From b4b1158a600a729890dc524034a6d0d294773c94 Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Mon, 16 Jan 2023 12:20:54 +0100 Subject: [PATCH 001/559] Change 02129_add_column_add_ttl to check differences between using materialize_ttl_recalculate_only=1/0 --- .../02129_add_column_add_ttl.reference | 39 ++++++++++--------- .../0_stateless/02129_add_column_add_ttl.sql | 16 +++++++- 2 files changed, 34 insertions(+), 21 deletions(-) diff --git a/tests/queries/0_stateless/02129_add_column_add_ttl.reference b/tests/queries/0_stateless/02129_add_column_add_ttl.reference index 8b3280ef095..5bffe7b5903 100644 --- a/tests/queries/0_stateless/02129_add_column_add_ttl.reference +++ b/tests/queries/0_stateless/02129_add_column_add_ttl.reference @@ -1,41 +1,42 @@ 0 2021-01-01 0 -0 2021-01-01 0 1 2021-01-01 0 -1 2021-01-01 0 -2 2021-01-01 0 2 2021-01-01 0 3 2021-01-01 0 -3 2021-01-01 0 -4 2021-01-01 0 4 2021-01-01 0 5 2021-01-01 0 -5 2021-01-01 0 -6 2021-01-01 0 6 2021-01-01 0 7 2021-01-01 0 -7 2021-01-01 0 8 2021-01-01 0 -8 2021-01-01 0 -9 2021-01-01 0 9 2021-01-01 0 ========== 0 2021-01-01 0 +0 2021-01-01 1 +1 2021-01-01 0 +1 2021-01-01 1 +2 2021-01-01 0 +2 2021-01-01 1 +3 2021-01-01 0 +3 2021-01-01 1 +4 2021-01-01 0 +4 2021-01-01 1 +5 2021-01-01 0 +5 2021-01-01 1 +6 2021-01-01 0 +6 2021-01-01 1 +7 2021-01-01 0 +7 2021-01-01 1 +8 2021-01-01 0 +8 2021-01-01 1 +9 2021-01-01 0 +9 2021-01-01 1 +========== 0 2021-01-01 0 1 2021-01-01 0 -1 2021-01-01 0 -2 2021-01-01 0 2 2021-01-01 0 3 2021-01-01 0 -3 2021-01-01 0 -4 2021-01-01 0 4 2021-01-01 0 5 2021-01-01 0 -5 2021-01-01 0 -6 2021-01-01 0 6 2021-01-01 0 7 2021-01-01 0 -7 2021-01-01 0 -8 2021-01-01 0 8 2021-01-01 0 9 2021-01-01 0 -9 2021-01-01 0 diff --git a/tests/queries/0_stateless/02129_add_column_add_ttl.sql b/tests/queries/0_stateless/02129_add_column_add_ttl.sql index 7a6dd928a3f..a68868ed3bc 100644 --- a/tests/queries/0_stateless/02129_add_column_add_ttl.sql +++ b/tests/queries/0_stateless/02129_add_column_add_ttl.sql @@ -4,10 +4,12 @@ create table ttl_test_02129(a Int64, b String, d Date) Engine=MergeTree partition by d order by a settings min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0, materialize_ttl_recalculate_only = 0; +system stop ttl merges ttl_test_02129; + insert into ttl_test_02129 select number, '', '2021-01-01' from numbers(10); alter table ttl_test_02129 add column c Int64 settings mutations_sync=2; -insert into ttl_test_02129 select number, '', '2021-01-01', 0 from numbers(10); +insert into ttl_test_02129 select number, '', '2021-01-01', 1 from numbers(10); alter table ttl_test_02129 modify TTL (d + INTERVAL 1 MONTH) DELETE WHERE c=1 settings mutations_sync=2; select * from ttl_test_02129 order by a, b, d, c; @@ -21,11 +23,21 @@ create table ttl_test_02129(a Int64, b String, d Date) Engine=MergeTree partition by d order by a settings min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0, materialize_ttl_recalculate_only = 1; +system stop ttl merges ttl_test_02129; + insert into ttl_test_02129 select number, '', '2021-01-01' from numbers(10); alter table ttl_test_02129 add column c Int64 settings mutations_sync=2; -insert into ttl_test_02129 select number, '', '2021-01-01', 0 from numbers(10); +insert into ttl_test_02129 select number, '', '2021-01-01', 1 from numbers(10); alter table ttl_test_02129 modify TTL (d + INTERVAL 1 MONTH) DELETE WHERE c=1 settings mutations_sync=2; +select * from ttl_test_02129 order by a, b, d, c; + +select '=========='; + +system start ttl merges ttl_test_02129; + +optimize table ttl_test_02129 final; + select * from ttl_test_02129 order by a, b, d, c; drop table ttl_test_02129; From 9c6c6d98444d160608690634cf794195fc37eaae Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 17 Jan 2023 20:12:34 +0000 Subject: [PATCH 002/559] Aggregate Projections analysis using query plan [In progress] --- src/Dictionaries/HashedDictionary.cpp | 24 +- src/Dictionaries/HashedDictionary.h | 4 +- .../QueryPlan/Optimizations/Optimizations.h | 1 + .../QueryPlan/Optimizations/matchTrees.cpp | 215 +++++++++++++ .../QueryPlan/Optimizations/matchTrees.h | 43 +++ .../Optimizations/optimizeReadInOrder.cpp | 241 +------------- .../Optimizations/optimizeUseProjections.cpp | 297 ++++++++++++++++++ 7 files changed, 571 insertions(+), 254 deletions(-) create mode 100644 src/Processors/QueryPlan/Optimizations/matchTrees.cpp create mode 100644 src/Processors/QueryPlan/Optimizations/matchTrees.h create mode 100644 src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 99c3dd05f45..331cd6cfde9 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -189,8 +189,8 @@ ColumnUInt8::Ptr HashedDictionary::hasKeys(const Co extractor.rollbackCurrentKey(); } - query_count.fetch_add(keys_size, std::memory_order_relaxed); - found_count.fetch_add(keys_found, std::memory_order_relaxed); + query_count.fetch_add(keys_size); + found_count.fetch_add(keys_found); return result; } @@ -214,8 +214,8 @@ ColumnUInt8::Ptr HashedDictionary::hasKeys(const Co } }); - query_count.fetch_add(keys_size, std::memory_order_relaxed); - found_count.fetch_add(keys_found, std::memory_order_relaxed); + query_count.fetch_add(keys_size); + found_count.fetch_add(keys_found); return result; } @@ -271,8 +271,8 @@ ColumnPtr HashedDictionary::getHierarchy(ColumnPtr auto dictionary_hierarchy_array = getKeysHierarchyArray(keys, is_key_valid_func, get_parent_func); - query_count.fetch_add(keys.size(), std::memory_order_relaxed); - found_count.fetch_add(keys_found, std::memory_order_relaxed); + query_count.fetch_add(keys.size()); + found_count.fetch_add(keys_found); return dictionary_hierarchy_array; } @@ -342,8 +342,8 @@ ColumnUInt8::Ptr HashedDictionary::isInHierarchy( auto result = getKeysIsInHierarchyColumn(keys, keys_in, is_key_valid_func, get_parent_key_func); - query_count.fetch_add(keys.size(), std::memory_order_relaxed); - found_count.fetch_add(keys_found, std::memory_order_relaxed); + query_count.fetch_add(keys.size()); + found_count.fetch_add(keys_found); return result; } @@ -392,8 +392,8 @@ ColumnPtr HashedDictionary::getDescendants( size_t keys_found; auto result = getKeysDescendantsArray(keys, *parent_to_child_index, level, keys_found); - query_count.fetch_add(keys.size(), std::memory_order_relaxed); - found_count.fetch_add(keys_found, std::memory_order_relaxed); + query_count.fetch_add(keys.size()); + found_count.fetch_add(keys_found); return result; } @@ -635,8 +635,8 @@ void HashedDictionary::getItemsImpl( keys_extractor.rollbackCurrentKey(); } - query_count.fetch_add(keys_size, std::memory_order_relaxed); - found_count.fetch_add(keys_found, std::memory_order_relaxed); + query_count.fetch_add(keys_size); + found_count.fetch_add(keys_found); } template diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 1fb38e8f9a6..26f4a4c7a88 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -62,10 +62,10 @@ public: double getFoundRate() const override { - size_t queries = query_count.load(std::memory_order_relaxed); + size_t queries = query_count.load(); if (!queries) return 0; - return static_cast(found_count.load(std::memory_order_relaxed)) / queries; + return static_cast(found_count.load()) / queries; } double getHitRate() const override { return 1.0; } diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 7f435463d64..6f733c75e88 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -93,6 +93,7 @@ using Stack = std::vector; void optimizePrimaryKeyCondition(const Stack & stack); void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); +void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes &); /// Enable memory bound merging of aggregation states for remote queries /// in case it was enabled for local plan diff --git a/src/Processors/QueryPlan/Optimizations/matchTrees.cpp b/src/Processors/QueryPlan/Optimizations/matchTrees.cpp new file mode 100644 index 00000000000..4e11f65f1e2 --- /dev/null +++ b/src/Processors/QueryPlan/Optimizations/matchTrees.cpp @@ -0,0 +1,215 @@ +#include +#include +#include +#include + +namespace DB::QueryPlanOptimizations +{ + +MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag) +{ + using Parents = std::set; + std::unordered_map inner_parents; + std::unordered_map inner_inputs; + + { + std::stack stack; + for (const auto * out : inner_dag.getOutputs()) + { + if (inner_parents.contains(out)) + continue; + + stack.push(out); + inner_parents.emplace(out, Parents()); + while (!stack.empty()) + { + const auto * node = stack.top(); + stack.pop(); + + if (node->type == ActionsDAG::ActionType::INPUT) + inner_inputs.emplace(node->result_name, node); + + for (const auto * child : node->children) + { + auto [it, inserted] = inner_parents.emplace(child, Parents()); + it->second.emplace(node); + + if (inserted) + stack.push(child); + } + } + } + } + + struct Frame + { + const ActionsDAG::Node * node; + ActionsDAG::NodeRawConstPtrs mapped_children; + }; + + MatchedTrees::Matches matches; + std::stack stack; + + for (const auto & node : outer_dag.getNodes()) + { + if (matches.contains(&node)) + continue; + + stack.push(Frame{&node, {}}); + while (!stack.empty()) + { + auto & frame = stack.top(); + frame.mapped_children.reserve(frame.node->children.size()); + + while (frame.mapped_children.size() < frame.node->children.size()) + { + const auto * child = frame.node->children[frame.mapped_children.size()]; + auto it = matches.find(child); + if (it == matches.end()) + { + /// If match map does not contain a child, it was not visited. + stack.push(Frame{child, {}}); + break; + } + /// A node from found match may be nullptr. + /// It means that node is visited, but no match was found. + frame.mapped_children.push_back(it->second.node); + } + + if (frame.mapped_children.size() < frame.node->children.size()) + continue; + + /// Create an empty match for current node. + /// natch.node will be set if match is found. + auto & match = matches[frame.node]; + + if (frame.node->type == ActionsDAG::ActionType::INPUT) + { + const ActionsDAG::Node * mapped = nullptr; + if (auto it = inner_inputs.find(frame.node->result_name); it != inner_inputs.end()) + mapped = it->second; + + match.node = mapped; + } + else if (frame.node->type == ActionsDAG::ActionType::ALIAS) + { + match = matches[frame.node->children.at(0)]; + } + else if (frame.node->type == ActionsDAG::ActionType::FUNCTION) + { + + //std::cerr << "... Processing " << frame.node->function_base->getName() << std::endl; + + bool found_all_children = true; + for (const auto * child : frame.mapped_children) + if (!child) + found_all_children = false; + + if (found_all_children && !frame.mapped_children.empty()) + { + Parents container; + Parents * intersection = &inner_parents[frame.mapped_children[0]]; + + if (frame.mapped_children.size() > 1) + { + std::vector other_parents; + size_t mapped_children_size = frame.mapped_children.size(); + other_parents.reserve(mapped_children_size); + for (size_t i = 1; i < mapped_children_size; ++i) + other_parents.push_back(&inner_parents[frame.mapped_children[i]]); + + for (const auto * parent : *intersection) + { + bool is_common = true; + for (const auto * set : other_parents) + { + if (!set->contains(parent)) + { + is_common = false; + break; + } + } + + if (is_common) + container.insert(parent); + } + + intersection = &container; + } + + //std::cerr << ".. Candidate parents " << intersection->size() << std::endl; + + if (!intersection->empty()) + { + auto func_name = frame.node->function_base->getName(); + for (const auto * parent : *intersection) + { + //std::cerr << ".. candidate " << parent->result_name << std::endl; + if (parent->type == ActionsDAG::ActionType::FUNCTION && func_name == parent->function_base->getName()) + { + const auto & children = parent->children; + size_t num_children = children.size(); + if (frame.mapped_children.size() == num_children) + { + bool all_children_matched = true; + for (size_t i = 0; all_children_matched && i < num_children; ++i) + all_children_matched = frame.mapped_children[i] == children[i]; + + if (all_children_matched) + { + match.node = parent; + break; + } + } + } + } + } + } + + if (!match.node && frame.node->function_base->hasInformationAboutMonotonicity()) + { + size_t num_const_args = 0; + const ActionsDAG::Node * monotonic_child = nullptr; + for (const auto * child : frame.node->children) + { + if (child->column) + ++num_const_args; + else + monotonic_child = child; + } + + if (monotonic_child && num_const_args + 1 == frame.node->children.size()) + { + const auto & child_match = matches[monotonic_child]; + if (child_match.node) + { + auto info = frame.node->function_base->getMonotonicityForRange(*monotonic_child->result_type, {}, {}); + if (info.is_monotonic) + { + MatchedTrees::Monotonicity monotonicity; + monotonicity.direction *= info.is_positive ? 1 : -1; + monotonicity.strict = info.is_strict; + + if (child_match.monotonicity) + { + monotonicity.direction *= child_match.monotonicity->direction; + if (!child_match.monotonicity->strict) + monotonicity.strict = false; + } + + match.node = child_match.node; + match.monotonicity = monotonicity; + } + } + } + } + } + + stack.pop(); + } + } + + return matches; +} + +} diff --git a/src/Processors/QueryPlan/Optimizations/matchTrees.h b/src/Processors/QueryPlan/Optimizations/matchTrees.h new file mode 100644 index 00000000000..18f0fe8b292 --- /dev/null +++ b/src/Processors/QueryPlan/Optimizations/matchTrees.h @@ -0,0 +1,43 @@ +#include + +namespace DB::QueryPlanOptimizations +{ + + +/// This structure stores a node mapping from one DAG to another. +/// The rule is following: +/// * Input nodes are mapped by name. +/// * Function is mapped to function if all children are mapped and function names are same. +/// * Alias is mapped to it's children mapping. +/// * Monotonic function can be mapped to it's children mapping if direct mapping does not exist. +/// In this case, information about monotonicity is filled. +/// * Mapped node is nullptr if there is no mapping found. +/// +/// Overall, directly mapped nodes represent equal calculations. +/// Notes: +/// * Mapped DAG can contain many nodes which represent the same calculation. +/// In this case mapping is ambiguous and only one node is mapped. +/// * Aliases for mapped DAG are not supported. +/// DAG for PK does not contain aliases and ambiguous nodes. +struct MatchedTrees +{ + /// Monotonicity is calculated for monotonic functions chain. + /// Chain is not strict if there is any non-strict monotonic function. + struct Monotonicity + { + int direction = 1; + bool strict = true; + }; + + struct Match + { + const ActionsDAG::Node * node = nullptr; + std::optional monotonicity; + }; + + using Matches = std::unordered_map; +}; + +MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag); + +} diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index bdf8f24f9d6..1876ebc4014 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -289,246 +290,6 @@ void enreachFixedColumns(const ActionsDAG & dag, FixedColumns & fixed_columns) } } -/// This structure stores a node mapping from one DAG to another. -/// The rule is following: -/// * Input nodes are mapped by name. -/// * Function is mapped to function if all children are mapped and function names are same. -/// * Alias is mapped to it's children mapping. -/// * Monotonic function can be mapped to it's children mapping if direct mapping does not exist. -/// In this case, information about monotonicity is filled. -/// * Mapped node is nullptr if there is no mapping found. -/// -/// Overall, directly mapped nodes represent equal calculations. -/// Notes: -/// * Mapped DAG can contain many nodes which represent the same calculation. -/// In this case mapping is ambiguous and only one node is mapped. -/// * Aliases for mapped DAG are not supported. -/// DAG for PK does not contain aliases and ambiguous nodes. -struct MatchedTrees -{ - /// Monotonicity is calculated for monotonic functions chain. - /// Chain is not strict if there is any non-strict monotonic function. - struct Monotonicity - { - int direction = 1; - bool strict = true; - }; - - struct Match - { - const ActionsDAG::Node * node = nullptr; - std::optional monotonicity; - }; - - using Matches = std::unordered_map; -}; - -MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag) -{ - using Parents = std::set; - std::unordered_map inner_parents; - std::unordered_map inner_inputs; - - { - std::stack stack; - for (const auto * out : inner_dag.getOutputs()) - { - if (inner_parents.contains(out)) - continue; - - stack.push(out); - inner_parents.emplace(out, Parents()); - while (!stack.empty()) - { - const auto * node = stack.top(); - stack.pop(); - - if (node->type == ActionsDAG::ActionType::INPUT) - inner_inputs.emplace(node->result_name, node); - - for (const auto * child : node->children) - { - auto [it, inserted] = inner_parents.emplace(child, Parents()); - it->second.emplace(node); - - if (inserted) - stack.push(child); - } - } - } - } - - struct Frame - { - const ActionsDAG::Node * node; - ActionsDAG::NodeRawConstPtrs mapped_children; - }; - - MatchedTrees::Matches matches; - std::stack stack; - - for (const auto & node : outer_dag.getNodes()) - { - if (matches.contains(&node)) - continue; - - stack.push(Frame{&node, {}}); - while (!stack.empty()) - { - auto & frame = stack.top(); - frame.mapped_children.reserve(frame.node->children.size()); - - while (frame.mapped_children.size() < frame.node->children.size()) - { - const auto * child = frame.node->children[frame.mapped_children.size()]; - auto it = matches.find(child); - if (it == matches.end()) - { - /// If match map does not contain a child, it was not visited. - stack.push(Frame{child, {}}); - break; - } - /// A node from found match may be nullptr. - /// It means that node is visited, but no match was found. - frame.mapped_children.push_back(it->second.node); - } - - if (frame.mapped_children.size() < frame.node->children.size()) - continue; - - /// Create an empty match for current node. - /// natch.node will be set if match is found. - auto & match = matches[frame.node]; - - if (frame.node->type == ActionsDAG::ActionType::INPUT) - { - const ActionsDAG::Node * mapped = nullptr; - if (auto it = inner_inputs.find(frame.node->result_name); it != inner_inputs.end()) - mapped = it->second; - - match.node = mapped; - } - else if (frame.node->type == ActionsDAG::ActionType::ALIAS) - { - match = matches[frame.node->children.at(0)]; - } - else if (frame.node->type == ActionsDAG::ActionType::FUNCTION) - { - - //std::cerr << "... Processing " << frame.node->function_base->getName() << std::endl; - - bool found_all_children = true; - for (const auto * child : frame.mapped_children) - if (!child) - found_all_children = false; - - if (found_all_children && !frame.mapped_children.empty()) - { - Parents container; - Parents * intersection = &inner_parents[frame.mapped_children[0]]; - - if (frame.mapped_children.size() > 1) - { - std::vector other_parents; - size_t mapped_children_size = frame.mapped_children.size(); - other_parents.reserve(mapped_children_size); - for (size_t i = 1; i < mapped_children_size; ++i) - other_parents.push_back(&inner_parents[frame.mapped_children[i]]); - - for (const auto * parent : *intersection) - { - bool is_common = true; - for (const auto * set : other_parents) - { - if (!set->contains(parent)) - { - is_common = false; - break; - } - } - - if (is_common) - container.insert(parent); - } - - intersection = &container; - } - - //std::cerr << ".. Candidate parents " << intersection->size() << std::endl; - - if (!intersection->empty()) - { - auto func_name = frame.node->function_base->getName(); - for (const auto * parent : *intersection) - { - //std::cerr << ".. candidate " << parent->result_name << std::endl; - if (parent->type == ActionsDAG::ActionType::FUNCTION && func_name == parent->function_base->getName()) - { - const auto & children = parent->children; - size_t num_children = children.size(); - if (frame.mapped_children.size() == num_children) - { - bool all_children_matched = true; - for (size_t i = 0; all_children_matched && i < num_children; ++i) - all_children_matched = frame.mapped_children[i] == children[i]; - - if (all_children_matched) - { - match.node = parent; - break; - } - } - } - } - } - } - - if (!match.node && frame.node->function_base->hasInformationAboutMonotonicity()) - { - size_t num_const_args = 0; - const ActionsDAG::Node * monotonic_child = nullptr; - for (const auto * child : frame.node->children) - { - if (child->column) - ++num_const_args; - else - monotonic_child = child; - } - - if (monotonic_child && num_const_args + 1 == frame.node->children.size()) - { - const auto & child_match = matches[monotonic_child]; - if (child_match.node) - { - auto info = frame.node->function_base->getMonotonicityForRange(*monotonic_child->result_type, {}, {}); - if (info.is_monotonic) - { - MatchedTrees::Monotonicity monotonicity; - monotonicity.direction *= info.is_positive ? 1 : -1; - monotonicity.strict = info.is_strict; - - if (child_match.monotonicity) - { - monotonicity.direction *= child_match.monotonicity->direction; - if (!child_match.monotonicity->strict) - monotonicity.strict = false; - } - - match.node = child_match.node; - match.monotonicity = monotonicity; - } - } - } - } - } - - stack.pop(); - } - } - - return matches; -} - InputOrderInfoPtr buildInputOrderInfo( const FixedColumns & fixed_columns, const ActionsDAGPtr & dag, diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp new file mode 100644 index 00000000000..f1e97558dbb --- /dev/null +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -0,0 +1,297 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB::QueryPlanOptimizations +{ + +QueryPlan::Node * findReadingStep(QueryPlan::Node & node) +{ + IQueryPlanStep * step = node.step.get();\ + if (auto * reading = typeid_cast(step)) + { + /// Already read-in-order, skip. + if (reading->getQueryInfo().input_order_info) + return nullptr; + + const auto & sorting_key = reading->getStorageMetadata()->getSortingKey(); + if (sorting_key.column_names.empty()) + return nullptr; + + return &node; + } + + if (node.children.size() != 1) + return nullptr; + + if (typeid_cast(step) || typeid_cast(step)) + return findReadingStep(*node.children.front()); + + return nullptr; +} + +void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) +{ + if (dag) + dag->mergeInplace(std::move(*expression->clone())); + else + dag = expression->clone(); +} + + +/// This function builds a common DAG which is a gerge of DAGs from Filter and Expression steps chain. +bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, ActionsDAG::NodeRawConstPtrs & filter_nodes) +{ + IQueryPlanStep * step = node.step.get(); + if (auto * reading = typeid_cast(step)) + { + if (const auto * prewhere_info = reading->getPrewhereInfo()) + { + if (prewhere_info->row_level_filter) + { + appendExpression(dag, prewhere_info->row_level_filter); + if (const auto * filter_node = dag->tryFindInOutputs(prewhere_info->row_level_column_name)) + filter_nodes.push_back(filter_node); + else + return false; + } + + if (prewhere_info->prewhere_actions) + { + appendExpression(dag, prewhere_info->prewhere_actions); + if (const auto * filter_node = dag->tryFindInOutputs(prewhere_info->prewhere_column_name)) + filter_nodes.push_back(filter_node); + else + return false; + } + } + return true; + } + + if (node.children.size() != 1) + return false; + + if (!buildAggregatingDAG(*node.children.front(), dag, filter_nodes)) + return false; + + if (auto * expression = typeid_cast(step)) + { + const auto & actions = expression->getExpression(); + if (actions->hasArrayJoin()) + return false; + + appendExpression(dag, actions); + } + + if (auto * filter = typeid_cast(step)) + { + const auto & actions = filter->getExpression(); + if (actions->hasArrayJoin()) + return false; + + appendExpression(dag, actions); + if (const auto * filter_expression = dag->tryFindInOutputs(filter->getFilterColumnName())) + filter_nodes.push_back(filter_expression); + else + return false; + } + + return false; +} + +struct AggregateProjectionInfo +{ + ActionsDAGPtr before_aggregation; + NamesAndTypesList keys; + AggregateDescriptions aggregates; + + /// A context copy from interpreter which was used for analysis. + /// Just in case it is used by some function. + ContextPtr context; +}; + +AggregateProjectionInfo getAggregatingProjectionInfo( + const ProjectionDescription & projection, + const ContextPtr & context, + StoragePtr & storage, + const StorageMetadataPtr & metadata_snapshot) +{ + /// This is a bad approach. + /// We'd better have a separate interpreter for projections. + /// Now it's not obvious we didn't miss anything here. + InterpreterSelectQuery interpreter( + projection.query_ast, + context, + storage, + metadata_snapshot, + SelectQueryOptions{QueryProcessingStage::WithMergeableState}); + + const auto & analysis_result = interpreter.getAnalysisResult(); + const auto & query_analyzer = interpreter.getQueryAnalyzer(); + + AggregateProjectionInfo info; + info.context = interpreter.getContext(); + info.before_aggregation = analysis_result.before_aggregation; + info.keys = query_analyzer->aggregationKeys(); + info.aggregates = query_analyzer->aggregates(); + + return info; +} + +struct AggregateProjectionCandidate +{ + AggregateProjectionInfo info; + ProjectionDescription * projection; +}; + +std::optional analyzeAggregateProjection( + ProjectionDescription & projection, + AggregateProjectionInfo info, + ActionsDAG & query_dag, + const Names & keys, + const AggregateDescriptions & aggregates) +{ + + ActionsDAG::NodeRawConstPtrs key_nodes; + std::unordered_set aggregate_args; + + std::unordered_map index; + for (const auto * output : query_dag.getOutputs()) + index.emplace(output->result_name, output); + + key_nodes.reserve(keys.size()); + for (const auto & key : keys) + { + auto it = index.find(key); + /// This should not happen ideally. + if (it == index.end()) + return {}; + + key_nodes.push_back(it->second); + } + + for (const auto & aggregate : aggregates) + { + for (const auto & argument : aggregate.argument_names) + { + auto it = index.find(argument); + /// This should not happen ideally. + if (it == index.end()) + return {}; + + aggregate_args.insert(it->second); + } + } + + MatchedTrees::Matches matches = matchTrees(*info.before_aggregation, query_dag); + + std::unordered_map> projection_aggregate_functions; + for (size_t i = 0; i < info.aggregates.size(); ++i) + projection_aggregate_functions[info.aggregates[i].function->getName()].push_back(i); + + struct AggFuncMatch + { + /// idx in projection + size_t idx; + /// nodes in query DAG + ActionsDAG::NodeRawConstPtrs args; + }; + + std::vector aggregate_function_matches; + aggregate_function_matches.reserve(aggregates.size()); + + for (const auto & aggregate : aggregates) + { + auto it = projection_aggregate_functions.find(aggregate.function->getName()); + if (it == projection_aggregate_functions.end()) + return {}; + auto & candidates = it->second; + + std::optional match; + + for (size_t idx : candidates) + { + const auto & candidate = info.aggregates[idx]; + + /// Note: this check is a bit strict. + /// We check that aggregate function names, arguemnt types and parameters are equal. + /// In some cases it's possilbe only to check that states are equal, + /// e.g. for quantile(0.3)(...) and quantile(0.5)(...). + /// But also functions sum(...) and sumIf(...) will have equal states, + /// and we can't replace one to another from projection. + if (!candidate.function->getStateType()->equals(*aggregate.function->getStateType())) + continue; + + ActionsDAG::NodeRawConstPtrs args; + args.reserve(aggregate.argument_names.size()); + for (const auto & name : aggregate.argument_names) + { + auto jt = index.find(name); + /// This should not happen ideally. + if (jt == index.end()) + break; + + const auto * outer_node = jt->second; + auto kt = matches.find(outer_node); + if (kt == matches.end()) + break; + + const auto & node_match = kt->second; + if (!node_match.node || node_match.monotonicity) + break; + + args.push_back(node_match.node); + } + + if (args.size() < aggregate.argument_names.size()) + continue; + + match = AggFuncMatch{idx, std::move(args)}; + } + + if (!match) + return {}; + + aggregate_function_matches.emplace_back(std::move(*match)); + } + + +} + +void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes &) +{ + if (node.children.size() != 1) + return; + + auto * aggregating = typeid_cast(node.step.get()); + if (!aggregating) + return; + + QueryPlan::Node * reading_node = findReadingStep(node); + if (!reading_node) + return; + + ActionsDAGPtr dag; + ActionsDAG::NodeRawConstPtrs filter_nodes; + if (!buildAggregatingDAG(node, dag, filter_nodes)) + return; + + const auto & keys = aggregating->getParams().keys; + const auto & aggregates = aggregating->getParams().aggregates; + + auto * reading = typeid_cast(reading_node->step.get()); + if (!reading) + return; + + // const auto metadata = reading->getStorageMetadata(); + // const auto & projections = metadata->projections; + + +} + +} From 134ac9b2dda076e63468ba84a406afd309025023 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 19 Jan 2023 19:53:22 +0000 Subject: [PATCH 003/559] Aggregate Projections analysis using query plan [In progress] --- .../Optimizations/optimizeUseProjections.cpp | 118 ++++++++++++++++-- 1 file changed, 106 insertions(+), 12 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index f1e97558dbb..18dd5c8adcb 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB::QueryPlanOptimizations { @@ -150,7 +151,7 @@ struct AggregateProjectionCandidate }; std::optional analyzeAggregateProjection( - ProjectionDescription & projection, + //ProjectionDescription & projection, AggregateProjectionInfo info, ActionsDAG & query_dag, const Names & keys, @@ -164,6 +165,10 @@ std::optional analyzeAggregateProjection( for (const auto * output : query_dag.getOutputs()) index.emplace(output->result_name, output); + std::unordered_map proj_index; + for (const auto * output : info.before_aggregation->getOutputs()) + proj_index.emplace(output->result_name, output); + key_nodes.reserve(keys.size()); for (const auto & key : keys) { @@ -194,6 +199,8 @@ std::optional analyzeAggregateProjection( for (size_t i = 0; i < info.aggregates.size(); ++i) projection_aggregate_functions[info.aggregates[i].function->getName()].push_back(i); + std::unordered_set split_nodes; + struct AggFuncMatch { /// idx in projection @@ -228,29 +235,44 @@ std::optional analyzeAggregateProjection( continue; ActionsDAG::NodeRawConstPtrs args; - args.reserve(aggregate.argument_names.size()); - for (const auto & name : aggregate.argument_names) + size_t num_args = aggregate.argument_names.size(); + args.reserve(num_args); + for (size_t arg = 0; arg < num_args; ++arg) { - auto jt = index.find(name); + const auto & query_name = aggregate.argument_names[arg]; + const auto & proj_name = candidate.argument_names[arg]; + + auto jt = index.find(query_name); /// This should not happen ideally. if (jt == index.end()) break; - const auto * outer_node = jt->second; - auto kt = matches.find(outer_node); - if (kt == matches.end()) + const auto * query_node = jt->second; + + auto kt = proj_index.find(proj_name); + /// This should not happen ideally. + if (kt == proj_index.end()) break; - const auto & node_match = kt->second; - if (!node_match.node || node_match.monotonicity) + const auto * proj_node = kt->second; + + auto mt = matches.find(query_node); + if (mt == matches.end()) break; - args.push_back(node_match.node); + const auto & node_match = mt->second; + if (node_match.node != proj_node || node_match.monotonicity) + break; + + args.push_back(query_node); } if (args.size() < aggregate.argument_names.size()) continue; + for (const auto * node : args) + split_nodes.insert(node); + match = AggFuncMatch{idx, std::move(args)}; } @@ -260,7 +282,79 @@ std::optional analyzeAggregateProjection( aggregate_function_matches.emplace_back(std::move(*match)); } + std::unordered_set proj_key_nodes; + for (const auto & key : info.keys) + { + auto it = proj_index.find(key.name); + /// This should not happen ideally. + if (it == proj_index.end()) + break; + proj_key_nodes.insert(it->second); + } + + std::unordered_set visited; + + struct Frame + { + const ActionsDAG::Node * node; + size_t next_child_to_visit = 0; + }; + + std::stack stack; + for (const auto & key : keys) + { + auto it = index.find(key); + /// This should not happen ideally. + if (it == index.end()) + break; + + const auto * key_node = it->second; + if (visited.contains(key_node)) + continue; + + stack.push({.node = key_node}); + + while (!stack.empty()) + { + auto & frame = stack.top(); + + if (frame.next_child_to_visit == 0) + { + auto jt = matches.find(frame.node); + if (jt != matches.end()) + { + auto & match = jt->second; + if (match.node && !match.monotonicity && proj_key_nodes.contains(match.node)) + { + visited.insert(frame.node); + split_nodes.insert(frame.node); + stack.pop(); + continue; + } + } + } + + if (frame.next_child_to_visit < frame.node->children.size()) + { + stack.push({.node = frame.node->children[frame.next_child_to_visit]}); + ++frame.next_child_to_visit; + continue; + } + + /// Not a match and there is no matched child. + if (frame.node->children.empty()) + return {}; + + /// Not a match, but all children matched. + visited.insert(frame.node); + stack.pop(); + } + } + + + + return {}; } void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes &) @@ -281,8 +375,8 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes &) if (!buildAggregatingDAG(node, dag, filter_nodes)) return; - const auto & keys = aggregating->getParams().keys; - const auto & aggregates = aggregating->getParams().aggregates; + // const auto & keys = aggregating->getParams().keys; + // const auto & aggregates = aggregating->getParams().aggregates; auto * reading = typeid_cast(reading_node->step.get()); if (!reading) From 1793730df85ba187e4943733bc208aa227aacaf0 Mon Sep 17 00:00:00 2001 From: exX512 Date: Thu, 26 Jan 2023 21:30:36 +0000 Subject: [PATCH 004/559] Change behaviour of cancel in PipelineExecutor --- src/Processors/Executors/ExecutingGraph.cpp | 4 +++- src/Processors/Executors/PipelineExecutor.cpp | 7 ++++++- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Processors/Executors/ExecutingGraph.cpp b/src/Processors/Executors/ExecutingGraph.cpp index f84efabdee1..f43f0ce8cff 100644 --- a/src/Processors/Executors/ExecutingGraph.cpp +++ b/src/Processors/Executors/ExecutingGraph.cpp @@ -400,7 +400,9 @@ void ExecutingGraph::cancel() { try { - processor->cancel(); + bool is_source = processor->getInputs().empty(); + if (is_source) + processor->cancel(); } catch (...) { diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index f1e044e470b..736098b267e 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -70,7 +70,6 @@ const Processors & PipelineExecutor::getProcessors() const void PipelineExecutor::cancel() { cancelled = true; - finish(); graph->cancel(); } @@ -147,7 +146,10 @@ bool PipelineExecutor::checkTimeLimitSoft() // We call cancel here so that all processors are notified and tasks waken up // so that the "break" is faster and doesn't wait for long events if (!continuing) + { cancel(); + finish(); + } return continuing; } @@ -227,7 +229,10 @@ void PipelineExecutor::executeStepImpl(size_t thread_num, std::atomic_bool * yie break; if (!context.executeTask()) + { cancel(); + finish(); + } if (tasks.isFinished()) break; From 286a58801e0e2ec770b8169e9a85b65233a30fc4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 27 Jan 2023 18:38:14 +0000 Subject: [PATCH 005/559] Aggregate Projections analysis using query plan [In progress] --- src/Interpreters/ActionsDAG.cpp | 73 +++++++++++++++++++ src/Interpreters/ActionsDAG.h | 2 + .../Optimizations/optimizeUseProjections.cpp | 9 ++- 3 files changed, 81 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 9968c43cc9d..d2a1d7becfe 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -723,6 +723,79 @@ NameSet ActionsDAG::foldActionsByProjection( return next_required_columns; } + +ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_map & new_inputs) +{ + auto dag = std::make_unique(); + std::unordered_map new_input_to_pos; + + std::unordered_map mapping; + struct Frame + { + const Node * node; + size_t next_child = 0; + }; + + std::vector stack; + for (const auto * output : outputs) + { + if (mapping.contains(output)) + continue; + + stack.push_back({.node = output}); + while (!stack.empty()) + { + auto & frame = stack.back(); + + if (frame.next_child == 0) + { + auto it = new_inputs.find(frame.node); + if (it != new_inputs.end()) + { + const auto & [new_input, rename] = *it; + + const auto * node = &dag->addInput(new_input->result_name, new_input->result_type); + if (!rename.empty() && new_input->result_name != rename) + node = &dag->addAlias(*node, rename); + + mapping.emplace(frame.node, node); + stack.pop_back(); + continue; + } + } + + const auto & children = frame.node->children; + + while (frame.next_child < children.size() && !mapping.emplace(children[frame.next_child], nullptr).second) + ++frame.next_child; + + if (frame.next_child < children.size()) + { + const auto * child = children[frame.next_child]; + ++frame.next_child; + stack.push_back({.node = child}); + continue; + } + + if (frame.node->type == ActionType::INPUT) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot fold actions for projection. Node {} requires input {} which does not belong to projection", + stack.front().node->result_name, frame.node->result_name); + + auto & node = dag->nodes.emplace_back(*frame.node); + for (auto & child : node.children) + child = mapping[child]; + + stack.pop_back(); + } + } + + for (const auto * output : outputs) + dag->outputs.push_back(mapping[output]); + + return dag; +} + void ActionsDAG::reorderAggregationKeysForProjection(const std::unordered_map & key_names_pos_map) { ::sort(outputs.begin(), outputs.end(), [&key_names_pos_map](const Node * lhs, const Node * rhs) diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index a26694e00f5..b23c87b4903 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -214,6 +214,8 @@ public: const String & predicate_column_name = {}, bool add_missing_keys = true); + ActionsDAGPtr foldActionsByProjection(const std::unordered_map & new_inputs); + /// Reorder the output nodes using given position mapping. void reorderAggregationKeysForProjection(const std::unordered_map & key_names_pos_map); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 18dd5c8adcb..3ec632843ff 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -148,9 +148,10 @@ struct AggregateProjectionCandidate { AggregateProjectionInfo info; ProjectionDescription * projection; + ActionsDAGPtr dag; }; -std::optional analyzeAggregateProjection( +ActionsDAGPtr analyzeAggregateProjection( //ProjectionDescription & projection, AggregateProjectionInfo info, ActionsDAG & query_dag, @@ -352,9 +353,11 @@ std::optional analyzeAggregateProjection( } } + std::unordered_map new_inputs; + for (const auto * node : split_nodes) + new_inputs[node] = matches[node].node->result_name; - - return {}; + return query_dag.foldActionsByProjection(new_inputs); } void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes &) From f09f8f80af5a62c1550928fa19dd8ef403aad5b2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 30 Jan 2023 20:09:58 +0000 Subject: [PATCH 006/559] Aggregate Projections analysis using query plan [In progress] --- .../Optimizations/optimizeUseProjections.cpp | 76 +++++++++++++++---- 1 file changed, 60 insertions(+), 16 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 3ec632843ff..5795687fbb3 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -6,6 +6,9 @@ #include #include #include +#include +#include +#include #include namespace DB::QueryPlanOptimizations @@ -119,7 +122,6 @@ struct AggregateProjectionInfo AggregateProjectionInfo getAggregatingProjectionInfo( const ProjectionDescription & projection, const ContextPtr & context, - StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot) { /// This is a bad approach. @@ -128,8 +130,7 @@ AggregateProjectionInfo getAggregatingProjectionInfo( InterpreterSelectQuery interpreter( projection.query_ast, context, - storage, - metadata_snapshot, + Pipe(std::make_shared(metadata_snapshot->getSampleBlock())), SelectQueryOptions{QueryProcessingStage::WithMergeableState}); const auto & analysis_result = interpreter.getAnalysisResult(); @@ -147,13 +148,12 @@ AggregateProjectionInfo getAggregatingProjectionInfo( struct AggregateProjectionCandidate { AggregateProjectionInfo info; - ProjectionDescription * projection; + const ProjectionDescription * projection; ActionsDAGPtr dag; }; ActionsDAGPtr analyzeAggregateProjection( - //ProjectionDescription & projection, - AggregateProjectionInfo info, + const AggregateProjectionInfo & info, ActionsDAG & query_dag, const Names & keys, const AggregateDescriptions & aggregates) @@ -373,22 +373,66 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes &) if (!reading_node) return; - ActionsDAGPtr dag; - ActionsDAG::NodeRawConstPtrs filter_nodes; - if (!buildAggregatingDAG(node, dag, filter_nodes)) - return; - - // const auto & keys = aggregating->getParams().keys; - // const auto & aggregates = aggregating->getParams().aggregates; - auto * reading = typeid_cast(reading_node->step.get()); if (!reading) return; - // const auto metadata = reading->getStorageMetadata(); - // const auto & projections = metadata->projections; + const auto metadata = reading->getStorageMetadata(); + const auto & projections = metadata->projections; + std::vector agg_projections; + for (const auto & projection : projections) + if (projection.type == ProjectionDescription::Type::Aggregate) + agg_projections.push_back(&projection); + if (agg_projections.empty()) + return; + + ActionsDAGPtr dag; + ActionsDAG::NodeRawConstPtrs filter_nodes; + if (!buildAggregatingDAG(*node.children.front(), dag, filter_nodes)) + return; + + const ActionsDAG::Node * filter_node = nullptr; + if (!filter_nodes.empty()) + { + filter_node = filter_nodes.front(); + if (filter_nodes.size() > 1) + { + FunctionOverloadResolverPtr func_builder_and = + std::make_unique( + std::make_shared()); + + filter_node = &dag->addFunction(func_builder_and, std::move(filter_nodes), {}); + } + + dag->getOutputs().insert(dag->getOutputs().begin(), filter_node); + } + + ContextPtr context = reading->getContext(); + + const auto & keys = aggregating->getParams().keys; + const auto & aggregates = aggregating->getParams().aggregates; + + std::vector candidates; + candidates.reserve(agg_projections.size()); + for (const auto * projection : agg_projections) + { + auto info = getAggregatingProjectionInfo(*projection, context, metadata); + if (auto proj_dag = analyzeAggregateProjection(info, *dag, keys, aggregates)) + { + candidates.emplace_back(AggregateProjectionCandidate{ + .info = std::move(info), + .projection = projection, + .dag = std::move(proj_dag), + }); + } + } + + if (candidates.empty()) + return; + + } } From e2c32ccbca9d1ee13af2f3febee9891ce9692fbb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 31 Jan 2023 20:33:01 +0000 Subject: [PATCH 007/559] Aggregate Projections analysis using query plan [In progress] --- src/Processors/QueryPlan/AggregatingStep.cpp | 23 +++ src/Processors/QueryPlan/AggregatingStep.h | 3 + .../Optimizations/optimizeUseProjections.cpp | 131 +++++++++++++++++- src/Processors/QueryPlan/ReadFromMergeTree.h | 11 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 22 ++- .../MergeTree/MergeTreeDataSelectExecutor.h | 2 +- src/Storages/StorageReplicatedMergeTree.h | 5 +- 7 files changed, 179 insertions(+), 18 deletions(-) diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 4fd6e7c11dd..b52982b0425 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -406,6 +406,11 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B return; } + if (input_streams.size() > 1) + { + + } + /// If there are several sources, then we perform parallel aggregation if (pipeline.getNumStreams() > 1) { @@ -465,6 +470,24 @@ void AggregatingStep::describePipeline(FormatSettings & settings) const } } +void AggregatingStep::requestOnlyMergeForAggregateProjection(const DataStream & input_stream) +{ + auto cur_header = getOutputStream().header; + input_streams.front() = input_stream; + params.only_merge = true; + updateOutputStream(); + assertBlocksHaveEqualStructure(cur_header, getOutputStream().header, "AggregatingStep"); +} + +void AggregatingStep::appendAggregateProjection(const DataStream & input_stream) +{ + input_streams.emplace_back(input_stream); + params.only_merge = true; + auto added_header = appendGroupingColumn(params.getHeader(input_streams.front().header, final), params.keys, !grouping_sets_params.empty(), group_by_use_nulls); + assertBlocksHaveEqualStructure(getOutputStream().header, added_header, "AggregatingStep"); + params.only_merge = false; +} + void AggregatingStep::updateOutputStream() { output_stream = createOutputStream( diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index 0dc06649d2d..2ceca926684 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -60,6 +60,9 @@ public: void applyOrder(SortDescription sort_description_for_merging_, SortDescription group_by_sort_description_); bool memoryBoundMergingWillBeUsed() const; + void requestOnlyMergeForAggregateProjection(const DataStream & input_stream); + void appendAggregateProjection(const DataStream & input_stream); + private: void updateOutputStream() override; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 5795687fbb3..2c9ad818ed4 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -9,6 +9,8 @@ #include #include #include +#include +#include #include namespace DB::QueryPlanOptimizations @@ -16,7 +18,7 @@ namespace DB::QueryPlanOptimizations QueryPlan::Node * findReadingStep(QueryPlan::Node & node) { - IQueryPlanStep * step = node.step.get();\ + IQueryPlanStep * step = node.step.get(); if (auto * reading = typeid_cast(step)) { /// Already read-in-order, skip. @@ -150,6 +152,9 @@ struct AggregateProjectionCandidate AggregateProjectionInfo info; const ProjectionDescription * projection; ActionsDAGPtr dag; + + MergeTreeDataSelectAnalysisResultPtr merge_tree_projection_select_result_ptr; + MergeTreeDataSelectAnalysisResultPtr merge_tree_normal_select_result_ptr; }; ActionsDAGPtr analyzeAggregateProjection( @@ -360,7 +365,7 @@ ActionsDAGPtr analyzeAggregateProjection( return query_dag.foldActionsByProjection(new_inputs); } -void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes &) +void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { if (node.children.size() != 1) return; @@ -432,6 +437,128 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes &) if (candidates.empty()) return; + AggregateProjectionCandidate * best_candidate = nullptr; + size_t best_candidate_marks = 0; + + const auto & parts = reading->getParts(); + const auto & query_info = reading->getQueryInfo(); + + MergeTreeDataSelectExecutor reader(reading->getMergeTreeData()); + + std::shared_ptr max_added_blocks; + if (context->getSettingsRef().select_sequential_consistency) + { + if (const StorageReplicatedMergeTree * replicated = dynamic_cast(&reading->getMergeTreeData())) + max_added_blocks = std::make_shared(replicated->getMaxAddedBlocks()); + } + + for (auto & candidate : candidates) + { + MergeTreeData::DataPartsVector projection_parts; + MergeTreeData::DataPartsVector normal_parts; + for (const auto & part : parts) + { + const auto & created_projections = part->getProjectionParts(); + auto it = created_projections.find(candidate.projection->name); + if (it != created_projections.end()) + projection_parts.push_back(it->second); + else + normal_parts.push_back(part); + } + + if (projection_parts.empty()) + continue; + + ActionDAGNodes added_filter_nodes; + if (filter_node) + added_filter_nodes.nodes.push_back(candidate.dag->getOutputs().front()); + + auto projection_result_ptr = reader.estimateNumMarksToRead( + projection_parts, + nullptr, + candidate.dag->getRequiredColumnsNames(), + metadata, + candidate.projection->metadata, + query_info, /// How it is actually used? I hope that for index we need only added_filter_nodes + added_filter_nodes, + context, + context->getSettingsRef().max_threads, + max_added_blocks); + + if (projection_result_ptr->error()) + continue; + + size_t sum_marks = projection_result_ptr->marks(); + + if (!normal_parts.empty()) + { + auto normal_result_ptr = reading->selectRangesToRead(std::move(normal_parts)); + + if (normal_result_ptr->error()) + continue; + + if (normal_result_ptr->marks() != 0) + { + sum_marks += normal_result_ptr->marks(); + candidate.merge_tree_normal_select_result_ptr = std::move(normal_result_ptr); + } + } + + candidate.merge_tree_projection_select_result_ptr = std::move(projection_result_ptr); + + if (best_candidate == nullptr || best_candidate_marks > sum_marks) + { + best_candidate = &candidate; + best_candidate_marks = sum_marks; + } + } + + if (!best_candidate) + return; + + auto projection_reading = reader.readFromParts( + {}, + best_candidate->dag->getRequiredColumnsNames(), + reading->getStorageSnapshot(), + query_info, + context, + reading->getMaxBlockSize(), + reading->getNumStreams(), + max_added_blocks, + best_candidate->merge_tree_projection_select_result_ptr, + reading->isParallelReadingEnabled()); + + projection_reading->setStepDescription(best_candidate->projection->name); + + if (!best_candidate->merge_tree_normal_select_result_ptr) + { + /// All parts are taken from projection + + auto & projection_reading_node = nodes.emplace_back(QueryPlan::Node{.step = std::move(projection_reading)}); + auto & expr_or_filter_node = nodes.emplace_back(); + + if (filter_node) + { + expr_or_filter_node.step = std::make_unique( + projection_reading_node.step->getOutputStream(), + best_candidate->dag, + best_candidate->dag->getOutputs().front()->result_name, + true); + } + else + expr_or_filter_node.step = std::make_unique( + projection_reading_node.step->getOutputStream(), + best_candidate->dag); + + expr_or_filter_node.children.push_back(&projection_reading_node); + aggregating->requestOnlyMergeForAggregateProjection(expr_or_filter_node.step->getOutputStream()); + node.children.front() = &expr_or_filter_node; + + optimizeAggregationInOrder(node, nodes); + + return; + } + } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index a3cea2a8afe..1c614ed09f1 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -147,13 +147,23 @@ public: bool sample_factor_column_queried, Poco::Logger * log); + MergeTreeDataSelectAnalysisResultPtr selectRangesToRead(MergeTreeData::DataPartsVector parts) const; + ContextPtr getContext() const { return context; } const SelectQueryInfo & getQueryInfo() const { return query_info; } StorageMetadataPtr getStorageMetadata() const { return metadata_for_reading; } + StorageSnapshotPtr getStorageSnapshot() const { return storage_snapshot; } const PrewhereInfo * getPrewhereInfo() const { return prewhere_info.get(); } void requestReadingInOrder(size_t prefix_size, int direction, size_t limit); + const MergeTreeData::DataPartsVector & getParts() const { return prepared_parts; } + const MergeTreeData & getMergeTreeData() const { return data; } + const Names & getRealColumnNames() const { return real_column_names; } + size_t getMaxBlockSize() const { return max_block_size; } + size_t getNumStreams() const { return requested_num_streams; } + bool isParallelReadingEnabled() const { return read_task_callback != std::nullopt; } + private: static MergeTreeDataSelectAnalysisResultPtr selectRangesToReadImpl( MergeTreeData::DataPartsVector parts, @@ -232,7 +242,6 @@ private: const Names & column_names, ActionsDAGPtr & out_projection); - MergeTreeDataSelectAnalysisResultPtr selectRangesToRead(MergeTreeData::DataPartsVector parts) const; ReadFromMergeTree::AnalysisResult getAnalysisResult() const; MergeTreeDataSelectAnalysisResultPtr analyzed_result_ptr; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 1ca1779e4b0..242f86e171c 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -156,7 +156,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( if (!query_info.projection) { - auto plan = readFromParts( + auto step = readFromParts( query_info.merge_tree_select_result_ptr ? MergeTreeData::DataPartsVector{} : parts, column_names_to_return, storage_snapshot, @@ -168,12 +168,14 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( query_info.merge_tree_select_result_ptr, enable_parallel_reading); - if (plan->isInitialized() && settings.allow_experimental_projection_optimization && settings.force_optimize_projection + if (!step && settings.allow_experimental_projection_optimization && settings.force_optimize_projection && !metadata_for_reading->projections.empty()) throw Exception( "No projection is used when allow_experimental_projection_optimization = 1 and force_optimize_projection = 1", ErrorCodes::PROJECTION_NOT_USED); + auto plan = std::make_unique(); + plan->addStep(std::move(step)); return plan; } @@ -197,7 +199,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( else if (query_info.projection->merge_tree_projection_select_result_ptr) { LOG_DEBUG(log, "projection required columns: {}", fmt::join(query_info.projection->required_columns, ", ")); - projection_plan = readFromParts( + projection_plan->addStep(readFromParts( {}, query_info.projection->required_columns, storage_snapshot, @@ -207,7 +209,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( num_streams, max_block_numbers_to_read, query_info.projection->merge_tree_projection_select_result_ptr, - enable_parallel_reading); + enable_parallel_reading)); } if (projection_plan->isInitialized()) @@ -1337,7 +1339,7 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar log); } -QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( +QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts( MergeTreeData::DataPartsVector parts, const Names & column_names_to_return, const StorageSnapshotPtr & storage_snapshot, @@ -1353,10 +1355,10 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( if (merge_tree_select_result_ptr) { if (merge_tree_select_result_ptr->marks() == 0) - return std::make_unique(); + return {}; } else if (parts.empty()) - return std::make_unique(); + return {}; Names real_column_names; Names virt_column_names; @@ -1366,7 +1368,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( selectColumnNames(column_names_to_return, data, real_column_names, virt_column_names, sample_factor_column_queried); - auto read_from_merge_tree = std::make_unique( + return std::make_unique( std::move(parts), real_column_names, virt_column_names, @@ -1382,10 +1384,6 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( merge_tree_select_result_ptr, enable_parallel_reading ); - - QueryPlanPtr plan = std::make_unique(); - plan->addStep(std::move(read_from_merge_tree)); - return plan; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 30d09312245..a337574bb64 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -39,7 +39,7 @@ public: bool enable_parallel_reading = false) const; /// The same as read, but with specified set of parts. - QueryPlanPtr readFromParts( + QueryPlanStepPtr readFromParts( MergeTreeData::DataPartsVector parts, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index c3bd682a29c..44f34adf3c0 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -322,12 +322,13 @@ public: const String & replica_name, const String & zookeeper_path, const ContextPtr & local_context, const zkutil::ZooKeeperPtr & zookeeper); bool canUseZeroCopyReplication() const; -private: - std::atomic_bool are_restoring_replica {false}; /// Get a sequential consistent view of current parts. ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock getMaxAddedBlocks() const; +private: + std::atomic_bool are_restoring_replica {false}; + /// Delete old parts from disk and from ZooKeeper. void clearOldPartsAndRemoveFromZK(); From ae420b30e50196876a87a57878e0c5baa80a9e2a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 1 Feb 2023 17:55:06 +0000 Subject: [PATCH 008/559] Aggregate Projections analysis using query plan [In progress] --- src/Processors/QueryPlan/AggregatingStep.cpp | 133 ++++++++++++++++-- src/Processors/QueryPlan/AggregatingStep.h | 40 +++++- .../Optimizations/optimizeUseProjections.cpp | 46 +++--- .../MergeTree/StorageFromMergeTreeDataPart.h | 2 +- 4 files changed, 188 insertions(+), 33 deletions(-) diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index b52982b0425..db85764ef33 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -406,11 +406,6 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B return; } - if (input_streams.size() > 1) - { - - } - /// If there are several sources, then we perform parallel aggregation if (pipeline.getNumStreams() > 1) { @@ -470,8 +465,16 @@ void AggregatingStep::describePipeline(FormatSettings & settings) const } } +bool AggregatingStep::canUseProjection() const +{ + return grouping_sets_params.empty() && sort_description_for_merging.empty() && !memory_bound_merging_of_aggregation_results_enabled; +} + void AggregatingStep::requestOnlyMergeForAggregateProjection(const DataStream & input_stream) { + if (!canUseProjection()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot aggregate from projection"); + auto cur_header = getOutputStream().header; input_streams.front() = input_stream; params.only_merge = true; @@ -479,13 +482,26 @@ void AggregatingStep::requestOnlyMergeForAggregateProjection(const DataStream & assertBlocksHaveEqualStructure(cur_header, getOutputStream().header, "AggregatingStep"); } -void AggregatingStep::appendAggregateProjection(const DataStream & input_stream) +std::unique_ptr AggregatingStep::convertToAggregatingProjection(const DataStream & input_stream) const { - input_streams.emplace_back(input_stream); - params.only_merge = true; - auto added_header = appendGroupingColumn(params.getHeader(input_streams.front().header, final), params.keys, !grouping_sets_params.empty(), group_by_use_nulls); - assertBlocksHaveEqualStructure(getOutputStream().header, added_header, "AggregatingStep"); - params.only_merge = false; + if (!canUseProjection()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot aggregate from projection"); + + auto aggregating_projection = std::make_unique( + DataStreams{input_streams.front(), input_stream}, + params, + final, + //max_block_size, + merge_threads, + temporary_data_merge_threads + //group_by_use_nulls, + //group_by_sort_description, + //should_produce_results_in_order_of_bucket_number + //memory_bound_merging_of_aggregation_results_enabled, + ); + + assertBlocksHaveEqualStructure(getOutputStream().header, aggregating_projection->getOutputStream().header, "AggregatingStep"); + return aggregating_projection; } void AggregatingStep::updateOutputStream() @@ -502,4 +518,99 @@ bool AggregatingStep::memoryBoundMergingWillBeUsed() const should_produce_results_in_order_of_bucket_number, memory_bound_merging_of_aggregation_results_enabled, sort_description_for_merging); } +AggregatingProjectionStep::AggregatingProjectionStep( + DataStreams input_streams_, + Aggregator::Params params_, + bool final_, + //size_t max_block_size_, + size_t merge_threads_, + size_t temporary_data_merge_threads_ + //bool group_by_use_nulls_, + //SortDescription group_by_sort_description_, + //bool should_produce_results_in_order_of_bucket_number_ + //bool memory_bound_merging_of_aggregation_results_enabled_ + ) + : params(std::move(params_)) + , final(final_) + //, max_block_size(max_block_size_) + , merge_threads(merge_threads_) + , temporary_data_merge_threads(temporary_data_merge_threads_) + //, group_by_use_nulls(group_by_use_nulls_) + //, group_by_sort_description(std::move(group_by_sort_description_)) + //, should_produce_results_in_order_of_bucket_number(should_produce_results_in_order_of_bucket_number_) + //, memory_bound_merging_of_aggregation_results_enabled(memory_bound_merging_of_aggregation_results_enabled_) +{ + input_streams = std::move(input_streams_); + + if (input_streams.size() != 2) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "AggregatingProjectionStep is expected to have two input streams, got {}", + input_streams.size()); + + auto normal_parts_header = params.getHeader(input_streams.front().header, final); + params.only_merge = true; + auto projection_parts_header = params.getHeader(input_streams.back().header, final); + params.only_merge = false; + + assertBlocksHaveEqualStructure(normal_parts_header, projection_parts_header, "AggregatingProjectionStep"); + output_stream.emplace(); + output_stream->header = std::move(normal_parts_header); +} + +QueryPipelineBuilderPtr AggregatingProjectionStep::updatePipeline( + QueryPipelineBuilders pipelines, + const BuildQueryPipelineSettings &) +{ + auto & normal_parts_pipeline = pipelines.front(); + auto & projection_parts_pipeline = pipelines.back(); + + /// Here we create shared ManyAggregatedData for both projection and ordinary data. + /// For ordinary data, AggregatedData is filled in a usual way. + /// For projection data, AggregatedData is filled by merging aggregation states. + /// When all AggregatedData is filled, we merge aggregation states together in a usual way. + /// Pipeline will look like: + /// ReadFromProjection -> Aggregating (only merge states) -> + /// ReadFromProjection -> Aggregating (only merge states) -> + /// ... -> Resize -> ConvertingAggregatedToChunks + /// ReadFromOrdinaryPart -> Aggregating (usual) -> (added by last Aggregating) + /// ReadFromOrdinaryPart -> Aggregating (usual) -> + /// ... + auto many_data = std::make_shared(normal_parts_pipeline->getNumStreams() + projection_parts_pipeline->getNumStreams()); + size_t counter = 0; + + AggregatorListPtr aggregator_list_ptr = std::make_shared(); + + /// TODO apply optimize_aggregation_in_order here too (like below) + auto build_aggregate_pipeline = [&](QueryPipelineBuilder & pipeline, bool projection) + { + auto params_copy = params; + if (projection) + params_copy.only_merge = true; + + AggregatingTransformParamsPtr transform_params = std::make_shared( + pipeline.getHeader(), std::move(params_copy), aggregator_list_ptr, final); + + pipeline.resize(pipeline.getNumStreams(), true, true); + + pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, transform_params, many_data, counter++, merge_threads, temporary_data_merge_threads); + }); + }; + + build_aggregate_pipeline(*normal_parts_pipeline, false); + build_aggregate_pipeline(*projection_parts_pipeline, true); + + auto pipeline = std::make_unique(); + + for (auto & cur_pipeline : pipelines) + assertBlocksHaveEqualStructure(cur_pipeline->getHeader(), getOutputStream().header, "AggregatingProjectionStep"); + + *pipeline = QueryPipelineBuilder::unitePipelines(std::move(pipelines), 0, &processors); + pipeline->resize(1); + return pipeline; +} + } diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index 2ceca926684..19c6a3a9d45 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -22,6 +22,8 @@ using GroupingSetsParamsList = std::vector; Block appendGroupingSetColumn(Block header); Block generateOutputHeader(const Block & input_header, const Names & keys, bool use_nulls); +class AggregatingProjectionStep; + /// Aggregation. See AggregatingTransform. class AggregatingStep : public ITransformingStep { @@ -60,8 +62,9 @@ public: void applyOrder(SortDescription sort_description_for_merging_, SortDescription group_by_sort_description_); bool memoryBoundMergingWillBeUsed() const; + bool canUseProjection() const; void requestOnlyMergeForAggregateProjection(const DataStream & input_stream); - void appendAggregateProjection(const DataStream & input_stream); + std::unique_ptr convertToAggregatingProjection(const DataStream & input_stream) const; private: void updateOutputStream() override; @@ -95,4 +98,39 @@ private: Processors aggregating; }; +class AggregatingProjectionStep : public IQueryPlanStep +{ +public: + AggregatingProjectionStep( + DataStreams input_streams_, + Aggregator::Params params_, + bool final_, + //size_t max_block_size_, + size_t merge_threads_, + size_t temporary_data_merge_threads_ + //bool group_by_use_nulls_, + //SortDescription group_by_sort_description_, + //bool should_produce_results_in_order_of_bucket_number_ + //bool memory_bound_merging_of_aggregation_results_enabled_ + ); + + String getName() const override { return "AggregatingProjection"; } + QueryPipelineBuilderPtr updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings &) override; + +private: + Aggregator::Params params; + bool final; + //size_t max_block_size; + size_t merge_threads; + size_t temporary_data_merge_threads; + + //bool storage_has_evenly_distributed_read; + //bool group_by_use_nulls; + //SortDescription group_by_sort_description; + //bool should_produce_results_in_order_of_bucket_number; + //bool memory_bound_merging_of_aggregation_results_enabled; + + Processors aggregating; +}; + } diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 2c9ad818ed4..e15a865a809 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -374,6 +374,9 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (!aggregating) return; + if (!aggregating->canUseProjection()) + return; + QueryPlan::Node * reading_node = findReadingStep(node); if (!reading_node) return; @@ -530,36 +533,39 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) projection_reading->setStepDescription(best_candidate->projection->name); + auto & projection_reading_node = nodes.emplace_back(QueryPlan::Node{.step = std::move(projection_reading)}); + auto & expr_or_filter_node = nodes.emplace_back(); + + if (filter_node) + { + expr_or_filter_node.step = std::make_unique( + projection_reading_node.step->getOutputStream(), + best_candidate->dag, + best_candidate->dag->getOutputs().front()->result_name, + true); + } + else + expr_or_filter_node.step = std::make_unique( + projection_reading_node.step->getOutputStream(), + best_candidate->dag); + + expr_or_filter_node.children.push_back(&projection_reading_node); + if (!best_candidate->merge_tree_normal_select_result_ptr) { /// All parts are taken from projection - auto & projection_reading_node = nodes.emplace_back(QueryPlan::Node{.step = std::move(projection_reading)}); - auto & expr_or_filter_node = nodes.emplace_back(); - if (filter_node) - { - expr_or_filter_node.step = std::make_unique( - projection_reading_node.step->getOutputStream(), - best_candidate->dag, - best_candidate->dag->getOutputs().front()->result_name, - true); - } - else - expr_or_filter_node.step = std::make_unique( - projection_reading_node.step->getOutputStream(), - best_candidate->dag); - - expr_or_filter_node.children.push_back(&projection_reading_node); aggregating->requestOnlyMergeForAggregateProjection(expr_or_filter_node.step->getOutputStream()); node.children.front() = &expr_or_filter_node; optimizeAggregationInOrder(node, nodes); - - return; } - - + else + { + node.step = aggregating->convertToAggregatingProjection(expr_or_filter_node.step->getOutputStream()); + node.children.push_back(&expr_or_filter_node); + } } } diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 7bad9947a88..2e0ad116d70 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -67,7 +67,7 @@ public: size_t max_block_size, size_t num_streams) override { - query_plan = std::move(*MergeTreeDataSelectExecutor(storage) + query_plan.addStep(MergeTreeDataSelectExecutor(storage) .readFromParts( parts, column_names, From 310a4c69af253d1013666108dac41fa080f6431e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 3 Feb 2023 18:30:06 +0000 Subject: [PATCH 009/559] Aggregate Projections analysis using query plan [In progress] --- src/Core/Settings.h | 1 + .../QueryPlanOptimizationSettings.cpp | 1 + .../QueryPlanOptimizationSettings.h | 3 ++ .../Optimizations/optimizeReadInOrder.cpp | 8 +-- .../QueryPlan/Optimizations/optimizeTree.cpp | 3 ++ .../Optimizations/optimizeUseProjections.cpp | 51 ++++++++++++------- src/Storages/MergeTree/MergeTreeData.cpp | 3 ++ 7 files changed, 47 insertions(+), 23 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b8d46244b6c..9ab1e274dff 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -587,6 +587,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, query_plan_optimize_primary_key, true, "Analyze primary key using query plan (instead of AST)", 0) \ M(Bool, query_plan_read_in_order, true, "Use query plan for read-in-order optimisation", 0) \ M(Bool, query_plan_aggregation_in_order, true, "Use query plan for aggregation-in-order optimisation", 0) \ + M(Bool, query_plan_optimize_projection, true, "Use query plan for aggregation-in-order optimisation", 0) \ M(UInt64, regexp_max_matches_per_row, 1000, "Max matches of any single regexp per row, used to safeguard 'extractAllGroupsHorizontal' against consuming too much memory with greedy RE.", 0) \ \ M(UInt64, limit, 0, "Limit on read rows from the most 'end' result for select query, default 0 means no limit length", 0) \ diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp index 00abd803d2a..dcc37d197cb 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp @@ -14,6 +14,7 @@ QueryPlanOptimizationSettings QueryPlanOptimizationSettings::fromSettings(const settings.distinct_in_order = from.optimize_distinct_in_order; settings.read_in_order = from.optimize_read_in_order && from.query_plan_read_in_order; settings.aggregation_in_order = from.optimize_aggregation_in_order && from.query_plan_aggregation_in_order; + settings.optimize_projection = from.allow_experimental_projection_optimization && from.query_plan_optimize_projection; return settings; } diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h index d4989b86b68..835a8307188 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h @@ -30,6 +30,9 @@ struct QueryPlanOptimizationSettings /// If aggregation-in-order optimisation is enabled bool aggregation_in_order = false; + /// If reading from projection can be applied + bool optimize_projection = false; + static QueryPlanOptimizationSettings fromSettings(const Settings & from); static QueryPlanOptimizationSettings fromContext(ContextPtr from); }; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 1876ebc4014..80dca5d3a27 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -29,7 +29,7 @@ namespace DB::QueryPlanOptimizations { -ISourceStep * checkSupportedReadingStep(IQueryPlanStep * step) +static ISourceStep * checkSupportedReadingStep(IQueryPlanStep * step) { if (auto * reading = typeid_cast(step)) { @@ -64,7 +64,7 @@ ISourceStep * checkSupportedReadingStep(IQueryPlanStep * step) return nullptr; } -QueryPlan::Node * findReadingStep(QueryPlan::Node & node) +static QueryPlan::Node * findReadingStep(QueryPlan::Node & node) { IQueryPlanStep * step = node.step.get(); if (auto * reading = checkSupportedReadingStep(step)) @@ -89,7 +89,7 @@ using FixedColumns = std::unordered_set; /// Right now we find only simple cases like 'and(..., and(..., and(column = value, ...), ...' /// Injective functions are supported here. For a condition 'injectiveFunction(x) = 5' column 'x' is fixed. -void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filter_expression, FixedColumns & fixed_columns) +static void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filter_expression, FixedColumns & fixed_columns) { std::stack stack; stack.push(&filter_expression); @@ -138,7 +138,7 @@ void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filter_expr } } -void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) +static void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) { if (dag) dag->mergeInplace(std::move(*expression->clone())); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 13095dfad47..e6568db1263 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -114,6 +114,9 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s if (optimization_settings.read_in_order) optimizeReadInOrder(*frame.node, nodes); + if (optimization_settings.optimize_projection) + optimizeUseProjections(*frame.node, nodes); + if (optimization_settings.aggregation_in_order) optimizeAggregationInOrder(*frame.node, nodes); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index e15a865a809..e2a942ed2dc 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -11,26 +11,17 @@ #include #include #include +#include #include namespace DB::QueryPlanOptimizations { -QueryPlan::Node * findReadingStep(QueryPlan::Node & node) +static QueryPlan::Node * findReadingStep(QueryPlan::Node & node) { IQueryPlanStep * step = node.step.get(); if (auto * reading = typeid_cast(step)) - { - /// Already read-in-order, skip. - if (reading->getQueryInfo().input_order_info) - return nullptr; - - const auto & sorting_key = reading->getStorageMetadata()->getSortingKey(); - if (sorting_key.column_names.empty()) - return nullptr; - return &node; - } if (node.children.size() != 1) return nullptr; @@ -41,7 +32,7 @@ QueryPlan::Node * findReadingStep(QueryPlan::Node & node) return nullptr; } -void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) +static void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) { if (dag) dag->mergeInplace(std::move(*expression->clone())); @@ -51,7 +42,7 @@ void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) /// This function builds a common DAG which is a gerge of DAGs from Filter and Expression steps chain. -bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, ActionsDAG::NodeRawConstPtrs & filter_nodes) +static bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, ActionsDAG::NodeRawConstPtrs & filter_nodes) { IQueryPlanStep * step = node.step.get(); if (auto * reading = typeid_cast(step)) @@ -92,6 +83,7 @@ bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, ActionsDAG return false; appendExpression(dag, actions); + return true; } if (auto * filter = typeid_cast(step)) @@ -101,10 +93,12 @@ bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, ActionsDAG return false; appendExpression(dag, actions); - if (const auto * filter_expression = dag->tryFindInOutputs(filter->getFilterColumnName())) - filter_nodes.push_back(filter_expression); - else + const auto * filter_expression = dag->tryFindInOutputs(filter->getFilterColumnName()); + if (!filter_expression) return false; + + filter_nodes.push_back(filter_expression); + return true; } return false; @@ -374,17 +368,23 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (!aggregating) return; + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 2"); if (!aggregating->canUseProjection()) return; - QueryPlan::Node * reading_node = findReadingStep(node); + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 3"); + QueryPlan::Node * reading_node = findReadingStep(*node.children.front()); if (!reading_node) return; + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 4"); + auto * reading = typeid_cast(reading_node->step.get()); if (!reading) return; + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 5"); + const auto metadata = reading->getStorageMetadata(); const auto & projections = metadata->projections; @@ -396,11 +396,15 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (agg_projections.empty()) return; + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Has agg projection"); + ActionsDAGPtr dag; ActionsDAG::NodeRawConstPtrs filter_nodes; if (!buildAggregatingDAG(*node.children.front(), dag, filter_nodes)) return; + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", dag->dumpDAG()); + const ActionsDAG::Node * filter_node = nullptr; if (!filter_nodes.empty()) { @@ -426,9 +430,13 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) candidates.reserve(agg_projections.size()); for (const auto * projection : agg_projections) { + + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try projection {}", projection->name); auto info = getAggregatingProjectionInfo(*projection, context, metadata); + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection DAG {}", info.before_aggregation->dumpDAG()); if (auto proj_dag = analyzeAggregateProjection(info, *dag, keys, aggregates)) { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); candidates.emplace_back(AggregateProjectionCandidate{ .info = std::move(info), .projection = projection, @@ -519,10 +527,15 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (!best_candidate) return; + auto storage_snapshot = reading->getStorageSnapshot(); + auto proj_snapshot = std::make_shared( + storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); //, storage_snapshot->data); + proj_snapshot->addProjection(best_candidate->projection); + auto projection_reading = reader.readFromParts( {}, best_candidate->dag->getRequiredColumnsNames(), - reading->getStorageSnapshot(), + proj_snapshot, query_info, context, reading->getMaxBlockSize(), @@ -559,7 +572,7 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) aggregating->requestOnlyMergeForAggregateProjection(expr_or_filter_node.step->getOutputStream()); node.children.front() = &expr_or_filter_node; - optimizeAggregationInOrder(node, nodes); + //optimizeAggregationInOrder(node, nodes); } else { diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 2a341b6f1de..dc6c70f66cd 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6159,6 +6159,9 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg if (analysis_result.join != nullptr || analysis_result.array_join != nullptr) can_use_aggregate_projection = false; + if (settings.query_plan_optimize_projection) + can_use_aggregate_projection = false; + /// Check if all needed columns can be provided by some aggregate projection. Here we also try /// to find expression matches. For example, suppose an aggregate projection contains a column /// named sum(x) and the given query also has an expression called sum(x), it's a match. This is From b31decc15a76474bebddd357d4009a7254331f70 Mon Sep 17 00:00:00 2001 From: alexX512 Date: Sun, 5 Feb 2023 12:54:50 +0000 Subject: [PATCH 010/559] Fix bug with wrong checking of execution finish in PullingAsyncPipeline --- src/Processors/Executors/PullingAsyncPipelineExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index fbbf8c119ce..0a7a9025b30 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -117,7 +117,7 @@ bool PullingAsyncPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds) data->rethrowExceptionIfHas(); bool is_execution_finished - = !data->executor->checkTimeLimitSoft() || lazy_format ? lazy_format->isFinished() : data->is_finished.load(); + = !data->executor->checkTimeLimitSoft() || (lazy_format ? lazy_format->isFinished() : data->is_finished.load()); if (is_execution_finished) { From a542626fa35a7f5a37eb98358738554188f74a3c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 6 Feb 2023 18:42:58 +0000 Subject: [PATCH 011/559] Make 01710_aggregate_projections work. --- src/Interpreters/ActionsDAG.cpp | 19 +++++--- src/Interpreters/ActionsDAG.h | 2 +- src/Processors/QueryPlan/AggregatingStep.cpp | 3 +- .../Optimizations/optimizeUseProjections.cpp | 47 ++++++++++++------- 4 files changed, 46 insertions(+), 25 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index d2a1d7becfe..0df7b71f719 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -724,7 +724,7 @@ NameSet ActionsDAG::foldActionsByProjection( } -ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_map & new_inputs) +ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_map & new_inputs, const NodeRawConstPtrs & required_outputs) { auto dag = std::make_unique(); std::unordered_map new_input_to_pos; @@ -737,7 +737,7 @@ ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_map stack; - for (const auto * output : outputs) + for (const auto * output : required_outputs) { if (mapping.contains(output)) continue; @@ -754,11 +754,15 @@ ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_mapaddInput(new_input->result_name, new_input->result_type); - if (!rename.empty() && new_input->result_name != rename) - node = &dag->addAlias(*node, rename); + auto & node = mapping[frame.node]; + + if (!node) + { + node = &dag->addInput(new_input->result_name, new_input->result_type); + if (!rename.empty() && new_input->result_name != rename) + node = &dag->addAlias(*node, rename); + } - mapping.emplace(frame.node, node); stack.pop_back(); continue; } @@ -786,11 +790,12 @@ ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_mapoutputs.push_back(mapping[output]); return dag; diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index b23c87b4903..93280c66668 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -214,7 +214,7 @@ public: const String & predicate_column_name = {}, bool add_missing_keys = true); - ActionsDAGPtr foldActionsByProjection(const std::unordered_map & new_inputs); + static ActionsDAGPtr foldActionsByProjection(const std::unordered_map & new_inputs, const NodeRawConstPtrs & required_outputs); /// Reorder the output nodes using given position mapping. void reorderAggregationKeysForProjection(const std::unordered_map & key_names_pos_map); diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index db85764ef33..6aacc2f8fae 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -467,7 +467,8 @@ void AggregatingStep::describePipeline(FormatSettings & settings) const bool AggregatingStep::canUseProjection() const { - return grouping_sets_params.empty() && sort_description_for_merging.empty() && !memory_bound_merging_of_aggregation_results_enabled; + //LOG_TRACE(&Poco::Logger::get("AggregatingStep"), "canUseProjection {} {} {}", grouping_sets_params.size(), sort_description_for_merging.size(), memory_bound_merging_of_aggregation_results_enabled); + return grouping_sets_params.empty() && sort_description_for_merging.empty(); // && !memory_bound_merging_of_aggregation_results_enabled; } void AggregatingStep::requestOnlyMergeForAggregateProjection(const DataStream & input_stream) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index e2a942ed2dc..185cc10e5ff 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -154,6 +154,7 @@ struct AggregateProjectionCandidate ActionsDAGPtr analyzeAggregateProjection( const AggregateProjectionInfo & info, ActionsDAG & query_dag, + const ActionsDAG::Node * filter_node, const Names & keys, const AggregateDescriptions & aggregates) { @@ -169,7 +170,11 @@ ActionsDAGPtr analyzeAggregateProjection( for (const auto * output : info.before_aggregation->getOutputs()) proj_index.emplace(output->result_name, output); - key_nodes.reserve(keys.size()); + key_nodes.reserve(keys.size() + 1); + + if (filter_node) + key_nodes.push_back(filter_node); + for (const auto & key : keys) { auto it = index.find(key); @@ -270,8 +275,8 @@ ActionsDAGPtr analyzeAggregateProjection( if (args.size() < aggregate.argument_names.size()) continue; - for (const auto * node : args) - split_nodes.insert(node); + // for (const auto * node : args) + // split_nodes.insert(node); match = AggFuncMatch{idx, std::move(args)}; } @@ -302,14 +307,8 @@ ActionsDAGPtr analyzeAggregateProjection( }; std::stack stack; - for (const auto & key : keys) + for (const auto * key_node : key_nodes) { - auto it = index.find(key); - /// This should not happen ideally. - if (it == index.end()) - break; - - const auto * key_node = it->second; if (visited.contains(key_node)) continue; @@ -343,8 +342,11 @@ ActionsDAGPtr analyzeAggregateProjection( } /// Not a match and there is no matched child. - if (frame.node->children.empty()) + if (frame.node->type == ActionsDAG::ActionType::INPUT) + { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot find match for {}", frame.node->result_name); return {}; + } /// Not a match, but all children matched. visited.insert(frame.node); @@ -356,7 +358,13 @@ ActionsDAGPtr analyzeAggregateProjection( for (const auto * node : split_nodes) new_inputs[node] = matches[node].node->result_name; - return query_dag.foldActionsByProjection(new_inputs); + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Folding actions by projection"); + auto proj_dag = query_dag.foldActionsByProjection(new_inputs, key_nodes); + auto & proj_dag_outputs = proj_dag->getOutputs(); + for (const auto & aggregate : aggregates) + proj_dag_outputs.push_back(&proj_dag->addInput(aggregate.column_name, aggregate.function->getResultType())); + + return proj_dag; } void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) @@ -418,7 +426,7 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) filter_node = &dag->addFunction(func_builder_and, std::move(filter_nodes), {}); } - dag->getOutputs().insert(dag->getOutputs().begin(), filter_node); + dag->getOutputs().push_back(filter_node); } ContextPtr context = reading->getContext(); @@ -434,7 +442,7 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try projection {}", projection->name); auto info = getAggregatingProjectionInfo(*projection, context, metadata); LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection DAG {}", info.before_aggregation->dumpDAG()); - if (auto proj_dag = analyzeAggregateProjection(info, *dag, keys, aggregates)) + if (auto proj_dag = analyzeAggregateProjection(info, *dag, filter_node, keys, aggregates)) { LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); candidates.emplace_back(AggregateProjectionCandidate{ @@ -532,11 +540,16 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); //, storage_snapshot->data); proj_snapshot->addProjection(best_candidate->projection); + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}", proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString()); + + auto query_info_copy = query_info; + query_info_copy.prewhere_info = nullptr; + auto projection_reading = reader.readFromParts( {}, best_candidate->dag->getRequiredColumnsNames(), proj_snapshot, - query_info, + query_info_copy, context, reading->getMaxBlockSize(), reading->getNumStreams(), @@ -544,6 +557,8 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) best_candidate->merge_tree_projection_select_result_ptr, reading->isParallelReadingEnabled()); + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}", projection_reading->getOutputStream().header.dumpStructure()); + projection_reading->setStepDescription(best_candidate->projection->name); auto & projection_reading_node = nodes.emplace_back(QueryPlan::Node{.step = std::move(projection_reading)}); @@ -568,7 +583,7 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { /// All parts are taken from projection - + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Expr stream {}", expr_or_filter_node.step->getOutputStream().header.dumpStructure()); aggregating->requestOnlyMergeForAggregateProjection(expr_or_filter_node.step->getOutputStream()); node.children.front() = &expr_or_filter_node; From b2cc2332f5883b9bdca2587aa9e4a33ee916c355 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 6 Feb 2023 19:20:03 +0000 Subject: [PATCH 012/559] Fix 01710_force_use_projection. --- .../QueryPlan/Optimizations/Optimizations.h | 2 +- .../QueryPlanOptimizationSettings.cpp | 1 + .../QueryPlanOptimizationSettings.h | 1 + .../QueryPlan/Optimizations/optimizeTree.cpp | 14 +++++++++++- .../Optimizations/optimizeUseProjections.cpp | 22 ++++++++++--------- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 5 +++-- 6 files changed, 31 insertions(+), 14 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 6f733c75e88..0ee2cecb4df 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -93,7 +93,7 @@ using Stack = std::vector; void optimizePrimaryKeyCondition(const Stack & stack); void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); -void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes &); +bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes &); /// Enable memory bound merging of aggregation states for remote queries /// in case it was enabled for local plan diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp index dcc37d197cb..68b078ecd0a 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp @@ -15,6 +15,7 @@ QueryPlanOptimizationSettings QueryPlanOptimizationSettings::fromSettings(const settings.read_in_order = from.optimize_read_in_order && from.query_plan_read_in_order; settings.aggregation_in_order = from.optimize_aggregation_in_order && from.query_plan_aggregation_in_order; settings.optimize_projection = from.allow_experimental_projection_optimization && from.query_plan_optimize_projection; + settings.force_use_projection = settings.optimize_projection && from.force_optimize_projection; return settings; } diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h index 835a8307188..7f9cc43eacd 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h @@ -32,6 +32,7 @@ struct QueryPlanOptimizationSettings /// If reading from projection can be applied bool optimize_projection = false; + bool force_use_projection = false; static QueryPlanOptimizationSettings fromSettings(const Settings & from); static QueryPlanOptimizationSettings fromContext(ContextPtr from); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index e6568db1263..d0d634d931d 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -11,6 +12,7 @@ namespace DB namespace ErrorCodes { extern const int TOO_MANY_QUERY_PLAN_OPTIMIZATIONS; + extern const int PROJECTION_NOT_USED; } namespace QueryPlanOptimizations @@ -102,6 +104,9 @@ void optimizeTreeFirstPass(const QueryPlanOptimizationSettings & settings, Query void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes) { + bool applied_projection = false; + bool has_reading_from_mt = false; + Stack stack; stack.push_back({.node = &root}); @@ -111,11 +116,13 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s if (frame.next_child == 0) { + has_reading_from_mt |= typeid_cast(frame.node->step.get()) != nullptr; + if (optimization_settings.read_in_order) optimizeReadInOrder(*frame.node, nodes); if (optimization_settings.optimize_projection) - optimizeUseProjections(*frame.node, nodes); + applied_projection |= optimizeUseProjections(*frame.node, nodes); if (optimization_settings.aggregation_in_order) optimizeAggregationInOrder(*frame.node, nodes); @@ -138,6 +145,11 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s stack.pop_back(); } + + if (optimization_settings.force_use_projection && has_reading_from_mt && !applied_projection) + throw Exception( + "No projection is used when allow_experimental_projection_optimization = 1 and force_optimize_projection = 1", + ErrorCodes::PROJECTION_NOT_USED); } } diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 185cc10e5ff..15ec1a9e0ca 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -367,29 +367,29 @@ ActionsDAGPtr analyzeAggregateProjection( return proj_dag; } -void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) +bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { if (node.children.size() != 1) - return; + return false; auto * aggregating = typeid_cast(node.step.get()); if (!aggregating) - return; + return false; LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 2"); if (!aggregating->canUseProjection()) - return; + return false; LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 3"); QueryPlan::Node * reading_node = findReadingStep(*node.children.front()); if (!reading_node) - return; + return false; LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 4"); auto * reading = typeid_cast(reading_node->step.get()); if (!reading) - return; + return false; LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 5"); @@ -402,14 +402,14 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) agg_projections.push_back(&projection); if (agg_projections.empty()) - return; + return false; LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Has agg projection"); ActionsDAGPtr dag; ActionsDAG::NodeRawConstPtrs filter_nodes; if (!buildAggregatingDAG(*node.children.front(), dag, filter_nodes)) - return; + return false; LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", dag->dumpDAG()); @@ -454,7 +454,7 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) } if (candidates.empty()) - return; + return false; AggregateProjectionCandidate * best_candidate = nullptr; size_t best_candidate_marks = 0; @@ -533,7 +533,7 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) } if (!best_candidate) - return; + return false; auto storage_snapshot = reading->getStorageSnapshot(); auto proj_snapshot = std::make_shared( @@ -594,6 +594,8 @@ void optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) node.step = aggregating->convertToAggregatingProjection(expr_or_filter_node.step->getOutputStream()); node.children.push_back(&expr_or_filter_node); } + + return true; } } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 242f86e171c..e4b0d266de0 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -169,13 +169,14 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( enable_parallel_reading); if (!step && settings.allow_experimental_projection_optimization && settings.force_optimize_projection - && !metadata_for_reading->projections.empty()) + && !metadata_for_reading->projections.empty() && !settings.query_plan_optimize_projection) throw Exception( "No projection is used when allow_experimental_projection_optimization = 1 and force_optimize_projection = 1", ErrorCodes::PROJECTION_NOT_USED); auto plan = std::make_unique(); - plan->addStep(std::move(step)); + if (step) + plan->addStep(std::move(step)); return plan; } From f30d1efe2f0d937943adb9d8c109f69adafdd6ca Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 8 Feb 2023 13:32:34 +0000 Subject: [PATCH 013/559] Fix test 01710_projections. --- .../QueryPlan/Optimizations/matchTrees.cpp | 31 +++++++++---- .../Optimizations/optimizeUseProjections.cpp | 44 +++++++++++++++++++ 2 files changed, 67 insertions(+), 8 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/matchTrees.cpp b/src/Processors/QueryPlan/Optimizations/matchTrees.cpp index 4e11f65f1e2..6c61b557477 100644 --- a/src/Processors/QueryPlan/Optimizations/matchTrees.cpp +++ b/src/Processors/QueryPlan/Optimizations/matchTrees.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include namespace DB::QueryPlanOptimizations @@ -101,14 +102,20 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG //std::cerr << "... Processing " << frame.node->function_base->getName() << std::endl; bool found_all_children = true; - for (const auto * child : frame.mapped_children) - if (!child) + const ActionsDAG::Node * any_child = nullptr; + size_t num_children = frame.node->children.size(); + for (size_t i = 0; i < num_children; ++i) + { + if (frame.mapped_children[i]) + any_child = frame.mapped_children[i]; + else if (!frame.node->children[i]->column || !isColumnConst(*frame.node->children[i]->column)) found_all_children = false; + } - if (found_all_children && !frame.mapped_children.empty()) + if (found_all_children && any_child) { Parents container; - Parents * intersection = &inner_parents[frame.mapped_children[0]]; + Parents * intersection = &inner_parents[any_child]; if (frame.mapped_children.size() > 1) { @@ -116,7 +123,8 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG size_t mapped_children_size = frame.mapped_children.size(); other_parents.reserve(mapped_children_size); for (size_t i = 1; i < mapped_children_size; ++i) - other_parents.push_back(&inner_parents[frame.mapped_children[i]]); + if (frame.mapped_children[i]) + other_parents.push_back(&inner_parents[frame.mapped_children[i]]); for (const auto * parent : *intersection) { @@ -148,12 +156,19 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG if (parent->type == ActionsDAG::ActionType::FUNCTION && func_name == parent->function_base->getName()) { const auto & children = parent->children; - size_t num_children = children.size(); - if (frame.mapped_children.size() == num_children) + if (children.size() == num_children) { bool all_children_matched = true; for (size_t i = 0; all_children_matched && i < num_children; ++i) - all_children_matched = frame.mapped_children[i] == children[i]; + { + if (frame.mapped_children[i] == nullptr) + { + all_children_matched = children[i]->column && isColumnConst(*children[i]->column) + && assert_cast(*children[i]->column).getField() == assert_cast(*frame.node->children[i]->column).getField(); + } + else + all_children_matched = frame.mapped_children[i] == children[i]; + } if (all_children_matched) { diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 15ec1a9e0ca..88eb366a3a8 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -12,6 +12,8 @@ #include #include #include +#include +#include #include namespace DB::QueryPlanOptimizations @@ -180,7 +182,10 @@ ActionsDAGPtr analyzeAggregateProjection( auto it = index.find(key); /// This should not happen ideally. if (it == index.end()) + { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot find key {} in query DAG", key); return {}; + } key_nodes.push_back(it->second); } @@ -192,13 +197,22 @@ ActionsDAGPtr analyzeAggregateProjection( auto it = index.find(argument); /// This should not happen ideally. if (it == index.end()) + { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot find arg {} for agg functions {}", argument, aggregate.column_name); return {}; + } aggregate_args.insert(it->second); } } MatchedTrees::Matches matches = matchTrees(*info.before_aggregation, query_dag); + for (const auto & [node, match] : matches) + { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Match {} {} -> {} {} (with monotonicity : {})", + static_cast(node), node->result_name, + static_cast(match.node), (match.node ? match.node->result_name : ""), match.monotonicity != std::nullopt); + } std::unordered_map> projection_aggregate_functions; for (size_t i = 0; i < info.aggregates.size(); ++i) @@ -221,7 +235,10 @@ ActionsDAGPtr analyzeAggregateProjection( { auto it = projection_aggregate_functions.find(aggregate.function->getName()); if (it == projection_aggregate_functions.end()) + { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot match agg func {} by name {}", aggregate.column_name, aggregate.function->getName()); return {}; + } auto & candidates = it->second; std::optional match; @@ -237,7 +254,12 @@ ActionsDAGPtr analyzeAggregateProjection( /// But also functions sum(...) and sumIf(...) will have equal states, /// and we can't replace one to another from projection. if (!candidate.function->getStateType()->equals(*aggregate.function->getStateType())) + { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot match agg func {} vs {} by state {} vs {}", + aggregate.column_name, candidate.column_name, + candidate.function->getStateType()->getName(), aggregate.function->getStateType()->getName()); continue; + } ActionsDAG::NodeRawConstPtrs args; size_t num_args = aggregate.argument_names.size(); @@ -250,24 +272,40 @@ ActionsDAGPtr analyzeAggregateProjection( auto jt = index.find(query_name); /// This should not happen ideally. if (jt == index.end()) + { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot match agg func {} vs {} : can't find arg {} in query dag", + aggregate.column_name, candidate.column_name, query_name); break; + } const auto * query_node = jt->second; auto kt = proj_index.find(proj_name); /// This should not happen ideally. if (kt == proj_index.end()) + { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot match agg func {} vs {} : can't find arg {} in proj dag", + aggregate.column_name, candidate.column_name, proj_name); break; + } const auto * proj_node = kt->second; auto mt = matches.find(query_node); if (mt == matches.end()) + { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot match agg func {} vs {} : can't match arg {} vs {} : no node in map", + aggregate.column_name, candidate.column_name, query_name, proj_name); break; + } const auto & node_match = mt->second; if (node_match.node != proj_node || node_match.monotonicity) + { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot match agg func {} vs {} : can't match arg {} vs {} : no match or monotonicity", + aggregate.column_name, candidate.column_name, query_name, proj_name); break; + } args.push_back(query_node); } @@ -557,6 +595,12 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) best_candidate->merge_tree_projection_select_result_ptr, reading->isParallelReadingEnabled()); + if (!projection_reading) + { + Pipe pipe(std::make_shared(proj_snapshot->getSampleBlockForColumns(best_candidate->dag->getRequiredColumnsNames()))); + projection_reading = std::make_unique(std::move(pipe)); + } + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}", projection_reading->getOutputStream().header.dumpStructure()); projection_reading->setStepDescription(best_candidate->projection->name); From d04b3a30cc8a53de6800e428706fc1147b0bd92a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 8 Feb 2023 14:26:20 +0000 Subject: [PATCH 014/559] Fix 01710_projections_partial_optimize_aggregation_in_order --- .../QueryPlan/Optimizations/optimizeUseProjections.cpp | 1 + src/Processors/QueryPlan/ReadFromMergeTree.h | 2 ++ 2 files changed, 3 insertions(+) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 88eb366a3a8..6ec521a9d77 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -637,6 +637,7 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { node.step = aggregating->convertToAggregatingProjection(expr_or_filter_node.step->getOutputStream()); node.children.push_back(&expr_or_filter_node); + reading->setAnalyzedResult(std::move(best_candidate->merge_tree_normal_select_result_ptr)); } return true; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 1c614ed09f1..5ab45331722 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -157,6 +157,8 @@ public: void requestReadingInOrder(size_t prefix_size, int direction, size_t limit); + void setAnalyzedResult(MergeTreeDataSelectAnalysisResultPtr analyzed_result_ptr_) { analyzed_result_ptr = std::move(analyzed_result_ptr_); } + const MergeTreeData::DataPartsVector & getParts() const { return prepared_parts; } const MergeTreeData & getMergeTreeData() const { return data; } const Names & getRealColumnNames() const { return real_column_names; } From e7dba2a85b96b77eac4bbcd61d9894247c5a2dd5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 8 Feb 2023 18:34:22 +0000 Subject: [PATCH 015/559] Partly support minmax projection. --- .../Optimizations/optimizeUseProjections.cpp | 147 +++++++++++++----- src/Processors/QueryPlan/ReadFromMergeTree.h | 1 + 2 files changed, 106 insertions(+), 42 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 6ec521a9d77..ca16bd09649 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -151,6 +152,8 @@ struct AggregateProjectionCandidate MergeTreeDataSelectAnalysisResultPtr merge_tree_projection_select_result_ptr; MergeTreeDataSelectAnalysisResultPtr merge_tree_normal_select_result_ptr; + + size_t sum_marks = 0; }; ActionsDAGPtr analyzeAggregateProjection( @@ -261,6 +264,16 @@ ActionsDAGPtr analyzeAggregateProjection( continue; } + if (typeid_cast(candidate.function.get())) + { + /// we can ignore arguments for count() + match = AggFuncMatch{idx, {}}; + break; + } + + if (aggregate.argument_names.size() != candidate.argument_names.size()) + continue; + ActionsDAG::NodeRawConstPtrs args; size_t num_args = aggregate.argument_names.size(); args.reserve(num_args); @@ -317,6 +330,7 @@ ActionsDAGPtr analyzeAggregateProjection( // split_nodes.insert(node); match = AggFuncMatch{idx, std::move(args)}; + break; } if (!match) @@ -434,12 +448,14 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) const auto metadata = reading->getStorageMetadata(); const auto & projections = metadata->projections; + bool can_use_minmax_projection = metadata->minmax_count_projection && !reading->getMergeTreeData().has_lightweight_delete_parts.load(); + std::vector agg_projections; for (const auto & projection : projections) if (projection.type == ProjectionDescription::Type::Aggregate) agg_projections.push_back(&projection); - if (agg_projections.empty()) + if (!can_use_minmax_projection && agg_projections.empty()) return false; LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Has agg projection"); @@ -473,29 +489,52 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) const auto & aggregates = aggregating->getParams().aggregates; std::vector candidates; - candidates.reserve(agg_projections.size()); - for (const auto * projection : agg_projections) - { + std::optional minmax_projection; + AggregateProjectionCandidate * best_candidate = nullptr; + + if (can_use_minmax_projection) + { + const auto * projection = &*(metadata->minmax_count_projection); LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try projection {}", projection->name); auto info = getAggregatingProjectionInfo(*projection, context, metadata); LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection DAG {}", info.before_aggregation->dumpDAG()); if (auto proj_dag = analyzeAggregateProjection(info, *dag, filter_node, keys, aggregates)) { LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); - candidates.emplace_back(AggregateProjectionCandidate{ + minmax_projection.emplace(AggregateProjectionCandidate{ .info = std::move(info), .projection = projection, .dag = std::move(proj_dag), }); } + + best_candidate = &*minmax_projection; } - if (candidates.empty()) - return false; + if (!minmax_projection) + { + candidates.reserve(agg_projections.size()); + for (const auto * projection : agg_projections) + { - AggregateProjectionCandidate * best_candidate = nullptr; - size_t best_candidate_marks = 0; + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try projection {}", projection->name); + auto info = getAggregatingProjectionInfo(*projection, context, metadata); + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection DAG {}", info.before_aggregation->dumpDAG()); + if (auto proj_dag = analyzeAggregateProjection(info, *dag, filter_node, keys, aggregates)) + { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); + candidates.emplace_back(AggregateProjectionCandidate{ + .info = std::move(info), + .projection = projection, + .dag = std::move(proj_dag), + }); + } + } + + if (candidates.empty()) + return false; + } const auto & parts = reading->getParts(); const auto & query_info = reading->getQueryInfo(); @@ -531,7 +570,7 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) added_filter_nodes.nodes.push_back(candidate.dag->getOutputs().front()); auto projection_result_ptr = reader.estimateNumMarksToRead( - projection_parts, + std::move(projection_parts), nullptr, candidate.dag->getRequiredColumnsNames(), metadata, @@ -545,7 +584,8 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (projection_result_ptr->error()) continue; - size_t sum_marks = projection_result_ptr->marks(); + candidate.merge_tree_projection_select_result_ptr = std::move(projection_result_ptr); + candidate.sum_marks += candidate.merge_tree_projection_select_result_ptr->marks(); if (!normal_parts.empty()) { @@ -556,49 +596,73 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (normal_result_ptr->marks() != 0) { - sum_marks += normal_result_ptr->marks(); + candidate.sum_marks += normal_result_ptr->marks(); candidate.merge_tree_normal_select_result_ptr = std::move(normal_result_ptr); } } - candidate.merge_tree_projection_select_result_ptr = std::move(projection_result_ptr); - - if (best_candidate == nullptr || best_candidate_marks > sum_marks) - { + if (best_candidate == nullptr || best_candidate->sum_marks > candidate.sum_marks) best_candidate = &candidate; - best_candidate_marks = sum_marks; - } } - if (!best_candidate) + if (!best_candidate && !minmax_projection) return false; - auto storage_snapshot = reading->getStorageSnapshot(); - auto proj_snapshot = std::make_shared( - storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); //, storage_snapshot->data); - proj_snapshot->addProjection(best_candidate->projection); - - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}", proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString()); - + QueryPlanStepPtr projection_reading; auto query_info_copy = query_info; query_info_copy.prewhere_info = nullptr; + bool has_nornal_parts; - auto projection_reading = reader.readFromParts( - {}, - best_candidate->dag->getRequiredColumnsNames(), - proj_snapshot, - query_info_copy, - context, - reading->getMaxBlockSize(), - reading->getNumStreams(), - max_added_blocks, - best_candidate->merge_tree_projection_select_result_ptr, - reading->isParallelReadingEnabled()); - - if (!projection_reading) + if (minmax_projection) { - Pipe pipe(std::make_shared(proj_snapshot->getSampleBlockForColumns(best_candidate->dag->getRequiredColumnsNames()))); + MergeTreeData::DataPartsVector normal_parts; + auto minmax_count_projection_block = reading->getMergeTreeData().getMinMaxCountProjectionBlock( + metadata, + minmax_projection->dag->getRequiredColumnsNames(), + filter_node != nullptr, + query_info, + parts, + normal_parts, + max_added_blocks.get(), + context); + + Pipe pipe(std::make_shared(std::move(minmax_count_projection_block))); projection_reading = std::make_unique(std::move(pipe)); + + has_nornal_parts = !normal_parts.empty(); + if (has_nornal_parts) + reading->resetParts(std::move(normal_parts)); + } + else + { + auto storage_snapshot = reading->getStorageSnapshot(); + auto proj_snapshot = std::make_shared( + storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); //, storage_snapshot->data); + proj_snapshot->addProjection(best_candidate->projection); + + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}", proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString()); + + projection_reading = reader.readFromParts( + {}, + best_candidate->dag->getRequiredColumnsNames(), + proj_snapshot, + query_info_copy, + context, + reading->getMaxBlockSize(), + reading->getNumStreams(), + max_added_blocks, + best_candidate->merge_tree_projection_select_result_ptr, + reading->isParallelReadingEnabled()); + + if (!projection_reading) + { + Pipe pipe(std::make_shared(proj_snapshot->getSampleBlockForColumns(best_candidate->dag->getRequiredColumnsNames()))); + projection_reading = std::make_unique(std::move(pipe)); + } + + has_nornal_parts = best_candidate->merge_tree_normal_select_result_ptr != nullptr; + if (has_nornal_parts) + reading->setAnalyzedResult(std::move(best_candidate->merge_tree_normal_select_result_ptr)); } LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}", projection_reading->getOutputStream().header.dumpStructure()); @@ -623,7 +687,7 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) expr_or_filter_node.children.push_back(&projection_reading_node); - if (!best_candidate->merge_tree_normal_select_result_ptr) + if (!has_nornal_parts) { /// All parts are taken from projection @@ -637,7 +701,6 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { node.step = aggregating->convertToAggregatingProjection(expr_or_filter_node.step->getOutputStream()); node.children.push_back(&expr_or_filter_node); - reading->setAnalyzedResult(std::move(best_candidate->merge_tree_normal_select_result_ptr)); } return true; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 5ab45331722..90832d8a7ae 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -158,6 +158,7 @@ public: void requestReadingInOrder(size_t prefix_size, int direction, size_t limit); void setAnalyzedResult(MergeTreeDataSelectAnalysisResultPtr analyzed_result_ptr_) { analyzed_result_ptr = std::move(analyzed_result_ptr_); } + void resetParts(MergeTreeData::DataPartsVector parts) { prepared_parts = std::move(parts); } const MergeTreeData::DataPartsVector & getParts() const { return prepared_parts; } const MergeTreeData & getMergeTreeData() const { return data; } From 9e04e57949674123b9b35b4070b387cec7353092 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 9 Feb 2023 15:52:11 +0100 Subject: [PATCH 016/559] Allow to give access only to certain named collections --- src/Access/AccessRights.cpp | 38 +++++++--- src/Access/Common/AccessFlags.cpp | 10 ++- src/Access/Common/AccessFlags.h | 5 ++ src/Access/Common/AccessRightsElement.cpp | 23 +++--- src/Access/Common/AccessRightsElement.h | 2 + src/Access/Common/AccessType.h | 10 +-- src/Access/ContextAccess.cpp | 4 +- src/Parsers/Access/ASTGrantQuery.cpp | 21 +++--- src/Parsers/Access/ParserGrantQuery.cpp | 34 ++++++++- .../System/StorageSystemNamedCollections.cpp | 6 +- .../System/StorageSystemPrivileges.cpp | 3 + .../test_named_collections/test.py | 70 +++++++++++++++++++ 12 files changed, 189 insertions(+), 37 deletions(-) diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index 7e21e3c2c4d..595afadfe1c 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -61,12 +61,22 @@ namespace res.any_database = true; res.any_table = true; res.any_column = true; + res.any_named_collection = !access_flags.isNamedCollectionAccessOnly(); break; } case 1: { - res.any_database = false; - res.database = full_name[0]; + res.any_named_collection = !access_flags.isNamedCollectionAccessOnly(); + if (!res.any_named_collection) + { + res.any_database = true; + res.named_collection = full_name[0]; + } + else + { + res.any_database = false; + res.database = full_name[0]; + } res.any_table = true; res.any_column = true; break; @@ -317,8 +327,8 @@ public: const Node * child = tryGetChild(name); if (child) return child->isGranted(flags_to_check, subnames...); - else - return flags.contains(flags_to_check); + + return flags.contains(flags_to_check); } template @@ -783,7 +793,9 @@ void AccessRights::grantImplHelper(const AccessRightsElement & element) { assert(!element.is_partial_revoke); assert(!element.grant_option || with_grant_option); - if (element.any_database) + if (!element.any_named_collection) + grantImpl(element.access_flags, element.named_collection); + else if (element.any_database) grantImpl(element.access_flags); else if (element.any_table) grantImpl(element.access_flags, element.database); @@ -825,7 +837,10 @@ void AccessRights::grant(const AccessFlags & flags, std::string_view database, s void AccessRights::grant(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) { grantImpl(flags, database, table, columns); } void AccessRights::grant(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) { grantImpl(flags, database, table, columns); } void AccessRights::grant(const AccessRightsElement & element) { grantImpl(element); } -void AccessRights::grant(const AccessRightsElements & elements) { grantImpl(elements); } +void AccessRights::grant(const AccessRightsElements & elements) +{ + grantImpl(elements); +} void AccessRights::grantWithGrantOption(const AccessFlags & flags) { grantImpl(flags); } void AccessRights::grantWithGrantOption(const AccessFlags & flags, std::string_view database) { grantImpl(flags, database); } @@ -858,7 +873,9 @@ template void AccessRights::revokeImplHelper(const AccessRightsElement & element) { assert(!element.grant_option || grant_option); - if (element.any_database) + if (!element.any_named_collection) + revokeImpl(element.access_flags, element.named_collection); + else if (element.any_database) revokeImpl(element.access_flags); else if (element.any_table) revokeImpl(element.access_flags, element.database); @@ -912,7 +929,7 @@ void AccessRights::revokeGrantOption(const AccessRightsElements & elements) { re AccessRightsElements AccessRights::getElements() const { -#if 0 +#if 1 logTree(); #endif if (!root) @@ -934,6 +951,7 @@ bool AccessRights::isGrantedImpl(const AccessFlags & flags, const Args &... args { auto helper = [&](const std::unique_ptr & root_node) -> bool { + logTree(); if (!root_node) return flags.isEmpty(); return root_node->isGranted(flags, args...); @@ -948,7 +966,9 @@ template bool AccessRights::isGrantedImplHelper(const AccessRightsElement & element) const { assert(!element.grant_option || grant_option); - if (element.any_database) + if (!element.any_named_collection) + return isGrantedImpl(element.access_flags, element.named_collection); + else if (element.any_database) return isGrantedImpl(element.access_flags); else if (element.any_table) return isGrantedImpl(element.access_flags, element.database); diff --git a/src/Access/Common/AccessFlags.cpp b/src/Access/Common/AccessFlags.cpp index bef165ba4e6..533fbb0e524 100644 --- a/src/Access/Common/AccessFlags.cpp +++ b/src/Access/Common/AccessFlags.cpp @@ -96,11 +96,13 @@ namespace const Flags & getAllFlags() const { return all_flags; } const Flags & getGlobalFlags() const { return all_flags_for_target[GLOBAL]; } + const Flags & getGlobalWithParameterFlags() const { return all_flags_for_target[GLOBAL_WITH_PARAMETER]; } const Flags & getDatabaseFlags() const { return all_flags_for_target[DATABASE]; } const Flags & getTableFlags() const { return all_flags_for_target[TABLE]; } const Flags & getColumnFlags() const { return all_flags_for_target[COLUMN]; } const Flags & getDictionaryFlags() const { return all_flags_for_target[DICTIONARY]; } const Flags & getAllFlagsGrantableOnGlobalLevel() const { return getAllFlags(); } + const Flags & getAllFlagsGrantableOnNamedCollectionLevel() const { return all_flags_for_target[NAMED_COLLECTION]; } const Flags & getAllFlagsGrantableOnDatabaseLevel() const { return all_flags_grantable_on_database_level; } const Flags & getAllFlagsGrantableOnTableLevel() const { return all_flags_grantable_on_table_level; } const Flags & getAllFlagsGrantableOnColumnLevel() const { return getColumnFlags(); } @@ -116,6 +118,8 @@ namespace VIEW = TABLE, COLUMN, DICTIONARY, + GLOBAL_WITH_PARAMETER, + NAMED_COLLECTION, }; struct Node; @@ -295,7 +299,7 @@ namespace collectAllFlags(child.get()); all_flags_grantable_on_table_level = all_flags_for_target[TABLE] | all_flags_for_target[DICTIONARY] | all_flags_for_target[COLUMN]; - all_flags_grantable_on_database_level = all_flags_for_target[DATABASE] | all_flags_grantable_on_table_level; + all_flags_grantable_on_database_level = all_flags_for_target[DATABASE] | all_flags_for_target[NAMED_COLLECTION] | all_flags_grantable_on_table_level; } Helper() @@ -345,7 +349,7 @@ namespace std::unordered_map keyword_to_flags_map; std::vector access_type_to_flags_mapping; Flags all_flags; - Flags all_flags_for_target[static_cast(DICTIONARY) + 1]; + Flags all_flags_for_target[static_cast(NAMED_COLLECTION) + 1]; Flags all_flags_grantable_on_database_level; Flags all_flags_grantable_on_table_level; }; @@ -361,11 +365,13 @@ std::vector AccessFlags::toAccessTypes() const { return Helper::inst std::vector AccessFlags::toKeywords() const { return Helper::instance().flagsToKeywords(flags); } AccessFlags AccessFlags::allFlags() { return Helper::instance().getAllFlags(); } AccessFlags AccessFlags::allGlobalFlags() { return Helper::instance().getGlobalFlags(); } +AccessFlags AccessFlags::allGlobalWithParameterFlags() { return Helper::instance().getGlobalWithParameterFlags(); } AccessFlags AccessFlags::allDatabaseFlags() { return Helper::instance().getDatabaseFlags(); } AccessFlags AccessFlags::allTableFlags() { return Helper::instance().getTableFlags(); } AccessFlags AccessFlags::allColumnFlags() { return Helper::instance().getColumnFlags(); } AccessFlags AccessFlags::allDictionaryFlags() { return Helper::instance().getDictionaryFlags(); } AccessFlags AccessFlags::allFlagsGrantableOnGlobalLevel() { return Helper::instance().getAllFlagsGrantableOnGlobalLevel(); } +AccessFlags AccessFlags::allFlagsGrantableOnNamedCollectionLevel() { return Helper::instance().getAllFlagsGrantableOnNamedCollectionLevel(); } AccessFlags AccessFlags::allFlagsGrantableOnDatabaseLevel() { return Helper::instance().getAllFlagsGrantableOnDatabaseLevel(); } AccessFlags AccessFlags::allFlagsGrantableOnTableLevel() { return Helper::instance().getAllFlagsGrantableOnTableLevel(); } AccessFlags AccessFlags::allFlagsGrantableOnColumnLevel() { return Helper::instance().getAllFlagsGrantableOnColumnLevel(); } diff --git a/src/Access/Common/AccessFlags.h b/src/Access/Common/AccessFlags.h index c4e0b7ac281..5443c505245 100644 --- a/src/Access/Common/AccessFlags.h +++ b/src/Access/Common/AccessFlags.h @@ -50,6 +50,7 @@ public: bool isEmpty() const { return flags.none(); } explicit operator bool() const { return !isEmpty(); } bool contains(const AccessFlags & other) const { return (flags & other.flags) == other.flags; } + bool isNamedCollectionAccessOnly() const { return (flags & ~allFlagsGrantableOnNamedCollectionLevel()).isEmpty(); } friend bool operator ==(const AccessFlags & left, const AccessFlags & right) { return left.flags == right.flags; } friend bool operator !=(const AccessFlags & left, const AccessFlags & right) { return !(left == right); } @@ -76,6 +77,8 @@ public: /// Returns all the global flags. static AccessFlags allGlobalFlags(); + static AccessFlags allGlobalWithParameterFlags(); + /// Returns all the flags related to a database. static AccessFlags allDatabaseFlags(); @@ -104,6 +107,8 @@ public: /// The same as allColumnFlags(). static AccessFlags allFlagsGrantableOnColumnLevel(); + static AccessFlags allFlagsGrantableOnNamedCollectionLevel(); + static constexpr size_t SIZE = 256; private: using Flags = std::bitset; diff --git a/src/Access/Common/AccessRightsElement.cpp b/src/Access/Common/AccessRightsElement.cpp index 69a2354f25d..e2eb14ad9cb 100644 --- a/src/Access/Common/AccessRightsElement.cpp +++ b/src/Access/Common/AccessRightsElement.cpp @@ -21,24 +21,31 @@ namespace result += ")"; } - void formatONClause(const String & database, bool any_database, const String & table, bool any_table, String & result) + void formatONClause(const AccessRightsElement & element, String & result) { result += "ON "; - if (any_database) + if (!element.any_named_collection) + { + if (element.named_collection.empty()) + result += "*"; + else + result += backQuoteIfNeed(element.named_collection); + } + else if (element.any_database) { result += "*.*"; } else { - if (!database.empty()) + if (!element.database.empty()) { - result += backQuoteIfNeed(database); + result += backQuoteIfNeed(element.database); result += "."; } - if (any_table) + if (element.any_table) result += "*"; else - result += backQuoteIfNeed(table); + result += backQuoteIfNeed(element.table); } } @@ -96,7 +103,7 @@ namespace String result; formatAccessFlagsWithColumns(element.access_flags, element.columns, element.any_column, result); result += " "; - formatONClause(element.database, element.any_database, element.table, element.any_table, result); + formatONClause(element, result); if (with_options) formatOptions(element.grant_option, element.is_partial_revoke, result); return result; @@ -129,7 +136,7 @@ namespace if (!next_element_uses_same_table_and_options) { part += " "; - formatONClause(element.database, element.any_database, element.table, element.any_table, part); + formatONClause(element, part); if (with_options) formatOptions(element.grant_option, element.is_partial_revoke, part); if (result.empty()) diff --git a/src/Access/Common/AccessRightsElement.h b/src/Access/Common/AccessRightsElement.h index 5f65b6bcd12..27657ea3960 100644 --- a/src/Access/Common/AccessRightsElement.h +++ b/src/Access/Common/AccessRightsElement.h @@ -14,9 +14,11 @@ struct AccessRightsElement String database; String table; Strings columns; + String named_collection; bool any_database = true; bool any_table = true; bool any_column = true; + bool any_named_collection = true; bool grant_option = false; bool is_partial_revoke = false; diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 497327c1bad..703b3106a9a 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -12,7 +12,7 @@ enum class AccessType /// Macro M should be defined as M(name, aliases, node_type, parent_group_name) /// where name is identifier with underscores (instead of spaces); /// aliases is a string containing comma-separated list; -/// node_type either specifies access type's level (GLOBAL/DATABASE/TABLE/DICTIONARY/VIEW/COLUMNS), +/// node_type either specifies access type's level (GLOBAL/NAMED_COLLECTION/DATABASE/TABLE/DICTIONARY/VIEW/COLUMNS), /// or specifies that the access type is a GROUP of other access types; /// parent_group_name is the name of the group containing this access type (or NONE if there is no such group). #define APPLY_FOR_ACCESS_TYPES(M) \ @@ -69,7 +69,7 @@ enum class AccessType M(ALTER_FREEZE_PARTITION, "FREEZE PARTITION, UNFREEZE", TABLE, ALTER_TABLE) \ \ M(ALTER_DATABASE_SETTINGS, "ALTER DATABASE SETTING, ALTER MODIFY DATABASE SETTING, MODIFY DATABASE SETTING", DATABASE, ALTER_DATABASE) /* allows to execute ALTER MODIFY SETTING */\ - M(ALTER_NAMED_COLLECTION, "", GROUP, ALTER) /* allows to execute ALTER NAMED COLLECTION */\ + M(ALTER_NAMED_COLLECTION, "", NAMED_COLLECTION, ALTER) /* allows to execute ALTER NAMED COLLECTION */\ \ M(ALTER_TABLE, "", GROUP, ALTER) \ M(ALTER_DATABASE, "", GROUP, ALTER) \ @@ -89,7 +89,7 @@ enum class AccessType M(CREATE_TEMPORARY_TABLE, "", GLOBAL, CREATE) /* allows to create and manipulate temporary tables; implicitly enabled by the grant CREATE_TABLE on any table */ \ M(CREATE_FUNCTION, "", GLOBAL, CREATE) /* allows to execute CREATE FUNCTION */ \ - M(CREATE_NAMED_COLLECTION, "", GLOBAL, CREATE) /* allows to execute CREATE NAMED COLLECTION */ \ + M(CREATE_NAMED_COLLECTION, "", NAMED_COLLECTION, CREATE) /* allows to execute CREATE NAMED COLLECTION */ \ M(CREATE, "", GROUP, ALL) /* allows to execute {CREATE|ATTACH} */ \ \ M(DROP_DATABASE, "", DATABASE, DROP) /* allows to execute {DROP|DETACH} DATABASE */\ @@ -98,7 +98,7 @@ enum class AccessType implicitly enabled by the grant DROP_TABLE */\ M(DROP_DICTIONARY, "", DICTIONARY, DROP) /* allows to execute {DROP|DETACH} DICTIONARY */\ M(DROP_FUNCTION, "", GLOBAL, DROP) /* allows to execute DROP FUNCTION */\ - M(DROP_NAMED_COLLECTION, "", GLOBAL, DROP) /* allows to execute DROP NAMED COLLECTION */\ + M(DROP_NAMED_COLLECTION, "", NAMED_COLLECTION, DROP) /* allows to execute DROP NAMED COLLECTION */\ M(DROP, "", GROUP, ALL) /* allows to execute {DROP|DETACH} */\ \ M(TRUNCATE, "TRUNCATE TABLE", TABLE, ALL) \ @@ -134,7 +134,7 @@ enum class AccessType M(SHOW_QUOTAS, "SHOW CREATE QUOTA", GLOBAL, SHOW_ACCESS) \ M(SHOW_SETTINGS_PROFILES, "SHOW PROFILES, SHOW CREATE SETTINGS PROFILE, SHOW CREATE PROFILE", GLOBAL, SHOW_ACCESS) \ M(SHOW_ACCESS, "", GROUP, ACCESS_MANAGEMENT) \ - M(SHOW_NAMED_COLLECTIONS, "SHOW NAMED COLLECTIONS", GLOBAL, ACCESS_MANAGEMENT) \ + M(SHOW_NAMED_COLLECTIONS, "SHOW NAMED COLLECTIONS", NAMED_COLLECTION, ACCESS_MANAGEMENT) \ M(ACCESS_MANAGEMENT, "", GROUP, ALL) \ \ M(SYSTEM_SHUTDOWN, "SYSTEM KILL, SHUTDOWN", GLOBAL, SYSTEM) \ diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index fbaacb2263b..6d6362a98b2 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -606,7 +606,9 @@ template bool ContextAccess::checkAccessImplHelper(const AccessRightsElement & element) const { assert(!element.grant_option || grant_option); - if (element.any_database) + if (!element.any_named_collection) + return checkAccessImpl(element.access_flags, element.named_collection); + else if (element.any_database) return checkAccessImpl(element.access_flags); else if (element.any_table) return checkAccessImpl(element.access_flags, element.database); diff --git a/src/Parsers/Access/ASTGrantQuery.cpp b/src/Parsers/Access/ASTGrantQuery.cpp index f92541ec672..e4aa11967c6 100644 --- a/src/Parsers/Access/ASTGrantQuery.cpp +++ b/src/Parsers/Access/ASTGrantQuery.cpp @@ -27,21 +27,25 @@ namespace } - void formatONClause(const String & database, bool any_database, const String & table, bool any_table, const IAST::FormatSettings & settings) + void formatONClause(const AccessRightsElement & element, const IAST::FormatSettings & settings) { settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "ON " << (settings.hilite ? IAST::hilite_none : ""); - if (any_database) + if (!element.any_named_collection) + { + settings.ostr << backQuoteIfNeed(element.named_collection); + } + else if (element.any_database) { settings.ostr << "*.*"; } else { - if (!database.empty()) - settings.ostr << backQuoteIfNeed(database) << "."; - if (any_table) + if (!element.database.empty()) + settings.ostr << backQuoteIfNeed(element.database) << "."; + if (element.any_table) settings.ostr << "*"; else - settings.ostr << backQuoteIfNeed(table); + settings.ostr << backQuoteIfNeed(element.table); } } @@ -71,14 +75,15 @@ namespace { const auto & next_element = elements[i + 1]; if ((element.database == next_element.database) && (element.any_database == next_element.any_database) - && (element.table == next_element.table) && (element.any_table == next_element.any_table)) + && (element.table == next_element.table) && (element.any_table == next_element.any_table) + && (element.named_collection == next_element.named_collection)) next_element_on_same_db_and_table = true; } if (!next_element_on_same_db_and_table) { settings.ostr << " "; - formatONClause(element.database, element.any_database, element.table, element.any_table, settings); + formatONClause(element, settings); } } diff --git a/src/Parsers/Access/ParserGrantQuery.cpp b/src/Parsers/Access/ParserGrantQuery.cpp index 2211969c61e..a2f4e2a4921 100644 --- a/src/Parsers/Access/ParserGrantQuery.cpp +++ b/src/Parsers/Access/ParserGrantQuery.cpp @@ -123,12 +123,38 @@ namespace if (!parseAccessFlagsWithColumns(pos, expected, access_and_columns)) return false; + String database_name, table_name, collection_name; + bool any_database = false, any_table = false, any_named_collection = true; + + size_t named_collection_access = 0; + for (const auto & elem : access_and_columns) + { + if (elem.first.isNamedCollectionAccessOnly()) + ++named_collection_access; + } + const bool grant_named_collection_access = named_collection_access == access_and_columns.size(); + if (!ParserKeyword{"ON"}.ignore(pos, expected)) return false; - String database_name, table_name; - bool any_database = false, any_table = false; - if (!parseDatabaseAndTableNameOrAsterisks(pos, expected, database_name, any_database, table_name, any_table)) + if (grant_named_collection_access) + { + ASTPtr collection; + if (ParserToken{TokenType::Asterisk}.ignore(pos, expected)) + { + any_named_collection = true; + } + else if (ParserIdentifier{}.parse(pos, collection, expected)) + { + any_named_collection = false; + collection_name = getIdentifierName(collection); + } + else + return false; + + any_database = any_table = true; + } + else if (!parseDatabaseAndTableNameOrAsterisks(pos, expected, database_name, any_database, table_name, any_table)) return false; for (auto & [access_flags, columns] : access_and_columns) @@ -140,6 +166,8 @@ namespace element.any_database = any_database; element.database = database_name; element.any_table = any_table; + element.any_named_collection = any_named_collection; + element.named_collection = collection_name; element.table = table_name; res_elements.emplace_back(std::move(element)); } diff --git a/src/Storages/System/StorageSystemNamedCollections.cpp b/src/Storages/System/StorageSystemNamedCollections.cpp index bc1e3a45e6b..0b7522d3845 100644 --- a/src/Storages/System/StorageSystemNamedCollections.cpp +++ b/src/Storages/System/StorageSystemNamedCollections.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -29,11 +30,14 @@ StorageSystemNamedCollections::StorageSystemNamedCollections(const StorageID & t void StorageSystemNamedCollections::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - context->checkAccess(AccessType::SHOW_NAMED_COLLECTIONS); + const auto & access = context->getAccess(); auto collections = NamedCollectionFactory::instance().getAll(); for (const auto & [name, collection] : collections) { + if (!access->isGranted(AccessType::SHOW_NAMED_COLLECTIONS, name)) + continue; + res_columns[0]->insert(name); auto * column_map = typeid_cast(res_columns[1].get()); diff --git a/src/Storages/System/StorageSystemPrivileges.cpp b/src/Storages/System/StorageSystemPrivileges.cpp index 70163979f72..14ca76df7d8 100644 --- a/src/Storages/System/StorageSystemPrivileges.cpp +++ b/src/Storages/System/StorageSystemPrivileges.cpp @@ -28,6 +28,8 @@ namespace DICTIONARY, VIEW, COLUMN, + GLOBAL_WITH_PARAMETER, + NAMED_COLLECTION, }; DataTypeEnum8::Values getLevelEnumValues() @@ -39,6 +41,7 @@ namespace enum_values.emplace_back("DICTIONARY", static_cast(DICTIONARY)); enum_values.emplace_back("VIEW", static_cast(VIEW)); enum_values.emplace_back("COLUMN", static_cast(COLUMN)); + enum_values.emplace_back("NAMED_COLLECTION", static_cast(NAMED_COLLECTION)); return enum_values; } } diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index 3b102f1aa70..0f8999f43dd 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -105,6 +105,76 @@ def test_access(cluster): assert int(node.query("select count() from system.named_collections")) > 0 +def test_granular_access(cluster): + node = cluster.instances["node"] + assert 1 == int(node.query("SELECT count() FROM system.named_collections")) + assert ( + "collection1" == node.query("SELECT name FROM system.named_collections").strip() + ) + + node.query("CREATE USER kek") + node.query("GRANT select ON *.* TO kek") + assert 0 == int( + node.query("SELECT count() FROM system.named_collections", user="kek") + ) + + node.query("GRANT show named collections ON collection1 TO kek") + assert 1 == int( + node.query("SELECT count() FROM system.named_collections", user="kek") + ) + assert ( + "collection1" + == node.query("SELECT name FROM system.named_collections", user="kek").strip() + ) + + node.query("CREATE NAMED COLLECTION collection2 AS key1=1, key2='value2'") + assert 2 == int(node.query("SELECT count() FROM system.named_collections")) + assert ( + "collection1\ncollection2" + == node.query("select name from system.named_collections").strip() + ) + + assert 1 == int( + node.query("SELECT count() FROM system.named_collections", user="kek") + ) + assert ( + "collection1" + == node.query("select name from system.named_collections", user="kek").strip() + ) + + node.query("GRANT show named collections ON collection2 TO kek") + assert 2 == int( + node.query("SELECT count() FROM system.named_collections", user="kek") + ) + assert ( + "collection1\ncollection2" + == node.query("select name from system.named_collections", user="kek").strip() + ) + node.restart_clickhouse() + assert ( + "collection1\ncollection2" + == node.query("select name from system.named_collections", user="kek").strip() + ) + + node.query("CREATE USER koko") + node.query("GRANT select ON *.* TO koko") + assert 0 == int( + node.query("SELECT count() FROM system.named_collections", user="koko") + ) + node.query("GRANT show named collections ON * TO koko") + assert ( + "collection1\ncollection2" + == node.query("select name from system.named_collections", user="koko").strip() + ) + node.restart_clickhouse() + assert ( + "collection1\ncollection2" + == node.query("select name from system.named_collections", user="koko").strip() + ) + + node.query("DROP NAMED COLLECTION collection2") + + def test_config_reload(cluster): node = cluster.instances["node"] assert ( From 214ffe0bb0f7d19b70649fac60886c900fb97925 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 9 Feb 2023 17:44:26 +0100 Subject: [PATCH 017/559] Remove unused code --- src/Access/Common/AccessFlags.cpp | 3 --- src/Access/Common/AccessFlags.h | 2 -- src/Access/Common/AccessRightsElement.cpp | 8 ++++---- src/Access/Common/AccessRightsElement.h | 2 ++ src/Storages/System/StorageSystemPrivileges.cpp | 1 - 5 files changed, 6 insertions(+), 10 deletions(-) diff --git a/src/Access/Common/AccessFlags.cpp b/src/Access/Common/AccessFlags.cpp index 533fbb0e524..c8d2b1ca4d9 100644 --- a/src/Access/Common/AccessFlags.cpp +++ b/src/Access/Common/AccessFlags.cpp @@ -96,7 +96,6 @@ namespace const Flags & getAllFlags() const { return all_flags; } const Flags & getGlobalFlags() const { return all_flags_for_target[GLOBAL]; } - const Flags & getGlobalWithParameterFlags() const { return all_flags_for_target[GLOBAL_WITH_PARAMETER]; } const Flags & getDatabaseFlags() const { return all_flags_for_target[DATABASE]; } const Flags & getTableFlags() const { return all_flags_for_target[TABLE]; } const Flags & getColumnFlags() const { return all_flags_for_target[COLUMN]; } @@ -118,7 +117,6 @@ namespace VIEW = TABLE, COLUMN, DICTIONARY, - GLOBAL_WITH_PARAMETER, NAMED_COLLECTION, }; @@ -365,7 +363,6 @@ std::vector AccessFlags::toAccessTypes() const { return Helper::inst std::vector AccessFlags::toKeywords() const { return Helper::instance().flagsToKeywords(flags); } AccessFlags AccessFlags::allFlags() { return Helper::instance().getAllFlags(); } AccessFlags AccessFlags::allGlobalFlags() { return Helper::instance().getGlobalFlags(); } -AccessFlags AccessFlags::allGlobalWithParameterFlags() { return Helper::instance().getGlobalWithParameterFlags(); } AccessFlags AccessFlags::allDatabaseFlags() { return Helper::instance().getDatabaseFlags(); } AccessFlags AccessFlags::allTableFlags() { return Helper::instance().getTableFlags(); } AccessFlags AccessFlags::allColumnFlags() { return Helper::instance().getColumnFlags(); } diff --git a/src/Access/Common/AccessFlags.h b/src/Access/Common/AccessFlags.h index 5443c505245..b81b73b8350 100644 --- a/src/Access/Common/AccessFlags.h +++ b/src/Access/Common/AccessFlags.h @@ -77,8 +77,6 @@ public: /// Returns all the global flags. static AccessFlags allGlobalFlags(); - static AccessFlags allGlobalWithParameterFlags(); - /// Returns all the flags related to a database. static AccessFlags allDatabaseFlags(); diff --git a/src/Access/Common/AccessRightsElement.cpp b/src/Access/Common/AccessRightsElement.cpp index e2eb14ad9cb..011db851bc4 100644 --- a/src/Access/Common/AccessRightsElement.cpp +++ b/src/Access/Common/AccessRightsElement.cpp @@ -24,12 +24,12 @@ namespace void formatONClause(const AccessRightsElement & element, String & result) { result += "ON "; - if (!element.any_named_collection) + if (element.isNamedCollectionAccess()) { - if (element.named_collection.empty()) - result += "*"; - else + if (!element.any_named_collection) result += backQuoteIfNeed(element.named_collection); + else + result += "*"; } else if (element.any_database) { diff --git a/src/Access/Common/AccessRightsElement.h b/src/Access/Common/AccessRightsElement.h index 27657ea3960..653f813ff35 100644 --- a/src/Access/Common/AccessRightsElement.h +++ b/src/Access/Common/AccessRightsElement.h @@ -69,6 +69,8 @@ struct AccessRightsElement /// If the database is empty, replaces it with `current_database`. Otherwise does nothing. void replaceEmptyDatabase(const String & current_database); + bool isNamedCollectionAccess() const { return access_flags.isNamedCollectionAccessOnly(); } + /// Returns a human-readable representation like "GRANT SELECT, UPDATE(x, y) ON db.table". String toString() const; String toStringWithoutOptions() const; diff --git a/src/Storages/System/StorageSystemPrivileges.cpp b/src/Storages/System/StorageSystemPrivileges.cpp index 14ca76df7d8..ee412d0e648 100644 --- a/src/Storages/System/StorageSystemPrivileges.cpp +++ b/src/Storages/System/StorageSystemPrivileges.cpp @@ -28,7 +28,6 @@ namespace DICTIONARY, VIEW, COLUMN, - GLOBAL_WITH_PARAMETER, NAMED_COLLECTION, }; From c13b0b8a065ec5fa2275fa7ed27705c1f481eebc Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 10 Feb 2023 13:14:11 +0100 Subject: [PATCH 018/559] Fix tests --- src/Access/AccessRights.cpp | 7 +++---- src/Access/Common/AccessFlags.h | 3 ++- src/Access/Common/AccessRightsElement.h | 2 +- src/Parsers/Access/ParserGrantQuery.cpp | 4 ++-- tests/queries/0_stateless/01271_show_privileges.reference | 8 ++++---- .../0_stateless/02117_show_create_table_system.reference | 2 +- 6 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index 595afadfe1c..ca118203541 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -61,15 +61,14 @@ namespace res.any_database = true; res.any_table = true; res.any_column = true; - res.any_named_collection = !access_flags.isNamedCollectionAccessOnly(); + res.any_named_collection = true; break; } case 1: { - res.any_named_collection = !access_flags.isNamedCollectionAccessOnly(); - if (!res.any_named_collection) + if (access_flags.isNamedCollectionAccess()) { - res.any_database = true; + res.any_named_collection = false; res.named_collection = full_name[0]; } else diff --git a/src/Access/Common/AccessFlags.h b/src/Access/Common/AccessFlags.h index b81b73b8350..f475896b6e5 100644 --- a/src/Access/Common/AccessFlags.h +++ b/src/Access/Common/AccessFlags.h @@ -50,7 +50,8 @@ public: bool isEmpty() const { return flags.none(); } explicit operator bool() const { return !isEmpty(); } bool contains(const AccessFlags & other) const { return (flags & other.flags) == other.flags; } - bool isNamedCollectionAccessOnly() const { return (flags & ~allFlagsGrantableOnNamedCollectionLevel()).isEmpty(); } + bool containsOnly(const AccessFlags & other) const { return flags == other.flags; } + bool isNamedCollectionAccess() const { return containsOnly(AccessFlags::allFlagsGrantableOnNamedCollectionLevel()); } friend bool operator ==(const AccessFlags & left, const AccessFlags & right) { return left.flags == right.flags; } friend bool operator !=(const AccessFlags & left, const AccessFlags & right) { return !(left == right); } diff --git a/src/Access/Common/AccessRightsElement.h b/src/Access/Common/AccessRightsElement.h index 653f813ff35..db6a1f6872f 100644 --- a/src/Access/Common/AccessRightsElement.h +++ b/src/Access/Common/AccessRightsElement.h @@ -69,7 +69,7 @@ struct AccessRightsElement /// If the database is empty, replaces it with `current_database`. Otherwise does nothing. void replaceEmptyDatabase(const String & current_database); - bool isNamedCollectionAccess() const { return access_flags.isNamedCollectionAccessOnly(); } + bool isNamedCollectionAccess() const { return access_flags.isNamedCollectionAccess(); } /// Returns a human-readable representation like "GRANT SELECT, UPDATE(x, y) ON db.table". String toString() const; diff --git a/src/Parsers/Access/ParserGrantQuery.cpp b/src/Parsers/Access/ParserGrantQuery.cpp index a2f4e2a4921..06660642fbf 100644 --- a/src/Parsers/Access/ParserGrantQuery.cpp +++ b/src/Parsers/Access/ParserGrantQuery.cpp @@ -129,14 +129,14 @@ namespace size_t named_collection_access = 0; for (const auto & elem : access_and_columns) { - if (elem.first.isNamedCollectionAccessOnly()) + if (elem.first.isNamedCollectionAccess()) ++named_collection_access; } - const bool grant_named_collection_access = named_collection_access == access_and_columns.size(); if (!ParserKeyword{"ON"}.ignore(pos, expected)) return false; + const bool grant_named_collection_access = named_collection_access && named_collection_access == access_and_columns.size(); if (grant_named_collection_access) { ASTPtr collection; diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 58b1cab6e20..518b1a84abb 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -39,7 +39,7 @@ ALTER MOVE PARTITION ['ALTER MOVE PART','MOVE PARTITION','MOVE PART'] TABLE ALTE ALTER FETCH PARTITION ['ALTER FETCH PART','FETCH PARTITION'] TABLE ALTER TABLE ALTER FREEZE PARTITION ['FREEZE PARTITION','UNFREEZE'] TABLE ALTER TABLE ALTER DATABASE SETTINGS ['ALTER DATABASE SETTING','ALTER MODIFY DATABASE SETTING','MODIFY DATABASE SETTING'] DATABASE ALTER DATABASE -ALTER NAMED COLLECTION [] \N ALTER +ALTER NAMED COLLECTION [] NAMED_COLLECTION ALTER ALTER TABLE [] \N ALTER ALTER DATABASE [] \N ALTER ALTER VIEW REFRESH ['ALTER LIVE VIEW REFRESH','REFRESH VIEW'] VIEW ALTER VIEW @@ -52,14 +52,14 @@ CREATE VIEW [] VIEW CREATE CREATE DICTIONARY [] DICTIONARY CREATE CREATE TEMPORARY TABLE [] GLOBAL CREATE CREATE FUNCTION [] GLOBAL CREATE -CREATE NAMED COLLECTION [] GLOBAL CREATE +CREATE NAMED COLLECTION [] NAMED_COLLECTION CREATE CREATE [] \N ALL DROP DATABASE [] DATABASE DROP DROP TABLE [] TABLE DROP DROP VIEW [] VIEW DROP DROP DICTIONARY [] DICTIONARY DROP DROP FUNCTION [] GLOBAL DROP -DROP NAMED COLLECTION [] GLOBAL DROP +DROP NAMED COLLECTION [] NAMED_COLLECTION DROP DROP [] \N ALL TRUNCATE ['TRUNCATE TABLE'] TABLE ALL OPTIMIZE ['OPTIMIZE TABLE'] TABLE ALL @@ -89,7 +89,7 @@ SHOW ROW POLICIES ['SHOW POLICIES','SHOW CREATE ROW POLICY','SHOW CREATE POLICY' SHOW QUOTAS ['SHOW CREATE QUOTA'] GLOBAL SHOW ACCESS SHOW SETTINGS PROFILES ['SHOW PROFILES','SHOW CREATE SETTINGS PROFILE','SHOW CREATE PROFILE'] GLOBAL SHOW ACCESS SHOW ACCESS [] \N ACCESS MANAGEMENT -SHOW NAMED COLLECTIONS ['SHOW NAMED COLLECTIONS'] GLOBAL ACCESS MANAGEMENT +SHOW NAMED COLLECTIONS ['SHOW NAMED COLLECTIONS'] NAMED_COLLECTION ACCESS MANAGEMENT ACCESS MANAGEMENT [] \N ALL SYSTEM SHUTDOWN ['SYSTEM KILL','SHUTDOWN'] GLOBAL SYSTEM SYSTEM DROP DNS CACHE ['SYSTEM DROP DNS','DROP DNS CACHE','DROP DNS'] GLOBAL SYSTEM DROP CACHE diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index dc7cdddf8ec..484243bd523 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -571,7 +571,7 @@ CREATE TABLE system.privileges ( `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP QUERY CACHE' = 98, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 99, 'SYSTEM DROP FILESYSTEM CACHE' = 100, 'SYSTEM DROP SCHEMA CACHE' = 101, 'SYSTEM DROP S3 CLIENT CACHE' = 102, 'SYSTEM DROP CACHE' = 103, 'SYSTEM RELOAD CONFIG' = 104, 'SYSTEM RELOAD USERS' = 105, 'SYSTEM RELOAD SYMBOLS' = 106, 'SYSTEM RELOAD DICTIONARY' = 107, 'SYSTEM RELOAD MODEL' = 108, 'SYSTEM RELOAD FUNCTION' = 109, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 110, 'SYSTEM RELOAD' = 111, 'SYSTEM RESTART DISK' = 112, 'SYSTEM MERGES' = 113, 'SYSTEM TTL MERGES' = 114, 'SYSTEM FETCHES' = 115, 'SYSTEM MOVES' = 116, 'SYSTEM DISTRIBUTED SENDS' = 117, 'SYSTEM REPLICATED SENDS' = 118, 'SYSTEM SENDS' = 119, 'SYSTEM REPLICATION QUEUES' = 120, 'SYSTEM DROP REPLICA' = 121, 'SYSTEM SYNC REPLICA' = 122, 'SYSTEM RESTART REPLICA' = 123, 'SYSTEM RESTORE REPLICA' = 124, 'SYSTEM WAIT LOADING PARTS' = 125, 'SYSTEM SYNC DATABASE REPLICA' = 126, 'SYSTEM SYNC TRANSACTION LOG' = 127, 'SYSTEM SYNC FILE CACHE' = 128, 'SYSTEM FLUSH DISTRIBUTED' = 129, 'SYSTEM FLUSH LOGS' = 130, 'SYSTEM FLUSH' = 131, 'SYSTEM THREAD FUZZER' = 132, 'SYSTEM UNFREEZE' = 133, 'SYSTEM' = 134, 'dictGet' = 135, 'addressToLine' = 136, 'addressToLineWithInlines' = 137, 'addressToSymbol' = 138, 'demangle' = 139, 'INTROSPECTION' = 140, 'FILE' = 141, 'URL' = 142, 'REMOTE' = 143, 'MONGO' = 144, 'MEILISEARCH' = 145, 'MYSQL' = 146, 'POSTGRES' = 147, 'SQLITE' = 148, 'ODBC' = 149, 'JDBC' = 150, 'HDFS' = 151, 'S3' = 152, 'HIVE' = 153, 'SOURCES' = 154, 'CLUSTER' = 155, 'ALL' = 156, 'NONE' = 157), `aliases` Array(String), - `level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5)), + `level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5, 'NAMED_COLLECTION' = 6)), `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP QUERY CACHE' = 98, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 99, 'SYSTEM DROP FILESYSTEM CACHE' = 100, 'SYSTEM DROP SCHEMA CACHE' = 101, 'SYSTEM DROP S3 CLIENT CACHE' = 102, 'SYSTEM DROP CACHE' = 103, 'SYSTEM RELOAD CONFIG' = 104, 'SYSTEM RELOAD USERS' = 105, 'SYSTEM RELOAD SYMBOLS' = 106, 'SYSTEM RELOAD DICTIONARY' = 107, 'SYSTEM RELOAD MODEL' = 108, 'SYSTEM RELOAD FUNCTION' = 109, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 110, 'SYSTEM RELOAD' = 111, 'SYSTEM RESTART DISK' = 112, 'SYSTEM MERGES' = 113, 'SYSTEM TTL MERGES' = 114, 'SYSTEM FETCHES' = 115, 'SYSTEM MOVES' = 116, 'SYSTEM DISTRIBUTED SENDS' = 117, 'SYSTEM REPLICATED SENDS' = 118, 'SYSTEM SENDS' = 119, 'SYSTEM REPLICATION QUEUES' = 120, 'SYSTEM DROP REPLICA' = 121, 'SYSTEM SYNC REPLICA' = 122, 'SYSTEM RESTART REPLICA' = 123, 'SYSTEM RESTORE REPLICA' = 124, 'SYSTEM WAIT LOADING PARTS' = 125, 'SYSTEM SYNC DATABASE REPLICA' = 126, 'SYSTEM SYNC TRANSACTION LOG' = 127, 'SYSTEM SYNC FILE CACHE' = 128, 'SYSTEM FLUSH DISTRIBUTED' = 129, 'SYSTEM FLUSH LOGS' = 130, 'SYSTEM FLUSH' = 131, 'SYSTEM THREAD FUZZER' = 132, 'SYSTEM UNFREEZE' = 133, 'SYSTEM' = 134, 'dictGet' = 135, 'addressToLine' = 136, 'addressToLineWithInlines' = 137, 'addressToSymbol' = 138, 'demangle' = 139, 'INTROSPECTION' = 140, 'FILE' = 141, 'URL' = 142, 'REMOTE' = 143, 'MONGO' = 144, 'MEILISEARCH' = 145, 'MYSQL' = 146, 'POSTGRES' = 147, 'SQLITE' = 148, 'ODBC' = 149, 'JDBC' = 150, 'HDFS' = 151, 'S3' = 152, 'HIVE' = 153, 'SOURCES' = 154, 'CLUSTER' = 155, 'ALL' = 156, 'NONE' = 157)) ) ENGINE = SystemPrivileges From 6f985b8ae0b8546dc0022862ae5ad913b83d6615 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 10 Feb 2023 18:49:18 +0100 Subject: [PATCH 019/559] Fix unit test, cleanup code --- src/Access/AccessRights.cpp | 12 ++++-------- src/Access/Common/AccessRightsElement.cpp | 6 +++--- src/Access/tests/gtest_access_rights_ops.cpp | 4 ++-- src/Parsers/Access/ASTGrantQuery.cpp | 7 +++++-- 4 files changed, 14 insertions(+), 15 deletions(-) diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index ca118203541..65363babb4f 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -326,8 +326,8 @@ public: const Node * child = tryGetChild(name); if (child) return child->isGranted(flags_to_check, subnames...); - - return flags.contains(flags_to_check); + else + return flags.contains(flags_to_check); } template @@ -836,10 +836,7 @@ void AccessRights::grant(const AccessFlags & flags, std::string_view database, s void AccessRights::grant(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) { grantImpl(flags, database, table, columns); } void AccessRights::grant(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) { grantImpl(flags, database, table, columns); } void AccessRights::grant(const AccessRightsElement & element) { grantImpl(element); } -void AccessRights::grant(const AccessRightsElements & elements) -{ - grantImpl(elements); -} +void AccessRights::grant(const AccessRightsElements & elements) { grantImpl(elements); } void AccessRights::grantWithGrantOption(const AccessFlags & flags) { grantImpl(flags); } void AccessRights::grantWithGrantOption(const AccessFlags & flags, std::string_view database) { grantImpl(flags, database); } @@ -928,7 +925,7 @@ void AccessRights::revokeGrantOption(const AccessRightsElements & elements) { re AccessRightsElements AccessRights::getElements() const { -#if 1 +#if 0 logTree(); #endif if (!root) @@ -950,7 +947,6 @@ bool AccessRights::isGrantedImpl(const AccessFlags & flags, const Args &... args { auto helper = [&](const std::unique_ptr & root_node) -> bool { - logTree(); if (!root_node) return flags.isEmpty(); return root_node->isGranted(flags, args...); diff --git a/src/Access/Common/AccessRightsElement.cpp b/src/Access/Common/AccessRightsElement.cpp index 011db851bc4..70a6b3bea57 100644 --- a/src/Access/Common/AccessRightsElement.cpp +++ b/src/Access/Common/AccessRightsElement.cpp @@ -26,10 +26,10 @@ namespace result += "ON "; if (element.isNamedCollectionAccess()) { - if (!element.any_named_collection) - result += backQuoteIfNeed(element.named_collection); - else + if (element.any_named_collection) result += "*"; + else + result += backQuoteIfNeed(element.named_collection); } else if (element.any_database) { diff --git a/src/Access/tests/gtest_access_rights_ops.cpp b/src/Access/tests/gtest_access_rights_ops.cpp index e21ebda2a31..d6f827a02c5 100644 --- a/src/Access/tests/gtest_access_rights_ops.cpp +++ b/src/Access/tests/gtest_access_rights_ops.cpp @@ -48,9 +48,9 @@ TEST(AccessRights, Union) ASSERT_EQ(lhs.toString(), "GRANT INSERT ON *.*, " "GRANT SHOW, SELECT, ALTER, CREATE DATABASE, CREATE TABLE, CREATE VIEW, " - "CREATE DICTIONARY, DROP DATABASE, DROP TABLE, DROP VIEW, DROP DICTIONARY, " + "CREATE DICTIONARY, CREATE NAMED COLLECTION, DROP DATABASE, DROP TABLE, DROP VIEW, DROP DICTIONARY, DROP NAMED COLLECTION, " "TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, " - "SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, " + "SHOW ROW POLICIES, SHOW NAMED COLLECTIONS, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, " "SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, " "SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, " "SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON db1.*"); diff --git a/src/Parsers/Access/ASTGrantQuery.cpp b/src/Parsers/Access/ASTGrantQuery.cpp index e4aa11967c6..8d66ac991be 100644 --- a/src/Parsers/Access/ASTGrantQuery.cpp +++ b/src/Parsers/Access/ASTGrantQuery.cpp @@ -30,9 +30,12 @@ namespace void formatONClause(const AccessRightsElement & element, const IAST::FormatSettings & settings) { settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "ON " << (settings.hilite ? IAST::hilite_none : ""); - if (!element.any_named_collection) + if (element.isNamedCollectionAccess()) { - settings.ostr << backQuoteIfNeed(element.named_collection); + if (element.any_named_collection) + settings.ostr << "*"; + else + settings.ostr << backQuoteIfNeed(element.named_collection); } else if (element.any_database) { From 78c809608cb7b0b8b81bc339062d6535af192880 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 10 Feb 2023 23:37:34 +0100 Subject: [PATCH 020/559] Fix --- src/Access/Common/AccessFlags.h | 3 +-- src/Parsers/Access/ParserGrantQuery.cpp | 2 ++ 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Access/Common/AccessFlags.h b/src/Access/Common/AccessFlags.h index f475896b6e5..68ff28516e5 100644 --- a/src/Access/Common/AccessFlags.h +++ b/src/Access/Common/AccessFlags.h @@ -50,8 +50,7 @@ public: bool isEmpty() const { return flags.none(); } explicit operator bool() const { return !isEmpty(); } bool contains(const AccessFlags & other) const { return (flags & other.flags) == other.flags; } - bool containsOnly(const AccessFlags & other) const { return flags == other.flags; } - bool isNamedCollectionAccess() const { return containsOnly(AccessFlags::allFlagsGrantableOnNamedCollectionLevel()); } + bool isNamedCollectionAccess() const { return AccessFlags::allFlagsGrantableOnNamedCollectionLevel().contains(*this); } friend bool operator ==(const AccessFlags & left, const AccessFlags & right) { return left.flags == right.flags; } friend bool operator !=(const AccessFlags & left, const AccessFlags & right) { return !(left == right); } diff --git a/src/Parsers/Access/ParserGrantQuery.cpp b/src/Parsers/Access/ParserGrantQuery.cpp index 06660642fbf..efc2e1c3ea5 100644 --- a/src/Parsers/Access/ParserGrantQuery.cpp +++ b/src/Parsers/Access/ParserGrantQuery.cpp @@ -155,7 +155,9 @@ namespace any_database = any_table = true; } else if (!parseDatabaseAndTableNameOrAsterisks(pos, expected, database_name, any_database, table_name, any_table)) + { return false; + } for (auto & [access_flags, columns] : access_and_columns) { From c79f252a568b12989eb0ab5f17caee949324ba5e Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 11 Feb 2023 12:14:37 +0100 Subject: [PATCH 021/559] Fix test --- src/Access/Common/AccessFlags.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Access/Common/AccessFlags.h b/src/Access/Common/AccessFlags.h index 68ff28516e5..f43e54f3f33 100644 --- a/src/Access/Common/AccessFlags.h +++ b/src/Access/Common/AccessFlags.h @@ -50,7 +50,7 @@ public: bool isEmpty() const { return flags.none(); } explicit operator bool() const { return !isEmpty(); } bool contains(const AccessFlags & other) const { return (flags & other.flags) == other.flags; } - bool isNamedCollectionAccess() const { return AccessFlags::allFlagsGrantableOnNamedCollectionLevel().contains(*this); } + bool isNamedCollectionAccess() const { return !isEmpty() && AccessFlags::allFlagsGrantableOnNamedCollectionLevel().contains(*this); } friend bool operator ==(const AccessFlags & left, const AccessFlags & right) { return left.flags == right.flags; } friend bool operator !=(const AccessFlags & left, const AccessFlags & right) { return !(left == right); } From 7e0a98cbf190e4bfd0d695285ae14db509e9e5d4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 11 Feb 2023 12:36:25 +0100 Subject: [PATCH 022/559] Add test --- .../test_named_collections/test.py | 52 ++++++++++++++++++- 1 file changed, 51 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index 0f8999f43dd..d2d6455caec 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -105,7 +105,7 @@ def test_access(cluster): assert int(node.query("select count() from system.named_collections")) > 0 -def test_granular_access(cluster): +def test_granular_access_show_query(cluster): node = cluster.instances["node"] assert 1 == int(node.query("SELECT count() FROM system.named_collections")) assert ( @@ -175,6 +175,56 @@ def test_granular_access(cluster): node.query("DROP NAMED COLLECTION collection2") +def test_granular_access_create_alter_drop_query(cluster): + node = cluster.instances["node"] + node.query("CREATE USER kek") + node.query("GRANT select ON *.* TO kek") + assert 0 == int( + node.query("SELECT count() FROM system.named_collections", user="kek") + ) + + assert ( + "DB::Exception: kek: Not enough privileges. To execute this query it's necessary to have grant CREATE NAMED COLLECTION" + in node.query_and_get_error( + "CREATE NAMED COLLECTION collection2 AS key1=1, key2='value2'", user="kek" + ) + ) + node.query("GRANT create named collection ON collection2 TO kek") + node.query_and_get_error( + "CREATE NAMED COLLECTION collection2 AS key1=1, key2='value2'", user="kek" + ) + assert 0 == int( + node.query("select count() from system.named_collections", user="kek") + ) + + node.query("GRANT show named collections ON collection2 TO kek") + # assert ( + # "collection2" + # == node.query("select name from system.named_collections", user="kek").strip() + # ) + # assert ( + # "1" + # == node.query( + # "select collection['key1'] from system.named_collections where name = 'collection2'" + # ).strip() + # ) + + # assert ( + # "DB::Exception: kek: Not enough privileges. To execute this query it's necessary to have grant ALTER NAMED COLLECTION" + # in node.query_and_get_error( + # "ALTER NAMED COLLECTION collection2 SET key1=2", user="kek" + # ) + # ) + # node.query("GRANT alter named collection ON collection2 TO kek") + # node.query("ALTER NAMED COLLECTION collection2 SET key1=2", user="kek") + # assert ( + # "2" + # == node.query( + # "select collection['key1'] from system.named_collections where name = 'collection2'" + # ).strip() + # ) + + def test_config_reload(cluster): node = cluster.instances["node"] assert ( From 7beb84365019617bfeeab5487a3e8dadfa647ca7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 11 Feb 2023 19:57:41 +0100 Subject: [PATCH 023/559] Fix --- .../InterpreterAlterNamedCollectionQuery.cpp | 5 +- .../InterpreterCreateNamedCollectionQuery.cpp | 4 +- .../InterpreterDropNamedCollectionQuery.cpp | 5 +- .../test_named_collections/test.py | 82 ++++++++++--------- 4 files changed, 52 insertions(+), 44 deletions(-) diff --git a/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp b/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp index 040a8714983..478735c432b 100644 --- a/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp @@ -12,9 +12,10 @@ namespace DB BlockIO InterpreterAlterNamedCollectionQuery::execute() { auto current_context = getContext(); - current_context->checkAccess(AccessType::ALTER_NAMED_COLLECTION); - const auto & query = query_ptr->as(); + + current_context->checkAccess(AccessType::ALTER_NAMED_COLLECTION, query.collection_name); + if (!query.cluster.empty()) { DDLQueryOnClusterParams params; diff --git a/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp b/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp index 8a1a8d9dde6..bac59998062 100644 --- a/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp @@ -13,10 +13,10 @@ namespace DB BlockIO InterpreterCreateNamedCollectionQuery::execute() { auto current_context = getContext(); - current_context->checkAccess(AccessType::CREATE_NAMED_COLLECTION); - const auto & query = query_ptr->as(); + current_context->checkAccess(AccessType::CREATE_NAMED_COLLECTION, query.collection_name); + if (!query.cluster.empty()) { DDLQueryOnClusterParams params; diff --git a/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp b/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp index 064a13012a6..cc3444bb4df 100644 --- a/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp @@ -12,9 +12,10 @@ namespace DB BlockIO InterpreterDropNamedCollectionQuery::execute() { auto current_context = getContext(); - current_context->checkAccess(AccessType::DROP_NAMED_COLLECTION); - const auto & query = query_ptr->as(); + + current_context->checkAccess(AccessType::DROP_NAMED_COLLECTION, query.collection_name); + if (!query.cluster.empty()) { DDLQueryOnClusterParams params; diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index d2d6455caec..ba403d3f48b 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -70,20 +70,15 @@ def replace_in_users_config(node, old, new): ) -def test_access(cluster): +def test_default_access(cluster): node = cluster.instances["node_no_default_access"] - assert ( - "DB::Exception: default: Not enough privileges. To execute this query it's necessary to have grant SHOW NAMED COLLECTIONS ON *.*" - in node.query_and_get_error("select count() from system.named_collections") - ) + assert 0 == int(node.query("select count() from system.named_collections")) node = cluster.instances["node_no_default_access_but_with_access_management"] - assert ( - "DB::Exception: default: Not enough privileges. To execute this query it's necessary to have grant SHOW NAMED COLLECTIONS ON *.*" - in node.query_and_get_error("select count() from system.named_collections") - ) + assert 0 == int(node.query("select count() from system.named_collections")) node = cluster.instances["node"] assert int(node.query("select count() from system.named_collections")) > 0 + replace_in_users_config( node, "show_named_collections>1", "show_named_collections>0" ) @@ -91,10 +86,8 @@ def test_access(cluster): ["bash", "-c", f"cat /etc/clickhouse-server/users.d/users.xml"] ) node.restart_clickhouse() - assert ( - "DB::Exception: default: Not enough privileges. To execute this query it's necessary to have grant SHOW NAMED COLLECTIONS ON *.*" - in node.query_and_get_error("select count() from system.named_collections") - ) + assert 0 == int(node.query("select count() from system.named_collections")) + replace_in_users_config( node, "show_named_collections>0", "show_named_collections>1" ) @@ -112,6 +105,7 @@ def test_granular_access_show_query(cluster): "collection1" == node.query("SELECT name FROM system.named_collections").strip() ) + node.query("DROP USER IF EXISTS kek") node.query("CREATE USER kek") node.query("GRANT select ON *.* TO kek") assert 0 == int( @@ -156,6 +150,7 @@ def test_granular_access_show_query(cluster): == node.query("select name from system.named_collections", user="kek").strip() ) + node.query("DROP USER IF EXISTS koko") node.query("CREATE USER koko") node.query("GRANT select ON *.* TO koko") assert 0 == int( @@ -177,6 +172,7 @@ def test_granular_access_show_query(cluster): def test_granular_access_create_alter_drop_query(cluster): node = cluster.instances["node"] + node.query("DROP USER IF EXISTS kek") node.query("CREATE USER kek") node.query("GRANT select ON *.* TO kek") assert 0 == int( @@ -190,7 +186,7 @@ def test_granular_access_create_alter_drop_query(cluster): ) ) node.query("GRANT create named collection ON collection2 TO kek") - node.query_and_get_error( + node.query( "CREATE NAMED COLLECTION collection2 AS key1=1, key2='value2'", user="kek" ) assert 0 == int( @@ -198,31 +194,41 @@ def test_granular_access_create_alter_drop_query(cluster): ) node.query("GRANT show named collections ON collection2 TO kek") - # assert ( - # "collection2" - # == node.query("select name from system.named_collections", user="kek").strip() - # ) - # assert ( - # "1" - # == node.query( - # "select collection['key1'] from system.named_collections where name = 'collection2'" - # ).strip() - # ) + assert ( + "collection2" + == node.query("select name from system.named_collections", user="kek").strip() + ) + assert ( + "1" + == node.query( + "select collection['key1'] from system.named_collections where name = 'collection2'" + ).strip() + ) - # assert ( - # "DB::Exception: kek: Not enough privileges. To execute this query it's necessary to have grant ALTER NAMED COLLECTION" - # in node.query_and_get_error( - # "ALTER NAMED COLLECTION collection2 SET key1=2", user="kek" - # ) - # ) - # node.query("GRANT alter named collection ON collection2 TO kek") - # node.query("ALTER NAMED COLLECTION collection2 SET key1=2", user="kek") - # assert ( - # "2" - # == node.query( - # "select collection['key1'] from system.named_collections where name = 'collection2'" - # ).strip() - # ) + assert ( + "DB::Exception: kek: Not enough privileges. To execute this query it's necessary to have grant ALTER NAMED COLLECTION" + in node.query_and_get_error( + "ALTER NAMED COLLECTION collection2 SET key1=2", user="kek" + ) + ) + node.query("GRANT alter named collection ON collection2 TO kek") + node.query("ALTER NAMED COLLECTION collection2 SET key1=2", user="kek") + assert ( + "2" + == node.query( + "select collection['key1'] from system.named_collections where name = 'collection2'" + ).strip() + ) + + assert ( + "DB::Exception: kek: Not enough privileges. To execute this query it's necessary to have grant DROP NAMED COLLECTION" + in node.query_and_get_error("DROP NAMED COLLECTION collection2", user="kek") + ) + node.query("GRANT drop named collection ON collection2 TO kek") + node.query("DROP NAMED COLLECTION collection2", user="kek") + assert 0 == int( + node.query("select count() from system.named_collections", user="kek") + ) def test_config_reload(cluster): From d6ea566b20673cc935823a7b7560dc76678f403c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Feb 2023 16:52:21 +0000 Subject: [PATCH 024/559] Mostly support minmax projection. --- .../Optimizations/optimizeUseProjections.cpp | 62 ++++++++++++------- 1 file changed, 40 insertions(+), 22 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index ca16bd09649..28bce3dea51 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include namespace DB::QueryPlanOptimizations @@ -121,8 +122,11 @@ struct AggregateProjectionInfo AggregateProjectionInfo getAggregatingProjectionInfo( const ProjectionDescription & projection, const ContextPtr & context, - const StorageMetadataPtr & metadata_snapshot) + const StorageMetadataPtr & metadata_snapshot, + const Block & key_virtual_columns) { + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj query : {}", queryToString(projection.query_ast)); + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Sample for keys : {}", projection.sample_block_for_keys.dumpStructure()); /// This is a bad approach. /// We'd better have a separate interpreter for projections. /// Now it's not obvious we didn't miss anything here. @@ -141,6 +145,13 @@ AggregateProjectionInfo getAggregatingProjectionInfo( info.keys = query_analyzer->aggregationKeys(); info.aggregates = query_analyzer->aggregates(); + for (const auto & virt_column : key_virtual_columns) + { + const auto * input = &info.before_aggregation->addInput(virt_column); + info.before_aggregation->getOutputs().push_back(input); + info.keys.push_back(NameAndTypePair{virt_column.name, virt_column.type}); + } + return info; } @@ -487,9 +498,18 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) const auto & keys = aggregating->getParams().keys; const auto & aggregates = aggregating->getParams().aggregates; + Block key_virtual_columns = reading->getMergeTreeData().getSampleBlockWithVirtualColumns(); std::vector candidates; std::optional minmax_projection; + Block minmax_count_projection_block; + MergeTreeData::DataPartsVector minmax_projection_normal_parts; + + const auto & parts = reading->getParts(); + const auto & query_info = reading->getQueryInfo(); + auto query_info_copy = query_info; + query_info_copy.prewhere_info = nullptr; + std::shared_ptr max_added_blocks; AggregateProjectionCandidate * best_candidate = nullptr; @@ -497,7 +517,7 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { const auto * projection = &*(metadata->minmax_count_projection); LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try projection {}", projection->name); - auto info = getAggregatingProjectionInfo(*projection, context, metadata); + auto info = getAggregatingProjectionInfo(*projection, context, metadata, key_virtual_columns); LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection DAG {}", info.before_aggregation->dumpDAG()); if (auto proj_dag = analyzeAggregateProjection(info, *dag, filter_node, keys, aggregates)) { @@ -509,7 +529,20 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) }); } - best_candidate = &*minmax_projection; + minmax_count_projection_block = reading->getMergeTreeData().getMinMaxCountProjectionBlock( + metadata, + minmax_projection->dag->getRequiredColumnsNames(), + filter_node != nullptr, + query_info, + parts, + minmax_projection_normal_parts, + max_added_blocks.get(), + context); + + if (!minmax_count_projection_block) + minmax_projection.reset(); + else + best_candidate = &*minmax_projection; } if (!minmax_projection) @@ -519,7 +552,7 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try projection {}", projection->name); - auto info = getAggregatingProjectionInfo(*projection, context, metadata); + auto info = getAggregatingProjectionInfo(*projection, context, metadata, key_virtual_columns); LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection DAG {}", info.before_aggregation->dumpDAG()); if (auto proj_dag = analyzeAggregateProjection(info, *dag, filter_node, keys, aggregates)) { @@ -536,12 +569,8 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) return false; } - const auto & parts = reading->getParts(); - const auto & query_info = reading->getQueryInfo(); - MergeTreeDataSelectExecutor reader(reading->getMergeTreeData()); - std::shared_ptr max_added_blocks; if (context->getSettingsRef().select_sequential_consistency) { if (const StorageReplicatedMergeTree * replicated = dynamic_cast(&reading->getMergeTreeData())) @@ -609,29 +638,18 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) return false; QueryPlanStepPtr projection_reading; - auto query_info_copy = query_info; - query_info_copy.prewhere_info = nullptr; bool has_nornal_parts; if (minmax_projection) { - MergeTreeData::DataPartsVector normal_parts; - auto minmax_count_projection_block = reading->getMergeTreeData().getMinMaxCountProjectionBlock( - metadata, - minmax_projection->dag->getRequiredColumnsNames(), - filter_node != nullptr, - query_info, - parts, - normal_parts, - max_added_blocks.get(), - context); + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Minmax proj block {}", minmax_count_projection_block.dumpStructure()); Pipe pipe(std::make_shared(std::move(minmax_count_projection_block))); projection_reading = std::make_unique(std::move(pipe)); - has_nornal_parts = !normal_parts.empty(); + has_nornal_parts = !minmax_projection_normal_parts.empty(); if (has_nornal_parts) - reading->resetParts(std::move(normal_parts)); + reading->resetParts(std::move(minmax_projection_normal_parts)); } else { From e6e486c4a14932db5516aa79a71a3a127fd6af2e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Feb 2023 17:10:10 +0000 Subject: [PATCH 025/559] Not crash with minmax projection. --- .../Optimizations/optimizeUseProjections.cpp | 30 +++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 28bce3dea51..968dfa7631d 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -527,22 +527,22 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) .projection = projection, .dag = std::move(proj_dag), }); + + minmax_count_projection_block = reading->getMergeTreeData().getMinMaxCountProjectionBlock( + metadata, + minmax_projection->dag->getRequiredColumnsNames(), + filter_node != nullptr, + query_info, + parts, + minmax_projection_normal_parts, + max_added_blocks.get(), + context); + + if (!minmax_count_projection_block) + minmax_projection.reset(); + else + best_candidate = &*minmax_projection; } - - minmax_count_projection_block = reading->getMergeTreeData().getMinMaxCountProjectionBlock( - metadata, - minmax_projection->dag->getRequiredColumnsNames(), - filter_node != nullptr, - query_info, - parts, - minmax_projection_normal_parts, - max_added_blocks.get(), - context); - - if (!minmax_count_projection_block) - minmax_projection.reset(); - else - best_candidate = &*minmax_projection; } if (!minmax_projection) From 522a39f93f3850534f81e40b1677b0c4ed1d20f0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Feb 2023 19:56:31 +0000 Subject: [PATCH 026/559] Normal Projections analysis using query plan [In progress] --- .../QueryPlan/Optimizations/Optimizations.h | 3 +- .../QueryPlan/Optimizations/optimizeTree.cpp | 10 +- .../Optimizations/optimizeUseProjections.cpp | 235 +++++++++++++++++- 3 files changed, 245 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 0ee2cecb4df..fb791a4bd09 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -93,7 +93,8 @@ using Stack = std::vector; void optimizePrimaryKeyCondition(const Stack & stack); void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); -bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes &); +bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes); +bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes); /// Enable memory bound merging of aggregation states for remote queries /// in case it was enabled for local plan diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index d0d634d931d..f8eb4b34316 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -122,7 +122,7 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s optimizeReadInOrder(*frame.node, nodes); if (optimization_settings.optimize_projection) - applied_projection |= optimizeUseProjections(*frame.node, nodes); + applied_projection |= optimizeUseAggProjections(*frame.node, nodes); if (optimization_settings.aggregation_in_order) optimizeAggregationInOrder(*frame.node, nodes); @@ -140,6 +140,14 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s continue; } + if (optimization_settings.optimize_projection) + { + bool applied = optimizeUseNormalProjections(stack, nodes); + applied_projection |= applied; + if (applied && stack.back().next_child == 0) + continue; + } + optimizePrimaryKeyCondition(stack); enableMemoryBoundMerging(*frame.node, nodes); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 968dfa7631d..b16fa317ef7 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -167,6 +168,16 @@ struct AggregateProjectionCandidate size_t sum_marks = 0; }; +struct NormalProjectionCandidate +{ + const ProjectionDescription * projection; + + MergeTreeDataSelectAnalysisResultPtr merge_tree_projection_select_result_ptr; + MergeTreeDataSelectAnalysisResultPtr merge_tree_normal_select_result_ptr; + + size_t sum_marks = 0; +}; + ActionsDAGPtr analyzeAggregateProjection( const AggregateProjectionInfo & info, ActionsDAG & query_dag, @@ -430,7 +441,7 @@ ActionsDAGPtr analyzeAggregateProjection( return proj_dag; } -bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) +bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { if (node.children.size() != 1) return false; @@ -724,4 +735,226 @@ bool optimizeUseProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) return true; } + +bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) +{ + const auto & frame = stack.back(); + + auto * reading = typeid_cast(frame.node->step.get()); + if (!reading) + return false; + + auto iter = stack.rbegin(); + while (iter != stack.rend()) + { + auto next = std::next(iter); + + if (!typeid_cast(next->node->step.get()) && + !typeid_cast(next->node->step.get())) + break; + + iter = next; + } + + if (iter == stack.rbegin()) + return false; + + const auto metadata = reading->getStorageMetadata(); + const auto & projections = metadata->projections; + + std::vector normal_projections; + for (const auto & projection : projections) + if (projection.type == ProjectionDescription::Type::Normal) + normal_projections.push_back(&projection); + + if (normal_projections.empty()) + return false; + + ActionsDAGPtr dag; + ActionsDAG::NodeRawConstPtrs filter_nodes; + if (!buildAggregatingDAG(*iter->node->children.front(), dag, filter_nodes)) + return false; + + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", dag->dumpDAG()); + + const ActionsDAG::Node * filter_node = nullptr; + if (!filter_nodes.empty()) + { + filter_node = filter_nodes.front(); + if (filter_nodes.size() > 1) + { + FunctionOverloadResolverPtr func_builder_and = + std::make_unique( + std::make_shared()); + + filter_node = &dag->addFunction(func_builder_and, std::move(filter_nodes), {}); + } + + dag->getOutputs().push_back(filter_node); + } + + std::list candidates; + NormalProjectionCandidate * best_candidate = nullptr; + + const Block & header = frame.node->step->getOutputStream().header; + const Names & required_columns = reading->getRealColumnNames(); + const auto & parts = reading->getParts(); + const auto & query_info = reading->getQueryInfo(); + ContextPtr context = reading->getContext(); + MergeTreeDataSelectExecutor reader(reading->getMergeTreeData()); + + std::shared_ptr max_added_blocks; + if (context->getSettingsRef().select_sequential_consistency) + { + if (const StorageReplicatedMergeTree * replicated = dynamic_cast(&reading->getMergeTreeData())) + max_added_blocks = std::make_shared(replicated->getMaxAddedBlocks()); + } + + for (const auto * projection : normal_projections) + { + bool has_all_columns = true; + for (const auto & col : required_columns) + { + if (!projection->sample_block.has(col)) + { + has_all_columns = false; + break; + } + } + + if (!has_all_columns) + continue; + + MergeTreeData::DataPartsVector projection_parts; + MergeTreeData::DataPartsVector normal_parts; + for (const auto & part : parts) + { + const auto & created_projections = part->getProjectionParts(); + auto it = created_projections.find(projection->name); + if (it != created_projections.end()) + projection_parts.push_back(it->second); + else + normal_parts.push_back(part); + } + + if (projection_parts.empty()) + continue; + + ActionDAGNodes added_filter_nodes; + if (filter_node) + added_filter_nodes.nodes.push_back(filter_node); + + auto projection_result_ptr = reader.estimateNumMarksToRead( + std::move(projection_parts), + nullptr, + header.getNames(), + metadata, + projection->metadata, + query_info, /// How it is actually used? I hope that for index we need only added_filter_nodes + added_filter_nodes, + context, + context->getSettingsRef().max_threads, + max_added_blocks); + + if (projection_result_ptr->error()) + continue; + + auto & candidate = candidates.emplace_back(); + candidate.merge_tree_projection_select_result_ptr = std::move(projection_result_ptr); + candidate.sum_marks += candidate.merge_tree_projection_select_result_ptr->marks(); + + if (!normal_parts.empty()) + { + auto normal_result_ptr = reading->selectRangesToRead(std::move(normal_parts)); + + if (normal_result_ptr->error()) + continue; + + if (normal_result_ptr->marks() != 0) + { + candidate.sum_marks += normal_result_ptr->marks(); + candidate.merge_tree_normal_select_result_ptr = std::move(normal_result_ptr); + } + } + + if (best_candidate == nullptr || best_candidate->sum_marks > candidate.sum_marks) + best_candidate = &candidate; + } + + if (!best_candidate) + return false; + + auto storage_snapshot = reading->getStorageSnapshot(); + auto proj_snapshot = std::make_shared( + storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); //, storage_snapshot->data); + proj_snapshot->addProjection(best_candidate->projection); + + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}", proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString()); + + auto projection_reading = reader.readFromParts( + {}, + header.getNames(), + proj_snapshot, + query_info, + context, + reading->getMaxBlockSize(), + reading->getNumStreams(), + max_added_blocks, + best_candidate->merge_tree_projection_select_result_ptr, + reading->isParallelReadingEnabled()); + + if (!projection_reading) + { + Pipe pipe(std::make_shared(proj_snapshot->getSampleBlockForColumns(header.getNames()))); + projection_reading = std::make_unique(std::move(pipe)); + } + + bool has_nornal_parts = best_candidate->merge_tree_normal_select_result_ptr != nullptr; + if (has_nornal_parts) + reading->setAnalyzedResult(std::move(best_candidate->merge_tree_normal_select_result_ptr)); + + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}", projection_reading->getOutputStream().header.dumpStructure()); + + projection_reading->setStepDescription(best_candidate->projection->name); + + auto & projection_reading_node = nodes.emplace_back(QueryPlan::Node{.step = std::move(projection_reading)}); + auto & expr_or_filter_node = nodes.emplace_back(); + + if (filter_node) + { + expr_or_filter_node.step = std::make_unique( + projection_reading_node.step->getOutputStream(), + dag, + dag->getOutputs().front()->result_name, + true); + } + else + expr_or_filter_node.step = std::make_unique( + projection_reading_node.step->getOutputStream(), + dag); + + expr_or_filter_node.children.push_back(&projection_reading_node); + + if (!has_nornal_parts) + { + /// All parts are taken from projection + iter->node->children.front() = &expr_or_filter_node; + + //optimizeAggregationInOrder(node, nodes); + } + else + { + auto & union_node = nodes.emplace_back(); + DataStreams input_streams = {iter->node->children.front()->step->getOutputStream(), expr_or_filter_node.step->getOutputStream()}; + union_node.step = std::make_unique(std::move(input_streams)); + union_node.children = {iter->node->children.front(), &expr_or_filter_node}; + iter->node->children.front() = &union_node; + + iter->next_child = 0; + stack.resize(iter.base() - stack.begin() + 1); + } + + return true; +} + } From 33572f9acd62e2d94109ae1c309e3aa8796d6fdc Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 14 Feb 2023 19:39:21 +0100 Subject: [PATCH 027/559] Better --- src/Access/Common/AccessRightsElement.h | 6 ++++-- src/Parsers/Access/ASTGrantQuery.cpp | 4 +--- src/Parsers/Access/ParserGrantQuery.cpp | 3 +-- 3 files changed, 6 insertions(+), 7 deletions(-) diff --git a/src/Access/Common/AccessRightsElement.h b/src/Access/Common/AccessRightsElement.h index db6a1f6872f..9e972fbc43c 100644 --- a/src/Access/Common/AccessRightsElement.h +++ b/src/Access/Common/AccessRightsElement.h @@ -52,8 +52,10 @@ struct AccessRightsElement bool sameDatabaseAndTable(const AccessRightsElement & other) const { - return (database == other.database) && (any_database == other.any_database) && (table == other.table) - && (any_table == other.any_table); + return (database == other.database) && (any_database == other.any_database) + && (table == other.table) && (any_table == other.any_table) + && (named_collection == other.named_collection) && (any_named_collection == other.any_named_collection) + && (isNamedCollectionAccess() == other.isNamedCollectionAccess()); } bool sameOptions(const AccessRightsElement & other) const diff --git a/src/Parsers/Access/ASTGrantQuery.cpp b/src/Parsers/Access/ASTGrantQuery.cpp index 8d66ac991be..d29ae3b8d3f 100644 --- a/src/Parsers/Access/ASTGrantQuery.cpp +++ b/src/Parsers/Access/ASTGrantQuery.cpp @@ -77,9 +77,7 @@ namespace if (i != elements.size() - 1) { const auto & next_element = elements[i + 1]; - if ((element.database == next_element.database) && (element.any_database == next_element.any_database) - && (element.table == next_element.table) && (element.any_table == next_element.any_table) - && (element.named_collection == next_element.named_collection)) + if (element.sameDatabaseAndTable(next_element)) next_element_on_same_db_and_table = true; } diff --git a/src/Parsers/Access/ParserGrantQuery.cpp b/src/Parsers/Access/ParserGrantQuery.cpp index efc2e1c3ea5..5d688c82187 100644 --- a/src/Parsers/Access/ParserGrantQuery.cpp +++ b/src/Parsers/Access/ParserGrantQuery.cpp @@ -136,8 +136,7 @@ namespace if (!ParserKeyword{"ON"}.ignore(pos, expected)) return false; - const bool grant_named_collection_access = named_collection_access && named_collection_access == access_and_columns.size(); - if (grant_named_collection_access) + if (named_collection_access && named_collection_access == access_and_columns.size()) { ASTPtr collection; if (ParserToken{TokenType::Asterisk}.ignore(pos, expected)) From 62bce41d3d2a9bf49d231c381b12d9ab2d412bc8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Feb 2023 19:31:20 +0000 Subject: [PATCH 028/559] Fix more tests. --- .../Optimizations/optimizeUseProjections.cpp | 34 +++++++++++++++---- 1 file changed, 27 insertions(+), 7 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index b16fa317ef7..d3c5c971162 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -47,7 +47,7 @@ static void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expressi /// This function builds a common DAG which is a gerge of DAGs from Filter and Expression steps chain. -static bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, ActionsDAG::NodeRawConstPtrs & filter_nodes) +static bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, ActionsDAG::NodeRawConstPtrs & filter_nodes, bool & need_remove_column) { IQueryPlanStep * step = node.step.get(); if (auto * reading = typeid_cast(step)) @@ -66,6 +66,7 @@ static bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, Act if (prewhere_info->prewhere_actions) { appendExpression(dag, prewhere_info->prewhere_actions); + need_remove_column = prewhere_info->remove_prewhere_column; if (const auto * filter_node = dag->tryFindInOutputs(prewhere_info->prewhere_column_name)) filter_nodes.push_back(filter_node); else @@ -78,7 +79,7 @@ static bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, Act if (node.children.size() != 1) return false; - if (!buildAggregatingDAG(*node.children.front(), dag, filter_nodes)) + if (!buildAggregatingDAG(*node.children.front(), dag, filter_nodes, need_remove_column)) return false; if (auto * expression = typeid_cast(step)) @@ -88,6 +89,7 @@ static bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, Act return false; appendExpression(dag, actions); + need_remove_column = false; return true; } @@ -98,6 +100,7 @@ static bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, Act return false; appendExpression(dag, actions); + need_remove_column = filter->removesFilterColumn(); const auto * filter_expression = dag->tryFindInOutputs(filter->getFilterColumnName()); if (!filter_expression) return false; @@ -483,8 +486,9 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Has agg projection"); ActionsDAGPtr dag; + bool need_remove_column = false; ActionsDAG::NodeRawConstPtrs filter_nodes; - if (!buildAggregatingDAG(*node.children.front(), dag, filter_nodes)) + if (!buildAggregatingDAG(*node.children.front(), dag, filter_nodes, need_remove_column)) return false; LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", dag->dumpDAG()); @@ -748,6 +752,8 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) while (iter != stack.rend()) { auto next = std::next(iter); + if (next == stack.rend()) + break; if (!typeid_cast(next->node->step.get()) && !typeid_cast(next->node->step.get())) @@ -772,7 +778,8 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) ActionsDAGPtr dag; ActionsDAG::NodeRawConstPtrs filter_nodes; - if (!buildAggregatingDAG(*iter->node->children.front(), dag, filter_nodes)) + bool need_remove_column = false; + if (!buildAggregatingDAG(*iter->node->children.front(), dag, filter_nodes, need_remove_column)) return false; LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", dag->dumpDAG()); @@ -780,17 +787,29 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) const ActionsDAG::Node * filter_node = nullptr; if (!filter_nodes.empty()) { - filter_node = filter_nodes.front(); + auto & outputs = dag->getOutputs(); + filter_node = filter_nodes.back(); if (filter_nodes.size() > 1) { + if (need_remove_column) + { + size_t pos = 0; + while (pos < outputs.size() && outputs[pos] != filter_node) + ++pos; + + if (pos < outputs.size()) + outputs.erase(outputs.begin() + pos); + } + FunctionOverloadResolverPtr func_builder_and = std::make_unique( std::make_shared()); filter_node = &dag->addFunction(func_builder_and, std::move(filter_nodes), {}); + outputs.insert(outputs.begin(), filter_node); } - - dag->getOutputs().push_back(filter_node); + else if (!need_remove_column) + outputs.insert(outputs.begin(), filter_node); } std::list candidates; @@ -860,6 +879,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) continue; auto & candidate = candidates.emplace_back(); + candidate.projection = projection; candidate.merge_tree_projection_select_result_ptr = std::move(projection_result_ptr); candidate.sum_marks += candidate.merge_tree_projection_select_result_ptr->marks(); From b466855e04f78b2b6c678350be9f0c2d6c8b203c Mon Sep 17 00:00:00 2001 From: alexX512 Date: Tue, 14 Feb 2023 21:09:15 +0000 Subject: [PATCH 029/559] Add hard_cancel mod --- src/Processors/Executors/ExecutingGraph.cpp | 4 ++-- src/Processors/Executors/ExecutingGraph.h | 2 +- src/Processors/Executors/PipelineExecutor.cpp | 13 +++++-------- src/Processors/Executors/PipelineExecutor.h | 2 +- .../Executors/PullingAsyncPipelineExecutor.cpp | 4 +++- 5 files changed, 12 insertions(+), 13 deletions(-) diff --git a/src/Processors/Executors/ExecutingGraph.cpp b/src/Processors/Executors/ExecutingGraph.cpp index f43f0ce8cff..97895c8a39d 100644 --- a/src/Processors/Executors/ExecutingGraph.cpp +++ b/src/Processors/Executors/ExecutingGraph.cpp @@ -390,7 +390,7 @@ bool ExecutingGraph::updateNode(uint64_t pid, Queue & queue, Queue & async_queue return true; } -void ExecutingGraph::cancel() +void ExecutingGraph::cancel(bool hard_cancel) { std::exception_ptr exception_ptr; @@ -401,7 +401,7 @@ void ExecutingGraph::cancel() try { bool is_source = processor->getInputs().empty(); - if (is_source) + if (hard_cancel || is_source) processor->cancel(); } catch (...) diff --git a/src/Processors/Executors/ExecutingGraph.h b/src/Processors/Executors/ExecutingGraph.h index 834ef5d4d9d..69e3525d5c7 100644 --- a/src/Processors/Executors/ExecutingGraph.h +++ b/src/Processors/Executors/ExecutingGraph.h @@ -137,7 +137,7 @@ public: /// If processor wants to be expanded, lock will be upgraded to get write access to pipeline. bool updateNode(uint64_t pid, Queue & queue, Queue & async_queue); - void cancel(); + void cancel(bool hard_cancel = false); private: /// Add single edge to edges list. Check processor is known. diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index 736098b267e..8d25589315c 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -67,10 +67,12 @@ const Processors & PipelineExecutor::getProcessors() const return graph->getProcessors(); } -void PipelineExecutor::cancel() +void PipelineExecutor::cancel(bool hard_cancel) { cancelled = true; - graph->cancel(); + if (hard_cancel) + finish(); + graph->cancel(hard_cancel); } void PipelineExecutor::finish() @@ -146,10 +148,8 @@ bool PipelineExecutor::checkTimeLimitSoft() // We call cancel here so that all processors are notified and tasks waken up // so that the "break" is faster and doesn't wait for long events if (!continuing) - { cancel(); - finish(); - } + return continuing; } @@ -229,10 +229,7 @@ void PipelineExecutor::executeStepImpl(size_t thread_num, std::atomic_bool * yie break; if (!context.executeTask()) - { cancel(); - finish(); - } if (tasks.isFinished()) break; diff --git a/src/Processors/Executors/PipelineExecutor.h b/src/Processors/Executors/PipelineExecutor.h index 21bde312cbc..a45a5b6a830 100644 --- a/src/Processors/Executors/PipelineExecutor.h +++ b/src/Processors/Executors/PipelineExecutor.h @@ -48,7 +48,7 @@ public: const Processors & getProcessors() const; /// Cancel execution. May be called from another thread. - void cancel(); + void cancel(bool hard_cancel = false); /// Checks the query time limits (cancelled or timeout). Throws on cancellation or when time limit is reached and the query uses "break" bool checkTimeLimit(); diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index 0a7a9025b30..cd23c818887 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -121,6 +121,7 @@ bool PullingAsyncPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds) if (is_execution_finished) { + LOG_DEBUG(&Poco::Logger::get("PullingAsyncPipelineExecutor::pull"), "execution_finished"); /// If lazy format is finished, we don't cancel pipeline but wait for main thread to be finished. data->is_finished = true; /// Wait thread and rethrow exception if any. @@ -130,6 +131,7 @@ bool PullingAsyncPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds) if (lazy_format) { + LOG_DEBUG(&Poco::Logger::get("PullingAsyncPipelineExecutor::pull"), "Get chunk. Ms: {}", milliseconds); chunk = lazy_format->getChunk(milliseconds); data->rethrowExceptionIfHas(); return true; @@ -182,7 +184,7 @@ void PullingAsyncPipelineExecutor::cancel() try { if (!data->is_finished && data->executor) - data->executor->cancel(); + data->executor->cancel(/*hard_cancel*/ true); } catch (...) { From e424ad9ba8d0ea40b3939cfedbb22369f43010b5 Mon Sep 17 00:00:00 2001 From: alexX512 Date: Tue, 14 Feb 2023 21:24:33 +0000 Subject: [PATCH 030/559] Delete unimportant logs --- src/Processors/Executors/PullingAsyncPipelineExecutor.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index cd23c818887..bc6ec198592 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -121,7 +121,6 @@ bool PullingAsyncPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds) if (is_execution_finished) { - LOG_DEBUG(&Poco::Logger::get("PullingAsyncPipelineExecutor::pull"), "execution_finished"); /// If lazy format is finished, we don't cancel pipeline but wait for main thread to be finished. data->is_finished = true; /// Wait thread and rethrow exception if any. @@ -131,7 +130,6 @@ bool PullingAsyncPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds) if (lazy_format) { - LOG_DEBUG(&Poco::Logger::get("PullingAsyncPipelineExecutor::pull"), "Get chunk. Ms: {}", milliseconds); chunk = lazy_format->getChunk(milliseconds); data->rethrowExceptionIfHas(); return true; From 5aabfe8644cc1177ee91ca032341335e2ab57ee7 Mon Sep 17 00:00:00 2001 From: alexX512 Date: Wed, 15 Feb 2023 06:50:33 +0000 Subject: [PATCH 031/559] Hard cancel for pipeline executor --- src/Processors/Executors/PipelineExecutor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index 8d25589315c..923987417bd 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -148,7 +148,7 @@ bool PipelineExecutor::checkTimeLimitSoft() // We call cancel here so that all processors are notified and tasks waken up // so that the "break" is faster and doesn't wait for long events if (!continuing) - cancel(); + cancel(/*hard_cancel*/ true); return continuing; } @@ -229,7 +229,7 @@ void PipelineExecutor::executeStepImpl(size_t thread_num, std::atomic_bool * yie break; if (!context.executeTask()) - cancel(); + cancel(/*hard_cancel*/ true); if (tasks.isFinished()) break; From 26fd12e0c7ec9bbf32573f49a20bb59accf285cc Mon Sep 17 00:00:00 2001 From: alexX512 Date: Wed, 15 Feb 2023 15:40:42 +0000 Subject: [PATCH 032/559] Add checking for source processors in executing graph --- src/Processors/Executors/ExecutingGraph.cpp | 21 ++++++++++++++++++--- src/Processors/Executors/ExecutingGraph.h | 2 ++ 2 files changed, 20 insertions(+), 3 deletions(-) diff --git a/src/Processors/Executors/ExecutingGraph.cpp b/src/Processors/Executors/ExecutingGraph.cpp index 97895c8a39d..eb06d45c81b 100644 --- a/src/Processors/Executors/ExecutingGraph.cpp +++ b/src/Processors/Executors/ExecutingGraph.cpp @@ -16,6 +16,7 @@ ExecutingGraph::ExecutingGraph(std::shared_ptr processors_, bool pro { uint64_t num_processors = processors->size(); nodes.reserve(num_processors); + source_processors.reserve(num_processors); /// Create nodes. for (uint64_t node = 0; node < num_processors; ++node) @@ -23,6 +24,9 @@ ExecutingGraph::ExecutingGraph(std::shared_ptr processors_, bool pro IProcessor * proc = processors->at(node).get(); processors_map[proc] = node; nodes.emplace_back(std::make_unique(proc, node)); + + bool is_source = proc->getInputs().empty(); + source_processors.emplace_back(is_source); } /// Create edges. @@ -117,6 +121,14 @@ bool ExecutingGraph::expandPipeline(std::stack & stack, uint64_t pid) return false; } processors->insert(processors->end(), new_processors.begin(), new_processors.end()); + + source_processors.reserve(source_processors.size() + new_processors.size()); + + for (auto& proc: new_processors) + { + bool is_source = proc->getInputs().empty(); + source_processors.emplace_back(is_source); + } } uint64_t num_processors = processors->size(); @@ -396,13 +408,16 @@ void ExecutingGraph::cancel(bool hard_cancel) { std::lock_guard guard(processors_mutex); - for (auto & processor : *processors) + uint64_t num_processors = processors->size(); + for (uint64_t proc = 0; proc < num_processors; ++proc) { try { - bool is_source = processor->getInputs().empty(); - if (hard_cancel || is_source) + if (hard_cancel || source_processors.at(proc)) + { + IProcessor * processor = processors->at(proc).get(); processor->cancel(); + } } catch (...) { diff --git a/src/Processors/Executors/ExecutingGraph.h b/src/Processors/Executors/ExecutingGraph.h index 69e3525d5c7..971c1f0e128 100644 --- a/src/Processors/Executors/ExecutingGraph.h +++ b/src/Processors/Executors/ExecutingGraph.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -152,6 +153,7 @@ private: bool expandPipeline(std::stack & stack, uint64_t pid); std::shared_ptr processors; + std::vector source_processors; std::mutex processors_mutex; UpgradableMutex nodes_mutex; From 0c53f33e6f1deb1c708dffe9364d8bba0acf3774 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 15 Feb 2023 16:59:43 +0100 Subject: [PATCH 033/559] Make code more common: add GlobalWithParameter level --- src/Access/AccessRights.cpp | 20 +++++++-------- src/Access/Common/AccessFlags.cpp | 30 +++++++++++++++++++++-- src/Access/Common/AccessFlags.h | 19 +++++++++++--- src/Access/Common/AccessRightsElement.cpp | 8 +++--- src/Access/Common/AccessRightsElement.h | 14 +++++++---- src/Access/ContextAccess.cpp | 22 ++++++++++------- src/Common/ErrorCodes.cpp | 1 + src/Parsers/Access/ASTGrantQuery.cpp | 6 ++--- src/Parsers/Access/ParserGrantQuery.cpp | 28 +++++++++++---------- 9 files changed, 100 insertions(+), 48 deletions(-) diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index 65363babb4f..a200111a311 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -61,15 +61,15 @@ namespace res.any_database = true; res.any_table = true; res.any_column = true; - res.any_named_collection = true; + res.any_global_with_parameter = true; break; } case 1: { - if (access_flags.isNamedCollectionAccess()) + if (access_flags.isGlobalWithParameter()) { - res.any_named_collection = false; - res.named_collection = full_name[0]; + res.any_global_with_parameter = false; + res.parameter = full_name[0]; } else { @@ -792,8 +792,8 @@ void AccessRights::grantImplHelper(const AccessRightsElement & element) { assert(!element.is_partial_revoke); assert(!element.grant_option || with_grant_option); - if (!element.any_named_collection) - grantImpl(element.access_flags, element.named_collection); + if (!element.any_global_with_parameter) + grantImpl(element.access_flags, element.parameter); else if (element.any_database) grantImpl(element.access_flags); else if (element.any_table) @@ -869,8 +869,8 @@ template void AccessRights::revokeImplHelper(const AccessRightsElement & element) { assert(!element.grant_option || grant_option); - if (!element.any_named_collection) - revokeImpl(element.access_flags, element.named_collection); + if (!element.any_global_with_parameter) + revokeImpl(element.access_flags, element.parameter); else if (element.any_database) revokeImpl(element.access_flags); else if (element.any_table) @@ -961,8 +961,8 @@ template bool AccessRights::isGrantedImplHelper(const AccessRightsElement & element) const { assert(!element.grant_option || grant_option); - if (!element.any_named_collection) - return isGrantedImpl(element.access_flags, element.named_collection); + if (!element.any_global_with_parameter) + return isGrantedImpl(element.access_flags, element.parameter); else if (element.any_database) return isGrantedImpl(element.access_flags); else if (element.any_table) diff --git a/src/Access/Common/AccessFlags.cpp b/src/Access/Common/AccessFlags.cpp index c8d2b1ca4d9..a018e7d3acb 100644 --- a/src/Access/Common/AccessFlags.cpp +++ b/src/Access/Common/AccessFlags.cpp @@ -15,6 +15,7 @@ namespace ErrorCodes { extern const int UNKNOWN_ACCESS_TYPE; extern const int LOGICAL_ERROR; + extern const int MIXED_ACCESS_PARAMETER_TYPES; } namespace @@ -96,12 +97,14 @@ namespace const Flags & getAllFlags() const { return all_flags; } const Flags & getGlobalFlags() const { return all_flags_for_target[GLOBAL]; } + const Flags & getGlobalWithParameterFlags() const { return all_flags_grantable_on_global_with_parameter_level; } const Flags & getDatabaseFlags() const { return all_flags_for_target[DATABASE]; } const Flags & getTableFlags() const { return all_flags_for_target[TABLE]; } const Flags & getColumnFlags() const { return all_flags_for_target[COLUMN]; } const Flags & getDictionaryFlags() const { return all_flags_for_target[DICTIONARY]; } + const Flags & getNamedCollectionFlags() const { return all_flags_for_target[NAMED_COLLECTION]; } const Flags & getAllFlagsGrantableOnGlobalLevel() const { return getAllFlags(); } - const Flags & getAllFlagsGrantableOnNamedCollectionLevel() const { return all_flags_for_target[NAMED_COLLECTION]; } + const Flags & getAllFlagsGrantableOnGlobalWithParameterLevel() const { return getGlobalWithParameterFlags(); } const Flags & getAllFlagsGrantableOnDatabaseLevel() const { return all_flags_grantable_on_database_level; } const Flags & getAllFlagsGrantableOnTableLevel() const { return all_flags_grantable_on_table_level; } const Flags & getAllFlagsGrantableOnColumnLevel() const { return getColumnFlags(); } @@ -297,6 +300,7 @@ namespace collectAllFlags(child.get()); all_flags_grantable_on_table_level = all_flags_for_target[TABLE] | all_flags_for_target[DICTIONARY] | all_flags_for_target[COLUMN]; + all_flags_grantable_on_global_with_parameter_level = all_flags_for_target[NAMED_COLLECTION]; all_flags_grantable_on_database_level = all_flags_for_target[DATABASE] | all_flags_for_target[NAMED_COLLECTION] | all_flags_grantable_on_table_level; } @@ -350,9 +354,29 @@ namespace Flags all_flags_for_target[static_cast(NAMED_COLLECTION) + 1]; Flags all_flags_grantable_on_database_level; Flags all_flags_grantable_on_table_level; + Flags all_flags_grantable_on_global_with_parameter_level; }; } +bool AccessFlags::isGlobalWithParameter() const +{ + return getParameterType() != AccessFlags::NONE; +} + +AccessFlags::ParameterType AccessFlags::getParameterType() const +{ + if (isEmpty() || contains(AccessFlags::allGlobalFlags())) + return AccessFlags::NONE; + + /// All flags refer to NAMED COLLECTION access type. + if (AccessFlags::allNamedCollectionFlags().contains(*this)) + return AccessFlags::NAMED_COLLECTION; + + if (!contains(AccessFlags::allGlobalWithParameterFlags())) + return AccessFlags::NONE; + + throw Exception(ErrorCodes::MIXED_ACCESS_PARAMETER_TYPES, "Having mixed parameter types: {}", toString()); +} AccessFlags::AccessFlags(AccessType type) : flags(Helper::instance().accessTypeToFlags(type)) {} AccessFlags::AccessFlags(std::string_view keyword) : flags(Helper::instance().keywordToFlags(keyword)) {} @@ -363,12 +387,14 @@ std::vector AccessFlags::toAccessTypes() const { return Helper::inst std::vector AccessFlags::toKeywords() const { return Helper::instance().flagsToKeywords(flags); } AccessFlags AccessFlags::allFlags() { return Helper::instance().getAllFlags(); } AccessFlags AccessFlags::allGlobalFlags() { return Helper::instance().getGlobalFlags(); } +AccessFlags AccessFlags::allGlobalWithParameterFlags() { return Helper::instance().getGlobalWithParameterFlags(); } AccessFlags AccessFlags::allDatabaseFlags() { return Helper::instance().getDatabaseFlags(); } AccessFlags AccessFlags::allTableFlags() { return Helper::instance().getTableFlags(); } AccessFlags AccessFlags::allColumnFlags() { return Helper::instance().getColumnFlags(); } AccessFlags AccessFlags::allDictionaryFlags() { return Helper::instance().getDictionaryFlags(); } +AccessFlags AccessFlags::allNamedCollectionFlags() { return Helper::instance().getNamedCollectionFlags(); } AccessFlags AccessFlags::allFlagsGrantableOnGlobalLevel() { return Helper::instance().getAllFlagsGrantableOnGlobalLevel(); } -AccessFlags AccessFlags::allFlagsGrantableOnNamedCollectionLevel() { return Helper::instance().getAllFlagsGrantableOnNamedCollectionLevel(); } +AccessFlags AccessFlags::allFlagsGrantableOnGlobalWithParameterLevel() { return Helper::instance().getAllFlagsGrantableOnGlobalWithParameterLevel(); } AccessFlags AccessFlags::allFlagsGrantableOnDatabaseLevel() { return Helper::instance().getAllFlagsGrantableOnDatabaseLevel(); } AccessFlags AccessFlags::allFlagsGrantableOnTableLevel() { return Helper::instance().getAllFlagsGrantableOnTableLevel(); } AccessFlags AccessFlags::allFlagsGrantableOnColumnLevel() { return Helper::instance().getAllFlagsGrantableOnColumnLevel(); } diff --git a/src/Access/Common/AccessFlags.h b/src/Access/Common/AccessFlags.h index f43e54f3f33..b923b24be47 100644 --- a/src/Access/Common/AccessFlags.h +++ b/src/Access/Common/AccessFlags.h @@ -48,9 +48,16 @@ public: AccessFlags operator ~() const { AccessFlags res; res.flags = ~flags; return res; } bool isEmpty() const { return flags.none(); } + bool isAll() const { return flags.all(); } explicit operator bool() const { return !isEmpty(); } bool contains(const AccessFlags & other) const { return (flags & other.flags) == other.flags; } - bool isNamedCollectionAccess() const { return !isEmpty() && AccessFlags::allFlagsGrantableOnNamedCollectionLevel().contains(*this); } + bool isGlobalWithParameter() const; + enum ParameterType + { + NONE, + NAMED_COLLECTION, + }; + ParameterType getParameterType() const; friend bool operator ==(const AccessFlags & left, const AccessFlags & right) { return left.flags == right.flags; } friend bool operator !=(const AccessFlags & left, const AccessFlags & right) { return !(left == right); } @@ -77,6 +84,8 @@ public: /// Returns all the global flags. static AccessFlags allGlobalFlags(); + static AccessFlags allGlobalWithParameterFlags(); + /// Returns all the flags related to a database. static AccessFlags allDatabaseFlags(); @@ -89,10 +98,16 @@ public: /// Returns all the flags related to a dictionary. static AccessFlags allDictionaryFlags(); + /// Returns all the flags related to a named collection. + static AccessFlags allNamedCollectionFlags(); + /// Returns all the flags which could be granted on the global level. /// The same as allFlags(). static AccessFlags allFlagsGrantableOnGlobalLevel(); + /// Returns all the flags which could be granted on the global with parameter level. + static AccessFlags allFlagsGrantableOnGlobalWithParameterLevel(); + /// Returns all the flags which could be granted on the database level. /// Returns allDatabaseFlags() | allTableFlags() | allDictionaryFlags() | allColumnFlags(). static AccessFlags allFlagsGrantableOnDatabaseLevel(); @@ -105,8 +120,6 @@ public: /// The same as allColumnFlags(). static AccessFlags allFlagsGrantableOnColumnLevel(); - static AccessFlags allFlagsGrantableOnNamedCollectionLevel(); - static constexpr size_t SIZE = 256; private: using Flags = std::bitset; diff --git a/src/Access/Common/AccessRightsElement.cpp b/src/Access/Common/AccessRightsElement.cpp index 70a6b3bea57..2f6f1264a65 100644 --- a/src/Access/Common/AccessRightsElement.cpp +++ b/src/Access/Common/AccessRightsElement.cpp @@ -24,12 +24,12 @@ namespace void formatONClause(const AccessRightsElement & element, String & result) { result += "ON "; - if (element.isNamedCollectionAccess()) + if (element.isGlobalWithParameter()) { - if (element.any_named_collection) + if (element.any_global_with_parameter) result += "*"; else - result += backQuoteIfNeed(element.named_collection); + result += backQuoteIfNeed(element.parameter); } else if (element.any_database) { @@ -206,6 +206,8 @@ void AccessRightsElement::eraseNonGrantable() access_flags &= AccessFlags::allFlagsGrantableOnTableLevel(); else if (!any_database) access_flags &= AccessFlags::allFlagsGrantableOnDatabaseLevel(); + else if (!any_global_with_parameter) + access_flags &= AccessFlags::allFlagsGrantableOnGlobalWithParameterLevel(); else access_flags &= AccessFlags::allFlagsGrantableOnGlobalLevel(); } diff --git a/src/Access/Common/AccessRightsElement.h b/src/Access/Common/AccessRightsElement.h index 9e972fbc43c..e881767b185 100644 --- a/src/Access/Common/AccessRightsElement.h +++ b/src/Access/Common/AccessRightsElement.h @@ -11,14 +11,17 @@ namespace DB struct AccessRightsElement { AccessFlags access_flags; + String database; String table; Strings columns; - String named_collection; + String parameter; + bool any_database = true; bool any_table = true; bool any_column = true; - bool any_named_collection = true; + bool any_global_with_parameter = true; + bool grant_option = false; bool is_partial_revoke = false; @@ -54,8 +57,9 @@ struct AccessRightsElement { return (database == other.database) && (any_database == other.any_database) && (table == other.table) && (any_table == other.any_table) - && (named_collection == other.named_collection) && (any_named_collection == other.any_named_collection) - && (isNamedCollectionAccess() == other.isNamedCollectionAccess()); + && (parameter == other.parameter) && (any_global_with_parameter == other.any_global_with_parameter) + && (access_flags.getParameterType() == other.access_flags.getParameterType()) + && (isGlobalWithParameter() == other.isGlobalWithParameter()); } bool sameOptions(const AccessRightsElement & other) const @@ -71,7 +75,7 @@ struct AccessRightsElement /// If the database is empty, replaces it with `current_database`. Otherwise does nothing. void replaceEmptyDatabase(const String & current_database); - bool isNamedCollectionAccess() const { return access_flags.isNamedCollectionAccess(); } + bool isGlobalWithParameter() const { return access_flags.isGlobalWithParameter(); } /// Returns a human-readable representation like "GRANT SELECT, UPDATE(x, y) ON db.table". String toString() const; diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 6d6362a98b2..4abb161fd80 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -502,13 +502,17 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg if (!flags) return true; - /// Access to temporary tables is controlled in an unusual way, not like normal tables. - /// Creating of temporary tables is controlled by AccessType::CREATE_TEMPORARY_TABLES grant, - /// and other grants are considered as always given. - /// The DatabaseCatalog class won't resolve StorageID for temporary tables - /// which shouldn't be accessed. - if (getDatabase(args...) == DatabaseCatalog::TEMPORARY_DATABASE) - return access_granted(); + const auto parameter_type = flags.getParameterType(); + if (parameter_type == AccessFlags::NONE) + { + /// Access to temporary tables is controlled in an unusual way, not like normal tables. + /// Creating of temporary tables is controlled by AccessType::CREATE_TEMPORARY_TABLES grant, + /// and other grants are considered as always given. + /// The DatabaseCatalog class won't resolve StorageID for temporary tables + /// which shouldn't be accessed. + if (getDatabase(args...) == DatabaseCatalog::TEMPORARY_DATABASE) + return access_granted(); + } auto acs = getAccessRightsWithImplicit(); bool granted; @@ -606,8 +610,8 @@ template bool ContextAccess::checkAccessImplHelper(const AccessRightsElement & element) const { assert(!element.grant_option || grant_option); - if (!element.any_named_collection) - return checkAccessImpl(element.access_flags, element.named_collection); + if (!element.any_global_with_parameter) + return checkAccessImpl(element.access_flags, element.parameter); else if (element.any_database) return checkAccessImpl(element.access_flags); else if (element.any_table) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 028663a2176..cffe5fd357c 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -648,6 +648,7 @@ M(677, THREAD_WAS_CANCELED) \ M(678, IO_URING_INIT_FAILED) \ M(679, IO_URING_SUBMIT_ERROR) \ + M(690, MIXED_ACCESS_PARAMETER_TYPES) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Parsers/Access/ASTGrantQuery.cpp b/src/Parsers/Access/ASTGrantQuery.cpp index d29ae3b8d3f..f1a1f9184a5 100644 --- a/src/Parsers/Access/ASTGrantQuery.cpp +++ b/src/Parsers/Access/ASTGrantQuery.cpp @@ -30,12 +30,12 @@ namespace void formatONClause(const AccessRightsElement & element, const IAST::FormatSettings & settings) { settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "ON " << (settings.hilite ? IAST::hilite_none : ""); - if (element.isNamedCollectionAccess()) + if (element.isGlobalWithParameter()) { - if (element.any_named_collection) + if (element.any_global_with_parameter) settings.ostr << "*"; else - settings.ostr << backQuoteIfNeed(element.named_collection); + settings.ostr << backQuoteIfNeed(element.parameter); } else if (element.any_database) { diff --git a/src/Parsers/Access/ParserGrantQuery.cpp b/src/Parsers/Access/ParserGrantQuery.cpp index 5d688c82187..d58599ead56 100644 --- a/src/Parsers/Access/ParserGrantQuery.cpp +++ b/src/Parsers/Access/ParserGrantQuery.cpp @@ -123,30 +123,30 @@ namespace if (!parseAccessFlagsWithColumns(pos, expected, access_and_columns)) return false; - String database_name, table_name, collection_name; - bool any_database = false, any_table = false, any_named_collection = true; + String database_name, table_name, parameter; + bool any_database = false, any_table = false, any_global_with_parameter = true; - size_t named_collection_access = 0; + size_t is_global_with_parameter = 0; for (const auto & elem : access_and_columns) { - if (elem.first.isNamedCollectionAccess()) - ++named_collection_access; + if (elem.first.isGlobalWithParameter()) + ++is_global_with_parameter; } if (!ParserKeyword{"ON"}.ignore(pos, expected)) return false; - if (named_collection_access && named_collection_access == access_and_columns.size()) + if (is_global_with_parameter && is_global_with_parameter == access_and_columns.size()) { - ASTPtr collection; + ASTPtr parameter_ast; if (ParserToken{TokenType::Asterisk}.ignore(pos, expected)) { - any_named_collection = true; + any_global_with_parameter = true; } - else if (ParserIdentifier{}.parse(pos, collection, expected)) + else if (ParserIdentifier{}.parse(pos, parameter_ast, expected)) { - any_named_collection = false; - collection_name = getIdentifierName(collection); + any_global_with_parameter = false; + parameter = getIdentifierName(parameter_ast); } else return false; @@ -167,9 +167,9 @@ namespace element.any_database = any_database; element.database = database_name; element.any_table = any_table; - element.any_named_collection = any_named_collection; - element.named_collection = collection_name; + element.any_global_with_parameter = any_global_with_parameter; element.table = table_name; + element.parameter = parameter; res_elements.emplace_back(std::move(element)); } @@ -202,6 +202,8 @@ namespace throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted on the table level", old_flags.toString()); else if (!element.any_database) throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted on the database level", old_flags.toString()); + else if (!element.any_global_with_parameter) + throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted on the global with parameter level", old_flags.toString()); else throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted", old_flags.toString()); }); From c6b371ddeaa7ff69d1372d838d46be5f126a3632 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 15 Feb 2023 18:32:00 +0100 Subject: [PATCH 034/559] Better --- src/Access/AccessRights.cpp | 2 +- src/Access/Common/AccessFlags.cpp | 7 ++----- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index a200111a311..6096612059a 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -214,7 +214,7 @@ namespace switch (level) { case GLOBAL_LEVEL: return AccessFlags::allFlagsGrantableOnGlobalLevel(); - case DATABASE_LEVEL: return AccessFlags::allFlagsGrantableOnDatabaseLevel(); + case DATABASE_LEVEL: return AccessFlags::allFlagsGrantableOnDatabaseLevel() | AccessFlags::allFlagsGrantableOnGlobalWithParameterLevel(); case TABLE_LEVEL: return AccessFlags::allFlagsGrantableOnTableLevel(); case COLUMN_LEVEL: return AccessFlags::allFlagsGrantableOnColumnLevel(); } diff --git a/src/Access/Common/AccessFlags.cpp b/src/Access/Common/AccessFlags.cpp index a018e7d3acb..4f8a9798ec4 100644 --- a/src/Access/Common/AccessFlags.cpp +++ b/src/Access/Common/AccessFlags.cpp @@ -301,7 +301,7 @@ namespace all_flags_grantable_on_table_level = all_flags_for_target[TABLE] | all_flags_for_target[DICTIONARY] | all_flags_for_target[COLUMN]; all_flags_grantable_on_global_with_parameter_level = all_flags_for_target[NAMED_COLLECTION]; - all_flags_grantable_on_database_level = all_flags_for_target[DATABASE] | all_flags_for_target[NAMED_COLLECTION] | all_flags_grantable_on_table_level; + all_flags_grantable_on_database_level = all_flags_for_target[DATABASE] | all_flags_grantable_on_table_level; } Helper() @@ -365,16 +365,13 @@ bool AccessFlags::isGlobalWithParameter() const AccessFlags::ParameterType AccessFlags::getParameterType() const { - if (isEmpty() || contains(AccessFlags::allGlobalFlags())) + if (isEmpty() || !AccessFlags::allGlobalWithParameterFlags().contains(*this)) return AccessFlags::NONE; /// All flags refer to NAMED COLLECTION access type. if (AccessFlags::allNamedCollectionFlags().contains(*this)) return AccessFlags::NAMED_COLLECTION; - if (!contains(AccessFlags::allGlobalWithParameterFlags())) - return AccessFlags::NONE; - throw Exception(ErrorCodes::MIXED_ACCESS_PARAMETER_TYPES, "Having mixed parameter types: {}", toString()); } From f524dae924022d38c967b7edc172439e17b7b7ff Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 15 Feb 2023 19:17:24 +0000 Subject: [PATCH 035/559] Fix more tests. --- .../QueryPlan/Optimizations/optimizeTree.cpp | 14 ++-- .../Optimizations/optimizeUseProjections.cpp | 81 ++++++++++++++----- .../QueryPlan/ReadFromMergeTree.cpp | 40 ++++++++- src/Processors/QueryPlan/ReadFromMergeTree.h | 3 + src/Storages/MergeTree/MergeTreeData.cpp | 6 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 37 --------- .../01710_projection_aggregation_in_order.sql | 2 +- 7 files changed, 113 insertions(+), 70 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index f8eb4b34316..cba0ef257f5 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -104,7 +104,7 @@ void optimizeTreeFirstPass(const QueryPlanOptimizationSettings & settings, Query void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes) { - bool applied_projection = false; + size_t num_applied_projection = 0; bool has_reading_from_mt = false; Stack stack; @@ -122,7 +122,7 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s optimizeReadInOrder(*frame.node, nodes); if (optimization_settings.optimize_projection) - applied_projection |= optimizeUseAggProjections(*frame.node, nodes); + num_applied_projection += optimizeUseAggProjections(*frame.node, nodes); if (optimization_settings.aggregation_in_order) optimizeAggregationInOrder(*frame.node, nodes); @@ -140,11 +140,13 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s continue; } - if (optimization_settings.optimize_projection) + if (num_applied_projection < 5 && optimization_settings.optimize_projection) { bool applied = optimizeUseNormalProjections(stack, nodes); - applied_projection |= applied; - if (applied && stack.back().next_child == 0) + /// This is actually some internal knowledge + bool stack_was_updated = !stack.back().node->children.empty(); + num_applied_projection += applied; + if (applied && stack_was_updated) continue; } @@ -154,7 +156,7 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s stack.pop_back(); } - if (optimization_settings.force_use_projection && has_reading_from_mt && !applied_projection) + if (optimization_settings.force_use_projection && has_reading_from_mt && num_applied_projection == 0) throw Exception( "No projection is used when allow_experimental_projection_optimization = 1 and force_optimize_projection = 1", ErrorCodes::PROJECTION_NOT_USED); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index d3c5c971162..d928cfbd0e6 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -52,8 +52,10 @@ static bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, Act IQueryPlanStep * step = node.step.get(); if (auto * reading = typeid_cast(step)) { + std::cerr << "============ Found ReadFromMergeTreen"; if (const auto * prewhere_info = reading->getPrewhereInfo()) { + std::cerr << "============ Found prewhere info\n"; if (prewhere_info->row_level_filter) { appendExpression(dag, prewhere_info->row_level_filter); @@ -65,7 +67,9 @@ static bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, Act if (prewhere_info->prewhere_actions) { + std::cerr << "============ Found prewhere actions\n"; appendExpression(dag, prewhere_info->prewhere_actions); + std::cerr << "============ Cur dag \n" << dag->dumpDAG(); need_remove_column = prewhere_info->remove_prewhere_column; if (const auto * filter_node = dag->tryFindInOutputs(prewhere_info->prewhere_column_name)) filter_nodes.push_back(filter_node); @@ -89,6 +93,7 @@ static bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, Act return false; appendExpression(dag, actions); + std::cerr << "============ Cur e dag \n" << dag->dumpDAG(); need_remove_column = false; return true; } @@ -100,6 +105,7 @@ static bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, Act return false; appendExpression(dag, actions); + std::cerr << "============ Cur f dag \n" << dag->dumpDAG(); need_remove_column = filter->removesFilterColumn(); const auto * filter_expression = dag->tryFindInOutputs(filter->getFilterColumnName()); if (!filter_expression) @@ -468,6 +474,10 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (!reading) return false; + /// Probably some projection already was applied. + if (reading->hasAnalyzedResult()) + return false; + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 5"); const auto metadata = reading->getStorageMetadata(); @@ -748,6 +758,14 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) if (!reading) return false; + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), + "Reading {} {} has analyzed result {}", + reading->getName(), reading->getStepDescription(), reading->hasAnalyzedResult()); + + /// Probably some projection already was applied. + if (reading->hasAnalyzedResult()) + return false; + auto iter = stack.rbegin(); while (iter != stack.rend()) { @@ -782,7 +800,11 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) if (!buildAggregatingDAG(*iter->node->children.front(), dag, filter_nodes, need_remove_column)) return false; - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", dag->dumpDAG()); + if (dag) + { + dag->removeUnusedActions(); + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", dag->dumpDAG()); + } const ActionsDAG::Node * filter_node = nullptr; if (!filter_nodes.empty()) @@ -815,13 +837,18 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) std::list candidates; NormalProjectionCandidate * best_candidate = nullptr; - const Block & header = frame.node->step->getOutputStream().header; + //const Block & header = frame.node->step->getOutputStream().header; const Names & required_columns = reading->getRealColumnNames(); const auto & parts = reading->getParts(); const auto & query_info = reading->getQueryInfo(); ContextPtr context = reading->getContext(); MergeTreeDataSelectExecutor reader(reading->getMergeTreeData()); + auto ordinary_reading_select_result = reading->selectRangesToRead(parts); + size_t ordinary_reading_marks = ordinary_reading_select_result->marks(); + + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Marks for ordinary reading {}", ordinary_reading_marks); + std::shared_ptr max_added_blocks; if (context->getSettingsRef().select_sequential_consistency) { @@ -866,7 +893,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) auto projection_result_ptr = reader.estimateNumMarksToRead( std::move(projection_parts), nullptr, - header.getNames(), + required_columns, metadata, projection->metadata, query_info, /// How it is actually used? I hope that for index we need only added_filter_nodes @@ -897,12 +924,16 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) } } - if (best_candidate == nullptr || best_candidate->sum_marks > candidate.sum_marks) + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Marks for projection {} {}", projection->name ,candidate.sum_marks); + if (candidate.sum_marks < ordinary_reading_marks && (best_candidate == nullptr || candidate.sum_marks < best_candidate->sum_marks)) best_candidate = &candidate; } if (!best_candidate) + { + reading->setAnalyzedResult(std::move(ordinary_reading_select_result)); return false; + } auto storage_snapshot = reading->getStorageSnapshot(); auto proj_snapshot = std::make_shared( @@ -913,7 +944,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) auto projection_reading = reader.readFromParts( {}, - header.getNames(), + required_columns, proj_snapshot, query_info, context, @@ -925,7 +956,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) if (!projection_reading) { - Pipe pipe(std::make_shared(proj_snapshot->getSampleBlockForColumns(header.getNames()))); + Pipe pipe(std::make_shared(proj_snapshot->getSampleBlockForColumns(required_columns))); projection_reading = std::make_unique(std::move(pipe)); } @@ -938,36 +969,42 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) projection_reading->setStepDescription(best_candidate->projection->name); auto & projection_reading_node = nodes.emplace_back(QueryPlan::Node{.step = std::move(projection_reading)}); - auto & expr_or_filter_node = nodes.emplace_back(); + auto * next_node = &projection_reading_node; - if (filter_node) + if (dag) { - expr_or_filter_node.step = std::make_unique( - projection_reading_node.step->getOutputStream(), - dag, - dag->getOutputs().front()->result_name, - true); - } - else - expr_or_filter_node.step = std::make_unique( - projection_reading_node.step->getOutputStream(), - dag); + auto & expr_or_filter_node = nodes.emplace_back(); - expr_or_filter_node.children.push_back(&projection_reading_node); + if (filter_node) + { + expr_or_filter_node.step = std::make_unique( + projection_reading_node.step->getOutputStream(), + dag, + dag->getOutputs().front()->result_name, + true); + } + else + expr_or_filter_node.step = std::make_unique( + projection_reading_node.step->getOutputStream(), + dag); + + expr_or_filter_node.children.push_back(&projection_reading_node); + next_node = &expr_or_filter_node; + } if (!has_nornal_parts) { /// All parts are taken from projection - iter->node->children.front() = &expr_or_filter_node; + iter->node->children.front() = next_node; //optimizeAggregationInOrder(node, nodes); } else { auto & union_node = nodes.emplace_back(); - DataStreams input_streams = {iter->node->children.front()->step->getOutputStream(), expr_or_filter_node.step->getOutputStream()}; + DataStreams input_streams = {iter->node->children.front()->step->getOutputStream(), next_node->step->getOutputStream()}; union_node.step = std::make_unique(std::move(input_streams)); - union_node.children = {iter->node->children.front(), &expr_or_filter_node}; + union_node.children = {iter->node->children.front(), next_node}; iter->node->children.front() = &union_node; iter->next_child = 0; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index dddbf1a570a..83a622f5ff7 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -52,6 +52,7 @@ namespace ErrorCodes { extern const int INDEX_NOT_USED; extern const int LOGICAL_ERROR; + extern const int TOO_MANY_ROWS; } static MergeTreeReaderSettings getMergeTreeReaderSettings( @@ -75,6 +76,41 @@ static const PrewhereInfoPtr & getPrewhereInfoFromQueryInfo(const SelectQueryInf : query_info.prewhere_info; } +void ReadFromMergeTree::AnalysisResult::checkLimits(const Settings & settings, const SelectQueryInfo & query_info_) const +{ + + /// Do not check number of read rows if we have reading + /// in order of sorting key with limit. + /// In general case, when there exists WHERE clause + /// it's impossible to estimate number of rows precisely, + /// because we can stop reading at any time. + + SizeLimits limits; + if (settings.read_overflow_mode == OverflowMode::THROW + && settings.max_rows_to_read + && !query_info_.input_order_info) + limits = SizeLimits(settings.max_rows_to_read, 0, settings.read_overflow_mode); + + SizeLimits leaf_limits; + if (settings.read_overflow_mode_leaf == OverflowMode::THROW + && settings.max_rows_to_read_leaf + && !query_info_.input_order_info) + leaf_limits = SizeLimits(settings.max_rows_to_read_leaf, 0, settings.read_overflow_mode_leaf); + + if (limits.max_rows || leaf_limits.max_rows) + { + /// Fail fast if estimated number of rows to read exceeds the limit + size_t total_rows_estimate = selected_rows; + if (query_info_.limit > 0 && total_rows_estimate > query_info_.limit) + { + total_rows_estimate = query_info_.limit; + } + limits.check(total_rows_estimate, 0, "rows (controlled by 'max_rows_to_read' setting)", ErrorCodes::TOO_MANY_ROWS); + leaf_limits.check( + total_rows_estimate, 0, "rows (controlled by 'max_rows_to_read_leaf' setting)", ErrorCodes::TOO_MANY_ROWS); + } +} + ReadFromMergeTree::ReadFromMergeTree( MergeTreeData::DataPartsVector parts_, Names real_column_names_, @@ -1206,12 +1242,14 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::getAnalysisResult() const if (std::holds_alternative(result_ptr->result)) std::rethrow_exception(std::get(result_ptr->result)); - return std::get(result_ptr->result); + return std::get(result_ptr->result); } void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { auto result = getAnalysisResult(); + result.checkLimits(context->getSettingsRef(), query_info); + LOG_DEBUG( log, "Selected {}/{} parts by partition key, {} parts by primary key, {}/{} marks by primary key, {} marks to read from {} ranges", diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 90832d8a7ae..aaf5266a4b7 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -85,6 +85,8 @@ public: UInt64 selected_marks_pk = 0; UInt64 total_marks_pk = 0; UInt64 selected_rows = 0; + + void checkLimits(const Settings & settings, const SelectQueryInfo & query_info_) const; }; ReadFromMergeTree( @@ -157,6 +159,7 @@ public: void requestReadingInOrder(size_t prefix_size, int direction, size_t limit); + bool hasAnalyzedResult() const { return analyzed_result_ptr != nullptr; } void setAnalyzedResult(MergeTreeDataSelectAnalysisResultPtr analyzed_result_ptr_) { analyzed_result_ptr = std::move(analyzed_result_ptr_); } void resetParts(MergeTreeData::DataPartsVector parts) { prepared_parts = std::move(parts); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index dc6c70f66cd..18eeb1399d3 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6072,6 +6072,9 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg const auto & metadata_snapshot = storage_snapshot->metadata; const auto & settings = query_context->getSettingsRef(); + if (settings.query_plan_optimize_projection) + return std::nullopt; + /// TODO: Analyzer syntax analyzer result if (!query_info.syntax_analyzer_result) return std::nullopt; @@ -6159,9 +6162,6 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg if (analysis_result.join != nullptr || analysis_result.array_join != nullptr) can_use_aggregate_projection = false; - if (settings.query_plan_optimize_projection) - can_use_aggregate_projection = false; - /// Check if all needed columns can be provided by some aggregate projection. Here we also try /// to find expression matches. For example, suppose an aggregate projection contains a column /// named sum(x) and the given query also has an expression called sum(x), it's a match. This is diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index e4b0d266de0..dc843abb884 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -980,26 +980,6 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd /// Let's find what range to read from each part. { - std::atomic total_rows{0}; - - /// Do not check number of read rows if we have reading - /// in order of sorting key with limit. - /// In general case, when there exists WHERE clause - /// it's impossible to estimate number of rows precisely, - /// because we can stop reading at any time. - - SizeLimits limits; - if (settings.read_overflow_mode == OverflowMode::THROW - && settings.max_rows_to_read - && !query_info.input_order_info) - limits = SizeLimits(settings.max_rows_to_read, 0, settings.read_overflow_mode); - - SizeLimits leaf_limits; - if (settings.read_overflow_mode_leaf == OverflowMode::THROW - && settings.max_rows_to_read_leaf - && !query_info.input_order_info) - leaf_limits = SizeLimits(settings.max_rows_to_read_leaf, 0, settings.read_overflow_mode_leaf); - auto mark_cache = context->getIndexMarkCache(); auto uncompressed_cache = context->getIndexUncompressedCache(); @@ -1074,24 +1054,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd } if (!ranges.ranges.empty()) - { - if (limits.max_rows || leaf_limits.max_rows) - { - /// Fail fast if estimated number of rows to read exceeds the limit - auto current_rows_estimate = ranges.getRowsCount(); - size_t prev_total_rows_estimate = total_rows.fetch_add(current_rows_estimate); - size_t total_rows_estimate = current_rows_estimate + prev_total_rows_estimate; - if (query_info.limit > 0 && total_rows_estimate > query_info.limit) - { - total_rows_estimate = query_info.limit; - } - limits.check(total_rows_estimate, 0, "rows (controlled by 'max_rows_to_read' setting)", ErrorCodes::TOO_MANY_ROWS); - leaf_limits.check( - total_rows_estimate, 0, "rows (controlled by 'max_rows_to_read_leaf' setting)", ErrorCodes::TOO_MANY_ROWS); - } - parts_with_ranges[part_index] = std::move(ranges); - } }; size_t num_threads = std::min(num_streams, parts.size()); diff --git a/tests/queries/0_stateless/01710_projection_aggregation_in_order.sql b/tests/queries/0_stateless/01710_projection_aggregation_in_order.sql index 31d32da0ed3..c7ed91eb19b 100644 --- a/tests/queries/0_stateless/01710_projection_aggregation_in_order.sql +++ b/tests/queries/0_stateless/01710_projection_aggregation_in_order.sql @@ -21,7 +21,7 @@ ENGINE = MergeTree ORDER BY (key, ts); INSERT INTO normal SELECT - 1, + number, toDateTime('2021-12-06 00:00:00') + number, number FROM numbers(100000); From 6de781afcc60dd5a5e101426e9e1989ea07ae721 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 17 Feb 2023 19:18:46 +0000 Subject: [PATCH 036/559] Fix 01710 tests. --- .../Optimizations/optimizeUseProjections.cpp | 41 ++++++++++++++++--- .../01710_normal_projection_fix1.sql | 1 + 2 files changed, 36 insertions(+), 6 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index d928cfbd0e6..81c142229cb 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -297,16 +297,42 @@ ActionsDAGPtr analyzeAggregateProjection( if (typeid_cast(candidate.function.get())) { - /// we can ignore arguments for count() - match = AggFuncMatch{idx, {}}; - break; + bool all_args_not_null = true; + for (const auto & query_name : aggregate.argument_names) + { + auto jt = index.find(query_name); + + if (jt == index.end() || jt->second->result_type->isNullable()) + { + all_args_not_null = false; + break; + } + } + + for (const auto & proj_name : candidate.argument_names) + { + auto kt = proj_index.find(proj_name); + + if (kt == proj_index.end() || kt->second->result_type->isNullable()) + { + all_args_not_null = false; + break; + } + } + + if (all_args_not_null) + { + /// we can ignore arguments for count() + match = AggFuncMatch{idx, {}}; + break; + } } if (aggregate.argument_names.size() != candidate.argument_names.size()) continue; - ActionsDAG::NodeRawConstPtrs args; size_t num_args = aggregate.argument_names.size(); + ActionsDAG::NodeRawConstPtrs args; args.reserve(num_args); for (size_t arg = 0; arg < num_args; ++arg) { @@ -942,11 +968,14 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}", proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString()); + auto query_info_copy = query_info; + query_info_copy.prewhere_info = nullptr; + auto projection_reading = reader.readFromParts( {}, required_columns, proj_snapshot, - query_info, + query_info_copy, context, reading->getMaxBlockSize(), reading->getNumStreams(), @@ -980,7 +1009,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) expr_or_filter_node.step = std::make_unique( projection_reading_node.step->getOutputStream(), dag, - dag->getOutputs().front()->result_name, + filter_node->result_name, true); } else diff --git a/tests/queries/0_stateless/01710_normal_projection_fix1.sql b/tests/queries/0_stateless/01710_normal_projection_fix1.sql index b4d7c6e8734..89632168709 100644 --- a/tests/queries/0_stateless/01710_normal_projection_fix1.sql +++ b/tests/queries/0_stateless/01710_normal_projection_fix1.sql @@ -7,6 +7,7 @@ insert into t values (1, 2); alter table t add projection x (select * order by j); insert into t values (1, 4); +insert into t values (1, 5); set allow_experimental_projection_optimization = 1, force_optimize_projection = 1; From 84e442e62092cac85b5036309a80fb6050087cb1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 20 Feb 2023 14:21:20 +0000 Subject: [PATCH 037/559] Fixing build and style. --- src/Processors/QueryPlan/AggregatingStep.cpp | 5 + .../Optimizations/actionsDAGUtils.cpp | 42 ++-- .../QueryPlan/Optimizations/matchTrees.cpp | 230 ------------------ .../QueryPlan/Optimizations/matchTrees.h | 43 ---- .../QueryPlan/Optimizations/optimizeTree.cpp | 4 +- .../Optimizations/optimizeUseProjections.cpp | 22 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 1 - 7 files changed, 46 insertions(+), 301 deletions(-) delete mode 100644 src/Processors/QueryPlan/Optimizations/matchTrees.cpp delete mode 100644 src/Processors/QueryPlan/Optimizations/matchTrees.h diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 12f3e6c54e8..b092b718ad5 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -25,6 +25,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + static bool memoryBoundMergingWillBeUsed( bool should_produce_results_in_order_of_bucket_number, bool memory_bound_merging_of_aggregation_results_enabled, diff --git a/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp b/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp index 643e93146f4..02b7de60a4a 100644 --- a/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp +++ b/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp @@ -2,6 +2,7 @@ #include #include +#include #include @@ -11,7 +12,7 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG { using Parents = std::set; std::unordered_map inner_parents; - std::unordered_map inner_inputs_and_constants; + std::unordered_map inner_inputs; { std::stack stack; @@ -27,8 +28,8 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG const auto * node = stack.top(); stack.pop(); - if (node->type == ActionsDAG::ActionType::INPUT || node->type == ActionsDAG::ActionType::COLUMN) - inner_inputs_and_constants.emplace(node->result_name, node); + if (node->type == ActionsDAG::ActionType::INPUT) + inner_inputs.emplace(node->result_name, node); for (const auto * child : node->children) { @@ -84,10 +85,10 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG /// match.node will be set if match is found. auto & match = matches[frame.node]; - if (frame.node->type == ActionsDAG::ActionType::INPUT || frame.node->type == ActionsDAG::ActionType::COLUMN) + if (frame.node->type == ActionsDAG::ActionType::INPUT) { const ActionsDAG::Node * mapped = nullptr; - if (auto it = inner_inputs_and_constants.find(frame.node->result_name); it != inner_inputs_and_constants.end()) + if (auto it = inner_inputs.find(frame.node->result_name); it != inner_inputs.end()) mapped = it->second; match.node = mapped; @@ -101,14 +102,20 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG //std::cerr << "... Processing " << frame.node->function_base->getName() << std::endl; bool found_all_children = true; - for (const auto * child : frame.mapped_children) - if (!child) + const ActionsDAG::Node * any_child = nullptr; + size_t num_children = frame.node->children.size(); + for (size_t i = 0; i < num_children; ++i) + { + if (frame.mapped_children[i]) + any_child = frame.mapped_children[i]; + else if (!frame.node->children[i]->column || !isColumnConst(*frame.node->children[i]->column)) found_all_children = false; + } - if (found_all_children && !frame.mapped_children.empty()) + if (found_all_children && any_child) { Parents container; - Parents * intersection = &inner_parents[frame.mapped_children[0]]; + Parents * intersection = &inner_parents[any_child]; if (frame.mapped_children.size() > 1) { @@ -116,7 +123,8 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG size_t mapped_children_size = frame.mapped_children.size(); other_parents.reserve(mapped_children_size); for (size_t i = 1; i < mapped_children_size; ++i) - other_parents.push_back(&inner_parents[frame.mapped_children[i]]); + if (frame.mapped_children[i]) + other_parents.push_back(&inner_parents[frame.mapped_children[i]]); for (const auto * parent : *intersection) { @@ -148,12 +156,19 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG if (parent->type == ActionsDAG::ActionType::FUNCTION && func_name == parent->function_base->getName()) { const auto & children = parent->children; - size_t num_children = children.size(); - if (frame.mapped_children.size() == num_children) + if (children.size() == num_children) { bool all_children_matched = true; for (size_t i = 0; all_children_matched && i < num_children; ++i) - all_children_matched = frame.mapped_children[i] == children[i]; + { + if (frame.mapped_children[i] == nullptr) + { + all_children_matched = children[i]->column && isColumnConst(*children[i]->column) + && assert_cast(*children[i]->column).getField() == assert_cast(*frame.node->children[i]->column).getField(); + } + else + all_children_matched = frame.mapped_children[i] == children[i]; + } if (all_children_matched) { @@ -212,5 +227,4 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG return matches; } - } diff --git a/src/Processors/QueryPlan/Optimizations/matchTrees.cpp b/src/Processors/QueryPlan/Optimizations/matchTrees.cpp deleted file mode 100644 index 6c61b557477..00000000000 --- a/src/Processors/QueryPlan/Optimizations/matchTrees.cpp +++ /dev/null @@ -1,230 +0,0 @@ -#include -#include -#include -#include -#include - -namespace DB::QueryPlanOptimizations -{ - -MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag) -{ - using Parents = std::set; - std::unordered_map inner_parents; - std::unordered_map inner_inputs; - - { - std::stack stack; - for (const auto * out : inner_dag.getOutputs()) - { - if (inner_parents.contains(out)) - continue; - - stack.push(out); - inner_parents.emplace(out, Parents()); - while (!stack.empty()) - { - const auto * node = stack.top(); - stack.pop(); - - if (node->type == ActionsDAG::ActionType::INPUT) - inner_inputs.emplace(node->result_name, node); - - for (const auto * child : node->children) - { - auto [it, inserted] = inner_parents.emplace(child, Parents()); - it->second.emplace(node); - - if (inserted) - stack.push(child); - } - } - } - } - - struct Frame - { - const ActionsDAG::Node * node; - ActionsDAG::NodeRawConstPtrs mapped_children; - }; - - MatchedTrees::Matches matches; - std::stack stack; - - for (const auto & node : outer_dag.getNodes()) - { - if (matches.contains(&node)) - continue; - - stack.push(Frame{&node, {}}); - while (!stack.empty()) - { - auto & frame = stack.top(); - frame.mapped_children.reserve(frame.node->children.size()); - - while (frame.mapped_children.size() < frame.node->children.size()) - { - const auto * child = frame.node->children[frame.mapped_children.size()]; - auto it = matches.find(child); - if (it == matches.end()) - { - /// If match map does not contain a child, it was not visited. - stack.push(Frame{child, {}}); - break; - } - /// A node from found match may be nullptr. - /// It means that node is visited, but no match was found. - frame.mapped_children.push_back(it->second.node); - } - - if (frame.mapped_children.size() < frame.node->children.size()) - continue; - - /// Create an empty match for current node. - /// natch.node will be set if match is found. - auto & match = matches[frame.node]; - - if (frame.node->type == ActionsDAG::ActionType::INPUT) - { - const ActionsDAG::Node * mapped = nullptr; - if (auto it = inner_inputs.find(frame.node->result_name); it != inner_inputs.end()) - mapped = it->second; - - match.node = mapped; - } - else if (frame.node->type == ActionsDAG::ActionType::ALIAS) - { - match = matches[frame.node->children.at(0)]; - } - else if (frame.node->type == ActionsDAG::ActionType::FUNCTION) - { - - //std::cerr << "... Processing " << frame.node->function_base->getName() << std::endl; - - bool found_all_children = true; - const ActionsDAG::Node * any_child = nullptr; - size_t num_children = frame.node->children.size(); - for (size_t i = 0; i < num_children; ++i) - { - if (frame.mapped_children[i]) - any_child = frame.mapped_children[i]; - else if (!frame.node->children[i]->column || !isColumnConst(*frame.node->children[i]->column)) - found_all_children = false; - } - - if (found_all_children && any_child) - { - Parents container; - Parents * intersection = &inner_parents[any_child]; - - if (frame.mapped_children.size() > 1) - { - std::vector other_parents; - size_t mapped_children_size = frame.mapped_children.size(); - other_parents.reserve(mapped_children_size); - for (size_t i = 1; i < mapped_children_size; ++i) - if (frame.mapped_children[i]) - other_parents.push_back(&inner_parents[frame.mapped_children[i]]); - - for (const auto * parent : *intersection) - { - bool is_common = true; - for (const auto * set : other_parents) - { - if (!set->contains(parent)) - { - is_common = false; - break; - } - } - - if (is_common) - container.insert(parent); - } - - intersection = &container; - } - - //std::cerr << ".. Candidate parents " << intersection->size() << std::endl; - - if (!intersection->empty()) - { - auto func_name = frame.node->function_base->getName(); - for (const auto * parent : *intersection) - { - //std::cerr << ".. candidate " << parent->result_name << std::endl; - if (parent->type == ActionsDAG::ActionType::FUNCTION && func_name == parent->function_base->getName()) - { - const auto & children = parent->children; - if (children.size() == num_children) - { - bool all_children_matched = true; - for (size_t i = 0; all_children_matched && i < num_children; ++i) - { - if (frame.mapped_children[i] == nullptr) - { - all_children_matched = children[i]->column && isColumnConst(*children[i]->column) - && assert_cast(*children[i]->column).getField() == assert_cast(*frame.node->children[i]->column).getField(); - } - else - all_children_matched = frame.mapped_children[i] == children[i]; - } - - if (all_children_matched) - { - match.node = parent; - break; - } - } - } - } - } - } - - if (!match.node && frame.node->function_base->hasInformationAboutMonotonicity()) - { - size_t num_const_args = 0; - const ActionsDAG::Node * monotonic_child = nullptr; - for (const auto * child : frame.node->children) - { - if (child->column) - ++num_const_args; - else - monotonic_child = child; - } - - if (monotonic_child && num_const_args + 1 == frame.node->children.size()) - { - const auto & child_match = matches[monotonic_child]; - if (child_match.node) - { - auto info = frame.node->function_base->getMonotonicityForRange(*monotonic_child->result_type, {}, {}); - if (info.is_monotonic) - { - MatchedTrees::Monotonicity monotonicity; - monotonicity.direction *= info.is_positive ? 1 : -1; - monotonicity.strict = info.is_strict; - - if (child_match.monotonicity) - { - monotonicity.direction *= child_match.monotonicity->direction; - if (!child_match.monotonicity->strict) - monotonicity.strict = false; - } - - match.node = child_match.node; - match.monotonicity = monotonicity; - } - } - } - } - } - - stack.pop(); - } - } - - return matches; -} - -} diff --git a/src/Processors/QueryPlan/Optimizations/matchTrees.h b/src/Processors/QueryPlan/Optimizations/matchTrees.h deleted file mode 100644 index 18f0fe8b292..00000000000 --- a/src/Processors/QueryPlan/Optimizations/matchTrees.h +++ /dev/null @@ -1,43 +0,0 @@ -#include - -namespace DB::QueryPlanOptimizations -{ - - -/// This structure stores a node mapping from one DAG to another. -/// The rule is following: -/// * Input nodes are mapped by name. -/// * Function is mapped to function if all children are mapped and function names are same. -/// * Alias is mapped to it's children mapping. -/// * Monotonic function can be mapped to it's children mapping if direct mapping does not exist. -/// In this case, information about monotonicity is filled. -/// * Mapped node is nullptr if there is no mapping found. -/// -/// Overall, directly mapped nodes represent equal calculations. -/// Notes: -/// * Mapped DAG can contain many nodes which represent the same calculation. -/// In this case mapping is ambiguous and only one node is mapped. -/// * Aliases for mapped DAG are not supported. -/// DAG for PK does not contain aliases and ambiguous nodes. -struct MatchedTrees -{ - /// Monotonicity is calculated for monotonic functions chain. - /// Chain is not strict if there is any non-strict monotonic function. - struct Monotonicity - { - int direction = 1; - bool strict = true; - }; - - struct Match - { - const ActionsDAG::Node * node = nullptr; - std::optional monotonicity; - }; - - using Matches = std::unordered_map; -}; - -MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag); - -} diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 7b126a9098a..ac9e778cc76 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -159,8 +159,8 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s if (optimization_settings.force_use_projection && has_reading_from_mt && num_applied_projection == 0) throw Exception( - "No projection is used when allow_experimental_projection_optimization = 1 and force_optimize_projection = 1", - ErrorCodes::PROJECTION_NOT_USED); + ErrorCodes::PROJECTION_NOT_USED, + "No projection is used when allow_experimental_projection_optimization = 1 and force_optimize_projection = 1"); } } diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 81c142229cb..61faf984674 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -52,10 +52,10 @@ static bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, Act IQueryPlanStep * step = node.step.get(); if (auto * reading = typeid_cast(step)) { - std::cerr << "============ Found ReadFromMergeTreen"; + //std::cerr << "============ Found ReadFromMergeTreen"; if (const auto * prewhere_info = reading->getPrewhereInfo()) { - std::cerr << "============ Found prewhere info\n"; + //std::cerr << "============ Found prewhere info\n"; if (prewhere_info->row_level_filter) { appendExpression(dag, prewhere_info->row_level_filter); @@ -67,9 +67,9 @@ static bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, Act if (prewhere_info->prewhere_actions) { - std::cerr << "============ Found prewhere actions\n"; + //std::cerr << "============ Found prewhere actions\n"; appendExpression(dag, prewhere_info->prewhere_actions); - std::cerr << "============ Cur dag \n" << dag->dumpDAG(); + //std::cerr << "============ Cur dag \n" << dag->dumpDAG(); need_remove_column = prewhere_info->remove_prewhere_column; if (const auto * filter_node = dag->tryFindInOutputs(prewhere_info->prewhere_column_name)) filter_nodes.push_back(filter_node); @@ -93,7 +93,7 @@ static bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, Act return false; appendExpression(dag, actions); - std::cerr << "============ Cur e dag \n" << dag->dumpDAG(); + //std::cerr << "============ Cur e dag \n" << dag->dumpDAG(); need_remove_column = false; return true; } @@ -105,7 +105,7 @@ static bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, Act return false; appendExpression(dag, actions); - std::cerr << "============ Cur f dag \n" << dag->dumpDAG(); + //std::cerr << "============ Cur f dag \n" << dag->dumpDAG(); need_remove_column = filter->removesFilterColumn(); const auto * filter_expression = dag->tryFindInOutputs(filter->getFilterColumnName()); if (!filter_expression) @@ -282,8 +282,8 @@ ActionsDAGPtr analyzeAggregateProjection( const auto & candidate = info.aggregates[idx]; /// Note: this check is a bit strict. - /// We check that aggregate function names, arguemnt types and parameters are equal. - /// In some cases it's possilbe only to check that states are equal, + /// We check that aggregate function names, argument types and parameters are equal. + /// In some cases it's possible only to check that states are equal, /// e.g. for quantile(0.3)(...) and quantile(0.5)(...). /// But also functions sum(...) and sumIf(...) will have equal states, /// and we can't replace one to another from projection. @@ -709,7 +709,7 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); //, storage_snapshot->data); proj_snapshot->addProjection(best_candidate->projection); - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}", proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString()); + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}", proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString()); projection_reading = reader.readFromParts( {}, @@ -966,7 +966,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); //, storage_snapshot->data); proj_snapshot->addProjection(best_candidate->projection); - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}", proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString()); + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}", proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString()); auto query_info_copy = query_info; query_info_copy.prewhere_info = nullptr; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 5ba5a14bf0f..a45f09f2f92 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -56,7 +56,6 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER; extern const int ILLEGAL_COLUMN; extern const int ARGUMENT_OUT_OF_BOUND; - extern const int TOO_MANY_ROWS; extern const int CANNOT_PARSE_TEXT; extern const int TOO_MANY_PARTITIONS; extern const int DUPLICATED_PART_UUIDS; From 413a8d38faed623f421d952eb1691e082da6d8cf Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 20 Feb 2023 16:40:35 +0000 Subject: [PATCH 038/559] Fix totals row for projections. --- src/Interpreters/Aggregator.cpp | 7 +++++++ .../02521_cannot-find-column-in-projection.reference | 1 + .../0_stateless/02521_cannot-find-column-in-projection.sql | 2 +- 3 files changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 55e49eb8d19..557aac2c89a 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -2890,6 +2890,13 @@ bool Aggregator::mergeOnBlock(Block block, AggregatedDataVariants & result, bool LOG_TRACE(log, "Aggregation method: {}", result.getMethodName()); } + if ((params.overflow_row || result.type == AggregatedDataVariants::Type::without_key) && !result.without_key) + { + AggregateDataPtr place = result.aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); + createAggregateStates(place); + result.without_key = place; + } + if (result.type == AggregatedDataVariants::Type::without_key || block.info.is_overflows) mergeBlockWithoutKeyStreamsImpl(std::move(block), result); #define M(NAME, IS_TWO_LEVEL) \ diff --git a/tests/queries/0_stateless/02521_cannot-find-column-in-projection.reference b/tests/queries/0_stateless/02521_cannot-find-column-in-projection.reference index e69de29bb2d..2cd767c8054 100644 --- a/tests/queries/0_stateless/02521_cannot-find-column-in-projection.reference +++ b/tests/queries/0_stateless/02521_cannot-find-column-in-projection.reference @@ -0,0 +1 @@ +2023-01-05 10 diff --git a/tests/queries/0_stateless/02521_cannot-find-column-in-projection.sql b/tests/queries/0_stateless/02521_cannot-find-column-in-projection.sql index 31602c5bae2..c763e17a854 100644 --- a/tests/queries/0_stateless/02521_cannot-find-column-in-projection.sql +++ b/tests/queries/0_stateless/02521_cannot-find-column-in-projection.sql @@ -1,3 +1,3 @@ create table test(day Date, id UInt32) engine=MergeTree partition by day order by tuple(); insert into test select toDate('2023-01-05') AS day, number from numbers(10); -with toUInt64(id) as id_with select day, count(id_with) from test where day >= '2023-01-01' group by day limit 1000; -- { serverError NOT_FOUND_COLUMN_IN_BLOCK } +with toUInt64(id) as id_with select day, count(id_with) from test where day >= '2023-01-01' group by day limit 1000; From 37a5e10b75e56e94412ef111efe47cff64ed0b5d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 20 Feb 2023 17:55:10 +0000 Subject: [PATCH 039/559] Fix more tests. --- .../Optimizations/optimizeUseProjections.cpp | 18 ++++++++++++++++++ src/Processors/QueryPlan/ReadFromMergeTree.cpp | 12 ++++++++++++ src/Processors/QueryPlan/ReadFromMergeTree.h | 4 ++-- ...al_count_with_partition_predicate.reference | 1 + ..._trivial_count_with_partition_predicate.sql | 2 +- ...19_no_columns_in_row_level_filter.reference | 2 +- .../02343_aggregation_pipeline.reference | 17 +++++++++++------ 7 files changed, 46 insertions(+), 10 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 61faf984674..b798d094bd5 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -504,6 +504,15 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (reading->hasAnalyzedResult()) return false; + if (reading->isQueryWithFinal()) + return false; + + if (reading->isQueryWithSampling()) + return false; + + if (reading->isParallelReadingEnabled()) + return false; + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 5"); const auto metadata = reading->getStorageMetadata(); @@ -792,6 +801,15 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) if (reading->hasAnalyzedResult()) return false; + if (reading->isQueryWithFinal()) + return false; + + if (reading->isQueryWithSampling()) + return false; + + if (reading->isParallelReadingEnabled()) + return false; + auto iter = stack.rbegin(); while (iter != stack.rend()) { diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 899cfa4b7c2..a55491df4e9 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1505,6 +1505,18 @@ bool ReadFromMergeTree::isQueryWithFinal() const return select.final(); } +bool ReadFromMergeTree::isQueryWithSampling() const +{ + if (context->getSettingsRef().parallel_replicas_count > 1 && data.supportsSampling()) + return true; + + const auto & select = query_info.query->as(); + if (query_info.table_expression_modifiers) + return query_info.table_expression_modifiers->getSampleSizeRatio() != std::nullopt; + else + return select.sampleSize() != nullptr; +} + Pipe ReadFromMergeTree::spreadMarkRanges( RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, ActionsDAGPtr & result_projection) { diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index e59cbf56c25..7882df26d76 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -167,6 +167,8 @@ public: bool requestReadingInOrder(size_t prefix_size, int direction, size_t limit); static bool isFinal(const SelectQueryInfo & query_info); + bool isQueryWithFinal() const; + bool isQueryWithSampling() const; /// Returns true if the optimisation is applicable (and applies it then). bool requestOutputEachPartitionThroughSeparatePort(); @@ -197,8 +199,6 @@ private: bool sample_factor_column_queried, Poco::Logger * log); - bool isQueryWithFinal() const; - int getSortDirection() const { const InputOrderInfoPtr & order_info = query_info.getInputOrderInfo(); diff --git a/tests/queries/0_stateless/01505_trivial_count_with_partition_predicate.reference b/tests/queries/0_stateless/01505_trivial_count_with_partition_predicate.reference index 5abc312652d..b434c50b070 100644 --- a/tests/queries/0_stateless/01505_trivial_count_with_partition_predicate.reference +++ b/tests/queries/0_stateless/01505_trivial_count_with_partition_predicate.reference @@ -1,3 +1,4 @@ +3 0 0 2 diff --git a/tests/queries/0_stateless/01505_trivial_count_with_partition_predicate.sql b/tests/queries/0_stateless/01505_trivial_count_with_partition_predicate.sql index e4e2e3dd76a..76a43395ee1 100644 --- a/tests/queries/0_stateless/01505_trivial_count_with_partition_predicate.sql +++ b/tests/queries/0_stateless/01505_trivial_count_with_partition_predicate.sql @@ -7,7 +7,7 @@ insert into test1 values ('2020-09-01 00:01:02', 1), ('2020-09-01 20:01:03', 2), set max_rows_to_read = 1; -- non-optimized -select count() from test1 settings max_parallel_replicas = 3; -- { serverError 158; } +select count() from test1 settings max_parallel_replicas = 3; -- optimized (toYear is monotonic and we provide the partition expr as is) select count() from test1 where toYear(toDate(p)) = 1999; -- non-optimized (toDate(DateTime) is always monotonic, but we cannot relaxing the predicates to do trivial count()) diff --git a/tests/queries/0_stateless/02319_no_columns_in_row_level_filter.reference b/tests/queries/0_stateless/02319_no_columns_in_row_level_filter.reference index c0911ffc598..d344f57649d 100644 --- a/tests/queries/0_stateless/02319_no_columns_in_row_level_filter.reference +++ b/tests/queries/0_stateless/02319_no_columns_in_row_level_filter.reference @@ -1,4 +1,4 @@ -1000000 +0 0 0 0 diff --git a/tests/queries/0_stateless/02343_aggregation_pipeline.reference b/tests/queries/0_stateless/02343_aggregation_pipeline.reference index ec9a394d05d..ca838fdf4e0 100644 --- a/tests/queries/0_stateless/02343_aggregation_pipeline.reference +++ b/tests/queries/0_stateless/02343_aggregation_pipeline.reference @@ -92,12 +92,15 @@ ExpressionTransform × 16 (MergingAggregated) Resize 1 → 16 MergingAggregatedTransform - Resize 2 → 1 + Resize 17 → 1 (Union) - (ReadFromStorage) - AggregatingTransform - ExpressionTransform - MergeTreeInOrder 0 → 1 + (Aggregating) + Resize 1 → 16 + AggregatingTransform + (Expression) + ExpressionTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 (ReadFromRemote) explain pipeline SELECT k1, k3, sum(value) v FROM remote('127.0.0.{1,2}', currentDatabase(), proj_agg_02343) GROUP BY k1, k3 SETTINGS distributed_aggregation_memory_efficient = 1; (Expression) @@ -109,9 +112,11 @@ ExpressionTransform × 16 Resize 1 → 16 GroupingAggregatedTransform 2 → 1 (Union) - (ReadFromStorage) + (Aggregating) AggregatingTransform + (Expression) ExpressionTransform + (ReadFromMergeTree) MergeTreeInOrder 0 → 1 (ReadFromRemote) -- { echoOn } From a54b0116704ca7a1c60e82d7944532680a1ddb6b Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 20 Feb 2023 21:37:38 +0100 Subject: [PATCH 040/559] Finish for mysql --- .../NamedCollections/NamedCollections.cpp | 11 ++++ .../NamedCollections/NamedCollections.h | 2 + src/Databases/DatabaseFactory.cpp | 46 ++----------- .../MySQL/ConnectionMySQLSettings.cpp | 65 ------------------- src/Databases/MySQL/ConnectionMySQLSettings.h | 38 ----------- src/Databases/MySQL/DatabaseMySQL.cpp | 6 +- src/Databases/MySQL/DatabaseMySQL.h | 6 +- src/Dictionaries/MySQLDictionarySource.cpp | 22 +++---- .../ExternalDataSourceConfiguration.cpp | 23 ------- .../ExternalDataSourceConfiguration.h | 6 -- src/Storages/MySQL/MySQLHelpers.cpp | 10 +-- src/Storages/MySQL/MySQLHelpers.h | 5 +- src/Storages/MySQL/MySQLSettings.cpp | 12 ++++ src/Storages/MySQL/MySQLSettings.h | 4 ++ src/Storages/NamedCollectionsHelpers.cpp | 25 +++++-- src/Storages/NamedCollectionsHelpers.h | 7 +- src/Storages/StorageMySQL.cpp | 63 ++++++++++++------ src/Storages/StorageMySQL.h | 24 ++++++- src/Storages/StoragePostgreSQL.cpp | 48 ++++++++------ src/Storages/StoragePostgreSQL.h | 4 +- src/Storages/StorageURL.h | 1 - src/TableFunctions/TableFunctionMySQL.cpp | 1 - src/TableFunctions/TableFunctionMySQL.h | 4 +- 23 files changed, 179 insertions(+), 254 deletions(-) delete mode 100644 src/Databases/MySQL/ConnectionMySQLSettings.cpp delete mode 100644 src/Databases/MySQL/ConnectionMySQLSettings.h diff --git a/src/Common/NamedCollections/NamedCollections.cpp b/src/Common/NamedCollections/NamedCollections.cpp index 5db46826b19..2f80392c9ab 100644 --- a/src/Common/NamedCollections/NamedCollections.cpp +++ b/src/Common/NamedCollections/NamedCollections.cpp @@ -200,6 +200,11 @@ public: return std::unique_ptr(new Impl(collection_config, keys)); } + bool has(const Key & key) const + { + return Configuration::hasConfigValue(*config, key); + } + template T get(const Key & key) const { return Configuration::getConfigValue(*config, key); @@ -341,6 +346,12 @@ MutableNamedCollectionPtr NamedCollection::create( new NamedCollection(std::move(impl), collection_name, source_id, is_mutable)); } +bool NamedCollection::has(const Key & key) const +{ + std::lock_guard lock(mutex); + return pimpl->has(key); +} + template T NamedCollection::get(const Key & key) const { std::lock_guard lock(mutex); diff --git a/src/Common/NamedCollections/NamedCollections.h b/src/Common/NamedCollections/NamedCollections.h index 5ff9404ed69..a5b4349aaa3 100644 --- a/src/Common/NamedCollections/NamedCollections.h +++ b/src/Common/NamedCollections/NamedCollections.h @@ -33,6 +33,8 @@ public: SourceId source_id_, bool is_mutable_); + bool has(const Key & key) const; + template T get(const Key & key) const; template T getOrDefault(const Key & key, const T & default_value) const; diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 51378b66b08..97ec0de9552 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -13,7 +13,6 @@ #include #include #include -#include #include #include #include @@ -24,11 +23,11 @@ #if USE_MYSQL # include -# include # include # include # include # include +# include # include # include #endif @@ -183,21 +182,13 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String if (!engine->arguments) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", engine_name); - StorageMySQLConfiguration configuration; + StorageMySQL::Configuration configuration; ASTs & arguments = engine->arguments->children; - auto mysql_settings = std::make_unique(); + auto mysql_settings = std::make_unique(); - if (auto named_collection = getExternalDataSourceConfiguration(arguments, context, true, true, *mysql_settings)) + if (auto named_collection = tryGetNamedCollectionWithOverrides(arguments)) { - auto [common_configuration, storage_specific_args, settings_changes] = named_collection.value(); - - configuration.set(common_configuration); - configuration.addresses = {std::make_pair(configuration.host, configuration.port)}; - mysql_settings->applyChanges(settings_changes); - - if (!storage_specific_args.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "MySQL database require mysql_hostname, mysql_database_name, mysql_username, mysql_password arguments."); + configuration = StorageMySQL::processNamedCollectionResult(*named_collection, *mysql_settings, false); } else { @@ -326,19 +317,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args)) { - validateNamedCollection( - *named_collection, - {"host", "port", "user", "password", "database"}, - {"schema", "on_conflict", "use_table_cache"}); - - configuration.host = named_collection->get("host"); - configuration.port = static_cast(named_collection->get("port")); - configuration.addresses = {std::make_pair(configuration.host, configuration.port)}; - configuration.username = named_collection->get("user"); - configuration.password = named_collection->get("password"); - configuration.database = named_collection->get("database"); - configuration.schema = named_collection->getOrDefault("schema", ""); - configuration.on_conflict = named_collection->getOrDefault("on_conflict", ""); + configuration = StoragePostgreSQL::processNamedCollectionResult(*named_collection); use_table_cache = named_collection->getOrDefault("use_tables_cache", 0); } else @@ -401,18 +380,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args)) { - validateNamedCollection( - *named_collection, - {"host", "port", "user", "password", "database"}, - {"schema"}); - - configuration.host = named_collection->get("host"); - configuration.port = static_cast(named_collection->get("port")); - configuration.addresses = {std::make_pair(configuration.host, configuration.port)}; - configuration.username = named_collection->get("user"); - configuration.password = named_collection->get("password"); - configuration.database = named_collection->get("database"); - configuration.schema = named_collection->getOrDefault("schema", ""); + configuration = StoragePostgreSQL::processNamedCollectionResult(*named_collection); } else { diff --git a/src/Databases/MySQL/ConnectionMySQLSettings.cpp b/src/Databases/MySQL/ConnectionMySQLSettings.cpp deleted file mode 100644 index 50c4c7e0613..00000000000 --- a/src/Databases/MySQL/ConnectionMySQLSettings.cpp +++ /dev/null @@ -1,65 +0,0 @@ -#include - -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int UNKNOWN_SETTING; - extern const int BAD_ARGUMENTS; -} - -IMPLEMENT_SETTINGS_TRAITS(ConnectionMySQLSettingsTraits, LIST_OF_MYSQL_DATABASE_SETTINGS) - -void ConnectionMySQLSettings::loadFromQuery(ASTStorage & storage_def) -{ - if (storage_def.settings) - { - try - { - applyChanges(storage_def.settings->changes); - } - catch (Exception & e) - { - if (e.code() == ErrorCodes::UNKNOWN_SETTING) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} for database {}", e.message(), storage_def.engine->name); - else - e.rethrow(); - } - } - else - { - auto settings_ast = std::make_shared(); - settings_ast->is_standalone = false; - storage_def.set(storage_def.settings, settings_ast); - } - - SettingsChanges & changes = storage_def.settings->changes; -#define ADD_IF_ABSENT(NAME) \ - if (std::find_if(changes.begin(), changes.end(), \ - [](const SettingChange & c) { return c.name == #NAME; }) \ - == changes.end()) \ - changes.push_back(SettingChange{#NAME, static_cast(NAME)}); - - APPLY_FOR_IMMUTABLE_CONNECTION_MYSQL_SETTINGS(ADD_IF_ABSENT) -#undef ADD_IF_ABSENT -} - -void ConnectionMySQLSettings::loadFromQueryContext(ContextPtr context) -{ - if (!context->hasQueryContext()) - return; - - const Settings & settings = context->getQueryContext()->getSettingsRef(); - - if (settings.mysql_datatypes_support_level.value != mysql_datatypes_support_level.value) - set("mysql_datatypes_support_level", settings.mysql_datatypes_support_level.toString()); -} - - -} diff --git a/src/Databases/MySQL/ConnectionMySQLSettings.h b/src/Databases/MySQL/ConnectionMySQLSettings.h deleted file mode 100644 index 34902cbe9be..00000000000 --- a/src/Databases/MySQL/ConnectionMySQLSettings.h +++ /dev/null @@ -1,38 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include - -namespace DB -{ - -class ASTStorage; - -#define LIST_OF_CONNECTION_MYSQL_SETTINGS(M, ALIAS) \ - M(MySQLDataTypesSupport, mysql_datatypes_support_level, 0, "Which MySQL types should be converted to corresponding ClickHouse types (rather than being represented as String). Can be empty or any combination of 'decimal' or 'datetime64'. When empty MySQL's DECIMAL and DATETIME/TIMESTAMP with non-zero precision are seen as String on ClickHouse's side.", 0) \ - -/// Settings that should not change after the creation of a database. -#define APPLY_FOR_IMMUTABLE_CONNECTION_MYSQL_SETTINGS(M) \ - M(mysql_datatypes_support_level) - -#define LIST_OF_MYSQL_DATABASE_SETTINGS(M, ALIAS) \ - LIST_OF_CONNECTION_MYSQL_SETTINGS(M, ALIAS) \ - LIST_OF_MYSQL_SETTINGS(M, ALIAS) - -DECLARE_SETTINGS_TRAITS(ConnectionMySQLSettingsTraits, LIST_OF_MYSQL_DATABASE_SETTINGS) - - -/** Settings for the MySQL database engine. - * Could be loaded from a CREATE DATABASE query (SETTINGS clause) and Query settings. - */ -struct ConnectionMySQLSettings : public BaseSettings -{ - void loadFromQuery(ASTStorage & storage_def); - - void loadFromQueryContext(ContextPtr context); -}; - -} diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 85944319999..df1726611b4 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -53,7 +53,7 @@ DatabaseMySQL::DatabaseMySQL( const String & metadata_path_, const ASTStorage * database_engine_define_, const String & database_name_in_mysql_, - std::unique_ptr settings_, + std::unique_ptr settings_, mysqlxx::PoolWithFailover && pool, bool attach) : IDatabase(database_name_) @@ -61,7 +61,7 @@ DatabaseMySQL::DatabaseMySQL( , metadata_path(metadata_path_) , database_engine_define(database_engine_define_->clone()) , database_name_in_mysql(database_name_in_mysql_) - , database_settings(std::move(settings_)) + , mysql_settings(std::move(settings_)) , mysql_pool(std::move(pool)) /// NOLINT { try @@ -309,7 +309,7 @@ DatabaseMySQL::fetchTablesColumnsList(const std::vector & tables_name, C database_name_in_mysql, tables_name, settings, - database_settings->mysql_datatypes_support_level); + mysql_settings->mysql_datatypes_support_level); } void DatabaseMySQL::shutdown() diff --git a/src/Databases/MySQL/DatabaseMySQL.h b/src/Databases/MySQL/DatabaseMySQL.h index a9c06074237..f34a2fff4f7 100644 --- a/src/Databases/MySQL/DatabaseMySQL.h +++ b/src/Databases/MySQL/DatabaseMySQL.h @@ -9,8 +9,8 @@ #include #include #include +#include #include -#include #include #include @@ -44,7 +44,7 @@ public: const String & metadata_path, const ASTStorage * database_engine_define, const String & database_name_in_mysql, - std::unique_ptr settings_, + std::unique_ptr settings_, mysqlxx::PoolWithFailover && pool, bool attach); @@ -93,7 +93,7 @@ private: String metadata_path; ASTPtr database_engine_define; String database_name_in_mysql; - std::unique_ptr database_settings; + std::unique_ptr mysql_settings; std::atomic quit{false}; std::condition_variable cond; diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index c8491d99255..82a2762e61e 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -13,9 +13,9 @@ #include #include #include -#include #include #include +#include #include #include #include @@ -68,27 +68,21 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory) auto settings_config_prefix = config_prefix + ".mysql"; std::shared_ptr pool; MySQLSettings mysql_settings; - auto has_config_key = [&](const String & key) - { - return dictionary_allowed_keys.contains(key) || key.starts_with("replica") || mysql_settings.has(key); - }; - StorageMySQLConfiguration configuration; - auto named_collection = created_from_ddl - ? getExternalDataSourceConfiguration(config, settings_config_prefix, global_context, has_config_key, mysql_settings) - : std::nullopt; + + StorageMySQL::Configuration configuration; + auto named_collection = created_from_ddl ? tryGetNamedCollectionWithOverrides(config, settings_config_prefix) : nullptr; if (named_collection) { - if (created_from_ddl) - global_context->getRemoteHostFilter().checkHostAndPort(configuration.host, toString(configuration.port)); + named_collection->remove("name"); + configuration = StorageMySQL::processNamedCollectionResult(*named_collection, mysql_settings); + global_context->getRemoteHostFilter().checkHostAndPort(configuration.host, toString(configuration.port)); - mysql_settings.applyChanges(named_collection->settings_changes); - configuration.set(named_collection->configuration); - configuration.addresses = {std::make_pair(configuration.host, configuration.port)}; const auto & settings = global_context->getSettingsRef(); if (!mysql_settings.isChanged("connect_timeout")) mysql_settings.connect_timeout = settings.external_storage_connect_timeout_sec; if (!mysql_settings.isChanged("read_write_timeout")) mysql_settings.read_write_timeout = settings.external_storage_rw_timeout_sec; + pool = std::make_shared(createMySQLPoolWithFailover(configuration, mysql_settings)); } else diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index d7c3fe44f38..4882c644f74 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -15,10 +15,6 @@ #if USE_RDKAFKA #include #endif -#if USE_MYSQL -#include -#include -#endif #if USE_NATSIO #include #endif @@ -473,23 +469,4 @@ template SettingsChanges getSettingsChangesFromConfig( const BaseSettings & settings, const Poco::Util::AbstractConfiguration & config, const String & config_prefix); -#if USE_MYSQL -template -std::optional getExternalDataSourceConfiguration( - const ASTs & args, ContextPtr context, bool is_database_engine, bool throw_on_no_collection, const BaseSettings & storage_settings); - -template -std::optional getExternalDataSourceConfiguration( - const ASTs & args, ContextPtr context, bool is_database_engine, bool throw_on_no_collection, const BaseSettings & storage_settings); - -template -std::optional getExternalDataSourceConfiguration( - const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, - ContextPtr context, HasConfigKeyFunc has_config_key, const BaseSettings & settings); - -template -SettingsChanges getSettingsChangesFromConfig( - const BaseSettings & settings, const Poco::Util::AbstractConfiguration & config, const String & config_prefix); - -#endif } diff --git a/src/Storages/ExternalDataSourceConfiguration.h b/src/Storages/ExternalDataSourceConfiguration.h index d042f763b11..7095f6b1b04 100644 --- a/src/Storages/ExternalDataSourceConfiguration.h +++ b/src/Storages/ExternalDataSourceConfiguration.h @@ -40,12 +40,6 @@ struct StoragePostgreSQLConfiguration : ExternalDataSourceConfiguration }; -struct StorageMySQLConfiguration : ExternalDataSourceConfiguration -{ - bool replace_query = false; - String on_duplicate_clause; -}; - using StorageSpecificArgs = std::vector>; struct ExternalDataSourceInfo diff --git a/src/Storages/MySQL/MySQLHelpers.cpp b/src/Storages/MySQL/MySQLHelpers.cpp index 127bdb96eaf..63a3436ea4a 100644 --- a/src/Storages/MySQL/MySQLHelpers.cpp +++ b/src/Storages/MySQL/MySQLHelpers.cpp @@ -2,9 +2,7 @@ #if USE_MYSQL #include -#include #include -#include namespace DB { @@ -14,8 +12,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -template mysqlxx::PoolWithFailover -createMySQLPoolWithFailover(const StorageMySQLConfiguration & configuration, const T & mysql_settings) +mysqlxx::PoolWithFailover createMySQLPoolWithFailover(const StorageMySQL::Configuration & configuration, const MySQLSettings & mysql_settings) { if (!mysql_settings.connection_pool_size) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Connection pool cannot have zero size"); @@ -30,11 +27,6 @@ createMySQLPoolWithFailover(const StorageMySQLConfiguration & configuration, con mysql_settings.read_write_timeout); } -template -mysqlxx::PoolWithFailover createMySQLPoolWithFailover(const StorageMySQLConfiguration & configuration, const MySQLSettings & mysql_settings); -template -mysqlxx::PoolWithFailover createMySQLPoolWithFailover(const StorageMySQLConfiguration & configuration, const ConnectionMySQLSettings & mysql_settings); - } #endif diff --git a/src/Storages/MySQL/MySQLHelpers.h b/src/Storages/MySQL/MySQLHelpers.h index 57b564c360c..71c331da16f 100644 --- a/src/Storages/MySQL/MySQLHelpers.h +++ b/src/Storages/MySQL/MySQLHelpers.h @@ -3,15 +3,14 @@ #if USE_MYSQL #include +#include namespace mysqlxx { class PoolWithFailover; } namespace DB { -struct StorageMySQLConfiguration; -template mysqlxx::PoolWithFailover -createMySQLPoolWithFailover(const StorageMySQLConfiguration & configuration, const T & mysql_settings); +mysqlxx::PoolWithFailover createMySQLPoolWithFailover(const StorageMySQL::Configuration & configuration, const MySQLSettings & mysql_settings); } diff --git a/src/Storages/MySQL/MySQLSettings.cpp b/src/Storages/MySQL/MySQLSettings.cpp index 5c1a2246ae9..b3bc11482f4 100644 --- a/src/Storages/MySQL/MySQLSettings.cpp +++ b/src/Storages/MySQL/MySQLSettings.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -43,4 +44,15 @@ void MySQLSettings::loadFromQuery(ASTStorage & storage_def) } } +void MySQLSettings::loadFromQueryContext(ContextPtr context) +{ + if (!context->hasQueryContext()) + return; + + const Settings & settings = context->getQueryContext()->getSettingsRef(); + + if (settings.mysql_datatypes_support_level.value != mysql_datatypes_support_level.value) + set("mysql_datatypes_support_level", settings.mysql_datatypes_support_level.toString()); +} + } diff --git a/src/Storages/MySQL/MySQLSettings.h b/src/Storages/MySQL/MySQLSettings.h index 603b477b856..40771d0aacb 100644 --- a/src/Storages/MySQL/MySQLSettings.h +++ b/src/Storages/MySQL/MySQLSettings.h @@ -2,6 +2,8 @@ #include #include +#include +#include namespace Poco::Util @@ -22,6 +24,7 @@ class ASTSetQuery; M(Bool, connection_auto_close, true, "Auto-close connection after query execution, i.e. disable connection reuse.", 0) \ M(UInt64, connect_timeout, DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, "Connect timeout (in seconds)", 0) \ M(UInt64, read_write_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "Read/write timeout (in seconds)", 0) \ + M(MySQLDataTypesSupport, mysql_datatypes_support_level, 0, "Which MySQL types should be converted to corresponding ClickHouse types (rather than being represented as String). Can be empty or any combination of 'decimal' or 'datetime64'. When empty MySQL's DECIMAL and DATETIME/TIMESTAMP with non-zero precision are seen as String on ClickHouse's side.", 0) \ DECLARE_SETTINGS_TRAITS(MySQLSettingsTraits, LIST_OF_MYSQL_SETTINGS) @@ -34,6 +37,7 @@ struct MySQLSettings : public MySQLBaseSettings { void loadFromQuery(ASTStorage & storage_def); void loadFromQuery(const ASTSetQuery & settings_def); + void loadFromQueryContext(ContextPtr context); }; diff --git a/src/Storages/NamedCollectionsHelpers.cpp b/src/Storages/NamedCollectionsHelpers.cpp index cefed555781..0c7426bb682 100644 --- a/src/Storages/NamedCollectionsHelpers.cpp +++ b/src/Storages/NamedCollectionsHelpers.cpp @@ -53,7 +53,7 @@ namespace } -NamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts) +MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts) { if (asts.empty()) return nullptr; @@ -62,11 +62,11 @@ NamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts) if (!collection) return nullptr; - if (asts.size() == 1) - return collection; - auto collection_copy = collection->duplicate(); + if (asts.size() == 1) + return collection_copy; + for (auto * it = std::next(asts.begin()); it != asts.end(); ++it) { auto value_override = getKeyValueFromAST(*it); @@ -82,6 +82,23 @@ NamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts) return collection_copy; } +MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) +{ + auto collection_name = config.getString(config_prefix + ".name", ""); + if (collection_name.empty()) + return nullptr; + + const auto & collection = NamedCollectionFactory::instance().get(collection_name); + auto collection_copy = collection->duplicate(); + + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(config_prefix, keys); + for (const auto & key : keys) + collection_copy->setOrUpdate(key, config.getString(config_prefix + '.' + key)); + + return collection_copy; +} + HTTPHeaderEntries getHeadersFromNamedCollection(const NamedCollection & collection) { HTTPHeaderEntries headers; diff --git a/src/Storages/NamedCollectionsHelpers.h b/src/Storages/NamedCollectionsHelpers.h index 9c70a46d755..5f8f316d023 100644 --- a/src/Storages/NamedCollectionsHelpers.h +++ b/src/Storages/NamedCollectionsHelpers.h @@ -16,7 +16,12 @@ namespace ErrorCodes namespace DB { -NamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts); +/// Helper function to get named collection for table engine. +/// Table engines have collection name as first argument of ast and other arguments are key-value overrides. +MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts); +/// Helper function to get named collection for dictionary source. +/// Dictionaries have collection name as name argument of dict configuration and other arguments are overrides. +MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); HTTPHeaderEntries getHeadersFromNamedCollection(const NamedCollection & collection); diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index ee647043407..bc39e76be29 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -20,6 +20,7 @@ #include #include #include +#include namespace DB @@ -235,31 +236,53 @@ SinkToStoragePtr StorageMySQL::write(const ASTPtr & /*query*/, const StorageMeta local_context->getSettingsRef().mysql_max_rows_to_insert); } - -StorageMySQLConfiguration StorageMySQL::getConfiguration(ASTs engine_args, ContextPtr context_, MySQLBaseSettings & storage_settings) +StorageMySQL::Configuration StorageMySQL::processNamedCollectionResult( + const NamedCollection & named_collection, MySQLSettings & storage_settings, bool require_table) { - StorageMySQLConfiguration configuration; + StorageMySQL::Configuration configuration; - if (auto named_collection = getExternalDataSourceConfiguration( - engine_args, context_, /* is_database_engine */false, /* throw_on_no_collection */true, storage_settings)) + std::unordered_set optional_arguments = {"replace_query", "on_duplicate_clause", "addresses_expr", "host", "port"}; + auto mysql_settings = storage_settings.all(); + for (const auto & setting : mysql_settings) + optional_arguments.insert(setting.getName()); + + std::unordered_set required_arguments = {"user", "password", "database", "table"}; + if (require_table) + required_arguments.insert("table"); + validateNamedCollection(named_collection, required_arguments, optional_arguments); + + configuration.addresses_expr = named_collection.getOrDefault("addresses_expr", ""); + if (configuration.addresses_expr.empty()) { - auto [common_configuration, storage_specific_args, settings_changes] = named_collection.value(); - configuration.set(common_configuration); + configuration.host = named_collection.get("host"); + configuration.port = static_cast(named_collection.get("port")); configuration.addresses = {std::make_pair(configuration.host, configuration.port)}; - storage_settings.applyChanges(settings_changes); + } - for (const auto & [arg_name, arg_value] : storage_specific_args) - { - if (arg_name == "replace_query") - configuration.replace_query = checkAndGetLiteralArgument(arg_value, "replace_query"); - else if (arg_name == "on_duplicate_clause") - configuration.on_duplicate_clause = checkAndGetLiteralArgument(arg_value, "on_duplicate_clause"); - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Unexpected key-value argument." - "Got: {}, but expected one of:" - "host, port, username, password, database, table, replace_query, on_duplicate_clause.", arg_name); - } + configuration.username = named_collection.get("user"); + configuration.password = named_collection.get("password"); + configuration.database = named_collection.get("database"); + if (require_table) + configuration.table = named_collection.get("table"); + configuration.replace_query = named_collection.getOrDefault("replace_query", false); + configuration.on_duplicate_clause = named_collection.getOrDefault("on_duplicate_clause", ""); + + for (const auto & setting : mysql_settings) + { + const auto & setting_name = setting.getName(); + if (named_collection.has(setting_name)) + storage_settings.set(setting_name, named_collection.get(setting_name)); + } + + return configuration; +} + +StorageMySQL::Configuration StorageMySQL::getConfiguration(ASTs engine_args, ContextPtr context_, MySQLSettings & storage_settings) +{ + StorageMySQL::Configuration configuration; + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args)) + { + configuration = StorageMySQL::processNamedCollectionResult(*named_collection, storage_settings); } else { diff --git a/src/Storages/StorageMySQL.h b/src/Storages/StorageMySQL.h index bf9a24c9bfe..1c0e2639717 100644 --- a/src/Storages/StorageMySQL.h +++ b/src/Storages/StorageMySQL.h @@ -6,7 +6,6 @@ #include #include -#include #include namespace Poco @@ -17,6 +16,8 @@ class Logger; namespace DB { +class NamedCollection; + /** Implements storage in the MySQL database. * Use ENGINE = mysql(host_port, database_name, table_name, user_name, password) * Read only. @@ -50,7 +51,26 @@ public: SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; - static StorageMySQLConfiguration getConfiguration(ASTs engine_args, ContextPtr context_, MySQLBaseSettings & storage_settings); + struct Configuration + { + String host; + UInt16 port = 0; + String username = "default"; + String password; + String database; + String table; + + bool replace_query = false; + String on_duplicate_clause; + + std::vector> addresses; /// Failover replicas. + String addresses_expr; + }; + + static Configuration getConfiguration(ASTs engine_args, ContextPtr context_, MySQLSettings & storage_settings); + + static Configuration processNamedCollectionResult( + const NamedCollection & named_collection, MySQLSettings & storage_settings, bool require_table = true); private: friend class StorageMySQLSink; diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 400430b9ea2..3fa0c137f7b 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -387,31 +387,41 @@ SinkToStoragePtr StoragePostgreSQL::write( return std::make_shared(metadata_snapshot, pool->get(), remote_table_name, remote_table_schema, on_conflict); } +StoragePostgreSQL::Configuration StoragePostgreSQL::processNamedCollectionResult(const NamedCollection & named_collection, bool require_table) +{ + StoragePostgreSQL::Configuration configuration; + std::unordered_set required_arguments = {"user", "password", "database", "table"}; + if (require_table) + required_arguments.insert("table"); + validateNamedCollection( + named_collection, required_arguments, + {"schema", "on_conflict", "addresses_expr", "host", "port"}); + + configuration.addresses_expr = named_collection.getOrDefault("addresses_expr", ""); + if (configuration.addresses_expr.empty()) + { + configuration.host = named_collection.get("host"); + configuration.port = static_cast(named_collection.get("port")); + configuration.addresses = {std::make_pair(configuration.host, configuration.port)}; + } + + configuration.username = named_collection.get("user"); + configuration.password = named_collection.get("password"); + configuration.database = named_collection.get("database"); + if (require_table) + configuration.table = named_collection.get("table"); + configuration.schema = named_collection.getOrDefault("schema", ""); + configuration.on_conflict = named_collection.getOrDefault("on_conflict", ""); + + return configuration; +} StoragePostgreSQL::Configuration StoragePostgreSQL::getConfiguration(ASTs engine_args, ContextPtr context) { StoragePostgreSQL::Configuration configuration; if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args)) { - validateNamedCollection( - *named_collection, - {"user", "password", "database", "table"}, - {"schema", "on_conflict", "addresses_expr", "host", "port"}); - - configuration.addresses_expr = named_collection->getOrDefault("addresses_expr", ""); - if (configuration.addresses_expr.empty()) - { - configuration.host = named_collection->get("host"); - configuration.port = static_cast(named_collection->get("port")); - configuration.addresses = {std::make_pair(configuration.host, configuration.port)}; - } - - configuration.username = named_collection->get("user"); - configuration.password = named_collection->get("password"); - configuration.database = named_collection->get("database"); - configuration.table = named_collection->get("table"); - configuration.schema = named_collection->getOrDefault("schema", ""); - configuration.on_conflict = named_collection->getOrDefault("on_conflict", ""); + configuration = StoragePostgreSQL::processNamedCollectionResult(*named_collection); } else { diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index f3df24ebd35..b3ff342da10 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -5,7 +5,6 @@ #if USE_LIBPQXX #include #include -#include namespace Poco { @@ -20,6 +19,7 @@ using PoolWithFailoverPtr = std::shared_ptr; namespace DB { +class NamedCollection; class StoragePostgreSQL final : public IStorage { @@ -64,6 +64,8 @@ public: static Configuration getConfiguration(ASTs engine_args, ContextPtr context); + static Configuration processNamedCollectionResult(const NamedCollection & named_collection, bool require_table = true); + private: String remote_table_name; String remote_table_schema; diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index acf49f3cb71..d033747d0a2 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -8,7 +8,6 @@ #include #include #include -#include #include #include diff --git a/src/TableFunctions/TableFunctionMySQL.cpp b/src/TableFunctions/TableFunctionMySQL.cpp index 0cbad7bd9fd..1080f12021f 100644 --- a/src/TableFunctions/TableFunctionMySQL.cpp +++ b/src/TableFunctions/TableFunctionMySQL.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include diff --git a/src/TableFunctions/TableFunctionMySQL.h b/src/TableFunctions/TableFunctionMySQL.h index 794e8632ae2..5a230530bc4 100644 --- a/src/TableFunctions/TableFunctionMySQL.h +++ b/src/TableFunctions/TableFunctionMySQL.h @@ -3,7 +3,7 @@ #if USE_MYSQL #include -#include +#include #include @@ -30,7 +30,7 @@ private: void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; mutable std::optional pool; - std::optional configuration; + std::optional configuration; }; } From 68e06ecb991a4e84492618c4592b47abf037dcaa Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 21 Feb 2023 14:33:37 +0100 Subject: [PATCH 041/559] Replace for table function remote, and external storage --- .../ExternalDataSourceConfiguration.cpp | 114 ------- .../ExternalDataSourceConfiguration.h | 20 -- src/Storages/MeiliSearch/StorageMeiliSearch.h | 1 - src/Storages/StorageExternalDistributed.cpp | 306 +++++------------- src/Storages/StorageExternalDistributed.h | 24 +- src/Storages/StorageMongoDB.h | 1 - src/Storages/StorageURL.h | 10 +- src/TableFunctions/TableFunctionRemote.cpp | 74 ++--- 8 files changed, 110 insertions(+), 440 deletions(-) diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index 4882c644f74..b742391bad5 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -90,116 +90,6 @@ void ExternalDataSourceConfiguration::set(const ExternalDataSourceConfiguration } -template -std::optional getExternalDataSourceConfiguration( - const ASTs & args, ContextPtr context, bool is_database_engine, bool throw_on_no_collection, const BaseSettings & storage_settings) -{ - if (args.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); - - ExternalDataSourceConfiguration configuration; - StorageSpecificArgs non_common_args; - - if (const auto * collection = typeid_cast(args[0].get())) - { - const auto & config = context->getConfigRef(); - const auto & collection_prefix = fmt::format("named_collections.{}", collection->name()); - - if (!config.has(collection_prefix)) - { - /// For table function remote we do not throw on no collection, because then we consider first arg - /// as cluster definition from config. - if (!throw_on_no_collection) - return std::nullopt; - - throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no collection named `{}` in config", collection->name()); - } - - SettingsChanges config_settings = getSettingsChangesFromConfig(storage_settings, config, collection_prefix); - - configuration.host = config.getString(collection_prefix + ".host", ""); - configuration.port = config.getInt(collection_prefix + ".port", 0); - configuration.username = config.getString(collection_prefix + ".user", ""); - configuration.password = config.getString(collection_prefix + ".password", ""); - configuration.quota_key = config.getString(collection_prefix + ".quota_key", ""); - configuration.database = config.getString(collection_prefix + ".database", ""); - configuration.table = config.getString(collection_prefix + ".table", config.getString(collection_prefix + ".collection", "")); - configuration.schema = config.getString(collection_prefix + ".schema", ""); - configuration.addresses_expr = config.getString(collection_prefix + ".addresses_expr", ""); - - if (!configuration.addresses_expr.empty() && !configuration.host.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot have `addresses_expr` and `host`, `port` in configuration at the same time"); - - if ((args.size() == 1) && ((configuration.addresses_expr.empty() && (configuration.host.empty() || configuration.port == 0)) - || configuration.database.empty() || (configuration.table.empty() && !is_database_engine))) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Named collection of connection parameters is missing some " - "of the parameters and no key-value arguments are added"); - } - - /// Check key-value arguments. - for (size_t i = 1; i < args.size(); ++i) - { - if (const auto * ast_function = typeid_cast(args[i].get())) - { - const auto * args_expr = assert_cast(ast_function->arguments.get()); - auto function_args = args_expr->children; - if (function_args.size() != 2) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument"); - - auto arg_name = function_args[0]->as()->name(); - if (function_args[1]->as()) - { - non_common_args.emplace_back(std::make_pair(arg_name, function_args[1])); - continue; - } - - auto arg_value_ast = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[1], context); - auto * arg_value_literal = arg_value_ast->as(); - if (arg_value_literal) - { - auto arg_value = arg_value_literal->value; - - if (arg_name == "host") - configuration.host = arg_value.safeGet(); - else if (arg_name == "port") - configuration.port = arg_value.safeGet(); - else if (arg_name == "user") - configuration.username = arg_value.safeGet(); - else if (arg_name == "password") - configuration.password = arg_value.safeGet(); - else if (arg_name == "quota_key") - configuration.quota_key = arg_value.safeGet(); - else if (arg_name == "database") - configuration.database = arg_value.safeGet(); - else if (arg_name == "table") - configuration.table = arg_value.safeGet(); - else if (arg_name == "schema") - configuration.schema = arg_value.safeGet(); - else if (arg_name == "addresses_expr") - configuration.addresses_expr = arg_value.safeGet(); - else if (storage_settings.has(arg_name)) - config_settings.emplace_back(arg_name, arg_value); - else - non_common_args.emplace_back(std::make_pair(arg_name, arg_value_ast)); - } - else - { - non_common_args.emplace_back(std::make_pair(arg_name, arg_value_ast)); - } - } - else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument"); - } - } - - return ExternalDataSourceInfo{ .configuration = configuration, .specific_args = non_common_args, .settings_changes = config_settings }; - } - return std::nullopt; -} - static void validateConfigKeys( const Poco::Util::AbstractConfiguration & dict_config, const String & config_prefix, HasConfigKeyFunc has_config_key_func) { @@ -456,10 +346,6 @@ template bool getExternalDataSourceConfiguration(const ASTs & args, BaseSettings & settings, ContextPtr context); #endif -template -std::optional getExternalDataSourceConfiguration( - const ASTs & args, ContextPtr context, bool is_database_engine, bool throw_on_no_collection, const BaseSettings & storage_settings); - template std::optional getExternalDataSourceConfiguration( const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, diff --git a/src/Storages/ExternalDataSourceConfiguration.h b/src/Storages/ExternalDataSourceConfiguration.h index 7095f6b1b04..6eab7c2e072 100644 --- a/src/Storages/ExternalDataSourceConfiguration.h +++ b/src/Storages/ExternalDataSourceConfiguration.h @@ -34,12 +34,6 @@ struct ExternalDataSourceConfiguration }; -struct StoragePostgreSQLConfiguration : ExternalDataSourceConfiguration -{ - String on_conflict; -}; - - using StorageSpecificArgs = std::vector>; struct ExternalDataSourceInfo @@ -49,20 +43,6 @@ struct ExternalDataSourceInfo SettingsChanges settings_changes; }; -/* If there is a storage engine's configuration specified in the named_collections, - * this function returns valid for usage ExternalDataSourceConfiguration struct - * otherwise std::nullopt is returned. - * - * If any configuration options are provided as key-value engine arguments, they will override - * configuration values, i.e. ENGINE = PostgreSQL(postgresql_configuration, database = 'postgres_database'); - * - * Any key-value engine argument except common (`host`, `port`, `username`, `password`, `database`) - * is returned in EngineArgs struct. - */ -template -std::optional getExternalDataSourceConfiguration( - const ASTs & args, ContextPtr context, bool is_database_engine = false, bool throw_on_no_collection = true, const BaseSettings & storage_settings = {}); - using HasConfigKeyFunc = std::function; template diff --git a/src/Storages/MeiliSearch/StorageMeiliSearch.h b/src/Storages/MeiliSearch/StorageMeiliSearch.h index 5fa7ac2c0e3..30ff2f9b9fa 100644 --- a/src/Storages/MeiliSearch/StorageMeiliSearch.h +++ b/src/Storages/MeiliSearch/StorageMeiliSearch.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp index 9eeb619b899..e07a4554a60 100644 --- a/src/Storages/StorageExternalDistributed.cpp +++ b/src/Storages/StorageExternalDistributed.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include #include @@ -31,154 +32,20 @@ namespace ErrorCodes StorageExternalDistributed::StorageExternalDistributed( const StorageID & table_id_, - ExternalStorageEngine table_engine, - const String & cluster_description, - const ExternalDataSourceConfiguration & configuration, + std::unordered_set && shards_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - const String & comment, - ContextPtr context) + const String & comment) : IStorage(table_id_) + , shards(shards_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - - size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements; - std::vector shards_descriptions = parseRemoteDescription(cluster_description, 0, cluster_description.size(), ',', max_addresses); - std::vector> addresses; - -#if USE_MYSQL || USE_LIBPQXX - - /// For each shard pass replicas description into storage, replicas are managed by storage's PoolWithFailover. - for (const auto & shard_description : shards_descriptions) - { - StoragePtr shard; - - switch (table_engine) - { -#if USE_MYSQL - case ExternalStorageEngine::MySQL: - { - addresses = parseRemoteDescriptionForExternalDatabase(shard_description, max_addresses, 3306); - - mysqlxx::PoolWithFailover pool( - configuration.database, - addresses, - configuration.username, - configuration.password); - - shard = std::make_shared( - table_id_, - std::move(pool), - configuration.database, - configuration.table, - /* replace_query = */ false, - /* on_duplicate_clause = */ "", - columns_, - constraints_, - String{}, - context, - MySQLSettings{}); - break; - } -#endif -#if USE_LIBPQXX - - case ExternalStorageEngine::PostgreSQL: - { - addresses = parseRemoteDescriptionForExternalDatabase(shard_description, max_addresses, 5432); - StoragePostgreSQL::Configuration postgres_conf; - postgres_conf.addresses = addresses; - postgres_conf.username = configuration.username; - postgres_conf.password = configuration.password; - postgres_conf.database = configuration.database; - postgres_conf.table = configuration.table; - postgres_conf.schema = configuration.schema; - - const auto & settings = context->getSettingsRef(); - auto pool = std::make_shared( - postgres_conf, - settings.postgresql_connection_pool_size, - settings.postgresql_connection_pool_wait_timeout, - POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, - settings.postgresql_connection_pool_auto_close_connection); - - shard = std::make_shared(table_id_, std::move(pool), configuration.table, columns_, constraints_, String{}); - break; - } -#endif - default: - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Unsupported table engine. Supported engines are: MySQL, PostgreSQL, URL"); - } - } - - shards.emplace(std::move(shard)); - } - -#else - (void)configuration; - (void)cluster_description; - (void)addresses; - (void)table_engine; -#endif } - -StorageExternalDistributed::StorageExternalDistributed( - const String & addresses_description, - const StorageID & table_id, - const String & format_name, - const std::optional & format_settings, - const String & compression_method, - const ColumnsDescription & columns, - const ConstraintsDescription & constraints, - ContextPtr context) - : IStorage(table_id) -{ - StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns); - storage_metadata.setConstraints(constraints); - setInMemoryMetadata(storage_metadata); - - size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements; - /// Generate addresses without splitting for failover options - std::vector url_descriptions = parseRemoteDescription(addresses_description, 0, addresses_description.size(), ',', max_addresses); - std::vector uri_options; - - for (const auto & url_description : url_descriptions) - { - /// For each uri (which acts like shard) check if it has failover options - uri_options = parseRemoteDescription(url_description, 0, url_description.size(), '|', max_addresses); - StoragePtr shard; - - if (uri_options.size() > 1) - { - shard = std::make_shared( - uri_options, - table_id, - format_name, - format_settings, - columns, constraints, context, - compression_method); - } - else - { - shard = std::make_shared( - url_description, table_id, format_name, format_settings, columns, constraints, String{}, context, compression_method); - - LOG_DEBUG(&Poco::Logger::get("StorageURLDistributed"), "Adding URL: {}", url_description); - } - - shards.emplace(std::move(shard)); - } -} - - void StorageExternalDistributed::read( QueryPlan & query_plan, const Names & column_names, @@ -226,7 +93,6 @@ void StorageExternalDistributed::read( query_plan.unitePlans(std::move(union_step), std::move(plans)); } - void registerStorageExternalDistributed(StorageFactory & factory) { factory.registerStorage("ExternalDistributed", [](const StorageFactory::Arguments & args) @@ -237,102 +103,94 @@ void registerStorageExternalDistributed(StorageFactory & factory) "Engine ExternalDistributed must have at least 2 arguments: " "engine_name, named_collection and/or description"); - auto engine_name = checkAndGetLiteralArgument(engine_args[0], "engine_name"); - StorageExternalDistributed::ExternalStorageEngine table_engine; - if (engine_name == "URL") - table_engine = StorageExternalDistributed::ExternalStorageEngine::URL; - else if (engine_name == "MySQL") - table_engine = StorageExternalDistributed::ExternalStorageEngine::MySQL; - else if (engine_name == "PostgreSQL") - table_engine = StorageExternalDistributed::ExternalStorageEngine::PostgreSQL; - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "External storage engine {} is not supported for StorageExternalDistributed. " - "Supported engines are: MySQL, PostgreSQL, URL", - engine_name); + auto context = args.getLocalContext(); + const auto & settings = context->getSettingsRef(); + size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements; + auto get_addresses = [&](const std::string addresses_expr) + { + return parseRemoteDescription(addresses_expr, 0, addresses_expr.size(), ',', max_addresses); + }; + std::unordered_set shards; ASTs inner_engine_args(engine_args.begin() + 1, engine_args.end()); - String cluster_description; + auto engine_name = checkAndGetLiteralArgument(engine_args[0], "engine_name"); if (engine_name == "URL") { - StorageURL::Configuration configuration; - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args)) - { - StorageURL::processNamedCollectionResult(configuration, *named_collection); - StorageURL::collectHeaders(engine_args, configuration.headers, args.getLocalContext()); - } - else - { - for (auto & engine_arg : engine_args) - engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getLocalContext()); - - cluster_description = checkAndGetLiteralArgument(engine_args[1], "cluster_description"); - configuration.format = checkAndGetLiteralArgument(engine_args[2], "format"); - configuration.compression_method = "auto"; - if (engine_args.size() == 4) - configuration.compression_method = checkAndGetLiteralArgument(engine_args[3], "compression_method"); - } - - + auto configuration = StorageURL::getConfiguration(inner_engine_args, context); + auto shards_addresses = get_addresses(configuration.addresses_expr); auto format_settings = StorageURL::getFormatSettingsFromArgs(args); - - return std::make_shared( - cluster_description, - args.table_id, - configuration.format, - format_settings, - configuration.compression_method, - args.columns, - args.constraints, - args.getContext()); + for (const auto & shard_address : shards_addresses) + { + auto uri_options = parseRemoteDescription(shard_address, 0, shard_address.size(), '|', max_addresses); + if (uri_options.size() > 1) + { + shards.insert( + std::make_shared( + uri_options, args.table_id, configuration.format, format_settings, + args.columns, args.constraints, context, configuration.compression_method)); + } + else + { + shards.insert(std::make_shared( + shard_address, args.table_id, configuration.format, format_settings, + args.columns, args.constraints, String{}, context, configuration.compression_method)); + } + } } +#if USE_MYSQL + else if (engine_name == "MySQL") + { + MySQLSettings mysql_settings; + auto configuration = StorageMySQL::getConfiguration(inner_engine_args, context, mysql_settings); + auto shards_addresses = get_addresses(configuration.addresses_expr); + for (const auto & shard_address : shards_addresses) + { + auto current_configuration{configuration}; + current_configuration.addresses = parseRemoteDescriptionForExternalDatabase(shard_address, max_addresses, 3306); + auto pool = createMySQLPoolWithFailover(configuration, mysql_settings); + shards.insert(std::make_shared( + args.table_id, std::move(pool), configuration.database, configuration.table, + /* replace_query = */ false, /* on_duplicate_clause = */ "", + args.columns, args.constraints, String{}, context, mysql_settings)); + } + } +#endif +#if USE_LIBPQXX + else if (engine_name == "PostgreSQL") + { + auto configuration = StoragePostgreSQL::getConfiguration(inner_engine_args, context); + auto shards_addresses = get_addresses(configuration.addresses_expr); + for (const auto & shard_address : shards_addresses) + { + auto current_configuration{configuration}; + current_configuration.addresses = parseRemoteDescriptionForExternalDatabase(shard_address, max_addresses, 5432); + auto pool = std::make_shared( + current_configuration, + settings.postgresql_connection_pool_size, + settings.postgresql_connection_pool_wait_timeout, + POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, + settings.postgresql_connection_pool_auto_close_connection); + shards.insert(std::make_shared( + args.table_id, std::move(pool), configuration.table, args.columns, args.constraints, String{})); + } + } +#endif else { - ExternalDataSourceConfiguration configuration; - if (auto named_collection = getExternalDataSourceConfiguration(inner_engine_args, args.getLocalContext())) - { - auto [common_configuration, storage_specific_args, _] = named_collection.value(); - configuration.set(common_configuration); - - for (const auto & [name, value] : storage_specific_args) - { - if (name == "description") - cluster_description = checkAndGetLiteralArgument(value, "cluster_description"); - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Unknown key-value argument {} for table function URL", name); - } - - if (cluster_description.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Engine ExternalDistribued must have `description` key-value argument or named collection parameter"); - } - else - { - if (engine_args.size() != 6) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage ExternalDistributed requires 5 parameters: " - "ExternalDistributed('engine_name', 'cluster_description', 'database', 'table', 'user', 'password')."); - - cluster_description = checkAndGetLiteralArgument(engine_args[1], "cluster_description"); - configuration.database = checkAndGetLiteralArgument(engine_args[2], "database"); - configuration.table = checkAndGetLiteralArgument(engine_args[3], "table"); - configuration.username = checkAndGetLiteralArgument(engine_args[4], "username"); - configuration.password = checkAndGetLiteralArgument(engine_args[5], "password"); - } - - - return std::make_shared( - args.table_id, - table_engine, - cluster_description, - configuration, - args.columns, - args.constraints, - args.comment, - args.getContext()); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "External storage engine {} is not supported for StorageExternalDistributed. " + "Supported engines are: MySQL, PostgreSQL, URL", + engine_name); } + + return std::make_shared( + args.table_id, + std::move(shards), + args.columns, + args.constraints, + args.comment); }, { .source_access_type = AccessType::SOURCES, diff --git a/src/Storages/StorageExternalDistributed.h b/src/Storages/StorageExternalDistributed.h index a1bdb41dded..c4d37c3e5cc 100644 --- a/src/Storages/StorageExternalDistributed.h +++ b/src/Storages/StorageExternalDistributed.h @@ -18,32 +18,12 @@ struct ExternalDataSourceConfiguration; class StorageExternalDistributed final : public DB::IStorage { public: - enum class ExternalStorageEngine - { - MySQL, - PostgreSQL, - URL - }; - StorageExternalDistributed( const StorageID & table_id_, - ExternalStorageEngine table_engine, - const String & cluster_description, - const ExternalDataSourceConfiguration & configuration, + std::unordered_set && shards_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - const String & comment, - ContextPtr context_); - - StorageExternalDistributed( - const String & addresses_description, - const StorageID & table_id, - const String & format_name, - const std::optional & format_settings, - const String & compression_method, - const ColumnsDescription & columns, - const ConstraintsDescription & constraints, - ContextPtr context); + const String & comment); std::string getName() const override { return "ExternalDistributed"; } diff --git a/src/Storages/StorageMongoDB.h b/src/Storages/StorageMongoDB.h index 941e9fcf4b0..682a027440d 100644 --- a/src/Storages/StorageMongoDB.h +++ b/src/Storages/StorageMongoDB.h @@ -3,7 +3,6 @@ #include #include -#include namespace DB { diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index d033747d0a2..24b1c7ee572 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -184,8 +184,9 @@ public: struct Configuration : public StatelessTableEngineConfiguration { std::string url; - std::string http_method; + std::string http_method = "auto"; HTTPHeaderEntries headers; + std::string addresses_expr; }; static Configuration getConfiguration(ASTs & args, ContextPtr context); @@ -219,13 +220,6 @@ public: size_t max_block_size, size_t num_streams) override; - struct Configuration - { - String url; - String compression_method = "auto"; - std::vector> headers; - }; - private: std::vector uri_options; }; diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 1877c9fe65b..dbc718536f2 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -2,8 +2,8 @@ #include #include -#include #include +#include #include #include #include @@ -34,10 +34,10 @@ namespace ErrorCodes void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr context) { ASTs & args_func = ast_function->children; - ExternalDataSourceConfiguration configuration; String cluster_name; String cluster_description; + String database, table, username, password; if (args_func.size() != 1) throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); @@ -50,47 +50,21 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr * For now named collection can be used only for remote as cluster does not require credentials. */ size_t max_args = is_cluster_function ? 4 : 6; - auto named_collection = getExternalDataSourceConfiguration(args, context, false, false); - if (named_collection) + if (auto named_collection = tryGetNamedCollectionWithOverrides(args)) { if (is_cluster_function) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Named collection cannot be used for table function cluster"); - /** - * Common arguments: database, table, username, password, addresses_expr. - * Specific args (remote): sharding_key, or database (in case it is not ASTLiteral). - * None of the common arguments is empty at this point, it is checked in getExternalDataSourceConfiguration. - */ - auto [common_configuration, storage_specific_args, _] = named_collection.value(); - configuration.set(common_configuration); + validateNamedCollection( + *named_collection, + {"addresses_expr", "database", "table"}, + {"username", "password", "sharding_key"}); - for (const auto & [arg_name, arg_value] : storage_specific_args) - { - if (arg_name == "sharding_key") - { - sharding_key = arg_value; - } - else if (arg_name == "database") - { - const auto * function = arg_value->as(); - if (function && TableFunctionFactory::instance().isTableFunctionName(function->name)) - { - remote_table_function_ptr = arg_value; - } - else - { - auto database_literal = evaluateConstantExpressionOrIdentifierAsLiteral(arg_value, context); - configuration.database = checkAndGetLiteralArgument(database_literal, "database"); - } - } - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Unexpected key-value argument." - "Got: {}, but expected: sharding_key", arg_name); - } - cluster_description = configuration.addresses_expr; - if (cluster_description.empty()) - cluster_description = configuration.port ? configuration.host + ':' + toString(configuration.port) : configuration.host; + cluster_description = named_collection->getOrDefault("addresses_expr", ""); + database = named_collection->get("database"); + table = named_collection->get("table"); + username = named_collection->getOrDefault("username", ""); + password = named_collection->getOrDefault("password", ""); } else { @@ -159,11 +133,11 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr else { args[arg_num] = evaluateConstantExpressionForDatabaseName(args[arg_num], context); - configuration.database = checkAndGetLiteralArgument(args[arg_num], "database"); + database = checkAndGetLiteralArgument(args[arg_num], "database"); ++arg_num; - auto qualified_name = QualifiedTableName::parseFromString(configuration.database); + auto qualified_name = QualifiedTableName::parseFromString(database); if (qualified_name.database.empty()) { if (arg_num >= args.size()) @@ -179,8 +153,8 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr } } - configuration.database = std::move(qualified_name.database); - configuration.table = std::move(qualified_name.table); + database = std::move(qualified_name.database); + table = std::move(qualified_name.table); /// Cluster function may have sharding key for insert if (is_cluster_function && arg_num < args.size()) @@ -195,9 +169,9 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr { if (arg_num < args.size()) { - if (!get_string_literal(*args[arg_num], configuration.username)) + if (!get_string_literal(*args[arg_num], username)) { - configuration.username = "default"; + username = "default"; sharding_key = args[arg_num]; } ++arg_num; @@ -205,7 +179,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr if (arg_num < args.size() && !sharding_key) { - if (!get_string_literal(*args[arg_num], configuration.password)) + if (!get_string_literal(*args[arg_num], password)) { sharding_key = args[arg_num]; } @@ -267,19 +241,19 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr cluster = std::make_shared( context->getSettingsRef(), names, - configuration.username, - configuration.password, + username, + password, (secure ? (maybe_secure_port ? *maybe_secure_port : DBMS_DEFAULT_SECURE_PORT) : context->getTCPPort()), treat_local_as_remote, treat_local_port_as_remote, secure); } - if (!remote_table_function_ptr && configuration.table.empty()) + if (!remote_table_function_ptr && table.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The name of remote table cannot be empty"); - remote_table_id.database_name = configuration.database; - remote_table_id.table_name = configuration.table; + remote_table_id.database_name = database; + remote_table_id.table_name = table; } StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const From b3ca976f5f4b51001ed7e4035f4fc4f1db181903 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 21 Feb 2023 14:50:55 +0100 Subject: [PATCH 042/559] Finish for streaming engines --- .../ExternalDataSourceConfiguration.cpp | 58 ------------------- .../ExternalDataSourceConfiguration.h | 4 -- src/Storages/Kafka/StorageKafka.cpp | 26 ++++++--- src/Storages/NATS/StorageNATS.cpp | 14 ++++- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 15 ++++- 5 files changed, 40 insertions(+), 77 deletions(-) diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index b742391bad5..28bd058b802 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -288,64 +288,6 @@ void URLBasedDataSourceConfiguration::set(const URLBasedDataSourceConfiguration headers = conf.headers; } -template -bool getExternalDataSourceConfiguration(const ASTs & args, BaseSettings & settings, ContextPtr context) -{ - if (args.empty()) - return false; - - if (const auto * collection = typeid_cast(args[0].get())) - { - const auto & config = context->getConfigRef(); - const auto & config_prefix = fmt::format("named_collections.{}", collection->name()); - - if (!config.has(config_prefix)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no collection named `{}` in config", collection->name()); - - auto config_settings = getSettingsChangesFromConfig(settings, config, config_prefix); - - /// Check key-value arguments. - for (size_t i = 1; i < args.size(); ++i) - { - if (const auto * ast_function = typeid_cast(args[i].get())) - { - const auto * args_expr = assert_cast(ast_function->arguments.get()); - auto function_args = args_expr->children; - if (function_args.size() != 2) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument"); - - auto arg_name = function_args[0]->as()->name(); - auto arg_value_ast = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[1], context); - auto arg_value = arg_value_ast->as()->value; - config_settings.emplace_back(arg_name, arg_value); - } - else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument"); - } - } - - settings.applyChanges(config_settings); - return true; - } - return false; -} - -#if USE_AMQPCPP -template -bool getExternalDataSourceConfiguration(const ASTs & args, BaseSettings & settings, ContextPtr context); -#endif - -#if USE_RDKAFKA -template -bool getExternalDataSourceConfiguration(const ASTs & args, BaseSettings & settings, ContextPtr context); -#endif - -#if USE_NATSIO -template -bool getExternalDataSourceConfiguration(const ASTs & args, BaseSettings & settings, ContextPtr context); -#endif - template std::optional getExternalDataSourceConfiguration( const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, diff --git a/src/Storages/ExternalDataSourceConfiguration.h b/src/Storages/ExternalDataSourceConfiguration.h index 6eab7c2e072..b825548debe 100644 --- a/src/Storages/ExternalDataSourceConfiguration.h +++ b/src/Storages/ExternalDataSourceConfiguration.h @@ -65,7 +65,6 @@ struct ExternalDataSourcesByPriority ExternalDataSourcesByPriority getExternalDataSourceConfigurationByPriority(const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context, HasConfigKeyFunc has_config_key); - struct URLBasedDataSourceConfiguration { String url; @@ -92,7 +91,4 @@ struct URLBasedDataSourceConfig std::optional getURLBasedDataSourceConfiguration( const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context); -template -bool getExternalDataSourceConfiguration(const ASTs & args, BaseSettings & settings, ContextPtr context); - } diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 50fb7dffa34..cfb066ee1b8 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -19,13 +19,13 @@ #include #include #include -#include #include #include #include #include #include #include +#include #include #include #include @@ -757,10 +757,23 @@ void registerStorageKafka(StorageFactory & factory) { ASTs & engine_args = args.engine_args; size_t args_count = engine_args.size(); - bool has_settings = args.storage_def->settings; + const bool has_settings = args.storage_def->settings; auto kafka_settings = std::make_unique(); - auto named_collection = getExternalDataSourceConfiguration(args.engine_args, *kafka_settings, args.getLocalContext()); + String collection_name; + if (auto named_collection = tryGetNamedCollectionWithOverrides(args.engine_args)) + { + for (const auto & setting : kafka_settings->all()) + { + const auto & setting_name = setting.getName(); + if (named_collection->has(setting_name)) + kafka_settings->set(setting_name, named_collection->get(setting_name)); + } + collection_name = assert_cast(args.engine_args[0].get())->name(); + } + else if (!has_settings) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Kafka engine must have settings"); + if (has_settings) { kafka_settings->loadFromQuery(*args.storage_def); @@ -824,12 +837,7 @@ void registerStorageKafka(StorageFactory & factory) * - Do intermediate commits when the batch consumed and handled */ - String collection_name; - if (named_collection) - { - collection_name = assert_cast(args.engine_args[0].get())->name(); - } - else + if (has_settings) { /* 0 = raw, 1 = evaluateConstantExpressionAsLiteral, 2=evaluateConstantExpressionOrIdentifierAsLiteral */ CHECK_KAFKA_STORAGE_ARGUMENT(1, kafka_broker_list, 0) diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index f1724b8c14c..feb44fe92e4 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -10,13 +10,13 @@ #include #include #include -#include #include #include #include #include #include #include +#include #include #include #include @@ -711,8 +711,16 @@ void registerStorageNATS(StorageFactory & factory) auto creator_fn = [](const StorageFactory::Arguments & args) { auto nats_settings = std::make_unique(); - bool with_named_collection = getExternalDataSourceConfiguration(args.engine_args, *nats_settings, args.getLocalContext()); - if (!with_named_collection && !args.storage_def->settings) + if (auto named_collection = tryGetNamedCollectionWithOverrides(args.engine_args)) + { + for (const auto & setting : nats_settings->all()) + { + const auto & setting_name = setting.getName(); + if (named_collection->has(setting_name)) + nats_settings->set(setting_name, named_collection->get(setting_name)); + } + } + else if (!args.storage_def->settings) throw Exception(ErrorCodes::BAD_ARGUMENTS, "NATS engine must have settings"); nats_settings->loadFromQuery(*args.storage_def); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index f10a60419d1..c5ea6f810ef 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -18,7 +18,7 @@ #include #include #include -#include +#include #include #include #include @@ -1194,8 +1194,17 @@ void registerStorageRabbitMQ(StorageFactory & factory) auto creator_fn = [](const StorageFactory::Arguments & args) { auto rabbitmq_settings = std::make_unique(); - bool with_named_collection = getExternalDataSourceConfiguration(args.engine_args, *rabbitmq_settings, args.getLocalContext()); - if (!with_named_collection && !args.storage_def->settings) + + if (auto named_collection = tryGetNamedCollectionWithOverrides(args.engine_args)) + { + for (const auto & setting : rabbitmq_settings->all()) + { + const auto & setting_name = setting.getName(); + if (named_collection->has(setting_name)) + rabbitmq_settings->set(setting_name, named_collection->get(setting_name)); + } + } + else if (!args.storage_def->settings) throw Exception(ErrorCodes::BAD_ARGUMENTS, "RabbitMQ engine must have settings"); if (args.storage_def->settings) From d84215d005195f787a493adf6c7093f062fe6b4a Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 21 Feb 2023 16:20:57 +0100 Subject: [PATCH 043/559] Fix style check --- src/Storages/StorageExternalDistributed.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp index e07a4554a60..a5c22d0807d 100644 --- a/src/Storages/StorageExternalDistributed.cpp +++ b/src/Storages/StorageExternalDistributed.cpp @@ -14,7 +14,6 @@ #include #include #include -#include #include #include #include @@ -26,7 +25,6 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; } @@ -104,7 +102,7 @@ void registerStorageExternalDistributed(StorageFactory & factory) "engine_name, named_collection and/or description"); auto context = args.getLocalContext(); - const auto & settings = context->getSettingsRef(); + [[maybe_unused]] const auto & settings = context->getSettingsRef(); size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements; auto get_addresses = [&](const std::string addresses_expr) { From c5f93eb1081de7dfd026ca5ca57771f118cfaef9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 21 Feb 2023 15:44:50 +0000 Subject: [PATCH 044/559] Fix more tests. --- src/Interpreters/Aggregator.cpp | 7 +++++++ .../QueryPlan/Optimizations/optimizeReadInOrder.cpp | 4 ++++ src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../0_stateless/02515_projections_with_totals.reference | 3 +++ .../queries/0_stateless/02515_projections_with_totals.sql | 4 ++++ 5 files changed, 19 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 557aac2c89a..3d07e40cb5a 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -975,6 +975,13 @@ void Aggregator::mergeOnBlockSmall( result.key_sizes = key_sizes; } + if ((params.overflow_row || result.type == AggregatedDataVariants::Type::without_key) && !result.without_key) + { + AggregateDataPtr place = result.aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); + createAggregateStates(place); + result.without_key = place; + } + if (false) {} // NOLINT #define M(NAME, IS_TWO_LEVEL) \ else if (result.type == AggregatedDataVariants::Type::NAME) \ diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 8c81372b89e..2c51950d567 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -981,6 +981,10 @@ void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &) if ((aggregating->inOrder() && !aggregating->explicitSortingRequired()) || aggregating->isGroupingSets()) return; + /// It just does not work, see 02515_projections_with_totals + if (aggregating->getParams().overflow_row) + return; + /// TODO: maybe add support for UNION later. std::vector steps_to_update; if (auto order_info = buildInputOrderInfo(*aggregating, *node.children.front(), steps_to_update); order_info.input_order) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 48721caa0b1..a1cfdd7ecd2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6207,7 +6207,7 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( agg_count->set(place, value.get()); else { - auto value_column = func->getResultType()->createColumnConst(1, value)->convertToFullColumnIfConst(); + auto value_column = func->getArgumentTypes().front()->createColumnConst(1, value)->convertToFullColumnIfConst(); const auto * value_column_ptr = value_column.get(); func->add(place, &value_column_ptr, 0, &arena); } diff --git a/tests/queries/0_stateless/02515_projections_with_totals.reference b/tests/queries/0_stateless/02515_projections_with_totals.reference index c6359cae032..f086fcb10e9 100644 --- a/tests/queries/0_stateless/02515_projections_with_totals.reference +++ b/tests/queries/0_stateless/02515_projections_with_totals.reference @@ -1,3 +1,6 @@ 0 0 +0 + +0 diff --git a/tests/queries/0_stateless/02515_projections_with_totals.sql b/tests/queries/0_stateless/02515_projections_with_totals.sql index 4d43d5381da..1e4b5c6f255 100644 --- a/tests/queries/0_stateless/02515_projections_with_totals.sql +++ b/tests/queries/0_stateless/02515_projections_with_totals.sql @@ -3,4 +3,8 @@ CREATE TABLE t (x UInt8, PROJECTION p (SELECT x GROUP BY x)) ENGINE = MergeTree INSERT INTO t VALUES (0); SET group_by_overflow_mode = 'any', max_rows_to_group_by = 1000, totals_mode = 'after_having_auto'; SELECT x FROM t GROUP BY x WITH TOTALS; + +SET optimize_aggregation_in_order=1; +SELECT x FROM t GROUP BY x WITH TOTALS; + DROP TABLE t; From b15be14792437e6298124fc96e15ab4a40b5d5b4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 21 Feb 2023 18:48:54 +0100 Subject: [PATCH 045/559] Fix table function remote --- src/Storages/NamedCollectionsHelpers.cpp | 10 ++++++---- src/Storages/NamedCollectionsHelpers.h | 2 +- src/TableFunctions/TableFunctionRemote.cpp | 2 +- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Storages/NamedCollectionsHelpers.cpp b/src/Storages/NamedCollectionsHelpers.cpp index 0c7426bb682..648707da027 100644 --- a/src/Storages/NamedCollectionsHelpers.cpp +++ b/src/Storages/NamedCollectionsHelpers.cpp @@ -15,7 +15,7 @@ namespace ErrorCodes namespace { - NamedCollectionPtr tryGetNamedCollectionFromASTs(ASTs asts) + NamedCollectionPtr tryGetNamedCollectionFromASTs(ASTs asts, bool throw_unknown_collection) { if (asts.empty()) return nullptr; @@ -25,7 +25,9 @@ namespace return nullptr; const auto & collection_name = identifier->name(); - return NamedCollectionFactory::instance().get(collection_name); + if (throw_unknown_collection) + return NamedCollectionFactory::instance().get(collection_name); + return NamedCollectionFactory::instance().tryGet(collection_name); } std::optional> getKeyValueFromAST(ASTPtr ast) @@ -53,12 +55,12 @@ namespace } -MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts) +MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts, bool throw_unknown_collection) { if (asts.empty()) return nullptr; - auto collection = tryGetNamedCollectionFromASTs(asts); + auto collection = tryGetNamedCollectionFromASTs(asts, throw_unknown_collection); if (!collection) return nullptr; diff --git a/src/Storages/NamedCollectionsHelpers.h b/src/Storages/NamedCollectionsHelpers.h index 5f8f316d023..6e80b360411 100644 --- a/src/Storages/NamedCollectionsHelpers.h +++ b/src/Storages/NamedCollectionsHelpers.h @@ -18,7 +18,7 @@ namespace DB /// Helper function to get named collection for table engine. /// Table engines have collection name as first argument of ast and other arguments are key-value overrides. -MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts); +MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts, bool throw_unknown_collection = true); /// Helper function to get named collection for dictionary source. /// Dictionaries have collection name as name argument of dict configuration and other arguments are overrides. MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index dbc718536f2..79f052854ec 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -50,7 +50,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr * For now named collection can be used only for remote as cluster does not require credentials. */ size_t max_args = is_cluster_function ? 4 : 6; - if (auto named_collection = tryGetNamedCollectionWithOverrides(args)) + if (auto named_collection = tryGetNamedCollectionWithOverrides(args, false)) { if (is_cluster_function) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Named collection cannot be used for table function cluster"); From a47860af17be242fbfd492c1debaa3bba852ad91 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 21 Feb 2023 19:50:17 +0000 Subject: [PATCH 046/559] Fixing more tests. --- src/Dictionaries/HashedDictionary.cpp | 24 +++---- src/Dictionaries/HashedDictionary.h | 4 +- src/Interpreters/ActionsDAG.h | 22 +++++- src/Processors/QueryPlan/AggregatingStep.cpp | 26 ++----- src/Processors/QueryPlan/AggregatingStep.h | 5 ++ .../QueryPlan/Optimizations/optimizeTree.cpp | 9 ++- .../Optimizations/optimizeUseProjections.cpp | 69 ++++++++++++++++--- .../01710_normal_projection_fix1.sql | 2 + .../00172_early_constant_folding.reference | 14 ++-- 9 files changed, 123 insertions(+), 52 deletions(-) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index b1dbfaee3c8..309a3dda2e4 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -396,8 +396,8 @@ ColumnUInt8::Ptr HashedDictionary::hasKeys extractor.rollbackCurrentKey(); } - query_count.fetch_add(keys_size); - found_count.fetch_add(keys_found); + query_count.fetch_add(keys_size, std::memory_order_relaxed); + found_count.fetch_add(keys_found, std::memory_order_relaxed); return result; } @@ -422,8 +422,8 @@ ColumnUInt8::Ptr HashedDictionary::hasKeys } }); - query_count.fetch_add(keys_size); - found_count.fetch_add(keys_found); + query_count.fetch_add(keys_size, std::memory_order_relaxed); + found_count.fetch_add(keys_found, std::memory_order_relaxed); return result; } @@ -482,8 +482,8 @@ ColumnPtr HashedDictionary::getHierarchy(C auto dictionary_hierarchy_array = getKeysHierarchyArray(keys, is_key_valid_func, get_parent_func); - query_count.fetch_add(keys.size()); - found_count.fetch_add(keys_found); + query_count.fetch_add(keys.size(), std::memory_order_relaxed); + found_count.fetch_add(keys_found, std::memory_order_relaxed); return dictionary_hierarchy_array; } @@ -556,8 +556,8 @@ ColumnUInt8::Ptr HashedDictionary::isInHie auto result = getKeysIsInHierarchyColumn(keys, keys_in, is_key_valid_func, get_parent_key_func); - query_count.fetch_add(keys.size()); - found_count.fetch_add(keys_found); + query_count.fetch_add(keys.size(), std::memory_order_relaxed); + found_count.fetch_add(keys_found, std::memory_order_relaxed); return result; } @@ -615,8 +615,8 @@ ColumnPtr HashedDictionary::getDescendants size_t keys_found; auto result = getKeysDescendantsArray(keys, *parent_to_child_index, level, keys_found); - query_count.fetch_add(keys.size()); - found_count.fetch_add(keys_found); + query_count.fetch_add(keys.size(), std::memory_order_relaxed); + found_count.fetch_add(keys_found, std::memory_order_relaxed); return result; } @@ -882,8 +882,8 @@ void HashedDictionary::getItemsImpl( keys_extractor.rollbackCurrentKey(); } - query_count.fetch_add(keys_size); - found_count.fetch_add(keys_found); + query_count.fetch_add(keys_size, std::memory_order_relaxed); + found_count.fetch_add(keys_found, std::memory_order_relaxed); } template diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 1e9b5572204..67683679647 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -69,10 +69,10 @@ public: double getFoundRate() const override { - size_t queries = query_count.load(); + size_t queries = query_count.load(std::memory_order_relaxed); if (!queries) return 0; - return static_cast(found_count.load()) / queries; + return static_cast(found_count.load(std::memory_order_relaxed)) / queries; } double getHitRate() const override { return 1.0; } diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 9a78e514b9d..66549488752 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -215,7 +215,27 @@ public: const String & predicate_column_name = {}, bool add_missing_keys = true); - static ActionsDAGPtr foldActionsByProjection(const std::unordered_map & new_inputs, const NodeRawConstPtrs & required_outputs); + /// Get an ActionsDAG where: + /// * Subtrees from new_inputs are converted to inputs with specified names. + /// * Outputs are taken from required_outputs. + /// Here want to substitute some expressions to columns from projection. + /// This function expects that all required_outputs can be calculated from nodes in new_inputs. + /// If not, excpetion will happen. + /// This function also expects that new_inputs and required_outputs are valid nodes from the same DAG. + /// Example: + /// DAG: new_inputs: Result DAG + /// a b c * d -> "(a + b) * d" + /// \ / e -> "" + /// a + b + /// \ required_outputs: => "(a + b) * d" e + /// c (alias) d c * d - e \ / + /// \ / c * d - e + /// c * d e + /// \ / + /// c * d - e + static ActionsDAGPtr foldActionsByProjection( + const std::unordered_map & new_inputs, + const NodeRawConstPtrs & required_outputs); /// Reorder the output nodes using given position mapping. void reorderAggregationKeysForProjection(const std::unordered_map & key_names_pos_map); diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index b092b718ad5..ddf08d8a124 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -516,8 +516,10 @@ void AggregatingStep::describePipeline(FormatSettings & settings) const bool AggregatingStep::canUseProjection() const { - //LOG_TRACE(&Poco::Logger::get("AggregatingStep"), "canUseProjection {} {} {}", grouping_sets_params.size(), sort_description_for_merging.size(), memory_bound_merging_of_aggregation_results_enabled); - return grouping_sets_params.empty() && sort_description_for_merging.empty(); // && !memory_bound_merging_of_aggregation_results_enabled; + /// For now, grouping sets are not supported. + /// Aggregation in order should be applied after projection optimization if projection is full. + /// Skip it here just in case. + return grouping_sets_params.empty() && sort_description_for_merging.empty(); } void AggregatingStep::requestOnlyMergeForAggregateProjection(const DataStream & input_stream) @@ -541,13 +543,8 @@ std::unique_ptr AggregatingStep::convertToAggregating DataStreams{input_streams.front(), input_stream}, params, final, - //max_block_size, merge_threads, temporary_data_merge_threads - //group_by_use_nulls, - //group_by_sort_description, - //should_produce_results_in_order_of_bucket_number - //memory_bound_merging_of_aggregation_results_enabled, ); assertBlocksHaveEqualStructure(getOutputStream().header, aggregating_projection->getOutputStream().header, "AggregatingStep"); @@ -572,23 +569,12 @@ AggregatingProjectionStep::AggregatingProjectionStep( DataStreams input_streams_, Aggregator::Params params_, bool final_, - //size_t max_block_size_, size_t merge_threads_, - size_t temporary_data_merge_threads_ - //bool group_by_use_nulls_, - //SortDescription group_by_sort_description_, - //bool should_produce_results_in_order_of_bucket_number_ - //bool memory_bound_merging_of_aggregation_results_enabled_ - ) + size_t temporary_data_merge_threads_) : params(std::move(params_)) , final(final_) - //, max_block_size(max_block_size_) , merge_threads(merge_threads_) , temporary_data_merge_threads(temporary_data_merge_threads_) - //, group_by_use_nulls(group_by_use_nulls_) - //, group_by_sort_description(std::move(group_by_sort_description_)) - //, should_produce_results_in_order_of_bucket_number(should_produce_results_in_order_of_bucket_number_) - //, memory_bound_merging_of_aggregation_results_enabled(memory_bound_merging_of_aggregation_results_enabled_) { input_streams = std::move(input_streams_); @@ -631,7 +617,7 @@ QueryPipelineBuilderPtr AggregatingProjectionStep::updatePipeline( AggregatorListPtr aggregator_list_ptr = std::make_shared(); - /// TODO apply optimize_aggregation_in_order here too (like below) + /// TODO apply optimize_aggregation_in_order here somehow auto build_aggregate_pipeline = [&](QueryPipelineBuilder & pipeline, bool projection) { auto params_copy = params; diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index b728fd8874f..f59718db522 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -66,7 +66,12 @@ public: void skipMerging() { skip_merging = true; } bool canUseProjection() const; + /// When we apply aggregate projection (whihc is full), this step will only merge data. + /// Agrument input_stream replaces current single input. + /// Probably we should replace this step to MergingAggregated later? (now, aggregation-in-order will not work) void requestOnlyMergeForAggregateProjection(const DataStream & input_stream); + /// When we apply aggregate projection (whihc is partial), this step should be replaced to AggregatingProjection. + /// Agrument input_stream would be the second input (from projection). std::unique_ptr convertToAggregatingProjection(const DataStream & input_stream) const; private: diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index ac9e778cc76..121cca610f1 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -105,6 +105,7 @@ void optimizeTreeFirstPass(const QueryPlanOptimizationSettings & settings, Query void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes) { + size_t max_optimizations_to_apply = optimization_settings.max_optimizations_to_apply; size_t num_applied_projection = 0; bool has_reading_from_mt = false; @@ -141,12 +142,18 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s continue; } - if (num_applied_projection < 5 && optimization_settings.optimize_projection) + if (optimization_settings.optimize_projection) { bool applied = optimizeUseNormalProjections(stack, nodes); /// This is actually some internal knowledge bool stack_was_updated = !stack.back().node->children.empty(); num_applied_projection += applied; + + if (max_optimizations_to_apply && max_optimizations_to_apply < num_applied_projection) + throw Exception(ErrorCodes::TOO_MANY_QUERY_PLAN_OPTIMIZATIONS, + "Too many projection optimizations applied to query plan. Current limit {}", + max_optimizations_to_apply); + if (applied && stack_was_updated) continue; } diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index b798d094bd5..0ed1b4d3f6f 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -513,6 +513,10 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (reading->isParallelReadingEnabled()) return false; + // Currently projection don't support deduplication when moving parts between shards. + if (reading->getContext()->getSettingsRef().allow_experimental_query_deduplication) + return false; + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 5"); const auto metadata = reading->getStorageMetadata(); @@ -810,6 +814,10 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) if (reading->isParallelReadingEnabled()) return false; + // Currently projection don't support deduplication when moving parts between shards. + if (reading->getContext()->getSettingsRef().allow_experimental_query_deduplication) + return false; + auto iter = stack.rbegin(); while (iter != stack.rend()) { @@ -844,17 +852,12 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) if (!buildAggregatingDAG(*iter->node->children.front(), dag, filter_nodes, need_remove_column)) return false; - if (dag) - { - dag->removeUnusedActions(); - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", dag->dumpDAG()); - } - const ActionsDAG::Node * filter_node = nullptr; if (!filter_nodes.empty()) { auto & outputs = dag->getOutputs(); filter_node = filter_nodes.back(); + if (filter_nodes.size() > 1) { if (need_remove_column) @@ -866,6 +869,20 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) if (pos < outputs.size()) outputs.erase(outputs.begin() + pos); } + else + { + for (auto & output : outputs) + { + if (output == filter_node) + { + ColumnWithTypeAndName col; + col.name = filter_node->result_name; + col.type = filter_node->result_type; + col.column = col.type->createColumnConst(1, 1); + output = &dag->addColumn(std::move(col)); + } + } + } FunctionOverloadResolverPtr func_builder_and = std::make_unique( @@ -873,9 +890,16 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) filter_node = &dag->addFunction(func_builder_and, std::move(filter_nodes), {}); outputs.insert(outputs.begin(), filter_node); + need_remove_column = true; } - else if (!need_remove_column) - outputs.insert(outputs.begin(), filter_node); + // else if (!need_remove_column) + // outputs.insert(outputs.begin(), filter_node); + } + + if (dag) + { + dag->removeUnusedActions(); + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", dag->dumpDAG()); } std::list candidates; @@ -1024,11 +1048,14 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) if (filter_node) { + //std::cerr << "======== " << projection_reading_node.step->getOutputStream().header.dumpStructure(); expr_or_filter_node.step = std::make_unique( projection_reading_node.step->getOutputStream(), dag, filter_node->result_name, - true); + need_remove_column); + + //std::cerr << "======2= " << expr_or_filter_node.step->getOutputStream().header.dumpStructure(); } else expr_or_filter_node.step = std::make_unique( @@ -1048,8 +1075,30 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) } else { + const auto & main_stream = iter->node->children.front()->step->getOutputStream(); + const auto * proj_stream = &next_node->step->getOutputStream(); + + if (!blocksHaveEqualStructure(proj_stream->header, main_stream.header)) + { + + //std::cerr << "======3= " << next_node->step->getOutputStream().header.dumpStructure(); + auto convert_actions_dag = ActionsDAG::makeConvertingActions( + proj_stream->header.getColumnsWithTypeAndName(), + main_stream.header.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name, + true); + + auto converting = std::make_unique(*proj_stream, convert_actions_dag); + proj_stream = &converting->getOutputStream(); + auto & expr_node = nodes.emplace_back(); + expr_node.step = std::move(converting); + expr_node.children.push_back(next_node); + next_node = &expr_node; + //std::cerr << "======4= " << next_node->step->getOutputStream().header.dumpStructure(); + } + auto & union_node = nodes.emplace_back(); - DataStreams input_streams = {iter->node->children.front()->step->getOutputStream(), next_node->step->getOutputStream()}; + DataStreams input_streams = {main_stream, *proj_stream}; union_node.step = std::make_unique(std::move(input_streams)); union_node.children = {iter->node->children.front(), next_node}; iter->node->children.front() = &union_node; diff --git a/tests/queries/0_stateless/01710_normal_projection_fix1.sql b/tests/queries/0_stateless/01710_normal_projection_fix1.sql index 89632168709..e8ba830e5a5 100644 --- a/tests/queries/0_stateless/01710_normal_projection_fix1.sql +++ b/tests/queries/0_stateless/01710_normal_projection_fix1.sql @@ -15,4 +15,6 @@ select i from t prewhere j = 4; SELECT j = 2, i FROM t PREWHERE j = 2; +SELECT j = -1, j = NULL FROM t WHERE j = -1; + drop table t; diff --git a/tests/queries/1_stateful/00172_early_constant_folding.reference b/tests/queries/1_stateful/00172_early_constant_folding.reference index da564dc694e..a8fda098e5d 100644 --- a/tests/queries/1_stateful/00172_early_constant_folding.reference +++ b/tests/queries/1_stateful/00172_early_constant_folding.reference @@ -1,6 +1,8 @@ -(Expression) -ExpressionTransform - (ReadFromStorage) - AggregatingTransform - ExpressionTransform - SourceFromSingleChunk 0 → 1 +ExpressionTransform × 25 + (Aggregating) + Resize 1 → 25 + AggregatingTransform + (Expression) + ExpressionTransform + (ReadFromPreparedSource) + SourceFromSingleChunk 0 → 1 From a0b88f0a2d35b5fa2e1712646f082422f9a86a7f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 21 Feb 2023 20:01:59 +0000 Subject: [PATCH 047/559] Fix typos. --- src/Interpreters/ActionsDAG.h | 2 +- src/Processors/QueryPlan/AggregatingStep.h | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 66549488752..71e9c77c1c3 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -220,7 +220,7 @@ public: /// * Outputs are taken from required_outputs. /// Here want to substitute some expressions to columns from projection. /// This function expects that all required_outputs can be calculated from nodes in new_inputs. - /// If not, excpetion will happen. + /// If not, exception will happen. /// This function also expects that new_inputs and required_outputs are valid nodes from the same DAG. /// Example: /// DAG: new_inputs: Result DAG diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index f59718db522..b2af653fbce 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -66,12 +66,12 @@ public: void skipMerging() { skip_merging = true; } bool canUseProjection() const; - /// When we apply aggregate projection (whihc is full), this step will only merge data. - /// Agrument input_stream replaces current single input. + /// When we apply aggregate projection (which is full), this step will only merge data. + /// Argument input_stream replaces current single input. /// Probably we should replace this step to MergingAggregated later? (now, aggregation-in-order will not work) void requestOnlyMergeForAggregateProjection(const DataStream & input_stream); - /// When we apply aggregate projection (whihc is partial), this step should be replaced to AggregatingProjection. - /// Agrument input_stream would be the second input (from projection). + /// When we apply aggregate projection (which is partial), this step should be replaced to AggregatingProjection. + /// Argument input_stream would be the second input (from projection). std::unique_ptr convertToAggregatingProjection(const DataStream & input_stream) const; private: From bb43e2c4e8ae80f6a83fdccd620e93eb6e599280 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 22 Feb 2023 18:05:53 +0800 Subject: [PATCH 048/559] wip --- src/Functions/parseDateTime.cpp | 841 ++++++++++++++++++++++++++++++++ 1 file changed, 841 insertions(+) create mode 100644 src/Functions/parseDateTime.cpp diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp new file mode 100644 index 00000000000..465e1fce860 --- /dev/null +++ b/src/Functions/parseDateTime.cpp @@ -0,0 +1,841 @@ +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + +namespace +{ + +struct Date +{ + Int32 year = 1970; + Int32 month = 1; + Int32 day = 1; + bool is_ad = true; // AD -> true, BC -> false. + + Int32 week = 1; // Week of year based on ISO week date, e.g: 27 + Int32 day_of_week = 1; // Day of week, Monday:1, Tuesday:2, ..., Sunday:7 + bool week_date_format = false; + + Int32 day_of_year = 1; + bool day_of_year_format = false; + + bool century_format = false; + + bool is_year_of_era = false; // Year of era cannot be zero or negative. + bool has_year = false; // Whether year was explicitly specified. + + Int32 hour = 0; + Int32 minute = 0; + Int32 second = 0; + // Int32 microsecond = 0; + bool is_am = true; // AM -> true, PM -> false + Int64 timezone_id = -1; + + bool is_clock_hour = false; // Whether most recent hour specifier is clockhour + bool is_hour_of_half_day = false; // Whether most recent hour specifier is of half day. + + std::vector day_of_month_values; + std::vector day_of_year_values; +}; + +constexpr std::string_view weekdaysShort[] = {"sun", "mon", "tue", "wed", "thu", "fri", "sat"}; +constexpr std::string_view weekdaysFull[] = {"sunday", "monday", "tuesday", "wednesday", "thursday", "friday", "saturday"}; + +constexpr std::string_view monthsShort[] + = {"jan", "feb", "mar", "apr", "may", "jun", "jul", "aug", "sep", "oct", "nov", "dec"}; + +constexpr char digits100[201] = "00010203040506070809" + "10111213141516171819" + "20212223242526272829" + "30313233343536373839" + "40414243444546474849" + "50515253545556575859" + "60616263646566676869" + "70717273747576777879" + "80818283848586878889" + "90919293949596979899"; + +using Pos = const char *; + +class Action +{ +private: + using Func = Pos (*)(Pos cur, Pos end, Date & date); + Func func; + std::string literal; + +public: + explicit Action(Func && func_) : func(std::move(func_)) {} + + template + explicit Action(const Literal & literal_) : literal(literal_) + { + } + + Pos perform(Pos cur, Pos end, Date & date) const + { + if (func) + return func(cur, end, date); + else + { + ensureSpace(cur, end, literal.size(), "requires size >= " + std::to_string(literal.size())); + if (std::string_view(cur, literal.size()) != literal) + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Expect literal {} but {} provided", literal, std::string_view(cur, literal.size())); + cur += literal.size(); + return cur; + } + } + + template + static Pos readNumber2(Pos cur, Pos end, T & res) + { + ensureSpace(cur, end, 2, "readNumber2 requires size >= 2"); + res = (*cur - '0') * 10; + ++cur; + res += *cur; + ++cur; + return cur; + } + + template + static Pos readNumber3(Pos cur, Pos end, T & res) + { + cur = readNumber2(cur, end, res); + + ensureSpace(cur, end, 1, "readNumber3 requires size >= 3"); + res = res * 10 + (*cur - '0'); + ++cur; + return cur; + } + + template + static Pos readNumber4(Pos cur, Pos end, T & res) + { + cur = readNumber2(cur, end, res); + + T tmp; + cur = readNumber2(cur, end, tmp); + res = res * 100 + tmp; + return cur; + } + + static ALWAYS_INLINE void ensureSpace(Pos cur, Pos end, size_t len, const String & msg) + { + if (cur > end || cur + len > end) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because {}", msg); + } + + static ALWAYS_INLINE Pos assertChar(Pos cur, Pos end, char ch) + { + ensureSpace(cur, end, 1, "assertChar requires size >= 1"); + + if (*cur != ch) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expect char {}, but {} provided", String(ch, 1), String(*cur, 1)); + + ++cur; + return cur; + } + + static Pos mysqlDayOfWeekTextShort(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 3, "Parsing DayOfWeekTextShort requires size >= 3"); + + String str(cur, 3); + Poco::toLower(str); + Int32 i = 0; + for (; i < 7; ++i) + if (str == weekdaysShort[i]) + break; + + if (i == 7) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown short week text"); + + date.day_of_week = i + 1; + date.week_date_format = true; + date.day_of_year_format = false; + if (!date.has_year) + { + date.has_year = true; + date.year = 2000; + } + cur += 3; + return cur; + } + + static Pos mysqlMonthOfYearTextShort(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 3, "Parsing MonthOfYearTextShort requires size >= 3"); + + String str(cur, 3); + Poco::toLower(str); + + Int32 i = 0; + for (; i < 12; ++i) + if (str == monthsShort[i]) + break; + + if (i == 12) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown short month text"); + + date.month = i + 1; + cur += 3; + return cur; + } + + static Pos mysqlMonth(Pos cur, Pos end, Date & date) + { + return readNumber2(cur, end, date.month); + } + + static Pos mysqlCentury(Pos cur, Pos end, Date & date) + { + Int32 centuray; + cur = readNumber2(cur, end, centuray); + date.century_format = true; + date.year = centuray * 100; + date.has_year = true; + return cur; + } + + static Pos mysqlDayOfMonth(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.day); + date.day_of_month_values.push_back(date.day); + date.week_date_format = false; + date.day_of_year_format = false; + if (!date.has_year) + { + date.has_year = true; + date.year = 2000; + } + return cur; + } + + static Pos mysqlAmericanDate(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.month); + cur = assertChar(cur, end, '/'); + + cur = readNumber2(cur, end, date.day); + cur = assertChar(cur, end, '/'); + + cur = readNumber2(cur, end, date.year); + cur = assertChar(cur, end, '/'); + + date.week_date_format = false; + date.day_of_year_format = false; + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; + return cur; + } + + + static Pos mysqlDayOfMonthSpacePadded(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 2, "mysqlDayOfMonthSpacePadded requires size >= 2"); + + date.day = *cur == ' ' ? 0 : (*cur - '0'); + ++cur; + + date.day = 10 * date.day + (*cur - '0'); + ++cur; + + date.week_date_format = false; + date.day_of_year_format = false; + if (!date.has_year) + { + date.has_year = true; + date.year = 2000; + } + return cur; + } + + /* + static Pos mysqlFractionalSecond(Pos, Pos, Date &) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for fractional second"); + } + */ + + static Pos mysqlISO8601Date(Pos cur, Pos end, Date & date) + { + cur = readNumber4(cur, end, date.year); + cur = readNumber2(cur, end, date.month); + cur = readNumber2(cur, end, date.day); + return cur; + } + + static Pos mysqlISO8601Year2(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.year); + date.year += 2000; + return cur; + } + + static Pos mysqlISO8601Year4(Pos cur, Pos end, Date & date) + { + cur = readNumber4(cur, end, date.year); + return cur; + } + + static Pos mysqlDayOfYear(Pos cur, Pos end, Date & date) + { + cur = readNumber3(cur, end, date.day_of_year); + date.day_of_year_format = true; + return cur; + } + + static Pos mysqlDayOfWeek(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1"); + + date.day_of_week = *cur - '0'; + return cur; + } + + static Pos mysqlISO8601Week(Pos cur, Pos end, Date & date) + { + return readNumber2(cur, end, date.week); + } + + static Pos mysqlDayOfWeek0To6(Pos cur, Pos end, Date & date) + { + Pos res = mysqlDayOfWeek(cur, end, date); + + if (date.day_of_week == 7) + date.day_of_week = 0; + return res; + } + + static Pos mysqlDayOfWeekTextLong(Pos cur, Pos end, Date & date) + { + mysqlDayOfWeekTextShort(cur, end, date); + auto expect_text = weekdaysFull[date.day_of_week]; + + ensureSpace(cur, end, expect_text.size(), "mysqlDayOfWeekTextLong requires size >= " + std::to_string(expect_text.size())); + std::string_view text(cur, expect_text.size()); + if (text != expect_text) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown full day of week text {}", expect_text); + + cur += expect_text.size(); + return cur; + } + + static Pos mysqlYear2(Pos cur, Pos end, Date & date) + { + Pos res = readNumber2(cur, end, date.year); + date.year += 2000; + return res; + } + + + static Pos mysqlYear4(Pos cur, Pos end, Date & date) + { + return readNumber4(cur, end, date.year); + } + + /* + static Pos mysqlQuarter(Pos cur, Pos end, Date & date) + { + /// TODO + } + */ + + static Pos mysqlTimezoneOffset(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 1, "Parse mysqlTimezoneOffset failed"); + Int32 sign = 1; + if (*cur == '-') + sign = -1; + ++cur; + + Int32 hour; + cur = readNumber2(cur, end, hour); + + Int32 minute; + cur = readNumber2(cur, end, minute); + + date.timezone_id = sign * (hour * 3600 + minute); + return cur; + } + + static Pos mysqlMinute(Pos cur, Pos end, Date & date) + { + return readNumber2(cur, end, date.minute); + } + + static Pos mysqlAMPM(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 2, "mysqlAMPM requires size >= 2"); + + std::string_view text(cur, 2); + if (text == "PM") + date.is_am = false; + else if (text == "AM") + date.is_am = true; + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Text should be AM or PM, but {} provided", text); + + cur += 2; + return cur; + } + + static Pos mysqlHHMM12(Pos cur, Pos end, Date & date) + { + Int32 hour; + Int32 minute; + cur = readNumber2(cur, end, hour); + cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, minute); + cur = assertChar(cur, end, ' '); + cur = mysqlAMPM(cur, end, date); + + /// TODO process hour and minute + return cur; + } + + static Pos mysqlHHMM24(Pos cur, Pos end, Date & date) + { + Int32 hour; + Int32 minute; + + cur = readNumber2(cur, end, hour); + cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, minute); + + /// TODO process hour and minute + return cur; + } + + static Pos mysqlSecond(Pos cur, Pos end, Date & date) + { + return readNumber2(cur, end, date.second); + } + + static Pos mysqlISO8601Time(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.hour); + cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, date.minute); + cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, date.second); + return cur; + } + + static Pos mysqlHour12(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.hour); + date.is_hour_of_half_day = true; + date.is_clock_hour = false; + return cur; + } + + static Pos mysqlHour24(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.hour); + date.is_hour_of_half_day = false; + date.is_clock_hour = false; + return cur; + } + + +}; + + +struct ParseDateTimeTraits +{ + /* + enum class SupportInteger + { + Yes, + No + }; + */ + + enum class ParseSyntax + { + MySQL, + Joda + }; +}; + + +/// _FUNC_(str[, format, timezone]) +template +class FunctionParseDateTimeImpl : public IFunction +{ +public: + static constexpr auto name = Name::name; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.size() != 1 && arguments.size() != 2 && arguments.size() != 3) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 1, 2 or 3", + getName(), + arguments.size()); + + if (!isString(arguments[0].type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {} when arguments size is 1. Should be string", + arguments[0].type->getName(), + getName()); + + if (arguments.size() > 1 && !isString(arguments[1].type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {} when arguments size is 1. Should be string", + arguments[0].type->getName(), + getName()); + + if (arguments.size() > 2 && !isString(arguments[2].type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of third argument of function {} when arguments size is 1. Should be string", + arguments[0].type->getName(), + getName()); + + return std::make_shared(); + } + + ColumnPtr executeImpl( + const ColumnsWithTypeAndName & arguments, + [[maybe_unused]] const DataTypePtr & result_type, + [[maybe_unused]] size_t input_rows_count) const override + { + const auto * col_str = checkAndGetColumn(arguments[0].column.get()); + if (!col_str) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of first ('str') argument of function {}. Must be string.", + arguments[0].column->getName(), getName()); + + String format = getFormat(arguments); + const auto & time_zone = getTimeZone(arguments); + + std::vector instructions; + } + + + +private: + + void parseFormat(const String & format, std::vector & instructions) + { + if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::MySQL) + parseMysqlFormat(format, instructions); + else if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::Joda) + parseJodaFormat(format, instructions); + else + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, + "Unknown datetime format style {} in function {}", + magic_enum::enum_name(parse_syntax), + getName()); + } + + void parseMysqlFormat(const String & format, std::vector & instructions) + { + Pos pos = format.data(); + Pos end = pos + format.size(); + while (true) + { + Pos percent_pos = find_first_symbols<'%'>(pos, end); + if (percent_pos < end) + { + if (pos < percent_pos) + instructions.emplace_back(std::string_view(pos, percent_pos - pos)); + + pos = percent_pos + 1; + if (pos >= end) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Sign '%' is the last in format, if you need it, use '%%'"); + + switch (*pos) + { + // Abbreviated weekday [Mon...Sun] + case 'a': + instructions.emplace_back(&Action::mysqlDayOfWeekTextShort); + break; + + // Abbreviated month [Jan...Dec] + case 'b': + instructions.emplace_back(&Action::mysqlMonthOfYearTextShort); + break; + + // Month as a decimal number (01-12) + case 'c': + instructions.emplace_back(&Action::mysqlMonth); + break; + + // Year, divided by 100, zero-padded + case 'C': + instructions.emplace_back(&Action::mysqlCentury); + break; + + // Day of month, zero-padded (01-31) + case 'd': + instructions.emplace_back(&Action::mysqlDayOfMonth); + break; + + // Short MM/DD/YY date, equivalent to %m/%d/%y + case 'D': + instructions.emplace_back(&Action::mysqlAmericanDate); + break; + + // Day of month, space-padded ( 1-31) 23 + case 'e': + instructions.emplace_back(&Action::mysqlDayOfMonthSpacePadded); + break; + + // Fractional seconds + case 'f': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for fractional seconds"); + + // Short YYYY-MM-DD date, equivalent to %Y-%m-%d 2001-08-23 + case 'F': + instructions.emplace_back(&Action::mysqlISO8601Date); + break; + + // Last two digits of year of ISO 8601 week number (see %G) + case 'g': + instructions.emplace_back(&Action::mysqlISO8601Year2); + break; + + // Year of ISO 8601 week number (see %V) + case 'G': + instructions.emplace_back(&Action::mysqlISO8601Year4); + break; + + // Day of the year (001-366) 235 + case 'j': + instructions.emplace_back(&Action::mysqlDayOfYear); + break; + + // Month as a decimal number (01-12) + case 'm': + instructions.emplace_back(&Action::mysqlMonth); + break; + + // ISO 8601 weekday as number with Monday as 1 (1-7) + case 'u': + instructions.emplace_back(&Action::mysqlDayOfWeek); + break; + + // ISO 8601 week number (01-53) + case 'V': + instructions.emplace_back(&Action::mysqlISO8601Week); + break; + + // Weekday as a decimal number with Sunday as 0 (0-6) 4 + case 'w': + instructions.emplace_back(&Action::mysqlDayOfWeek0To6); + break; + + // Full weekday [Monday...Sunday] + case 'W': + instructions.emplace_back(&Action::mysqlDayOfWeekTextLong); + break; + + // Two digits year + case 'y': + instructions.emplace_back(&Action::mysqlYear2); + break; + + // Four digits year + case 'Y': + instructions.emplace_back(&Action::mysqlYear4); + break; + + // Quarter (1-4) + case 'Q': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for quarter"); + break; + + // Offset from UTC timezone as +hhmm or -hhmm + case 'z': + instructions.emplace_back(&Action::mysqlTimezoneOffset); + break; + + /// Time components. If the argument is Date, not a DateTime, then this components will have default value. + + // Minute (00-59) + case 'M': + instructions.emplace_back(&Action::mysqlMinute); + break; + + // AM or PM + case 'p': + instructions.emplace_back(&Action::mysqlAMPM); + break; + + // 12-hour HH:MM time, equivalent to %h:%i %p 2:55 PM + case 'r': + instructions.emplace_back(&Action::mysqlHHMM12); + break; + + // 24-hour HH:MM time, equivalent to %H:%i 14:55 + case 'R': + instructions.emplace_back(&Action::mysqlHHMM24); + break; + + // Seconds + case 's': + instructions.emplace_back(&Action::mysqlSecond); + break; + + // Seconds + case 'S': + instructions.emplace_back(&Action::mysqlSecond); + break; + + // ISO 8601 time format (HH:MM:SS), equivalent to %H:%i:%S 14:55:02 + case 'T': + instructions.emplace_back(&Action::mysqlISO8601Time); + break; + + // Hour in 12h format (01-12) + case 'h': + instructions.emplace_back(&Action::mysqlHour12); + break; + + // Hour in 24h format (00-23) + case 'H': + instructions.emplace_back(&Action::mysqlHour24); + break; + + // Minute of hour range [0, 59] + case 'i': + instructions.emplace_back(&Action::mysqlMinute); + break; + + // Hour in 12h format (01-12) + case 'I': + instructions.emplace_back(&Action::mysqlHour12); + break; + + // Hour in 24h format (00-23) + case 'k': + instructions.emplace_back(&Action::mysqlHour24); + break; + + // Hour in 12h format (01-12) + case 'l': + instructions.emplace_back(&Action::mysqlHour12); + break; + + case 't': + instructions.emplace_back("\t"); + break; + + case 'n': + instructions.emplace_back("\n"); + break; + + // Escaped literal characters. + case '%': + instructions.emplace_back("\n"); + break; + + // Unimplemented + case 'U': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for WEEK (Sun-Sat)"); + case 'v': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for WEEK (Mon-Sun)"); + case 'x': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for YEAR for week (Mon-Sun)"); + case 'X': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for YEAR for week (Sun-Sat)"); + + default: + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Incorrect syntax '{}', symbol is not supported '{}' for function {}", + format, + *pos, + getName()); + } + + ++pos; + } + else + { + instructions.emplace_back(std::string_view(pos, end - pos)); + break; + } + } + } + + void parseJodaFormat(const String & format, std::vector & instructions) + { + /// TODO + } + + + + String getFormat(const ColumnsWithTypeAndName & arguments) const + { + if (arguments.size() < 2) + { + if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::Joda) + return "yyyy-MM-dd HH:mm:ss"; + else + return "%F %T"; + } + + const auto * format_column = checkAndGetColumnConst(arguments[1].column.get()); + if (!format_column) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of second ('format') argument of function {}. Must be constant string.", + arguments[1].column->getName(), getName()); + return format_column->getValue(); + } + + const DateLUTImpl & getTimeZone(const ColumnsWithTypeAndName & arguments) const + { + if (arguments.size() < 3) + return DateLUT::instance(); + + const auto * col = checkAndGetColumnConst(arguments[2].column.get()); + if (!col) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of third ('timezone') argument of function {}. Must be constant string.", + arguments[2].column->getName(), getName()); + + String time_zone = col->getValue(); + if (time_zone.empty()) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Provided time zone must be non-empty and be a valid time zone"); + return DateLUT::instance(time_zone); + } +}; + +} + +} From 9749bd491bf8f34476994bdcd8c24308cddd6d60 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 22 Feb 2023 10:26:13 +0000 Subject: [PATCH 049/559] Fixing test. --- .../1_stateful/00172_early_constant_folding.reference | 5 +++-- tests/queries/1_stateful/00172_early_constant_folding.sql | 1 + 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/1_stateful/00172_early_constant_folding.reference b/tests/queries/1_stateful/00172_early_constant_folding.reference index a8fda098e5d..6b72183c066 100644 --- a/tests/queries/1_stateful/00172_early_constant_folding.reference +++ b/tests/queries/1_stateful/00172_early_constant_folding.reference @@ -1,6 +1,7 @@ -ExpressionTransform × 25 +(Expression) +ExpressionTransform × 10 (Aggregating) - Resize 1 → 25 + Resize 1 → 10 AggregatingTransform (Expression) ExpressionTransform diff --git a/tests/queries/1_stateful/00172_early_constant_folding.sql b/tests/queries/1_stateful/00172_early_constant_folding.sql index b31e418b492..1ed7b8719b4 100644 --- a/tests/queries/1_stateful/00172_early_constant_folding.sql +++ b/tests/queries/1_stateful/00172_early_constant_folding.sql @@ -1,3 +1,4 @@ -- Tags: no-parallel-replicas +set max_threads=10; EXPLAIN PIPELINE SELECT count(JavaEnable) FROM test.hits WHERE WatchID = 1 OR Title = 'next' OR URL = 'prev' OR URL = '???' OR 1; From 81d0a3bca3ef52b75eb1dd65f070c1ab49c542f1 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 22 Feb 2023 20:42:59 +0800 Subject: [PATCH 050/559] wip --- src/Functions/parseDateTime.cpp | 176 ++++++++++++++++++++++---------- 1 file changed, 122 insertions(+), 54 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 465e1fce860..3fad90ea1ff 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -267,18 +267,18 @@ public: return cur; } - /* - static Pos mysqlFractionalSecond(Pos, Pos, Date &) - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for fractional second"); - } - */ - static Pos mysqlISO8601Date(Pos cur, Pos end, Date & date) { cur = readNumber4(cur, end, date.year); cur = readNumber2(cur, end, date.month); cur = readNumber2(cur, end, date.day); + + date.week_date_format = false; + date.day_of_year_format = false; + + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; return cur; } @@ -286,19 +286,33 @@ public: { cur = readNumber2(cur, end, date.year); date.year += 2000; + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; return cur; } static Pos mysqlISO8601Year4(Pos cur, Pos end, Date & date) { cur = readNumber4(cur, end, date.year); + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; return cur; } static Pos mysqlDayOfYear(Pos cur, Pos end, Date & date) { cur = readNumber3(cur, end, date.day_of_year); + + date.day_of_year_values.push_back(date.day_of_year); date.day_of_year_format = true; + date.week_date_format = false; + if (!date.has_year) + { + date.has_year = true; + date.year = 2000; + } return cur; } @@ -307,27 +321,42 @@ public: ensureSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1"); date.day_of_week = *cur - '0'; + date.week_date_format = true; + date.day_of_year_format = false; + if (!date.has_year) + { + date.has_year = true; + date.year = 2000; + } return cur; } static Pos mysqlISO8601Week(Pos cur, Pos end, Date & date) { - return readNumber2(cur, end, date.week); + cur = readNumber2(cur, end, date.week); + date.week_date_format = true; + date.day_of_year_format = false; + if (date.has_year) + { + date.has_year = true; + date.year = 2000; + } + return cur; } static Pos mysqlDayOfWeek0To6(Pos cur, Pos end, Date & date) { - Pos res = mysqlDayOfWeek(cur, end, date); + cur = mysqlDayOfWeek(cur, end, date); + if (date.day_of_week == 0) + date.day_of_week = 7; - if (date.day_of_week == 7) - date.day_of_week = 0; - return res; + return cur; } static Pos mysqlDayOfWeekTextLong(Pos cur, Pos end, Date & date) { mysqlDayOfWeekTextShort(cur, end, date); - auto expect_text = weekdaysFull[date.day_of_week]; + auto expect_text = weekdaysFull[date.day_of_week - 1]; ensureSpace(cur, end, expect_text.size(), "mysqlDayOfWeekTextLong requires size >= " + std::to_string(expect_text.size())); std::string_view text(cur, expect_text.size()); @@ -340,26 +369,27 @@ public: static Pos mysqlYear2(Pos cur, Pos end, Date & date) { - Pos res = readNumber2(cur, end, date.year); + cur = readNumber2(cur, end, date.year); date.year += 2000; - return res; + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; + return cur; } static Pos mysqlYear4(Pos cur, Pos end, Date & date) { - return readNumber4(cur, end, date.year); + cur = readNumber4(cur, end, date.year); + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; + return cur; } - /* - static Pos mysqlQuarter(Pos cur, Pos end, Date & date) - { - /// TODO - } - */ - static Pos mysqlTimezoneOffset(Pos cur, Pos end, Date & date) { + /// TODO figure out what timezone_id mean ensureSpace(cur, end, 1, "Parse mysqlTimezoneOffset failed"); Int32 sign = 1; if (*cur == '-') @@ -385,11 +415,12 @@ public: { ensureSpace(cur, end, 2, "mysqlAMPM requires size >= 2"); - std::string_view text(cur, 2); + std::string text(cur, 2); + Poco::toUpper(text); if (text == "PM") - date.is_am = false; - else if (text == "AM") date.is_am = true; + else if (text == "AM") + date.is_am = false; else throw Exception(ErrorCodes::LOGICAL_ERROR, "Text should be AM or PM, but {} provided", text); @@ -399,28 +430,25 @@ public: static Pos mysqlHHMM12(Pos cur, Pos end, Date & date) { - Int32 hour; - Int32 minute; - cur = readNumber2(cur, end, hour); + cur = readNumber2(cur, end, date.hour); + date.is_clock_hour = false; + date.is_hour_of_half_day = true; + cur = assertChar(cur, end, ':'); - cur = readNumber2(cur, end, minute); + cur = readNumber2(cur, end, date.minute); cur = assertChar(cur, end, ' '); cur = mysqlAMPM(cur, end, date); - - /// TODO process hour and minute return cur; } static Pos mysqlHHMM24(Pos cur, Pos end, Date & date) { - Int32 hour; - Int32 minute; + cur = readNumber2(cur, end, date.hour); + date.is_clock_hour = false; + date.is_hour_of_half_day = false; - cur = readNumber2(cur, end, hour); cur = assertChar(cur, end, ':'); - cur = readNumber2(cur, end, minute); - - /// TODO process hour and minute + cur = readNumber2(cur, end, date.minute); return cur; } @@ -432,6 +460,9 @@ public: static Pos mysqlISO8601Time(Pos cur, Pos end, Date & date) { cur = readNumber2(cur, end, date.hour); + date.is_clock_hour = false; + date.is_hour_of_half_day = false; + cur = assertChar(cur, end, ':'); cur = readNumber2(cur, end, date.minute); cur = assertChar(cur, end, ':'); @@ -454,21 +485,11 @@ public: date.is_clock_hour = false; return cur; } - - }; struct ParseDateTimeTraits { - /* - enum class SupportInteger - { - Yes, - No - }; - */ - enum class ParseSyntax { MySQL, @@ -542,8 +563,59 @@ public: const auto & time_zone = getTimeZone(arguments); std::vector instructions; - } + parseFormat(format, instructions); + for (size_t i = 0; igetDataAt(i); + Date date; + Pos cur = str_ref.data; + Pos end = str_ref.data + str_ref.size; + for (const auto & instruction: instructions) + { + cur = instruction.perform(cur, end, date); + } + + // Ensure all input was consumed. + if (cur < end) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Invalid format input {} is malformed at {}", + str_ref.toView(), + std::string_view(cur, end - cur)); + + // Era is BC and year of era is provided + if (date.is_year_of_era && !date.is_ad) + date.year = -1 * (date.year - 1); + + if (date.is_hour_of_half_day && !date.is_am) + date.hour += 12; + + // Ensure all day of month values are valid for ending month value + for (size_t i = 0; i < date.day_of_month_values.size(); ++i) + { + if (!util::isValidDate(date.year, date.month, date.dayOfMonthValues[i])) + { + VELOX_USER_FAIL( + "Value {} for dayOfMonth must be in the range [1,{}]", + date.dayOfMonthValues[i], + util::getMaxDayOfMonth(date.year, date.month)); + } + } + + // Ensure all day of year values are valid for ending year value + for (int i = 0; i < date.dayOfYearValues.size(); i++) + { + if (!util::isValidDayOfYear(date.year, date.dayOfYearValues[i])) + { + VELOX_USER_FAIL( + "Value {} for dayOfMonth must be in the range [1,{}]", + date.dayOfYearValues[i], + util::isLeapYear(date.year) ? 366 : 365); + } + } + } + } private: @@ -793,11 +865,7 @@ private: } } - void parseJodaFormat(const String & format, std::vector & instructions) - { - /// TODO - } - + void parseJodaFormat(const String & /*format*/, std::vector & /*instructions*/) { } String getFormat(const ColumnsWithTypeAndName & arguments) const From 67db9123d85b94ed5831dedd2427ce07ed1b2c38 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 22 Feb 2023 16:40:40 +0000 Subject: [PATCH 051/559] Refactor a bit. --- .../Optimizations/optimizeReadInOrder.cpp | 2 +- .../Optimizations/optimizeUseProjections.cpp | 571 +++++++++--------- ...01710_projections_in_distributed_query.sql | 2 + 3 files changed, 285 insertions(+), 290 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 2c51950d567..556ed5cb8b4 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -176,7 +176,7 @@ static void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expressi dag = expression->clone(); } -/// This function builds a common DAG which is a gerge of DAGs from Filter and Expression steps chain. +/// 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) { diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 0ed1b4d3f6f..11ef581f698 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -22,102 +22,7 @@ namespace DB::QueryPlanOptimizations { -static QueryPlan::Node * findReadingStep(QueryPlan::Node & node) -{ - IQueryPlanStep * step = node.step.get(); - if (auto * reading = typeid_cast(step)) - return &node; - - if (node.children.size() != 1) - return nullptr; - - if (typeid_cast(step) || typeid_cast(step)) - return findReadingStep(*node.children.front()); - - return nullptr; -} - -static void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) -{ - if (dag) - dag->mergeInplace(std::move(*expression->clone())); - else - dag = expression->clone(); -} - - -/// This function builds a common DAG which is a gerge of DAGs from Filter and Expression steps chain. -static bool buildAggregatingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, ActionsDAG::NodeRawConstPtrs & filter_nodes, bool & need_remove_column) -{ - IQueryPlanStep * step = node.step.get(); - if (auto * reading = typeid_cast(step)) - { - //std::cerr << "============ Found ReadFromMergeTreen"; - if (const auto * prewhere_info = reading->getPrewhereInfo()) - { - //std::cerr << "============ Found prewhere info\n"; - if (prewhere_info->row_level_filter) - { - appendExpression(dag, prewhere_info->row_level_filter); - if (const auto * filter_node = dag->tryFindInOutputs(prewhere_info->row_level_column_name)) - filter_nodes.push_back(filter_node); - else - return false; - } - - if (prewhere_info->prewhere_actions) - { - //std::cerr << "============ Found prewhere actions\n"; - appendExpression(dag, prewhere_info->prewhere_actions); - //std::cerr << "============ Cur dag \n" << dag->dumpDAG(); - need_remove_column = prewhere_info->remove_prewhere_column; - if (const auto * filter_node = dag->tryFindInOutputs(prewhere_info->prewhere_column_name)) - filter_nodes.push_back(filter_node); - else - return false; - } - } - return true; - } - - if (node.children.size() != 1) - return false; - - if (!buildAggregatingDAG(*node.children.front(), dag, filter_nodes, need_remove_column)) - return false; - - if (auto * expression = typeid_cast(step)) - { - const auto & actions = expression->getExpression(); - if (actions->hasArrayJoin()) - return false; - - appendExpression(dag, actions); - //std::cerr << "============ Cur e dag \n" << dag->dumpDAG(); - need_remove_column = false; - return true; - } - - if (auto * filter = typeid_cast(step)) - { - const auto & actions = filter->getExpression(); - if (actions->hasArrayJoin()) - return false; - - appendExpression(dag, actions); - //std::cerr << "============ Cur f dag \n" << dag->dumpDAG(); - need_remove_column = filter->removesFilterColumn(); - const auto * filter_expression = dag->tryFindInOutputs(filter->getFilterColumnName()); - if (!filter_expression) - return false; - - filter_nodes.push_back(filter_expression); - return true; - } - - return false; -} - +/// Required analysis info from aggregate projection. struct AggregateProjectionInfo { ActionsDAGPtr before_aggregation; @@ -129,14 +34,48 @@ struct AggregateProjectionInfo ContextPtr context; }; -AggregateProjectionInfo getAggregatingProjectionInfo( +struct ProjectionCandidate +{ + const ProjectionDescription * projection; + + /// The number of marks we are going to read + size_t sum_marks = 0; + + /// Analysis result, separate for parts with and without projection. + /// Analysis is done in order to estimate the number of marks we are going to read. + /// For chosen projection, it is reused for reading step. + MergeTreeDataSelectAnalysisResultPtr merge_tree_projection_select_result_ptr; + MergeTreeDataSelectAnalysisResultPtr merge_tree_normal_select_result_ptr; +}; + +/// Aggregate projection analysis result in case it can be applied. +struct AggregateProjectionCandidate : public ProjectionCandidate +{ + AggregateProjectionInfo info; + + /// Actions which need to be applied to columns from projection + /// in order to get all the columns required for aggregation. + ActionsDAGPtr dag; +}; + +/// Normal projection analysis result in case it can be applied. +/// For now, it is empty. +/// Normal projection can be used only if it contains all required source columns. +/// It would not be hard to support pre-computed expressions and filtration. +struct NormalProjectionCandidate : public ProjectionCandidate +{ +}; + +/// Get required info from aggregate projection. +/// Ideally, this should be pre-calculated and stored inside ProjectionDescription. +static AggregateProjectionInfo getAggregatingProjectionInfo( const ProjectionDescription & projection, const ContextPtr & context, const StorageMetadataPtr & metadata_snapshot, const Block & key_virtual_columns) { - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj query : {}", queryToString(projection.query_ast)); - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Sample for keys : {}", projection.sample_block_for_keys.dumpStructure()); + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection query : {}", queryToString(projection.query_ast)); + /// This is a bad approach. /// We'd better have a separate interpreter for projections. /// Now it's not obvious we didn't miss anything here. @@ -155,6 +94,8 @@ AggregateProjectionInfo getAggregatingProjectionInfo( info.keys = query_analyzer->aggregationKeys(); info.aggregates = query_analyzer->aggregates(); + /// Add part/partition virtual columns to projection aggregation keys. + /// We can do it because projection is stored for every part separately. for (const auto & virt_column : key_virtual_columns) { const auto * input = &info.before_aggregation->addInput(virt_column); @@ -165,117 +106,54 @@ AggregateProjectionInfo getAggregatingProjectionInfo( return info; } -struct AggregateProjectionCandidate +using DAGIndex = std::unordered_map; +static DAGIndex buildDAGIndex(const ActionsDAG & dag) { - AggregateProjectionInfo info; - const ProjectionDescription * projection; - ActionsDAGPtr dag; - - MergeTreeDataSelectAnalysisResultPtr merge_tree_projection_select_result_ptr; - MergeTreeDataSelectAnalysisResultPtr merge_tree_normal_select_result_ptr; - - size_t sum_marks = 0; -}; - -struct NormalProjectionCandidate -{ - const ProjectionDescription * projection; - - MergeTreeDataSelectAnalysisResultPtr merge_tree_projection_select_result_ptr; - MergeTreeDataSelectAnalysisResultPtr merge_tree_normal_select_result_ptr; - - size_t sum_marks = 0; -}; - -ActionsDAGPtr analyzeAggregateProjection( - const AggregateProjectionInfo & info, - ActionsDAG & query_dag, - const ActionsDAG::Node * filter_node, - const Names & keys, - const AggregateDescriptions & aggregates) -{ - - ActionsDAG::NodeRawConstPtrs key_nodes; - std::unordered_set aggregate_args; - - std::unordered_map index; - for (const auto * output : query_dag.getOutputs()) + DAGIndex index; + for (const auto * output : dag.getOutputs()) index.emplace(output->result_name, output); - std::unordered_map proj_index; - for (const auto * output : info.before_aggregation->getOutputs()) - proj_index.emplace(output->result_name, output); - - key_nodes.reserve(keys.size() + 1); - - if (filter_node) - key_nodes.push_back(filter_node); - - for (const auto & key : keys) - { - auto it = index.find(key); - /// This should not happen ideally. - if (it == index.end()) - { - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot find key {} in query DAG", key); - return {}; - } - - key_nodes.push_back(it->second); - } - - for (const auto & aggregate : aggregates) - { - for (const auto & argument : aggregate.argument_names) - { - auto it = index.find(argument); - /// This should not happen ideally. - if (it == index.end()) - { - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot find arg {} for agg functions {}", argument, aggregate.column_name); - return {}; - } - - aggregate_args.insert(it->second); - } - } - - MatchedTrees::Matches matches = matchTrees(*info.before_aggregation, query_dag); - for (const auto & [node, match] : matches) - { - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Match {} {} -> {} {} (with monotonicity : {})", - static_cast(node), node->result_name, - static_cast(match.node), (match.node ? match.node->result_name : ""), match.monotonicity != std::nullopt); - } + return index; +} +bool areAggregatesMatch( + const AggregateProjectionInfo & info, + const AggregateDescriptions & aggregates, + const MatchedTrees::Matches & matches, + const DAGIndex & query_index, + const DAGIndex & proj_index) +{ std::unordered_map> projection_aggregate_functions; for (size_t i = 0; i < info.aggregates.size(); ++i) projection_aggregate_functions[info.aggregates[i].function->getName()].push_back(i); - std::unordered_set split_nodes; + // struct AggFuncMatch + // { + // /// idx in projection + // size_t idx; + // /// nodes in query DAG + // ActionsDAG::NodeRawConstPtrs args; + // }; - struct AggFuncMatch - { - /// idx in projection - size_t idx; - /// nodes in query DAG - ActionsDAG::NodeRawConstPtrs args; - }; - - std::vector aggregate_function_matches; - aggregate_function_matches.reserve(aggregates.size()); + // std::vector aggregate_function_matches; + // aggregate_function_matches.reserve(aggregates.size()); for (const auto & aggregate : aggregates) { auto it = projection_aggregate_functions.find(aggregate.function->getName()); if (it == projection_aggregate_functions.end()) { - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot match agg func {} by name {}", aggregate.column_name, aggregate.function->getName()); - return {}; + // LOG_TRACE( + // &Poco::Logger::get("optimizeUseProjections"), + // "Cannot match agg func {} by name {}", + // aggregate.column_name, aggregate.function->getName()); + + return false; } auto & candidates = it->second; - std::optional match; + // std::optional match; + bool found_match = false; for (size_t idx : candidates) { @@ -300,9 +178,9 @@ ActionsDAGPtr analyzeAggregateProjection( bool all_args_not_null = true; for (const auto & query_name : aggregate.argument_names) { - auto jt = index.find(query_name); + auto jt = query_index.find(query_name); - if (jt == index.end() || jt->second->result_type->isNullable()) + if (jt == query_index.end() || jt->second->result_type->isNullable()) { all_args_not_null = false; break; @@ -323,7 +201,8 @@ ActionsDAGPtr analyzeAggregateProjection( if (all_args_not_null) { /// we can ignore arguments for count() - match = AggFuncMatch{idx, {}}; + /// match = AggFuncMatch{idx, {}}; + found_match = true; break; } } @@ -332,83 +211,120 @@ ActionsDAGPtr analyzeAggregateProjection( continue; size_t num_args = aggregate.argument_names.size(); - ActionsDAG::NodeRawConstPtrs args; - args.reserve(num_args); - for (size_t arg = 0; arg < num_args; ++arg) - { - const auto & query_name = aggregate.argument_names[arg]; - const auto & proj_name = candidate.argument_names[arg]; + // ActionsDAG::NodeRawConstPtrs args; + // args.reserve(num_args); + + size_t next_arg = 0; + while (next_arg < num_args) + { + const auto & query_name = aggregate.argument_names[next_arg]; + const auto & proj_name = candidate.argument_names[next_arg]; + + auto jt = query_index.find(query_name); + auto kt = proj_index.find(proj_name); - auto jt = index.find(query_name); /// This should not happen ideally. - if (jt == index.end()) - { - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot match agg func {} vs {} : can't find arg {} in query dag", - aggregate.column_name, candidate.column_name, query_name); + if (jt == query_index.end() || kt == proj_index.end()) break; - } const auto * query_node = jt->second; - - auto kt = proj_index.find(proj_name); - /// This should not happen ideally. - if (kt == proj_index.end()) - { - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot match agg func {} vs {} : can't find arg {} in proj dag", - aggregate.column_name, candidate.column_name, proj_name); - break; - } - const auto * proj_node = kt->second; auto mt = matches.find(query_node); if (mt == matches.end()) { - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot match agg func {} vs {} : can't match arg {} vs {} : no node in map", - aggregate.column_name, candidate.column_name, query_name, proj_name); + // LOG_TRACE( + // &Poco::Logger::get("optimizeUseProjections"), + // "Cannot match agg func {} vs {} : can't match arg {} vs {} : no node in map", + // aggregate.column_name, candidate.column_name, query_name, proj_name); + break; } const auto & node_match = mt->second; if (node_match.node != proj_node || node_match.monotonicity) { - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot match agg func {} vs {} : can't match arg {} vs {} : no match or monotonicity", - aggregate.column_name, candidate.column_name, query_name, proj_name); + // LOG_TRACE( + // &Poco::Logger::get("optimizeUseProjections"), + // "Cannot match agg func {} vs {} : can't match arg {} vs {} : no match or monotonicity", + // aggregate.column_name, candidate.column_name, query_name, proj_name); + break; } - args.push_back(query_node); + // args.push_back(query_node); + ++next_arg; } - if (args.size() < aggregate.argument_names.size()) + if (next_arg < aggregate.argument_names.size()) continue; - // for (const auto * node : args) - // split_nodes.insert(node); - - match = AggFuncMatch{idx, std::move(args)}; + // match = AggFuncMatch{idx, std::move(args)}; + found_match = true; break; } - if (!match) - return {}; + if (!found_match) + return false; - aggregate_function_matches.emplace_back(std::move(*match)); + // aggregate_function_matches.emplace_back(std::move(*match)); } + return true; +} + +ActionsDAGPtr analyzeAggregateProjection( + const AggregateProjectionInfo & info, + ActionsDAG & query_dag, + const ActionsDAG::Node * filter_node, + const Names & keys, + const AggregateDescriptions & aggregates) +{ + auto query_index = buildDAGIndex(query_dag); + auto proj_index = buildDAGIndex(*info.before_aggregation); + + MatchedTrees::Matches matches = matchTrees(*info.before_aggregation, query_dag); + + // for (const auto & [node, match] : matches) + // { + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Match {} {} -> {} {} (with monotonicity : {})", + // static_cast(node), node->result_name, + // static_cast(match.node), (match.node ? match.node->result_name : ""), match.monotonicity != std::nullopt); + // } + + if (!areAggregatesMatch(info, aggregates, matches, query_index, proj_index)) + return {}; + + ActionsDAG::NodeRawConstPtrs query_key_nodes; std::unordered_set proj_key_nodes; - for (const auto & key : info.keys) + { - auto it = proj_index.find(key.name); - /// This should not happen ideally. - if (it == proj_index.end()) - break; + for (const auto & key : info.keys) + { + auto it = proj_index.find(key.name); + /// This should not happen ideally. + if (it == proj_index.end()) + return {}; - proj_key_nodes.insert(it->second); + proj_key_nodes.insert(it->second); + } + + query_key_nodes.reserve(keys.size() + 1); + + if (filter_node) + query_key_nodes.push_back(filter_node); + + for (const auto & key : keys) + { + auto it = query_index.find(key); + /// This should not happen ideally. + if (it == query_index.end()) + return {}; + + query_key_nodes.push_back(it->second); + } } - std::unordered_set visited; - struct Frame { const ActionsDAG::Node * node; @@ -416,7 +332,10 @@ ActionsDAGPtr analyzeAggregateProjection( }; std::stack stack; - for (const auto * key_node : key_nodes) + std::unordered_set visited; + std::unordered_map new_inputs; + + for (const auto * key_node : query_key_nodes) { if (visited.contains(key_node)) continue; @@ -436,7 +355,7 @@ ActionsDAGPtr analyzeAggregateProjection( if (match.node && !match.monotonicity && proj_key_nodes.contains(match.node)) { visited.insert(frame.node); - split_nodes.insert(frame.node); + new_inputs[frame.node] = match.node->result_name; stack.pop(); continue; } @@ -463,12 +382,10 @@ ActionsDAGPtr analyzeAggregateProjection( } } - std::unordered_map new_inputs; - for (const auto * node : split_nodes) - new_inputs[node] = matches[node].node->result_name; + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Folding actions by projection"); + + auto proj_dag = query_dag.foldActionsByProjection(new_inputs, query_key_nodes); - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Folding actions by projection"); - auto proj_dag = query_dag.foldActionsByProjection(new_inputs, key_nodes); auto & proj_dag_outputs = proj_dag->getOutputs(); for (const auto & aggregate : aggregates) proj_dag_outputs.push_back(&proj_dag->addInput(aggregate.column_name, aggregate.function->getResultType())); @@ -476,30 +393,105 @@ ActionsDAGPtr analyzeAggregateProjection( return proj_dag; } -bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) +static QueryPlan::Node * findReadingStep(QueryPlan::Node & node) { + IQueryPlanStep * step = node.step.get(); + if (auto * reading = typeid_cast(step)) + return &node; + + if (node.children.size() != 1) + return nullptr; + + if (typeid_cast(step) || typeid_cast(step)) + return findReadingStep(*node.children.front()); + + return nullptr; +} + +static void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) +{ + if (dag) + dag->mergeInplace(std::move(*expression->clone())); + else + dag = expression->clone(); +} + + +/// This function builds a common DAG which is a merge of DAGs from Filter and Expression steps chain. +/// Additionally, for all the Filter steps, we collect filter conditions into filter_nodes. +/// Flag need_remove_column is set in case if the last step is a Filter step and it should remove filter column. +static bool buildAggregatingDAG( + QueryPlan::Node & node, + ActionsDAGPtr & dag, + ActionsDAG::NodeRawConstPtrs & filter_nodes, + bool & need_remove_column) +{ + IQueryPlanStep * step = node.step.get(); + if (auto * reading = typeid_cast(step)) + { + if (const auto * prewhere_info = reading->getPrewhereInfo()) + { + if (prewhere_info->row_level_filter) + { + need_remove_column = false; + appendExpression(dag, prewhere_info->row_level_filter); + if (const auto * filter_node = dag->tryFindInOutputs(prewhere_info->row_level_column_name)) + filter_nodes.push_back(filter_node); + else + return false; + } + + if (prewhere_info->prewhere_actions) + { + need_remove_column = prewhere_info->remove_prewhere_column; + appendExpression(dag, prewhere_info->prewhere_actions); + if (const auto * filter_node = dag->tryFindInOutputs(prewhere_info->prewhere_column_name)) + filter_nodes.push_back(filter_node); + else + return false; + } + } + return true; + } + if (node.children.size() != 1) return false; - auto * aggregating = typeid_cast(node.step.get()); - if (!aggregating) + if (!buildAggregatingDAG(*node.children.front(), dag, filter_nodes, need_remove_column)) return false; - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 2"); - if (!aggregating->canUseProjection()) - return false; + if (auto * expression = typeid_cast(step)) + { + const auto & actions = expression->getExpression(); + if (actions->hasArrayJoin()) + return false; - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 3"); - QueryPlan::Node * reading_node = findReadingStep(*node.children.front()); - if (!reading_node) - return false; + appendExpression(dag, actions); + need_remove_column = false; + return true; + } - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 4"); + if (auto * filter = typeid_cast(step)) + { + const auto & actions = filter->getExpression(); + if (actions->hasArrayJoin()) + return false; - auto * reading = typeid_cast(reading_node->step.get()); - if (!reading) - return false; + appendExpression(dag, actions); + need_remove_column = filter->removesFilterColumn(); + const auto * filter_expression = dag->tryFindInOutputs(filter->getFilterColumnName()); + if (!filter_expression) + return false; + filter_nodes.push_back(filter_expression); + return true; + } + + return false; +} + +bool canUseProjectionForReadingStep(ReadFromMergeTree * reading) +{ /// Probably some projection already was applied. if (reading->hasAnalyzedResult()) return false; @@ -517,7 +509,31 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (reading->getContext()->getSettingsRef().allow_experimental_query_deduplication) return false; - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try optimize projection 5"); + return true; +} + +bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) +{ + if (node.children.size() != 1) + return false; + + auto * aggregating = typeid_cast(node.step.get()); + if (!aggregating) + return false; + + if (!aggregating->canUseProjection()) + return false; + + QueryPlan::Node * reading_node = findReadingStep(*node.children.front()); + if (!reading_node) + return false; + + auto * reading = typeid_cast(reading_node->step.get()); + if (!reading) + return false; + + if (!canUseProjectionForReadingStep(reading)) + return false; const auto metadata = reading->getStorageMetadata(); const auto & projections = metadata->projections; @@ -586,11 +602,8 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (auto proj_dag = analyzeAggregateProjection(info, *dag, filter_node, keys, aggregates)) { LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); - minmax_projection.emplace(AggregateProjectionCandidate{ - .info = std::move(info), - .projection = projection, - .dag = std::move(proj_dag), - }); + minmax_projection.emplace(AggregateProjectionCandidate{.info = std::move(info), .dag = std::move(proj_dag)}); + minmax_projection->projection = projection; minmax_count_projection_block = reading->getMergeTreeData().getMinMaxCountProjectionBlock( metadata, @@ -621,11 +634,9 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (auto proj_dag = analyzeAggregateProjection(info, *dag, filter_node, keys, aggregates)) { LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); - candidates.emplace_back(AggregateProjectionCandidate{ - .info = std::move(info), - .projection = projection, - .dag = std::move(proj_dag), - }); + AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(proj_dag)}; + candidate.projection = projection; + candidates.emplace_back(std::move(candidate)); } } @@ -797,25 +808,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) if (!reading) return false; - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), - "Reading {} {} has analyzed result {}", - reading->getName(), reading->getStepDescription(), reading->hasAnalyzedResult()); - - /// Probably some projection already was applied. - if (reading->hasAnalyzedResult()) - return false; - - if (reading->isQueryWithFinal()) - return false; - - if (reading->isQueryWithSampling()) - return false; - - if (reading->isParallelReadingEnabled()) - return false; - - // Currently projection don't support deduplication when moving parts between shards. - if (reading->getContext()->getSettingsRef().allow_experimental_query_deduplication) + if (!canUseProjectionForReadingStep(reading)) return false; auto iter = stack.rbegin(); diff --git a/tests/queries/0_stateless/01710_projections_in_distributed_query.sql b/tests/queries/0_stateless/01710_projections_in_distributed_query.sql index fa734b605cd..7600340579d 100644 --- a/tests/queries/0_stateless/01710_projections_in_distributed_query.sql +++ b/tests/queries/0_stateless/01710_projections_in_distributed_query.sql @@ -1,5 +1,7 @@ -- Tags: distributed +set enable_memory_bound_merging_of_aggregation_results=0; + drop table if exists projection_test; create table projection_test (dt DateTime, cost Int64, projection p (select toStartOfMinute(dt) dt_m, sum(cost) group by dt_m)) engine MergeTree partition by toDate(dt) order by dt; From c8ce1f2b50a794d2640b2a93fb33bf9e7c6cc51f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 22 Feb 2023 17:27:21 +0000 Subject: [PATCH 052/559] Refactor a bit. --- .../Optimizations/optimizeUseProjections.cpp | 83 +++++++++---------- 1 file changed, 37 insertions(+), 46 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 11ef581f698..ea81208df8e 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -116,6 +116,20 @@ static DAGIndex buildDAGIndex(const ActionsDAG & dag) return index; } +static bool hasNullableOrMissingColumn(const DAGIndex & index, const Names & names) +{ + for (const auto & query_name : names) + { + auto jt = index.find(query_name); + if (jt == index.end() || jt->second->result_type->isNullable()) + return true; + } + + return false; +} + +/// Here we try to match aggregate functions from the query to +/// aggregate functions from projection. bool areAggregatesMatch( const AggregateProjectionInfo & info, const AggregateDescriptions & aggregates, @@ -123,23 +137,14 @@ bool areAggregatesMatch( const DAGIndex & query_index, const DAGIndex & proj_index) { + /// Index (projection agg function name) -> pos std::unordered_map> projection_aggregate_functions; for (size_t i = 0; i < info.aggregates.size(); ++i) projection_aggregate_functions[info.aggregates[i].function->getName()].push_back(i); - // struct AggFuncMatch - // { - // /// idx in projection - // size_t idx; - // /// nodes in query DAG - // ActionsDAG::NodeRawConstPtrs args; - // }; - - // std::vector aggregate_function_matches; - // aggregate_function_matches.reserve(aggregates.size()); - for (const auto & aggregate : aggregates) { + /// Get a list of candidates by name first. auto it = projection_aggregate_functions.find(aggregate.function->getName()); if (it == projection_aggregate_functions.end()) { @@ -150,9 +155,8 @@ bool areAggregatesMatch( return false; } - auto & candidates = it->second; - // std::optional match; + auto & candidates = it->second; bool found_match = false; for (size_t idx : candidates) @@ -173,46 +177,28 @@ bool areAggregatesMatch( continue; } + /// This is a special case for the function count(). + /// We can assume that 'count(expr) == count()' if expr is not nullable. if (typeid_cast(candidate.function.get())) { - bool all_args_not_null = true; - for (const auto & query_name : aggregate.argument_names) - { - auto jt = query_index.find(query_name); + bool has_nullable_or_missing_arg = false; + has_nullable_or_missing_arg |= hasNullableOrMissingColumn(query_index, aggregate.argument_names); + has_nullable_or_missing_arg |= hasNullableOrMissingColumn(proj_index, candidate.argument_names); - if (jt == query_index.end() || jt->second->result_type->isNullable()) - { - all_args_not_null = false; - break; - } - } - - for (const auto & proj_name : candidate.argument_names) - { - auto kt = proj_index.find(proj_name); - - if (kt == proj_index.end() || kt->second->result_type->isNullable()) - { - all_args_not_null = false; - break; - } - } - - if (all_args_not_null) + if (!has_nullable_or_missing_arg) { /// we can ignore arguments for count() - /// match = AggFuncMatch{idx, {}}; found_match = true; break; } } - if (aggregate.argument_names.size() != candidate.argument_names.size()) - continue; + /// Now, function names and types matched. + /// Next, match arguments from DAGs. size_t num_args = aggregate.argument_names.size(); - // ActionsDAG::NodeRawConstPtrs args; - // args.reserve(num_args); + if (num_args != candidate.argument_names.size()) + continue; size_t next_arg = 0; while (next_arg < num_args) @@ -252,22 +238,18 @@ bool areAggregatesMatch( break; } - // args.push_back(query_node); ++next_arg; } if (next_arg < aggregate.argument_names.size()) continue; - // match = AggFuncMatch{idx, std::move(args)}; found_match = true; break; } if (!found_match) return false; - - // aggregate_function_matches.emplace_back(std::move(*match)); } return true; @@ -299,6 +281,8 @@ ActionsDAGPtr analyzeAggregateProjection( std::unordered_set proj_key_nodes; { + /// Just, filling the set above. + for (const auto & key : info.keys) { auto it = proj_index.find(key.name); @@ -311,6 +295,9 @@ ActionsDAGPtr analyzeAggregateProjection( query_key_nodes.reserve(keys.size() + 1); + /// We need to add filter column to keys set. + /// It should be computable from projection keys. + /// It will be removed in FilterStep. if (filter_node) query_key_nodes.push_back(filter_node); @@ -325,6 +312,9 @@ ActionsDAGPtr analyzeAggregateProjection( } } + /// Here we want to match query keys with projection keys. + /// Query key can be any expression depending on projection keys. + struct Frame { const ActionsDAG::Node * node; @@ -386,6 +376,7 @@ ActionsDAGPtr analyzeAggregateProjection( auto proj_dag = query_dag.foldActionsByProjection(new_inputs, query_key_nodes); + /// Just add all the aggregates to dag inputs. auto & proj_dag_outputs = proj_dag->getOutputs(); for (const auto & aggregate : aggregates) proj_dag_outputs.push_back(&proj_dag->addInput(aggregate.column_name, aggregate.function->getResultType())); @@ -548,7 +539,7 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (!can_use_minmax_projection && agg_projections.empty()) return false; - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Has agg projection"); + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Has agg projection"); ActionsDAGPtr dag; bool need_remove_column = false; From d4793afec374fc6d322281c207a5d8e04164567f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 23 Feb 2023 19:56:03 +0800 Subject: [PATCH 053/559] finish dev of to_timestamp in mysql syntax --- src/Functions/parseDateTime.cpp | 1810 ++++++++++++++++--------------- 1 file changed, 962 insertions(+), 848 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 3fad90ea1ff..89a4716b6cd 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -1,9 +1,10 @@ -#include +#include +#include #include #include #include #include -#include +#include #include #include @@ -16,893 +17,1006 @@ namespace DB { - namespace { + using Pos = const char *; -struct Date -{ - Int32 year = 1970; - Int32 month = 1; - Int32 day = 1; - bool is_ad = true; // AD -> true, BC -> false. + constexpr std::string_view weekdaysShort[] = {"sun", "mon", "tue", "wed", "thu", "fri", "sat"}; + constexpr std::string_view weekdaysFull[] = {"sunday", "monday", "tuesday", "wednesday", "thursday", "friday", "saturday"}; + constexpr std::string_view monthsShort[] = {"jan", "feb", "mar", "apr", "may", "jun", "jul", "aug", "sep", "oct", "nov", "dec"}; - Int32 week = 1; // Week of year based on ISO week date, e.g: 27 - Int32 day_of_week = 1; // Day of week, Monday:1, Tuesday:2, ..., Sunday:7 - bool week_date_format = false; + constexpr Int32 leapDays[] = {0, 31, 29, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}; + constexpr Int32 normalDays[] = {0, 31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}; - Int32 day_of_year = 1; - bool day_of_year_format = false; + constexpr Int32 cumulativeLeapDays[] = {0, 31, 60, 91, 121, 152, 182, 213, 244, 274, 305, 335, 366}; + constexpr Int32 cumulativeDays[] = {0, 31, 59, 90, 120, 151, 181, 212, 243, 273, 304, 334, 365}; + constexpr Int32 cumulativeYearDays[] + = {0, 365, 730, 1096, 1461, 1826, 2191, 2557, 2922, 3287, 3652, 4018, 4383, 4748, 5113, 5479, 5844, 6209, + 6574, 6940, 7305, 7670, 8035, 8401, 8766, 9131, 9496, 9862, 10227, 10592, 10957, 11323, 11688, 12053, 12418, 12784, + 13149, 13514, 13879, 14245, 14610, 14975, 15340, 15706, 16071, 16436, 16801, 17167, 17532, 17897, 18262, 18628, 18993, 19358, + 19723, 20089, 20454, 20819, 21184, 21550, 21915, 22280, 22645, 23011, 23376, 23741, 24106, 24472, 24837, 25202, 25567, 25933, + 26298, 26663, 27028, 27394, 27759, 28124, 28489, 28855, 29220, 29585, 29950, 30316, 30681, 31046, 31411, 31777, 32142, 32507, + 32872, 33238, 33603, 33968, 34333, 34699, 35064, 35429, 35794, 36160, 36525, 36890, 37255, 37621, 37986, 38351, 38716, 39082, + 39447, 39812, 40177, 40543, 40908, 41273, 41638, 42004, 42369, 42734, 43099, 43465, 43830, 44195, 44560, 44926, 45291, 45656, + 46021, 46387, 46752, 47117, 47482, 47847, 48212, 48577, 48942, 49308, 49673}; - bool century_format = false; - bool is_year_of_era = false; // Year of era cannot be zero or negative. - bool has_year = false; // Whether year was explicitly specified. + constexpr Int32 minYear = 1970; + constexpr Int32 maxYear = 2106; + constexpr Int32 yearInterval = 400; - Int32 hour = 0; - Int32 minute = 0; - Int32 second = 0; - // Int32 microsecond = 0; - bool is_am = true; // AM -> true, PM -> false - Int64 timezone_id = -1; - - bool is_clock_hour = false; // Whether most recent hour specifier is clockhour - bool is_hour_of_half_day = false; // Whether most recent hour specifier is of half day. - - std::vector day_of_month_values; - std::vector day_of_year_values; -}; - -constexpr std::string_view weekdaysShort[] = {"sun", "mon", "tue", "wed", "thu", "fri", "sat"}; -constexpr std::string_view weekdaysFull[] = {"sunday", "monday", "tuesday", "wednesday", "thursday", "friday", "saturday"}; - -constexpr std::string_view monthsShort[] - = {"jan", "feb", "mar", "apr", "may", "jun", "jul", "aug", "sep", "oct", "nov", "dec"}; - -constexpr char digits100[201] = "00010203040506070809" - "10111213141516171819" - "20212223242526272829" - "30313233343536373839" - "40414243444546474849" - "50515253545556575859" - "60616263646566676869" - "70717273747576777879" - "80818283848586878889" - "90919293949596979899"; - -using Pos = const char *; - -class Action -{ -private: - using Func = Pos (*)(Pos cur, Pos end, Date & date); - Func func; - std::string literal; - -public: - explicit Action(Func && func_) : func(std::move(func_)) {} - - template - explicit Action(const Literal & literal_) : literal(literal_) + struct Date { - } + Int32 year = 1970; + Int32 month = 1; + Int32 day = 1; + bool is_ad = true; // AD -> true, BC -> false. - Pos perform(Pos cur, Pos end, Date & date) const - { - if (func) - return func(cur, end, date); - else + Int32 week = 1; // Week of year based on ISO week date, e.g: 27 + Int32 day_of_week = 1; // Day of week, Monday:1, Tuesday:2, ..., Sunday:7 + bool week_date_format = false; + + Int32 day_of_year = 1; + bool day_of_year_format = false; + + bool century_format = false; + + bool is_year_of_era = false; // Year of era cannot be zero or negative. + bool has_year = false; // Whether year was explicitly specified. + + Int32 hour = 0; + Int32 minute = 0; + Int32 second = 0; + // Int32 microsecond = 0; + bool is_am = true; // AM -> true, PM -> false + std::optional time_zone_offset = 0; + + bool is_clock_hour = false; // Whether most recent hour specifier is clockhour + bool is_hour_of_half_day = false; // Whether most recent hour specifier is of half day. + + std::vector day_of_month_values; + std::vector day_of_year_values; + + static bool isLeapYear(Int32 year_) { return year_ % 4 == 0 && (year_ % 100 != 0 || year_ % 400 == 0); } + + static bool isDateValid(Int32 year_, Int32 month_, Int32 day_) { - ensureSpace(cur, end, literal.size(), "requires size >= " + std::to_string(literal.size())); - if (std::string_view(cur, literal.size()) != literal) - throw Exception( - ErrorCodes::LOGICAL_ERROR, "Expect literal {} but {} provided", literal, std::string_view(cur, literal.size())); - cur += literal.size(); - return cur; + if (month_ < 1 || month_ > 12) + return false; + + if (year_ < minYear || year_ > maxYear) + return false; + + bool leap = isLeapYear(year_); + if (day_ < 1) + return false; + + if (leap && day_ > leapDays[month_]) + return false; + + if (!leap && day_ > normalDays[month_]) + return false; + return true; } - } - template - static Pos readNumber2(Pos cur, Pos end, T & res) - { - ensureSpace(cur, end, 2, "readNumber2 requires size >= 2"); - res = (*cur - '0') * 10; - ++cur; - res += *cur; - ++cur; - return cur; - } - - template - static Pos readNumber3(Pos cur, Pos end, T & res) - { - cur = readNumber2(cur, end, res); - - ensureSpace(cur, end, 1, "readNumber3 requires size >= 3"); - res = res * 10 + (*cur - '0'); - ++cur; - return cur; - } - - template - static Pos readNumber4(Pos cur, Pos end, T & res) - { - cur = readNumber2(cur, end, res); - - T tmp; - cur = readNumber2(cur, end, tmp); - res = res * 100 + tmp; - return cur; - } - - static ALWAYS_INLINE void ensureSpace(Pos cur, Pos end, size_t len, const String & msg) - { - if (cur > end || cur + len > end) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because {}", msg); - } - - static ALWAYS_INLINE Pos assertChar(Pos cur, Pos end, char ch) - { - ensureSpace(cur, end, 1, "assertChar requires size >= 1"); - - if (*cur != ch) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expect char {}, but {} provided", String(ch, 1), String(*cur, 1)); - - ++cur; - return cur; - } - - static Pos mysqlDayOfWeekTextShort(Pos cur, Pos end, Date & date) - { - ensureSpace(cur, end, 3, "Parsing DayOfWeekTextShort requires size >= 3"); - - String str(cur, 3); - Poco::toLower(str); - Int32 i = 0; - for (; i < 7; ++i) - if (str == weekdaysShort[i]) - break; - - if (i == 7) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown short week text"); - - date.day_of_week = i + 1; - date.week_date_format = true; - date.day_of_year_format = false; - if (!date.has_year) + static bool isDayOfYearValid(Int32 year_, Int32 day_of_year_) { - date.has_year = true; - date.year = 2000; + if (year_ < minYear || year_ > maxYear) + return false; + + if (day_of_year_ < 1 || day_of_year_ > 365 + (isLeapYear(year_) ? 1 : 0)) + return false; + + return true; } - cur += 3; - return cur; - } - static Pos mysqlMonthOfYearTextShort(Pos cur, Pos end, Date & date) - { - ensureSpace(cur, end, 3, "Parsing MonthOfYearTextShort requires size >= 3"); - - String str(cur, 3); - Poco::toLower(str); - - Int32 i = 0; - for (; i < 12; ++i) - if (str == monthsShort[i]) - break; - - if (i == 12) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown short month text"); - - date.month = i + 1; - cur += 3; - return cur; - } - - static Pos mysqlMonth(Pos cur, Pos end, Date & date) - { - return readNumber2(cur, end, date.month); - } - - static Pos mysqlCentury(Pos cur, Pos end, Date & date) - { - Int32 centuray; - cur = readNumber2(cur, end, centuray); - date.century_format = true; - date.year = centuray * 100; - date.has_year = true; - return cur; - } - - static Pos mysqlDayOfMonth(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.day); - date.day_of_month_values.push_back(date.day); - date.week_date_format = false; - date.day_of_year_format = false; - if (!date.has_year) + static bool isWeekDateValid(Int32 week_year_, Int32 week_of_year_, Int32 day_of_week_) { - date.has_year = true; - date.year = 2000; + if (day_of_week_ < 1 || day_of_week_ > 7) + return false; + + if (week_of_year_ < 1 || week_of_year_ > 52) + return false; + + if (week_year_ < minYear || week_year_ > maxYear) + return false; + + return true; } - return cur; - } - static Pos mysqlAmericanDate(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.month); - cur = assertChar(cur, end, '/'); - - cur = readNumber2(cur, end, date.day); - cur = assertChar(cur, end, '/'); - - cur = readNumber2(cur, end, date.year); - cur = assertChar(cur, end, '/'); - - date.week_date_format = false; - date.day_of_year_format = false; - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; - return cur; - } - - - static Pos mysqlDayOfMonthSpacePadded(Pos cur, Pos end, Date & date) - { - ensureSpace(cur, end, 2, "mysqlDayOfMonthSpacePadded requires size >= 2"); - - date.day = *cur == ' ' ? 0 : (*cur - '0'); - ++cur; - - date.day = 10 * date.day + (*cur - '0'); - ++cur; - - date.week_date_format = false; - date.day_of_year_format = false; - if (!date.has_year) + static Int32 extractISODayOfTheWeek(Int32 days_since_epoch) { - date.has_year = true; - date.year = 2000; - } - return cur; - } - - static Pos mysqlISO8601Date(Pos cur, Pos end, Date & date) - { - cur = readNumber4(cur, end, date.year); - cur = readNumber2(cur, end, date.month); - cur = readNumber2(cur, end, date.day); - - date.week_date_format = false; - date.day_of_year_format = false; - - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; - return cur; - } - - static Pos mysqlISO8601Year2(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.year); - date.year += 2000; - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; - return cur; - } - - static Pos mysqlISO8601Year4(Pos cur, Pos end, Date & date) - { - cur = readNumber4(cur, end, date.year); - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; - return cur; - } - - static Pos mysqlDayOfYear(Pos cur, Pos end, Date & date) - { - cur = readNumber3(cur, end, date.day_of_year); - - date.day_of_year_values.push_back(date.day_of_year); - date.day_of_year_format = true; - date.week_date_format = false; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } - return cur; - } - - static Pos mysqlDayOfWeek(Pos cur, Pos end, Date & date) - { - ensureSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1"); - - date.day_of_week = *cur - '0'; - date.week_date_format = true; - date.day_of_year_format = false; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } - return cur; - } - - static Pos mysqlISO8601Week(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.week); - date.week_date_format = true; - date.day_of_year_format = false; - if (date.has_year) - { - date.has_year = true; - date.year = 2000; - } - return cur; - } - - static Pos mysqlDayOfWeek0To6(Pos cur, Pos end, Date & date) - { - cur = mysqlDayOfWeek(cur, end, date); - if (date.day_of_week == 0) - date.day_of_week = 7; - - return cur; - } - - static Pos mysqlDayOfWeekTextLong(Pos cur, Pos end, Date & date) - { - mysqlDayOfWeekTextShort(cur, end, date); - auto expect_text = weekdaysFull[date.day_of_week - 1]; - - ensureSpace(cur, end, expect_text.size(), "mysqlDayOfWeekTextLong requires size >= " + std::to_string(expect_text.size())); - std::string_view text(cur, expect_text.size()); - if (text != expect_text) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown full day of week text {}", expect_text); - - cur += expect_text.size(); - return cur; - } - - static Pos mysqlYear2(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.year); - date.year += 2000; - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; - return cur; - } - - - static Pos mysqlYear4(Pos cur, Pos end, Date & date) - { - cur = readNumber4(cur, end, date.year); - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; - return cur; - } - - static Pos mysqlTimezoneOffset(Pos cur, Pos end, Date & date) - { - /// TODO figure out what timezone_id mean - ensureSpace(cur, end, 1, "Parse mysqlTimezoneOffset failed"); - Int32 sign = 1; - if (*cur == '-') - sign = -1; - ++cur; - - Int32 hour; - cur = readNumber2(cur, end, hour); - - Int32 minute; - cur = readNumber2(cur, end, minute); - - date.timezone_id = sign * (hour * 3600 + minute); - return cur; - } - - static Pos mysqlMinute(Pos cur, Pos end, Date & date) - { - return readNumber2(cur, end, date.minute); - } - - static Pos mysqlAMPM(Pos cur, Pos end, Date & date) - { - ensureSpace(cur, end, 2, "mysqlAMPM requires size >= 2"); - - std::string text(cur, 2); - Poco::toUpper(text); - if (text == "PM") - date.is_am = true; - else if (text == "AM") - date.is_am = false; - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Text should be AM or PM, but {} provided", text); - - cur += 2; - return cur; - } - - static Pos mysqlHHMM12(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.hour); - date.is_clock_hour = false; - date.is_hour_of_half_day = true; - - cur = assertChar(cur, end, ':'); - cur = readNumber2(cur, end, date.minute); - cur = assertChar(cur, end, ' '); - cur = mysqlAMPM(cur, end, date); - return cur; - } - - static Pos mysqlHHMM24(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.hour); - date.is_clock_hour = false; - date.is_hour_of_half_day = false; - - cur = assertChar(cur, end, ':'); - cur = readNumber2(cur, end, date.minute); - return cur; - } - - static Pos mysqlSecond(Pos cur, Pos end, Date & date) - { - return readNumber2(cur, end, date.second); - } - - static Pos mysqlISO8601Time(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.hour); - date.is_clock_hour = false; - date.is_hour_of_half_day = false; - - cur = assertChar(cur, end, ':'); - cur = readNumber2(cur, end, date.minute); - cur = assertChar(cur, end, ':'); - cur = readNumber2(cur, end, date.second); - return cur; - } - - static Pos mysqlHour12(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.hour); - date.is_hour_of_half_day = true; - date.is_clock_hour = false; - return cur; - } - - static Pos mysqlHour24(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.hour); - date.is_hour_of_half_day = false; - date.is_clock_hour = false; - return cur; - } -}; - - -struct ParseDateTimeTraits -{ - enum class ParseSyntax - { - MySQL, - Joda - }; -}; - - -/// _FUNC_(str[, format, timezone]) -template -class FunctionParseDateTimeImpl : public IFunction -{ -public: - static constexpr auto name = Name::name; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } - - String getName() const override { return name; } - - bool useDefaultImplementationForConstants() const override { return true; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - if (arguments.size() != 1 && arguments.size() != 2 && arguments.size() != 3) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be 1, 2 or 3", - getName(), - arguments.size()); - - if (!isString(arguments[0].type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of first argument of function {} when arguments size is 1. Should be string", - arguments[0].type->getName(), - getName()); - - if (arguments.size() > 1 && !isString(arguments[1].type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of second argument of function {} when arguments size is 1. Should be string", - arguments[0].type->getName(), - getName()); - - if (arguments.size() > 2 && !isString(arguments[2].type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of third argument of function {} when arguments size is 1. Should be string", - arguments[0].type->getName(), - getName()); - - return std::make_shared(); - } - - ColumnPtr executeImpl( - const ColumnsWithTypeAndName & arguments, - [[maybe_unused]] const DataTypePtr & result_type, - [[maybe_unused]] size_t input_rows_count) const override - { - const auto * col_str = checkAndGetColumn(arguments[0].column.get()); - if (!col_str) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of first ('str') argument of function {}. Must be string.", - arguments[0].column->getName(), getName()); - - String format = getFormat(arguments); - const auto & time_zone = getTimeZone(arguments); - - std::vector instructions; - parseFormat(format, instructions); - - for (size_t i = 0; igetDataAt(i); - Date date; - Pos cur = str_ref.data; - Pos end = str_ref.data + str_ref.size; - for (const auto & instruction: instructions) + if (days_since_epoch < 0) { - cur = instruction.perform(cur, end, date); + // negative date: start off at 4 and cycle downwards + return (7 - ((-int64_t(days_since_epoch) + 3) % 7)); } - - // Ensure all input was consumed. - if (cur < end) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Invalid format input {} is malformed at {}", - str_ref.toView(), - std::string_view(cur, end - cur)); - - // Era is BC and year of era is provided - if (date.is_year_of_era && !date.is_ad) - date.year = -1 * (date.year - 1); - - if (date.is_hour_of_half_day && !date.is_am) - date.hour += 12; - - // Ensure all day of month values are valid for ending month value - for (size_t i = 0; i < date.day_of_month_values.size(); ++i) + else { - if (!util::isValidDate(date.year, date.month, date.dayOfMonthValues[i])) - { - VELOX_USER_FAIL( - "Value {} for dayOfMonth must be in the range [1,{}]", - date.dayOfMonthValues[i], - util::getMaxDayOfMonth(date.year, date.month)); - } + // positive date: start off at 4 and cycle upwards + return ((int64_t(days_since_epoch) + 3) % 7) + 1; + } + } + + static Int32 daysSinceEpochFromWeekDate(int32_t week_year_, int32_t week_of_year_, int32_t day_of_week_) + { + if (!isWeekDateValid(week_year_, week_of_year_, day_of_week_)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid week date"); + + Int32 days_since_epoch_of_jan_fourth = daysSinceEpochFromDate(week_year_, 1, 4); + Int32 first_day_of_week_year = extractISODayOfTheWeek(days_since_epoch_of_jan_fourth); + return days_since_epoch_of_jan_fourth - (first_day_of_week_year - 1) + 7 * (week_of_year_ - 1) + day_of_week_ - 1; + } + + static Int32 daysSinceEpochFromDayOfYear(Int32 year_, Int32 day_of_year_) + { + if (!isDayOfYearValid(year_, day_of_year_)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of year"); + + Int32 res = daysSinceEpochFromDate(year_, 1, 1); + res += day_of_year_ - 1; + return res; + } + + static Int32 daysSinceEpochFromDate(Int32 year_, Int32 month_, Int32 day_) + { + if (!isDateValid(year_, month_, day_)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid date"); + + Int32 res = cumulativeYearDays[year_ - 1970]; + res += isLeapYear(year_) ? cumulativeLeapDays[month_ - 1] : cumulativeDays[month_ - 1]; + res += day_ - 1; + return res; + } + + + Int64 checkAndGetDateTime(const DateLUTImpl & time_zone) + { + /// Era is BC and year of era is provided + if (is_year_of_era && !is_ad) + year = -1 * (year - 1); + + if (is_hour_of_half_day && !is_am) + hour += 12; + + + /// Ensure all day of year values are valid for ending year value + for (const auto d : day_of_month_values) + { + if (!isDateValid(year, month, d)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of month."); } // Ensure all day of year values are valid for ending year value - for (int i = 0; i < date.dayOfYearValues.size(); i++) + for (const auto d : day_of_year_values) { - if (!util::isValidDayOfYear(date.year, date.dayOfYearValues[i])) + if (!isDayOfYearValid(year, d)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of year."); + } + + // Convert the parsed date/time into a timestamp. + Int32 days_since_epoch; + if (week_date_format) + days_since_epoch = daysSinceEpochFromWeekDate(year, week, day_of_week); + else if (day_of_year_format) + days_since_epoch = daysSinceEpochFromDayOfYear(year, day_of_year); + else + days_since_epoch = daysSinceEpochFromDate(year, month, day); + + Int64 seconds_since_epoch = days_since_epoch * 86400 + hour * 3600 + minute * 60 + second; + /// Time zone is not specified, use local time zone + if (!time_zone_offset) + *time_zone_offset = time_zone.timezoneOffset(seconds_since_epoch); + + /// Time zone is specified in format string. + seconds_since_epoch -= *time_zone_offset; + return seconds_since_epoch; + } + }; + + class Action + { + private: + using Func = Pos (*)(Pos cur, Pos end, Date & date); + Func func; + std::string literal; + + public: + explicit Action(Func && func_) : func(std::move(func_)) { } + + template + explicit Action(const Literal & literal_) : literal(literal_) + { + } + + Pos perform(Pos cur, Pos end, Date & date) const + { + if (func) + return func(cur, end, date); + else + { + ensureSpace(cur, end, literal.size(), "requires size >= " + std::to_string(literal.size())); + if (std::string_view(cur, literal.size()) != literal) + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Expect literal {} but {} provided", literal, std::string_view(cur, literal.size())); + cur += literal.size(); + return cur; + } + } + + template + static Pos readNumber2(Pos cur, Pos end, T & res) + { + ensureSpace(cur, end, 2, "readNumber2 requires size >= 2"); + res = (*cur - '0') * 10; + ++cur; + res += *cur; + ++cur; + return cur; + } + + template + static Pos readNumber3(Pos cur, Pos end, T & res) + { + cur = readNumber2(cur, end, res); + + ensureSpace(cur, end, 1, "readNumber3 requires size >= 3"); + res = res * 10 + (*cur - '0'); + ++cur; + return cur; + } + + template + static Pos readNumber4(Pos cur, Pos end, T & res) + { + cur = readNumber2(cur, end, res); + + T tmp; + cur = readNumber2(cur, end, tmp); + res = res * 100 + tmp; + return cur; + } + + static ALWAYS_INLINE void ensureSpace(Pos cur, Pos end, size_t len, const String & msg) + { + if (cur > end || cur + len > end) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because {}", msg); + } + + static ALWAYS_INLINE Pos assertChar(Pos cur, Pos end, char ch) + { + ensureSpace(cur, end, 1, "assertChar requires size >= 1"); + + if (*cur != ch) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expect char {}, but {} provided", String(ch, 1), String(*cur, 1)); + + ++cur; + return cur; + } + + static Pos mysqlDayOfWeekTextShort(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 3, "Parsing DayOfWeekTextShort requires size >= 3"); + + String str(cur, 3); + Poco::toLower(str); + Int32 i = 0; + for (; i < 7; ++i) + if (str == weekdaysShort[i]) + break; + + if (i == 7) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown short week text"); + + date.day_of_week = i + 1; + date.week_date_format = true; + date.day_of_year_format = false; + if (!date.has_year) + { + date.has_year = true; + date.year = 2000; + } + cur += 3; + return cur; + } + + static Pos mysqlMonthOfYearTextShort(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 3, "Parsing MonthOfYearTextShort requires size >= 3"); + + String str(cur, 3); + Poco::toLower(str); + + Int32 i = 0; + for (; i < 12; ++i) + if (str == monthsShort[i]) + break; + + if (i == 12) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown short month text"); + + date.month = i + 1; + cur += 3; + return cur; + } + + static Pos mysqlMonth(Pos cur, Pos end, Date & date) { return readNumber2(cur, end, date.month); } + + static Pos mysqlCentury(Pos cur, Pos end, Date & date) + { + Int32 centuray; + cur = readNumber2(cur, end, centuray); + date.century_format = true; + date.year = centuray * 100; + date.has_year = true; + return cur; + } + + static Pos mysqlDayOfMonth(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.day); + date.day_of_month_values.push_back(date.day); + date.week_date_format = false; + date.day_of_year_format = false; + if (!date.has_year) + { + date.has_year = true; + date.year = 2000; + } + return cur; + } + + static Pos mysqlAmericanDate(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.month); + cur = assertChar(cur, end, '/'); + + cur = readNumber2(cur, end, date.day); + cur = assertChar(cur, end, '/'); + + cur = readNumber2(cur, end, date.year); + cur = assertChar(cur, end, '/'); + + date.week_date_format = false; + date.day_of_year_format = false; + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; + return cur; + } + + + static Pos mysqlDayOfMonthSpacePadded(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 2, "mysqlDayOfMonthSpacePadded requires size >= 2"); + + date.day = *cur == ' ' ? 0 : (*cur - '0'); + ++cur; + + date.day = 10 * date.day + (*cur - '0'); + ++cur; + + date.week_date_format = false; + date.day_of_year_format = false; + if (!date.has_year) + { + date.has_year = true; + date.year = 2000; + } + return cur; + } + + static Pos mysqlISO8601Date(Pos cur, Pos end, Date & date) + { + cur = readNumber4(cur, end, date.year); + cur = readNumber2(cur, end, date.month); + cur = readNumber2(cur, end, date.day); + + date.week_date_format = false; + date.day_of_year_format = false; + + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; + return cur; + } + + static Pos mysqlISO8601Year2(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.year); + date.year += 2000; + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; + return cur; + } + + static Pos mysqlISO8601Year4(Pos cur, Pos end, Date & date) + { + cur = readNumber4(cur, end, date.year); + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; + return cur; + } + + static Pos mysqlDayOfYear(Pos cur, Pos end, Date & date) + { + cur = readNumber3(cur, end, date.day_of_year); + + date.day_of_year_values.push_back(date.day_of_year); + date.day_of_year_format = true; + date.week_date_format = false; + if (!date.has_year) + { + date.has_year = true; + date.year = 2000; + } + return cur; + } + + static Pos mysqlDayOfWeek(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1"); + + date.day_of_week = *cur - '0'; + date.week_date_format = true; + date.day_of_year_format = false; + if (!date.has_year) + { + date.has_year = true; + date.year = 2000; + } + return cur; + } + + static Pos mysqlISO8601Week(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.week); + date.week_date_format = true; + date.day_of_year_format = false; + if (date.has_year) + { + date.has_year = true; + date.year = 2000; + } + return cur; + } + + static Pos mysqlDayOfWeek0To6(Pos cur, Pos end, Date & date) + { + cur = mysqlDayOfWeek(cur, end, date); + if (date.day_of_week == 0) + date.day_of_week = 7; + + return cur; + } + + static Pos mysqlDayOfWeekTextLong(Pos cur, Pos end, Date & date) + { + mysqlDayOfWeekTextShort(cur, end, date); + auto expect_text = weekdaysFull[date.day_of_week - 1]; + + ensureSpace(cur, end, expect_text.size(), "mysqlDayOfWeekTextLong requires size >= " + std::to_string(expect_text.size())); + std::string_view text(cur, expect_text.size()); + if (text != expect_text) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown full day of week text {}", expect_text); + + cur += expect_text.size(); + return cur; + } + + static Pos mysqlYear2(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.year); + date.year += 2000; + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; + return cur; + } + + + static Pos mysqlYear4(Pos cur, Pos end, Date & date) + { + cur = readNumber4(cur, end, date.year); + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; + return cur; + } + + static Pos mysqlTimezoneOffset(Pos cur, Pos end, Date & date) + { + /// TODO figure out what timezone_id mean + ensureSpace(cur, end, 1, "Parse mysqlTimezoneOffset failed"); + Int32 sign = 1; + if (*cur == '-') + sign = -1; + ++cur; + + Int32 hour; + cur = readNumber2(cur, end, hour); + + Int32 minute; + cur = readNumber2(cur, end, minute); + + *date.time_zone_offset = sign * (hour * 3600 + minute * 60); + return cur; + } + + static Pos mysqlMinute(Pos cur, Pos end, Date & date) { return readNumber2(cur, end, date.minute); } + + static Pos mysqlAMPM(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 2, "mysqlAMPM requires size >= 2"); + + std::string text(cur, 2); + Poco::toUpper(text); + if (text == "PM") + date.is_am = true; + else if (text == "AM") + date.is_am = false; + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Text should be AM or PM, but {} provided", text); + + cur += 2; + return cur; + } + + static Pos mysqlHHMM12(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.hour); + date.is_clock_hour = false; + date.is_hour_of_half_day = true; + + cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, date.minute); + cur = assertChar(cur, end, ' '); + cur = mysqlAMPM(cur, end, date); + return cur; + } + + static Pos mysqlHHMM24(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.hour); + date.is_clock_hour = false; + date.is_hour_of_half_day = false; + + cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, date.minute); + return cur; + } + + static Pos mysqlSecond(Pos cur, Pos end, Date & date) { return readNumber2(cur, end, date.second); } + + static Pos mysqlISO8601Time(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.hour); + date.is_clock_hour = false; + date.is_hour_of_half_day = false; + + cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, date.minute); + cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, date.second); + return cur; + } + + static Pos mysqlHour12(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.hour); + date.is_hour_of_half_day = true; + date.is_clock_hour = false; + return cur; + } + + static Pos mysqlHour24(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.hour); + date.is_hour_of_half_day = false; + date.is_clock_hour = false; + return cur; + } + }; + + + struct ParseDateTimeTraits + { + enum class ParseSyntax + { + MySQL, + Joda + }; + }; + + + /// _FUNC_(str[, format, timezone]) + template + class FunctionParseDateTimeImpl : public IFunction + { + public: + static constexpr auto name = Name::name; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.size() != 1 && arguments.size() != 2 && arguments.size() != 3) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 1, 2 or 3", + getName(), + arguments.size()); + + if (!isString(arguments[0].type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {} when arguments size is 1. Should be string", + arguments[0].type->getName(), + getName()); + + if (arguments.size() > 1 && !isString(arguments[1].type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {} when arguments size is 1. Should be string", + arguments[0].type->getName(), + getName()); + + if (arguments.size() > 2 && !isString(arguments[2].type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of third argument of function {} when arguments size is 1. Should be string", + arguments[0].type->getName(), + getName()); + + String time_zone_name = getTimeZone(arguments).second; + return std::make_shared(time_zone_name); + } + + ColumnPtr executeImpl( + const ColumnsWithTypeAndName & arguments, + [[maybe_unused]] const DataTypePtr & result_type, + [[maybe_unused]] size_t input_rows_count) const override + { + const auto * col_str = checkAndGetColumn(arguments[0].column.get()); + if (!col_str) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of first ('str') argument of function {}. Must be string.", + arguments[0].column->getName(), + getName()); + + String format = getFormat(arguments); + const auto * time_zone = getTimeZone(arguments).first; + + std::vector instructions; + parseFormat(format, instructions); + + auto col_res = ColumnDateTime::create(); + col_res->reserve(input_rows_count); + auto & data_res = col_res->getData(); + for (size_t i = 0; i < input_rows_count; ++i) + { + StringRef str_ref = col_str->getDataAt(i); + Date date; + Pos cur = str_ref.data; + Pos end = str_ref.data + str_ref.size; + for (const auto & instruction : instructions) + cur = instruction.perform(cur, end, date); + + // Ensure all input was consumed. + if (cur < end) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Invalid format input {} is malformed at {}", + str_ref.toView(), + std::string_view(cur, end - cur)); + + auto time = date.checkAndGetDateTime(*time_zone); + data_res.push_back(time); + } + + return col_res; + } + + + private: + void parseFormat(const String & format, std::vector & instructions) + { + if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::MySQL) + parseMysqlFormat(format, instructions); + else if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::Joda) + parseJodaFormat(format, instructions); + else + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, + "Unknown datetime format style {} in function {}", + magic_enum::enum_name(parse_syntax), + getName()); + } + + void parseMysqlFormat(const String & format, std::vector & instructions) + { + Pos pos = format.data(); + Pos end = pos + format.size(); + while (true) + { + Pos percent_pos = find_first_symbols<'%'>(pos, end); + if (percent_pos < end) { - VELOX_USER_FAIL( - "Value {} for dayOfMonth must be in the range [1,{}]", - date.dayOfYearValues[i], - util::isLeapYear(date.year) ? 366 : 365); + if (pos < percent_pos) + instructions.emplace_back(std::string_view(pos, percent_pos - pos)); + + pos = percent_pos + 1; + if (pos >= end) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Sign '%' is the last in format, if you need it, use '%%'"); + + switch (*pos) + { + // Abbreviated weekday [Mon...Sun] + case 'a': + instructions.emplace_back(&Action::mysqlDayOfWeekTextShort); + break; + + // Abbreviated month [Jan...Dec] + case 'b': + instructions.emplace_back(&Action::mysqlMonthOfYearTextShort); + break; + + // Month as a decimal number (01-12) + case 'c': + instructions.emplace_back(&Action::mysqlMonth); + break; + + // Year, divided by 100, zero-padded + case 'C': + instructions.emplace_back(&Action::mysqlCentury); + break; + + // Day of month, zero-padded (01-31) + case 'd': + instructions.emplace_back(&Action::mysqlDayOfMonth); + break; + + // Short MM/DD/YY date, equivalent to %m/%d/%y + case 'D': + instructions.emplace_back(&Action::mysqlAmericanDate); + break; + + // Day of month, space-padded ( 1-31) 23 + case 'e': + instructions.emplace_back(&Action::mysqlDayOfMonthSpacePadded); + break; + + // Fractional seconds + case 'f': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for fractional seconds"); + + // Short YYYY-MM-DD date, equivalent to %Y-%m-%d 2001-08-23 + case 'F': + instructions.emplace_back(&Action::mysqlISO8601Date); + break; + + // Last two digits of year of ISO 8601 week number (see %G) + case 'g': + instructions.emplace_back(&Action::mysqlISO8601Year2); + break; + + // Year of ISO 8601 week number (see %V) + case 'G': + instructions.emplace_back(&Action::mysqlISO8601Year4); + break; + + // Day of the year (001-366) 235 + case 'j': + instructions.emplace_back(&Action::mysqlDayOfYear); + break; + + // Month as a decimal number (01-12) + case 'm': + instructions.emplace_back(&Action::mysqlMonth); + break; + + // ISO 8601 weekday as number with Monday as 1 (1-7) + case 'u': + instructions.emplace_back(&Action::mysqlDayOfWeek); + break; + + // ISO 8601 week number (01-53) + case 'V': + instructions.emplace_back(&Action::mysqlISO8601Week); + break; + + // Weekday as a decimal number with Sunday as 0 (0-6) 4 + case 'w': + instructions.emplace_back(&Action::mysqlDayOfWeek0To6); + break; + + // Full weekday [Monday...Sunday] + case 'W': + instructions.emplace_back(&Action::mysqlDayOfWeekTextLong); + break; + + // Two digits year + case 'y': + instructions.emplace_back(&Action::mysqlYear2); + break; + + // Four digits year + case 'Y': + instructions.emplace_back(&Action::mysqlYear4); + break; + + // Quarter (1-4) + case 'Q': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for quarter"); + break; + + // Offset from UTC timezone as +hhmm or -hhmm + case 'z': + instructions.emplace_back(&Action::mysqlTimezoneOffset); + break; + + /// Time components. If the argument is Date, not a DateTime, then this components will have default value. + + // Minute (00-59) + case 'M': + instructions.emplace_back(&Action::mysqlMinute); + break; + + // AM or PM + case 'p': + instructions.emplace_back(&Action::mysqlAMPM); + break; + + // 12-hour HH:MM time, equivalent to %h:%i %p 2:55 PM + case 'r': + instructions.emplace_back(&Action::mysqlHHMM12); + break; + + // 24-hour HH:MM time, equivalent to %H:%i 14:55 + case 'R': + instructions.emplace_back(&Action::mysqlHHMM24); + break; + + // Seconds + case 's': + instructions.emplace_back(&Action::mysqlSecond); + break; + + // Seconds + case 'S': + instructions.emplace_back(&Action::mysqlSecond); + break; + + // ISO 8601 time format (HH:MM:SS), equivalent to %H:%i:%S 14:55:02 + case 'T': + instructions.emplace_back(&Action::mysqlISO8601Time); + break; + + // Hour in 12h format (01-12) + case 'h': + instructions.emplace_back(&Action::mysqlHour12); + break; + + // Hour in 24h format (00-23) + case 'H': + instructions.emplace_back(&Action::mysqlHour24); + break; + + // Minute of hour range [0, 59] + case 'i': + instructions.emplace_back(&Action::mysqlMinute); + break; + + // Hour in 12h format (01-12) + case 'I': + instructions.emplace_back(&Action::mysqlHour12); + break; + + // Hour in 24h format (00-23) + case 'k': + instructions.emplace_back(&Action::mysqlHour24); + break; + + // Hour in 12h format (01-12) + case 'l': + instructions.emplace_back(&Action::mysqlHour12); + break; + + case 't': + instructions.emplace_back("\t"); + break; + + case 'n': + instructions.emplace_back("\n"); + break; + + // Escaped literal characters. + case '%': + instructions.emplace_back("\n"); + break; + + // Unimplemented + case 'U': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for WEEK (Sun-Sat)"); + case 'v': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for WEEK (Mon-Sun)"); + case 'x': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for YEAR for week (Mon-Sun)"); + case 'X': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for YEAR for week (Sun-Sat)"); + + default: + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Incorrect syntax '{}', symbol is not supported '{}' for function {}", + format, + *pos, + getName()); + } + + ++pos; + } + else + { + instructions.emplace_back(std::string_view(pos, end - pos)); + break; } } } - } + + void parseJodaFormat(const String & /*format*/, std::vector & /*instructions*/) { } -private: - - void parseFormat(const String & format, std::vector & instructions) - { - if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::MySQL) - parseMysqlFormat(format, instructions); - else if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::Joda) - parseJodaFormat(format, instructions); - else - throw Exception( - ErrorCodes::NOT_IMPLEMENTED, - "Unknown datetime format style {} in function {}", - magic_enum::enum_name(parse_syntax), - getName()); - } - - void parseMysqlFormat(const String & format, std::vector & instructions) - { - Pos pos = format.data(); - Pos end = pos + format.size(); - while (true) + String getFormat(const ColumnsWithTypeAndName & arguments) const { - Pos percent_pos = find_first_symbols<'%'>(pos, end); - if (percent_pos < end) + if (arguments.size() < 2) { - if (pos < percent_pos) - instructions.emplace_back(std::string_view(pos, percent_pos - pos)); - - pos = percent_pos + 1; - if (pos >= end) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Sign '%' is the last in format, if you need it, use '%%'"); - - switch (*pos) - { - // Abbreviated weekday [Mon...Sun] - case 'a': - instructions.emplace_back(&Action::mysqlDayOfWeekTextShort); - break; - - // Abbreviated month [Jan...Dec] - case 'b': - instructions.emplace_back(&Action::mysqlMonthOfYearTextShort); - break; - - // Month as a decimal number (01-12) - case 'c': - instructions.emplace_back(&Action::mysqlMonth); - break; - - // Year, divided by 100, zero-padded - case 'C': - instructions.emplace_back(&Action::mysqlCentury); - break; - - // Day of month, zero-padded (01-31) - case 'd': - instructions.emplace_back(&Action::mysqlDayOfMonth); - break; - - // Short MM/DD/YY date, equivalent to %m/%d/%y - case 'D': - instructions.emplace_back(&Action::mysqlAmericanDate); - break; - - // Day of month, space-padded ( 1-31) 23 - case 'e': - instructions.emplace_back(&Action::mysqlDayOfMonthSpacePadded); - break; - - // Fractional seconds - case 'f': - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for fractional seconds"); - - // Short YYYY-MM-DD date, equivalent to %Y-%m-%d 2001-08-23 - case 'F': - instructions.emplace_back(&Action::mysqlISO8601Date); - break; - - // Last two digits of year of ISO 8601 week number (see %G) - case 'g': - instructions.emplace_back(&Action::mysqlISO8601Year2); - break; - - // Year of ISO 8601 week number (see %V) - case 'G': - instructions.emplace_back(&Action::mysqlISO8601Year4); - break; - - // Day of the year (001-366) 235 - case 'j': - instructions.emplace_back(&Action::mysqlDayOfYear); - break; - - // Month as a decimal number (01-12) - case 'm': - instructions.emplace_back(&Action::mysqlMonth); - break; - - // ISO 8601 weekday as number with Monday as 1 (1-7) - case 'u': - instructions.emplace_back(&Action::mysqlDayOfWeek); - break; - - // ISO 8601 week number (01-53) - case 'V': - instructions.emplace_back(&Action::mysqlISO8601Week); - break; - - // Weekday as a decimal number with Sunday as 0 (0-6) 4 - case 'w': - instructions.emplace_back(&Action::mysqlDayOfWeek0To6); - break; - - // Full weekday [Monday...Sunday] - case 'W': - instructions.emplace_back(&Action::mysqlDayOfWeekTextLong); - break; - - // Two digits year - case 'y': - instructions.emplace_back(&Action::mysqlYear2); - break; - - // Four digits year - case 'Y': - instructions.emplace_back(&Action::mysqlYear4); - break; - - // Quarter (1-4) - case 'Q': - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for quarter"); - break; - - // Offset from UTC timezone as +hhmm or -hhmm - case 'z': - instructions.emplace_back(&Action::mysqlTimezoneOffset); - break; - - /// Time components. If the argument is Date, not a DateTime, then this components will have default value. - - // Minute (00-59) - case 'M': - instructions.emplace_back(&Action::mysqlMinute); - break; - - // AM or PM - case 'p': - instructions.emplace_back(&Action::mysqlAMPM); - break; - - // 12-hour HH:MM time, equivalent to %h:%i %p 2:55 PM - case 'r': - instructions.emplace_back(&Action::mysqlHHMM12); - break; - - // 24-hour HH:MM time, equivalent to %H:%i 14:55 - case 'R': - instructions.emplace_back(&Action::mysqlHHMM24); - break; - - // Seconds - case 's': - instructions.emplace_back(&Action::mysqlSecond); - break; - - // Seconds - case 'S': - instructions.emplace_back(&Action::mysqlSecond); - break; - - // ISO 8601 time format (HH:MM:SS), equivalent to %H:%i:%S 14:55:02 - case 'T': - instructions.emplace_back(&Action::mysqlISO8601Time); - break; - - // Hour in 12h format (01-12) - case 'h': - instructions.emplace_back(&Action::mysqlHour12); - break; - - // Hour in 24h format (00-23) - case 'H': - instructions.emplace_back(&Action::mysqlHour24); - break; - - // Minute of hour range [0, 59] - case 'i': - instructions.emplace_back(&Action::mysqlMinute); - break; - - // Hour in 12h format (01-12) - case 'I': - instructions.emplace_back(&Action::mysqlHour12); - break; - - // Hour in 24h format (00-23) - case 'k': - instructions.emplace_back(&Action::mysqlHour24); - break; - - // Hour in 12h format (01-12) - case 'l': - instructions.emplace_back(&Action::mysqlHour12); - break; - - case 't': - instructions.emplace_back("\t"); - break; - - case 'n': - instructions.emplace_back("\n"); - break; - - // Escaped literal characters. - case '%': - instructions.emplace_back("\n"); - break; - - // Unimplemented - case 'U': - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for WEEK (Sun-Sat)"); - case 'v': - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for WEEK (Mon-Sun)"); - case 'x': - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for YEAR for week (Mon-Sun)"); - case 'X': - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for YEAR for week (Sun-Sat)"); - - default: - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Incorrect syntax '{}', symbol is not supported '{}' for function {}", - format, - *pos, - getName()); - } - - ++pos; + if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::Joda) + return "yyyy-MM-dd HH:mm:ss"; + else + return "%F %T"; } - else - { - instructions.emplace_back(std::string_view(pos, end - pos)); - break; - } - } - } - void parseJodaFormat(const String & /*format*/, std::vector & /*instructions*/) { } - - - String getFormat(const ColumnsWithTypeAndName & arguments) const - { - if (arguments.size() < 2) - { - if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::Joda) - return "yyyy-MM-dd HH:mm:ss"; - else - return "%F %T"; + const auto * format_column = checkAndGetColumnConst(arguments[1].column.get()); + if (!format_column) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of second ('format') argument of function {}. Must be constant string.", + arguments[1].column->getName(), + getName()); + return format_column->getValue(); } - const auto * format_column = checkAndGetColumnConst(arguments[1].column.get()); - if (!format_column) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of second ('format') argument of function {}. Must be constant string.", - arguments[1].column->getName(), getName()); - return format_column->getValue(); - } + std::pair getTimeZone(const ColumnsWithTypeAndName & arguments) const + { + if (arguments.size() < 3) + return {&DateLUT::instance(), ""}; - const DateLUTImpl & getTimeZone(const ColumnsWithTypeAndName & arguments) const - { - if (arguments.size() < 3) - return DateLUT::instance(); + const auto * col = checkAndGetColumnConst(arguments[2].column.get()); + if (!col) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of third ('timezone') argument of function {}. Must be constant string.", + arguments[2].column->getName(), + getName()); - const auto * col = checkAndGetColumnConst(arguments[2].column.get()); - if (!col) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of third ('timezone') argument of function {}. Must be constant string.", - arguments[2].column->getName(), getName()); - - String time_zone = col->getValue(); - if (time_zone.empty()) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Provided time zone must be non-empty and be a valid time zone"); - return DateLUT::instance(time_zone); - } -}; + String time_zone = col->getValue(); + if (time_zone.empty()) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Provided time zone must be non-empty and be a valid time zone"); + return {&DateLUT::instance(time_zone), time_zone}; + } + }; } From c63bbbb37401ff7cef7cd2b3355abcc2001b5d3f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 23 Feb 2023 20:22:12 +0000 Subject: [PATCH 054/559] Refactor a bit more. --- .../Optimizations/optimizeUseProjections.cpp | 273 ++++++++++-------- 1 file changed, 153 insertions(+), 120 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index ea81208df8e..ce7818a3298 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -257,7 +257,7 @@ bool areAggregatesMatch( ActionsDAGPtr analyzeAggregateProjection( const AggregateProjectionInfo & info, - ActionsDAG & query_dag, + const ActionsDAG & query_dag, const ActionsDAG::Node * filter_node, const Names & keys, const AggregateDescriptions & aggregates) @@ -411,7 +411,7 @@ static void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expressi /// This function builds a common DAG which is a merge of DAGs from Filter and Expression steps chain. /// Additionally, for all the Filter steps, we collect filter conditions into filter_nodes. /// Flag need_remove_column is set in case if the last step is a Filter step and it should remove filter column. -static bool buildAggregatingDAG( +static bool buildQueryDAG( QueryPlan::Node & node, ActionsDAGPtr & dag, ActionsDAG::NodeRawConstPtrs & filter_nodes, @@ -448,7 +448,7 @@ static bool buildAggregatingDAG( if (node.children.size() != 1) return false; - if (!buildAggregatingDAG(*node.children.front(), dag, filter_nodes, need_remove_column)) + if (!buildQueryDAG(*node.children.front(), dag, filter_nodes, need_remove_column)) return false; if (auto * expression = typeid_cast(step)) @@ -503,6 +503,131 @@ bool canUseProjectionForReadingStep(ReadFromMergeTree * reading) return true; } +struct MinMaxProjectionCandidate +{ + AggregateProjectionCandidate candidate; + Block minmax_count_projection_block; + MergeTreeData::DataPartsVector minmax_projection_normal_parts; +}; + +struct AggregateProjectionCandidates +{ + std::vector real; + std::optional minmax_projection; + bool has_filter = false; +}; + +AggregateProjectionCandidates getAggregateProjectionCandidates( + QueryPlan::Node & node, + AggregatingStep & aggregating, + ReadFromMergeTree & reading, + const std::shared_ptr & max_added_blocks) +{ + const auto & keys = aggregating.getParams().keys; + const auto & aggregates = aggregating.getParams().aggregates; + Block key_virtual_columns = reading.getMergeTreeData().getSampleBlockWithVirtualColumns(); + + AggregateProjectionCandidates candidates; + + const auto & parts = reading.getParts(); + const auto & query_info = reading.getQueryInfo(); + + const auto metadata = reading.getStorageMetadata(); + ContextPtr context = reading.getContext(); + + const auto & projections = metadata->projections; + std::vector agg_projections; + for (const auto & projection : projections) + if (projection.type == ProjectionDescription::Type::Aggregate) + agg_projections.push_back(&projection); + + bool can_use_minmax_projection = metadata->minmax_count_projection && !reading.getMergeTreeData().has_lightweight_delete_parts.load(); + + if (!can_use_minmax_projection && agg_projections.empty()) + return candidates; + + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Has agg projection"); + + ActionsDAGPtr dag; + bool need_remove_column = false; // not used here + ActionsDAG::NodeRawConstPtrs filter_nodes; + if (!buildQueryDAG(*node.children.front(), dag, filter_nodes, need_remove_column)) + return candidates; + + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", dag->dumpDAG()); + + const ActionsDAG::Node * filter_node = nullptr; + if (!filter_nodes.empty()) + { + filter_node = filter_nodes.front(); + if (filter_nodes.size() > 1) + { + FunctionOverloadResolverPtr func_builder_and = + std::make_unique( + std::make_shared()); + + filter_node = &dag->addFunction(func_builder_and, std::move(filter_nodes), {}); + } + + dag->getOutputs().push_back(filter_node); + } + candidates.has_filter = filter_node; + + if (can_use_minmax_projection) + { + const auto * projection = &*(metadata->minmax_count_projection); + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try projection {}", projection->name); + auto info = getAggregatingProjectionInfo(*projection, context, metadata, key_virtual_columns); + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection DAG {}", info.before_aggregation->dumpDAG()); + if (auto proj_dag = analyzeAggregateProjection(info, *dag, filter_node, keys, aggregates)) + { + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); + AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(proj_dag)}; + MergeTreeData::DataPartsVector minmax_projection_normal_parts; + + auto block = reading.getMergeTreeData().getMinMaxCountProjectionBlock( + metadata, + candidate.dag->getRequiredColumnsNames(), + filter_node != nullptr, + query_info, + parts, + minmax_projection_normal_parts, + max_added_blocks.get(), + context); + + if (block) + { + MinMaxProjectionCandidate minmax; + minmax.candidate = std::move(candidate); + minmax.minmax_count_projection_block = std::move(block); + minmax.minmax_projection_normal_parts = std::move(minmax_projection_normal_parts); + minmax.candidate.projection = projection; + candidates.minmax_projection.emplace(std::move(minmax)); + } + } + } + + if (!candidates.minmax_projection) + { + candidates.real.reserve(agg_projections.size()); + for (const auto * projection : agg_projections) + { + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try projection {}", projection->name); + auto info = getAggregatingProjectionInfo(*projection, context, metadata, key_virtual_columns); + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection DAG {}", info.before_aggregation->dumpDAG()); + if (auto proj_dag = analyzeAggregateProjection(info, *dag, filter_node, keys, aggregates)) + { + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); + AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(proj_dag)}; + candidate.projection = projection; + candidates.real.emplace_back(std::move(candidate)); + } + } + } + + return candidates; +} + bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { if (node.children.size() != 1) @@ -527,123 +652,28 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) return false; const auto metadata = reading->getStorageMetadata(); - const auto & projections = metadata->projections; - - bool can_use_minmax_projection = metadata->minmax_count_projection && !reading->getMergeTreeData().has_lightweight_delete_parts.load(); - - std::vector agg_projections; - for (const auto & projection : projections) - if (projection.type == ProjectionDescription::Type::Aggregate) - agg_projections.push_back(&projection); - - if (!can_use_minmax_projection && agg_projections.empty()) - return false; - - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Has agg projection"); - - ActionsDAGPtr dag; - bool need_remove_column = false; - ActionsDAG::NodeRawConstPtrs filter_nodes; - if (!buildAggregatingDAG(*node.children.front(), dag, filter_nodes, need_remove_column)) - return false; - - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", dag->dumpDAG()); - - const ActionsDAG::Node * filter_node = nullptr; - if (!filter_nodes.empty()) - { - filter_node = filter_nodes.front(); - if (filter_nodes.size() > 1) - { - FunctionOverloadResolverPtr func_builder_and = - std::make_unique( - std::make_shared()); - - filter_node = &dag->addFunction(func_builder_and, std::move(filter_nodes), {}); - } - - dag->getOutputs().push_back(filter_node); - } - ContextPtr context = reading->getContext(); - const auto & keys = aggregating->getParams().keys; - const auto & aggregates = aggregating->getParams().aggregates; - Block key_virtual_columns = reading->getMergeTreeData().getSampleBlockWithVirtualColumns(); - - std::vector candidates; - std::optional minmax_projection; - Block minmax_count_projection_block; - MergeTreeData::DataPartsVector minmax_projection_normal_parts; - - const auto & parts = reading->getParts(); - const auto & query_info = reading->getQueryInfo(); - auto query_info_copy = query_info; - query_info_copy.prewhere_info = nullptr; std::shared_ptr max_added_blocks; - - AggregateProjectionCandidate * best_candidate = nullptr; - - if (can_use_minmax_projection) - { - const auto * projection = &*(metadata->minmax_count_projection); - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try projection {}", projection->name); - auto info = getAggregatingProjectionInfo(*projection, context, metadata, key_virtual_columns); - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection DAG {}", info.before_aggregation->dumpDAG()); - if (auto proj_dag = analyzeAggregateProjection(info, *dag, filter_node, keys, aggregates)) - { - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); - minmax_projection.emplace(AggregateProjectionCandidate{.info = std::move(info), .dag = std::move(proj_dag)}); - minmax_projection->projection = projection; - - minmax_count_projection_block = reading->getMergeTreeData().getMinMaxCountProjectionBlock( - metadata, - minmax_projection->dag->getRequiredColumnsNames(), - filter_node != nullptr, - query_info, - parts, - minmax_projection_normal_parts, - max_added_blocks.get(), - context); - - if (!minmax_count_projection_block) - minmax_projection.reset(); - else - best_candidate = &*minmax_projection; - } - } - - if (!minmax_projection) - { - candidates.reserve(agg_projections.size()); - for (const auto * projection : agg_projections) - { - - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try projection {}", projection->name); - auto info = getAggregatingProjectionInfo(*projection, context, metadata, key_virtual_columns); - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection DAG {}", info.before_aggregation->dumpDAG()); - if (auto proj_dag = analyzeAggregateProjection(info, *dag, filter_node, keys, aggregates)) - { - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); - AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(proj_dag)}; - candidate.projection = projection; - candidates.emplace_back(std::move(candidate)); - } - } - - if (candidates.empty()) - return false; - } - - MergeTreeDataSelectExecutor reader(reading->getMergeTreeData()); - if (context->getSettingsRef().select_sequential_consistency) { if (const StorageReplicatedMergeTree * replicated = dynamic_cast(&reading->getMergeTreeData())) max_added_blocks = std::make_shared(replicated->getMaxAddedBlocks()); } - for (auto & candidate : candidates) + auto candidates = getAggregateProjectionCandidates(node, *aggregating, *reading, max_added_blocks); + + AggregateProjectionCandidate * best_candidate = nullptr; + if (candidates.minmax_projection) + best_candidate = &candidates.minmax_projection->candidate; + else if (candidates.real.empty()) + return false; + + MergeTreeDataSelectExecutor reader(reading->getMergeTreeData()); + const auto & parts = reading->getParts(); + const auto & query_info = reading->getQueryInfo(); + + for (auto & candidate : candidates.real) { MergeTreeData::DataPartsVector projection_parts; MergeTreeData::DataPartsVector normal_parts; @@ -661,7 +691,7 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) continue; ActionDAGNodes added_filter_nodes; - if (filter_node) + if (candidates.has_filter) added_filter_nodes.nodes.push_back(candidate.dag->getOutputs().front()); auto projection_result_ptr = reader.estimateNumMarksToRead( @@ -700,22 +730,22 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) best_candidate = &candidate; } - if (!best_candidate && !minmax_projection) + if (!best_candidate) return false; QueryPlanStepPtr projection_reading; bool has_nornal_parts; - if (minmax_projection) + if (candidates.minmax_projection) { - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Minmax proj block {}", minmax_count_projection_block.dumpStructure()); + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Minmax proj block {}", candidates.minmax_projection->minmax_count_projection_block.dumpStructure()); - Pipe pipe(std::make_shared(std::move(minmax_count_projection_block))); + Pipe pipe(std::make_shared(std::move(candidates.minmax_projection->minmax_count_projection_block))); projection_reading = std::make_unique(std::move(pipe)); - has_nornal_parts = !minmax_projection_normal_parts.empty(); + has_nornal_parts = !candidates.minmax_projection->minmax_projection_normal_parts.empty(); if (has_nornal_parts) - reading->resetParts(std::move(minmax_projection_normal_parts)); + reading->resetParts(std::move(candidates.minmax_projection->minmax_projection_normal_parts)); } else { @@ -726,6 +756,9 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}", proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString()); + auto query_info_copy = query_info; + query_info_copy.prewhere_info = nullptr; + projection_reading = reader.readFromParts( {}, best_candidate->dag->getRequiredColumnsNames(), @@ -756,7 +789,7 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) auto & projection_reading_node = nodes.emplace_back(QueryPlan::Node{.step = std::move(projection_reading)}); auto & expr_or_filter_node = nodes.emplace_back(); - if (filter_node) + if (candidates.has_filter) { expr_or_filter_node.step = std::make_unique( projection_reading_node.step->getOutputStream(), @@ -833,7 +866,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) ActionsDAGPtr dag; ActionsDAG::NodeRawConstPtrs filter_nodes; bool need_remove_column = false; - if (!buildAggregatingDAG(*iter->node->children.front(), dag, filter_nodes, need_remove_column)) + if (!buildQueryDAG(*iter->node->children.front(), dag, filter_nodes, need_remove_column)) return false; const ActionsDAG::Node * filter_node = nullptr; From 203048eedbcbb1362556f3e9600c793d7cbe5a27 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 24 Feb 2023 17:07:27 +0800 Subject: [PATCH 055/559] wip --- src/Functions/parseDateTime.cpp | 1802 ++++++++++++++++--------------- 1 file changed, 909 insertions(+), 893 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 89a4716b6cd..d81eef7e162 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -13,1011 +13,1027 @@ #include #include +#include +#pragma GCC diagnostic ignored "-Wunneeded-member-function" namespace DB { + namespace { - using Pos = const char *; +using Pos = const char *; - constexpr std::string_view weekdaysShort[] = {"sun", "mon", "tue", "wed", "thu", "fri", "sat"}; - constexpr std::string_view weekdaysFull[] = {"sunday", "monday", "tuesday", "wednesday", "thursday", "friday", "saturday"}; - constexpr std::string_view monthsShort[] = {"jan", "feb", "mar", "apr", "may", "jun", "jul", "aug", "sep", "oct", "nov", "dec"}; +constexpr std::string_view weekdaysShort[] = {"sun", "mon", "tue", "wed", "thu", "fri", "sat"}; +constexpr std::string_view weekdaysFull[] = {"sunday", "monday", "tuesday", "wednesday", "thursday", "friday", "saturday"}; +constexpr std::string_view monthsShort[] = {"jan", "feb", "mar", "apr", "may", "jun", "jul", "aug", "sep", "oct", "nov", "dec"}; - constexpr Int32 leapDays[] = {0, 31, 29, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}; - constexpr Int32 normalDays[] = {0, 31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}; +constexpr Int32 leapDays[] = {0, 31, 29, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}; +constexpr Int32 normalDays[] = {0, 31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}; - constexpr Int32 cumulativeLeapDays[] = {0, 31, 60, 91, 121, 152, 182, 213, 244, 274, 305, 335, 366}; - constexpr Int32 cumulativeDays[] = {0, 31, 59, 90, 120, 151, 181, 212, 243, 273, 304, 334, 365}; - constexpr Int32 cumulativeYearDays[] - = {0, 365, 730, 1096, 1461, 1826, 2191, 2557, 2922, 3287, 3652, 4018, 4383, 4748, 5113, 5479, 5844, 6209, - 6574, 6940, 7305, 7670, 8035, 8401, 8766, 9131, 9496, 9862, 10227, 10592, 10957, 11323, 11688, 12053, 12418, 12784, - 13149, 13514, 13879, 14245, 14610, 14975, 15340, 15706, 16071, 16436, 16801, 17167, 17532, 17897, 18262, 18628, 18993, 19358, - 19723, 20089, 20454, 20819, 21184, 21550, 21915, 22280, 22645, 23011, 23376, 23741, 24106, 24472, 24837, 25202, 25567, 25933, - 26298, 26663, 27028, 27394, 27759, 28124, 28489, 28855, 29220, 29585, 29950, 30316, 30681, 31046, 31411, 31777, 32142, 32507, - 32872, 33238, 33603, 33968, 34333, 34699, 35064, 35429, 35794, 36160, 36525, 36890, 37255, 37621, 37986, 38351, 38716, 39082, - 39447, 39812, 40177, 40543, 40908, 41273, 41638, 42004, 42369, 42734, 43099, 43465, 43830, 44195, 44560, 44926, 45291, 45656, - 46021, 46387, 46752, 47117, 47482, 47847, 48212, 48577, 48942, 49308, 49673}; +constexpr Int32 cumulativeLeapDays[] = {0, 31, 60, 91, 121, 152, 182, 213, 244, 274, 305, 335, 366}; +constexpr Int32 cumulativeDays[] = {0, 31, 59, 90, 120, 151, 181, 212, 243, 273, 304, 334, 365}; +constexpr Int32 cumulativeYearDays[] + = {0, 365, 730, 1096, 1461, 1826, 2191, 2557, 2922, 3287, 3652, 4018, 4383, 4748, 5113, 5479, 5844, 6209, + 6574, 6940, 7305, 7670, 8035, 8401, 8766, 9131, 9496, 9862, 10227, 10592, 10957, 11323, 11688, 12053, 12418, 12784, + 13149, 13514, 13879, 14245, 14610, 14975, 15340, 15706, 16071, 16436, 16801, 17167, 17532, 17897, 18262, 18628, 18993, 19358, + 19723, 20089, 20454, 20819, 21184, 21550, 21915, 22280, 22645, 23011, 23376, 23741, 24106, 24472, 24837, 25202, 25567, 25933, + 26298, 26663, 27028, 27394, 27759, 28124, 28489, 28855, 29220, 29585, 29950, 30316, 30681, 31046, 31411, 31777, 32142, 32507, + 32872, 33238, 33603, 33968, 34333, 34699, 35064, 35429, 35794, 36160, 36525, 36890, 37255, 37621, 37986, 38351, 38716, 39082, + 39447, 39812, 40177, 40543, 40908, 41273, 41638, 42004, 42369, 42734, 43099, 43465, 43830, 44195, 44560, 44926, 45291, 45656, + 46021, 46387, 46752, 47117, 47482, 47847, 48212, 48577, 48942, 49308, 49673}; - constexpr Int32 minYear = 1970; - constexpr Int32 maxYear = 2106; - constexpr Int32 yearInterval = 400; +constexpr Int32 minYear = 1970; +constexpr Int32 maxYear = 2106; - struct Date +struct Date +{ + Int32 year = 1970; + Int32 month = 1; + Int32 day = 1; + bool is_ad = true; // AD -> true, BC -> false. + + Int32 week = 1; // Week of year based on ISO week date, e.g: 27 + Int32 day_of_week = 1; // Day of week, Monday:1, Tuesday:2, ..., Sunday:7 + bool week_date_format = false; + + Int32 day_of_year = 1; + bool day_of_year_format = false; + + bool century_format = false; + + bool is_year_of_era = false; // Year of era cannot be zero or negative. + bool has_year = false; // Whether year was explicitly specified. + + Int32 hour = 0; + Int32 minute = 0; + Int32 second = 0; + // Int32 microsecond = 0; + bool is_am = true; // AM -> true, PM -> false + std::optional time_zone_offset = 0; + + bool is_clock_hour = false; // Whether most recent hour specifier is clockhour + bool is_hour_of_half_day = false; // Whether most recent hour specifier is of half day. + + std::vector day_of_month_values; + std::vector day_of_year_values; + + static bool isLeapYear(Int32 year_) { return year_ % 4 == 0 && (year_ % 100 != 0 || year_ % 400 == 0); } + + static bool isDateValid(Int32 year_, Int32 month_, Int32 day_) { - Int32 year = 1970; - Int32 month = 1; - Int32 day = 1; - bool is_ad = true; // AD -> true, BC -> false. + if (month_ < 1 || month_ > 12) + return false; - Int32 week = 1; // Week of year based on ISO week date, e.g: 27 - Int32 day_of_week = 1; // Day of week, Monday:1, Tuesday:2, ..., Sunday:7 - bool week_date_format = false; + if (year_ < minYear || year_ > maxYear) + return false; - Int32 day_of_year = 1; - bool day_of_year_format = false; + bool leap = isLeapYear(year_); + if (day_ < 1) + return false; - bool century_format = false; + if (leap && day_ > leapDays[month_]) + return false; - bool is_year_of_era = false; // Year of era cannot be zero or negative. - bool has_year = false; // Whether year was explicitly specified. + if (!leap && day_ > normalDays[month_]) + return false; + return true; + } - Int32 hour = 0; - Int32 minute = 0; - Int32 second = 0; - // Int32 microsecond = 0; - bool is_am = true; // AM -> true, PM -> false - std::optional time_zone_offset = 0; + static bool isDayOfYearValid(Int32 year_, Int32 day_of_year_) + { + if (year_ < minYear || year_ > maxYear) + return false; - bool is_clock_hour = false; // Whether most recent hour specifier is clockhour - bool is_hour_of_half_day = false; // Whether most recent hour specifier is of half day. + if (day_of_year_ < 1 || day_of_year_ > 365 + (isLeapYear(year_) ? 1 : 0)) + return false; - std::vector day_of_month_values; - std::vector day_of_year_values; + return true; + } - static bool isLeapYear(Int32 year_) { return year_ % 4 == 0 && (year_ % 100 != 0 || year_ % 400 == 0); } + static bool isWeekDateValid(Int32 week_year_, Int32 week_of_year_, Int32 day_of_week_) + { + if (day_of_week_ < 1 || day_of_week_ > 7) + return false; - static bool isDateValid(Int32 year_, Int32 month_, Int32 day_) + if (week_of_year_ < 1 || week_of_year_ > 52) + return false; + + if (week_year_ < minYear || week_year_ > maxYear) + return false; + + return true; + } + + static Int32 extractISODayOfTheWeek(Int32 days_since_epoch) + { + if (days_since_epoch < 0) { - if (month_ < 1 || month_ > 12) - return false; + // negative date: start off at 4 and cycle downwards + return (7 - ((-int64_t(days_since_epoch) + 3) % 7)); + } + else + { + // positive date: start off at 4 and cycle upwards + return ((int64_t(days_since_epoch) + 3) % 7) + 1; + } + } - if (year_ < minYear || year_ > maxYear) - return false; + static Int32 daysSinceEpochFromWeekDate(int32_t week_year_, int32_t week_of_year_, int32_t day_of_week_) + { + if (!isWeekDateValid(week_year_, week_of_year_, day_of_week_)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid week date"); - bool leap = isLeapYear(year_); - if (day_ < 1) - return false; + Int32 days_since_epoch_of_jan_fourth = daysSinceEpochFromDate(week_year_, 1, 4); + Int32 first_day_of_week_year = extractISODayOfTheWeek(days_since_epoch_of_jan_fourth); + return days_since_epoch_of_jan_fourth - (first_day_of_week_year - 1) + 7 * (week_of_year_ - 1) + day_of_week_ - 1; + } - if (leap && day_ > leapDays[month_]) - return false; + static Int32 daysSinceEpochFromDayOfYear(Int32 year_, Int32 day_of_year_) + { + if (!isDayOfYearValid(year_, day_of_year_)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of year"); - if (!leap && day_ > normalDays[month_]) - return false; - return true; + Int32 res = daysSinceEpochFromDate(year_, 1, 1); + res += day_of_year_ - 1; + return res; + } + + static Int32 daysSinceEpochFromDate(Int32 year_, Int32 month_, Int32 day_) + { + if (!isDateValid(year_, month_, day_)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid date"); + + Int32 res = cumulativeYearDays[year_ - 1970]; + res += isLeapYear(year_) ? cumulativeLeapDays[month_ - 1] : cumulativeDays[month_ - 1]; + res += day_ - 1; + return res; + } + + + Int64 checkAndGetDateTime(const DateLUTImpl & time_zone) + { + /// Era is BC and year of era is provided + if (is_year_of_era && !is_ad) + year = -1 * (year - 1); + + if (is_hour_of_half_day && !is_am) + hour += 12; + + + /// Ensure all day of year values are valid for ending year value + for (const auto d : day_of_month_values) + { + if (!isDateValid(year, month, d)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of month."); } - static bool isDayOfYearValid(Int32 year_, Int32 day_of_year_) + // Ensure all day of year values are valid for ending year value + for (const auto d : day_of_year_values) { - if (year_ < minYear || year_ > maxYear) - return false; - - if (day_of_year_ < 1 || day_of_year_ > 365 + (isLeapYear(year_) ? 1 : 0)) - return false; - - return true; + if (!isDayOfYearValid(year, d)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of year."); } - static bool isWeekDateValid(Int32 week_year_, Int32 week_of_year_, Int32 day_of_week_) + // Convert the parsed date/time into a timestamp. + Int32 days_since_epoch; + if (week_date_format) + days_since_epoch = daysSinceEpochFromWeekDate(year, week, day_of_week); + else if (day_of_year_format) + days_since_epoch = daysSinceEpochFromDayOfYear(year, day_of_year); + else + days_since_epoch = daysSinceEpochFromDate(year, month, day); + + Int64 seconds_since_epoch = days_since_epoch * 86400 + hour * 3600 + minute * 60 + second; + /// Time zone is not specified, use local time zone + if (!time_zone_offset) + *time_zone_offset = time_zone.timezoneOffset(seconds_since_epoch); + + /// Time zone is specified in format string. + seconds_since_epoch -= *time_zone_offset; + return seconds_since_epoch; + } +}; + +class Action +{ +private: + using Func = Pos (*)(Pos cur, Pos end, Date & date); + Func func{nullptr}; + std::string func_name; + + std::string literal; + +public: + explicit Action(Func func_, const char * func_name_) : func(func_), func_name(func_name_) { } + + template + explicit Action(const Literal & literal_) : literal(literal_) + { + } + + Pos perform(Pos cur, Pos end, Date & date) const + { + if (func) + return func(cur, end, date); + else { - if (day_of_week_ < 1 || day_of_week_ > 7) - return false; - - if (week_of_year_ < 1 || week_of_year_ > 52) - return false; - - if (week_year_ < minYear || week_year_ > maxYear) - return false; - - return true; + ensureSpace(cur, end, literal.size(), "requires size >= " + std::to_string(literal.size())); + if (std::string_view(cur, literal.size()) != literal) + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Expect literal {} but {} provided", literal, std::string_view(cur, literal.size())); + cur += literal.size(); + return cur; } + } - static Int32 extractISODayOfTheWeek(Int32 days_since_epoch) + template + static Pos readNumber2(Pos cur, Pos end, T & res) + { + ensureSpace(cur, end, 2, "readNumber2 requires size >= 2"); + res = (*cur - '0') * 10; + ++cur; + res += *cur; + ++cur; + return cur; + } + + template + static Pos readNumber3(Pos cur, Pos end, T & res) + { + cur = readNumber2(cur, end, res); + + ensureSpace(cur, end, 1, "readNumber3 requires size >= 3"); + res = res * 10 + (*cur - '0'); + ++cur; + return cur; + } + + template + static Pos readNumber4(Pos cur, Pos end, T & res) + { + cur = readNumber2(cur, end, res); + + T tmp; + cur = readNumber2(cur, end, tmp); + res = res * 100 + tmp; + return cur; + } + + static ALWAYS_INLINE void ensureSpace(Pos cur, Pos end, size_t len, const String & msg) + { + if (cur > end || cur + len > end) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because {}", msg); + } + + static ALWAYS_INLINE Pos assertChar(Pos cur, Pos end, char ch) + { + ensureSpace(cur, end, 1, "assertChar requires size >= 1"); + + if (*cur != ch) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expect char {}, but {} provided", String(ch, 1), String(*cur, 1)); + + ++cur; + return cur; + } + + static Pos mysqlDayOfWeekTextShort(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 3, "Parsing DayOfWeekTextShort requires size >= 3"); + + String str(cur, 3); + Poco::toLower(str); + Int32 i = 0; + for (; i < 7; ++i) + if (str == weekdaysShort[i]) + break; + + if (i == 7) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown short week text"); + + date.day_of_week = i + 1; + date.week_date_format = true; + date.day_of_year_format = false; + if (!date.has_year) { - if (days_since_epoch < 0) - { - // negative date: start off at 4 and cycle downwards - return (7 - ((-int64_t(days_since_epoch) + 3) % 7)); - } - else - { - // positive date: start off at 4 and cycle upwards - return ((int64_t(days_since_epoch) + 3) % 7) + 1; - } + date.has_year = true; + date.year = 2000; } + cur += 3; + return cur; + } - static Int32 daysSinceEpochFromWeekDate(int32_t week_year_, int32_t week_of_year_, int32_t day_of_week_) + static Pos mysqlMonthOfYearTextShort(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 3, "Parsing MonthOfYearTextShort requires size >= 3"); + + String str(cur, 3); + Poco::toLower(str); + + Int32 i = 0; + for (; i < 12; ++i) + if (str == monthsShort[i]) + break; + + if (i == 12) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown short month text"); + + date.month = i + 1; + cur += 3; + return cur; + } + + static Pos mysqlMonth(Pos cur, Pos end, Date & date) { return readNumber2(cur, end, date.month); } + + static Pos mysqlCentury(Pos cur, Pos end, Date & date) + { + Int32 centuray; + cur = readNumber2(cur, end, centuray); + date.century_format = true; + date.year = centuray * 100; + date.has_year = true; + return cur; + } + + static Pos mysqlDayOfMonth(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.day); + date.day_of_month_values.push_back(date.day); + date.week_date_format = false; + date.day_of_year_format = false; + if (!date.has_year) { - if (!isWeekDateValid(week_year_, week_of_year_, day_of_week_)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid week date"); - - Int32 days_since_epoch_of_jan_fourth = daysSinceEpochFromDate(week_year_, 1, 4); - Int32 first_day_of_week_year = extractISODayOfTheWeek(days_since_epoch_of_jan_fourth); - return days_since_epoch_of_jan_fourth - (first_day_of_week_year - 1) + 7 * (week_of_year_ - 1) + day_of_week_ - 1; + date.has_year = true; + date.year = 2000; } + return cur; + } - static Int32 daysSinceEpochFromDayOfYear(Int32 year_, Int32 day_of_year_) + static Pos mysqlAmericanDate(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.month); + cur = assertChar(cur, end, '/'); + + cur = readNumber2(cur, end, date.day); + cur = assertChar(cur, end, '/'); + + cur = readNumber2(cur, end, date.year); + cur = assertChar(cur, end, '/'); + + date.week_date_format = false; + date.day_of_year_format = false; + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; + return cur; + } + + + static Pos mysqlDayOfMonthSpacePadded(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 2, "mysqlDayOfMonthSpacePadded requires size >= 2"); + + date.day = *cur == ' ' ? 0 : (*cur - '0'); + ++cur; + + date.day = 10 * date.day + (*cur - '0'); + ++cur; + + date.week_date_format = false; + date.day_of_year_format = false; + if (!date.has_year) { - if (!isDayOfYearValid(year_, day_of_year_)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of year"); - - Int32 res = daysSinceEpochFromDate(year_, 1, 1); - res += day_of_year_ - 1; - return res; + date.has_year = true; + date.year = 2000; } + return cur; + } - static Int32 daysSinceEpochFromDate(Int32 year_, Int32 month_, Int32 day_) + static Pos mysqlISO8601Date(Pos cur, Pos end, Date & date) + { + cur = readNumber4(cur, end, date.year); + cur = readNumber2(cur, end, date.month); + cur = readNumber2(cur, end, date.day); + + date.week_date_format = false; + date.day_of_year_format = false; + + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; + return cur; + } + + static Pos mysqlISO8601Year2(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.year); + date.year += 2000; + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; + return cur; + } + + static Pos mysqlISO8601Year4(Pos cur, Pos end, Date & date) + { + cur = readNumber4(cur, end, date.year); + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; + return cur; + } + + static Pos mysqlDayOfYear(Pos cur, Pos end, Date & date) + { + cur = readNumber3(cur, end, date.day_of_year); + + date.day_of_year_values.push_back(date.day_of_year); + date.day_of_year_format = true; + date.week_date_format = false; + if (!date.has_year) { - if (!isDateValid(year_, month_, day_)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid date"); - - Int32 res = cumulativeYearDays[year_ - 1970]; - res += isLeapYear(year_) ? cumulativeLeapDays[month_ - 1] : cumulativeDays[month_ - 1]; - res += day_ - 1; - return res; + date.has_year = true; + date.year = 2000; } + return cur; + } + static Pos mysqlDayOfWeek(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1"); - Int64 checkAndGetDateTime(const DateLUTImpl & time_zone) + date.day_of_week = *cur - '0'; + date.week_date_format = true; + date.day_of_year_format = false; + if (!date.has_year) { - /// Era is BC and year of era is provided - if (is_year_of_era && !is_ad) - year = -1 * (year - 1); - - if (is_hour_of_half_day && !is_am) - hour += 12; - - - /// Ensure all day of year values are valid for ending year value - for (const auto d : day_of_month_values) - { - if (!isDateValid(year, month, d)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of month."); - } - - // Ensure all day of year values are valid for ending year value - for (const auto d : day_of_year_values) - { - if (!isDayOfYearValid(year, d)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of year."); - } - - // Convert the parsed date/time into a timestamp. - Int32 days_since_epoch; - if (week_date_format) - days_since_epoch = daysSinceEpochFromWeekDate(year, week, day_of_week); - else if (day_of_year_format) - days_since_epoch = daysSinceEpochFromDayOfYear(year, day_of_year); - else - days_since_epoch = daysSinceEpochFromDate(year, month, day); - - Int64 seconds_since_epoch = days_since_epoch * 86400 + hour * 3600 + minute * 60 + second; - /// Time zone is not specified, use local time zone - if (!time_zone_offset) - *time_zone_offset = time_zone.timezoneOffset(seconds_since_epoch); - - /// Time zone is specified in format string. - seconds_since_epoch -= *time_zone_offset; - return seconds_since_epoch; + date.has_year = true; + date.year = 2000; } + return cur; + } + + static Pos mysqlISO8601Week(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.week); + date.week_date_format = true; + date.day_of_year_format = false; + if (date.has_year) + { + date.has_year = true; + date.year = 2000; + } + return cur; + } + + static Pos mysqlDayOfWeek0To6(Pos cur, Pos end, Date & date) + { + cur = mysqlDayOfWeek(cur, end, date); + if (date.day_of_week == 0) + date.day_of_week = 7; + + return cur; + } + + static Pos mysqlDayOfWeekTextLong(Pos cur, Pos end, Date & date) + { + mysqlDayOfWeekTextShort(cur, end, date); + auto expect_text = weekdaysFull[date.day_of_week - 1]; + + ensureSpace(cur, end, expect_text.size(), "mysqlDayOfWeekTextLong requires size >= " + std::to_string(expect_text.size())); + std::string_view text(cur, expect_text.size()); + if (text != expect_text) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown full day of week text {}", expect_text); + + cur += expect_text.size(); + return cur; + } + + static Pos mysqlYear2(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.year); + date.year += 2000; + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; + return cur; + } + + static Pos mysqlYear4(Pos cur, Pos end, Date & date) + { + cur = readNumber4(cur, end, date.year); + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; + return cur; + } + + static Pos mysqlTimezoneOffset(Pos cur, Pos end, Date & date) + { + /// TODO figure out what timezone_id mean + ensureSpace(cur, end, 1, "Parse mysqlTimezoneOffset failed"); + Int32 sign = 1; + if (*cur == '-') + sign = -1; + ++cur; + + Int32 hour; + cur = readNumber2(cur, end, hour); + + Int32 minute; + cur = readNumber2(cur, end, minute); + + *date.time_zone_offset = sign * (hour * 3600 + minute * 60); + return cur; + } + + static Pos mysqlMinute(Pos cur, Pos end, Date & date) { return readNumber2(cur, end, date.minute); } + + static Pos mysqlAMPM(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 2, "mysqlAMPM requires size >= 2"); + + std::string text(cur, 2); + Poco::toUpper(text); + if (text == "PM") + date.is_am = true; + else if (text == "AM") + date.is_am = false; + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Text should be AM or PM, but {} provided", text); + + cur += 2; + return cur; + } + + static Pos mysqlHHMM12(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.hour); + date.is_clock_hour = false; + date.is_hour_of_half_day = true; + + cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, date.minute); + cur = assertChar(cur, end, ' '); + cur = mysqlAMPM(cur, end, date); + return cur; + } + + static Pos mysqlHHMM24(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.hour); + date.is_clock_hour = false; + date.is_hour_of_half_day = false; + + cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, date.minute); + return cur; + } + + static Pos mysqlSecond(Pos cur, Pos end, Date & date) { return readNumber2(cur, end, date.second); } + + static Pos mysqlISO8601Time(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.hour); + date.is_clock_hour = false; + date.is_hour_of_half_day = false; + + cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, date.minute); + cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, date.second); + return cur; + } + + static Pos mysqlHour12(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.hour); + date.is_hour_of_half_day = true; + date.is_clock_hour = false; + return cur; + } + + static Pos mysqlHour24(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.hour); + date.is_hour_of_half_day = false; + date.is_clock_hour = false; + return cur; + } +}; + + +struct ParseDateTimeTraits +{ + enum class ParseSyntax + { + MySQL, + Joda }; +}; - class Action +#define ACTION_ARGS(func) &func, #func + + +/// _FUNC_(str[, format, timezone]) +template +class FunctionParseDateTimeImpl : public IFunction +{ +public: + static constexpr auto name = Name::name; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - private: - using Func = Pos (*)(Pos cur, Pos end, Date & date); - Func func; - std::string literal; - - public: - explicit Action(Func && func_) : func(std::move(func_)) { } - - template - explicit Action(const Literal & literal_) : literal(literal_) - { - } - - Pos perform(Pos cur, Pos end, Date & date) const - { - if (func) - return func(cur, end, date); - else - { - ensureSpace(cur, end, literal.size(), "requires size >= " + std::to_string(literal.size())); - if (std::string_view(cur, literal.size()) != literal) - throw Exception( - ErrorCodes::LOGICAL_ERROR, "Expect literal {} but {} provided", literal, std::string_view(cur, literal.size())); - cur += literal.size(); - return cur; - } - } - - template - static Pos readNumber2(Pos cur, Pos end, T & res) - { - ensureSpace(cur, end, 2, "readNumber2 requires size >= 2"); - res = (*cur - '0') * 10; - ++cur; - res += *cur; - ++cur; - return cur; - } - - template - static Pos readNumber3(Pos cur, Pos end, T & res) - { - cur = readNumber2(cur, end, res); - - ensureSpace(cur, end, 1, "readNumber3 requires size >= 3"); - res = res * 10 + (*cur - '0'); - ++cur; - return cur; - } - - template - static Pos readNumber4(Pos cur, Pos end, T & res) - { - cur = readNumber2(cur, end, res); - - T tmp; - cur = readNumber2(cur, end, tmp); - res = res * 100 + tmp; - return cur; - } - - static ALWAYS_INLINE void ensureSpace(Pos cur, Pos end, size_t len, const String & msg) - { - if (cur > end || cur + len > end) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because {}", msg); - } - - static ALWAYS_INLINE Pos assertChar(Pos cur, Pos end, char ch) - { - ensureSpace(cur, end, 1, "assertChar requires size >= 1"); - - if (*cur != ch) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expect char {}, but {} provided", String(ch, 1), String(*cur, 1)); - - ++cur; - return cur; - } - - static Pos mysqlDayOfWeekTextShort(Pos cur, Pos end, Date & date) - { - ensureSpace(cur, end, 3, "Parsing DayOfWeekTextShort requires size >= 3"); - - String str(cur, 3); - Poco::toLower(str); - Int32 i = 0; - for (; i < 7; ++i) - if (str == weekdaysShort[i]) - break; - - if (i == 7) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown short week text"); - - date.day_of_week = i + 1; - date.week_date_format = true; - date.day_of_year_format = false; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } - cur += 3; - return cur; - } - - static Pos mysqlMonthOfYearTextShort(Pos cur, Pos end, Date & date) - { - ensureSpace(cur, end, 3, "Parsing MonthOfYearTextShort requires size >= 3"); - - String str(cur, 3); - Poco::toLower(str); - - Int32 i = 0; - for (; i < 12; ++i) - if (str == monthsShort[i]) - break; - - if (i == 12) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown short month text"); - - date.month = i + 1; - cur += 3; - return cur; - } - - static Pos mysqlMonth(Pos cur, Pos end, Date & date) { return readNumber2(cur, end, date.month); } - - static Pos mysqlCentury(Pos cur, Pos end, Date & date) - { - Int32 centuray; - cur = readNumber2(cur, end, centuray); - date.century_format = true; - date.year = centuray * 100; - date.has_year = true; - return cur; - } - - static Pos mysqlDayOfMonth(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.day); - date.day_of_month_values.push_back(date.day); - date.week_date_format = false; - date.day_of_year_format = false; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } - return cur; - } - - static Pos mysqlAmericanDate(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.month); - cur = assertChar(cur, end, '/'); - - cur = readNumber2(cur, end, date.day); - cur = assertChar(cur, end, '/'); - - cur = readNumber2(cur, end, date.year); - cur = assertChar(cur, end, '/'); - - date.week_date_format = false; - date.day_of_year_format = false; - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; - return cur; - } - - - static Pos mysqlDayOfMonthSpacePadded(Pos cur, Pos end, Date & date) - { - ensureSpace(cur, end, 2, "mysqlDayOfMonthSpacePadded requires size >= 2"); - - date.day = *cur == ' ' ? 0 : (*cur - '0'); - ++cur; - - date.day = 10 * date.day + (*cur - '0'); - ++cur; - - date.week_date_format = false; - date.day_of_year_format = false; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } - return cur; - } - - static Pos mysqlISO8601Date(Pos cur, Pos end, Date & date) - { - cur = readNumber4(cur, end, date.year); - cur = readNumber2(cur, end, date.month); - cur = readNumber2(cur, end, date.day); - - date.week_date_format = false; - date.day_of_year_format = false; - - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; - return cur; - } - - static Pos mysqlISO8601Year2(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.year); - date.year += 2000; - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; - return cur; - } - - static Pos mysqlISO8601Year4(Pos cur, Pos end, Date & date) - { - cur = readNumber4(cur, end, date.year); - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; - return cur; - } - - static Pos mysqlDayOfYear(Pos cur, Pos end, Date & date) - { - cur = readNumber3(cur, end, date.day_of_year); - - date.day_of_year_values.push_back(date.day_of_year); - date.day_of_year_format = true; - date.week_date_format = false; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } - return cur; - } - - static Pos mysqlDayOfWeek(Pos cur, Pos end, Date & date) - { - ensureSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1"); - - date.day_of_week = *cur - '0'; - date.week_date_format = true; - date.day_of_year_format = false; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } - return cur; - } - - static Pos mysqlISO8601Week(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.week); - date.week_date_format = true; - date.day_of_year_format = false; - if (date.has_year) - { - date.has_year = true; - date.year = 2000; - } - return cur; - } - - static Pos mysqlDayOfWeek0To6(Pos cur, Pos end, Date & date) - { - cur = mysqlDayOfWeek(cur, end, date); - if (date.day_of_week == 0) - date.day_of_week = 7; - - return cur; - } - - static Pos mysqlDayOfWeekTextLong(Pos cur, Pos end, Date & date) - { - mysqlDayOfWeekTextShort(cur, end, date); - auto expect_text = weekdaysFull[date.day_of_week - 1]; - - ensureSpace(cur, end, expect_text.size(), "mysqlDayOfWeekTextLong requires size >= " + std::to_string(expect_text.size())); - std::string_view text(cur, expect_text.size()); - if (text != expect_text) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown full day of week text {}", expect_text); - - cur += expect_text.size(); - return cur; - } - - static Pos mysqlYear2(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.year); - date.year += 2000; - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; - return cur; - } - - - static Pos mysqlYear4(Pos cur, Pos end, Date & date) - { - cur = readNumber4(cur, end, date.year); - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; - return cur; - } - - static Pos mysqlTimezoneOffset(Pos cur, Pos end, Date & date) - { - /// TODO figure out what timezone_id mean - ensureSpace(cur, end, 1, "Parse mysqlTimezoneOffset failed"); - Int32 sign = 1; - if (*cur == '-') - sign = -1; - ++cur; - - Int32 hour; - cur = readNumber2(cur, end, hour); - - Int32 minute; - cur = readNumber2(cur, end, minute); - - *date.time_zone_offset = sign * (hour * 3600 + minute * 60); - return cur; - } - - static Pos mysqlMinute(Pos cur, Pos end, Date & date) { return readNumber2(cur, end, date.minute); } - - static Pos mysqlAMPM(Pos cur, Pos end, Date & date) - { - ensureSpace(cur, end, 2, "mysqlAMPM requires size >= 2"); - - std::string text(cur, 2); - Poco::toUpper(text); - if (text == "PM") - date.is_am = true; - else if (text == "AM") - date.is_am = false; - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Text should be AM or PM, but {} provided", text); - - cur += 2; - return cur; - } - - static Pos mysqlHHMM12(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.hour); - date.is_clock_hour = false; - date.is_hour_of_half_day = true; - - cur = assertChar(cur, end, ':'); - cur = readNumber2(cur, end, date.minute); - cur = assertChar(cur, end, ' '); - cur = mysqlAMPM(cur, end, date); - return cur; - } - - static Pos mysqlHHMM24(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.hour); - date.is_clock_hour = false; - date.is_hour_of_half_day = false; - - cur = assertChar(cur, end, ':'); - cur = readNumber2(cur, end, date.minute); - return cur; - } - - static Pos mysqlSecond(Pos cur, Pos end, Date & date) { return readNumber2(cur, end, date.second); } - - static Pos mysqlISO8601Time(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.hour); - date.is_clock_hour = false; - date.is_hour_of_half_day = false; - - cur = assertChar(cur, end, ':'); - cur = readNumber2(cur, end, date.minute); - cur = assertChar(cur, end, ':'); - cur = readNumber2(cur, end, date.second); - return cur; - } - - static Pos mysqlHour12(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.hour); - date.is_hour_of_half_day = true; - date.is_clock_hour = false; - return cur; - } - - static Pos mysqlHour24(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.hour); - date.is_hour_of_half_day = false; - date.is_clock_hour = false; - return cur; - } - }; - - - struct ParseDateTimeTraits + if (arguments.size() != 1 && arguments.size() != 2 && arguments.size() != 3) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 1, 2 or 3", + getName(), + arguments.size()); + + if (!isString(arguments[0].type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {} when arguments size is 1. Should be string", + arguments[0].type->getName(), + getName()); + + if (arguments.size() > 1 && !isString(arguments[1].type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {} when arguments size is 1. Should be string", + arguments[0].type->getName(), + getName()); + + if (arguments.size() > 2 && !isString(arguments[2].type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of third argument of function {} when arguments size is 1. Should be string", + arguments[0].type->getName(), + getName()); + + String time_zone_name = getTimeZone(arguments).second; + return std::make_shared(time_zone_name); + } + + ColumnPtr + executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override { - enum class ParseSyntax + const auto * col_str = checkAndGetColumn(arguments[0].column.get()); + if (!col_str) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of first ('str') argument of function {}. Must be string.", + arguments[0].column->getName(), + getName()); + + String format = getFormat(arguments); + const auto * time_zone = getTimeZone(arguments).first; + + std::vector instructions; + parseFormat(format, instructions); + + auto col_res = ColumnDateTime::create(); + col_res->reserve(input_rows_count); + auto & data_res = col_res->getData(); + for (size_t i = 0; i < input_rows_count; ++i) { - MySQL, - Joda - }; - }; + StringRef str_ref = col_str->getDataAt(i); + Date date; + Pos cur = str_ref.data; + Pos end = str_ref.data + str_ref.size; + for (const auto & instruction : instructions) + cur = instruction.perform(cur, end, date); + + // Ensure all input was consumed. + if (cur < end) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Invalid format input {} is malformed at {}", + str_ref.toView(), + std::string_view(cur, end - cur)); + + Int64 time = date.checkAndGetDateTime(*time_zone); + data_res.push_back(static_cast(time)); + } + + return col_res; + } - /// _FUNC_(str[, format, timezone]) - template - class FunctionParseDateTimeImpl : public IFunction +private: + ALWAYS_INLINE void parseFormat(const String & format, std::vector & instructions) const { - public: - static constexpr auto name = Name::name; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } + if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::MySQL) + parseMysqlFormat(format, instructions); + else if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::Joda) + parseJodaFormat(format, instructions); + else + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, + "Unknown datetime format style {} in function {}", + magic_enum::enum_name(parse_syntax), + getName()); + } - String getName() const override { return name; } - - bool useDefaultImplementationForConstants() const override { return true; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + ALWAYS_INLINE void parseMysqlFormat(const String & format, std::vector & instructions) const + { + Pos pos = format.data(); + Pos end = pos + format.size(); + while (true) { - if (arguments.size() != 1 && arguments.size() != 2 && arguments.size() != 3) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be 1, 2 or 3", - getName(), - arguments.size()); - - if (!isString(arguments[0].type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of first argument of function {} when arguments size is 1. Should be string", - arguments[0].type->getName(), - getName()); - - if (arguments.size() > 1 && !isString(arguments[1].type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of second argument of function {} when arguments size is 1. Should be string", - arguments[0].type->getName(), - getName()); - - if (arguments.size() > 2 && !isString(arguments[2].type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of third argument of function {} when arguments size is 1. Should be string", - arguments[0].type->getName(), - getName()); - - String time_zone_name = getTimeZone(arguments).second; - return std::make_shared(time_zone_name); - } - - ColumnPtr executeImpl( - const ColumnsWithTypeAndName & arguments, - [[maybe_unused]] const DataTypePtr & result_type, - [[maybe_unused]] size_t input_rows_count) const override - { - const auto * col_str = checkAndGetColumn(arguments[0].column.get()); - if (!col_str) - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of first ('str') argument of function {}. Must be string.", - arguments[0].column->getName(), - getName()); - - String format = getFormat(arguments); - const auto * time_zone = getTimeZone(arguments).first; - - std::vector instructions; - parseFormat(format, instructions); - - auto col_res = ColumnDateTime::create(); - col_res->reserve(input_rows_count); - auto & data_res = col_res->getData(); - for (size_t i = 0; i < input_rows_count; ++i) + Pos percent_pos = find_first_symbols<'%'>(pos, end); + if (percent_pos < end) { - StringRef str_ref = col_str->getDataAt(i); - Date date; - Pos cur = str_ref.data; - Pos end = str_ref.data + str_ref.size; - for (const auto & instruction : instructions) - cur = instruction.perform(cur, end, date); + if (pos < percent_pos) + instructions.emplace_back(std::string_view(pos, percent_pos - pos)); - // Ensure all input was consumed. - if (cur < end) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Invalid format input {} is malformed at {}", - str_ref.toView(), - std::string_view(cur, end - cur)); + pos = percent_pos + 1; + if (pos >= end) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Sign '%' is the last in format, if you need it, use '%%'"); - auto time = date.checkAndGetDateTime(*time_zone); - data_res.push_back(time); - } - - return col_res; - } - - - private: - void parseFormat(const String & format, std::vector & instructions) - { - if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::MySQL) - parseMysqlFormat(format, instructions); - else if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::Joda) - parseJodaFormat(format, instructions); - else - throw Exception( - ErrorCodes::NOT_IMPLEMENTED, - "Unknown datetime format style {} in function {}", - magic_enum::enum_name(parse_syntax), - getName()); - } - - void parseMysqlFormat(const String & format, std::vector & instructions) - { - Pos pos = format.data(); - Pos end = pos + format.size(); - while (true) - { - Pos percent_pos = find_first_symbols<'%'>(pos, end); - if (percent_pos < end) + switch (*pos) { - if (pos < percent_pos) - instructions.emplace_back(std::string_view(pos, percent_pos - pos)); + // Abbreviated weekday [Mon...Sun] + case 'a': + instructions.emplace_back(&Action::mysqlDayOfWeekTextShort); + break; - pos = percent_pos + 1; - if (pos >= end) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Sign '%' is the last in format, if you need it, use '%%'"); + // Abbreviated month [Jan...Dec] + case 'b': + instructions.emplace_back(&Action::mysqlMonthOfYearTextShort); + break; - switch (*pos) - { - // Abbreviated weekday [Mon...Sun] - case 'a': - instructions.emplace_back(&Action::mysqlDayOfWeekTextShort); - break; + // Month as a decimal number (01-12) + case 'c': + instructions.emplace_back(&Action::mysqlMonth); + break; - // Abbreviated month [Jan...Dec] - case 'b': - instructions.emplace_back(&Action::mysqlMonthOfYearTextShort); - break; + // Year, divided by 100, zero-padded + case 'C': + instructions.emplace_back(&Action::mysqlCentury); + break; - // Month as a decimal number (01-12) - case 'c': - instructions.emplace_back(&Action::mysqlMonth); - break; + // Day of month, zero-padded (01-31) + case 'd': + instructions.emplace_back(&Action::mysqlDayOfMonth); + break; - // Year, divided by 100, zero-padded - case 'C': - instructions.emplace_back(&Action::mysqlCentury); - break; + // Short MM/DD/YY date, equivalent to %m/%d/%y + case 'D': + instructions.emplace_back(&Action::mysqlAmericanDate); + break; - // Day of month, zero-padded (01-31) - case 'd': - instructions.emplace_back(&Action::mysqlDayOfMonth); - break; + // Day of month, space-padded ( 1-31) 23 + case 'e': + instructions.emplace_back(&Action::mysqlDayOfMonthSpacePadded); + break; - // Short MM/DD/YY date, equivalent to %m/%d/%y - case 'D': - instructions.emplace_back(&Action::mysqlAmericanDate); - break; + // Fractional seconds + case 'f': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for fractional seconds"); - // Day of month, space-padded ( 1-31) 23 - case 'e': - instructions.emplace_back(&Action::mysqlDayOfMonthSpacePadded); - break; + // Short YYYY-MM-DD date, equivalent to %Y-%m-%d 2001-08-23 + case 'F': + instructions.emplace_back(&Action::mysqlISO8601Date); + break; - // Fractional seconds - case 'f': - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for fractional seconds"); + // Last two digits of year of ISO 8601 week number (see %G) + case 'g': + instructions.emplace_back(&Action::mysqlISO8601Year2); + break; - // Short YYYY-MM-DD date, equivalent to %Y-%m-%d 2001-08-23 - case 'F': - instructions.emplace_back(&Action::mysqlISO8601Date); - break; + // Year of ISO 8601 week number (see %V) + case 'G': + instructions.emplace_back(&Action::mysqlISO8601Year4); + break; - // Last two digits of year of ISO 8601 week number (see %G) - case 'g': - instructions.emplace_back(&Action::mysqlISO8601Year2); - break; + // Day of the year (001-366) 235 + case 'j': + instructions.emplace_back(&Action::mysqlDayOfYear); + break; - // Year of ISO 8601 week number (see %V) - case 'G': - instructions.emplace_back(&Action::mysqlISO8601Year4); - break; + // Month as a decimal number (01-12) + case 'm': + instructions.emplace_back(&Action::mysqlMonth); + break; - // Day of the year (001-366) 235 - case 'j': - instructions.emplace_back(&Action::mysqlDayOfYear); - break; + // ISO 8601 weekday as number with Monday as 1 (1-7) + case 'u': + instructions.emplace_back(&Action::mysqlDayOfWeek); + break; - // Month as a decimal number (01-12) - case 'm': - instructions.emplace_back(&Action::mysqlMonth); - break; + // ISO 8601 week number (01-53) + case 'V': + instructions.emplace_back(&Action::mysqlISO8601Week); + break; - // ISO 8601 weekday as number with Monday as 1 (1-7) - case 'u': - instructions.emplace_back(&Action::mysqlDayOfWeek); - break; + // Weekday as a decimal number with Sunday as 0 (0-6) 4 + case 'w': + instructions.emplace_back(&Action::mysqlDayOfWeek0To6); + break; - // ISO 8601 week number (01-53) - case 'V': - instructions.emplace_back(&Action::mysqlISO8601Week); - break; + // Full weekday [Monday...Sunday] + case 'W': + instructions.emplace_back(&Action::mysqlDayOfWeekTextLong); + break; - // Weekday as a decimal number with Sunday as 0 (0-6) 4 - case 'w': - instructions.emplace_back(&Action::mysqlDayOfWeek0To6); - break; + // Two digits year + case 'y': + instructions.emplace_back(&Action::mysqlYear2); + break; - // Full weekday [Monday...Sunday] - case 'W': - instructions.emplace_back(&Action::mysqlDayOfWeekTextLong); - break; + // Four digits year + case 'Y': + instructions.emplace_back(&Action::mysqlYear4); + break; - // Two digits year - case 'y': - instructions.emplace_back(&Action::mysqlYear2); - break; + // Quarter (1-4) + case 'Q': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for quarter"); + break; - // Four digits year - case 'Y': - instructions.emplace_back(&Action::mysqlYear4); - break; + // Offset from UTC timezone as +hhmm or -hhmm + case 'z': + instructions.emplace_back(&Action::mysqlTimezoneOffset); + break; - // Quarter (1-4) - case 'Q': - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for quarter"); - break; + /// Time components. If the argument is Date, not a DateTime, then this components will have default value. - // Offset from UTC timezone as +hhmm or -hhmm - case 'z': - instructions.emplace_back(&Action::mysqlTimezoneOffset); - break; + // Minute (00-59) + case 'M': + instructions.emplace_back(&Action::mysqlMinute); + break; - /// Time components. If the argument is Date, not a DateTime, then this components will have default value. + // AM or PM + case 'p': + instructions.emplace_back(&Action::mysqlAMPM); + break; - // Minute (00-59) - case 'M': - instructions.emplace_back(&Action::mysqlMinute); - break; + // 12-hour HH:MM time, equivalent to %h:%i %p 2:55 PM + case 'r': + instructions.emplace_back(&Action::mysqlHHMM12); + break; - // AM or PM - case 'p': - instructions.emplace_back(&Action::mysqlAMPM); - break; + // 24-hour HH:MM time, equivalent to %H:%i 14:55 + case 'R': + instructions.emplace_back(&Action::mysqlHHMM24); + break; - // 12-hour HH:MM time, equivalent to %h:%i %p 2:55 PM - case 'r': - instructions.emplace_back(&Action::mysqlHHMM12); - break; + // Seconds + case 's': + instructions.emplace_back(&Action::mysqlSecond); + break; - // 24-hour HH:MM time, equivalent to %H:%i 14:55 - case 'R': - instructions.emplace_back(&Action::mysqlHHMM24); - break; + // Seconds + case 'S': + instructions.emplace_back(&Action::mysqlSecond); + break; - // Seconds - case 's': - instructions.emplace_back(&Action::mysqlSecond); - break; + // ISO 8601 time format (HH:MM:SS), equivalent to %H:%i:%S 14:55:02 + case 'T': + instructions.emplace_back(&Action::mysqlISO8601Time); + break; - // Seconds - case 'S': - instructions.emplace_back(&Action::mysqlSecond); - break; + // Hour in 12h format (01-12) + case 'h': + instructions.emplace_back(&Action::mysqlHour12); + break; - // ISO 8601 time format (HH:MM:SS), equivalent to %H:%i:%S 14:55:02 - case 'T': - instructions.emplace_back(&Action::mysqlISO8601Time); - break; + // Hour in 24h format (00-23) + case 'H': + instructions.emplace_back(&Action::mysqlHour24); + break; - // Hour in 12h format (01-12) - case 'h': - instructions.emplace_back(&Action::mysqlHour12); - break; + // Minute of hour range [0, 59] + case 'i': + instructions.emplace_back(&Action::mysqlMinute); + break; - // Hour in 24h format (00-23) - case 'H': - instructions.emplace_back(&Action::mysqlHour24); - break; + // Hour in 12h format (01-12) + case 'I': + instructions.emplace_back(&Action::mysqlHour12); + break; - // Minute of hour range [0, 59] - case 'i': - instructions.emplace_back(&Action::mysqlMinute); - break; + // Hour in 24h format (00-23) + case 'k': + instructions.emplace_back(&Action::mysqlHour24); + break; - // Hour in 12h format (01-12) - case 'I': - instructions.emplace_back(&Action::mysqlHour12); - break; + // Hour in 12h format (01-12) + case 'l': + instructions.emplace_back(&Action::mysqlHour12); + break; - // Hour in 24h format (00-23) - case 'k': - instructions.emplace_back(&Action::mysqlHour24); - break; + case 't': + instructions.emplace_back("\t"); + break; - // Hour in 12h format (01-12) - case 'l': - instructions.emplace_back(&Action::mysqlHour12); - break; + case 'n': + instructions.emplace_back("\n"); + break; - case 't': - instructions.emplace_back("\t"); - break; + // Escaped literal characters. + case '%': + instructions.emplace_back("\n"); + break; - case 'n': - instructions.emplace_back("\n"); - break; + // Unimplemented + case 'U': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for WEEK (Sun-Sat)"); + case 'v': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for WEEK (Mon-Sun)"); + case 'x': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for YEAR for week (Mon-Sun)"); + case 'X': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for YEAR for week (Sun-Sat)"); - // Escaped literal characters. - case '%': - instructions.emplace_back("\n"); - break; - - // Unimplemented - case 'U': - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for WEEK (Sun-Sat)"); - case 'v': - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for WEEK (Mon-Sun)"); - case 'x': - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for YEAR for week (Mon-Sun)"); - case 'X': - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for YEAR for week (Sun-Sat)"); - - default: - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Incorrect syntax '{}', symbol is not supported '{}' for function {}", - format, - *pos, - getName()); - } - - ++pos; - } - else - { - instructions.emplace_back(std::string_view(pos, end - pos)); - break; + default: + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Incorrect syntax '{}', symbol is not supported '{}' for function {}", + format, + *pos, + getName()); } + + ++pos; } - } - - void parseJodaFormat(const String & /*format*/, std::vector & /*instructions*/) { } - - - String getFormat(const ColumnsWithTypeAndName & arguments) const - { - if (arguments.size() < 2) + else { - if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::Joda) - return "yyyy-MM-dd HH:mm:ss"; - else - return "%F %T"; + instructions.emplace_back(std::string_view(pos, end - pos)); + break; } - - const auto * format_column = checkAndGetColumnConst(arguments[1].column.get()); - if (!format_column) - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of second ('format') argument of function {}. Must be constant string.", - arguments[1].column->getName(), - getName()); - return format_column->getValue(); } + } - std::pair getTimeZone(const ColumnsWithTypeAndName & arguments) const + void parseJodaFormat(const String & /*format*/, std::vector & /*instructions*/) const { } + + + ALWAYS_INLINE String getFormat(const ColumnsWithTypeAndName & arguments) const + { + if (arguments.size() < 2) { - if (arguments.size() < 3) - return {&DateLUT::instance(), ""}; - - const auto * col = checkAndGetColumnConst(arguments[2].column.get()); - if (!col) - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of third ('timezone') argument of function {}. Must be constant string.", - arguments[2].column->getName(), - getName()); - - String time_zone = col->getValue(); - if (time_zone.empty()) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Provided time zone must be non-empty and be a valid time zone"); - return {&DateLUT::instance(time_zone), time_zone}; + if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::Joda) + return "yyyy-MM-dd HH:mm:ss"; + else + return "%Y-%m-%d %H:%M:%S"; } - }; + const auto * format_column = checkAndGetColumnConst(arguments[1].column.get()); + if (!format_column) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of second ('format') argument of function {}. Must be constant string.", + arguments[1].column->getName(), + getName()); + return format_column->getValue(); + } + + ALWAYS_INLINE std::pair getTimeZone(const ColumnsWithTypeAndName & arguments) const + { + if (arguments.size() < 3) + return {&DateLUT::instance(), ""}; + + const auto * col = checkAndGetColumnConst(arguments[2].column.get()); + if (!col) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of third ('timezone') argument of function {}. Must be constant string.", + arguments[2].column->getName(), + getName()); + + String time_zone = col->getValue(); + if (time_zone.empty()) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Provided time zone must be non-empty and be a valid time zone"); + return {&DateLUT::instance(time_zone), time_zone}; + } +}; + +struct NameParseDateTime +{ + static constexpr auto name = "parseDateTime"; +}; + +using FunctionParseDateTime = FunctionParseDateTimeImpl; } +REGISTER_FUNCTION(ParseDateTime) +{ + factory.registerFunction(); + factory.registerAlias("TO_UNIXTIME", "parseDateTime"); +} + + } From 00edf977f83fb5b422624a0035bb61c837642dd9 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 24 Feb 2023 18:27:21 +0800 Subject: [PATCH 056/559] finish debug --- src/Functions/parseDateTime.cpp | 122 +++++++++++++++++++++----------- 1 file changed, 80 insertions(+), 42 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index d81eef7e162..2e986835276 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -15,8 +15,6 @@ #include #include -#pragma GCC diagnostic ignored "-Wunneeded-member-function" - namespace DB { @@ -71,7 +69,7 @@ struct Date Int32 second = 0; // Int32 microsecond = 0; bool is_am = true; // AM -> true, PM -> false - std::optional time_zone_offset = 0; + std::optional time_zone_offset; bool is_clock_hour = false; // Whether most recent hour specifier is clockhour bool is_hour_of_half_day = false; // Whether most recent hour specifier is of half day. @@ -79,6 +77,24 @@ struct Date std::vector day_of_month_values; std::vector day_of_year_values; + /// For debug + [[maybe_unused]] String toString() const + { + String res; + res += "year:" + std::to_string(year); + res += ","; + res += "month:" + std::to_string(month); + res += ","; + res += "day:" + std::to_string(day); + res += ","; + res += "hour:" + std::to_string(hour); + res += ","; + res += "minute:" + std::to_string(minute); + res += ","; + res += "second:" + std::to_string(second); + return res; + } + static bool isLeapYear(Int32 year_) { return year_ % 4 == 0 && (year_ % 100 != 0 || year_ % 400 == 0); } static bool isDateValid(Int32 year_, Int32 month_, Int32 day_) @@ -206,12 +222,17 @@ struct Date days_since_epoch = daysSinceEpochFromDate(year, month, day); Int64 seconds_since_epoch = days_since_epoch * 86400 + hour * 3600 + minute * 60 + second; + /// Time zone is not specified, use local time zone if (!time_zone_offset) - *time_zone_offset = time_zone.timezoneOffset(seconds_since_epoch); + *time_zone_offset = time_zone.getOffsetAtStartOfEpoch(); + // std::cout << "timezonename:" << time_zone.getTimeZone() << std::endl; + // std::cout << "time_zone_offset:" << *time_zone_offset << time_zone.getOffsetAtStartOfEpoch() << std::endl; + // std::cout << "before timestamp:" << seconds_since_epoch << std::endl; /// Time zone is specified in format string. seconds_since_epoch -= *time_zone_offset; + // std::cout << "after timestamp:" << seconds_since_epoch << std::endl; return seconds_since_epoch; } }; @@ -233,6 +254,15 @@ public: { } + /// For debug + [[maybe_unused]] String toString()const + { + if (func) + return "func:" + func_name; + else + return "literal:" + literal; + } + Pos perform(Pos cur, Pos end, Date & date) const { if (func) @@ -254,7 +284,7 @@ public: ensureSpace(cur, end, 2, "readNumber2 requires size >= 2"); res = (*cur - '0') * 10; ++cur; - res += *cur; + res += *cur - '0'; ++cur; return cur; } @@ -413,7 +443,9 @@ public: static Pos mysqlISO8601Date(Pos cur, Pos end, Date & date) { cur = readNumber4(cur, end, date.year); + cur = assertChar(cur, end, '-'); cur = readNumber2(cur, end, date.month); + cur = assertChar(cur, end, '-'); cur = readNumber2(cur, end, date.day); date.week_date_format = false; @@ -596,13 +628,13 @@ public: static Pos mysqlISO8601Time(Pos cur, Pos end, Date & date) { cur = readNumber2(cur, end, date.hour); - date.is_clock_hour = false; - date.is_hour_of_half_day = false; - cur = assertChar(cur, end, ':'); cur = readNumber2(cur, end, date.minute); cur = assertChar(cur, end, ':'); cur = readNumber2(cur, end, date.second); + + date.is_clock_hour = false; + date.is_hour_of_half_day = false; return cur; } @@ -700,6 +732,7 @@ public: String format = getFormat(arguments); const auto * time_zone = getTimeZone(arguments).first; + // std::cout << "timezonename:" << getTimeZone(arguments).second << std::endl; std::vector instructions; parseFormat(format, instructions); @@ -710,11 +743,15 @@ public: for (size_t i = 0; i < input_rows_count; ++i) { StringRef str_ref = col_str->getDataAt(i); - Date date; Pos cur = str_ref.data; Pos end = str_ref.data + str_ref.size; + Date date; for (const auto & instruction : instructions) + { cur = instruction.perform(cur, end, date); + // std::cout << "instruction:" << instruction.toString() << std::endl; + // std::cout << "date:" << date.toString() << std::endl; + } // Ensure all input was consumed. if (cur < end) @@ -767,37 +804,37 @@ private: { // Abbreviated weekday [Mon...Sun] case 'a': - instructions.emplace_back(&Action::mysqlDayOfWeekTextShort); + instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfWeekTextShort)); break; // Abbreviated month [Jan...Dec] case 'b': - instructions.emplace_back(&Action::mysqlMonthOfYearTextShort); + instructions.emplace_back(ACTION_ARGS(Action::mysqlMonthOfYearTextShort)); break; // Month as a decimal number (01-12) case 'c': - instructions.emplace_back(&Action::mysqlMonth); + instructions.emplace_back(ACTION_ARGS(Action::mysqlMonth)); break; // Year, divided by 100, zero-padded case 'C': - instructions.emplace_back(&Action::mysqlCentury); + instructions.emplace_back(ACTION_ARGS(Action::mysqlCentury)); break; // Day of month, zero-padded (01-31) case 'd': - instructions.emplace_back(&Action::mysqlDayOfMonth); + instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfMonth)); break; // Short MM/DD/YY date, equivalent to %m/%d/%y case 'D': - instructions.emplace_back(&Action::mysqlAmericanDate); + instructions.emplace_back(ACTION_ARGS(Action::mysqlAmericanDate)); break; // Day of month, space-padded ( 1-31) 23 case 'e': - instructions.emplace_back(&Action::mysqlDayOfMonthSpacePadded); + instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfMonthSpacePadded)); break; // Fractional seconds @@ -806,57 +843,57 @@ private: // Short YYYY-MM-DD date, equivalent to %Y-%m-%d 2001-08-23 case 'F': - instructions.emplace_back(&Action::mysqlISO8601Date); + instructions.emplace_back(ACTION_ARGS(Action::mysqlISO8601Date)); break; // Last two digits of year of ISO 8601 week number (see %G) case 'g': - instructions.emplace_back(&Action::mysqlISO8601Year2); + instructions.emplace_back(ACTION_ARGS(Action::mysqlISO8601Year2)); break; // Year of ISO 8601 week number (see %V) case 'G': - instructions.emplace_back(&Action::mysqlISO8601Year4); + instructions.emplace_back(ACTION_ARGS(Action::mysqlISO8601Year4)); break; // Day of the year (001-366) 235 case 'j': - instructions.emplace_back(&Action::mysqlDayOfYear); + instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfYear)); break; // Month as a decimal number (01-12) case 'm': - instructions.emplace_back(&Action::mysqlMonth); + instructions.emplace_back(ACTION_ARGS(Action::mysqlMonth)); break; // ISO 8601 weekday as number with Monday as 1 (1-7) case 'u': - instructions.emplace_back(&Action::mysqlDayOfWeek); + instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfWeek)); break; // ISO 8601 week number (01-53) case 'V': - instructions.emplace_back(&Action::mysqlISO8601Week); + instructions.emplace_back(ACTION_ARGS(Action::mysqlISO8601Week)); break; // Weekday as a decimal number with Sunday as 0 (0-6) 4 case 'w': - instructions.emplace_back(&Action::mysqlDayOfWeek0To6); + instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfWeek0To6)); break; // Full weekday [Monday...Sunday] case 'W': - instructions.emplace_back(&Action::mysqlDayOfWeekTextLong); + instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfWeekTextLong)); break; // Two digits year case 'y': - instructions.emplace_back(&Action::mysqlYear2); + instructions.emplace_back(ACTION_ARGS(Action::mysqlYear2)); break; // Four digits year case 'Y': - instructions.emplace_back(&Action::mysqlYear4); + instructions.emplace_back(ACTION_ARGS(Action::mysqlYear4)); break; // Quarter (1-4) @@ -866,74 +903,74 @@ private: // Offset from UTC timezone as +hhmm or -hhmm case 'z': - instructions.emplace_back(&Action::mysqlTimezoneOffset); + instructions.emplace_back(ACTION_ARGS(Action::mysqlTimezoneOffset)); break; /// Time components. If the argument is Date, not a DateTime, then this components will have default value. // Minute (00-59) case 'M': - instructions.emplace_back(&Action::mysqlMinute); + instructions.emplace_back(ACTION_ARGS(Action::mysqlMinute)); break; // AM or PM case 'p': - instructions.emplace_back(&Action::mysqlAMPM); + instructions.emplace_back(ACTION_ARGS(Action::mysqlAMPM)); break; // 12-hour HH:MM time, equivalent to %h:%i %p 2:55 PM case 'r': - instructions.emplace_back(&Action::mysqlHHMM12); + instructions.emplace_back(ACTION_ARGS(Action::mysqlHHMM12)); break; // 24-hour HH:MM time, equivalent to %H:%i 14:55 case 'R': - instructions.emplace_back(&Action::mysqlHHMM24); + instructions.emplace_back(ACTION_ARGS(Action::mysqlHHMM24)); break; // Seconds case 's': - instructions.emplace_back(&Action::mysqlSecond); + instructions.emplace_back(ACTION_ARGS(Action::mysqlSecond)); break; // Seconds case 'S': - instructions.emplace_back(&Action::mysqlSecond); + instructions.emplace_back(ACTION_ARGS(Action::mysqlSecond)); break; // ISO 8601 time format (HH:MM:SS), equivalent to %H:%i:%S 14:55:02 case 'T': - instructions.emplace_back(&Action::mysqlISO8601Time); + instructions.emplace_back(ACTION_ARGS(Action::mysqlISO8601Time)); break; // Hour in 12h format (01-12) case 'h': - instructions.emplace_back(&Action::mysqlHour12); + instructions.emplace_back(ACTION_ARGS(Action::mysqlHour12)); break; // Hour in 24h format (00-23) case 'H': - instructions.emplace_back(&Action::mysqlHour24); + instructions.emplace_back(ACTION_ARGS(Action::mysqlHour24)); break; // Minute of hour range [0, 59] case 'i': - instructions.emplace_back(&Action::mysqlMinute); + instructions.emplace_back(ACTION_ARGS(Action::mysqlMinute)); break; // Hour in 12h format (01-12) case 'I': - instructions.emplace_back(&Action::mysqlHour12); + instructions.emplace_back(ACTION_ARGS(Action::mysqlHour12)); break; // Hour in 24h format (00-23) case 'k': - instructions.emplace_back(&Action::mysqlHour24); + instructions.emplace_back(ACTION_ARGS(Action::mysqlHour24)); break; // Hour in 12h format (01-12) case 'l': - instructions.emplace_back(&Action::mysqlHour12); + instructions.emplace_back(ACTION_ARGS(Action::mysqlHour12)); break; case 't': @@ -972,7 +1009,8 @@ private: } else { - instructions.emplace_back(std::string_view(pos, end - pos)); + if (pos < end) + instructions.emplace_back(std::string_view(pos, end - pos)); break; } } From 65caf40c3d829d53c2b25079c7be9d6649e5494a Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 24 Feb 2023 18:34:46 +0800 Subject: [PATCH 057/559] fix style --- src/Functions/parseDateTime.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 2e986835276..a80f26ec1c9 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -665,8 +665,7 @@ struct ParseDateTimeTraits }; }; -#define ACTION_ARGS(func) &func, #func - +#define ACTION_ARGS(func) &(func), #func /// _FUNC_(str[, format, timezone]) template From d372bf9e128327faced5ec10aad7558c6bb9cdf4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Feb 2023 11:47:22 +0100 Subject: [PATCH 058/559] Fix test --- src/TableFunctions/TableFunctionRemote.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 79f052854ec..66f04346f16 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -37,7 +37,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr String cluster_name; String cluster_description; - String database, table, username, password; + String database, table, username = "default", password; if (args_func.size() != 1) throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); From 0555ca27240c6864a080b60ddaec154f57ebf3e3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Feb 2023 14:44:47 +0100 Subject: [PATCH 059/559] Review fixes --- src/Access/AccessRights.cpp | 58 +++++++++++++++---- src/Access/Common/AccessFlags.cpp | 15 +++++ src/Access/Common/AccessFlags.h | 1 + src/Access/Common/AccessRightsElement.cpp | 19 +++--- src/Access/Common/AccessRightsElement.h | 8 +-- src/Access/ContextAccess.cpp | 9 ++- .../Access/InterpreterShowGrantsQuery.cpp | 2 +- src/Parsers/Access/ASTGrantQuery.cpp | 11 +++- src/Parsers/Access/ParserGrantQuery.cpp | 10 ++-- .../test_named_collections/test.py | 40 +++++++++++++ 10 files changed, 140 insertions(+), 33 deletions(-) diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index 6096612059a..420b1e34db4 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -61,23 +61,24 @@ namespace res.any_database = true; res.any_table = true; res.any_column = true; - res.any_global_with_parameter = true; + res.any_parameter = true; break; } case 1: { if (access_flags.isGlobalWithParameter()) { - res.any_global_with_parameter = false; + res.any_parameter = false; res.parameter = full_name[0]; } else { res.any_database = false; res.database = full_name[0]; + + res.any_table = true; + res.any_column = true; } - res.any_table = true; - res.any_column = true; break; } case 2: @@ -119,10 +120,26 @@ namespace size_t count_elements_with_diff_columns = sorted.countElementsWithDifferenceInColumnOnly(i); if (count_elements_with_diff_columns == 1) { - /// Easy case: one Element is converted to one AccessRightsElement. const auto & element = sorted[i]; if (element.access_flags) - res.emplace_back(element.getResult()); + { + auto per_parameter = element.access_flags.splitIntoParameterTypes(); + if (per_parameter.size() == 1) + { + /// Easy case: one Element is converted to one AccessRightsElement. + res.emplace_back(element.getResult()); + } + else + { + /// Difficult case: one element is converted into multiple AccessRightsElements. + for (const auto & [_, parameter_flags] : per_parameter) + { + auto current_element{element}; + current_element.access_flags = parameter_flags; + res.emplace_back(current_element.getResult()); + } + } + } ++i; } else @@ -146,6 +163,8 @@ namespace { return (element.full_name.size() != 3) || (element.full_name[0] != start_element.full_name[0]) || (element.full_name[1] != start_element.full_name[1]) || (element.grant_option != start_element.grant_option) + || (element.access_flags.isGlobalWithParameter() != start_element.access_flags.isGlobalWithParameter()) + || (element.access_flags.getParameterType() != start_element.access_flags.getParameterType()) || (element.is_partial_revoke != start_element.is_partial_revoke); }); @@ -792,8 +811,13 @@ void AccessRights::grantImplHelper(const AccessRightsElement & element) { assert(!element.is_partial_revoke); assert(!element.grant_option || with_grant_option); - if (!element.any_global_with_parameter) - grantImpl(element.access_flags, element.parameter); + if (element.isGlobalWithParameter()) + { + if (element.any_parameter) + grantImpl(element.access_flags); + else + grantImpl(element.access_flags, element.parameter); + } else if (element.any_database) grantImpl(element.access_flags); else if (element.any_table) @@ -869,8 +893,13 @@ template void AccessRights::revokeImplHelper(const AccessRightsElement & element) { assert(!element.grant_option || grant_option); - if (!element.any_global_with_parameter) - revokeImpl(element.access_flags, element.parameter); + if (element.isGlobalWithParameter()) + { + if (element.any_parameter) + revokeImpl(element.access_flags); + else + revokeImpl(element.access_flags, element.parameter); + } else if (element.any_database) revokeImpl(element.access_flags); else if (element.any_table) @@ -961,8 +990,13 @@ template bool AccessRights::isGrantedImplHelper(const AccessRightsElement & element) const { assert(!element.grant_option || grant_option); - if (!element.any_global_with_parameter) - return isGrantedImpl(element.access_flags, element.parameter); + if (element.isGlobalWithParameter()) + { + if (element.any_parameter) + return isGrantedImpl(element.access_flags); + else + return isGrantedImpl(element.access_flags, element.parameter); + } else if (element.any_database) return isGrantedImpl(element.access_flags); else if (element.any_table) diff --git a/src/Access/Common/AccessFlags.cpp b/src/Access/Common/AccessFlags.cpp index 4f8a9798ec4..8612fc2309e 100644 --- a/src/Access/Common/AccessFlags.cpp +++ b/src/Access/Common/AccessFlags.cpp @@ -363,6 +363,21 @@ bool AccessFlags::isGlobalWithParameter() const return getParameterType() != AccessFlags::NONE; } +std::unordered_map AccessFlags::splitIntoParameterTypes() const +{ + std::unordered_map result; + + auto named_collection_flags = AccessFlags::allNamedCollectionFlags() & *this; + if (named_collection_flags) + result.emplace(ParameterType::NAMED_COLLECTION, named_collection_flags); + + auto other_flags = (~AccessFlags::allNamedCollectionFlags()) & *this; + if (other_flags) + result.emplace(ParameterType::NONE, other_flags); + + return result; +} + AccessFlags::ParameterType AccessFlags::getParameterType() const { if (isEmpty() || !AccessFlags::allGlobalWithParameterFlags().contains(*this)) diff --git a/src/Access/Common/AccessFlags.h b/src/Access/Common/AccessFlags.h index b923b24be47..270ee1c0045 100644 --- a/src/Access/Common/AccessFlags.h +++ b/src/Access/Common/AccessFlags.h @@ -58,6 +58,7 @@ public: NAMED_COLLECTION, }; ParameterType getParameterType() const; + std::unordered_map splitIntoParameterTypes() const; friend bool operator ==(const AccessFlags & left, const AccessFlags & right) { return left.flags == right.flags; } friend bool operator !=(const AccessFlags & left, const AccessFlags & right) { return !(left == right); } diff --git a/src/Access/Common/AccessRightsElement.cpp b/src/Access/Common/AccessRightsElement.cpp index 2f6f1264a65..8d849297246 100644 --- a/src/Access/Common/AccessRightsElement.cpp +++ b/src/Access/Common/AccessRightsElement.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB @@ -26,7 +27,7 @@ namespace result += "ON "; if (element.isGlobalWithParameter()) { - if (element.any_global_with_parameter) + if (element.any_parameter) result += "*"; else result += backQuoteIfNeed(element.parameter); @@ -129,8 +130,10 @@ namespace if (i != elements.size() - 1) { const auto & next_element = elements[i + 1]; - if (element.sameDatabaseAndTable(next_element) && element.sameOptions(next_element)) + if (element.sameDatabaseAndTableAndParameter(next_element) && element.sameOptions(next_element)) + { next_element_uses_same_table_and_options = true; + } } if (!next_element_uses_same_table_and_options) @@ -171,6 +174,7 @@ AccessRightsElement::AccessRightsElement( , any_database(false) , any_table(false) , any_column(false) + , any_parameter(false) { } @@ -195,19 +199,20 @@ AccessRightsElement::AccessRightsElement( , any_database(false) , any_table(false) , any_column(false) + , any_parameter(false) { } void AccessRightsElement::eraseNonGrantable() { - if (!any_column) + if (isGlobalWithParameter() && !any_parameter) + access_flags &= AccessFlags::allFlagsGrantableOnGlobalWithParameterLevel(); + else if (!any_column) access_flags &= AccessFlags::allFlagsGrantableOnColumnLevel(); else if (!any_table) access_flags &= AccessFlags::allFlagsGrantableOnTableLevel(); else if (!any_database) access_flags &= AccessFlags::allFlagsGrantableOnDatabaseLevel(); - else if (!any_global_with_parameter) - access_flags &= AccessFlags::allFlagsGrantableOnGlobalWithParameterLevel(); else access_flags &= AccessFlags::allFlagsGrantableOnGlobalLevel(); } @@ -224,9 +229,9 @@ String AccessRightsElement::toStringWithoutOptions() const { return toStringImpl bool AccessRightsElements::empty() const { return std::all_of(begin(), end(), [](const AccessRightsElement & e) { return e.empty(); }); } -bool AccessRightsElements::sameDatabaseAndTable() const +bool AccessRightsElements::sameDatabaseAndTableAndParameter() const { - return (size() < 2) || std::all_of(std::next(begin()), end(), [this](const AccessRightsElement & e) { return e.sameDatabaseAndTable(front()); }); + return (size() < 2) || std::all_of(std::next(begin()), end(), [this](const AccessRightsElement & e) { return e.sameDatabaseAndTableAndParameter(front()); }); } bool AccessRightsElements::sameOptions() const diff --git a/src/Access/Common/AccessRightsElement.h b/src/Access/Common/AccessRightsElement.h index e881767b185..247b1e4e455 100644 --- a/src/Access/Common/AccessRightsElement.h +++ b/src/Access/Common/AccessRightsElement.h @@ -20,7 +20,7 @@ struct AccessRightsElement bool any_database = true; bool any_table = true; bool any_column = true; - bool any_global_with_parameter = true; + bool any_parameter = false; bool grant_option = false; bool is_partial_revoke = false; @@ -53,11 +53,11 @@ struct AccessRightsElement friend bool operator==(const AccessRightsElement & left, const AccessRightsElement & right) { return left.toTuple() == right.toTuple(); } friend bool operator!=(const AccessRightsElement & left, const AccessRightsElement & right) { return !(left == right); } - bool sameDatabaseAndTable(const AccessRightsElement & other) const + bool sameDatabaseAndTableAndParameter(const AccessRightsElement & other) const { return (database == other.database) && (any_database == other.any_database) && (table == other.table) && (any_table == other.any_table) - && (parameter == other.parameter) && (any_global_with_parameter == other.any_global_with_parameter) + && (parameter == other.parameter) && (any_parameter == other.any_parameter) && (access_flags.getParameterType() == other.access_flags.getParameterType()) && (isGlobalWithParameter() == other.isGlobalWithParameter()); } @@ -91,7 +91,7 @@ public: using Base::Base; bool empty() const; - bool sameDatabaseAndTable() const; + bool sameDatabaseAndTableAndParameter() const; bool sameOptions() const; /// Resets flags which cannot be granted. diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 4abb161fd80..5f84326d210 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -610,8 +610,13 @@ template bool ContextAccess::checkAccessImplHelper(const AccessRightsElement & element) const { assert(!element.grant_option || grant_option); - if (!element.any_global_with_parameter) - return checkAccessImpl(element.access_flags, element.parameter); + if (element.isGlobalWithParameter()) + { + if (element.any_parameter) + return checkAccessImpl(element.access_flags); + else + return checkAccessImpl(element.access_flags, element.parameter); + } else if (element.any_database) return checkAccessImpl(element.access_flags); else if (element.any_table) diff --git a/src/Interpreters/Access/InterpreterShowGrantsQuery.cpp b/src/Interpreters/Access/InterpreterShowGrantsQuery.cpp index 2ff6d44e041..56fbb34a577 100644 --- a/src/Interpreters/Access/InterpreterShowGrantsQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowGrantsQuery.cpp @@ -48,7 +48,7 @@ namespace if (current_query) { const auto & prev_element = current_query->access_rights_elements.back(); - bool continue_with_current_query = element.sameDatabaseAndTable(prev_element) && element.sameOptions(prev_element); + bool continue_with_current_query = element.sameDatabaseAndTableAndParameter(prev_element) && element.sameOptions(prev_element); if (!continue_with_current_query) current_query = nullptr; } diff --git a/src/Parsers/Access/ASTGrantQuery.cpp b/src/Parsers/Access/ASTGrantQuery.cpp index f1a1f9184a5..71eff476965 100644 --- a/src/Parsers/Access/ASTGrantQuery.cpp +++ b/src/Parsers/Access/ASTGrantQuery.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB @@ -32,7 +33,7 @@ namespace settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "ON " << (settings.hilite ? IAST::hilite_none : ""); if (element.isGlobalWithParameter()) { - if (element.any_global_with_parameter) + if (element.any_parameter) settings.ostr << "*"; else settings.ostr << backQuoteIfNeed(element.parameter); @@ -56,6 +57,8 @@ namespace void formatElementsWithoutOptions(const AccessRightsElements & elements, const IAST::FormatSettings & settings) { bool no_output = true; + auto * log = &Poco::Logger::get("kssenii"); + LOG_TEST(log, "kssenii 0 - {}", elements.size()); for (size_t i = 0; i != elements.size(); ++i) { const auto & element = elements[i]; @@ -77,12 +80,16 @@ namespace if (i != elements.size() - 1) { const auto & next_element = elements[i + 1]; - if (element.sameDatabaseAndTable(next_element)) + if (element.sameDatabaseAndTableAndParameter(next_element)) + { + LOG_TEST(log, "kssenii 1"); next_element_on_same_db_and_table = true; + } } if (!next_element_on_same_db_and_table) { + LOG_TEST(log, "kssenii 2"); settings.ostr << " "; formatONClause(element, settings); } diff --git a/src/Parsers/Access/ParserGrantQuery.cpp b/src/Parsers/Access/ParserGrantQuery.cpp index d58599ead56..28a1846df74 100644 --- a/src/Parsers/Access/ParserGrantQuery.cpp +++ b/src/Parsers/Access/ParserGrantQuery.cpp @@ -124,7 +124,7 @@ namespace return false; String database_name, table_name, parameter; - bool any_database = false, any_table = false, any_global_with_parameter = true; + bool any_database = false, any_table = false, any_parameter = false; size_t is_global_with_parameter = 0; for (const auto & elem : access_and_columns) @@ -141,11 +141,11 @@ namespace ASTPtr parameter_ast; if (ParserToken{TokenType::Asterisk}.ignore(pos, expected)) { - any_global_with_parameter = true; + any_parameter = true; } else if (ParserIdentifier{}.parse(pos, parameter_ast, expected)) { - any_global_with_parameter = false; + any_parameter = false; parameter = getIdentifierName(parameter_ast); } else @@ -167,7 +167,7 @@ namespace element.any_database = any_database; element.database = database_name; element.any_table = any_table; - element.any_global_with_parameter = any_global_with_parameter; + element.any_parameter = any_parameter; element.table = table_name; element.parameter = parameter; res_elements.emplace_back(std::move(element)); @@ -202,7 +202,7 @@ namespace throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted on the table level", old_flags.toString()); else if (!element.any_database) throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted on the database level", old_flags.toString()); - else if (!element.any_global_with_parameter) + else if (!element.any_parameter) throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted on the global with parameter level", old_flags.toString()); else throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted", old_flags.toString()); diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index ba403d3f48b..4e9b8324aea 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -150,6 +150,10 @@ def test_granular_access_show_query(cluster): == node.query("select name from system.named_collections", user="kek").strip() ) + # check: + # GRANT show named collections ON * + # REVOKE show named collections ON collection + node.query("DROP USER IF EXISTS koko") node.query("CREATE USER koko") node.query("GRANT select ON *.* TO koko") @@ -167,6 +171,35 @@ def test_granular_access_show_query(cluster): == node.query("select name from system.named_collections", user="koko").strip() ) + node.query("REVOKE show named collections ON collection1 FROM koko;") + assert ( + "collection2" + == node.query("select name from system.named_collections", user="koko").strip() + ) + node.restart_clickhouse() + assert ( + "collection2" + == node.query("select name from system.named_collections", user="koko").strip() + ) + node.query("REVOKE show named collections ON collection2 FROM koko;") + assert ( + "" == node.query("select * from system.named_collections", user="koko").strip() + ) + + # check: + # GRANT show named collections ON collection + # REVOKE show named collections ON * + + node.query("GRANT show named collections ON collection2 TO koko") + assert ( + "collection2" + == node.query("select name from system.named_collections", user="koko").strip() + ) + node.query("REVOKE show named collections ON * FROM koko;") + assert ( + "" == node.query("select * from system.named_collections", user="koko").strip() + ) + node.query("DROP NAMED COLLECTION collection2") @@ -219,6 +252,13 @@ def test_granular_access_create_alter_drop_query(cluster): "select collection['key1'] from system.named_collections where name = 'collection2'" ).strip() ) + node.query("REVOKE create named collection ON collection2 FROM kek") + assert ( + "DB::Exception: kek: Not enough privileges. To execute this query it's necessary to have grant ALTER NAMED COLLECTION" + in node.query_and_get_error( + "ALTER NAMED COLLECTION collection2 SET key1=3", user="kek" + ) + ) assert ( "DB::Exception: kek: Not enough privileges. To execute this query it's necessary to have grant DROP NAMED COLLECTION" From 8b40723fedf11b162864e56abf9c337e1b775a04 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Feb 2023 16:19:36 +0100 Subject: [PATCH 060/559] Better fix --- .../test_named_collections/test.py | 50 ++++++++++++++++++- 1 file changed, 49 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index f219fa7a632..899ae130404 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -228,6 +228,54 @@ def test_granular_access_show_query(cluster): node.query("DROP NAMED COLLECTION collection2") +def test_show_grants(cluster): + node = cluster.instances["node"] + node.query("DROP USER IF EXISTS koko") + node.query("CREATE USER koko") + node.query("GRANT CREATE NAMED COLLECTION ON name1 TO koko") + node.query("GRANT select ON name1.* TO koko") + assert ( + "GRANT SELECT ON name1.* TO koko\nGRANT CREATE NAMED COLLECTION ON name1 TO koko" + in node.query("SHOW GRANTS FOR koko;").strip() + ) + + node.query("DROP USER IF EXISTS koko") + node.query("CREATE USER koko") + node.query("GRANT CREATE NAMED COLLECTION ON name1 TO koko") + node.query("GRANT select ON name1 TO koko") + assert ( + "GRANT SELECT ON default.name1 TO koko\nGRANT CREATE NAMED COLLECTION ON name1 TO koko" + in node.query("SHOW GRANTS FOR koko;").strip() + ) + + node.query("DROP USER IF EXISTS koko") + node.query("CREATE USER koko") + node.query("GRANT select ON name1 TO koko") + node.query("GRANT CREATE NAMED COLLECTION ON name1 TO koko") + assert ( + "GRANT SELECT ON default.name1 TO koko\nGRANT CREATE NAMED COLLECTION ON name1 TO koko" + in node.query("SHOW GRANTS FOR koko;").strip() + ) + + node.query("DROP USER IF EXISTS koko") + node.query("CREATE USER koko") + node.query("GRANT select ON *.* TO koko") + node.query("GRANT CREATE NAMED COLLECTION ON * TO koko") + assert ( + "GRANT SELECT ON *.* TO koko\nGRANT CREATE NAMED COLLECTION ON * TO koko" + in node.query("SHOW GRANTS FOR koko;").strip() + ) + + node.query("DROP USER IF EXISTS koko") + node.query("CREATE USER koko") + node.query("GRANT CREATE NAMED COLLECTION ON * TO koko") + node.query("GRANT select ON *.* TO koko") + assert ( + "GRANT SELECT ON *.* TO koko\nGRANT CREATE NAMED COLLECTION ON * TO koko" + in node.query("SHOW GRANTS FOR koko;").strip() + ) + + def test_granular_access_create_alter_drop_query(cluster): node = cluster.instances["node"] node.query("DROP USER IF EXISTS kek") @@ -277,7 +325,7 @@ def test_granular_access_create_alter_drop_query(cluster): "select collection['key1'] from system.named_collections where name = 'collection2'" ).strip() ) - node.query("REVOKE create named collection ON collection2 FROM kek") + node.query("REVOKE alter named collection ON collection2 FROM kek") assert ( "DB::Exception: kek: Not enough privileges. To execute this query it's necessary to have grant ALTER NAMED COLLECTION" in node.query_and_get_error( From 357ffcb61f4e568642a4e8270d37c883e22c203f Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Feb 2023 16:49:25 +0100 Subject: [PATCH 061/559] More review fixes --- src/Access/Common/AccessType.h | 9 +++++---- .../integration/test_named_collections/test.py | 18 ++++++++++++++++++ 2 files changed, 23 insertions(+), 4 deletions(-) diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index ba39a984358..84ec93d58f6 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -89,7 +89,7 @@ enum class AccessType M(CREATE_TEMPORARY_TABLE, "", GLOBAL, CREATE) /* allows to create and manipulate temporary tables; implicitly enabled by the grant CREATE_TABLE on any table */ \ M(CREATE_FUNCTION, "", GLOBAL, CREATE) /* allows to execute CREATE FUNCTION */ \ - M(CREATE_NAMED_COLLECTION, "", NAMED_COLLECTION, CREATE) /* allows to execute CREATE NAMED COLLECTION */ \ + M(CREATE_NAMED_COLLECTION, "", NAMED_COLLECTION, NAMED_COLLECTION_CONTROL) /* allows to execute CREATE NAMED COLLECTION */ \ M(CREATE, "", GROUP, ALL) /* allows to execute {CREATE|ATTACH} */ \ \ M(DROP_DATABASE, "", DATABASE, DROP) /* allows to execute {DROP|DETACH} DATABASE */\ @@ -98,7 +98,7 @@ enum class AccessType implicitly enabled by the grant DROP_TABLE */\ M(DROP_DICTIONARY, "", DICTIONARY, DROP) /* allows to execute {DROP|DETACH} DICTIONARY */\ M(DROP_FUNCTION, "", GLOBAL, DROP) /* allows to execute DROP FUNCTION */\ - M(DROP_NAMED_COLLECTION, "", NAMED_COLLECTION, DROP) /* allows to execute DROP NAMED COLLECTION */\ + M(DROP_NAMED_COLLECTION, "", NAMED_COLLECTION, NAMED_COLLECTION_CONTROL) /* allows to execute DROP NAMED COLLECTION */\ M(DROP, "", GROUP, ALL) /* allows to execute {DROP|DETACH} */\ \ M(TRUNCATE, "TRUNCATE TABLE", TABLE, ALL) \ @@ -134,9 +134,10 @@ enum class AccessType M(SHOW_QUOTAS, "SHOW CREATE QUOTA", GLOBAL, SHOW_ACCESS) \ M(SHOW_SETTINGS_PROFILES, "SHOW PROFILES, SHOW CREATE SETTINGS PROFILE, SHOW CREATE PROFILE", GLOBAL, SHOW_ACCESS) \ M(SHOW_ACCESS, "", GROUP, ACCESS_MANAGEMENT) \ - M(SHOW_NAMED_COLLECTIONS, "SHOW NAMED COLLECTIONS", NAMED_COLLECTION, ACCESS_MANAGEMENT) \ - M(SHOW_NAMED_COLLECTIONS_SECRETS, "SHOW NAMED COLLECTIONS SECRETS", NAMED_COLLECTION, ACCESS_MANAGEMENT) \ M(ACCESS_MANAGEMENT, "", GROUP, ALL) \ + M(SHOW_NAMED_COLLECTIONS, "SHOW NAMED COLLECTIONS", NAMED_COLLECTION, NAMED_COLLECTION_CONTROL) \ + M(SHOW_NAMED_COLLECTIONS_SECRETS, "SHOW NAMED COLLECTIONS SECRETS", NAMED_COLLECTION, NAMED_COLLECTION_CONTROL) \ + M(NAMED_COLLECTION_CONTROL, "", NAMED_COLLECTION, ALL) \ \ M(SYSTEM_SHUTDOWN, "SYSTEM KILL, SHUTDOWN", GLOBAL, SYSTEM) \ M(SYSTEM_DROP_DNS_CACHE, "SYSTEM DROP DNS, DROP DNS CACHE, DROP DNS", GLOBAL, SYSTEM_DROP_CACHE) \ diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index 899ae130404..1e9995a0603 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -275,6 +275,24 @@ def test_show_grants(cluster): in node.query("SHOW GRANTS FOR koko;").strip() ) + node.query("DROP USER IF EXISTS koko") + node.query("CREATE USER koko") + node.query("GRANT CREATE NAMED COLLECTION ON * TO koko") + node.query("GRANT select ON * TO koko") + assert ( + "GRANT CREATE NAMED COLLECTION ON * TO koko\nGRANT SELECT ON default.* TO koko" + in node.query("SHOW GRANTS FOR koko;").strip() + ) + + node.query("DROP USER IF EXISTS koko") + node.query("CREATE USER koko") + node.query("GRANT select ON * TO koko") + node.query("GRANT CREATE NAMED COLLECTION ON * TO koko") + assert ( + "GRANT SELECT ON default.* TO koko\nGRANT CREATE NAMED COLLECTION ON * TO koko" + in node.query("SHOW GRANTS FOR koko;").strip() + ) + def test_granular_access_create_alter_drop_query(cluster): node = cluster.instances["node"] From 6224ca9cd68c03a7b7f257baa235e7f0cf1d7311 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Feb 2023 17:08:05 +0100 Subject: [PATCH 062/559] Fix --- src/Access/AccessRights.cpp | 8 ++++++++ src/Storages/System/StorageSystemNamedCollections.cpp | 1 - tests/integration/test_named_collections/test.py | 2 +- 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index 420b1e34db4..424135acb81 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -219,11 +219,19 @@ namespace } }; + /** + * Levels: + * 1. GLOBAL + * 2. DATABASE_LEVEL 2. GLOBAL_WITH_PARAMETER (parameter example: named collection) + * 3. TABLE_LEVEL + * 4. COLUMN_LEVEL + */ enum Level { GLOBAL_LEVEL, DATABASE_LEVEL, + GLOBAL_WITH_PARAMETER = DATABASE_LEVEL, TABLE_LEVEL, COLUMN_LEVEL, }; diff --git a/src/Storages/System/StorageSystemNamedCollections.cpp b/src/Storages/System/StorageSystemNamedCollections.cpp index baba93aa3d5..1d94b0afd1b 100644 --- a/src/Storages/System/StorageSystemNamedCollections.cpp +++ b/src/Storages/System/StorageSystemNamedCollections.cpp @@ -10,7 +10,6 @@ #include #include #include -#include namespace DB diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index 1e9995a0603..6a53f7e0a58 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -289,7 +289,7 @@ def test_show_grants(cluster): node.query("GRANT select ON * TO koko") node.query("GRANT CREATE NAMED COLLECTION ON * TO koko") assert ( - "GRANT SELECT ON default.* TO koko\nGRANT CREATE NAMED COLLECTION ON * TO koko" + "GRANT CREATE NAMED COLLECTION ON * TO koko\nGRANT SELECT ON default.* TO koko" in node.query("SHOW GRANTS FOR koko;").strip() ) From d55fed77e365514ffb77a99ec626bd5cc3eefe34 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Feb 2023 18:02:31 +0100 Subject: [PATCH 063/559] Fix --- src/Access/AccessRights.cpp | 7 ++++--- src/Access/Common/AccessRightsElement.h | 2 +- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index 424135acb81..37597552a41 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -68,14 +68,15 @@ namespace { if (access_flags.isGlobalWithParameter()) { - res.any_parameter = false; res.parameter = full_name[0]; + res.any_parameter = false; + res.any_database = false; } else { - res.any_database = false; res.database = full_name[0]; - + res.any_database = false; + res.any_parameter = false; res.any_table = true; res.any_column = true; } diff --git a/src/Access/Common/AccessRightsElement.h b/src/Access/Common/AccessRightsElement.h index 247b1e4e455..96850f0880e 100644 --- a/src/Access/Common/AccessRightsElement.h +++ b/src/Access/Common/AccessRightsElement.h @@ -49,7 +49,7 @@ struct AccessRightsElement bool empty() const { return !access_flags || (!any_column && columns.empty()); } - auto toTuple() const { return std::tie(access_flags, any_database, database, any_table, table, any_column, columns, grant_option, is_partial_revoke); } + auto toTuple() const { return std::tie(access_flags, any_database, database, any_table, table, any_column, columns, any_parameter, parameter, grant_option, is_partial_revoke); } friend bool operator==(const AccessRightsElement & left, const AccessRightsElement & right) { return left.toTuple() == right.toTuple(); } friend bool operator!=(const AccessRightsElement & left, const AccessRightsElement & right) { return !(left == right); } From 95f414200fd2a99ae5f9a9173784b70d488b403b Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Feb 2023 18:05:03 +0100 Subject: [PATCH 064/559] Update .reference --- .../queries/0_stateless/01271_show_privileges.reference | 9 +++++---- .../0_stateless/02117_show_create_table_system.reference | 6 +++--- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index aa43c91ed72..03661d2469f 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -52,14 +52,14 @@ CREATE VIEW [] VIEW CREATE CREATE DICTIONARY [] DICTIONARY CREATE CREATE TEMPORARY TABLE [] GLOBAL CREATE CREATE FUNCTION [] GLOBAL CREATE -CREATE NAMED COLLECTION [] NAMED_COLLECTION CREATE +CREATE NAMED COLLECTION [] NAMED_COLLECTION NAMED COLLECTION CONTROL CREATE [] \N ALL DROP DATABASE [] DATABASE DROP DROP TABLE [] TABLE DROP DROP VIEW [] VIEW DROP DROP DICTIONARY [] DICTIONARY DROP DROP FUNCTION [] GLOBAL DROP -DROP NAMED COLLECTION [] NAMED_COLLECTION DROP +DROP NAMED COLLECTION [] NAMED_COLLECTION NAMED COLLECTION CONTROL DROP [] \N ALL TRUNCATE ['TRUNCATE TABLE'] TABLE ALL OPTIMIZE ['OPTIMIZE TABLE'] TABLE ALL @@ -89,9 +89,10 @@ SHOW ROW POLICIES ['SHOW POLICIES','SHOW CREATE ROW POLICY','SHOW CREATE POLICY' SHOW QUOTAS ['SHOW CREATE QUOTA'] GLOBAL SHOW ACCESS SHOW SETTINGS PROFILES ['SHOW PROFILES','SHOW CREATE SETTINGS PROFILE','SHOW CREATE PROFILE'] GLOBAL SHOW ACCESS SHOW ACCESS [] \N ACCESS MANAGEMENT -SHOW NAMED COLLECTIONS ['SHOW NAMED COLLECTIONS'] NAMED_COLLECTION ACCESS MANAGEMENT -SHOW NAMED COLLECTIONS SECRETS ['SHOW NAMED COLLECTIONS SECRETS'] NAMED_COLLECTION ACCESS MANAGEMENT ACCESS MANAGEMENT [] \N ALL +SHOW NAMED COLLECTIONS ['SHOW NAMED COLLECTIONS'] NAMED_COLLECTION NAMED COLLECTION CONTROL +SHOW NAMED COLLECTIONS SECRETS ['SHOW NAMED COLLECTIONS SECRETS'] NAMED_COLLECTION NAMED COLLECTION CONTROL +NAMED COLLECTION CONTROL [] NAMED_COLLECTION ALL SYSTEM SHUTDOWN ['SYSTEM KILL','SHUTDOWN'] GLOBAL SYSTEM SYSTEM DROP DNS CACHE ['SYSTEM DROP DNS','DROP DNS CACHE','DROP DNS'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP MARK CACHE ['SYSTEM DROP MARK','DROP MARK CACHE','DROP MARKS'] GLOBAL SYSTEM DROP CACHE diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 4c00b65d269..90e018703b1 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -289,7 +289,7 @@ CREATE TABLE system.grants ( `user_name` Nullable(String), `role_name` Nullable(String), - `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'SHOW NAMED COLLECTIONS SECRETS' = 92, 'ACCESS MANAGEMENT' = 93, 'SYSTEM SHUTDOWN' = 94, 'SYSTEM DROP DNS CACHE' = 95, 'SYSTEM DROP MARK CACHE' = 96, 'SYSTEM DROP UNCOMPRESSED CACHE' = 97, 'SYSTEM DROP MMAP CACHE' = 98, 'SYSTEM DROP QUERY CACHE' = 99, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 100, 'SYSTEM DROP FILESYSTEM CACHE' = 101, 'SYSTEM DROP SCHEMA CACHE' = 102, 'SYSTEM DROP S3 CLIENT CACHE' = 103, 'SYSTEM DROP CACHE' = 104, 'SYSTEM RELOAD CONFIG' = 105, 'SYSTEM RELOAD USERS' = 106, 'SYSTEM RELOAD SYMBOLS' = 107, 'SYSTEM RELOAD DICTIONARY' = 108, 'SYSTEM RELOAD MODEL' = 109, 'SYSTEM RELOAD FUNCTION' = 110, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 111, 'SYSTEM RELOAD' = 112, 'SYSTEM RESTART DISK' = 113, 'SYSTEM MERGES' = 114, 'SYSTEM TTL MERGES' = 115, 'SYSTEM FETCHES' = 116, 'SYSTEM MOVES' = 117, 'SYSTEM DISTRIBUTED SENDS' = 118, 'SYSTEM REPLICATED SENDS' = 119, 'SYSTEM SENDS' = 120, 'SYSTEM REPLICATION QUEUES' = 121, 'SYSTEM DROP REPLICA' = 122, 'SYSTEM SYNC REPLICA' = 123, 'SYSTEM RESTART REPLICA' = 124, 'SYSTEM RESTORE REPLICA' = 125, 'SYSTEM WAIT LOADING PARTS' = 126, 'SYSTEM SYNC DATABASE REPLICA' = 127, 'SYSTEM SYNC TRANSACTION LOG' = 128, 'SYSTEM SYNC FILE CACHE' = 129, 'SYSTEM FLUSH DISTRIBUTED' = 130, 'SYSTEM FLUSH LOGS' = 131, 'SYSTEM FLUSH' = 132, 'SYSTEM THREAD FUZZER' = 133, 'SYSTEM UNFREEZE' = 134, 'SYSTEM' = 135, 'dictGet' = 136, 'addressToLine' = 137, 'addressToLineWithInlines' = 138, 'addressToSymbol' = 139, 'demangle' = 140, 'INTROSPECTION' = 141, 'FILE' = 142, 'URL' = 143, 'REMOTE' = 144, 'MONGO' = 145, 'MEILISEARCH' = 146, 'MYSQL' = 147, 'POSTGRES' = 148, 'SQLITE' = 149, 'ODBC' = 150, 'JDBC' = 151, 'HDFS' = 152, 'S3' = 153, 'HIVE' = 154, 'SOURCES' = 155, 'CLUSTER' = 156, 'ALL' = 157, 'NONE' = 158), + `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'ACCESS MANAGEMENT' = 91, 'SHOW NAMED COLLECTIONS' = 92, 'SHOW NAMED COLLECTIONS SECRETS' = 93, 'NAMED COLLECTION CONTROL' = 94, 'SYSTEM SHUTDOWN' = 95, 'SYSTEM DROP DNS CACHE' = 96, 'SYSTEM DROP MARK CACHE' = 97, 'SYSTEM DROP UNCOMPRESSED CACHE' = 98, 'SYSTEM DROP MMAP CACHE' = 99, 'SYSTEM DROP QUERY CACHE' = 100, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 101, 'SYSTEM DROP FILESYSTEM CACHE' = 102, 'SYSTEM DROP SCHEMA CACHE' = 103, 'SYSTEM DROP S3 CLIENT CACHE' = 104, 'SYSTEM DROP CACHE' = 105, 'SYSTEM RELOAD CONFIG' = 106, 'SYSTEM RELOAD USERS' = 107, 'SYSTEM RELOAD SYMBOLS' = 108, 'SYSTEM RELOAD DICTIONARY' = 109, 'SYSTEM RELOAD MODEL' = 110, 'SYSTEM RELOAD FUNCTION' = 111, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 112, 'SYSTEM RELOAD' = 113, 'SYSTEM RESTART DISK' = 114, 'SYSTEM MERGES' = 115, 'SYSTEM TTL MERGES' = 116, 'SYSTEM FETCHES' = 117, 'SYSTEM MOVES' = 118, 'SYSTEM DISTRIBUTED SENDS' = 119, 'SYSTEM REPLICATED SENDS' = 120, 'SYSTEM SENDS' = 121, 'SYSTEM REPLICATION QUEUES' = 122, 'SYSTEM DROP REPLICA' = 123, 'SYSTEM SYNC REPLICA' = 124, 'SYSTEM RESTART REPLICA' = 125, 'SYSTEM RESTORE REPLICA' = 126, 'SYSTEM WAIT LOADING PARTS' = 127, 'SYSTEM SYNC DATABASE REPLICA' = 128, 'SYSTEM SYNC TRANSACTION LOG' = 129, 'SYSTEM SYNC FILE CACHE' = 130, 'SYSTEM FLUSH DISTRIBUTED' = 131, 'SYSTEM FLUSH LOGS' = 132, 'SYSTEM FLUSH' = 133, 'SYSTEM THREAD FUZZER' = 134, 'SYSTEM UNFREEZE' = 135, 'SYSTEM' = 136, 'dictGet' = 137, 'addressToLine' = 138, 'addressToLineWithInlines' = 139, 'addressToSymbol' = 140, 'demangle' = 141, 'INTROSPECTION' = 142, 'FILE' = 143, 'URL' = 144, 'REMOTE' = 145, 'MONGO' = 146, 'MEILISEARCH' = 147, 'MYSQL' = 148, 'POSTGRES' = 149, 'SQLITE' = 150, 'ODBC' = 151, 'JDBC' = 152, 'HDFS' = 153, 'S3' = 154, 'HIVE' = 155, 'SOURCES' = 156, 'CLUSTER' = 157, 'ALL' = 158, 'NONE' = 159), `database` Nullable(String), `table` Nullable(String), `column` Nullable(String), @@ -570,10 +570,10 @@ ENGINE = SystemPartsColumns COMMENT 'SYSTEM TABLE is built on the fly.' CREATE TABLE system.privileges ( - `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'SHOW NAMED COLLECTIONS SECRETS' = 92, 'ACCESS MANAGEMENT' = 93, 'SYSTEM SHUTDOWN' = 94, 'SYSTEM DROP DNS CACHE' = 95, 'SYSTEM DROP MARK CACHE' = 96, 'SYSTEM DROP UNCOMPRESSED CACHE' = 97, 'SYSTEM DROP MMAP CACHE' = 98, 'SYSTEM DROP QUERY CACHE' = 99, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 100, 'SYSTEM DROP FILESYSTEM CACHE' = 101, 'SYSTEM DROP SCHEMA CACHE' = 102, 'SYSTEM DROP S3 CLIENT CACHE' = 103, 'SYSTEM DROP CACHE' = 104, 'SYSTEM RELOAD CONFIG' = 105, 'SYSTEM RELOAD USERS' = 106, 'SYSTEM RELOAD SYMBOLS' = 107, 'SYSTEM RELOAD DICTIONARY' = 108, 'SYSTEM RELOAD MODEL' = 109, 'SYSTEM RELOAD FUNCTION' = 110, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 111, 'SYSTEM RELOAD' = 112, 'SYSTEM RESTART DISK' = 113, 'SYSTEM MERGES' = 114, 'SYSTEM TTL MERGES' = 115, 'SYSTEM FETCHES' = 116, 'SYSTEM MOVES' = 117, 'SYSTEM DISTRIBUTED SENDS' = 118, 'SYSTEM REPLICATED SENDS' = 119, 'SYSTEM SENDS' = 120, 'SYSTEM REPLICATION QUEUES' = 121, 'SYSTEM DROP REPLICA' = 122, 'SYSTEM SYNC REPLICA' = 123, 'SYSTEM RESTART REPLICA' = 124, 'SYSTEM RESTORE REPLICA' = 125, 'SYSTEM WAIT LOADING PARTS' = 126, 'SYSTEM SYNC DATABASE REPLICA' = 127, 'SYSTEM SYNC TRANSACTION LOG' = 128, 'SYSTEM SYNC FILE CACHE' = 129, 'SYSTEM FLUSH DISTRIBUTED' = 130, 'SYSTEM FLUSH LOGS' = 131, 'SYSTEM FLUSH' = 132, 'SYSTEM THREAD FUZZER' = 133, 'SYSTEM UNFREEZE' = 134, 'SYSTEM' = 135, 'dictGet' = 136, 'addressToLine' = 137, 'addressToLineWithInlines' = 138, 'addressToSymbol' = 139, 'demangle' = 140, 'INTROSPECTION' = 141, 'FILE' = 142, 'URL' = 143, 'REMOTE' = 144, 'MONGO' = 145, 'MEILISEARCH' = 146, 'MYSQL' = 147, 'POSTGRES' = 148, 'SQLITE' = 149, 'ODBC' = 150, 'JDBC' = 151, 'HDFS' = 152, 'S3' = 153, 'HIVE' = 154, 'SOURCES' = 155, 'CLUSTER' = 156, 'ALL' = 157, 'NONE' = 158), + `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'ACCESS MANAGEMENT' = 91, 'SHOW NAMED COLLECTIONS' = 92, 'SHOW NAMED COLLECTIONS SECRETS' = 93, 'NAMED COLLECTION CONTROL' = 94, 'SYSTEM SHUTDOWN' = 95, 'SYSTEM DROP DNS CACHE' = 96, 'SYSTEM DROP MARK CACHE' = 97, 'SYSTEM DROP UNCOMPRESSED CACHE' = 98, 'SYSTEM DROP MMAP CACHE' = 99, 'SYSTEM DROP QUERY CACHE' = 100, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 101, 'SYSTEM DROP FILESYSTEM CACHE' = 102, 'SYSTEM DROP SCHEMA CACHE' = 103, 'SYSTEM DROP S3 CLIENT CACHE' = 104, 'SYSTEM DROP CACHE' = 105, 'SYSTEM RELOAD CONFIG' = 106, 'SYSTEM RELOAD USERS' = 107, 'SYSTEM RELOAD SYMBOLS' = 108, 'SYSTEM RELOAD DICTIONARY' = 109, 'SYSTEM RELOAD MODEL' = 110, 'SYSTEM RELOAD FUNCTION' = 111, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 112, 'SYSTEM RELOAD' = 113, 'SYSTEM RESTART DISK' = 114, 'SYSTEM MERGES' = 115, 'SYSTEM TTL MERGES' = 116, 'SYSTEM FETCHES' = 117, 'SYSTEM MOVES' = 118, 'SYSTEM DISTRIBUTED SENDS' = 119, 'SYSTEM REPLICATED SENDS' = 120, 'SYSTEM SENDS' = 121, 'SYSTEM REPLICATION QUEUES' = 122, 'SYSTEM DROP REPLICA' = 123, 'SYSTEM SYNC REPLICA' = 124, 'SYSTEM RESTART REPLICA' = 125, 'SYSTEM RESTORE REPLICA' = 126, 'SYSTEM WAIT LOADING PARTS' = 127, 'SYSTEM SYNC DATABASE REPLICA' = 128, 'SYSTEM SYNC TRANSACTION LOG' = 129, 'SYSTEM SYNC FILE CACHE' = 130, 'SYSTEM FLUSH DISTRIBUTED' = 131, 'SYSTEM FLUSH LOGS' = 132, 'SYSTEM FLUSH' = 133, 'SYSTEM THREAD FUZZER' = 134, 'SYSTEM UNFREEZE' = 135, 'SYSTEM' = 136, 'dictGet' = 137, 'addressToLine' = 138, 'addressToLineWithInlines' = 139, 'addressToSymbol' = 140, 'demangle' = 141, 'INTROSPECTION' = 142, 'FILE' = 143, 'URL' = 144, 'REMOTE' = 145, 'MONGO' = 146, 'MEILISEARCH' = 147, 'MYSQL' = 148, 'POSTGRES' = 149, 'SQLITE' = 150, 'ODBC' = 151, 'JDBC' = 152, 'HDFS' = 153, 'S3' = 154, 'HIVE' = 155, 'SOURCES' = 156, 'CLUSTER' = 157, 'ALL' = 158, 'NONE' = 159), `aliases` Array(String), `level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5, 'NAMED_COLLECTION' = 6)), - `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'SHOW NAMED COLLECTIONS SECRETS' = 92, 'ACCESS MANAGEMENT' = 93, 'SYSTEM SHUTDOWN' = 94, 'SYSTEM DROP DNS CACHE' = 95, 'SYSTEM DROP MARK CACHE' = 96, 'SYSTEM DROP UNCOMPRESSED CACHE' = 97, 'SYSTEM DROP MMAP CACHE' = 98, 'SYSTEM DROP QUERY CACHE' = 99, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 100, 'SYSTEM DROP FILESYSTEM CACHE' = 101, 'SYSTEM DROP SCHEMA CACHE' = 102, 'SYSTEM DROP S3 CLIENT CACHE' = 103, 'SYSTEM DROP CACHE' = 104, 'SYSTEM RELOAD CONFIG' = 105, 'SYSTEM RELOAD USERS' = 106, 'SYSTEM RELOAD SYMBOLS' = 107, 'SYSTEM RELOAD DICTIONARY' = 108, 'SYSTEM RELOAD MODEL' = 109, 'SYSTEM RELOAD FUNCTION' = 110, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 111, 'SYSTEM RELOAD' = 112, 'SYSTEM RESTART DISK' = 113, 'SYSTEM MERGES' = 114, 'SYSTEM TTL MERGES' = 115, 'SYSTEM FETCHES' = 116, 'SYSTEM MOVES' = 117, 'SYSTEM DISTRIBUTED SENDS' = 118, 'SYSTEM REPLICATED SENDS' = 119, 'SYSTEM SENDS' = 120, 'SYSTEM REPLICATION QUEUES' = 121, 'SYSTEM DROP REPLICA' = 122, 'SYSTEM SYNC REPLICA' = 123, 'SYSTEM RESTART REPLICA' = 124, 'SYSTEM RESTORE REPLICA' = 125, 'SYSTEM WAIT LOADING PARTS' = 126, 'SYSTEM SYNC DATABASE REPLICA' = 127, 'SYSTEM SYNC TRANSACTION LOG' = 128, 'SYSTEM SYNC FILE CACHE' = 129, 'SYSTEM FLUSH DISTRIBUTED' = 130, 'SYSTEM FLUSH LOGS' = 131, 'SYSTEM FLUSH' = 132, 'SYSTEM THREAD FUZZER' = 133, 'SYSTEM UNFREEZE' = 134, 'SYSTEM' = 135, 'dictGet' = 136, 'addressToLine' = 137, 'addressToLineWithInlines' = 138, 'addressToSymbol' = 139, 'demangle' = 140, 'INTROSPECTION' = 141, 'FILE' = 142, 'URL' = 143, 'REMOTE' = 144, 'MONGO' = 145, 'MEILISEARCH' = 146, 'MYSQL' = 147, 'POSTGRES' = 148, 'SQLITE' = 149, 'ODBC' = 150, 'JDBC' = 151, 'HDFS' = 152, 'S3' = 153, 'HIVE' = 154, 'SOURCES' = 155, 'CLUSTER' = 156, 'ALL' = 157, 'NONE' = 158)) + `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'ACCESS MANAGEMENT' = 91, 'SHOW NAMED COLLECTIONS' = 92, 'SHOW NAMED COLLECTIONS SECRETS' = 93, 'NAMED COLLECTION CONTROL' = 94, 'SYSTEM SHUTDOWN' = 95, 'SYSTEM DROP DNS CACHE' = 96, 'SYSTEM DROP MARK CACHE' = 97, 'SYSTEM DROP UNCOMPRESSED CACHE' = 98, 'SYSTEM DROP MMAP CACHE' = 99, 'SYSTEM DROP QUERY CACHE' = 100, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 101, 'SYSTEM DROP FILESYSTEM CACHE' = 102, 'SYSTEM DROP SCHEMA CACHE' = 103, 'SYSTEM DROP S3 CLIENT CACHE' = 104, 'SYSTEM DROP CACHE' = 105, 'SYSTEM RELOAD CONFIG' = 106, 'SYSTEM RELOAD USERS' = 107, 'SYSTEM RELOAD SYMBOLS' = 108, 'SYSTEM RELOAD DICTIONARY' = 109, 'SYSTEM RELOAD MODEL' = 110, 'SYSTEM RELOAD FUNCTION' = 111, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 112, 'SYSTEM RELOAD' = 113, 'SYSTEM RESTART DISK' = 114, 'SYSTEM MERGES' = 115, 'SYSTEM TTL MERGES' = 116, 'SYSTEM FETCHES' = 117, 'SYSTEM MOVES' = 118, 'SYSTEM DISTRIBUTED SENDS' = 119, 'SYSTEM REPLICATED SENDS' = 120, 'SYSTEM SENDS' = 121, 'SYSTEM REPLICATION QUEUES' = 122, 'SYSTEM DROP REPLICA' = 123, 'SYSTEM SYNC REPLICA' = 124, 'SYSTEM RESTART REPLICA' = 125, 'SYSTEM RESTORE REPLICA' = 126, 'SYSTEM WAIT LOADING PARTS' = 127, 'SYSTEM SYNC DATABASE REPLICA' = 128, 'SYSTEM SYNC TRANSACTION LOG' = 129, 'SYSTEM SYNC FILE CACHE' = 130, 'SYSTEM FLUSH DISTRIBUTED' = 131, 'SYSTEM FLUSH LOGS' = 132, 'SYSTEM FLUSH' = 133, 'SYSTEM THREAD FUZZER' = 134, 'SYSTEM UNFREEZE' = 135, 'SYSTEM' = 136, 'dictGet' = 137, 'addressToLine' = 138, 'addressToLineWithInlines' = 139, 'addressToSymbol' = 140, 'demangle' = 141, 'INTROSPECTION' = 142, 'FILE' = 143, 'URL' = 144, 'REMOTE' = 145, 'MONGO' = 146, 'MEILISEARCH' = 147, 'MYSQL' = 148, 'POSTGRES' = 149, 'SQLITE' = 150, 'ODBC' = 151, 'JDBC' = 152, 'HDFS' = 153, 'S3' = 154, 'HIVE' = 155, 'SOURCES' = 156, 'CLUSTER' = 157, 'ALL' = 158, 'NONE' = 159)) ) ENGINE = SystemPrivileges COMMENT 'SYSTEM TABLE is built on the fly.' From 536446d526852fcf5dd5fdc8a9abcc4c95246078 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Feb 2023 18:09:00 +0100 Subject: [PATCH 065/559] Remove debug logs --- src/Access/Common/AccessRightsElement.cpp | 1 - src/Parsers/Access/ASTGrantQuery.cpp | 5 ----- 2 files changed, 6 deletions(-) diff --git a/src/Access/Common/AccessRightsElement.cpp b/src/Access/Common/AccessRightsElement.cpp index 8d849297246..81cebd68b4c 100644 --- a/src/Access/Common/AccessRightsElement.cpp +++ b/src/Access/Common/AccessRightsElement.cpp @@ -1,7 +1,6 @@ #include #include #include -#include namespace DB diff --git a/src/Parsers/Access/ASTGrantQuery.cpp b/src/Parsers/Access/ASTGrantQuery.cpp index 71eff476965..44d727c7177 100644 --- a/src/Parsers/Access/ASTGrantQuery.cpp +++ b/src/Parsers/Access/ASTGrantQuery.cpp @@ -2,7 +2,6 @@ #include #include #include -#include namespace DB @@ -57,8 +56,6 @@ namespace void formatElementsWithoutOptions(const AccessRightsElements & elements, const IAST::FormatSettings & settings) { bool no_output = true; - auto * log = &Poco::Logger::get("kssenii"); - LOG_TEST(log, "kssenii 0 - {}", elements.size()); for (size_t i = 0; i != elements.size(); ++i) { const auto & element = elements[i]; @@ -82,14 +79,12 @@ namespace const auto & next_element = elements[i + 1]; if (element.sameDatabaseAndTableAndParameter(next_element)) { - LOG_TEST(log, "kssenii 1"); next_element_on_same_db_and_table = true; } } if (!next_element_on_same_db_and_table) { - LOG_TEST(log, "kssenii 2"); settings.ostr << " "; formatONClause(element, settings); } From 030a0ba7fb7feedefe1efa575cb4823c58fde472 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 24 Feb 2023 18:39:27 +0000 Subject: [PATCH 066/559] Refactor a bit more. --- .../Optimizations/optimizeUseProjections.cpp | 525 ++++++++++-------- 1 file changed, 297 insertions(+), 228 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index ce7818a3298..18b6140686d 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -22,6 +22,131 @@ namespace DB::QueryPlanOptimizations { +static QueryPlan::Node * findReadingStep(QueryPlan::Node & node) +{ + IQueryPlanStep * step = node.step.get(); + if (auto * reading = typeid_cast(step)) + return &node; + + if (node.children.size() != 1) + return nullptr; + + if (typeid_cast(step) || typeid_cast(step)) + return findReadingStep(*node.children.front()); + + return nullptr; +} + +/// This is a common DAG which is a merge of DAGs from Filter and Expression steps chain. +/// Additionally, for all the Filter steps, we collect filter conditions into filter_nodes. +/// Flag remove_last_filter_node is set in case if the last step is a Filter step and it should remove filter column. +struct QueryDAG +{ + ActionsDAGPtr dag; + ActionsDAG::NodeRawConstPtrs filter_nodes; + bool remove_last_filter_node = false; + + bool build(QueryPlan::Node & node); + +private: + void appendExpression(const ActionsDAGPtr & expression) + { + if (dag) + dag->mergeInplace(std::move(*expression->clone())); + else + dag = expression->clone(); + } +}; + +bool QueryDAG::build(QueryPlan::Node & node) +{ + IQueryPlanStep * step = node.step.get(); + if (auto * reading = typeid_cast(step)) + { + if (const auto * prewhere_info = reading->getPrewhereInfo()) + { + if (prewhere_info->row_level_filter) + { + remove_last_filter_node = false; + appendExpression(prewhere_info->row_level_filter); + if (const auto * filter_node = dag->tryFindInOutputs(prewhere_info->row_level_column_name)) + filter_nodes.push_back(filter_node); + else + return false; + } + + if (prewhere_info->prewhere_actions) + { + remove_last_filter_node = prewhere_info->remove_prewhere_column; + appendExpression(prewhere_info->prewhere_actions); + if (const auto * filter_node = dag->tryFindInOutputs(prewhere_info->prewhere_column_name)) + filter_nodes.push_back(filter_node); + else + return false; + } + } + return true; + } + + if (node.children.size() != 1) + return false; + + if (!build(*node.children.front())) + return false; + + if (auto * expression = typeid_cast(step)) + { + const auto & actions = expression->getExpression(); + if (actions->hasArrayJoin()) + return false; + + appendExpression(actions); + remove_last_filter_node = false; + return true; + } + + if (auto * filter = typeid_cast(step)) + { + const auto & actions = filter->getExpression(); + if (actions->hasArrayJoin()) + return false; + + appendExpression(actions); + remove_last_filter_node = filter->removesFilterColumn(); + const auto * filter_expression = dag->tryFindInOutputs(filter->getFilterColumnName()); + if (!filter_expression) + return false; + + filter_nodes.push_back(filter_expression); + return true; + } + + return false; +} + +bool canUseProjectionForReadingStep(ReadFromMergeTree * reading) +{ + /// Probably some projection already was applied. + if (reading->hasAnalyzedResult()) + return false; + + if (reading->isQueryWithFinal()) + return false; + + if (reading->isQueryWithSampling()) + return false; + + if (reading->isParallelReadingEnabled()) + return false; + + // Currently projection don't support deduplication when moving parts between shards. + if (reading->getContext()->getSettingsRef().allow_experimental_query_deduplication) + return false; + + return true; +} + + /// Required analysis info from aggregate projection. struct AggregateProjectionInfo { @@ -255,17 +380,120 @@ bool areAggregatesMatch( return true; } +struct AggregateQueryDAG +{ + ActionsDAGPtr dag; + const ActionsDAG::Node * filter_node = nullptr; + + bool build(QueryPlan::Node & node) + { + QueryDAG query; + if (!query.build(node)) + return false; + + dag = std::move(query.dag); + auto filter_nodes = std::move(query.filter_nodes); + + if (!filter_nodes.empty()) + { + filter_node = filter_nodes.front(); + if (filter_nodes.size() > 1) + { + FunctionOverloadResolverPtr func_builder_and = + std::make_unique( + std::make_shared()); + + filter_node = &dag->addFunction(func_builder_and, std::move(filter_nodes), {}); + } + + dag->getOutputs().push_back(filter_node); + } + + return true; + } +}; + +struct NormalQueryDAG +{ + ActionsDAGPtr dag; + bool need_remove_column = false; + const ActionsDAG::Node * filter_node = nullptr; + + bool build(QueryPlan::Node & node) + { + QueryDAG query; + if (!query.build(node)) + return false; + + dag = std::move(query.dag); + auto filter_nodes = std::move(query.filter_nodes); + need_remove_column = query.remove_last_filter_node; + + if (!filter_nodes.empty()) + { + auto & outputs = dag->getOutputs(); + filter_node = filter_nodes.back(); + + if (filter_nodes.size() > 1) + { + /// Add a conjunction of all the filters. + if (need_remove_column) + { + /// Last filter column is not needed; remove it right here + size_t pos = 0; + while (pos < outputs.size() && outputs[pos] != filter_node) + ++pos; + + if (pos < outputs.size()) + outputs.erase(outputs.begin() + pos); + } + else + { + /// Last filter is needed; we must replace it to constant 1, + /// As well as FilterStep does to make a compatible header. + for (auto & output : outputs) + { + if (output == filter_node) + { + ColumnWithTypeAndName col; + col.name = filter_node->result_name; + col.type = filter_node->result_type; + col.column = col.type->createColumnConst(1, 1); + output = &dag->addColumn(std::move(col)); + } + } + } + + FunctionOverloadResolverPtr func_builder_and = + std::make_unique( + std::make_shared()); + + filter_node = &dag->addFunction(func_builder_and, std::move(filter_nodes), {}); + outputs.insert(outputs.begin(), filter_node); + need_remove_column = true; + } + } + + if (dag) + { + dag->removeUnusedActions(); + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Header {}, Query DAG: {}", header.dumpStructure(), dag->dumpDAG()); + } + + return true; + } +}; + ActionsDAGPtr analyzeAggregateProjection( const AggregateProjectionInfo & info, - const ActionsDAG & query_dag, - const ActionsDAG::Node * filter_node, + const AggregateQueryDAG & query, const Names & keys, const AggregateDescriptions & aggregates) { - auto query_index = buildDAGIndex(query_dag); + auto query_index = buildDAGIndex(*query.dag); auto proj_index = buildDAGIndex(*info.before_aggregation); - MatchedTrees::Matches matches = matchTrees(*info.before_aggregation, query_dag); + MatchedTrees::Matches matches = matchTrees(*info.before_aggregation, *query.dag); // for (const auto & [node, match] : matches) // { @@ -298,8 +526,8 @@ ActionsDAGPtr analyzeAggregateProjection( /// We need to add filter column to keys set. /// It should be computable from projection keys. /// It will be removed in FilterStep. - if (filter_node) - query_key_nodes.push_back(filter_node); + if (query.filter_node) + query_key_nodes.push_back(query.filter_node); for (const auto & key : keys) { @@ -362,7 +590,7 @@ ActionsDAGPtr analyzeAggregateProjection( /// Not a match and there is no matched child. if (frame.node->type == ActionsDAG::ActionType::INPUT) { - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot find match for {}", frame.node->result_name); + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot find match for {}", frame.node->result_name); return {}; } @@ -374,7 +602,7 @@ ActionsDAGPtr analyzeAggregateProjection( // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Folding actions by projection"); - auto proj_dag = query_dag.foldActionsByProjection(new_inputs, query_key_nodes); + auto proj_dag = query.dag->foldActionsByProjection(new_inputs, query_key_nodes); /// Just add all the aggregates to dag inputs. auto & proj_dag_outputs = proj_dag->getOutputs(); @@ -384,125 +612,6 @@ ActionsDAGPtr analyzeAggregateProjection( return proj_dag; } -static QueryPlan::Node * findReadingStep(QueryPlan::Node & node) -{ - IQueryPlanStep * step = node.step.get(); - if (auto * reading = typeid_cast(step)) - return &node; - - if (node.children.size() != 1) - return nullptr; - - if (typeid_cast(step) || typeid_cast(step)) - return findReadingStep(*node.children.front()); - - return nullptr; -} - -static void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) -{ - if (dag) - dag->mergeInplace(std::move(*expression->clone())); - else - dag = expression->clone(); -} - - -/// This function builds a common DAG which is a merge of DAGs from Filter and Expression steps chain. -/// Additionally, for all the Filter steps, we collect filter conditions into filter_nodes. -/// Flag need_remove_column is set in case if the last step is a Filter step and it should remove filter column. -static bool buildQueryDAG( - QueryPlan::Node & node, - ActionsDAGPtr & dag, - ActionsDAG::NodeRawConstPtrs & filter_nodes, - bool & need_remove_column) -{ - IQueryPlanStep * step = node.step.get(); - if (auto * reading = typeid_cast(step)) - { - if (const auto * prewhere_info = reading->getPrewhereInfo()) - { - if (prewhere_info->row_level_filter) - { - need_remove_column = false; - appendExpression(dag, prewhere_info->row_level_filter); - if (const auto * filter_node = dag->tryFindInOutputs(prewhere_info->row_level_column_name)) - filter_nodes.push_back(filter_node); - else - return false; - } - - if (prewhere_info->prewhere_actions) - { - need_remove_column = prewhere_info->remove_prewhere_column; - appendExpression(dag, prewhere_info->prewhere_actions); - if (const auto * filter_node = dag->tryFindInOutputs(prewhere_info->prewhere_column_name)) - filter_nodes.push_back(filter_node); - else - return false; - } - } - return true; - } - - if (node.children.size() != 1) - return false; - - if (!buildQueryDAG(*node.children.front(), dag, filter_nodes, need_remove_column)) - return false; - - if (auto * expression = typeid_cast(step)) - { - const auto & actions = expression->getExpression(); - if (actions->hasArrayJoin()) - return false; - - appendExpression(dag, actions); - need_remove_column = false; - return true; - } - - if (auto * filter = typeid_cast(step)) - { - const auto & actions = filter->getExpression(); - if (actions->hasArrayJoin()) - return false; - - appendExpression(dag, actions); - need_remove_column = filter->removesFilterColumn(); - const auto * filter_expression = dag->tryFindInOutputs(filter->getFilterColumnName()); - if (!filter_expression) - return false; - - filter_nodes.push_back(filter_expression); - return true; - } - - return false; -} - -bool canUseProjectionForReadingStep(ReadFromMergeTree * reading) -{ - /// Probably some projection already was applied. - if (reading->hasAnalyzedResult()) - return false; - - if (reading->isQueryWithFinal()) - return false; - - if (reading->isQueryWithSampling()) - return false; - - if (reading->isParallelReadingEnabled()) - return false; - - // Currently projection don't support deduplication when moving parts between shards. - if (reading->getContext()->getSettingsRef().allow_experimental_query_deduplication) - return false; - - return true; -} - struct MinMaxProjectionCandidate { AggregateProjectionCandidate candidate; @@ -514,6 +623,8 @@ struct AggregateProjectionCandidates { std::vector real; std::optional minmax_projection; + + /// This flag means that DAG for projection candidate should be used in FilterStep. bool has_filter = false; }; @@ -548,30 +659,13 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Has agg projection"); - ActionsDAGPtr dag; - bool need_remove_column = false; // not used here - ActionsDAG::NodeRawConstPtrs filter_nodes; - if (!buildQueryDAG(*node.children.front(), dag, filter_nodes, need_remove_column)) + AggregateQueryDAG dag; + if (!dag.build(*node.children.front())) return candidates; - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", dag->dumpDAG()); + LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", dag.dag->dumpDAG()); - const ActionsDAG::Node * filter_node = nullptr; - if (!filter_nodes.empty()) - { - filter_node = filter_nodes.front(); - if (filter_nodes.size() > 1) - { - FunctionOverloadResolverPtr func_builder_and = - std::make_unique( - std::make_shared()); - - filter_node = &dag->addFunction(func_builder_and, std::move(filter_nodes), {}); - } - - dag->getOutputs().push_back(filter_node); - } - candidates.has_filter = filter_node; + candidates.has_filter = dag.filter_node; if (can_use_minmax_projection) { @@ -579,7 +673,7 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try projection {}", projection->name); auto info = getAggregatingProjectionInfo(*projection, context, metadata, key_virtual_columns); // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection DAG {}", info.before_aggregation->dumpDAG()); - if (auto proj_dag = analyzeAggregateProjection(info, *dag, filter_node, keys, aggregates)) + if (auto proj_dag = analyzeAggregateProjection(info, dag, keys, aggregates)) { // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(proj_dag)}; @@ -588,7 +682,7 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( auto block = reading.getMergeTreeData().getMinMaxCountProjectionBlock( metadata, candidate.dag->getRequiredColumnsNames(), - filter_node != nullptr, + dag.filter_node != nullptr, query_info, parts, minmax_projection_normal_parts, @@ -615,7 +709,7 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try projection {}", projection->name); auto info = getAggregatingProjectionInfo(*projection, context, metadata, key_virtual_columns); // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection DAG {}", info.before_aggregation->dumpDAG()); - if (auto proj_dag = analyzeAggregateProjection(info, *dag, filter_node, keys, aggregates)) + if (auto proj_dag = analyzeAggregateProjection(info, dag, keys, aggregates)) { // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(proj_dag)}; @@ -823,6 +917,40 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) return true; } +ActionsDAGPtr 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. + + 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; + + std::vector const_positions; + for (size_t i = 0; i < num_columns; ++i) + { + auto col_proj = proj_header.getByPosition(i).column; + auto col_main = main_header.getByPosition(i).column; + bool is_proj_const = col_proj && isColumnConst(*col_proj); + bool is_main_proj = col_main && isColumnConst(*col_main); + if (is_proj_const && !is_main_proj) + const_positions.push_back(i); + } + + if (const_positions.empty()) + return nullptr; + + ActionsDAGPtr dag = std::make_unique(proj_header.getColumnsWithTypeAndName()); + for (auto pos : const_positions) + { + auto & output = dag->getOutputs()[pos]; + output = &dag->materializeNode(*output); + } + + return dag; +} + bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) { @@ -863,66 +991,15 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) if (normal_projections.empty()) return false; - ActionsDAGPtr dag; - ActionsDAG::NodeRawConstPtrs filter_nodes; - bool need_remove_column = false; - if (!buildQueryDAG(*iter->node->children.front(), dag, filter_nodes, need_remove_column)) - return false; - - const ActionsDAG::Node * filter_node = nullptr; - if (!filter_nodes.empty()) + NormalQueryDAG query; { - auto & outputs = dag->getOutputs(); - filter_node = filter_nodes.back(); - - if (filter_nodes.size() > 1) - { - if (need_remove_column) - { - size_t pos = 0; - while (pos < outputs.size() && outputs[pos] != filter_node) - ++pos; - - if (pos < outputs.size()) - outputs.erase(outputs.begin() + pos); - } - else - { - for (auto & output : outputs) - { - if (output == filter_node) - { - ColumnWithTypeAndName col; - col.name = filter_node->result_name; - col.type = filter_node->result_type; - col.column = col.type->createColumnConst(1, 1); - output = &dag->addColumn(std::move(col)); - } - } - } - - FunctionOverloadResolverPtr func_builder_and = - std::make_unique( - std::make_shared()); - - filter_node = &dag->addFunction(func_builder_and, std::move(filter_nodes), {}); - outputs.insert(outputs.begin(), filter_node); - need_remove_column = true; - } - // else if (!need_remove_column) - // outputs.insert(outputs.begin(), filter_node); - } - - if (dag) - { - dag->removeUnusedActions(); - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", dag->dumpDAG()); + if (!query.build(*iter->node->children.front())) + return false; } std::list candidates; NormalProjectionCandidate * best_candidate = nullptr; - //const Block & header = frame.node->step->getOutputStream().header; const Names & required_columns = reading->getRealColumnNames(); const auto & parts = reading->getParts(); const auto & query_info = reading->getQueryInfo(); @@ -972,8 +1049,8 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) continue; ActionDAGNodes added_filter_nodes; - if (filter_node) - added_filter_nodes.nodes.push_back(filter_node); + if (query.filter_node) + added_filter_nodes.nodes.push_back(query.filter_node); auto projection_result_ptr = reader.estimateNumMarksToRead( std::move(projection_parts), @@ -1059,25 +1136,22 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) auto & projection_reading_node = nodes.emplace_back(QueryPlan::Node{.step = std::move(projection_reading)}); auto * next_node = &projection_reading_node; - if (dag) + if (query.dag) { auto & expr_or_filter_node = nodes.emplace_back(); - if (filter_node) + if (query.filter_node) { - //std::cerr << "======== " << projection_reading_node.step->getOutputStream().header.dumpStructure(); expr_or_filter_node.step = std::make_unique( projection_reading_node.step->getOutputStream(), - dag, - filter_node->result_name, - need_remove_column); - - //std::cerr << "======2= " << expr_or_filter_node.step->getOutputStream().header.dumpStructure(); + query.dag, + query.filter_node->result_name, + query.need_remove_column); } else expr_or_filter_node.step = std::make_unique( projection_reading_node.step->getOutputStream(), - dag); + query.dag); expr_or_filter_node.children.push_back(&projection_reading_node); next_node = &expr_or_filter_node; @@ -1087,31 +1161,26 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) { /// All parts are taken from projection iter->node->children.front() = next_node; - - //optimizeAggregationInOrder(node, nodes); } else { const auto & main_stream = iter->node->children.front()->step->getOutputStream(); const auto * proj_stream = &next_node->step->getOutputStream(); - if (!blocksHaveEqualStructure(proj_stream->header, main_stream.header)) + if (auto materializing = makeMaterializingDAG(proj_stream->header, main_stream.header)) { + // auto convert_actions_dag = ActionsDAG::makeConvertingActions( + // proj_stream->header.getColumnsWithTypeAndName(), + // main_stream.header.getColumnsWithTypeAndName(), + // ActionsDAG::MatchColumnsMode::Name, + // true); - //std::cerr << "======3= " << next_node->step->getOutputStream().header.dumpStructure(); - auto convert_actions_dag = ActionsDAG::makeConvertingActions( - proj_stream->header.getColumnsWithTypeAndName(), - main_stream.header.getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Name, - true); - - auto converting = std::make_unique(*proj_stream, convert_actions_dag); + auto converting = std::make_unique(*proj_stream, materializing); proj_stream = &converting->getOutputStream(); auto & expr_node = nodes.emplace_back(); expr_node.step = std::move(converting); expr_node.children.push_back(next_node); next_node = &expr_node; - //std::cerr << "======4= " << next_node->step->getOutputStream().header.dumpStructure(); } auto & union_node = nodes.emplace_back(); From 96b57c8a6594026a32d5e94fea3a49e8b19a2dc0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Feb 2023 20:38:40 +0100 Subject: [PATCH 067/559] Better support for equal keys --- src/Storages/NamedCollectionsHelpers.h | 38 +++++++++++++++++++--- src/Storages/StorageMongoDB.cpp | 16 +++------ src/Storages/StorageMySQL.cpp | 12 +++---- src/Storages/StoragePostgreSQL.cpp | 14 ++++---- src/TableFunctions/TableFunctionRemote.cpp | 10 +++--- 5 files changed, 57 insertions(+), 33 deletions(-) diff --git a/src/Storages/NamedCollectionsHelpers.h b/src/Storages/NamedCollectionsHelpers.h index 6e80b360411..085e21937ee 100644 --- a/src/Storages/NamedCollectionsHelpers.h +++ b/src/Storages/NamedCollectionsHelpers.h @@ -25,12 +25,42 @@ MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(const Poco::Util::A HTTPHeaderEntries getHeadersFromNamedCollection(const NamedCollection & collection); -template , - typename OptionalKeys = std::unordered_set> +struct ExternalDatabaseEqualKeysSet +{ + static constexpr std::array, 3> equal_keys{ + std::pair{"username", "user"}, std::pair{"database", "db"}, std::pair{"hostname", "host"}}; +}; +struct MongoDBEqualKeysSet +{ + static constexpr std::array, 4> equal_keys{ + std::pair{"username", "user"}, std::pair{"database", "db"}, std::pair{"hostname", "host"}, std::pair{"table", "collection"}}; +}; + +template struct ValidateKeysCmp +{ + constexpr bool operator()(const auto & lhs, const auto & rhs) const + { + if (lhs == rhs) + return true; + + for (const auto & equal : EqualKeys::equal_keys) + { + if (((equal.first == lhs) && (equal.second == rhs)) || ((equal.first == rhs) && (equal.second == lhs))) + return true; + } + return false; + } +}; + +template using ValidateKeysMultiset = std::unordered_multiset, ValidateKeysCmp>; +using ValidateKeysSet = std::unordered_multiset>; + + +template void validateNamedCollection( const NamedCollection & collection, - const RequiredKeys & required_keys, - const OptionalKeys & optional_keys, + const Keys & required_keys, + const Keys & optional_keys, const std::vector & optional_regex_keys = {}) { NamedCollection::Keys keys = collection.getKeys(); diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 2cb85878000..59ecab03bd8 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -18,6 +18,7 @@ #include #include #include +#include namespace DB { @@ -171,13 +172,6 @@ SinkToStoragePtr StorageMongoDB::write(const ASTPtr & /* query */, const Storage return std::make_shared(collection_name, database_name, metadata_snapshot, connection); } -struct KeysCmp -{ - constexpr bool operator()(const auto & lhs, const auto & rhs) const - { - return lhs == rhs || ((lhs == "table") && (rhs == "collection")) || ((rhs == "table") && (lhs == "collection")); - } -}; StorageMongoDB::Configuration StorageMongoDB::getConfiguration(ASTs engine_args, ContextPtr context) { Configuration configuration; @@ -186,14 +180,14 @@ StorageMongoDB::Configuration StorageMongoDB::getConfiguration(ASTs engine_args, { validateNamedCollection( *named_collection, - std::unordered_multiset, KeysCmp>{"host", "port", "user", "password", "database", "collection", "table"}, + ValidateKeysMultiset{"host", "port", "user", "username", "password", "database", "db", "collection", "table"}, {"options"}); - configuration.host = named_collection->get("host"); + configuration.host = named_collection->getOrDefault("host", named_collection->getOrDefault("hostname", "")); configuration.port = static_cast(named_collection->get("port")); - configuration.username = named_collection->get("user"); + configuration.username = named_collection->getOrDefault("user", named_collection->getOrDefault("username", "")); configuration.password = named_collection->get("password"); - configuration.database = named_collection->get("database"); + configuration.database = named_collection->getOrDefault("database", named_collection->getOrDefault("db", "")); configuration.table = named_collection->getOrDefault("collection", named_collection->getOrDefault("table", "")); configuration.options = named_collection->getOrDefault("options", ""); } diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index bc39e76be29..6bc9232a29a 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -241,27 +241,27 @@ StorageMySQL::Configuration StorageMySQL::processNamedCollectionResult( { StorageMySQL::Configuration configuration; - std::unordered_set optional_arguments = {"replace_query", "on_duplicate_clause", "addresses_expr", "host", "port"}; + ValidateKeysMultiset optional_arguments = {"replace_query", "on_duplicate_clause", "addresses_expr", "host", "hostname", "port"}; auto mysql_settings = storage_settings.all(); for (const auto & setting : mysql_settings) optional_arguments.insert(setting.getName()); - std::unordered_set required_arguments = {"user", "password", "database", "table"}; + ValidateKeysMultiset required_arguments = {"user", "username", "password", "database", "db", "table"}; if (require_table) required_arguments.insert("table"); - validateNamedCollection(named_collection, required_arguments, optional_arguments); + validateNamedCollection>(named_collection, required_arguments, optional_arguments); configuration.addresses_expr = named_collection.getOrDefault("addresses_expr", ""); if (configuration.addresses_expr.empty()) { - configuration.host = named_collection.get("host"); + configuration.host = named_collection.getOrDefault("host", named_collection.getOrDefault("hostname", "")); configuration.port = static_cast(named_collection.get("port")); configuration.addresses = {std::make_pair(configuration.host, configuration.port)}; } - configuration.username = named_collection.get("user"); + configuration.username = named_collection.getOrDefault("username", named_collection.getOrDefault("user", "")); configuration.password = named_collection.get("password"); - configuration.database = named_collection.get("database"); + configuration.database = named_collection.getOrDefault("db", named_collection.getOrDefault("database", "")); if (require_table) configuration.table = named_collection.get("table"); configuration.replace_query = named_collection.getOrDefault("replace_query", false); diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 3fa0c137f7b..e736f9edc18 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -390,24 +390,24 @@ SinkToStoragePtr StoragePostgreSQL::write( StoragePostgreSQL::Configuration StoragePostgreSQL::processNamedCollectionResult(const NamedCollection & named_collection, bool require_table) { StoragePostgreSQL::Configuration configuration; - std::unordered_set required_arguments = {"user", "password", "database", "table"}; + ValidateKeysMultiset required_arguments = {"user", "username", "password", "database", "db", "table"}; if (require_table) required_arguments.insert("table"); - validateNamedCollection( - named_collection, required_arguments, - {"schema", "on_conflict", "addresses_expr", "host", "port"}); + + validateNamedCollection>( + named_collection, required_arguments, {"schema", "on_conflict", "addresses_expr", "host", "hostname", "port"}); configuration.addresses_expr = named_collection.getOrDefault("addresses_expr", ""); if (configuration.addresses_expr.empty()) { - configuration.host = named_collection.get("host"); + configuration.host = named_collection.getOrDefault("host", named_collection.getOrDefault("hostname", "")); configuration.port = static_cast(named_collection.get("port")); configuration.addresses = {std::make_pair(configuration.host, configuration.port)}; } - configuration.username = named_collection.get("user"); + configuration.username = named_collection.getOrDefault("username", named_collection.getOrDefault("user", "")); configuration.password = named_collection.get("password"); - configuration.database = named_collection.get("database"); + configuration.database = named_collection.getOrDefault("db", named_collection.getOrDefault("database", "")); if (require_table) configuration.table = named_collection.get("table"); configuration.schema = named_collection.getOrDefault("schema", ""); diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 66f04346f16..ff1c714bbc9 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -55,15 +55,15 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr if (is_cluster_function) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Named collection cannot be used for table function cluster"); - validateNamedCollection( + validateNamedCollection>( *named_collection, - {"addresses_expr", "database", "table"}, - {"username", "password", "sharding_key"}); + {"addresses_expr", "database", "db", "table"}, + {"username", "user", "password", "sharding_key"}); cluster_description = named_collection->getOrDefault("addresses_expr", ""); - database = named_collection->get("database"); + database = named_collection->getOrDefault("db", named_collection->getOrDefault("database", "")); table = named_collection->get("table"); - username = named_collection->getOrDefault("username", ""); + username = named_collection->getOrDefault("username", named_collection->getOrDefault("user", "")); password = named_collection->getOrDefault("password", ""); } else From c2bcc4119fa706926e54ed99ddb108a601cb9853 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Feb 2023 20:57:37 +0100 Subject: [PATCH 068/559] Better --- .../NamedCollections/NamedCollections.cpp | 34 +++++++++++++++++++ .../NamedCollections/NamedCollections.h | 4 +++ src/Storages/StorageMongoDB.cpp | 8 ++--- src/Storages/StorageMySQL.cpp | 4 +-- src/Storages/StoragePostgreSQL.cpp | 6 ++-- src/TableFunctions/TableFunctionRemote.cpp | 4 +-- 6 files changed, 49 insertions(+), 11 deletions(-) diff --git a/src/Common/NamedCollections/NamedCollections.cpp b/src/Common/NamedCollections/NamedCollections.cpp index 2f80392c9ab..50f88adab36 100644 --- a/src/Common/NamedCollections/NamedCollections.cpp +++ b/src/Common/NamedCollections/NamedCollections.cpp @@ -364,6 +364,28 @@ template T NamedCollection::getOrDefault(const Key & key, const T & return pimpl->getOrDefault(key, default_value); } +template T NamedCollection::getAny(const std::initializer_list & keys) const +{ + std::lock_guard lock(mutex); + for (const auto & key : keys) + { + if (pimpl->has(key)) + return pimpl->get(key); + } + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such keys: {}", fmt::join(keys, ", ")); +} + +template T NamedCollection::getAnyOrDefault(const std::initializer_list & keys, const T & default_value) const +{ + std::lock_guard lock(mutex); + for (const auto & key : keys) + { + if (pimpl->has(key)) + return pimpl->get(key); + } + return default_value; +} + template void NamedCollection::set(const Key & key, const T & value) { assertMutable(); @@ -455,6 +477,18 @@ template Int64 NamedCollection::getOrDefault(const NamedCollection::Key & template Float64 NamedCollection::getOrDefault(const NamedCollection::Key & key, const Float64 & default_value) const; template bool NamedCollection::getOrDefault(const NamedCollection::Key & key, const bool & default_value) const; +template String NamedCollection::getAny(const std::initializer_list & key) const; +template UInt64 NamedCollection::getAny(const std::initializer_list & key) const; +template Int64 NamedCollection::getAny(const std::initializer_list & key) const; +template Float64 NamedCollection::getAny(const std::initializer_list & key) const; +template bool NamedCollection::getAny(const std::initializer_list & key) const; + +template String NamedCollection::getAnyOrDefault(const std::initializer_list & key, const String & default_value) const; +template UInt64 NamedCollection::getAnyOrDefault(const std::initializer_list & key, const UInt64 & default_value) const; +template Int64 NamedCollection::getAnyOrDefault(const std::initializer_list & key, const Int64 & default_value) const; +template Float64 NamedCollection::getAnyOrDefault(const std::initializer_list & key, const Float64 & default_value) const; +template bool NamedCollection::getAnyOrDefault(const std::initializer_list & key, const bool & default_value) const; + template void NamedCollection::set(const NamedCollection::Key & key, const String & value); template void NamedCollection::set(const NamedCollection::Key & key, const String & value); template void NamedCollection::set(const NamedCollection::Key & key, const UInt64 & value); diff --git a/src/Common/NamedCollections/NamedCollections.h b/src/Common/NamedCollections/NamedCollections.h index a5b4349aaa3..b82d5eb3152 100644 --- a/src/Common/NamedCollections/NamedCollections.h +++ b/src/Common/NamedCollections/NamedCollections.h @@ -39,6 +39,10 @@ public: template T getOrDefault(const Key & key, const T & default_value) const; + template T getAny(const std::initializer_list & keys) const; + + template T getAnyOrDefault(const std::initializer_list & keys, const T & default_value) const; + std::unique_lock lock(); template void set(const Key & key, const T & value); diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 59ecab03bd8..57aa81efe0a 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -183,12 +183,12 @@ StorageMongoDB::Configuration StorageMongoDB::getConfiguration(ASTs engine_args, ValidateKeysMultiset{"host", "port", "user", "username", "password", "database", "db", "collection", "table"}, {"options"}); - configuration.host = named_collection->getOrDefault("host", named_collection->getOrDefault("hostname", "")); + configuration.host = named_collection->getAny({"host", "hostname"}); configuration.port = static_cast(named_collection->get("port")); - configuration.username = named_collection->getOrDefault("user", named_collection->getOrDefault("username", "")); + configuration.username = named_collection->getAny({"user", "username"}); configuration.password = named_collection->get("password"); - configuration.database = named_collection->getOrDefault("database", named_collection->getOrDefault("db", "")); - configuration.table = named_collection->getOrDefault("collection", named_collection->getOrDefault("table", "")); + configuration.database = named_collection->getAny({"database", "db"}); + configuration.table = named_collection->getAny({"collection", "table"}); configuration.options = named_collection->getOrDefault("options", ""); } else diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 6bc9232a29a..fdeea044dee 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -259,9 +259,9 @@ StorageMySQL::Configuration StorageMySQL::processNamedCollectionResult( configuration.addresses = {std::make_pair(configuration.host, configuration.port)}; } - configuration.username = named_collection.getOrDefault("username", named_collection.getOrDefault("user", "")); + configuration.username = named_collection.getAny({"username", "user"}); configuration.password = named_collection.get("password"); - configuration.database = named_collection.getOrDefault("db", named_collection.getOrDefault("database", "")); + configuration.database = named_collection.getAny({"db", "database"}); if (require_table) configuration.table = named_collection.get("table"); configuration.replace_query = named_collection.getOrDefault("replace_query", false); diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index e736f9edc18..1bfc056f316 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -400,14 +400,14 @@ StoragePostgreSQL::Configuration StoragePostgreSQL::processNamedCollectionResult configuration.addresses_expr = named_collection.getOrDefault("addresses_expr", ""); if (configuration.addresses_expr.empty()) { - configuration.host = named_collection.getOrDefault("host", named_collection.getOrDefault("hostname", "")); + configuration.host = named_collection.getAny({"host", "hostname"}); configuration.port = static_cast(named_collection.get("port")); configuration.addresses = {std::make_pair(configuration.host, configuration.port)}; } - configuration.username = named_collection.getOrDefault("username", named_collection.getOrDefault("user", "")); + configuration.username = named_collection.getAny({"username", "user"}); configuration.password = named_collection.get("password"); - configuration.database = named_collection.getOrDefault("db", named_collection.getOrDefault("database", "")); + configuration.database = named_collection.getAny({"db", "database"}); if (require_table) configuration.table = named_collection.get("table"); configuration.schema = named_collection.getOrDefault("schema", ""); diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index ff1c714bbc9..f6c773b0b97 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -61,9 +61,9 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr {"username", "user", "password", "sharding_key"}); cluster_description = named_collection->getOrDefault("addresses_expr", ""); - database = named_collection->getOrDefault("db", named_collection->getOrDefault("database", "")); + database = named_collection->getAnyOrDefault({"db", "database"}, "default"); table = named_collection->get("table"); - username = named_collection->getOrDefault("username", named_collection->getOrDefault("user", "")); + username = named_collection->getAnyOrDefault({"username", "user"}, "default"); password = named_collection->getOrDefault("password", ""); } else From 53b006dd5cef3501d01e781025e99f097dfa7c36 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 24 Feb 2023 19:59:13 +0000 Subject: [PATCH 069/559] Refactor a bit more. --- .../Optimizations/optimizeUseProjections.cpp | 122 ++++++++++-------- 1 file changed, 67 insertions(+), 55 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 18b6140686d..8d21539a252 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -124,7 +124,7 @@ bool QueryDAG::build(QueryPlan::Node & node) return false; } -bool canUseProjectionForReadingStep(ReadFromMergeTree * reading) +static bool canUseProjectionForReadingStep(ReadFromMergeTree * reading) { /// Probably some projection already was applied. if (reading->hasAnalyzedResult()) @@ -615,8 +615,8 @@ ActionsDAGPtr analyzeAggregateProjection( struct MinMaxProjectionCandidate { AggregateProjectionCandidate candidate; - Block minmax_count_projection_block; - MergeTreeData::DataPartsVector minmax_projection_normal_parts; + Block block; + MergeTreeData::DataPartsVector normal_parts; }; struct AggregateProjectionCandidates @@ -693,8 +693,8 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( { MinMaxProjectionCandidate minmax; minmax.candidate = std::move(candidate); - minmax.minmax_count_projection_block = std::move(block); - minmax.minmax_projection_normal_parts = std::move(minmax_projection_normal_parts); + minmax.block = std::move(block); + minmax.normal_parts = std::move(minmax_projection_normal_parts); minmax.candidate.projection = projection; candidates.minmax_projection.emplace(std::move(minmax)); } @@ -722,6 +722,19 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( return candidates; } +static std::shared_ptr getMaxAddedBlocks(ReadFromMergeTree * reading) +{ + ContextPtr context = reading->getContext(); + + if (context->getSettingsRef().select_sequential_consistency) + { + if (const auto * replicated = dynamic_cast(&reading->getMergeTreeData())) + return std::make_shared(replicated->getMaxAddedBlocks()); + } + + return {}; +} + bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { if (node.children.size() != 1) @@ -745,15 +758,7 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (!canUseProjectionForReadingStep(reading)) return false; - const auto metadata = reading->getStorageMetadata(); - ContextPtr context = reading->getContext(); - - std::shared_ptr max_added_blocks; - if (context->getSettingsRef().select_sequential_consistency) - { - if (const StorageReplicatedMergeTree * replicated = dynamic_cast(&reading->getMergeTreeData())) - max_added_blocks = std::make_shared(replicated->getMaxAddedBlocks()); - } + std::shared_ptr max_added_blocks = getMaxAddedBlocks(reading); auto candidates = getAggregateProjectionCandidates(node, *aggregating, *reading, max_added_blocks); @@ -763,10 +768,13 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) else if (candidates.real.empty()) return false; - MergeTreeDataSelectExecutor reader(reading->getMergeTreeData()); const auto & parts = reading->getParts(); const auto & query_info = reading->getQueryInfo(); + const auto metadata = reading->getStorageMetadata(); + ContextPtr context = reading->getContext(); + MergeTreeDataSelectExecutor reader(reading->getMergeTreeData()); + /// Selecting best candidate. for (auto & candidate : candidates.real) { MergeTreeData::DataPartsVector projection_parts; @@ -830,25 +838,28 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) QueryPlanStepPtr projection_reading; bool has_nornal_parts; + /// Add reading from projection step. if (candidates.minmax_projection) { - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Minmax proj block {}", candidates.minmax_projection->minmax_count_projection_block.dumpStructure()); + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Minmax proj block {}", + // candidates.minmax_projection->block.dumpStructure()); - Pipe pipe(std::make_shared(std::move(candidates.minmax_projection->minmax_count_projection_block))); + Pipe pipe(std::make_shared(std::move(candidates.minmax_projection->block))); projection_reading = std::make_unique(std::move(pipe)); - has_nornal_parts = !candidates.minmax_projection->minmax_projection_normal_parts.empty(); + has_nornal_parts = !candidates.minmax_projection->normal_parts.empty(); if (has_nornal_parts) - reading->resetParts(std::move(candidates.minmax_projection->minmax_projection_normal_parts)); + reading->resetParts(std::move(candidates.minmax_projection->normal_parts)); } else { auto storage_snapshot = reading->getStorageSnapshot(); auto proj_snapshot = std::make_shared( - storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); //, storage_snapshot->data); + storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); proj_snapshot->addProjection(best_candidate->projection); - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}", proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString()); + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}", + // proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString()); auto query_info_copy = query_info; query_info_copy.prewhere_info = nullptr; @@ -867,7 +878,8 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (!projection_reading) { - Pipe pipe(std::make_shared(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)); } @@ -876,7 +888,8 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) reading->setAnalyzedResult(std::move(best_candidate->merge_tree_normal_select_result_ptr)); } - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}", projection_reading->getOutputStream().header.dumpStructure()); + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}", + // projection_reading->getOutputStream().header.dumpStructure()); projection_reading->setStepDescription(best_candidate->projection->name); @@ -901,12 +914,8 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (!has_nornal_parts) { /// All parts are taken from projection - - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Expr stream {}", expr_or_filter_node.step->getOutputStream().header.dumpStructure()); aggregating->requestOnlyMergeForAggregateProjection(expr_or_filter_node.step->getOutputStream()); node.children.front() = &expr_or_filter_node; - - //optimizeAggregationInOrder(node, nodes); } else { @@ -917,7 +926,8 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) return true; } -ActionsDAGPtr makeMaterializingDAG(const Block & proj_header, const Block main_header) + +static ActionsDAGPtr 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. @@ -951,6 +961,16 @@ ActionsDAGPtr makeMaterializingDAG(const Block & proj_header, const Block main_h return dag; } +static bool hasAllRequiredColumns(const ProjectionDescription * projection, const Names & required_columns) +{ + for (const auto & col : required_columns) + { + if (!projection->sample_block.has(col)) + return false; + } + + return true; +} bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) { @@ -1009,28 +1029,14 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) auto ordinary_reading_select_result = reading->selectRangesToRead(parts); size_t ordinary_reading_marks = ordinary_reading_select_result->marks(); - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Marks for ordinary reading {}", ordinary_reading_marks); + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), + // "Marks for ordinary reading {}", ordinary_reading_marks); - std::shared_ptr max_added_blocks; - if (context->getSettingsRef().select_sequential_consistency) - { - if (const StorageReplicatedMergeTree * replicated = dynamic_cast(&reading->getMergeTreeData())) - max_added_blocks = std::make_shared(replicated->getMaxAddedBlocks()); - } + std::shared_ptr max_added_blocks = getMaxAddedBlocks(reading); for (const auto * projection : normal_projections) { - bool has_all_columns = true; - for (const auto & col : required_columns) - { - if (!projection->sample_block.has(col)) - { - has_all_columns = false; - break; - } - } - - if (!has_all_columns) + if (!hasAllRequiredColumns(projection, required_columns)) continue; MergeTreeData::DataPartsVector projection_parts; @@ -1086,7 +1092,15 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) } } - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Marks for projection {} {}", projection->name ,candidate.sum_marks); + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), + // "Marks for projection {} {}", projection->name ,candidate.sum_marks); + + // if (candidate.sum_marks > ordinary_reading_marks) + // continue; + + // if (best_candidate == nullptr || candidate.sum_marks < best_candidate->sum_marks) + // best_candidate = &candidate; + if (candidate.sum_marks < ordinary_reading_marks && (best_candidate == nullptr || candidate.sum_marks < best_candidate->sum_marks)) best_candidate = &candidate; } @@ -1102,7 +1116,8 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); //, storage_snapshot->data); proj_snapshot->addProjection(best_candidate->projection); - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}", proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString()); + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}", + // proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString()); auto query_info_copy = query_info; query_info_copy.prewhere_info = nullptr; @@ -1129,7 +1144,8 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) if (has_nornal_parts) reading->setAnalyzedResult(std::move(best_candidate->merge_tree_normal_select_result_ptr)); - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}", projection_reading->getOutputStream().header.dumpStructure()); + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}", + // projection_reading->getOutputStream().header.dumpStructure()); projection_reading->setStepDescription(best_candidate->projection->name); @@ -1169,12 +1185,6 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) if (auto materializing = makeMaterializingDAG(proj_stream->header, main_stream.header)) { - // auto convert_actions_dag = ActionsDAG::makeConvertingActions( - // proj_stream->header.getColumnsWithTypeAndName(), - // main_stream.header.getColumnsWithTypeAndName(), - // ActionsDAG::MatchColumnsMode::Name, - // true); - auto converting = std::make_unique(*proj_stream, materializing); proj_stream = &converting->getOutputStream(); auto & expr_node = nodes.emplace_back(); @@ -1189,6 +1199,8 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) union_node.children = {iter->node->children.front(), next_node}; iter->node->children.front() = &union_node; + /// Here we remove last steps from stack to be able to optimize again. + /// In theory, read-in-order can be applied to projection. iter->next_child = 0; stack.resize(iter.base() - stack.begin() + 1); } From 7045ed539f548cedce43fe541afc9561bd6205a5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 24 Feb 2023 20:04:12 +0000 Subject: [PATCH 070/559] Refactor a bit more. --- .../QueryPlan/Optimizations/optimizeUseProjections.cpp | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index 8d21539a252..c400bba4339 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -1095,13 +1095,10 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), // "Marks for projection {} {}", projection->name ,candidate.sum_marks); - // if (candidate.sum_marks > ordinary_reading_marks) - // continue; + if (candidate.sum_marks >= ordinary_reading_marks) + continue; - // if (best_candidate == nullptr || candidate.sum_marks < best_candidate->sum_marks) - // best_candidate = &candidate; - - if (candidate.sum_marks < ordinary_reading_marks && (best_candidate == nullptr || candidate.sum_marks < best_candidate->sum_marks)) + if (best_candidate == nullptr || candidate.sum_marks < best_candidate->sum_marks) best_candidate = &candidate; } From f398c5d4717102fdd5d6338f994673f34c9a3c31 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 27 Feb 2023 12:22:27 +0100 Subject: [PATCH 071/559] Fix style --- src/Common/NamedCollections/NamedCollections.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/NamedCollections/NamedCollections.cpp b/src/Common/NamedCollections/NamedCollections.cpp index 50f88adab36..533481f792a 100644 --- a/src/Common/NamedCollections/NamedCollections.cpp +++ b/src/Common/NamedCollections/NamedCollections.cpp @@ -17,6 +17,7 @@ namespace ErrorCodes extern const int NAMED_COLLECTION_DOESNT_EXIST; extern const int NAMED_COLLECTION_ALREADY_EXISTS; extern const int NAMED_COLLECTION_IS_IMMUTABLE; + extern const int BAD_ARGUMENTS; } namespace Configuration = NamedCollectionConfiguration; From b19264cf9f059c4a6c1c0e40fd9f6a00cc9ba168 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 27 Feb 2023 12:32:13 +0100 Subject: [PATCH 072/559] Remove redundant --- src/Databases/PostgreSQL/DatabasePostgreSQL.h | 1 - src/Storages/ExternalDataSourceConfiguration.cpp | 10 ---------- 2 files changed, 11 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.h b/src/Databases/PostgreSQL/DatabasePostgreSQL.h index 074a8728d0a..08583f4b6d9 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.h @@ -8,7 +8,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index 28bd058b802..e503c5edaab 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -9,16 +9,6 @@ #include #include -#if USE_AMQPCPP -#include -#endif -#if USE_RDKAFKA -#include -#endif -#if USE_NATSIO -#include -#endif - #include namespace DB From a2f9ac88d94df5ecf21af09918d78fcd7db3c069 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 27 Feb 2023 13:00:54 +0100 Subject: [PATCH 073/559] Fix unit test --- src/Access/Common/AccessType.h | 2 +- src/Access/tests/gtest_access_rights_ops.cpp | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 84ec93d58f6..0a8ea908cff 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -69,7 +69,7 @@ enum class AccessType M(ALTER_FREEZE_PARTITION, "FREEZE PARTITION, UNFREEZE", TABLE, ALTER_TABLE) \ \ M(ALTER_DATABASE_SETTINGS, "ALTER DATABASE SETTING, ALTER MODIFY DATABASE SETTING, MODIFY DATABASE SETTING", DATABASE, ALTER_DATABASE) /* allows to execute ALTER MODIFY SETTING */\ - M(ALTER_NAMED_COLLECTION, "", NAMED_COLLECTION, ALTER) /* allows to execute ALTER NAMED COLLECTION */\ + M(ALTER_NAMED_COLLECTION, "", NAMED_COLLECTION, NAMED_COLLECTION_CONTROL) /* allows to execute ALTER NAMED COLLECTION */\ \ M(ALTER_TABLE, "", GROUP, ALTER) \ M(ALTER_DATABASE, "", GROUP, ALTER) \ diff --git a/src/Access/tests/gtest_access_rights_ops.cpp b/src/Access/tests/gtest_access_rights_ops.cpp index d6f827a02c5..025f70af587 100644 --- a/src/Access/tests/gtest_access_rights_ops.cpp +++ b/src/Access/tests/gtest_access_rights_ops.cpp @@ -48,12 +48,12 @@ TEST(AccessRights, Union) ASSERT_EQ(lhs.toString(), "GRANT INSERT ON *.*, " "GRANT SHOW, SELECT, ALTER, CREATE DATABASE, CREATE TABLE, CREATE VIEW, " - "CREATE DICTIONARY, CREATE NAMED COLLECTION, DROP DATABASE, DROP TABLE, DROP VIEW, DROP DICTIONARY, DROP NAMED COLLECTION, " + "CREATE DICTIONARY, DROP DATABASE, DROP TABLE, DROP VIEW, DROP DICTIONARY, " "TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, " - "SHOW ROW POLICIES, SHOW NAMED COLLECTIONS, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, " + "SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, " "SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, " "SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, " - "SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON db1.*"); + "SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON db1.*, GRANT NAMED COLLECTION CONTROL ON db1"); } From 03c9eeb1064078ed5d39776e4b27020373c1d52d Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 27 Feb 2023 14:24:22 +0100 Subject: [PATCH 074/559] Fix tests --- src/Access/UsersConfigAccessStorage.cpp | 6 ++-- .../configs/users.d/users.xml | 3 +- .../configs/users.xml | 3 +- .../configs/users.xml | 3 +- .../configs/users.d/users.xml | 3 +- .../configs/users.d/users.xml | 1 + ...> users_only_named_collection_control.xml} | 2 +- .../test_named_collections/test.py | 36 ++++++++++--------- .../configs/users.d/users.xml | 3 +- .../configs/users.d/users.xml | 3 +- .../01271_show_privileges.reference | 2 +- 11 files changed, 32 insertions(+), 33 deletions(-) rename tests/integration/test_named_collections/configs/users.d/{users_no_default_access_with_access_management.xml => users_only_named_collection_control.xml} (74%) diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index b893554cb8a..562df61e8aa 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -233,10 +233,10 @@ namespace user->access.revokeGrantOption(AccessType::ALL); } - bool show_named_collections = config.getBool(user_config + ".show_named_collections", false); - if (!show_named_collections) + bool named_collection_control = config.getBool(user_config + ".named_collection_control", false); + if (!named_collection_control) { - user->access.revoke(AccessType::SHOW_NAMED_COLLECTIONS); + user->access.revoke(AccessType::NAMED_COLLECTION_CONTROL); } bool show_named_collections_secrets = config.getBool(user_config + ".show_named_collections_secrets", false); diff --git a/tests/integration/test_access_control_on_cluster/configs/users.d/users.xml b/tests/integration/test_access_control_on_cluster/configs/users.d/users.xml index 8556e73c82f..775c63350b0 100644 --- a/tests/integration/test_access_control_on_cluster/configs/users.d/users.xml +++ b/tests/integration/test_access_control_on_cluster/configs/users.d/users.xml @@ -4,8 +4,7 @@ default default - 1 - 1 + 1 diff --git a/tests/integration/test_create_query_constraints/configs/users.xml b/tests/integration/test_create_query_constraints/configs/users.xml index 8556e73c82f..775c63350b0 100644 --- a/tests/integration/test_create_query_constraints/configs/users.xml +++ b/tests/integration/test_create_query_constraints/configs/users.xml @@ -4,8 +4,7 @@ default default - 1 - 1 + 1 diff --git a/tests/integration/test_global_overcommit_tracker/configs/users.xml b/tests/integration/test_global_overcommit_tracker/configs/users.xml index 8556e73c82f..775c63350b0 100644 --- a/tests/integration/test_global_overcommit_tracker/configs/users.xml +++ b/tests/integration/test_global_overcommit_tracker/configs/users.xml @@ -4,8 +4,7 @@ default default - 1 - 1 + 1 diff --git a/tests/integration/test_grant_and_revoke/configs/users.d/users.xml b/tests/integration/test_grant_and_revoke/configs/users.d/users.xml index 8556e73c82f..775c63350b0 100644 --- a/tests/integration/test_grant_and_revoke/configs/users.d/users.xml +++ b/tests/integration/test_grant_and_revoke/configs/users.d/users.xml @@ -4,8 +4,7 @@ default default - 1 - 1 + 1 diff --git a/tests/integration/test_named_collections/configs/users.d/users.xml b/tests/integration/test_named_collections/configs/users.d/users.xml index 8556e73c82f..15da914f666 100644 --- a/tests/integration/test_named_collections/configs/users.d/users.xml +++ b/tests/integration/test_named_collections/configs/users.d/users.xml @@ -4,6 +4,7 @@ default default + 1 1 1 diff --git a/tests/integration/test_named_collections/configs/users.d/users_no_default_access_with_access_management.xml b/tests/integration/test_named_collections/configs/users.d/users_only_named_collection_control.xml similarity index 74% rename from tests/integration/test_named_collections/configs/users.d/users_no_default_access_with_access_management.xml rename to tests/integration/test_named_collections/configs/users.d/users_only_named_collection_control.xml index 83dc04f03aa..775c63350b0 100644 --- a/tests/integration/test_named_collections/configs/users.d/users_no_default_access_with_access_management.xml +++ b/tests/integration/test_named_collections/configs/users.d/users_only_named_collection_control.xml @@ -4,7 +4,7 @@ default default - 1 + 1 diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index 6a53f7e0a58..1f27826d213 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -24,6 +24,16 @@ def cluster(): ], stay_alive=True, ) + cluster.add_instance( + "node_only_named_collection_control", + main_configs=[ + "configs/config.d/named_collections.xml", + ], + user_configs=[ + "configs/users.d/users_only_named_collection_control.xml", + ], + stay_alive=True, + ) cluster.add_instance( "node_no_default_access", main_configs=[ @@ -34,16 +44,6 @@ def cluster(): ], stay_alive=True, ) - cluster.add_instance( - "node_no_default_access_but_with_access_management", - main_configs=[ - "configs/config.d/named_collections.xml", - ], - user_configs=[ - "configs/users.d/users_no_default_access_with_access_management.xml", - ], - stay_alive=True, - ) logging.info("Starting cluster...") cluster.start() @@ -73,25 +73,29 @@ def replace_in_users_config(node, old, new): def test_default_access(cluster): node = cluster.instances["node_no_default_access"] assert 0 == int(node.query("select count() from system.named_collections")) - node = cluster.instances["node_no_default_access_but_with_access_management"] - assert 0 == int(node.query("select count() from system.named_collections")) + node = cluster.instances["node_only_named_collection_control"] + assert 1 == int(node.query("select count() from system.named_collections")) + assert ( + node.query("select collection['key1'] from system.named_collections").strip() + == "[HIDDEN]" + ) node = cluster.instances["node"] assert int(node.query("select count() from system.named_collections")) > 0 replace_in_users_config( - node, "show_named_collections>1", "show_named_collections>0" + node, "named_collection_control>1", "named_collection_control>0" ) - assert "show_named_collections>0" in node.exec_in_container( + assert "named_collection_control>0" in node.exec_in_container( ["bash", "-c", f"cat /etc/clickhouse-server/users.d/users.xml"] ) node.restart_clickhouse() assert 0 == int(node.query("select count() from system.named_collections")) replace_in_users_config( - node, "show_named_collections>0", "show_named_collections>1" + node, "named_collection_control>0", "named_collection_control>1" ) - assert "show_named_collections>1" in node.exec_in_container( + assert "named_collection_control>1" in node.exec_in_container( ["bash", "-c", f"cat /etc/clickhouse-server/users.d/users.xml"] ) node.restart_clickhouse() diff --git a/tests/integration/test_overcommit_tracker/configs/users.d/users.xml b/tests/integration/test_overcommit_tracker/configs/users.d/users.xml index 8556e73c82f..775c63350b0 100644 --- a/tests/integration/test_overcommit_tracker/configs/users.d/users.xml +++ b/tests/integration/test_overcommit_tracker/configs/users.d/users.xml @@ -4,8 +4,7 @@ default default - 1 - 1 + 1 diff --git a/tests/integration/test_settings_constraints_distributed/configs/users.d/users.xml b/tests/integration/test_settings_constraints_distributed/configs/users.d/users.xml index 8556e73c82f..775c63350b0 100644 --- a/tests/integration/test_settings_constraints_distributed/configs/users.d/users.xml +++ b/tests/integration/test_settings_constraints_distributed/configs/users.d/users.xml @@ -4,8 +4,7 @@ default default - 1 - 1 + 1 diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 03661d2469f..2d54531aff4 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -39,7 +39,7 @@ ALTER MOVE PARTITION ['ALTER MOVE PART','MOVE PARTITION','MOVE PART'] TABLE ALTE ALTER FETCH PARTITION ['ALTER FETCH PART','FETCH PARTITION'] TABLE ALTER TABLE ALTER FREEZE PARTITION ['FREEZE PARTITION','UNFREEZE'] TABLE ALTER TABLE ALTER DATABASE SETTINGS ['ALTER DATABASE SETTING','ALTER MODIFY DATABASE SETTING','MODIFY DATABASE SETTING'] DATABASE ALTER DATABASE -ALTER NAMED COLLECTION [] NAMED_COLLECTION ALTER +ALTER NAMED COLLECTION [] NAMED_COLLECTION NAMED COLLECTION CONTROL ALTER TABLE [] \N ALTER ALTER DATABASE [] \N ALTER ALTER VIEW REFRESH ['ALTER LIVE VIEW REFRESH','REFRESH VIEW'] VIEW ALTER VIEW From 05823ffb9a2e8fdfd9c884abc9e34ea7969c8f3f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 27 Feb 2023 21:41:38 +0800 Subject: [PATCH 075/559] finish dev joda parsing --- src/Functions/parseDateTime.cpp | 2646 +++++++++++++++++++------------ 1 file changed, 1637 insertions(+), 1009 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index a80f26ec1c9..9d331e4d11b 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -1,9 +1,6 @@ #include #include -#include -#include #include -#include #include #include @@ -17,1059 +14,1690 @@ namespace DB { - namespace { -using Pos = const char *; - -constexpr std::string_view weekdaysShort[] = {"sun", "mon", "tue", "wed", "thu", "fri", "sat"}; -constexpr std::string_view weekdaysFull[] = {"sunday", "monday", "tuesday", "wednesday", "thursday", "friday", "saturday"}; -constexpr std::string_view monthsShort[] = {"jan", "feb", "mar", "apr", "may", "jun", "jul", "aug", "sep", "oct", "nov", "dec"}; - -constexpr Int32 leapDays[] = {0, 31, 29, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}; -constexpr Int32 normalDays[] = {0, 31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}; - -constexpr Int32 cumulativeLeapDays[] = {0, 31, 60, 91, 121, 152, 182, 213, 244, 274, 305, 335, 366}; -constexpr Int32 cumulativeDays[] = {0, 31, 59, 90, 120, 151, 181, 212, 243, 273, 304, 334, 365}; -constexpr Int32 cumulativeYearDays[] - = {0, 365, 730, 1096, 1461, 1826, 2191, 2557, 2922, 3287, 3652, 4018, 4383, 4748, 5113, 5479, 5844, 6209, - 6574, 6940, 7305, 7670, 8035, 8401, 8766, 9131, 9496, 9862, 10227, 10592, 10957, 11323, 11688, 12053, 12418, 12784, - 13149, 13514, 13879, 14245, 14610, 14975, 15340, 15706, 16071, 16436, 16801, 17167, 17532, 17897, 18262, 18628, 18993, 19358, - 19723, 20089, 20454, 20819, 21184, 21550, 21915, 22280, 22645, 23011, 23376, 23741, 24106, 24472, 24837, 25202, 25567, 25933, - 26298, 26663, 27028, 27394, 27759, 28124, 28489, 28855, 29220, 29585, 29950, 30316, 30681, 31046, 31411, 31777, 32142, 32507, - 32872, 33238, 33603, 33968, 34333, 34699, 35064, 35429, 35794, 36160, 36525, 36890, 37255, 37621, 37986, 38351, 38716, 39082, - 39447, 39812, 40177, 40543, 40908, 41273, 41638, 42004, 42369, 42734, 43099, 43465, 43830, 44195, 44560, 44926, 45291, 45656, - 46021, 46387, 46752, 47117, 47482, 47847, 48212, 48577, 48942, 49308, 49673}; - - -constexpr Int32 minYear = 1970; -constexpr Int32 maxYear = 2106; - -struct Date -{ - Int32 year = 1970; - Int32 month = 1; - Int32 day = 1; - bool is_ad = true; // AD -> true, BC -> false. - - Int32 week = 1; // Week of year based on ISO week date, e.g: 27 - Int32 day_of_week = 1; // Day of week, Monday:1, Tuesday:2, ..., Sunday:7 - bool week_date_format = false; - - Int32 day_of_year = 1; - bool day_of_year_format = false; - - bool century_format = false; - - bool is_year_of_era = false; // Year of era cannot be zero or negative. - bool has_year = false; // Whether year was explicitly specified. - - Int32 hour = 0; - Int32 minute = 0; - Int32 second = 0; - // Int32 microsecond = 0; - bool is_am = true; // AM -> true, PM -> false - std::optional time_zone_offset; - - bool is_clock_hour = false; // Whether most recent hour specifier is clockhour - bool is_hour_of_half_day = false; // Whether most recent hour specifier is of half day. - - std::vector day_of_month_values; - std::vector day_of_year_values; - - /// For debug - [[maybe_unused]] String toString() const - { - String res; - res += "year:" + std::to_string(year); - res += ","; - res += "month:" + std::to_string(month); - res += ","; - res += "day:" + std::to_string(day); - res += ","; - res += "hour:" + std::to_string(hour); - res += ","; - res += "minute:" + std::to_string(minute); - res += ","; - res += "second:" + std::to_string(second); - return res; - } - - static bool isLeapYear(Int32 year_) { return year_ % 4 == 0 && (year_ % 100 != 0 || year_ % 400 == 0); } - - static bool isDateValid(Int32 year_, Int32 month_, Int32 day_) - { - if (month_ < 1 || month_ > 12) - return false; - - if (year_ < minYear || year_ > maxYear) - return false; - - bool leap = isLeapYear(year_); - if (day_ < 1) - return false; - - if (leap && day_ > leapDays[month_]) - return false; - - if (!leap && day_ > normalDays[month_]) - return false; - return true; - } - - static bool isDayOfYearValid(Int32 year_, Int32 day_of_year_) - { - if (year_ < minYear || year_ > maxYear) - return false; - - if (day_of_year_ < 1 || day_of_year_ > 365 + (isLeapYear(year_) ? 1 : 0)) - return false; - - return true; - } - - static bool isWeekDateValid(Int32 week_year_, Int32 week_of_year_, Int32 day_of_week_) - { - if (day_of_week_ < 1 || day_of_week_ > 7) - return false; - - if (week_of_year_ < 1 || week_of_year_ > 52) - return false; - - if (week_year_ < minYear || week_year_ > maxYear) - return false; - - return true; - } - - static Int32 extractISODayOfTheWeek(Int32 days_since_epoch) - { - if (days_since_epoch < 0) - { - // negative date: start off at 4 and cycle downwards - return (7 - ((-int64_t(days_since_epoch) + 3) % 7)); - } - else - { - // positive date: start off at 4 and cycle upwards - return ((int64_t(days_since_epoch) + 3) % 7) + 1; - } - } - - static Int32 daysSinceEpochFromWeekDate(int32_t week_year_, int32_t week_of_year_, int32_t day_of_week_) - { - if (!isWeekDateValid(week_year_, week_of_year_, day_of_week_)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid week date"); - - Int32 days_since_epoch_of_jan_fourth = daysSinceEpochFromDate(week_year_, 1, 4); - Int32 first_day_of_week_year = extractISODayOfTheWeek(days_since_epoch_of_jan_fourth); - return days_since_epoch_of_jan_fourth - (first_day_of_week_year - 1) + 7 * (week_of_year_ - 1) + day_of_week_ - 1; - } - - static Int32 daysSinceEpochFromDayOfYear(Int32 year_, Int32 day_of_year_) - { - if (!isDayOfYearValid(year_, day_of_year_)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of year"); - - Int32 res = daysSinceEpochFromDate(year_, 1, 1); - res += day_of_year_ - 1; - return res; - } - - static Int32 daysSinceEpochFromDate(Int32 year_, Int32 month_, Int32 day_) - { - if (!isDateValid(year_, month_, day_)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid date"); - - Int32 res = cumulativeYearDays[year_ - 1970]; - res += isLeapYear(year_) ? cumulativeLeapDays[month_ - 1] : cumulativeDays[month_ - 1]; - res += day_ - 1; - return res; - } - - - Int64 checkAndGetDateTime(const DateLUTImpl & time_zone) - { - /// Era is BC and year of era is provided - if (is_year_of_era && !is_ad) - year = -1 * (year - 1); - - if (is_hour_of_half_day && !is_am) - hour += 12; - - - /// Ensure all day of year values are valid for ending year value - for (const auto d : day_of_month_values) - { - if (!isDateValid(year, month, d)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of month."); - } - - // Ensure all day of year values are valid for ending year value - for (const auto d : day_of_year_values) - { - if (!isDayOfYearValid(year, d)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of year."); - } - - // Convert the parsed date/time into a timestamp. - Int32 days_since_epoch; - if (week_date_format) - days_since_epoch = daysSinceEpochFromWeekDate(year, week, day_of_week); - else if (day_of_year_format) - days_since_epoch = daysSinceEpochFromDayOfYear(year, day_of_year); - else - days_since_epoch = daysSinceEpochFromDate(year, month, day); - - Int64 seconds_since_epoch = days_since_epoch * 86400 + hour * 3600 + minute * 60 + second; - - /// Time zone is not specified, use local time zone - if (!time_zone_offset) - *time_zone_offset = time_zone.getOffsetAtStartOfEpoch(); - - // std::cout << "timezonename:" << time_zone.getTimeZone() << std::endl; - // std::cout << "time_zone_offset:" << *time_zone_offset << time_zone.getOffsetAtStartOfEpoch() << std::endl; - // std::cout << "before timestamp:" << seconds_since_epoch << std::endl; - /// Time zone is specified in format string. - seconds_since_epoch -= *time_zone_offset; - // std::cout << "after timestamp:" << seconds_since_epoch << std::endl; - return seconds_since_epoch; - } -}; - -class Action -{ -private: - using Func = Pos (*)(Pos cur, Pos end, Date & date); - Func func{nullptr}; - std::string func_name; - - std::string literal; - -public: - explicit Action(Func func_, const char * func_name_) : func(func_), func_name(func_name_) { } - - template - explicit Action(const Literal & literal_) : literal(literal_) - { - } - - /// For debug - [[maybe_unused]] String toString()const - { - if (func) - return "func:" + func_name; - else - return "literal:" + literal; - } - - Pos perform(Pos cur, Pos end, Date & date) const - { - if (func) - return func(cur, end, date); - else - { - ensureSpace(cur, end, literal.size(), "requires size >= " + std::to_string(literal.size())); - if (std::string_view(cur, literal.size()) != literal) - throw Exception( - ErrorCodes::LOGICAL_ERROR, "Expect literal {} but {} provided", literal, std::string_view(cur, literal.size())); - cur += literal.size(); - return cur; - } - } - - template - static Pos readNumber2(Pos cur, Pos end, T & res) - { - ensureSpace(cur, end, 2, "readNumber2 requires size >= 2"); - res = (*cur - '0') * 10; - ++cur; - res += *cur - '0'; - ++cur; - return cur; - } - - template - static Pos readNumber3(Pos cur, Pos end, T & res) - { - cur = readNumber2(cur, end, res); - - ensureSpace(cur, end, 1, "readNumber3 requires size >= 3"); - res = res * 10 + (*cur - '0'); - ++cur; - return cur; - } - - template - static Pos readNumber4(Pos cur, Pos end, T & res) - { - cur = readNumber2(cur, end, res); - - T tmp; - cur = readNumber2(cur, end, tmp); - res = res * 100 + tmp; - return cur; - } - - static ALWAYS_INLINE void ensureSpace(Pos cur, Pos end, size_t len, const String & msg) - { - if (cur > end || cur + len > end) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because {}", msg); - } - - static ALWAYS_INLINE Pos assertChar(Pos cur, Pos end, char ch) - { - ensureSpace(cur, end, 1, "assertChar requires size >= 1"); - - if (*cur != ch) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expect char {}, but {} provided", String(ch, 1), String(*cur, 1)); - - ++cur; - return cur; - } - - static Pos mysqlDayOfWeekTextShort(Pos cur, Pos end, Date & date) - { - ensureSpace(cur, end, 3, "Parsing DayOfWeekTextShort requires size >= 3"); - - String str(cur, 3); - Poco::toLower(str); - Int32 i = 0; - for (; i < 7; ++i) - if (str == weekdaysShort[i]) - break; - - if (i == 7) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown short week text"); - - date.day_of_week = i + 1; - date.week_date_format = true; - date.day_of_year_format = false; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } - cur += 3; - return cur; - } - - static Pos mysqlMonthOfYearTextShort(Pos cur, Pos end, Date & date) - { - ensureSpace(cur, end, 3, "Parsing MonthOfYearTextShort requires size >= 3"); - - String str(cur, 3); - Poco::toLower(str); - - Int32 i = 0; - for (; i < 12; ++i) - if (str == monthsShort[i]) - break; - - if (i == 12) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown short month text"); - - date.month = i + 1; - cur += 3; - return cur; - } - - static Pos mysqlMonth(Pos cur, Pos end, Date & date) { return readNumber2(cur, end, date.month); } - - static Pos mysqlCentury(Pos cur, Pos end, Date & date) - { - Int32 centuray; - cur = readNumber2(cur, end, centuray); - date.century_format = true; - date.year = centuray * 100; - date.has_year = true; - return cur; - } - - static Pos mysqlDayOfMonth(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.day); - date.day_of_month_values.push_back(date.day); - date.week_date_format = false; - date.day_of_year_format = false; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } - return cur; - } - - static Pos mysqlAmericanDate(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.month); - cur = assertChar(cur, end, '/'); - - cur = readNumber2(cur, end, date.day); - cur = assertChar(cur, end, '/'); - - cur = readNumber2(cur, end, date.year); - cur = assertChar(cur, end, '/'); - - date.week_date_format = false; - date.day_of_year_format = false; - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; - return cur; - } - - - static Pos mysqlDayOfMonthSpacePadded(Pos cur, Pos end, Date & date) - { - ensureSpace(cur, end, 2, "mysqlDayOfMonthSpacePadded requires size >= 2"); - - date.day = *cur == ' ' ? 0 : (*cur - '0'); - ++cur; - - date.day = 10 * date.day + (*cur - '0'); - ++cur; - - date.week_date_format = false; - date.day_of_year_format = false; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } - return cur; - } - - static Pos mysqlISO8601Date(Pos cur, Pos end, Date & date) - { - cur = readNumber4(cur, end, date.year); - cur = assertChar(cur, end, '-'); - cur = readNumber2(cur, end, date.month); - cur = assertChar(cur, end, '-'); - cur = readNumber2(cur, end, date.day); - - date.week_date_format = false; - date.day_of_year_format = false; - - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; - return cur; - } - - static Pos mysqlISO8601Year2(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.year); - date.year += 2000; - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; - return cur; - } - - static Pos mysqlISO8601Year4(Pos cur, Pos end, Date & date) - { - cur = readNumber4(cur, end, date.year); - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; - return cur; - } - - static Pos mysqlDayOfYear(Pos cur, Pos end, Date & date) - { - cur = readNumber3(cur, end, date.day_of_year); - - date.day_of_year_values.push_back(date.day_of_year); - date.day_of_year_format = true; - date.week_date_format = false; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } - return cur; - } - - static Pos mysqlDayOfWeek(Pos cur, Pos end, Date & date) - { - ensureSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1"); - - date.day_of_week = *cur - '0'; - date.week_date_format = true; - date.day_of_year_format = false; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } - return cur; - } - - static Pos mysqlISO8601Week(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.week); - date.week_date_format = true; - date.day_of_year_format = false; - if (date.has_year) - { - date.has_year = true; - date.year = 2000; - } - return cur; - } - - static Pos mysqlDayOfWeek0To6(Pos cur, Pos end, Date & date) - { - cur = mysqlDayOfWeek(cur, end, date); - if (date.day_of_week == 0) - date.day_of_week = 7; - - return cur; - } - - static Pos mysqlDayOfWeekTextLong(Pos cur, Pos end, Date & date) - { - mysqlDayOfWeekTextShort(cur, end, date); - auto expect_text = weekdaysFull[date.day_of_week - 1]; - - ensureSpace(cur, end, expect_text.size(), "mysqlDayOfWeekTextLong requires size >= " + std::to_string(expect_text.size())); - std::string_view text(cur, expect_text.size()); - if (text != expect_text) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown full day of week text {}", expect_text); - - cur += expect_text.size(); - return cur; - } - - static Pos mysqlYear2(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.year); - date.year += 2000; - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; - return cur; - } - - static Pos mysqlYear4(Pos cur, Pos end, Date & date) - { - cur = readNumber4(cur, end, date.year); - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; - return cur; - } - - static Pos mysqlTimezoneOffset(Pos cur, Pos end, Date & date) - { - /// TODO figure out what timezone_id mean - ensureSpace(cur, end, 1, "Parse mysqlTimezoneOffset failed"); - Int32 sign = 1; - if (*cur == '-') - sign = -1; - ++cur; - - Int32 hour; - cur = readNumber2(cur, end, hour); - - Int32 minute; - cur = readNumber2(cur, end, minute); - - *date.time_zone_offset = sign * (hour * 3600 + minute * 60); - return cur; - } - - static Pos mysqlMinute(Pos cur, Pos end, Date & date) { return readNumber2(cur, end, date.minute); } - - static Pos mysqlAMPM(Pos cur, Pos end, Date & date) - { - ensureSpace(cur, end, 2, "mysqlAMPM requires size >= 2"); - - std::string text(cur, 2); - Poco::toUpper(text); - if (text == "PM") - date.is_am = true; - else if (text == "AM") - date.is_am = false; - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Text should be AM or PM, but {} provided", text); - - cur += 2; - return cur; - } - - static Pos mysqlHHMM12(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.hour); - date.is_clock_hour = false; - date.is_hour_of_half_day = true; - - cur = assertChar(cur, end, ':'); - cur = readNumber2(cur, end, date.minute); - cur = assertChar(cur, end, ' '); - cur = mysqlAMPM(cur, end, date); - return cur; - } - - static Pos mysqlHHMM24(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.hour); - date.is_clock_hour = false; - date.is_hour_of_half_day = false; - - cur = assertChar(cur, end, ':'); - cur = readNumber2(cur, end, date.minute); - return cur; - } - - static Pos mysqlSecond(Pos cur, Pos end, Date & date) { return readNumber2(cur, end, date.second); } - - static Pos mysqlISO8601Time(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.hour); - cur = assertChar(cur, end, ':'); - cur = readNumber2(cur, end, date.minute); - cur = assertChar(cur, end, ':'); - cur = readNumber2(cur, end, date.second); - - date.is_clock_hour = false; - date.is_hour_of_half_day = false; - return cur; - } - - static Pos mysqlHour12(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.hour); - date.is_hour_of_half_day = true; - date.is_clock_hour = false; - return cur; - } - - static Pos mysqlHour24(Pos cur, Pos end, Date & date) - { - cur = readNumber2(cur, end, date.hour); - date.is_hour_of_half_day = false; - date.is_clock_hour = false; - return cur; - } -}; - - -struct ParseDateTimeTraits -{ - enum class ParseSyntax - { - MySQL, - Joda + using Pos = const char *; + + constexpr std::string_view weekdaysShort[] = {"sun", "mon", "tue", "wed", "thu", "fri", "sat"}; + constexpr std::string_view weekdaysFull[] = {"sunday", "monday", "tuesday", "wednesday", "thursday", "friday", "saturday"}; + constexpr std::string_view monthsShort[] = {"jan", "feb", "mar", "apr", "may", "jun", "jul", "aug", "sep", "oct", "nov", "dec"}; + const std::unordered_map> dayOfWeekMap{ + {"mon", {"day", 1}}, + {"tue", {"sday", 2}}, + {"wed", {"nesday", 3}}, + {"thu", {"rsday", 4}}, + {"fri", {"day", 5}}, + {"sat", {"urday", 6}}, + {"sun", {"day", 7}}, }; -}; -#define ACTION_ARGS(func) &(func), #func + const std::unordered_map> monthMap{ + {"jan", {"uary", 1}}, + {"feb", {"ruary", 2}}, + {"mar", {"rch", 3}}, + {"apr", {"il", 4}}, + {"may", {"", 5}}, + {"jun", {"e", 6}}, + {"jul", {"y", 7}}, + {"aug", {"ust", 8}}, + {"sep", {"tember", 9}}, + {"oct", {"ober", 10}}, + {"nov", {"ember", 11}}, + {"dec", {"ember", 12}}, + }; -/// _FUNC_(str[, format, timezone]) -template -class FunctionParseDateTimeImpl : public IFunction -{ -public: - static constexpr auto name = Name::name; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } + constexpr Int32 leapDays[] = {0, 31, 29, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}; + constexpr Int32 normalDays[] = {0, 31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}; - String getName() const override { return name; } + constexpr Int32 cumulativeLeapDays[] = {0, 31, 60, 91, 121, 152, 182, 213, 244, 274, 305, 335, 366}; + constexpr Int32 cumulativeDays[] = {0, 31, 59, 90, 120, 151, 181, 212, 243, 273, 304, 334, 365}; + constexpr Int32 cumulativeYearDays[] + = {0, 365, 730, 1096, 1461, 1826, 2191, 2557, 2922, 3287, 3652, 4018, 4383, 4748, 5113, 5479, 5844, 6209, + 6574, 6940, 7305, 7670, 8035, 8401, 8766, 9131, 9496, 9862, 10227, 10592, 10957, 11323, 11688, 12053, 12418, 12784, + 13149, 13514, 13879, 14245, 14610, 14975, 15340, 15706, 16071, 16436, 16801, 17167, 17532, 17897, 18262, 18628, 18993, 19358, + 19723, 20089, 20454, 20819, 21184, 21550, 21915, 22280, 22645, 23011, 23376, 23741, 24106, 24472, 24837, 25202, 25567, 25933, + 26298, 26663, 27028, 27394, 27759, 28124, 28489, 28855, 29220, 29585, 29950, 30316, 30681, 31046, 31411, 31777, 32142, 32507, + 32872, 33238, 33603, 33968, 34333, 34699, 35064, 35429, 35794, 36160, 36525, 36890, 37255, 37621, 37986, 38351, 38716, 39082, + 39447, 39812, 40177, 40543, 40908, 41273, 41638, 42004, 42369, 42734, 43099, 43465, 43830, 44195, 44560, 44926, 45291, 45656, + 46021, 46387, 46752, 47117, 47482, 47847, 48212, 48577, 48942, 49308, 49673}; - bool useDefaultImplementationForConstants() const override { return true; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } + constexpr Int32 minYear = 1970; + constexpr Int32 maxYear = 2106; - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + /// Counts the number of literal characters in Joda format string until the next closing literal + /// sequence single quote. Returns -1 if no literal single quote was found. + /// In Joda format string(https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html) + /// literal content must be quoted with single quote. and two single quote means literal with one single quote. + /// For example: + /// Format string: "'aaaa'", unescaped literal: "aaaa"; + /// Format string: "'aa''aa'", unescaped literal: "aa'aa"; + /// Format string: "'aaa''aa" is not valid because of missing of end single quote. + [[maybe_unused]] Int64 numLiteralChars(const char * cur, const char * end) { - if (arguments.size() != 1 && arguments.size() != 2 && arguments.size() != 3) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be 1, 2 or 3", - getName(), - arguments.size()); - - if (!isString(arguments[0].type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of first argument of function {} when arguments size is 1. Should be string", - arguments[0].type->getName(), - getName()); - - if (arguments.size() > 1 && !isString(arguments[1].type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of second argument of function {} when arguments size is 1. Should be string", - arguments[0].type->getName(), - getName()); - - if (arguments.size() > 2 && !isString(arguments[2].type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of third argument of function {} when arguments size is 1. Should be string", - arguments[0].type->getName(), - getName()); - - String time_zone_name = getTimeZone(arguments).second; - return std::make_shared(time_zone_name); + bool found = false; + Int64 count = 0; + while (cur < end) + { + if (*cur == '\'') + { + if (cur + 1 < end && *(cur + 1) == '\'') + { + count += 2; + cur += 2; + } + else + { + found = true; + break; + } + } + else + { + ++count; + ++cur; + } + } + return found ? count : -1; } - ColumnPtr - executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override + struct Date { - const auto * col_str = checkAndGetColumn(arguments[0].column.get()); - if (!col_str) - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of first ('str') argument of function {}. Must be string.", - arguments[0].column->getName(), - getName()); + Int32 year = 1970; + Int32 month = 1; + Int32 day = 1; + bool is_ad = true; // AD -> true, BC -> false. - String format = getFormat(arguments); - const auto * time_zone = getTimeZone(arguments).first; - // std::cout << "timezonename:" << getTimeZone(arguments).second << std::endl; + Int32 week = 1; // Week of year based on ISO week date, e.g: 27 + Int32 day_of_week = 1; // Day of week, Monday:1, Tuesday:2, ..., Sunday:7 + bool week_date_format = false; - std::vector instructions; - parseFormat(format, instructions); + Int32 day_of_year = 1; + bool day_of_year_format = false; - auto col_res = ColumnDateTime::create(); - col_res->reserve(input_rows_count); - auto & data_res = col_res->getData(); - for (size_t i = 0; i < input_rows_count; ++i) + bool century_format = false; + + bool is_year_of_era = false; // Year of era cannot be zero or negative. + bool has_year = false; // Whether year was explicitly specified. + + Int32 hour = 0; + Int32 minute = 0; + Int32 second = 0; + // Int32 microsecond = 0; + bool is_am = true; // AM -> true, PM -> false + std::optional time_zone_offset; + + bool is_clock_hour = false; // Whether most recent hour specifier is clockhour + bool is_hour_of_half_day = false; // Whether most recent hour specifier is of half day. + + std::vector day_of_month_values; + std::vector day_of_year_values; + + /// For debug + [[maybe_unused]] String toString() const { - StringRef str_ref = col_str->getDataAt(i); - Pos cur = str_ref.data; - Pos end = str_ref.data + str_ref.size; - Date date; - for (const auto & instruction : instructions) + String res; + res += "year:" + std::to_string(year); + res += ","; + res += "month:" + std::to_string(month); + res += ","; + res += "day:" + std::to_string(day); + res += ","; + res += "hour:" + std::to_string(hour); + res += ","; + res += "minute:" + std::to_string(minute); + res += ","; + res += "second:" + std::to_string(second); + return res; + } + + static bool isLeapYear(Int32 year_) { return year_ % 4 == 0 && (year_ % 100 != 0 || year_ % 400 == 0); } + + static bool isDateValid(Int32 year_, Int32 month_, Int32 day_) + { + if (month_ < 1 || month_ > 12) + return false; + + if (year_ < minYear || year_ > maxYear) + return false; + + bool leap = isLeapYear(year_); + if (day_ < 1) + return false; + + if (leap && day_ > leapDays[month_]) + return false; + + if (!leap && day_ > normalDays[month_]) + return false; + return true; + } + + static bool isDayOfYearValid(Int32 year_, Int32 day_of_year_) + { + if (year_ < minYear || year_ > maxYear) + return false; + + if (day_of_year_ < 1 || day_of_year_ > 365 + (isLeapYear(year_) ? 1 : 0)) + return false; + + return true; + } + + static bool isWeekDateValid(Int32 week_year_, Int32 week_of_year_, Int32 day_of_week_) + { + if (day_of_week_ < 1 || day_of_week_ > 7) + return false; + + if (week_of_year_ < 1 || week_of_year_ > 52) + return false; + + if (week_year_ < minYear || week_year_ > maxYear) + return false; + + return true; + } + + static Int32 extractISODayOfTheWeek(Int32 days_since_epoch) + { + if (days_since_epoch < 0) { - cur = instruction.perform(cur, end, date); - // std::cout << "instruction:" << instruction.toString() << std::endl; - // std::cout << "date:" << date.toString() << std::endl; + // negative date: start off at 4 and cycle downwards + return (7 - ((-int64_t(days_since_epoch) + 3) % 7)); + } + else + { + // positive date: start off at 4 and cycle upwards + return ((int64_t(days_since_epoch) + 3) % 7) + 1; + } + } + + static Int32 daysSinceEpochFromWeekDate(int32_t week_year_, int32_t week_of_year_, int32_t day_of_week_) + { + if (!isWeekDateValid(week_year_, week_of_year_, day_of_week_)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid week date"); + + Int32 days_since_epoch_of_jan_fourth = daysSinceEpochFromDate(week_year_, 1, 4); + Int32 first_day_of_week_year = extractISODayOfTheWeek(days_since_epoch_of_jan_fourth); + return days_since_epoch_of_jan_fourth - (first_day_of_week_year - 1) + 7 * (week_of_year_ - 1) + day_of_week_ - 1; + } + + static Int32 daysSinceEpochFromDayOfYear(Int32 year_, Int32 day_of_year_) + { + if (!isDayOfYearValid(year_, day_of_year_)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of year"); + + Int32 res = daysSinceEpochFromDate(year_, 1, 1); + res += day_of_year_ - 1; + return res; + } + + static Int32 daysSinceEpochFromDate(Int32 year_, Int32 month_, Int32 day_) + { + if (!isDateValid(year_, month_, day_)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid date"); + + Int32 res = cumulativeYearDays[year_ - 1970]; + res += isLeapYear(year_) ? cumulativeLeapDays[month_ - 1] : cumulativeDays[month_ - 1]; + res += day_ - 1; + return res; + } + + + Int64 checkAndGetDateTime(const DateLUTImpl & time_zone) + { + /// Era is BC and year of era is provided + if (is_year_of_era && !is_ad) + year = -1 * (year - 1); + + if (is_hour_of_half_day && !is_am) + hour += 12; + + + /// Ensure all day of year values are valid for ending year value + for (const auto d : day_of_month_values) + { + if (!isDateValid(year, month, d)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of month."); } - // Ensure all input was consumed. - if (cur < end) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Invalid format input {} is malformed at {}", - str_ref.toView(), - std::string_view(cur, end - cur)); + // Ensure all day of year values are valid for ending year value + for (const auto d : day_of_year_values) + { + if (!isDayOfYearValid(year, d)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of year."); + } - Int64 time = date.checkAndGetDateTime(*time_zone); - data_res.push_back(static_cast(time)); + // Convert the parsed date/time into a timestamp. + Int32 days_since_epoch; + if (week_date_format) + days_since_epoch = daysSinceEpochFromWeekDate(year, week, day_of_week); + else if (day_of_year_format) + days_since_epoch = daysSinceEpochFromDayOfYear(year, day_of_year); + else + days_since_epoch = daysSinceEpochFromDate(year, month, day); + + Int64 seconds_since_epoch = days_since_epoch * 86400 + hour * 3600 + minute * 60 + second; + + /// Time zone is not specified, use local time zone + if (!time_zone_offset) + *time_zone_offset = time_zone.getOffsetAtStartOfEpoch(); + + // std::cout << "timezonename:" << time_zone.getTimeZone() << std::endl; + // std::cout << "time_zone_offset:" << *time_zone_offset << time_zone.getOffsetAtStartOfEpoch() << std::endl; + // std::cout << "before timestamp:" << seconds_since_epoch << std::endl; + /// Time zone is specified in format string. + seconds_since_epoch -= *time_zone_offset; + // std::cout << "after timestamp:" << seconds_since_epoch << std::endl; + return seconds_since_epoch; + } + }; + + + struct ParseDateTimeTraits + { + enum class ParseSyntax + { + MySQL, + Joda + }; + }; + + + /// _FUNC_(str[, format, timezone]) + template + class FunctionParseDateTimeImpl : public IFunction + { + public: + static constexpr auto name = Name::name; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.size() != 1 && arguments.size() != 2 && arguments.size() != 3) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 1, 2 or 3", + getName(), + arguments.size()); + + if (!isString(arguments[0].type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {} when arguments size is 1. Should be string", + arguments[0].type->getName(), + getName()); + + if (arguments.size() > 1 && !isString(arguments[1].type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {} when arguments size is 1. Should be string", + arguments[0].type->getName(), + getName()); + + if (arguments.size() > 2 && !isString(arguments[2].type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of third argument of function {} when arguments size is 1. Should be string", + arguments[0].type->getName(), + getName()); + + String time_zone_name = getTimeZone(arguments).second; + return std::make_shared(time_zone_name); } - return col_res; - } - - -private: - ALWAYS_INLINE void parseFormat(const String & format, std::vector & instructions) const - { - if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::MySQL) - parseMysqlFormat(format, instructions); - else if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::Joda) - parseJodaFormat(format, instructions); - else - throw Exception( - ErrorCodes::NOT_IMPLEMENTED, - "Unknown datetime format style {} in function {}", - magic_enum::enum_name(parse_syntax), - getName()); - } - - ALWAYS_INLINE void parseMysqlFormat(const String & format, std::vector & instructions) const - { - Pos pos = format.data(); - Pos end = pos + format.size(); - while (true) + ColumnPtr + executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override { - Pos percent_pos = find_first_symbols<'%'>(pos, end); - if (percent_pos < end) + const auto * col_str = checkAndGetColumn(arguments[0].column.get()); + if (!col_str) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of first ('str') argument of function {}. Must be string.", + arguments[0].column->getName(), + getName()); + + String format = getFormat(arguments); + const auto * time_zone = getTimeZone(arguments).first; + // std::cout << "timezonename:" << getTimeZone(arguments).second << std::endl; + + std::vector instructions; + parseFormat(format, instructions); + + auto col_res = ColumnDateTime::create(); + col_res->reserve(input_rows_count); + auto & data_res = col_res->getData(); + for (size_t i = 0; i < input_rows_count; ++i) { - if (pos < percent_pos) - instructions.emplace_back(std::string_view(pos, percent_pos - pos)); - - pos = percent_pos + 1; - if (pos >= end) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Sign '%' is the last in format, if you need it, use '%%'"); - - switch (*pos) + StringRef str_ref = col_str->getDataAt(i); + Pos cur = str_ref.data; + Pos end = str_ref.data + str_ref.size; + Date date; + for (const auto & instruction : instructions) { - // Abbreviated weekday [Mon...Sun] - case 'a': - instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfWeekTextShort)); - break; - - // Abbreviated month [Jan...Dec] - case 'b': - instructions.emplace_back(ACTION_ARGS(Action::mysqlMonthOfYearTextShort)); - break; - - // Month as a decimal number (01-12) - case 'c': - instructions.emplace_back(ACTION_ARGS(Action::mysqlMonth)); - break; - - // Year, divided by 100, zero-padded - case 'C': - instructions.emplace_back(ACTION_ARGS(Action::mysqlCentury)); - break; - - // Day of month, zero-padded (01-31) - case 'd': - instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfMonth)); - break; - - // Short MM/DD/YY date, equivalent to %m/%d/%y - case 'D': - instructions.emplace_back(ACTION_ARGS(Action::mysqlAmericanDate)); - break; - - // Day of month, space-padded ( 1-31) 23 - case 'e': - instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfMonthSpacePadded)); - break; - - // Fractional seconds - case 'f': - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for fractional seconds"); - - // Short YYYY-MM-DD date, equivalent to %Y-%m-%d 2001-08-23 - case 'F': - instructions.emplace_back(ACTION_ARGS(Action::mysqlISO8601Date)); - break; - - // Last two digits of year of ISO 8601 week number (see %G) - case 'g': - instructions.emplace_back(ACTION_ARGS(Action::mysqlISO8601Year2)); - break; - - // Year of ISO 8601 week number (see %V) - case 'G': - instructions.emplace_back(ACTION_ARGS(Action::mysqlISO8601Year4)); - break; - - // Day of the year (001-366) 235 - case 'j': - instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfYear)); - break; - - // Month as a decimal number (01-12) - case 'm': - instructions.emplace_back(ACTION_ARGS(Action::mysqlMonth)); - break; - - // ISO 8601 weekday as number with Monday as 1 (1-7) - case 'u': - instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfWeek)); - break; - - // ISO 8601 week number (01-53) - case 'V': - instructions.emplace_back(ACTION_ARGS(Action::mysqlISO8601Week)); - break; - - // Weekday as a decimal number with Sunday as 0 (0-6) 4 - case 'w': - instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfWeek0To6)); - break; - - // Full weekday [Monday...Sunday] - case 'W': - instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfWeekTextLong)); - break; - - // Two digits year - case 'y': - instructions.emplace_back(ACTION_ARGS(Action::mysqlYear2)); - break; - - // Four digits year - case 'Y': - instructions.emplace_back(ACTION_ARGS(Action::mysqlYear4)); - break; - - // Quarter (1-4) - case 'Q': - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for quarter"); - break; - - // Offset from UTC timezone as +hhmm or -hhmm - case 'z': - instructions.emplace_back(ACTION_ARGS(Action::mysqlTimezoneOffset)); - break; - - /// Time components. If the argument is Date, not a DateTime, then this components will have default value. - - // Minute (00-59) - case 'M': - instructions.emplace_back(ACTION_ARGS(Action::mysqlMinute)); - break; - - // AM or PM - case 'p': - instructions.emplace_back(ACTION_ARGS(Action::mysqlAMPM)); - break; - - // 12-hour HH:MM time, equivalent to %h:%i %p 2:55 PM - case 'r': - instructions.emplace_back(ACTION_ARGS(Action::mysqlHHMM12)); - break; - - // 24-hour HH:MM time, equivalent to %H:%i 14:55 - case 'R': - instructions.emplace_back(ACTION_ARGS(Action::mysqlHHMM24)); - break; - - // Seconds - case 's': - instructions.emplace_back(ACTION_ARGS(Action::mysqlSecond)); - break; - - // Seconds - case 'S': - instructions.emplace_back(ACTION_ARGS(Action::mysqlSecond)); - break; - - // ISO 8601 time format (HH:MM:SS), equivalent to %H:%i:%S 14:55:02 - case 'T': - instructions.emplace_back(ACTION_ARGS(Action::mysqlISO8601Time)); - break; - - // Hour in 12h format (01-12) - case 'h': - instructions.emplace_back(ACTION_ARGS(Action::mysqlHour12)); - break; - - // Hour in 24h format (00-23) - case 'H': - instructions.emplace_back(ACTION_ARGS(Action::mysqlHour24)); - break; - - // Minute of hour range [0, 59] - case 'i': - instructions.emplace_back(ACTION_ARGS(Action::mysqlMinute)); - break; - - // Hour in 12h format (01-12) - case 'I': - instructions.emplace_back(ACTION_ARGS(Action::mysqlHour12)); - break; - - // Hour in 24h format (00-23) - case 'k': - instructions.emplace_back(ACTION_ARGS(Action::mysqlHour24)); - break; - - // Hour in 12h format (01-12) - case 'l': - instructions.emplace_back(ACTION_ARGS(Action::mysqlHour12)); - break; - - case 't': - instructions.emplace_back("\t"); - break; - - case 'n': - instructions.emplace_back("\n"); - break; - - // Escaped literal characters. - case '%': - instructions.emplace_back("\n"); - break; - - // Unimplemented - case 'U': - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for WEEK (Sun-Sat)"); - case 'v': - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for WEEK (Mon-Sun)"); - case 'x': - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for YEAR for week (Mon-Sun)"); - case 'X': - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for YEAR for week (Sun-Sat)"); - - default: - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Incorrect syntax '{}', symbol is not supported '{}' for function {}", - format, - *pos, - getName()); + cur = instruction.perform(cur, end, date); + // std::cout << "instruction:" << instruction.toString() << std::endl; + // std::cout << "date:" << date.toString() << std::endl; } - ++pos; - } - else - { - if (pos < end) - instructions.emplace_back(std::string_view(pos, end - pos)); - break; + // Ensure all input was consumed. + if (cur < end) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Invalid format input {} is malformed at {}", + str_ref.toView(), + std::string_view(cur, end - cur)); + + Int64 time = date.checkAndGetDateTime(*time_zone); + data_res.push_back(static_cast(time)); } + + return col_res; } - } - - void parseJodaFormat(const String & /*format*/, std::vector & /*instructions*/) const { } - ALWAYS_INLINE String getFormat(const ColumnsWithTypeAndName & arguments) const - { - if (arguments.size() < 2) + private: + class Action { - if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::Joda) - return "yyyy-MM-dd HH:mm:ss"; + private: + using Func = std::conditional_t< + parse_syntax == ParseDateTimeTraits::ParseSyntax::MySQL, + Pos (*)(Pos, Pos, Date &), + std::function>; + Func func{}; + std::string func_name; + + std::string literal; + + public: + explicit Action(Func && func_, const char * func_name_) : func(std::move(func_)), func_name(func_name_) { } + + explicit Action(const String & literal_) : literal(literal_) { } + explicit Action(String && literal_) : literal(std::move(literal_)) { } + + /// For debug + [[maybe_unused]] String toString() const + { + if (func) + return "func:" + func_name; + else + return "literal:" + literal; + } + + Pos perform(Pos cur, Pos end, Date & date) const + { + if (func) + return func(cur, end, date); + else + { + ensureSpace(cur, end, literal.size(), "requires size >= " + std::to_string(literal.size())); + if (std::string_view(cur, literal.size()) != literal) + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Expect literal {} but {} provided", literal, std::string_view(cur, literal.size())); + cur += literal.size(); + return cur; + } + } + + template + static Pos readNumber2(Pos cur, Pos end, T & res) + { + ensureSpace(cur, end, 2, "readNumber2 requires size >= 2"); + res = (*cur - '0') * 10; + ++cur; + res += *cur - '0'; + ++cur; + return cur; + } + + template + static Pos readNumber3(Pos cur, Pos end, T & res) + { + cur = readNumber2(cur, end, res); + + ensureSpace(cur, end, 1, "readNumber3 requires size >= 3"); + res = res * 10 + (*cur - '0'); + ++cur; + return cur; + } + + template + static Pos readNumber4(Pos cur, Pos end, T & res) + { + cur = readNumber2(cur, end, res); + + T tmp; + cur = readNumber2(cur, end, tmp); + res = res * 100 + tmp; + return cur; + } + + static ALWAYS_INLINE void ensureSpace(Pos cur, Pos end, size_t len, const String & msg) + { + if (cur > end || cur + len > end) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because {}", msg); + } + + static ALWAYS_INLINE Pos assertChar(Pos cur, Pos end, char ch) + { + ensureSpace(cur, end, 1, "assertChar requires size >= 1"); + + if (*cur != ch) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expect char {}, but {} provided", String(ch, 1), String(*cur, 1)); + + ++cur; + return cur; + } + + static Pos mysqlDayOfWeekTextShort(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 3, "Parsing DayOfWeekTextShort requires size >= 3"); + + String str(cur, 3); + Poco::toLowerInPlace(str); + Int32 i = 0; + for (; i < 7; ++i) + if (str == weekdaysShort[i]) + break; + + if (i == 7) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown short week text"); + + date.day_of_week = i + 1; + date.week_date_format = true; + date.day_of_year_format = false; + if (!date.has_year) + { + date.has_year = true; + date.year = 2000; + } + cur += 3; + return cur; + } + + static Pos mysqlMonthOfYearTextShort(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 3, "Parsing MonthOfYearTextShort requires size >= 3"); + + String str(cur, 3); + Poco::toLowerInPlace(str); + + Int32 i = 0; + for (; i < 12; ++i) + if (str == monthsShort[i]) + break; + + if (i == 12) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown short month text"); + + date.month = i + 1; + cur += 3; + return cur; + } + + static Pos mysqlMonth(Pos cur, Pos end, Date & date) { return readNumber2(cur, end, date.month); } + + static Pos mysqlCentury(Pos cur, Pos end, Date & date) + { + Int32 centuray; + cur = readNumber2(cur, end, centuray); + date.century_format = true; + date.year = centuray * 100; + date.has_year = true; + return cur; + } + + static Pos mysqlDayOfMonth(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.day); + date.day_of_month_values.push_back(date.day); + date.week_date_format = false; + date.day_of_year_format = false; + if (!date.has_year) + { + date.has_year = true; + date.year = 2000; + } + return cur; + } + + static Pos mysqlAmericanDate(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.month); + cur = assertChar(cur, end, '/'); + + cur = readNumber2(cur, end, date.day); + cur = assertChar(cur, end, '/'); + + cur = readNumber2(cur, end, date.year); + cur = assertChar(cur, end, '/'); + + date.week_date_format = false; + date.day_of_year_format = false; + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; + return cur; + } + + + static Pos mysqlDayOfMonthSpacePadded(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 2, "mysqlDayOfMonthSpacePadded requires size >= 2"); + + date.day = *cur == ' ' ? 0 : (*cur - '0'); + ++cur; + + date.day = 10 * date.day + (*cur - '0'); + ++cur; + + date.week_date_format = false; + date.day_of_year_format = false; + if (!date.has_year) + { + date.has_year = true; + date.year = 2000; + } + return cur; + } + + static Pos mysqlISO8601Date(Pos cur, Pos end, Date & date) + { + cur = readNumber4(cur, end, date.year); + cur = assertChar(cur, end, '-'); + cur = readNumber2(cur, end, date.month); + cur = assertChar(cur, end, '-'); + cur = readNumber2(cur, end, date.day); + + date.week_date_format = false; + date.day_of_year_format = false; + + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; + return cur; + } + + static Pos mysqlISO8601Year2(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.year); + date.year += 2000; + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; + return cur; + } + + static Pos mysqlISO8601Year4(Pos cur, Pos end, Date & date) + { + cur = readNumber4(cur, end, date.year); + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; + return cur; + } + + static Pos mysqlDayOfYear(Pos cur, Pos end, Date & date) + { + cur = readNumber3(cur, end, date.day_of_year); + + date.day_of_year_values.push_back(date.day_of_year); + date.day_of_year_format = true; + date.week_date_format = false; + if (!date.has_year) + { + date.has_year = true; + date.year = 2000; + } + return cur; + } + + static Pos mysqlDayOfWeek(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1"); + + date.day_of_week = *cur - '0'; + date.week_date_format = true; + date.day_of_year_format = false; + if (!date.has_year) + { + date.has_year = true; + date.year = 2000; + } + return cur; + } + + static Pos mysqlISO8601Week(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.week); + date.week_date_format = true; + date.day_of_year_format = false; + if (date.has_year) + { + date.has_year = true; + date.year = 2000; + } + return cur; + } + + static Pos mysqlDayOfWeek0To6(Pos cur, Pos end, Date & date) + { + cur = mysqlDayOfWeek(cur, end, date); + if (date.day_of_week == 0) + date.day_of_week = 7; + + return cur; + } + + static Pos mysqlDayOfWeekTextLong(Pos cur, Pos end, Date & date) + { + mysqlDayOfWeekTextShort(cur, end, date); + auto expect_text = weekdaysFull[date.day_of_week - 1]; + + ensureSpace(cur, end, expect_text.size(), "mysqlDayOfWeekTextLong requires size >= " + std::to_string(expect_text.size())); + std::string_view text(cur, expect_text.size()); + if (text != expect_text) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown full day of week text {}", expect_text); + + cur += expect_text.size(); + return cur; + } + + static Pos mysqlYear2(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.year); + date.year += 2000; + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; + return cur; + } + + static Pos mysqlYear4(Pos cur, Pos end, Date & date) + { + cur = readNumber4(cur, end, date.year); + date.century_format = false; + date.is_year_of_era = false; + date.has_year = true; + return cur; + } + + static Pos mysqlTimezoneOffset(Pos cur, Pos end, Date & date) + { + /// TODO figure out what timezone_id mean + ensureSpace(cur, end, 1, "Parse mysqlTimezoneOffset failed"); + Int32 sign = 1; + if (*cur == '-') + sign = -1; + ++cur; + + Int32 hour; + cur = readNumber2(cur, end, hour); + + Int32 minute; + cur = readNumber2(cur, end, minute); + + *date.time_zone_offset = sign * (hour * 3600 + minute * 60); + return cur; + } + + static Pos mysqlMinute(Pos cur, Pos end, Date & date) { return readNumber2(cur, end, date.minute); } + + static Pos mysqlAMPM(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 2, "mysqlAMPM requires size >= 2"); + + std::string text(cur, 2); + Poco::toUpper(text); + if (text == "PM") + date.is_am = true; + else if (text == "AM") + date.is_am = false; + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Text should be AM or PM, but {} provided", text); + + cur += 2; + return cur; + } + + static Pos mysqlHHMM12(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.hour); + date.is_clock_hour = false; + date.is_hour_of_half_day = true; + + cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, date.minute); + cur = assertChar(cur, end, ' '); + cur = mysqlAMPM(cur, end, date); + return cur; + } + + static Pos mysqlHHMM24(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.hour); + date.is_clock_hour = false; + date.is_hour_of_half_day = false; + + cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, date.minute); + return cur; + } + + static Pos mysqlSecond(Pos cur, Pos end, Date & date) { return readNumber2(cur, end, date.second); } + + static Pos mysqlISO8601Time(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.hour); + cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, date.minute); + cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, date.second); + + date.is_clock_hour = false; + date.is_hour_of_half_day = false; + return cur; + } + + static Pos mysqlHour12(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.hour); + date.is_hour_of_half_day = true; + date.is_clock_hour = false; + return cur; + } + + static Pos mysqlHour24(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.hour); + date.is_hour_of_half_day = false; + date.is_clock_hour = false; + return cur; + } + + static Pos readNumberWithVariableLength( + Pos cur, + Pos end, + bool allow_negative, + bool allow_plus_sign, + bool is_year, + int repetitions, + int max_digits_consume, + Int32 & number) + { + bool negative = false; + if (allow_negative && cur < end && *cur == '-') + { + negative = true; + ++cur; + } + else if (allow_plus_sign && cur < end && *cur == '+') + { + negative = false; + ++cur; + } + + number = 0; + Pos start = cur; + if (is_year && repetitions == 2) + { + // If abbreviated two year digit is provided in format string, try to read + // in two digits of year and convert to appropriate full length year The + // two-digit mapping is as follows: [00, 69] -> [2000, 2069] + // [70, 99] -> [1970, 1999] + // If more than two digits are provided, then simply read in full year + // normally without conversion + int count = 0; + while (cur < end && cur < start + max_digits_consume && *cur >= '0' && *cur <= '9') + { + number = number * 10 + (*cur - '0'); + ++cur; + ++count; + } + if (count == 2) + { + if (number >= 70) + number += 1900; + else if (number >= 0 && number < 70) + number += 2000; + } + else + { + while (cur < end && cur < start + max_digits_consume && *cur >= '0' && *cur <= '9') + { + number = number * 10 + (*cur - '0'); + ++cur; + } + } + } + else + { + while (cur < end && cur < start + max_digits_consume && *cur >= '0' and *cur <= '9') + { + number = number * 10 + (*cur - '0'); + ++cur; + } + } + + /// Need to have read at least one digit. + if (cur <= start) + throw Exception(ErrorCodes::LOGICAL_ERROR, "read number from {} failed", String(cur, end - cur)); + + if (negative) + number *= -1; + + return cur; + } + + static Pos jodaEra(int, Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 2, "jodaEra requires size >= 2"); + if (std::strncmp(cur, "AD", 2) == 0 || std::strncmp(cur, "ad", 2) == 0) + date.is_ad = true; + else if (std::strncmp(cur, "BC", 2) == 0 || std::strncmp(cur, "bc", 2) == 0) + date.is_ad = false; + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown era {}", std::string(cur, 2)); + + cur += 2; + return cur; + } + + static Pos jodaCenturyOfEra(int repetitions, Pos cur, Pos end, Date & date) + { + Int32 number; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, repetitions, number); + + if (number < 0 || number > 2922789) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for century of era must be in the range [0, 2922789]", number); + + date.century_format = true; + date.year = 100 * number; + date.has_year = true; + return cur; + } + + static Pos jodaYearOfEra(int repetitions, Pos cur, Pos end, Date & date) + { + Int32 number; + cur = readNumberWithVariableLength(cur, end, false, false, true, repetitions, repetitions, number); + + date.century_format = false; + date.is_year_of_era = true; + if (number > 292278993 || number < 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for year of era must be in the range [1, 292278993]", number); + + date.has_year = true; + date.year = number; + return cur; + } + + static Pos jodaWeekYear(int repetitions, Pos cur, Pos end, Date & date) + { + Int32 number; + cur = readNumberWithVariableLength(cur, end, true, true, true, repetitions, repetitions, number); + if (number < -292275054 || number > 292278993) + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Value {} for week year must be in the range [-292275054,292278993]", number); + + date.year = number; + date.week_date_format = true; + date.day_of_year_format = false; + date.century_format = false; + date.has_year = true; + return cur; + } + + static Pos jodaWeekOfWeekYear(int repetitions, Pos cur, Pos end, Date & date) + { + Int32 number; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), number); + if (number < 1 || number > 52) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for week of week year must be in the range [1, 52]", number); + + date.week = number; + date.week_date_format = true; + date.day_of_year_format = false; + if (!date.has_year) + { + date.has_year = true; + date.year = 2000; + } + return cur; + } + + static Pos jodaDayOfWeek1Based(int repetitions, Pos cur, Pos end, Date & date) + { + Int32 number; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, repetitions, number); + if (number < 1 || number > 7) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for day of week 1-based must be in the range [1, 7]", number); + + date.day_of_week = number; + date.week_date_format = true; + date.day_of_year_format = false; + if (!date.has_year) + { + date.has_year = true; + date.year = 2000; + } + return cur; + } + + static Pos jodaDayOfWeekText(size_t /*min_represent_digits*/, Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 3, "jodaDayOfWeekText requires size >= 3"); + + String text1(cur, 3); + Poco::toLowerInPlace(text1); + auto it = dayOfWeekMap.find(text1); + if (it == dayOfWeekMap.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown day of week text: {}", text1); + + date.day_of_week = it->second.second; + cur += 3; + if (cur + it->second.first.size() <= end) + { + String text2(cur, it->second.first.size()); + Poco::toLowerInPlace(text2); + if (text2 == it->second.first) + { + cur += it->second.first.size(); + return cur; + } + } + return cur; + } + + static Pos jodaYear(int repetitions, Pos cur, Pos end, Date & date) + { + Int32 number; + cur = readNumberWithVariableLength(cur, end, true, true, true, repetitions, repetitions, number); + + date.century_format = false; + date.is_year_of_era = false; + if (number > 292278994 || number < -292275055) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for year must be in the range [-292275055,292278994]", number); + + date.has_year = true; + date.year = number; + return cur; + } + + static Pos jodaDayOfYear(int repetitions, Pos cur, Pos end, Date & date) + { + Int32 number; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 3), number); + + date.day_of_year_values.push_back(number); + date.day_of_year = true; + date.day_of_year_format = true; + date.week_date_format = false; + if (date.has_year) + { + date.has_year = true; + date.year = 2000; + } + return cur; + } + + static Pos jodaMonthOfYear(int repetitions, Pos cur, Pos end, Date & date) + { + Int32 number; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, 2, number); + if (number < 1 || number > 12) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for month of year must be in the range [1, 12]", number); + + date.month = number; + date.week_date_format = false; + date.day_of_year_format = false; + if (!date.has_year) + { + date.has_year = true; + date.year = 2000; + } + return cur; + } + + static Pos jodaMonthOfYearText(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 3, "jodaMonthOfYearText requires size >= 3"); + + String text1(cur, 3); + Poco::toLowerInPlace(text1); + auto it = monthMap.find(text1); + if (it == monthMap.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown month of year text: {}", text1); + + date.month = it->second.second; + cur += 3; + if (cur + it->second.first.size() <= end) + { + String text2(cur, it->second.first.size()); + Poco::toLowerInPlace(text2); + if (text2 == it->second.first) + { + cur += it->second.first.size(); + return cur; + } + } + return cur; + } + + static Pos jodaDayOfMonth(int repetitions, Pos cur, Pos end, Date & date) + { + Int32 number; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), number); + if (number < 1 || number > 12) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for day of month must be in the range [1, 12]", number); + + date.day_of_month_values.push_back(number); + date.day = number; + date.week_date_format = false; + date.day_of_year_format = false; + if (!date.has_year) + { + date.has_year = true; + date.year = 2000; + } + return cur; + } + + static Pos jodaHalfDayOfDay(Pos cur, Pos end, Date & date) + { + ensureSpace(cur, end, 2, "jodaHalfDayOfDay requires size >= 2"); + + String text(cur, 2); + Poco::toLowerInPlace(cur); + if (text == "am") + date.is_am = true; + else if (text == "pm") + date.is_am = false; + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown half day of day: {}", text); + + cur += 2; + return cur; + } + + static Pos jodaHourOfHalfDay(int repetitions, Pos cur, Pos end, Date & date) + { + Int32 number; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), number); + if (number > 11 || number < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for hour of half day must be in the range [0, 11]", number); + + date.is_clock_hour = false; + date.is_hour_of_half_day = true; + date.hour = number; + return cur; + } + + static Pos jodaClockHourOfHalfDay(int repetitions, Pos cur, Pos end, Date & date) + { + Int32 number; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), number); + if (number > 12 || number < 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for clock hour of half day must be in the range [1, 12]", number); + + date.is_clock_hour = true; + date.is_hour_of_half_day = true; + date.hour = number; + return cur; + } + + static Pos jodaHourOfDay(int repetitions, Pos cur, Pos end, Date & date) + { + Int32 number; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), number); + if (number > 23 || number < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for hour of day must be in the range [0, 23]", number); + + date.is_clock_hour = false; + date.is_hour_of_half_day = false; + date.hour = number; + return cur; + } + + static Pos jodaClockHourOfDay(int repetitions, Pos cur, Pos end, Date & date) + { + Int32 number; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), number); + if (number > 24 || number < 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for clock hour of day must be in the range [1, 24]", number); + + date.is_clock_hour = true; + date.is_hour_of_half_day = false; + date.hour = number % 24; + return cur; + } + + static Pos jodaMinuteOfHour(int repetitions, Pos cur, Pos end, Date & date) + { + Int32 number; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), number); + if (number > 59 || number < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for minute of hour must be in the range [0, 59]", number); + + date.minute = number; + return cur; + } + + static Pos jodaSecondOfMinute(int repetitions, Pos cur, Pos end, Date & date) + { + Int32 number; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), number); + if (number > 59 || number < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for second of minute must be in the range [0, 59]", number); + + date.second = number; + return cur; + } + }; + + + ALWAYS_INLINE void parseFormat(const String & format, std::vector & instructions) const + { + if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::MySQL) + parseMysqlFormat(format, instructions); + else if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::Joda) + parseJodaFormat(format, instructions); else - return "%Y-%m-%d %H:%M:%S"; + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, + "Unknown datetime format style {} in function {}", + magic_enum::enum_name(parse_syntax), + getName()); } - const auto * format_column = checkAndGetColumnConst(arguments[1].column.get()); - if (!format_column) - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of second ('format') argument of function {}. Must be constant string.", - arguments[1].column->getName(), - getName()); - return format_column->getValue(); - } + ALWAYS_INLINE void parseMysqlFormat(const String & format, std::vector & instructions) const + { +#define ACTION_ARGS(func) &(func), #func - ALWAYS_INLINE std::pair getTimeZone(const ColumnsWithTypeAndName & arguments) const + Pos pos = format.data(); + Pos end = pos + format.size(); + while (true) + { + Pos percent_pos = find_first_symbols<'%'>(pos, end); + if (percent_pos < end) + { + if (pos < percent_pos) + instructions.emplace_back(String(pos, percent_pos - pos)); + + pos = percent_pos + 1; + if (pos >= end) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Sign '%' is the last in format, if you need it, use '%%'"); + + switch (*pos) + { + // Abbreviated weekday [Mon...Sun] + case 'a': + instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfWeekTextShort)); + break; + + // Abbreviated month [Jan...Dec] + case 'b': + instructions.emplace_back(ACTION_ARGS(Action::mysqlMonthOfYearTextShort)); + break; + + // Month as a decimal number (01-12) + case 'c': + instructions.emplace_back(ACTION_ARGS(Action::mysqlMonth)); + break; + + // Year, divided by 100, zero-padded + case 'C': + instructions.emplace_back(ACTION_ARGS(Action::mysqlCentury)); + break; + + // Day of month, zero-padded (01-31) + case 'd': + instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfMonth)); + break; + + // Short MM/DD/YY date, equivalent to %m/%d/%y + case 'D': + instructions.emplace_back(ACTION_ARGS(Action::mysqlAmericanDate)); + break; + + // Day of month, space-padded ( 1-31) 23 + case 'e': + instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfMonthSpacePadded)); + break; + + // Fractional seconds + case 'f': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for fractional seconds"); + + // Short YYYY-MM-DD date, equivalent to %Y-%m-%d 2001-08-23 + case 'F': + instructions.emplace_back(ACTION_ARGS(Action::mysqlISO8601Date)); + break; + + // Last two digits of year of ISO 8601 week number (see %G) + case 'g': + instructions.emplace_back(ACTION_ARGS(Action::mysqlISO8601Year2)); + break; + + // Year of ISO 8601 week number (see %V) + case 'G': + instructions.emplace_back(ACTION_ARGS(Action::mysqlISO8601Year4)); + break; + + // Day of the year (001-366) 235 + case 'j': + instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfYear)); + break; + + // Month as a decimal number (01-12) + case 'm': + instructions.emplace_back(ACTION_ARGS(Action::mysqlMonth)); + break; + + // ISO 8601 weekday as number with Monday as 1 (1-7) + case 'u': + instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfWeek)); + break; + + // ISO 8601 week number (01-53) + case 'V': + instructions.emplace_back(ACTION_ARGS(Action::mysqlISO8601Week)); + break; + + // Weekday as a decimal number with Sunday as 0 (0-6) 4 + case 'w': + instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfWeek0To6)); + break; + + // Full weekday [Monday...Sunday] + case 'W': + instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfWeekTextLong)); + break; + + // Two digits year + case 'y': + instructions.emplace_back(ACTION_ARGS(Action::mysqlYear2)); + break; + + // Four digits year + case 'Y': + instructions.emplace_back(ACTION_ARGS(Action::mysqlYear4)); + break; + + // Quarter (1-4) + case 'Q': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for quarter"); + break; + + // Offset from UTC timezone as +hhmm or -hhmm + case 'z': + instructions.emplace_back(ACTION_ARGS(Action::mysqlTimezoneOffset)); + break; + + /// Time components. If the argument is Date, not a DateTime, then this components will have default value. + + // Minute (00-59) + case 'M': + instructions.emplace_back(ACTION_ARGS(Action::mysqlMinute)); + break; + + // AM or PM + case 'p': + instructions.emplace_back(ACTION_ARGS(Action::mysqlAMPM)); + break; + + // 12-hour HH:MM time, equivalent to %h:%i %p 2:55 PM + case 'r': + instructions.emplace_back(ACTION_ARGS(Action::mysqlHHMM12)); + break; + + // 24-hour HH:MM time, equivalent to %H:%i 14:55 + case 'R': + instructions.emplace_back(ACTION_ARGS(Action::mysqlHHMM24)); + break; + + // Seconds + case 's': + instructions.emplace_back(ACTION_ARGS(Action::mysqlSecond)); + break; + + // Seconds + case 'S': + instructions.emplace_back(ACTION_ARGS(Action::mysqlSecond)); + break; + + // ISO 8601 time format (HH:MM:SS), equivalent to %H:%i:%S 14:55:02 + case 'T': + instructions.emplace_back(ACTION_ARGS(Action::mysqlISO8601Time)); + break; + + // Hour in 12h format (01-12) + case 'h': + instructions.emplace_back(ACTION_ARGS(Action::mysqlHour12)); + break; + + // Hour in 24h format (00-23) + case 'H': + instructions.emplace_back(ACTION_ARGS(Action::mysqlHour24)); + break; + + // Minute of hour range [0, 59] + case 'i': + instructions.emplace_back(ACTION_ARGS(Action::mysqlMinute)); + break; + + // Hour in 12h format (01-12) + case 'I': + instructions.emplace_back(ACTION_ARGS(Action::mysqlHour12)); + break; + + // Hour in 24h format (00-23) + case 'k': + instructions.emplace_back(ACTION_ARGS(Action::mysqlHour24)); + break; + + // Hour in 12h format (01-12) + case 'l': + instructions.emplace_back(ACTION_ARGS(Action::mysqlHour12)); + break; + + case 't': + instructions.emplace_back("\t"); + break; + + case 'n': + instructions.emplace_back("\n"); + break; + + // Escaped literal characters. + case '%': + instructions.emplace_back("\n"); + break; + + // Unimplemented + case 'U': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for WEEK (Sun-Sat)"); + case 'v': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for WEEK (Mon-Sun)"); + case 'x': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for YEAR for week (Mon-Sun)"); + case 'X': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for YEAR for week (Sun-Sat)"); + + default: + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Incorrect syntax '{}', symbol is not supported '{}' for function {}", + format, + *pos, + getName()); + } + + ++pos; + } + else + { + if (pos < end) + instructions.emplace_back(String(pos, end - pos)); + break; + } + } +#undef ACTION_ARGS + } + + void parseJodaFormat(const String & format, std::vector & instructions) const + { +#define ACTION_ARGS_WITH_BIND(func, arg) std::bind_front(&(func), (arg)), #func + + // size_t reserve_size = 0; + const char * pos = format.data(); + const char * end = pos + format.size(); + + while (pos < end) + { + const char * cur_token = pos; + + // Literal case + if (*cur_token == '\'') + { + // Case 1: 2 consecutive single quote + if (pos + 1 < end && *(pos + 1) == '\'') + { + instructions.emplace_back(String(cur_token, 1)); + // ++reserve_size; + pos += 2; + } + else + { + // Case 2: find closing single quote + Int64 count = numLiteralChars(cur_token + 1, end); + if (count == -1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No closing single quote for literal"); + else + { + for (Int64 i = 1; i <= count; i++) + { + instructions.emplace_back(String(cur_token + i, 1)); + // ++reserve_size; + if (*(cur_token + i) == '\'') + i += 1; + } + pos += count + 2; + } + } + } + else + { + int repetitions = 1; + ++pos; + while (pos < end && *cur_token == *pos) + { + ++repetitions; + ++pos; + } + switch (*cur_token) + { + case 'G': + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaEra, repetitions)); + // reserve_size += repetitions <= 3 ? 2 : 13; + break; + /* + case 'C': + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaCenturyOfEra, repetitions)); + /// Year range [1900, 2299] + // reserve_size += std::max(repetitions, 2); + break; + case 'Y': + + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaYearOfEra, repetitions)); + /// Year range [1900, 2299] + // reserve_size += repetitions == 2 ? 2 : std::max(repetitions, 4); + break; + case 'x': + + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaWeekYear, repetitions)); + /// weekyear range [1900, 2299] + // reserve_size += std::max(repetitions, 4); + break; + case 'w': + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaWeekOfWeekYear, repetitions)); + /// Week of weekyear range [1, 52] + // reserve_size += std::max(repetitions, 2); + break; + case 'e': + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaDayOfWeek1Based, repetitions)); + /// Day of week range [1, 7] + // reserve_size += std::max(repetitions, 1); + break; + case 'E': + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaDayOfWeekText, repetitions)); + /// Maximum length of short name is 3, maximum length of full name is 9. + // reserve_size += repetitions <= 3 ? 3 : 9; + break; + case 'y': + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaYear, repetitions)); + /// Year range [1900, 2299] + // reserve_size += repetitions == 2 ? 2 : std::max(repetitions, 4); + break; + case 'D': + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaDayOfYear, repetitions)); + /// Day of year range [1, 366] + // reserve_size += std::max(repetitions, 3); + break; + case 'M': + if (repetitions <= 2) + { + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaMonthOfYear, repetitions)); + /// Month of year range [1, 12] + // reserve_size += 2; + } + else + { + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaMonthOfYearText, repetitions)); + /// Maximum length of short name is 3, maximum length of full name is 9. + // reserve_size += repetitions <= 3 ? 3 : 9; + } + break; + case 'd': + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaDayOfMonth, repetitions)); + /// Day of month range [1, 3] + // reserve_size += std::max(repetitions, 3); + break; + case 'a': + /// Default half day of day is "AM" + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaHalfDayOfDay, repetitions)); + // reserve_size += 2; + break; + case 'K': + /// Default hour of half day is 0 + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaHourOfHalfDay, repetitions)); + /// Hour of half day range [0, 11] + // reserve_size += std::max(repetitions, 2); + break; + case 'h': + /// Default clock hour of half day is 12 + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaClockHourOfHalfDay, repetitions)); + /// Clock hour of half day range [1, 12] + // reserve_size += std::max(repetitions, 2); + break; + case 'H': + /// Default hour of day is 0 + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaHourOfDay, repetitions)); + /// Hour of day range [0, 23] + // reserve_size += std::max(repetitions, 2); + break; + case 'k': + /// Default clock hour of day is 24 + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaClockHourOfDay, repetitions)); + /// Clock hour of day range [1, 24] + // reserve_size += std::max(repetitions, 2); + break; + case 'm': + /// Default minute of hour is 0 + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaMinuteOfHour, repetitions)); + /// Minute of hour range [0, 59] + // reserve_size += std::max(repetitions, 2); + break; + case 's': + /// Default second of minute is 0 + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaSecondOfMinute, repetitions)); + /// Second of minute range [0, 59] + // reserve_size += std::max(repetitions, 2); + break; + case 'S': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for fractional seconds"); + break; + case 'z': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for timezone"); + break; + case 'Z': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for timezone offset id"); + */ + default: + if (isalpha(*cur_token)) + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, "format is not supported for {}", String(cur_token, repetitions)); + + instructions.emplace_back(String(cur_token, pos - cur_token)); + // reserve_size += pos - cur_token; + break; + } + } + } + } + + + ALWAYS_INLINE String getFormat(const ColumnsWithTypeAndName & arguments) const + { + if (arguments.size() < 2) + { + if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::Joda) + return "yyyy-MM-dd HH:mm:ss"; + else + return "%Y-%m-%d %H:%M:%S"; + } + + const auto * format_column = checkAndGetColumnConst(arguments[1].column.get()); + if (!format_column) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of second ('format') argument of function {}. Must be constant string.", + arguments[1].column->getName(), + getName()); + return format_column->getValue(); + } + + ALWAYS_INLINE std::pair getTimeZone(const ColumnsWithTypeAndName & arguments) const + { + if (arguments.size() < 3) + return {&DateLUT::instance(), ""}; + + const auto * col = checkAndGetColumnConst(arguments[2].column.get()); + if (!col) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of third ('timezone') argument of function {}. Must be constant string.", + arguments[2].column->getName(), + getName()); + + String time_zone = col->getValue(); + if (time_zone.empty()) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Provided time zone must be non-empty and be a valid time zone"); + return {&DateLUT::instance(time_zone), time_zone}; + } + }; + + struct NameParseDateTime { - if (arguments.size() < 3) - return {&DateLUT::instance(), ""}; + static constexpr auto name = "parseDateTime"; + }; - const auto * col = checkAndGetColumnConst(arguments[2].column.get()); - if (!col) - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of third ('timezone') argument of function {}. Must be constant string.", - arguments[2].column->getName(), - getName()); + struct NameParseDateTimeInJodaSyntax + { + static constexpr auto name = "parseDateTimeInJodaSyntax"; + }; - String time_zone = col->getValue(); - if (time_zone.empty()) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Provided time zone must be non-empty and be a valid time zone"); - return {&DateLUT::instance(time_zone), time_zone}; - } -}; -struct NameParseDateTime -{ - static constexpr auto name = "parseDateTime"; -}; - -using FunctionParseDateTime = FunctionParseDateTimeImpl; + using FunctionParseDateTime = FunctionParseDateTimeImpl; + using FunctionParseDateTimeInJodaSyntax + = FunctionParseDateTimeImpl; } REGISTER_FUNCTION(ParseDateTime) { factory.registerFunction(); factory.registerAlias("TO_UNIXTIME", "parseDateTime"); + + factory.registerFunction(); } From 040067748387c1565e5e73399e61b31d85fa3104 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 27 Feb 2023 15:41:50 +0000 Subject: [PATCH 076/559] makeMaterializingDAG --- .../QueryPlan/Optimizations/optimizeUseProjections.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index c400bba4339..d6515f67dbf 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -951,10 +951,14 @@ static ActionsDAGPtr makeMaterializingDAG(const Block & proj_header, const Block if (const_positions.empty()) return nullptr; - ActionsDAGPtr dag = std::make_unique(proj_header.getColumnsWithTypeAndName()); + ActionsDAGPtr dag = std::make_unique(); + auto & outputs = dag->getOutputs(); + for (const auto & col : proj_header.getColumnsWithTypeAndName()) + outputs.push_back(&dag->addInput(col)); + for (auto pos : const_positions) { - auto & output = dag->getOutputs()[pos]; + auto & output = outputs[pos]; output = &dag->materializeNode(*output); } From ad88251ee785139660735884f56bd5573b013944 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 27 Feb 2023 17:42:04 +0100 Subject: [PATCH 077/559] Fix tests --- src/Common/NamedCollections/NamedCollections.cpp | 9 +++++++++ src/Common/NamedCollections/NamedCollections.h | 2 ++ src/Storages/NamedCollectionsHelpers.h | 4 ++-- src/Storages/StorageURL.h | 2 +- src/TableFunctions/TableFunctionRemote.cpp | 6 ++++-- 5 files changed, 18 insertions(+), 5 deletions(-) diff --git a/src/Common/NamedCollections/NamedCollections.cpp b/src/Common/NamedCollections/NamedCollections.cpp index 533481f792a..0a0f29a8a82 100644 --- a/src/Common/NamedCollections/NamedCollections.cpp +++ b/src/Common/NamedCollections/NamedCollections.cpp @@ -353,6 +353,15 @@ bool NamedCollection::has(const Key & key) const return pimpl->has(key); } +bool NamedCollection::hasAny(const std::initializer_list & keys) const +{ + std::lock_guard lock(mutex); + for (const auto & key : keys) + if (pimpl->has(key)) + return true; + return false; +} + template T NamedCollection::get(const Key & key) const { std::lock_guard lock(mutex); diff --git a/src/Common/NamedCollections/NamedCollections.h b/src/Common/NamedCollections/NamedCollections.h index b82d5eb3152..4a0f020db21 100644 --- a/src/Common/NamedCollections/NamedCollections.h +++ b/src/Common/NamedCollections/NamedCollections.h @@ -35,6 +35,8 @@ public: bool has(const Key & key) const; + bool hasAny(const std::initializer_list & keys) const; + template T get(const Key & key) const; template T getOrDefault(const Key & key, const T & default_value) const; diff --git a/src/Storages/NamedCollectionsHelpers.h b/src/Storages/NamedCollectionsHelpers.h index 085e21937ee..a2aed38ed08 100644 --- a/src/Storages/NamedCollectionsHelpers.h +++ b/src/Storages/NamedCollectionsHelpers.h @@ -27,8 +27,8 @@ HTTPHeaderEntries getHeadersFromNamedCollection(const NamedCollection & collecti struct ExternalDatabaseEqualKeysSet { - static constexpr std::array, 3> equal_keys{ - std::pair{"username", "user"}, std::pair{"database", "db"}, std::pair{"hostname", "host"}}; + static constexpr std::array, 5> equal_keys{ + std::pair{"username", "user"}, std::pair{"database", "db"}, std::pair{"hostname", "host"}, std::pair{"addresses_expr", "host"}, std::pair{"addresses_expr", "hostname"}}; }; struct MongoDBEqualKeysSet { diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 24b1c7ee572..c95cfa69e54 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -184,7 +184,7 @@ public: struct Configuration : public StatelessTableEngineConfiguration { std::string url; - std::string http_method = "auto"; + std::string http_method; HTTPHeaderEntries headers; std::string addresses_expr; }; diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index f6c773b0b97..1e093e957a7 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -57,10 +57,12 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr validateNamedCollection>( *named_collection, - {"addresses_expr", "database", "db", "table"}, - {"username", "user", "password", "sharding_key"}); + {"addresses_expr", "host", "database", "db", "table"}, + {"username", "user", "password", "sharding_key", "port"}); cluster_description = named_collection->getOrDefault("addresses_expr", ""); + if (cluster_description.empty() && named_collection->hasAny({"host", "hostname"})) + cluster_description = named_collection->has("port") ? named_collection->getAny({"host", "hostname"}) + ':' + toString(named_collection->get("port")) : named_collection->getAny({"host", "hostname"}); database = named_collection->getAnyOrDefault({"db", "database"}, "default"); table = named_collection->get("table"); username = named_collection->getAnyOrDefault({"username", "user"}, "default"); From 9bf828cc98c490f3d405628fbaedd6641c85c737 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 27 Feb 2023 16:43:54 +0000 Subject: [PATCH 078/559] Refactor a bit. --- .../Optimizations/optimizeUseProjections.cpp | 165 ++++++++---------- 1 file changed, 73 insertions(+), 92 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index d6515f67dbf..a444538c8ce 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -735,6 +735,68 @@ static std::shared_ptr getMaxAddedBlocks(ReadFromMergeTre return {}; } +static bool analyzeProjectionCandidate( + ProjectionCandidate & candidate, + const ReadFromMergeTree & reading, + const MergeTreeDataSelectExecutor & reader, + const Names & required_column_names, + const MergeTreeData::DataPartsVector & parts, + const StorageMetadataPtr & metadata, + const SelectQueryInfo & query_info, + const ContextPtr & context, + const std::shared_ptr & max_added_blocks, + const ActionDAGNodes & added_filter_nodes) +{ + MergeTreeData::DataPartsVector projection_parts; + MergeTreeData::DataPartsVector normal_parts; + for (const auto & part : parts) + { + const auto & created_projections = part->getProjectionParts(); + auto it = created_projections.find(candidate.projection->name); + if (it != created_projections.end()) + projection_parts.push_back(it->second); + else + normal_parts.push_back(part); + } + + if (projection_parts.empty()) + return false; + + auto projection_result_ptr = reader.estimateNumMarksToRead( + std::move(projection_parts), + nullptr, + required_column_names, + metadata, + candidate.projection->metadata, + query_info, /// How it is actually used? I hope that for index we need only added_filter_nodes + added_filter_nodes, + context, + context->getSettingsRef().max_threads, + max_added_blocks); + + if (projection_result_ptr->error()) + return false; + + candidate.merge_tree_projection_select_result_ptr = std::move(projection_result_ptr); + candidate.sum_marks += candidate.merge_tree_projection_select_result_ptr->marks(); + + if (!normal_parts.empty()) + { + auto normal_result_ptr = reading.selectRangesToRead(std::move(normal_parts)); + + if (normal_result_ptr->error()) + return false; + + if (normal_result_ptr->marks() != 0) + { + candidate.sum_marks += normal_result_ptr->marks(); + candidate.merge_tree_normal_select_result_ptr = std::move(normal_result_ptr); + } + } + + return true; +} + bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { if (node.children.size() != 1) @@ -777,57 +839,18 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) /// Selecting best candidate. for (auto & candidate : candidates.real) { - MergeTreeData::DataPartsVector projection_parts; - MergeTreeData::DataPartsVector normal_parts; - for (const auto & part : parts) - { - const auto & created_projections = part->getProjectionParts(); - auto it = created_projections.find(candidate.projection->name); - if (it != created_projections.end()) - projection_parts.push_back(it->second); - else - normal_parts.push_back(part); - } - - if (projection_parts.empty()) - continue; - + auto required_column_names = candidate.dag->getRequiredColumnsNames(); ActionDAGNodes added_filter_nodes; if (candidates.has_filter) added_filter_nodes.nodes.push_back(candidate.dag->getOutputs().front()); - auto projection_result_ptr = reader.estimateNumMarksToRead( - std::move(projection_parts), - nullptr, - candidate.dag->getRequiredColumnsNames(), - metadata, - candidate.projection->metadata, - query_info, /// How it is actually used? I hope that for index we need only added_filter_nodes - added_filter_nodes, - context, - context->getSettingsRef().max_threads, - max_added_blocks); + bool analyzed = analyzeProjectionCandidate( + candidate, *reading, reader, required_column_names, parts, + metadata, query_info, context, max_added_blocks, added_filter_nodes); - if (projection_result_ptr->error()) + if (!analyzed) continue; - candidate.merge_tree_projection_select_result_ptr = std::move(projection_result_ptr); - candidate.sum_marks += candidate.merge_tree_projection_select_result_ptr->marks(); - - if (!normal_parts.empty()) - { - auto normal_result_ptr = reading->selectRangesToRead(std::move(normal_parts)); - - if (normal_result_ptr->error()) - continue; - - if (normal_result_ptr->marks() != 0) - { - candidate.sum_marks += normal_result_ptr->marks(); - candidate.merge_tree_normal_select_result_ptr = std::move(normal_result_ptr); - } - } - if (best_candidate == nullptr || best_candidate->sum_marks > candidate.sum_marks) best_candidate = &candidate; } @@ -858,9 +881,6 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); proj_snapshot->addProjection(best_candidate->projection); - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}", - // proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString()); - auto query_info_copy = query_info; query_info_copy.prewhere_info = nullptr; @@ -1043,59 +1063,20 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) if (!hasAllRequiredColumns(projection, required_columns)) continue; - MergeTreeData::DataPartsVector projection_parts; - MergeTreeData::DataPartsVector normal_parts; - for (const auto & part : parts) - { - const auto & created_projections = part->getProjectionParts(); - auto it = created_projections.find(projection->name); - if (it != created_projections.end()) - projection_parts.push_back(it->second); - else - normal_parts.push_back(part); - } - - if (projection_parts.empty()) - continue; + auto & candidate = candidates.emplace_back(); + candidate.projection = projection; ActionDAGNodes added_filter_nodes; if (query.filter_node) added_filter_nodes.nodes.push_back(query.filter_node); - auto projection_result_ptr = reader.estimateNumMarksToRead( - std::move(projection_parts), - nullptr, - required_columns, - metadata, - projection->metadata, - query_info, /// How it is actually used? I hope that for index we need only added_filter_nodes - added_filter_nodes, - context, - context->getSettingsRef().max_threads, - max_added_blocks); + bool analyzed = analyzeProjectionCandidate( + candidate, *reading, reader, required_columns, parts, + metadata, query_info, context, max_added_blocks, added_filter_nodes); - if (projection_result_ptr->error()) + if (!analyzed) continue; - auto & candidate = candidates.emplace_back(); - candidate.projection = projection; - candidate.merge_tree_projection_select_result_ptr = std::move(projection_result_ptr); - candidate.sum_marks += candidate.merge_tree_projection_select_result_ptr->marks(); - - if (!normal_parts.empty()) - { - auto normal_result_ptr = reading->selectRangesToRead(std::move(normal_parts)); - - if (normal_result_ptr->error()) - continue; - - if (normal_result_ptr->marks() != 0) - { - candidate.sum_marks += normal_result_ptr->marks(); - candidate.merge_tree_normal_select_result_ptr = std::move(normal_result_ptr); - } - } - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), // "Marks for projection {} {}", projection->name ,candidate.sum_marks); From 1e0ea2446e63ef526316632c4384265feac3021b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 27 Feb 2023 16:59:32 +0000 Subject: [PATCH 079/559] Refactor a bit. --- .../Optimizations/optimizeUseProjections.cpp | 267 +++++++++--------- 1 file changed, 133 insertions(+), 134 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp index a444538c8ce..2694285c2c9 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp @@ -22,21 +22,6 @@ namespace DB::QueryPlanOptimizations { -static QueryPlan::Node * findReadingStep(QueryPlan::Node & node) -{ - IQueryPlanStep * step = node.step.get(); - if (auto * reading = typeid_cast(step)) - return &node; - - if (node.children.size() != 1) - return nullptr; - - if (typeid_cast(step) || typeid_cast(step)) - return findReadingStep(*node.children.front()); - - return nullptr; -} - /// This is a common DAG which is a merge of DAGs from Filter and Expression steps chain. /// Additionally, for all the Filter steps, we collect filter conditions into filter_nodes. /// Flag remove_last_filter_node is set in case if the last step is a Filter step and it should remove filter column. @@ -124,28 +109,109 @@ bool QueryDAG::build(QueryPlan::Node & node) return false; } -static bool canUseProjectionForReadingStep(ReadFromMergeTree * reading) +struct AggregateQueryDAG { - /// Probably some projection already was applied. - if (reading->hasAnalyzedResult()) - return false; + ActionsDAGPtr dag; + const ActionsDAG::Node * filter_node = nullptr; - if (reading->isQueryWithFinal()) - return false; + bool build(QueryPlan::Node & node) + { + QueryDAG query; + if (!query.build(node)) + return false; - if (reading->isQueryWithSampling()) - return false; + dag = std::move(query.dag); + auto filter_nodes = std::move(query.filter_nodes); - if (reading->isParallelReadingEnabled()) - return false; + if (!filter_nodes.empty()) + { + filter_node = filter_nodes.front(); + if (filter_nodes.size() > 1) + { + FunctionOverloadResolverPtr func_builder_and = + std::make_unique( + std::make_shared()); - // Currently projection don't support deduplication when moving parts between shards. - if (reading->getContext()->getSettingsRef().allow_experimental_query_deduplication) - return false; + filter_node = &dag->addFunction(func_builder_and, std::move(filter_nodes), {}); + } - return true; -} + dag->getOutputs().push_back(filter_node); + } + return true; + } +}; + +struct NormalQueryDAG +{ + ActionsDAGPtr dag; + bool need_remove_column = false; + const ActionsDAG::Node * filter_node = nullptr; + + bool build(QueryPlan::Node & node) + { + QueryDAG query; + if (!query.build(node)) + return false; + + dag = std::move(query.dag); + auto filter_nodes = std::move(query.filter_nodes); + need_remove_column = query.remove_last_filter_node; + + if (!filter_nodes.empty()) + { + auto & outputs = dag->getOutputs(); + filter_node = filter_nodes.back(); + + if (filter_nodes.size() > 1) + { + /// Add a conjunction of all the filters. + if (need_remove_column) + { + /// Last filter column is not needed; remove it right here + size_t pos = 0; + while (pos < outputs.size() && outputs[pos] != filter_node) + ++pos; + + if (pos < outputs.size()) + outputs.erase(outputs.begin() + pos); + } + else + { + /// Last filter is needed; we must replace it to constant 1, + /// As well as FilterStep does to make a compatible header. + for (auto & output : outputs) + { + if (output == filter_node) + { + ColumnWithTypeAndName col; + col.name = filter_node->result_name; + col.type = filter_node->result_type; + col.column = col.type->createColumnConst(1, 1); + output = &dag->addColumn(std::move(col)); + } + } + } + + FunctionOverloadResolverPtr func_builder_and = + std::make_unique( + std::make_shared()); + + filter_node = &dag->addFunction(func_builder_and, std::move(filter_nodes), {}); + outputs.insert(outputs.begin(), filter_node); + need_remove_column = true; + } + } + + if (dag) + { + dag->removeUnusedActions(); + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Header {}, Query DAG: {}", header.dumpStructure(), dag->dumpDAG()); + } + + return true; + } +}; /// Required analysis info from aggregate projection. struct AggregateProjectionInfo @@ -380,110 +446,6 @@ bool areAggregatesMatch( return true; } -struct AggregateQueryDAG -{ - ActionsDAGPtr dag; - const ActionsDAG::Node * filter_node = nullptr; - - bool build(QueryPlan::Node & node) - { - QueryDAG query; - if (!query.build(node)) - return false; - - dag = std::move(query.dag); - auto filter_nodes = std::move(query.filter_nodes); - - if (!filter_nodes.empty()) - { - filter_node = filter_nodes.front(); - if (filter_nodes.size() > 1) - { - FunctionOverloadResolverPtr func_builder_and = - std::make_unique( - std::make_shared()); - - filter_node = &dag->addFunction(func_builder_and, std::move(filter_nodes), {}); - } - - dag->getOutputs().push_back(filter_node); - } - - return true; - } -}; - -struct NormalQueryDAG -{ - ActionsDAGPtr dag; - bool need_remove_column = false; - const ActionsDAG::Node * filter_node = nullptr; - - bool build(QueryPlan::Node & node) - { - QueryDAG query; - if (!query.build(node)) - return false; - - dag = std::move(query.dag); - auto filter_nodes = std::move(query.filter_nodes); - need_remove_column = query.remove_last_filter_node; - - if (!filter_nodes.empty()) - { - auto & outputs = dag->getOutputs(); - filter_node = filter_nodes.back(); - - if (filter_nodes.size() > 1) - { - /// Add a conjunction of all the filters. - if (need_remove_column) - { - /// Last filter column is not needed; remove it right here - size_t pos = 0; - while (pos < outputs.size() && outputs[pos] != filter_node) - ++pos; - - if (pos < outputs.size()) - outputs.erase(outputs.begin() + pos); - } - else - { - /// Last filter is needed; we must replace it to constant 1, - /// As well as FilterStep does to make a compatible header. - for (auto & output : outputs) - { - if (output == filter_node) - { - ColumnWithTypeAndName col; - col.name = filter_node->result_name; - col.type = filter_node->result_type; - col.column = col.type->createColumnConst(1, 1); - output = &dag->addColumn(std::move(col)); - } - } - } - - FunctionOverloadResolverPtr func_builder_and = - std::make_unique( - std::make_shared()); - - filter_node = &dag->addFunction(func_builder_and, std::move(filter_nodes), {}); - outputs.insert(outputs.begin(), filter_node); - need_remove_column = true; - } - } - - if (dag) - { - dag->removeUnusedActions(); - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Header {}, Query DAG: {}", header.dumpStructure(), dag->dumpDAG()); - } - - return true; - } -}; - ActionsDAGPtr analyzeAggregateProjection( const AggregateProjectionInfo & info, const AggregateQueryDAG & query, @@ -797,6 +759,43 @@ static bool analyzeProjectionCandidate( return true; } +static QueryPlan::Node * findReadingStep(QueryPlan::Node & node) +{ + IQueryPlanStep * step = node.step.get(); + if (auto * reading = typeid_cast(step)) + return &node; + + if (node.children.size() != 1) + return nullptr; + + if (typeid_cast(step) || typeid_cast(step)) + return findReadingStep(*node.children.front()); + + return nullptr; +} + +static bool canUseProjectionForReadingStep(ReadFromMergeTree * reading) +{ + /// Probably some projection already was applied. + if (reading->hasAnalyzedResult()) + return false; + + if (reading->isQueryWithFinal()) + return false; + + if (reading->isQueryWithSampling()) + return false; + + if (reading->isParallelReadingEnabled()) + return false; + + // Currently projection don't support deduplication when moving parts between shards. + if (reading->getContext()->getSettingsRef().allow_experimental_query_deduplication) + return false; + + return true; +} + bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { if (node.children.size() != 1) From 353fca74f073edc51163ad09558c6131626dae23 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 27 Feb 2023 16:33:59 +0100 Subject: [PATCH 080/559] Fix config --- tests/config/users.d/access_management.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/users.d/access_management.xml b/tests/config/users.d/access_management.xml index f7963cdb7f2..45e7c23227e 100644 --- a/tests/config/users.d/access_management.xml +++ b/tests/config/users.d/access_management.xml @@ -2,7 +2,7 @@ 1 - 1 + 1 1 From d4e6fc454631e85a43575c3fc36480f95c6ade39 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 27 Feb 2023 21:43:45 +0100 Subject: [PATCH 081/559] Fix test --- src/Storages/NamedCollectionsHelpers.h | 51 +++++++++++++++++----- src/TableFunctions/TableFunctionRemote.cpp | 14 +++--- 2 files changed, 48 insertions(+), 17 deletions(-) diff --git a/src/Storages/NamedCollectionsHelpers.h b/src/Storages/NamedCollectionsHelpers.h index a2aed38ed08..619adfc54b6 100644 --- a/src/Storages/NamedCollectionsHelpers.h +++ b/src/Storages/NamedCollectionsHelpers.h @@ -36,24 +36,40 @@ struct MongoDBEqualKeysSet std::pair{"username", "user"}, std::pair{"database", "db"}, std::pair{"hostname", "host"}, std::pair{"table", "collection"}}; }; -template struct ValidateKeysCmp +template struct NamedCollectionValidateKey { - constexpr bool operator()(const auto & lhs, const auto & rhs) const + NamedCollectionValidateKey() = default; + NamedCollectionValidateKey(const char * value_) : value(value_) {} + NamedCollectionValidateKey(std::string_view value_) : value(value_) {} + NamedCollectionValidateKey(const String & value_) : value(value_) {} + + std::string_view value; + + bool operator==(const auto & other) const { - if (lhs == rhs) + if (value == other.value) return true; for (const auto & equal : EqualKeys::equal_keys) { - if (((equal.first == lhs) && (equal.second == rhs)) || ((equal.first == rhs) && (equal.second == lhs))) + if (((equal.first == value) && (equal.second == other.value)) || ((equal.first == other.value) && (equal.second == value))) + { return true; + } } return false; } + + bool operator<(const auto & other) const + { + if (*this == other) + return false; + return value < other.value; + } }; -template using ValidateKeysMultiset = std::unordered_multiset, ValidateKeysCmp>; -using ValidateKeysSet = std::unordered_multiset>; +template using ValidateKeysMultiset = std::multiset>; +using ValidateKeysSet = std::multiset; template @@ -84,10 +100,10 @@ void validateNamedCollection( if (!match) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Unexpected key {} in named collection. Required keys: {}, optional keys: {}", - backQuoteIfNeed(key), fmt::join(required_keys, ", "), fmt::join(optional_keys, ", ")); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Unexpected key {} in named collection. Required keys: {}, optional keys: {}", + backQuoteIfNeed(key), fmt::join(required_keys, ", "), fmt::join(optional_keys, ", ")); } } @@ -101,3 +117,18 @@ void validateNamedCollection( } } + +template +struct fmt::formatter> +{ + constexpr static auto parse(format_parse_context & context) + { + return context.begin(); + } + + template + auto format(const DB::NamedCollectionValidateKey & elem, FormatContext & context) + { + return fmt::format_to(context.out(), "{}", elem.value); + } +}; diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 1e093e957a7..4952aa16efa 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -50,19 +50,19 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr * For now named collection can be used only for remote as cluster does not require credentials. */ size_t max_args = is_cluster_function ? 4 : 6; - if (auto named_collection = tryGetNamedCollectionWithOverrides(args, false)) + NamedCollectionPtr named_collection; + if (!is_cluster_function && (named_collection = tryGetNamedCollectionWithOverrides(args))) { - if (is_cluster_function) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Named collection cannot be used for table function cluster"); - validateNamedCollection>( *named_collection, - {"addresses_expr", "host", "database", "db", "table"}, - {"username", "user", "password", "sharding_key", "port"}); + {"addresses_expr", "host", "hostname", "table"}, + {"username", "user", "password", "sharding_key", "port", "database", "db"}); cluster_description = named_collection->getOrDefault("addresses_expr", ""); if (cluster_description.empty() && named_collection->hasAny({"host", "hostname"})) - cluster_description = named_collection->has("port") ? named_collection->getAny({"host", "hostname"}) + ':' + toString(named_collection->get("port")) : named_collection->getAny({"host", "hostname"}); + cluster_description = named_collection->has("port") + ? named_collection->getAny({"host", "hostname"}) + ':' + toString(named_collection->get("port")) + : named_collection->getAny({"host", "hostname"}); database = named_collection->getAnyOrDefault({"db", "database"}, "default"); table = named_collection->get("table"); username = named_collection->getAnyOrDefault({"username", "user"}, "default"); From e288c92d62a47caff1d8723c5931abe25e56fe67 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 28 Feb 2023 14:08:03 +0800 Subject: [PATCH 082/559] fix some bugs --- src/Functions/parseDateTime.cpp | 72 ++++++++--- .../0_stateless/02668_parse_datetime.sql | 118 ++++++++++++++++++ 2 files changed, 170 insertions(+), 20 deletions(-) create mode 100644 tests/queries/0_stateless/02668_parse_datetime.sql diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 9d331e4d11b..9511ab45efe 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -253,7 +253,6 @@ namespace if (is_hour_of_half_day && !is_am) hour += 12; - /// Ensure all day of year values are valid for ending year value for (const auto d : day_of_month_values) { @@ -275,9 +274,14 @@ namespace else if (day_of_year_format) days_since_epoch = daysSinceEpochFromDayOfYear(year, day_of_year); else + { days_since_epoch = daysSinceEpochFromDate(year, month, day); + std::cout << "year:" << year << "month:" << month << "day:" << day << std::endl; + } + std::cout << "days_since_epoch:" << days_since_epoch << std::endl; Int64 seconds_since_epoch = days_since_epoch * 86400 + hour * 3600 + minute * 60 + second; + std::cout << "seconds_since_epoch:" << seconds_since_epoch << std::endl; /// Time zone is not specified, use local time zone if (!time_zone_offset) @@ -287,8 +291,12 @@ namespace // std::cout << "time_zone_offset:" << *time_zone_offset << time_zone.getOffsetAtStartOfEpoch() << std::endl; // std::cout << "before timestamp:" << seconds_since_epoch << std::endl; /// Time zone is specified in format string. - seconds_since_epoch -= *time_zone_offset; - // std::cout << "after timestamp:" << seconds_since_epoch << std::endl; + if (seconds_since_epoch >= *time_zone_offset) + seconds_since_epoch -= *time_zone_offset; + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Seconds since epoch is negative"); + + std::cout << "after adjustment:" << seconds_since_epoch << std::endl; return seconds_since_epoch; } }; @@ -368,7 +376,7 @@ namespace String format = getFormat(arguments); const auto * time_zone = getTimeZone(arguments).first; - // std::cout << "timezonename:" << getTimeZone(arguments).second << std::endl; + std::cout << "timezonename:" << getTimeZone(arguments).second << std::endl; std::vector instructions; parseFormat(format, instructions); @@ -385,8 +393,8 @@ namespace for (const auto & instruction : instructions) { cur = instruction.perform(cur, end, date); - // std::cout << "instruction:" << instruction.toString() << std::endl; - // std::cout << "date:" << date.toString() << std::endl; + std::cout << "instruction:" << instruction.toString() << std::endl; + std::cout << "date:" << date.toString() << std::endl; } // Ensure all input was consumed. @@ -544,7 +552,13 @@ namespace return cur; } - static Pos mysqlMonth(Pos cur, Pos end, Date & date) { return readNumber2(cur, end, date.month); } + static Pos mysqlMonth(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.month); + if (date.month < 1 || date.month > 12) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for month must be in the range [1, 12]", date.month); + return cur; + } static Pos mysqlCentury(Pos cur, Pos end, Date & date) { @@ -750,14 +764,21 @@ namespace return cur; } - static Pos mysqlMinute(Pos cur, Pos end, Date & date) { return readNumber2(cur, end, date.minute); } + static Pos mysqlMinute(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.minute); + if (date.minute < 0 || date.minute > 59) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for minute must be in the range [0, 59]", date.minute); + + return cur; + } static Pos mysqlAMPM(Pos cur, Pos end, Date & date) { ensureSpace(cur, end, 2, "mysqlAMPM requires size >= 2"); std::string text(cur, 2); - Poco::toUpper(text); + Poco::toUpperInPlace(text); if (text == "PM") date.is_am = true; else if (text == "AM") @@ -793,7 +814,13 @@ namespace return cur; } - static Pos mysqlSecond(Pos cur, Pos end, Date & date) { return readNumber2(cur, end, date.second); } + static Pos mysqlSecond(Pos cur, Pos end, Date & date) + { + cur = readNumber2(cur, end, date.second); + if (date.second < 0 || date.second > 59) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for second must be in the range [0,59]", date.second); + return cur; + } static Pos mysqlISO8601Time(Pos cur, Pos end, Date & date) { @@ -811,6 +838,8 @@ namespace static Pos mysqlHour12(Pos cur, Pos end, Date & date) { cur = readNumber2(cur, end, date.hour); + if (date.hour < 1 || date.hour > 12) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for mysql hour12 must be in the range [1,12]", date.hour); date.is_hour_of_half_day = true; date.is_clock_hour = false; return cur; @@ -819,6 +848,9 @@ namespace static Pos mysqlHour24(Pos cur, Pos end, Date & date) { cur = readNumber2(cur, end, date.hour); + if (date.hour < 0 || date.hour > 23) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for mysql hour24 must be in the range [0,23]", date.hour); + date.is_hour_of_half_day = false; date.is_clock_hour = false; return cur; @@ -1022,12 +1054,11 @@ namespace { Int32 number; cur = readNumberWithVariableLength(cur, end, true, true, true, repetitions, repetitions, number); - - date.century_format = false; - date.is_year_of_era = false; if (number > 292278994 || number < -292275055) throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for year must be in the range [-292275055,292278994]", number); + date.century_format = false; + date.is_year_of_era = false; date.has_year = true; date.year = number; return cur; @@ -1037,6 +1068,8 @@ namespace { Int32 number; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 3), number); + if (number < 1 || number > 366) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for day of year must be in the range [1, 366]", number); date.day_of_year_values.push_back(number); date.day_of_year = true; @@ -1068,7 +1101,7 @@ namespace return cur; } - static Pos jodaMonthOfYearText(Pos cur, Pos end, Date & date) + static Pos jodaMonthOfYearText(int, Pos cur, Pos end, Date & date) { ensureSpace(cur, end, 3, "jodaMonthOfYearText requires size >= 3"); @@ -1097,8 +1130,8 @@ namespace { Int32 number; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), number); - if (number < 1 || number > 12) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for day of month must be in the range [1, 12]", number); + if (number < 1 || number > 31) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for day of month must be in the range [1, 31]", number); date.day_of_month_values.push_back(number); date.day = number; @@ -1112,12 +1145,12 @@ namespace return cur; } - static Pos jodaHalfDayOfDay(Pos cur, Pos end, Date & date) + static Pos jodaHalfDayOfDay(int, Pos cur, Pos end, Date & date) { ensureSpace(cur, end, 2, "jodaHalfDayOfDay requires size >= 2"); String text(cur, 2); - Poco::toLowerInPlace(cur); + Poco::toLowerInPlace(text); if (text == "am") date.is_am = true; else if (text == "pm") @@ -1510,7 +1543,6 @@ namespace instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaEra, repetitions)); // reserve_size += repetitions <= 3 ? 2 : 13; break; - /* case 'C': instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaCenturyOfEra, repetitions)); /// Year range [1900, 2299] @@ -1621,7 +1653,6 @@ namespace break; case 'Z': throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for timezone offset id"); - */ default: if (isalpha(*cur_token)) throw Exception( @@ -1633,6 +1664,7 @@ namespace } } } +#undef ACTION_ARGS_WITH_BIND } diff --git a/tests/queries/0_stateless/02668_parse_datetime.sql b/tests/queries/0_stateless/02668_parse_datetime.sql new file mode 100644 index 00000000000..61a2259fecc --- /dev/null +++ b/tests/queries/0_stateless/02668_parse_datetime.sql @@ -0,0 +1,118 @@ +-- { echoOn } +-- year +select parseDateTime('2020', '%Y') = toDateTime('2020-01-01'); + +-- month +select parseDateTime('02', '%m') = toDateTime('1970-02-01'); +select parseDateTime('07', '%m') = toDateTime('1970-07-01'); +select parseDateTime('11-', '%m-') = toDateTime('1970-11-01'); +select parseDateTime('02', '%c') = toDateTime('1970-01-01'); +select parseDateTime('jun', '%b') = toDateTime('1970-06-01'); +select parseDateTime('02', '%m'); -- { serverError LOGICAL_ERROR } + +-- day of month +select parseDateTime('07', '%d') = toDateTime('2020-01-07'); +select parseDateTime('01', '%d') = toDateTime('2020-01-01'); +select parseDateTime('/11', '/%d') = toDateTime('2020-01-11'); +select parseDateTime('32', '%d'); -- { serverError LOGICAL_ERROR } +select parseDateTime('02-31', '%m-%d'); -- { serverError LOGICAL_ERROR } +select parseDateTime('04-31', '%m-%d'); -- { serverError LOGICAL_ERROR } +select parseDateTime('2000-02-29', '%Y-%m-%d') = toDateTime('2000-02-29'); +select parseDateTime('2001-02-29', '%Y-%m-%d'); -- { serverError LOGICAL_ERROR } + +-- day of year +select parseDateTime('001', '%j') = toDateTime('2000-01-01'); +select parseDateTime('007', '%j') = toDateTime('2000-01-07'); +select parseDateTime('/031/', '/%j/') = toDateTime('2000-01-31'); +select parseDateTime('032', '%j') = toDateTime('2000-02-01'); +select parseDateTime('060', '%j') = toDateTime('2000-02-29'); +select parseDateTime('365', '%j') = toDateTime('2000-12-30'); +select parseDateTime('366', '%j') = toDateTime('2000-12-31'); +select parseDateTime('1980 001', '%Y %j') = toDateTime('1980-01-01'); +select parseDateTime('1980 007', '%Y %j') = toDateTime('1980-01-07'); +select parseDateTime('1980 /007', '%Y /%j') = toDateTime('1980-01-11'); +select parseDateTime('1980 /031/', '%Y /%j/') = toDateTime('1980-01-31'); +select parseDateTime('1980 032', '%Y %j') = toDateTime('1980-02-01'); +select parseDateTime('1980 060', '%Y %j') = toDateTime('1980-02-29'); +select parseDateTime('1980 366', '%Y %j') = toDateTime('1980-12-31'); +select parseDateTime('367', '%j'); -- { serverError LOGICAL_ERROR } +select parseDateTime('000', '%j'); -- { serverError LOGICAL_ERROR } + +-- hour of day +select parseDateTime('07', '%H') = toDateTime('1970-01-01 07:00:00'); +select parseDateTime('23', '%H') = toDateTime('1970-01-01 23:00:00'); +select parseDateTime('00', '%H') = toDateTime('1970-01-01 00:00:00'); +select parseDateTime('10', '%H') = toDateTime('1970-01-01 10:00:00'); +select parseDateTime('24', '%H'); -- { serverError LOGICAL_ERROR } +select parseDateTime('-1', '%H'); -- { serverError LOGICAL_ERROR } +select parseDateTime('1234567', '%H'); -- { serverError LOGICAL_ERROR } +select parseDateTime('07', '%k') = toDateTime('1970-01-01 07:00:00'); +select parseDateTime('23', '%k') = toDateTime('1970-01-01 23:00:00'); +select parseDateTime('00', '%k') = toDateTime('1970-01-01 00:00:00'); +select parseDateTime('10', '%k') = toDateTime('1970-01-01 10:00:00'); +select parseDateTime('24', '%k'); -- { serverError LOGICAL_ERROR } +select parseDateTime('-1', '%k'); -- { serverError LOGICAL_ERROR } +select parseDateTime('1234567', '%k'); -- { serverError LOGICAL_ERROR } + +-- clock hour of half day +select parseDateTime('07', '%h') = toDateTime('1970-01-01 07:00:00'); +select parseDateTime('12', '%h') = toDateTime('1970-01-01 00:00:00'); +select parseDateTime('01', '%h') = toDateTime('1970-01-01 00:00:00'); +select parseDateTime('10', '%h') = toDateTime('1970-01-01 10:00:00'); +select parseDateTime('00', '%h'); -- { serverError LOGICAL_ERROR } +select parseDateTime('13', '%h'); -- { serverError LOGICAL_ERROR } +select parseDateTime('123456789', '%h'); -- { serverError LOGICAL_ERROR } +select parseDateTime('07', '%I') = toDateTime('1970-01-01 07:00:00'); +select parseDateTime('12', '%I') = toDateTime('1970-01-01 00:00:00'); +select parseDateTime('01', '%I') = toDateTime('1970-01-01 00:00:00'); +select parseDateTime('10', '%I') = toDateTime('1970-01-01 10:00:00'); +select parseDateTime('00', '%I'); -- { serverError LOGICAL_ERROR } +select parseDateTime('13', '%I'); -- { serverError LOGICAL_ERROR } +select parseDateTime('123456789', '%I'); -- { serverError LOGICAL_ERROR } +select parseDateTime('07', '%l') = toDateTime('1970-01-01 07:00:00'); +select parseDateTime('12', '%l') = toDateTime('1970-01-01 00:00:00'); +select parseDateTime('01', '%l') = toDateTime('1970-01-01 00:00:00'); +select parseDateTime('10', '%l') = toDateTime('1970-01-01 10:00:00'); +select parseDateTime('00', '%l'); -- { serverError LOGICAL_ERROR } +select parseDateTime('13', '%l'); -- { serverError LOGICAL_ERROR } +select parseDateTime('123456789', '%l'); -- { serverError LOGICAL_ERROR } + +-- half of day +select parseDateTime('07 PM', '%H %p') = toDateTime('1970-01-01 07:00:00'); +select parseDateTime('07 AM', '%H %p') = toDateTime('1970-01-01 07:00:00'); +select parseDateTime('07 pm', '%H %p') = toDateTime('1970-01-01 07:00:00'); +select parseDateTime('07 am', '%H %p') = toDateTime('1970-01-01 07:00:00'); +select parseDateTime('00 AM', '%H %p') = toDateTime('1970-01-01 00:00:00'); +select parseDateTime('00 PM', '%H %p') = toDateTime('1970-01-01 00:00:00'); +select parseDateTime('00 am', '%H %p') = toDateTime('1970-01-01 00:00:00'); +select parseDateTime('00 pm', '%H %p') = toDateTime('1970-01-01 00:00:00'); + +select parseDateTime('01 PM', '%h %p') = toDateTime('1970-01-01 13:00:00'); +select parseDateTime('01 AM', '%h %p') = toDateTime('1970-01-01 01:00:00'); +select parseDateTime('06 PM', '%h %p') = toDateTime('1970-01-01 18:00:00'); +select parseDateTime('06 AM', '%h %p') = toDateTime('1970-01-01 06:00:00'); +select parseDateTime('12 PM', '%h %p') = toDateTime('1970-01-01 12:00:00'); +select parseDateTime('12 AM', '%h %p') = toDateTime('1970-01-01 00:00:00'); + +-- minute +select parseDateTime('08', '%i') = toDateTime('1970-01-01 00:08:00'); +select parseDateTime('59', '%i') = toDateTime('1970-01-01 00:59:00'); +select parseDateTime('00/', '%i/') = toDateTime('1970-01-01 00:00:00'); +select parseDateTime('60', '%i'); -- { serverError LOGICAL_ERROR } +select parseDateTime('-1', '%i'); -- { serverError LOGICAL_ERROR } +select parseDateTime('123456789', '%i'); -- { serverError LOGICAL_ERROR } + +-- second +select parseDateTime('09', '%s') = toDateTime('1970-01-01 00:00:09'); +select parseDateTime('58', '%s') = toDateTime('1970-01-01 00:00:58'); +select parseDateTime('00/', '%s/') = toDateTime('1970-01-01 00:00:00'); +select parseDateTime('60', '%s'); -- { serverError LOGICAL_ERROR } +select parseDateTime('-1', '%s'); -- { serverError LOGICAL_ERROR } +select parseDateTime('123456789', '%s'); -- { serverError LOGICAL_ERROR } + +-- mixed YMD format +select parseDateTime('2021-01-04+23:00:00', '%Y-%m-%d+%H:%i:%s') = toDateTime('2021-01-04 23:00:00'); +select parseDateTime('2019-07-03 11:04:10', '%Y-%m-%d %H:%i:%s') = toDateTime('2019-07-03 11:04:10'); +select parseDateTime('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y') = toDateTime('2019-07-03 11:04:10'); + +-- { echoOff } From beced6fd527f20cdc7e4de8b09c5913da1600bd9 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 28 Feb 2023 14:52:13 +0800 Subject: [PATCH 083/559] fix all bugs in mysql syntax --- src/Functions/parseDateTime.cpp | 23 ++- .../0_stateless/02668_parse_datetime.sql | 142 +++++++++--------- 2 files changed, 91 insertions(+), 74 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 9511ab45efe..e303bb536f0 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -148,6 +148,8 @@ namespace res += "minute:" + std::to_string(minute); res += ","; res += "second:" + std::to_string(second); + res += ","; + res += "AM:" + std::to_string(is_am); return res; } @@ -548,6 +550,11 @@ namespace throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown short month text"); date.month = i + 1; + if (!date.has_year) + { + date.has_year = true; + date.year = 2000; + } cur += 3; return cur; } @@ -557,6 +564,12 @@ namespace cur = readNumber2(cur, end, date.month); if (date.month < 1 || date.month > 12) throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for month must be in the range [1, 12]", date.month); + + if (!date.has_year) + { + date.has_year = true; + date.year = 2000; + } return cur; } @@ -780,9 +793,9 @@ namespace std::string text(cur, 2); Poco::toUpperInPlace(text); if (text == "PM") - date.is_am = true; - else if (text == "AM") date.is_am = false; + else if (text == "AM") + date.is_am = true; else throw Exception(ErrorCodes::LOGICAL_ERROR, "Text should be AM or PM, but {} provided", text); @@ -839,6 +852,7 @@ namespace { cur = readNumber2(cur, end, date.hour); if (date.hour < 1 || date.hour > 12) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for mysql hour12 must be in the range [1,12]", date.hour); date.is_hour_of_half_day = true; date.is_clock_hour = false; @@ -1112,6 +1126,11 @@ namespace throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown month of year text: {}", text1); date.month = it->second.second; + if (!date.has_year) + { + date.has_year = true; + date.year = 2000; + } cur += 3; if (cur + it->second.first.size() <= end) { diff --git a/tests/queries/0_stateless/02668_parse_datetime.sql b/tests/queries/0_stateless/02668_parse_datetime.sql index 61a2259fecc..99ca8d1d4e4 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.sql +++ b/tests/queries/0_stateless/02668_parse_datetime.sql @@ -3,17 +3,16 @@ select parseDateTime('2020', '%Y') = toDateTime('2020-01-01'); -- month -select parseDateTime('02', '%m') = toDateTime('1970-02-01'); -select parseDateTime('07', '%m') = toDateTime('1970-07-01'); -select parseDateTime('11-', '%m-') = toDateTime('1970-11-01'); -select parseDateTime('02', '%c') = toDateTime('1970-01-01'); -select parseDateTime('jun', '%b') = toDateTime('1970-06-01'); -select parseDateTime('02', '%m'); -- { serverError LOGICAL_ERROR } +select parseDateTime('02', '%m') = toDateTime('2000-02-01'); +select parseDateTime('07', '%m') = toDateTime('2000-07-01'); +select parseDateTime('11-', '%m-') = toDateTime('2000-11-01'); +select parseDateTime('02', '%c') = toDateTime('2000-02-01'); +select parseDateTime('jun', '%b') = toDateTime('2000-06-01'); -- day of month -select parseDateTime('07', '%d') = toDateTime('2020-01-07'); -select parseDateTime('01', '%d') = toDateTime('2020-01-01'); -select parseDateTime('/11', '/%d') = toDateTime('2020-01-11'); +select parseDateTime('07', '%d') = toDateTime('2000-01-07'); +select parseDateTime('01', '%d') = toDateTime('2000-01-01'); +select parseDateTime('/11', '/%d') = toDateTime('2000-01-11'); select parseDateTime('32', '%d'); -- { serverError LOGICAL_ERROR } select parseDateTime('02-31', '%m-%d'); -- { serverError LOGICAL_ERROR } select parseDateTime('04-31', '%m-%d'); -- { serverError LOGICAL_ERROR } @@ -30,7 +29,7 @@ select parseDateTime('365', '%j') = toDateTime('2000-12-30'); select parseDateTime('366', '%j') = toDateTime('2000-12-31'); select parseDateTime('1980 001', '%Y %j') = toDateTime('1980-01-01'); select parseDateTime('1980 007', '%Y %j') = toDateTime('1980-01-07'); -select parseDateTime('1980 /007', '%Y /%j') = toDateTime('1980-01-11'); +select parseDateTime('1980 /007', '%Y /%j') = toDateTime('1980-01-07'); select parseDateTime('1980 /031/', '%Y /%j/') = toDateTime('1980-01-31'); select parseDateTime('1980 032', '%Y %j') = toDateTime('1980-02-01'); select parseDateTime('1980 060', '%Y %j') = toDateTime('1980-02-29'); @@ -39,76 +38,75 @@ select parseDateTime('367', '%j'); -- { serverError LOGICAL_ERROR } select parseDateTime('000', '%j'); -- { serverError LOGICAL_ERROR } -- hour of day -select parseDateTime('07', '%H') = toDateTime('1970-01-01 07:00:00'); -select parseDateTime('23', '%H') = toDateTime('1970-01-01 23:00:00'); -select parseDateTime('00', '%H') = toDateTime('1970-01-01 00:00:00'); -select parseDateTime('10', '%H') = toDateTime('1970-01-01 10:00:00'); -select parseDateTime('24', '%H'); -- { serverError LOGICAL_ERROR } -select parseDateTime('-1', '%H'); -- { serverError LOGICAL_ERROR } -select parseDateTime('1234567', '%H'); -- { serverError LOGICAL_ERROR } -select parseDateTime('07', '%k') = toDateTime('1970-01-01 07:00:00'); -select parseDateTime('23', '%k') = toDateTime('1970-01-01 23:00:00'); -select parseDateTime('00', '%k') = toDateTime('1970-01-01 00:00:00'); -select parseDateTime('10', '%k') = toDateTime('1970-01-01 10:00:00'); -select parseDateTime('24', '%k'); -- { serverError LOGICAL_ERROR } -select parseDateTime('-1', '%k'); -- { serverError LOGICAL_ERROR } -select parseDateTime('1234567', '%k'); -- { serverError LOGICAL_ERROR } +select parseDateTime('07', '%H', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +select parseDateTime('23', '%H', 'UTC') = toDateTime('1970-01-01 23:00:00', 'UTC'); +select parseDateTime('00', '%H', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTime('10', '%H', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); +select parseDateTime('24', '%H', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('-1', '%H', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('1234567', '%H', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('07', '%k', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +select parseDateTime('23', '%k', 'UTC') = toDateTime('1970-01-01 23:00:00', 'UTC'); +select parseDateTime('00', '%k', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTime('10', '%k', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); +select parseDateTime('24', '%k', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('-1', '%k', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('1234567', '%k', 'UTC'); -- { serverError LOGICAL_ERROR } -- clock hour of half day -select parseDateTime('07', '%h') = toDateTime('1970-01-01 07:00:00'); -select parseDateTime('12', '%h') = toDateTime('1970-01-01 00:00:00'); -select parseDateTime('01', '%h') = toDateTime('1970-01-01 00:00:00'); -select parseDateTime('10', '%h') = toDateTime('1970-01-01 10:00:00'); -select parseDateTime('00', '%h'); -- { serverError LOGICAL_ERROR } -select parseDateTime('13', '%h'); -- { serverError LOGICAL_ERROR } -select parseDateTime('123456789', '%h'); -- { serverError LOGICAL_ERROR } -select parseDateTime('07', '%I') = toDateTime('1970-01-01 07:00:00'); -select parseDateTime('12', '%I') = toDateTime('1970-01-01 00:00:00'); -select parseDateTime('01', '%I') = toDateTime('1970-01-01 00:00:00'); -select parseDateTime('10', '%I') = toDateTime('1970-01-01 10:00:00'); -select parseDateTime('00', '%I'); -- { serverError LOGICAL_ERROR } -select parseDateTime('13', '%I'); -- { serverError LOGICAL_ERROR } -select parseDateTime('123456789', '%I'); -- { serverError LOGICAL_ERROR } -select parseDateTime('07', '%l') = toDateTime('1970-01-01 07:00:00'); -select parseDateTime('12', '%l') = toDateTime('1970-01-01 00:00:00'); -select parseDateTime('01', '%l') = toDateTime('1970-01-01 00:00:00'); -select parseDateTime('10', '%l') = toDateTime('1970-01-01 10:00:00'); -select parseDateTime('00', '%l'); -- { serverError LOGICAL_ERROR } -select parseDateTime('13', '%l'); -- { serverError LOGICAL_ERROR } -select parseDateTime('123456789', '%l'); -- { serverError LOGICAL_ERROR } +select parseDateTime('07', '%h', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +select parseDateTime('12', '%h', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); +select parseDateTime('01', '%h', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); +select parseDateTime('10', '%h', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); +select parseDateTime('00', '%h', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('13', '%h', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('123456789', '%h', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('07', '%I', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +select parseDateTime('12', '%I', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); +select parseDateTime('01', '%I', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); +select parseDateTime('10', '%I', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); +select parseDateTime('00', '%I', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('13', '%I', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('123456789', '%I', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('07', '%l', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +select parseDateTime('12', '%l', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); +select parseDateTime('01', '%l', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); +select parseDateTime('10', '%l', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); +select parseDateTime('00', '%l', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('13', '%l', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('123456789', '%l', 'UTC'); -- { serverError LOGICAL_ERROR } -- half of day -select parseDateTime('07 PM', '%H %p') = toDateTime('1970-01-01 07:00:00'); -select parseDateTime('07 AM', '%H %p') = toDateTime('1970-01-01 07:00:00'); -select parseDateTime('07 pm', '%H %p') = toDateTime('1970-01-01 07:00:00'); -select parseDateTime('07 am', '%H %p') = toDateTime('1970-01-01 07:00:00'); -select parseDateTime('00 AM', '%H %p') = toDateTime('1970-01-01 00:00:00'); -select parseDateTime('00 PM', '%H %p') = toDateTime('1970-01-01 00:00:00'); -select parseDateTime('00 am', '%H %p') = toDateTime('1970-01-01 00:00:00'); -select parseDateTime('00 pm', '%H %p') = toDateTime('1970-01-01 00:00:00'); - -select parseDateTime('01 PM', '%h %p') = toDateTime('1970-01-01 13:00:00'); -select parseDateTime('01 AM', '%h %p') = toDateTime('1970-01-01 01:00:00'); -select parseDateTime('06 PM', '%h %p') = toDateTime('1970-01-01 18:00:00'); -select parseDateTime('06 AM', '%h %p') = toDateTime('1970-01-01 06:00:00'); -select parseDateTime('12 PM', '%h %p') = toDateTime('1970-01-01 12:00:00'); -select parseDateTime('12 AM', '%h %p') = toDateTime('1970-01-01 00:00:00'); +select parseDateTime('07 PM', '%H %p', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +select parseDateTime('07 AM', '%H %p', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +select parseDateTime('07 pm', '%H %p', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +select parseDateTime('07 am', '%H %p', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +select parseDateTime('00 AM', '%H %p', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTime('00 PM', '%H %p', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTime('00 am', '%H %p', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTime('00 pm', '%H %p', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTime('01 PM', '%h %p', 'UTC') = toDateTime('1970-01-01 13:00:00', 'UTC'); +select parseDateTime('01 AM', '%h %p', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); +select parseDateTime('06 PM', '%h %p', 'UTC') = toDateTime('1970-01-01 18:00:00', 'UTC'); +select parseDateTime('06 AM', '%h %p', 'UTC') = toDateTime('1970-01-01 06:00:00', 'UTC'); +select parseDateTime('12 PM', '%h %p', 'UTC') = toDateTime('1970-01-02 00:00:00', 'UTC'); +select parseDateTime('12 AM', '%h %p', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); -- minute -select parseDateTime('08', '%i') = toDateTime('1970-01-01 00:08:00'); -select parseDateTime('59', '%i') = toDateTime('1970-01-01 00:59:00'); -select parseDateTime('00/', '%i/') = toDateTime('1970-01-01 00:00:00'); -select parseDateTime('60', '%i'); -- { serverError LOGICAL_ERROR } -select parseDateTime('-1', '%i'); -- { serverError LOGICAL_ERROR } -select parseDateTime('123456789', '%i'); -- { serverError LOGICAL_ERROR } +select parseDateTime('08', '%i', 'UTC') = toDateTime('1970-01-01 00:08:00', 'UTC'); +select parseDateTime('59', '%i', 'UTC') = toDateTime('1970-01-01 00:59:00', 'UTC'); +select parseDateTime('00/', '%i/', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTime('60', '%i', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('-1', '%i', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('123456789', '%i', 'UTC'); -- { serverError LOGICAL_ERROR } -- second -select parseDateTime('09', '%s') = toDateTime('1970-01-01 00:00:09'); -select parseDateTime('58', '%s') = toDateTime('1970-01-01 00:00:58'); -select parseDateTime('00/', '%s/') = toDateTime('1970-01-01 00:00:00'); -select parseDateTime('60', '%s'); -- { serverError LOGICAL_ERROR } -select parseDateTime('-1', '%s'); -- { serverError LOGICAL_ERROR } -select parseDateTime('123456789', '%s'); -- { serverError LOGICAL_ERROR } +select parseDateTime('09', '%s', 'UTC') = toDateTime('1970-01-01 00:00:09', 'UTC'); +select parseDateTime('58', '%s', 'UTC') = toDateTime('1970-01-01 00:00:58', 'UTC'); +select parseDateTime('00/', '%s/', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTime('60', '%s', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('-1', '%s', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('123456789', '%s', 'UTC'); -- { serverError LOGICAL_ERROR } -- mixed YMD format select parseDateTime('2021-01-04+23:00:00', '%Y-%m-%d+%H:%i:%s') = toDateTime('2021-01-04 23:00:00'); From e4cb6c47f80c2d76151bccc7b51d83f45310a2fb Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 28 Feb 2023 15:23:29 +0800 Subject: [PATCH 084/559] finish all uts for mysql syntax --- .../02668_parse_datetime.reference | 197 ++++++++++++++++++ 1 file changed, 197 insertions(+) create mode 100644 tests/queries/0_stateless/02668_parse_datetime.reference diff --git a/tests/queries/0_stateless/02668_parse_datetime.reference b/tests/queries/0_stateless/02668_parse_datetime.reference new file mode 100644 index 00000000000..24df3a8bd76 --- /dev/null +++ b/tests/queries/0_stateless/02668_parse_datetime.reference @@ -0,0 +1,197 @@ +-- { echoOn } +-- year +select parseDateTime('2020', '%Y') = toDateTime('2020-01-01'); +1 +-- month +select parseDateTime('02', '%m') = toDateTime('2000-02-01'); +1 +select parseDateTime('07', '%m') = toDateTime('2000-07-01'); +1 +select parseDateTime('11-', '%m-') = toDateTime('2000-11-01'); +1 +select parseDateTime('00', '%m'); -- { serverError LOGICAL_ERROR } +select parseDateTime('13', '%m'); -- { serverError LOGICAL_ERROR } +select parseDateTime('12345', '%m'); -- { serverError LOGICAL_ERROR } +select parseDateTime('02', '%c') = toDateTime('2000-02-01'); +1 +select parseDateTime('07', '%c') = toDateTime('2000-07-01'); +1 +select parseDateTime('11-', '%c-') = toDateTime('2000-11-01'); +1 +select parseDateTime('00', '%c'); -- { serverError LOGICAL_ERROR } +select parseDateTime('13', '%c'); -- { serverError LOGICAL_ERROR } +select parseDateTime('12345', '%c'); -- { serverError LOGICAL_ERROR } +select parseDateTime('jun', '%b') = toDateTime('2000-06-01'); +1 +select parseDateTime('JUN', '%b') = toDateTime('2000-06-01'); +1 +select parseDateTime('abc', '%b'); -- { serverError LOGICAL_ERROR } +-- day of month +select parseDateTime('07', '%d') = toDateTime('2000-01-07'); +1 +select parseDateTime('01', '%d') = toDateTime('2000-01-01'); +1 +select parseDateTime('/11', '/%d') = toDateTime('2000-01-11'); +1 +select parseDateTime('00', '%d'); -- { serverError LOGICAL_ERROR } +select parseDateTime('32', '%d'); -- { serverError LOGICAL_ERROR } +select parseDateTime('12345', '%d'); -- { serverError LOGICAL_ERROR } +select parseDateTime('02-31', '%m-%d'); -- { serverError LOGICAL_ERROR } +select parseDateTime('04-31', '%m-%d'); -- { serverError LOGICAL_ERROR } +-- Ensure all days of month are checked against final selected month +select parseDateTime('01 31 20 02', '%m %d %d %m'); -- { serverError LOGICAL_ERROR } +select parseDateTime('02 31 20 04', '%m %d %d %m'); -- { serverError LOGICAL_ERROR } +select parseDateTime('02 31 01', '%m %d %m') = toDateTime('2000-01-31'); +1 +select parseDateTime('2000-02-29', '%Y-%m-%d') = toDateTime('2000-02-29'); +1 +select parseDateTime('2001-02-29', '%Y-%m-%d'); -- { serverError LOGICAL_ERROR } +-- day of year +select parseDateTime('001', '%j') = toDateTime('2000-01-01'); +1 +select parseDateTime('007', '%j') = toDateTime('2000-01-07'); +1 +select parseDateTime('/031/', '/%j/') = toDateTime('2000-01-31'); +1 +select parseDateTime('032', '%j') = toDateTime('2000-02-01'); +1 +select parseDateTime('060', '%j') = toDateTime('2000-02-29'); +1 +select parseDateTime('365', '%j') = toDateTime('2000-12-30'); +1 +select parseDateTime('366', '%j') = toDateTime('2000-12-31'); +1 +select parseDateTime('1980 001', '%Y %j') = toDateTime('1980-01-01'); +1 +select parseDateTime('1980 007', '%Y %j') = toDateTime('1980-01-07'); +1 +select parseDateTime('1980 /007', '%Y /%j') = toDateTime('1980-01-07'); +1 +select parseDateTime('1980 /031/', '%Y /%j/') = toDateTime('1980-01-31'); +1 +select parseDateTime('1980 032', '%Y %j') = toDateTime('1980-02-01'); +1 +select parseDateTime('1980 060', '%Y %j') = toDateTime('1980-02-29'); +1 +select parseDateTime('1980 366', '%Y %j') = toDateTime('1980-12-31'); +1 +select parseDateTime('1981 366', '%Y %j'); -- { serverError LOGICAL_ERROR } +select parseDateTime('367', '%j'); -- { serverError LOGICAL_ERROR } +select parseDateTime('000', '%j'); -- { serverError LOGICAL_ERROR } +-- Ensure all days of year are checked against final selected year +select parseDateTime('2000 366 2001', '%Y %j %Y'); -- { serverError LOGICAL_ERROR } +select parseDateTime('2001 366 2000', '%Y %j %Y') = toDateTime('2000-12-31'); +1 +-- hour of day +select parseDateTime('07', '%H', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +1 +select parseDateTime('23', '%H', 'UTC') = toDateTime('1970-01-01 23:00:00', 'UTC'); +1 +select parseDateTime('00', '%H', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTime('10', '%H', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); +1 +select parseDateTime('24', '%H', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('-1', '%H', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('1234567', '%H', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('07', '%k', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +1 +select parseDateTime('23', '%k', 'UTC') = toDateTime('1970-01-01 23:00:00', 'UTC'); +1 +select parseDateTime('00', '%k', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTime('10', '%k', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); +1 +select parseDateTime('24', '%k', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('-1', '%k', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('1234567', '%k', 'UTC'); -- { serverError LOGICAL_ERROR } +-- hour of half day +select parseDateTime('07', '%h', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +1 +select parseDateTime('12', '%h', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); +1 +select parseDateTime('01', '%h', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); +1 +select parseDateTime('10', '%h', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); +1 +select parseDateTime('00', '%h', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('13', '%h', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('123456789', '%h', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('07', '%I', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +1 +select parseDateTime('12', '%I', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); +1 +select parseDateTime('01', '%I', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); +1 +select parseDateTime('10', '%I', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); +1 +select parseDateTime('00', '%I', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('13', '%I', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('123456789', '%I', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('07', '%l', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +1 +select parseDateTime('12', '%l', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); +1 +select parseDateTime('01', '%l', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); +1 +select parseDateTime('10', '%l', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); +1 +select parseDateTime('00', '%l', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('13', '%l', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('123456789', '%l', 'UTC'); -- { serverError LOGICAL_ERROR } +-- half of day +select parseDateTime('07 PM', '%H %p', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +1 +select parseDateTime('07 AM', '%H %p', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +1 +select parseDateTime('07 pm', '%H %p', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +1 +select parseDateTime('07 am', '%H %p', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +1 +select parseDateTime('00 AM', '%H %p', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTime('00 PM', '%H %p', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTime('00 am', '%H %p', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTime('00 pm', '%H %p', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTime('01 PM', '%h %p', 'UTC') = toDateTime('1970-01-01 13:00:00', 'UTC'); +1 +select parseDateTime('01 AM', '%h %p', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); +1 +select parseDateTime('06 PM', '%h %p', 'UTC') = toDateTime('1970-01-01 18:00:00', 'UTC'); +1 +select parseDateTime('06 AM', '%h %p', 'UTC') = toDateTime('1970-01-01 06:00:00', 'UTC'); +1 +select parseDateTime('12 PM', '%h %p', 'UTC') = toDateTime('1970-01-02 00:00:00', 'UTC'); +1 +select parseDateTime('12 AM', '%h %p', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); +1 +-- minute +select parseDateTime('08', '%i', 'UTC') = toDateTime('1970-01-01 00:08:00', 'UTC'); +1 +select parseDateTime('59', '%i', 'UTC') = toDateTime('1970-01-01 00:59:00', 'UTC'); +1 +select parseDateTime('00/', '%i/', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTime('60', '%i', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('-1', '%i', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('123456789', '%i', 'UTC'); -- { serverError LOGICAL_ERROR } +-- second +select parseDateTime('09', '%s', 'UTC') = toDateTime('1970-01-01 00:00:09', 'UTC'); +1 +select parseDateTime('58', '%s', 'UTC') = toDateTime('1970-01-01 00:00:58', 'UTC'); +1 +select parseDateTime('00/', '%s/', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTime('60', '%s', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('-1', '%s', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('123456789', '%s', 'UTC'); -- { serverError LOGICAL_ERROR } +-- mixed YMD format +select parseDateTime('2021-01-04+23:00:00', '%Y-%m-%d+%H:%i:%s') = toDateTime('2021-01-04 23:00:00'); +1 +select parseDateTime('2019-07-03 11:04:10', '%Y-%m-%d %H:%i:%s') = toDateTime('2019-07-03 11:04:10'); +1 +select parseDateTime('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y') = toDateTime('2019-07-03 11:04:10'); +1 From fe9c4ea0ce5bd5e580189bc72b09a9424ad86ca7 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 28 Feb 2023 17:27:59 +0800 Subject: [PATCH 085/559] add test of joda syntax --- src/Functions/parseDateTime.cpp | 20 ++- .../0_stateless/02668_parse_datetime.sql | 24 ++- .../02668_parse_datetime_in_joda_syntax.sql | 138 ++++++++++++++++++ 3 files changed, 173 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index e303bb536f0..b01d0315855 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -217,7 +217,12 @@ namespace static Int32 daysSinceEpochFromWeekDate(int32_t week_year_, int32_t week_of_year_, int32_t day_of_week_) { if (!isWeekDateValid(week_year_, week_of_year_, day_of_week_)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid week date"); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Invalid week date, week year:{} week of year:{} day of week:{}", + week_year_, + week_of_year_, + day_of_week_); Int32 days_since_epoch_of_jan_fourth = daysSinceEpochFromDate(week_year_, 1, 4); Int32 first_day_of_week_year = extractISODayOfTheWeek(days_since_epoch_of_jan_fourth); @@ -227,7 +232,7 @@ namespace static Int32 daysSinceEpochFromDayOfYear(Int32 year_, Int32 day_of_year_) { if (!isDayOfYearValid(year_, day_of_year_)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of year"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of year, year:{} day of year:{}", year_, day_of_year_); Int32 res = daysSinceEpochFromDate(year_, 1, 1); res += day_of_year_ - 1; @@ -237,7 +242,7 @@ namespace static Int32 daysSinceEpochFromDate(Int32 year_, Int32 month_, Int32 day_) { if (!isDateValid(year_, month_, day_)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid date"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid date, year:{} month:{} day:{}", year_, month_, day_); Int32 res = cumulativeYearDays[year_ - 1970]; res += isLeapYear(year_) ? cumulativeLeapDays[month_ - 1] : cumulativeDays[month_ - 1]; @@ -259,14 +264,14 @@ namespace for (const auto d : day_of_month_values) { if (!isDateValid(year, month, d)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of month."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of month, year:{} month:{} day:{}", year, month, d); } // Ensure all day of year values are valid for ending year value for (const auto d : day_of_year_values) { if (!isDayOfYearValid(year, d)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of year."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of year, year:{} day of year:{}", year, d); } // Convert the parsed date/time into a timestamp. @@ -1086,10 +1091,10 @@ namespace throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for day of year must be in the range [1, 366]", number); date.day_of_year_values.push_back(number); - date.day_of_year = true; + date.day_of_year = number; date.day_of_year_format = true; date.week_date_format = false; - if (date.has_year) + if (!date.has_year) { date.has_year = true; date.year = 2000; @@ -1568,7 +1573,6 @@ namespace // reserve_size += std::max(repetitions, 2); break; case 'Y': - instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaYearOfEra, repetitions)); /// Year range [1900, 2299] // reserve_size += repetitions == 2 ? 2 : std::max(repetitions, 4); diff --git a/tests/queries/0_stateless/02668_parse_datetime.sql b/tests/queries/0_stateless/02668_parse_datetime.sql index 99ca8d1d4e4..ac8e68c4f93 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.sql +++ b/tests/queries/0_stateless/02668_parse_datetime.sql @@ -6,16 +6,34 @@ select parseDateTime('2020', '%Y') = toDateTime('2020-01-01'); select parseDateTime('02', '%m') = toDateTime('2000-02-01'); select parseDateTime('07', '%m') = toDateTime('2000-07-01'); select parseDateTime('11-', '%m-') = toDateTime('2000-11-01'); +select parseDateTime('00', '%m'); -- { serverError LOGICAL_ERROR } +select parseDateTime('13', '%m'); -- { serverError LOGICAL_ERROR } +select parseDateTime('12345', '%m'); -- { serverError LOGICAL_ERROR } + select parseDateTime('02', '%c') = toDateTime('2000-02-01'); +select parseDateTime('07', '%c') = toDateTime('2000-07-01'); +select parseDateTime('11-', '%c-') = toDateTime('2000-11-01'); +select parseDateTime('00', '%c'); -- { serverError LOGICAL_ERROR } +select parseDateTime('13', '%c'); -- { serverError LOGICAL_ERROR } +select parseDateTime('12345', '%c'); -- { serverError LOGICAL_ERROR } + select parseDateTime('jun', '%b') = toDateTime('2000-06-01'); +select parseDateTime('JUN', '%b') = toDateTime('2000-06-01'); +select parseDateTime('abc', '%b'); -- { serverError LOGICAL_ERROR } -- day of month select parseDateTime('07', '%d') = toDateTime('2000-01-07'); select parseDateTime('01', '%d') = toDateTime('2000-01-01'); select parseDateTime('/11', '/%d') = toDateTime('2000-01-11'); +select parseDateTime('00', '%d'); -- { serverError LOGICAL_ERROR } select parseDateTime('32', '%d'); -- { serverError LOGICAL_ERROR } +select parseDateTime('12345', '%d'); -- { serverError LOGICAL_ERROR } select parseDateTime('02-31', '%m-%d'); -- { serverError LOGICAL_ERROR } select parseDateTime('04-31', '%m-%d'); -- { serverError LOGICAL_ERROR } +-- Ensure all days of month are checked against final selected month +select parseDateTime('01 31 20 02', '%m %d %d %m'); -- { serverError LOGICAL_ERROR } +select parseDateTime('02 31 20 04', '%m %d %d %m'); -- { serverError LOGICAL_ERROR } +select parseDateTime('02 31 01', '%m %d %m') = toDateTime('2000-01-31'); select parseDateTime('2000-02-29', '%Y-%m-%d') = toDateTime('2000-02-29'); select parseDateTime('2001-02-29', '%Y-%m-%d'); -- { serverError LOGICAL_ERROR } @@ -34,8 +52,12 @@ select parseDateTime('1980 /031/', '%Y /%j/') = toDateTime('1980-01-31'); select parseDateTime('1980 032', '%Y %j') = toDateTime('1980-02-01'); select parseDateTime('1980 060', '%Y %j') = toDateTime('1980-02-29'); select parseDateTime('1980 366', '%Y %j') = toDateTime('1980-12-31'); +select parseDateTime('1981 366', '%Y %j'); -- { serverError LOGICAL_ERROR } select parseDateTime('367', '%j'); -- { serverError LOGICAL_ERROR } select parseDateTime('000', '%j'); -- { serverError LOGICAL_ERROR } +-- Ensure all days of year are checked against final selected year +select parseDateTime('2000 366 2001', '%Y %j %Y'); -- { serverError LOGICAL_ERROR } +select parseDateTime('2001 366 2000', '%Y %j %Y') = toDateTime('2000-12-31'); -- hour of day select parseDateTime('07', '%H', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); @@ -53,7 +75,7 @@ select parseDateTime('24', '%k', 'UTC'); -- { serverError LOGICAL_ERROR } select parseDateTime('-1', '%k', 'UTC'); -- { serverError LOGICAL_ERROR } select parseDateTime('1234567', '%k', 'UTC'); -- { serverError LOGICAL_ERROR } --- clock hour of half day +-- hour of half day select parseDateTime('07', '%h', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); select parseDateTime('12', '%h', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); select parseDateTime('01', '%h', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); diff --git a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql new file mode 100644 index 00000000000..841be265f3d --- /dev/null +++ b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql @@ -0,0 +1,138 @@ +-- empty +select parseDateTimeInJodaSyntax(' ', ' ', 'UTC') = toDateTime('1970-01-01', 'UTC'); + +-- era +select parseDateTimeInJodaSyntax('AD 1999', 'G YYYY') = toDateTime('1999-01-01'); +select parseDateTimeInJodaSyntax('ad 1999', 'G YYYY') = toDateTime('1999-01-01'); +select parseDateTimeInJodaSyntax('Ad 1999', 'G YYYY') = toDateTime('1999-01-01'); +select parseDateTimeInJodaSyntax('AD 1999', 'G YYYY') = toDateTime('1999-01-01'); +select parseDateTimeInJodaSyntax('AD 1999', 'G yyyy') = toDateTime('1999-01-01'); +select parseDateTimeInJodaSyntax('AD 1999 2000', 'G YYYY yyyy') = toDateTime('2000-01-01'); +select parseDateTimeInJodaSyntax('AD 1999 2000', 'G yyyy YYYY') = toDateTime('2000-01-01'); +select parseDateTimeInJodaSyntax('AD 1999', 'G Y'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('AD 1999', 'G YY'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('AD 1999', 'G YYY'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('BC', 'G'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('AB', 'G'); -- { serverError LOGICAL_ERROR } + +-- year of era +select parseDateTimeInJodaSyntax('2106', 'YYYY', 'UTC') = toDateTime('2106-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('1970', 'YYYY', 'UTC') = toDateTime('1970-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('1969', 'YYYY', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('2107', 'YYYY', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('+1999', 'YYYY', 'UTC') -- { serverError LOGICAL_ERROR } + +select parseDateTimeInJodaSyntax('12', 'YY', 'UTC') = toDateTime('2012-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('69', 'YY', 'UTC') = toDateTime('2069-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('70', 'YY', 'UTC') = toDateTime('1970-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('99', 'YY', 'UTC') = toDateTime('1999-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('01', 'YY', 'UTC') = toDateTime('2001-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('1', 'YY', 'UTC'); -- { serverError LOGICAL_ERROR } + +select parseDateTimeInJodaSyntax('99 98 97', 'YY YY YY', 'UTC') = toDateTime('1997-01-01', 'UTC'); + +-- year +select parseDateTimeInJodaSyntax('12', 'yy', 'UTC') = toDateTime('2012-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('69', 'yy', 'UTC') = toDateTime('2069-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('70', 'yy', 'UTC') = toDateTime('1970-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('99', 'yy', 'UTC') = toDateTime('1999-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('+99', 'yy', 'UTC') = toDateTime('1999-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('+99 02', 'yy MM', 'UTC') = toDateTime('1999-02-01', 'UTC'); +select parseDateTimeInJodaSyntax('10 +10', 'MM yy', 'UTC') = toDateTime('2010-10-01', 'UTC'); +select parseDateTimeInJodaSyntax('10+2001', 'MMyyyy', 'UTC') = toDateTime('2001-10-01', 'UTC'); +select parseDateTimeInJodaSyntax('+200110', 'yyyyMM', 'UTC') = toDateTime('2001-10-01', 'UTC'); +select parseDateTimeInJodaSyntax('1970', 'yyyy', 'UTC') = toDateTime('1970-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('2106', 'yyyy', 'UTC') = toDateTime('2106-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('1969', 'yyyy', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('2107', 'yyyy', 'UTC'); -- { serverError LOGICAL_ERROR } + +-- week year +select parseDateTimeInJodaSyntax('2106', 'xxxx', 'UTC') = toDateTime('2106-01-04', 'UTC'); +select parseDateTimeInJodaSyntax('1971', 'xxxx', 'UTC') = toDateTime('1971-01-04', 'UTC'); +select parseDateTimeInJodaSyntax('2025', 'xxxx', 'UTC') = toDateTime('2024-12-30', 'UTC'); +select parseDateTimeInJodaSyntax('12', 'xx', 'UTC') = toDateTime('2012-01-02', 'UTC'); +select parseDateTimeInJodaSyntax('69', 'xx', 'UTC') = toDateTime('2068-12-31', 'UTC'); +select parseDateTimeInJodaSyntax('99', 'xx', 'UTC') = toDateTime('1999-01-04', 'UTC'); +select parseDateTimeInJodaSyntax('01', 'xx', 'UTC') = toDateTime('2001-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('+10', 'xx', 'UTC') = toDateTime('2001-01-04', 'UTC'); +select parseDateTimeInJodaSyntax('+99 01', 'xx ww', 'UTC') = toDateTime('1999-01-04', 'UTC'); +select parseDateTimeInJodaSyntax('+99 02', 'xx ww', 'UTC') = toDateTime('1999-01-11', 'UTC'); +select parseDateTimeInJodaSyntax('10 +10', 'ww xx', 'UTC') = toDateTime('2010-03-08', 'UTC'); +select parseDateTimeInJodaSyntax('2+10', 'wwxx', 'UTC') = toDateTime('2010-01-11', 'UTC'); +select parseDateTimeInJodaSyntax('+102', 'xxM', 'UTC') = toDateTime('2010-02-01', 'UTC'); +select parseDateTimeInJodaSyntax('+20102', 'xxxxM', 'UTC') = toDateTime('2010-02-01', 'UTC'); +select parseDateTimeInJodaSyntax('1970', 'xxxx', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('1969', 'xxxx', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('2107', 'xxxx', 'UTC'); -- { serverError LOGICAL_ERROR } + +-- century of era +select parseDateTimeInJodaSyntax('20', 'CC', 'UTC') = toDateTime('2000-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('21', 'CC', 'UTC') = toDateTime('2100-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('19', 'CC', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('22', 'CC', 'UTC'); -- { serverError LOGICAL_ERROR } + +-- month +select parseDateTimeInJodaSyntax('1', 'M', 'UTC') = toDateTime('2000-01-01', 'UTC'); +select parseDateTimeInJodaSyntax(' 7', ' MM', 'UTC') = toDateTime('2000-07-01', 'UTC'); +select parseDateTimeInJodaSyntax('11', 'M', 'UTC') = toDateTime('2000-11-01', 'UTC'); +select parseDateTimeInJodaSyntax('10-', 'M-', 'UTC') = toDateTime('2000-10-01', 'UTC'); +select parseDateTimeInJodaSyntax('-12-', '-M-', 'UTC') = toDateTime('2000-12-01', 'UTC'); +select parseDateTimeInJodaSyntax('0', 'M', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('13', 'M', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('12345', 'M', 'UTC'); -- { serverError LOGICAL_ERROR } +--- Ensure MMM and MMMM specifiers consume both short- and long-form month names +select parseDateTimeInJodaSyntax('Aug', 'MMM', 'UTC') = toDateTime('2000-08-01', 'UTC'); +select parseDateTimeInJodaSyntax('AuG', 'MMM', 'UTC') = toDateTime('2000-08-01', 'UTC'); +select parseDateTimeInJodaSyntax('august', 'MMM', 'UTC') = toDateTime('2000-08-01', 'UTC'); +select parseDateTimeInJodaSyntax('Aug', 'MMMM', 'UTC') = toDateTime('2000-08-01', 'UTC'); +select parseDateTimeInJodaSyntax('AuG', 'MMMM', 'UTC') = toDateTime('2000-08-01', 'UTC'); +select parseDateTimeInJodaSyntax('august', 'MMMM', 'UTC') = toDateTime('2000-08-01', 'UTC'); +--- invalid month names +select parseDateTimeInJodaSyntax('Decembr', 'MMM', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('Decembr', 'MMMM', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('Decemberary', 'MMM', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('Decemberary', 'MMMM', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('asdf', 'MMM', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('asdf', 'MMMM', 'UTC'); -- { serverError LOGICAL_ERROR } + +-- day of month +select parseDateTimeInJodaSyntax('1', 'd', 'UTC') = toDateTime('2000-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('7 ', 'dd ', 'UTC') = toDateTime('2000-01-07', 'UTC'); +select parseDateTimeInJodaSyntax('/11', '/dd', 'UTC') = toDateTime('2000-01-11', 'UTC'); +select parseDateTimeInJodaSyntax('/31/', '/d/', 'UTC') = toDateTime('2000-01-31', 'UTC'); +select parseDateTimeInJodaSyntax('0', 'd', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('32', 'd', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('12345', 'd', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('02-31', 'M-d', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('04-31', 'M-d', 'UTC'); -- { serverError LOGICAL_ERROR } +--- Ensure all days of month are checked against final selected month +select parseDateTimeInJodaSyntax('2 31 1', 'M d M') = toDateTime('2000-01-31', 'UTC'); +select parseDateTimeInJodaSyntax('1 31 20 2', 'M d d M'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('2 31 20 4', 'M d d M'); -- { serverError LOGICAL_ERROR } +--- Leap year +select parseDateTimeInJodaSyntax('2020-02-29', 'YYYY-M-d') = toDateTime('2020-02-29', 'UTC'); +select parseDateTimeInJodaSyntax('2001-02-29', 'YYYY-M-d'); -- { serverError LOGICAL_ERROR } + +-- day of year +select parseDateTimeInJodaSyntax('1', 'D', 'UTC') = toDateTime('2000-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('7 ', 'DD ', 'UTC') = toDateTime('2000-01-07', 'UTC'); +select parseDateTimeInJodaSyntax('/11', '/DD', 'UTC') = toDateTime('2000-01-11', 'UTC'); +select parseDateTimeInJodaSyntax('/31/', '/DDD/', 'UTC') = toDateTime('2000-01-31', 'UTC'); +select parseDateTimeInJodaSyntax('32', 'D', 'UTC') = toDateTime('2000-02-01', 'UTC'); +select parseDateTimeInJodaSyntax('60', 'D', 'UTC') = toDateTime('2000-02-29', 'UTC'); +select parseDateTimeInJodaSyntax('365', 'D', 'UTC') = toDateTime('2000-12-30', 'UTC'); +select parseDateTimeInJodaSyntax('366', 'D', 'UTC') = toDateTime('2000-12-31', 'UTC'); +select parseDateTimeInJodaSyntax('1999 1', 'yyyy D', 'UTC') = toDateTime('1999-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('1999 7 ', 'yyyy DD ', 'UTC') = toDateTime('1999-01-07', 'UTC'); +select parseDateTimeInJodaSyntax('1999 /11', 'yyyy /DD', 'UTC') = toDateTime('1999-01-11', 'UTC'); +select parseDateTimeInJodaSyntax('1999 /31/', 'yyyy /DD/', 'UTC') = toDateTime('1999-01-31', 'UTC'); +select parseDateTimeInJodaSyntax('1999 32', 'yyyy D', 'UTC') = toDateTime('1999-02-01', 'UTC'); +select parseDateTimeInJodaSyntax('1999 60', 'yyyy D', 'UTC') = toDateTime('1999-03-01', 'UTC'); +select parseDateTimeInJodaSyntax('1999 365', 'yyyy D', 'UTC') = toDateTime('1999-12-31', 'UTC'); +select parseDateTimeInJodaSyntax('1999 366', 'yyyy D', 'UTC'); -- { serverError LOGICAL_ERROR } +--- Ensure all days of year are checked against final selected year +select parseDateTimeInJodaSyntax('2001 366 2000', 'yyyy D yyyy', 'UTC') = toDateTime('2000-12-31', 'UTC'); +select parseDateTimeInJodaSyntax('2000 366 2001', 'yyyy D yyyy', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('0', 'D', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('367', 'D', 'UTC'); -- { serverError LOGICAL_ERROR } + From de81a5f92da88a6ff43d0d1ee155eec268638093 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 28 Feb 2023 13:36:52 +0100 Subject: [PATCH 086/559] Fix integration tests --- .../test_access_control_on_cluster/configs/users.d/users.xml | 1 + .../integration/test_create_query_constraints/configs/users.xml | 1 + .../integration/test_global_overcommit_tracker/configs/users.xml | 1 + .../integration/test_grant_and_revoke/configs/users.d/users.xml | 1 + .../test_overcommit_tracker/configs/users.d/users.xml | 1 + .../configs/users.d/users.xml | 1 + 6 files changed, 6 insertions(+) diff --git a/tests/integration/test_access_control_on_cluster/configs/users.d/users.xml b/tests/integration/test_access_control_on_cluster/configs/users.d/users.xml index 775c63350b0..1c5f4d5a21f 100644 --- a/tests/integration/test_access_control_on_cluster/configs/users.d/users.xml +++ b/tests/integration/test_access_control_on_cluster/configs/users.d/users.xml @@ -5,6 +5,7 @@ default default 1 + 1 diff --git a/tests/integration/test_create_query_constraints/configs/users.xml b/tests/integration/test_create_query_constraints/configs/users.xml index 775c63350b0..1c5f4d5a21f 100644 --- a/tests/integration/test_create_query_constraints/configs/users.xml +++ b/tests/integration/test_create_query_constraints/configs/users.xml @@ -5,6 +5,7 @@ default default 1 + 1 diff --git a/tests/integration/test_global_overcommit_tracker/configs/users.xml b/tests/integration/test_global_overcommit_tracker/configs/users.xml index 775c63350b0..1c5f4d5a21f 100644 --- a/tests/integration/test_global_overcommit_tracker/configs/users.xml +++ b/tests/integration/test_global_overcommit_tracker/configs/users.xml @@ -5,6 +5,7 @@ default default 1 + 1 diff --git a/tests/integration/test_grant_and_revoke/configs/users.d/users.xml b/tests/integration/test_grant_and_revoke/configs/users.d/users.xml index 775c63350b0..1c5f4d5a21f 100644 --- a/tests/integration/test_grant_and_revoke/configs/users.d/users.xml +++ b/tests/integration/test_grant_and_revoke/configs/users.d/users.xml @@ -5,6 +5,7 @@ default default 1 + 1 diff --git a/tests/integration/test_overcommit_tracker/configs/users.d/users.xml b/tests/integration/test_overcommit_tracker/configs/users.d/users.xml index 775c63350b0..1c5f4d5a21f 100644 --- a/tests/integration/test_overcommit_tracker/configs/users.d/users.xml +++ b/tests/integration/test_overcommit_tracker/configs/users.d/users.xml @@ -5,6 +5,7 @@ default default 1 + 1 diff --git a/tests/integration/test_settings_constraints_distributed/configs/users.d/users.xml b/tests/integration/test_settings_constraints_distributed/configs/users.d/users.xml index 775c63350b0..1c5f4d5a21f 100644 --- a/tests/integration/test_settings_constraints_distributed/configs/users.d/users.xml +++ b/tests/integration/test_settings_constraints_distributed/configs/users.d/users.xml @@ -5,6 +5,7 @@ default default 1 + 1 From 8d33a961ab09a496c67c3b92f9ec1b54471b2968 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 28 Feb 2023 20:42:16 +0800 Subject: [PATCH 087/559] fix all bugs --- src/Functions/parseDateTime.cpp | 9 +- ...68_parse_datetime_in_joda_syntax.reference | 344 ++++++++++++++++++ .../02668_parse_datetime_in_joda_syntax.sql | 109 +++++- 3 files changed, 452 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index b01d0315855..63c0f98d53d 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -952,9 +952,12 @@ namespace static Pos jodaEra(int, Pos cur, Pos end, Date & date) { ensureSpace(cur, end, 2, "jodaEra requires size >= 2"); - if (std::strncmp(cur, "AD", 2) == 0 || std::strncmp(cur, "ad", 2) == 0) + + String text(cur, 2); + Poco::toLowerInPlace(text); + if (text == "ad") date.is_ad = true; - else if (std::strncmp(cur, "BC", 2) == 0 || std::strncmp(cur, "bc", 2) == 0) + else if (text == "bc") date.is_ad = false; else throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown era {}", std::string(cur, 2)); @@ -1208,7 +1211,7 @@ namespace date.is_clock_hour = true; date.is_hour_of_half_day = true; - date.hour = number; + date.hour = number % 12; return cur; } diff --git a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference new file mode 100644 index 00000000000..5d338022a26 --- /dev/null +++ b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference @@ -0,0 +1,344 @@ +-- { echoOn } +-- empty +select parseDateTimeInJodaSyntax(' ', ' ', 'UTC') = toDateTime('1970-01-01', 'UTC'); +1 +-- era +select parseDateTimeInJodaSyntax('AD 1999', 'G YYYY') = toDateTime('1999-01-01'); +1 +select parseDateTimeInJodaSyntax('ad 1999', 'G YYYY') = toDateTime('1999-01-01'); +1 +select parseDateTimeInJodaSyntax('Ad 1999', 'G YYYY') = toDateTime('1999-01-01'); +1 +select parseDateTimeInJodaSyntax('AD 1999', 'G YYYY') = toDateTime('1999-01-01'); +1 +select parseDateTimeInJodaSyntax('AD 1999', 'G yyyy') = toDateTime('1999-01-01'); +1 +select parseDateTimeInJodaSyntax('AD 1999 2000', 'G YYYY yyyy') = toDateTime('2000-01-01'); +1 +select parseDateTimeInJodaSyntax('AD 1999 2000', 'G yyyy YYYY') = toDateTime('2000-01-01'); +1 +select parseDateTimeInJodaSyntax('AD 1999', 'G Y'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('AD 1999', 'G YY'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('AD 1999', 'G YYY'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('BC', 'G'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('AB', 'G'); -- { serverError LOGICAL_ERROR } +-- year of era +select parseDateTimeInJodaSyntax('2106', 'YYYY', 'UTC') = toDateTime('2106-01-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('1970', 'YYYY', 'UTC') = toDateTime('1970-01-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('1969', 'YYYY', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('2107', 'YYYY', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('+1999', 'YYYY', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('12', 'YY', 'UTC') = toDateTime('2012-01-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('69', 'YY', 'UTC') = toDateTime('2069-01-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('70', 'YY', 'UTC') = toDateTime('1970-01-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('99', 'YY', 'UTC') = toDateTime('1999-01-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('01', 'YY', 'UTC') = toDateTime('2001-01-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('1', 'YY', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('99 98 97', 'YY YY YY', 'UTC') = toDateTime('1997-01-01', 'UTC'); +1 +-- year +select parseDateTimeInJodaSyntax('12', 'yy', 'UTC') = toDateTime('2012-01-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('69', 'yy', 'UTC') = toDateTime('2069-01-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('70', 'yy', 'UTC') = toDateTime('1970-01-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('99', 'yy', 'UTC') = toDateTime('1999-01-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('+99', 'yy', 'UTC') = toDateTime('1999-01-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('+99 02', 'yy MM', 'UTC') = toDateTime('1999-02-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('10 +10', 'MM yy', 'UTC') = toDateTime('2010-10-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('10+2001', 'MMyyyy', 'UTC') = toDateTime('2001-10-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('+200110', 'yyyyMM', 'UTC') = toDateTime('2001-10-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('1970', 'yyyy', 'UTC') = toDateTime('1970-01-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('2106', 'yyyy', 'UTC') = toDateTime('2106-01-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('1969', 'yyyy', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('2107', 'yyyy', 'UTC'); -- { serverError LOGICAL_ERROR } +-- week year +select parseDateTimeInJodaSyntax('2106', 'xxxx', 'UTC') = toDateTime('2106-01-04', 'UTC'); +1 +select parseDateTimeInJodaSyntax('1971', 'xxxx', 'UTC') = toDateTime('1971-01-04', 'UTC'); +1 +select parseDateTimeInJodaSyntax('2025', 'xxxx', 'UTC') = toDateTime('2024-12-30', 'UTC'); +1 +select parseDateTimeInJodaSyntax('12', 'xx', 'UTC') = toDateTime('2012-01-02', 'UTC'); +1 +select parseDateTimeInJodaSyntax('69', 'xx', 'UTC') = toDateTime('2068-12-31', 'UTC'); +1 +select parseDateTimeInJodaSyntax('99', 'xx', 'UTC') = toDateTime('1999-01-04', 'UTC'); +1 +select parseDateTimeInJodaSyntax('01', 'xx', 'UTC') = toDateTime('2001-01-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('+10', 'xx', 'UTC') = toDateTime('2010-01-04', 'UTC'); +1 +select parseDateTimeInJodaSyntax('+99 01', 'xx ww', 'UTC') = toDateTime('1999-01-04', 'UTC'); +1 +select parseDateTimeInJodaSyntax('+99 02', 'xx ww', 'UTC') = toDateTime('1999-01-11', 'UTC'); +1 +select parseDateTimeInJodaSyntax('10 +10', 'ww xx', 'UTC') = toDateTime('2010-03-08', 'UTC'); +1 +select parseDateTimeInJodaSyntax('2+10', 'wwxx', 'UTC') = toDateTime('2010-01-11', 'UTC'); +1 +select parseDateTimeInJodaSyntax('+102', 'xxM', 'UTC') = toDateTime('2010-02-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('+20102', 'xxxxM', 'UTC') = toDateTime('2010-02-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('1970', 'xxxx', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('1969', 'xxxx', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('2107', 'xxxx', 'UTC'); -- { serverError LOGICAL_ERROR } +-- century of era +select parseDateTimeInJodaSyntax('20', 'CC', 'UTC') = toDateTime('2000-01-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('21', 'CC', 'UTC') = toDateTime('2100-01-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('19', 'CC', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('22', 'CC', 'UTC'); -- { serverError LOGICAL_ERROR } +-- month +select parseDateTimeInJodaSyntax('1', 'M', 'UTC') = toDateTime('2000-01-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax(' 7', ' MM', 'UTC') = toDateTime('2000-07-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('11', 'M', 'UTC') = toDateTime('2000-11-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('10-', 'M-', 'UTC') = toDateTime('2000-10-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('-12-', '-M-', 'UTC') = toDateTime('2000-12-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('0', 'M', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('13', 'M', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('12345', 'M', 'UTC'); -- { serverError LOGICAL_ERROR } +--- Ensure MMM and MMMM specifiers consume both short- and long-form month names +select parseDateTimeInJodaSyntax('Aug', 'MMM', 'UTC') = toDateTime('2000-08-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('AuG', 'MMM', 'UTC') = toDateTime('2000-08-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('august', 'MMM', 'UTC') = toDateTime('2000-08-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('Aug', 'MMMM', 'UTC') = toDateTime('2000-08-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('AuG', 'MMMM', 'UTC') = toDateTime('2000-08-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('august', 'MMMM', 'UTC') = toDateTime('2000-08-01', 'UTC'); +1 +--- invalid month names +select parseDateTimeInJodaSyntax('Decembr', 'MMM', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('Decembr', 'MMMM', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('Decemberary', 'MMM', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('Decemberary', 'MMMM', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('asdf', 'MMM', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('asdf', 'MMMM', 'UTC'); -- { serverError LOGICAL_ERROR } +-- day of month +select parseDateTimeInJodaSyntax('1', 'd', 'UTC') = toDateTime('2000-01-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('7 ', 'dd ', 'UTC') = toDateTime('2000-01-07', 'UTC'); +1 +select parseDateTimeInJodaSyntax('/11', '/dd', 'UTC') = toDateTime('2000-01-11', 'UTC'); +1 +select parseDateTimeInJodaSyntax('/31/', '/d/', 'UTC') = toDateTime('2000-01-31', 'UTC'); +1 +select parseDateTimeInJodaSyntax('0', 'd', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('32', 'd', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('12345', 'd', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('02-31', 'M-d', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('04-31', 'M-d', 'UTC'); -- { serverError LOGICAL_ERROR } +--- Ensure all days of month are checked against final selected month +select parseDateTimeInJodaSyntax('2 31 1', 'M d M', 'UTC') = toDateTime('2000-01-31', 'UTC'); +1 +select parseDateTimeInJodaSyntax('1 31 20 2', 'M d d M', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('2 31 20 4', 'M d d M', 'UTC'); -- { serverError LOGICAL_ERROR } +--- Leap year +select parseDateTimeInJodaSyntax('2020-02-29', 'YYYY-M-d', 'UTC') = toDateTime('2020-02-29', 'UTC'); +1 +select parseDateTimeInJodaSyntax('2001-02-29', 'YYYY-M-d', 'UTC'); -- { serverError LOGICAL_ERROR } +-- day of year +select parseDateTimeInJodaSyntax('1', 'D', 'UTC') = toDateTime('2000-01-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('7 ', 'DD ', 'UTC') = toDateTime('2000-01-07', 'UTC'); +1 +select parseDateTimeInJodaSyntax('/11', '/DD', 'UTC') = toDateTime('2000-01-11', 'UTC'); +1 +select parseDateTimeInJodaSyntax('/31/', '/DDD/', 'UTC') = toDateTime('2000-01-31', 'UTC'); +1 +select parseDateTimeInJodaSyntax('32', 'D', 'UTC') = toDateTime('2000-02-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('60', 'D', 'UTC') = toDateTime('2000-02-29', 'UTC'); +1 +select parseDateTimeInJodaSyntax('365', 'D', 'UTC') = toDateTime('2000-12-30', 'UTC'); +1 +select parseDateTimeInJodaSyntax('366', 'D', 'UTC') = toDateTime('2000-12-31', 'UTC'); +1 +select parseDateTimeInJodaSyntax('1999 1', 'yyyy D', 'UTC') = toDateTime('1999-01-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('1999 7 ', 'yyyy DD ', 'UTC') = toDateTime('1999-01-07', 'UTC'); +1 +select parseDateTimeInJodaSyntax('1999 /11', 'yyyy /DD', 'UTC') = toDateTime('1999-01-11', 'UTC'); +1 +select parseDateTimeInJodaSyntax('1999 /31/', 'yyyy /DD/', 'UTC') = toDateTime('1999-01-31', 'UTC'); +1 +select parseDateTimeInJodaSyntax('1999 32', 'yyyy D', 'UTC') = toDateTime('1999-02-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('1999 60', 'yyyy D', 'UTC') = toDateTime('1999-03-01', 'UTC'); +1 +select parseDateTimeInJodaSyntax('1999 365', 'yyyy D', 'UTC') = toDateTime('1999-12-31', 'UTC'); +1 +select parseDateTimeInJodaSyntax('1999 366', 'yyyy D', 'UTC'); -- { serverError LOGICAL_ERROR } +--- Ensure all days of year are checked against final selected year +select parseDateTimeInJodaSyntax('2001 366 2000', 'yyyy D yyyy', 'UTC') = toDateTime('2000-12-31', 'UTC'); +1 +select parseDateTimeInJodaSyntax('2000 366 2001', 'yyyy D yyyy', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('0', 'D', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('367', 'D', 'UTC'); -- { serverError LOGICAL_ERROR } +-- hour of day +select parseDateTimeInJodaSyntax('7', 'H', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('23', 'HH', 'UTC') = toDateTime('1970-01-01 23:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('0', 'HHH', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('10', 'HHHHHHHH', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); +1 +--- invalid hour od day +select parseDateTimeInJodaSyntax('24', 'H', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('-1', 'H', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('123456789', 'H', 'UTC'); -- { serverError LOGICAL_ERROR } +-- clock hour of day +select parseDateTimeInJodaSyntax('7', 'k', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('24', 'kk', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('1', 'kkk', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('10', 'kkkkkkkk', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); +1 +-- invalid clock hour of day +select parseDateTimeInJodaSyntax('25', 'k', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('0', 'k', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('123456789', 'k', 'UTC'); -- { serverError LOGICAL_ERROR } +-- hour of half day +select parseDateTimeInJodaSyntax('7', 'K', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('11', 'KK', 'UTC') = toDateTime('1970-01-01 11:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('0', 'KKK', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('10', 'KKKKKKKK', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); +1 +-- invalid hour of half day +select parseDateTimeInJodaSyntax('12', 'K', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('-1', 'K', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('123456789', 'K', 'UTC'); -- { serverError LOGICAL_ERROR } +-- clock hour of half day +select parseDateTimeInJodaSyntax('7', 'h', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('12', 'hh', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('1', 'hhh', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('10', 'hhhhhhhh', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); +1 +-- invalid clock hour of half day +select parseDateTimeInJodaSyntax('13', 'h', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('0', 'h', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('123456789', 'h', 'UTC'); -- { serverError LOGICAL_ERROR } +-- half of day +--- Half of day has no effect if hour or clockhour of day is provided hour of day tests +select parseDateTimeInJodaSyntax('7 PM', 'H a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('7 AM', 'H a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('7 pm', 'H a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('7 am', 'H a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('0 PM', 'H a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('0 AM', 'H a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('0 pm', 'H a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('0 am', 'H a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('7 PM', 'k a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('7 AM', 'k a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('7 pm', 'k a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('7 am', 'k a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('24 PM', 'k a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('24 AM', 'k a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('24 pm', 'k a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('24 am', 'k a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +-- Half of day has effect if hour or clockhour of halfday is provided +select parseDateTimeInJodaSyntax('0 PM', 'K a', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('0 AM', 'K a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('6 PM', 'K a', 'UTC') = toDateTime('1970-01-01 18:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('6 AM', 'K a', 'UTC') = toDateTime('1970-01-01 06:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('11 PM', 'K a', 'UTC') = toDateTime('1970-01-01 23:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('11 AM', 'K a', 'UTC') = toDateTime('1970-01-01 11:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('1 PM', 'h a', 'UTC') = toDateTime('1970-01-01 13:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('1 AM', 'h a', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('6 PM', 'h a', 'UTC') = toDateTime('1970-01-01 18:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('6 AM', 'h a', 'UTC') = toDateTime('1970-01-01 06:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('12 PM', 'h a', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('12 AM', 'h a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +-- time gives precendent to most recent time specifier +select parseDateTimeInJodaSyntax('0 1 AM', 'H h a', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('12 1 PM', 'H h a', 'UTC') = toDateTime('1970-01-01 13:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('1 AM 0', 'h a H', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('1 AM 12', 'h a H', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); +1 +-- minute +select parseDateTimeInJodaSyntax('8', 'm', 'UTC') = toDateTime('1970-01-01 00:08:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('59', 'mm', 'UTC') = toDateTime('1970-01-01 00:59:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('0/', 'mmm/', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('60', 'm', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('-1', 'm', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('123456789', 'm', 'UTC'); -- { serverError LOGICAL_ERROR } +-- second +select parseDateTimeInJodaSyntax('9', 's', 'UTC') = toDateTime('1970-01-01 00:00:09', 'UTC'); +1 +select parseDateTimeInJodaSyntax('58', 'ss', 'UTC') = toDateTime('1970-01-01 00:00:58', 'UTC'); +1 +select parseDateTimeInJodaSyntax('0/', 's/', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTimeInJodaSyntax('60', 's', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('-1', 's', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('123456789', 's', 'UTC'); -- { serverError LOGICAL_ERROR } diff --git a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql index 841be265f3d..2f2b0d26b2c 100644 --- a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql +++ b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql @@ -1,3 +1,4 @@ +-- { echoOn } -- empty select parseDateTimeInJodaSyntax(' ', ' ', 'UTC') = toDateTime('1970-01-01', 'UTC'); @@ -20,7 +21,7 @@ select parseDateTimeInJodaSyntax('2106', 'YYYY', 'UTC') = toDateTime('2106-01-01 select parseDateTimeInJodaSyntax('1970', 'YYYY', 'UTC') = toDateTime('1970-01-01', 'UTC'); select parseDateTimeInJodaSyntax('1969', 'YYYY', 'UTC'); -- { serverError LOGICAL_ERROR } select parseDateTimeInJodaSyntax('2107', 'YYYY', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('+1999', 'YYYY', 'UTC') -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('+1999', 'YYYY', 'UTC'); -- { serverError LOGICAL_ERROR } select parseDateTimeInJodaSyntax('12', 'YY', 'UTC') = toDateTime('2012-01-01', 'UTC'); select parseDateTimeInJodaSyntax('69', 'YY', 'UTC') = toDateTime('2069-01-01', 'UTC'); @@ -54,7 +55,7 @@ select parseDateTimeInJodaSyntax('12', 'xx', 'UTC') = toDateTime('2012-01-02', ' select parseDateTimeInJodaSyntax('69', 'xx', 'UTC') = toDateTime('2068-12-31', 'UTC'); select parseDateTimeInJodaSyntax('99', 'xx', 'UTC') = toDateTime('1999-01-04', 'UTC'); select parseDateTimeInJodaSyntax('01', 'xx', 'UTC') = toDateTime('2001-01-01', 'UTC'); -select parseDateTimeInJodaSyntax('+10', 'xx', 'UTC') = toDateTime('2001-01-04', 'UTC'); +select parseDateTimeInJodaSyntax('+10', 'xx', 'UTC') = toDateTime('2010-01-04', 'UTC'); select parseDateTimeInJodaSyntax('+99 01', 'xx ww', 'UTC') = toDateTime('1999-01-04', 'UTC'); select parseDateTimeInJodaSyntax('+99 02', 'xx ww', 'UTC') = toDateTime('1999-01-11', 'UTC'); select parseDateTimeInJodaSyntax('10 +10', 'ww xx', 'UTC') = toDateTime('2010-03-08', 'UTC'); @@ -106,12 +107,12 @@ select parseDateTimeInJodaSyntax('12345', 'd', 'UTC'); -- { serverError LOGICAL_ select parseDateTimeInJodaSyntax('02-31', 'M-d', 'UTC'); -- { serverError LOGICAL_ERROR } select parseDateTimeInJodaSyntax('04-31', 'M-d', 'UTC'); -- { serverError LOGICAL_ERROR } --- Ensure all days of month are checked against final selected month -select parseDateTimeInJodaSyntax('2 31 1', 'M d M') = toDateTime('2000-01-31', 'UTC'); -select parseDateTimeInJodaSyntax('1 31 20 2', 'M d d M'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('2 31 20 4', 'M d d M'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('2 31 1', 'M d M', 'UTC') = toDateTime('2000-01-31', 'UTC'); +select parseDateTimeInJodaSyntax('1 31 20 2', 'M d d M', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('2 31 20 4', 'M d d M', 'UTC'); -- { serverError LOGICAL_ERROR } --- Leap year -select parseDateTimeInJodaSyntax('2020-02-29', 'YYYY-M-d') = toDateTime('2020-02-29', 'UTC'); -select parseDateTimeInJodaSyntax('2001-02-29', 'YYYY-M-d'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('2020-02-29', 'YYYY-M-d', 'UTC') = toDateTime('2020-02-29', 'UTC'); +select parseDateTimeInJodaSyntax('2001-02-29', 'YYYY-M-d', 'UTC'); -- { serverError LOGICAL_ERROR } -- day of year select parseDateTimeInJodaSyntax('1', 'D', 'UTC') = toDateTime('2000-01-01', 'UTC'); @@ -136,3 +137,97 @@ select parseDateTimeInJodaSyntax('2000 366 2001', 'yyyy D yyyy', 'UTC'); -- { se select parseDateTimeInJodaSyntax('0', 'D', 'UTC'); -- { serverError LOGICAL_ERROR } select parseDateTimeInJodaSyntax('367', 'D', 'UTC'); -- { serverError LOGICAL_ERROR } +-- hour of day +select parseDateTimeInJodaSyntax('7', 'H', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('23', 'HH', 'UTC') = toDateTime('1970-01-01 23:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('0', 'HHH', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('10', 'HHHHHHHH', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); +--- invalid hour od day +select parseDateTimeInJodaSyntax('24', 'H', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('-1', 'H', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('123456789', 'H', 'UTC'); -- { serverError LOGICAL_ERROR } + +-- clock hour of day +select parseDateTimeInJodaSyntax('7', 'k', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('24', 'kk', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('1', 'kkk', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('10', 'kkkkkkkk', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); +-- invalid clock hour of day +select parseDateTimeInJodaSyntax('25', 'k', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('0', 'k', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('123456789', 'k', 'UTC'); -- { serverError LOGICAL_ERROR } + +-- hour of half day +select parseDateTimeInJodaSyntax('7', 'K', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('11', 'KK', 'UTC') = toDateTime('1970-01-01 11:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('0', 'KKK', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('10', 'KKKKKKKK', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); +-- invalid hour of half day +select parseDateTimeInJodaSyntax('12', 'K', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('-1', 'K', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('123456789', 'K', 'UTC'); -- { serverError LOGICAL_ERROR } + +-- clock hour of half day +select parseDateTimeInJodaSyntax('7', 'h', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('12', 'hh', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('1', 'hhh', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('10', 'hhhhhhhh', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); +-- invalid clock hour of half day +select parseDateTimeInJodaSyntax('13', 'h', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('0', 'h', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('123456789', 'h', 'UTC'); -- { serverError LOGICAL_ERROR } + +-- half of day +--- Half of day has no effect if hour or clockhour of day is provided hour of day tests +select parseDateTimeInJodaSyntax('7 PM', 'H a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('7 AM', 'H a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('7 pm', 'H a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('7 am', 'H a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('0 PM', 'H a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('0 AM', 'H a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('0 pm', 'H a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('0 am', 'H a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('7 PM', 'k a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('7 AM', 'k a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('7 pm', 'k a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('7 am', 'k a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('24 PM', 'k a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('24 AM', 'k a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('24 pm', 'k a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('24 am', 'k a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +-- Half of day has effect if hour or clockhour of halfday is provided +select parseDateTimeInJodaSyntax('0 PM', 'K a', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('0 AM', 'K a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('6 PM', 'K a', 'UTC') = toDateTime('1970-01-01 18:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('6 AM', 'K a', 'UTC') = toDateTime('1970-01-01 06:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('11 PM', 'K a', 'UTC') = toDateTime('1970-01-01 23:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('11 AM', 'K a', 'UTC') = toDateTime('1970-01-01 11:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('1 PM', 'h a', 'UTC') = toDateTime('1970-01-01 13:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('1 AM', 'h a', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('6 PM', 'h a', 'UTC') = toDateTime('1970-01-01 18:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('6 AM', 'h a', 'UTC') = toDateTime('1970-01-01 06:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('12 PM', 'h a', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('12 AM', 'h a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +-- time gives precendent to most recent time specifier +select parseDateTimeInJodaSyntax('0 1 AM', 'H h a', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('12 1 PM', 'H h a', 'UTC') = toDateTime('1970-01-01 13:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('1 AM 0', 'h a H', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('1 AM 12', 'h a H', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); + +-- minute +select parseDateTimeInJodaSyntax('8', 'm', 'UTC') = toDateTime('1970-01-01 00:08:00', 'UTC'); +select parseDateTimeInJodaSyntax('59', 'mm', 'UTC') = toDateTime('1970-01-01 00:59:00', 'UTC'); +select parseDateTimeInJodaSyntax('0/', 'mmm/', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('60', 'm', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('-1', 'm', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('123456789', 'm', 'UTC'); -- { serverError LOGICAL_ERROR } + +-- second +select parseDateTimeInJodaSyntax('9', 's', 'UTC') = toDateTime('1970-01-01 00:00:09', 'UTC'); +select parseDateTimeInJodaSyntax('58', 'ss', 'UTC') = toDateTime('1970-01-01 00:00:58', 'UTC'); +select parseDateTimeInJodaSyntax('0/', 's/', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTimeInJodaSyntax('60', 's', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('-1', 's', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('123456789', 's', 'UTC'); -- { serverError LOGICAL_ERROR } + +-- { echoOff } From ad48ad37878e0abd5b5fc24cc17c1d44857a2a20 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 28 Feb 2023 23:02:02 +0800 Subject: [PATCH 088/559] some refactor works --- src/Functions/parseDateTime.cpp | 742 ++++++++++++++++---------------- 1 file changed, 379 insertions(+), 363 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 63c0f98d53d..20f9838ad2f 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -19,7 +19,7 @@ namespace using Pos = const char *; constexpr std::string_view weekdaysShort[] = {"sun", "mon", "tue", "wed", "thu", "fri", "sat"}; - constexpr std::string_view weekdaysFull[] = {"sunday", "monday", "tuesday", "wednesday", "thursday", "friday", "saturday"}; + // constexpr std::string_view weekdaysFull[] = {"sunday", "monday", "tuesday", "wednesday", "thursday", "friday", "saturday"}; constexpr std::string_view monthsShort[] = {"jan", "feb", "mar", "apr", "may", "jun", "jul", "aug", "sep", "oct", "nov", "dec"}; const std::unordered_map> dayOfWeekMap{ {"mon", {"day", 1}}, @@ -65,15 +65,7 @@ namespace constexpr Int32 minYear = 1970; constexpr Int32 maxYear = 2106; - /// Counts the number of literal characters in Joda format string until the next closing literal - /// sequence single quote. Returns -1 if no literal single quote was found. - /// In Joda format string(https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html) - /// literal content must be quoted with single quote. and two single quote means literal with one single quote. - /// For example: - /// Format string: "'aaaa'", unescaped literal: "aaaa"; - /// Format string: "'aa''aa'", unescaped literal: "aa'aa"; - /// Format string: "'aaa''aa" is not valid because of missing of end single quote. - [[maybe_unused]] Int64 numLiteralChars(const char * cur, const char * end) + Int64 numLiteralChars(const char * cur, const char * end) { bool found = false; Int64 count = 0; @@ -133,6 +125,202 @@ namespace std::vector day_of_month_values; std::vector day_of_year_values; + void setCentrury(Int32 century) + { + if (century < 19 || century > 21) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for century must be in the range [19, 21]", century); + + century_format = true; + year = 100 * century; + has_year = true; + } + + void setDayOfWeek(Int32 day_of_week_) + { + if (day_of_week_ < 1 || day_of_week_ > 7) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for day of week must be in the range [1, 7]", day_of_week_); + + day_of_week = day_of_week_; + week_date_format = true; + day_of_year_format = false; + if (!has_year) + { + has_year = true; + year = 2000; + } + } + + void setMonth(Int32 month_) + { + if (month_ < 1 || month_ > 12) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for month of year must be in the range [1, 12]", month_); + + month = month_; + week_date_format = false; + day_of_year_format = false; + if (!has_year) + { + has_year = true; + year = 2000; + } + } + + void appendDayOfMonth(Int32 day_of_month) + { + if (day_of_month < 1 || day_of_month > 31) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for day of month must be in the range [1, 31]", day_of_month); + + day_of_month_values.push_back(day_of_month); + day = day_of_month; + week_date_format = false; + day_of_year_format = false; + if (!has_year) + { + has_year = true; + year = 2000; + } + } + + void appendDayOfYear(Int32 day_of_year_) + { + if (day_of_year_ < 1 || day_of_year_ > 366) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for day of year must be in the range [1, 366]", day_of_year_); + + day_of_year_values.push_back(day_of_year_); + day_of_year = day_of_year_; + day_of_year_format = true; + week_date_format = false; + if (!has_year) + { + has_year = true; + year = 2000; + } + } + + void setYear2(Int32 year_, bool is_year_of_era_ = false, bool is_week_year = false) + { + if (year_ >= 70 && year_ < 100) + year_ += 1900; + else if (year_ >= 0 && year_ < 70) + year_ += 2000; + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for year2 must be in the range [0, 99]", year_); + + setYear(year_, is_year_of_era_, is_week_year); + } + + void setYear(Int32 year_, bool is_year_of_era_ = false, bool is_week_year = false) + { + if (year_ < minYear || year_ > maxYear) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for year must be in the range [{}, {}]", year_, minYear, maxYear); + + year = year_; + century_format = false; + has_year = true; + is_year_of_era = is_year_of_era_; + if (is_week_year) + { + week_date_format = true; + day_of_year_format = false; + } + } + + void setWeek(Int32 week_) + { + if (week_ < 1 || week_ > 53) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for week of week year must be in the range [1, 53]", week_); + + week = week_; + week_date_format = true; + day_of_year_format = false; + if (!has_year) + { + has_year = true; + year = 2000; + } + } + + void setMinute(Int32 minute_) + { + if (minute_ < 0 || minute_ > 59) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for minute must be in the range [0, 59]", minute_); + + minute = minute_; + } + + void setSecond(Int32 second_) + { + if (second_ < 0 || second_ > 59) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for second must be in the range [0, 59]", second_); + + second = second_; + } + + void setEra(String & text) + { + Poco::toLowerInPlace(text); + if (text == "ad") + is_ad = true; + else if (text == "bc") + is_ad = false; + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown era {}", text); + } + + void setAMPM(String & text) + { + Poco::toLowerInPlace(text); + if (text == "am") + is_am = true; + else if (text == "pm") + is_am = false; + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown half day of day: {}", text); + } + + void setHour(Int32 hour_, bool is_hour_of_half_day_ = false, bool is_clock_hour_ = false) + { + Int32 max_hour; + Int32 min_hour; + Int32 new_hour = hour; + if (!is_hour_of_half_day_ && !is_clock_hour_) + { + max_hour = 23; + min_hour = 0; + } + else if (!is_hour_of_half_day_ && is_clock_hour_) + { + max_hour = 24; + min_hour = 1; + new_hour = hour_ % 24; + } + else if (is_hour_of_half_day_ && !is_clock_hour_) + { + max_hour = 11; + min_hour = 0; + } + else + { + max_hour = 12; + min_hour = 1; + new_hour = hour_ % 12; + } + + if (hour_ < min_hour || hour_ > max_hour) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Value {} for hour must be in the range [{}, {}] if_hour_of_half_day={} and is_clock_hour={}", + hour, + max_hour, + min_hour, + is_hour_of_half_day_, + is_clock_hour_); + + hour = new_hour; + is_hour_of_half_day = is_hour_of_half_day_; + is_clock_hour = is_clock_hour_; + } + /// For debug [[maybe_unused]] String toString() const { @@ -308,7 +496,6 @@ namespace } }; - struct ParseDateTimeTraits { enum class ParseSyntax @@ -517,24 +704,17 @@ namespace { ensureSpace(cur, end, 3, "Parsing DayOfWeekTextShort requires size >= 3"); - String str(cur, 3); - Poco::toLowerInPlace(str); + String text(cur, 3); + Poco::toLowerInPlace(text); Int32 i = 0; for (; i < 7; ++i) - if (str == weekdaysShort[i]) + if (text == weekdaysShort[i]) break; if (i == 7) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown short week text"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown short week text {}", text); - date.day_of_week = i + 1; - date.week_date_format = true; - date.day_of_year_format = false; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } + date.setDayOfWeek(i+1); cur += 3; return cur; } @@ -550,146 +730,109 @@ namespace for (; i < 12; ++i) if (str == monthsShort[i]) break; - if (i == 12) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown short month text"); - date.month = i + 1; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } + date.setMonth(i+1); cur += 3; return cur; } static Pos mysqlMonth(Pos cur, Pos end, Date & date) { - cur = readNumber2(cur, end, date.month); - if (date.month < 1 || date.month > 12) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for month must be in the range [1, 12]", date.month); - - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } + Int32 month; + cur = readNumber2(cur, end, month); + date.setMonth(month); return cur; } static Pos mysqlCentury(Pos cur, Pos end, Date & date) { - Int32 centuray; - cur = readNumber2(cur, end, centuray); - date.century_format = true; - date.year = centuray * 100; - date.has_year = true; + Int32 century; + cur = readNumber2(cur, end, century); + date.setCentrury(century); return cur; } static Pos mysqlDayOfMonth(Pos cur, Pos end, Date & date) { - cur = readNumber2(cur, end, date.day); - date.day_of_month_values.push_back(date.day); - date.week_date_format = false; - date.day_of_year_format = false; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } + Int32 day_of_month; + cur = readNumber2(cur, end, day_of_month); + date.appendDayOfMonth(day_of_month); return cur; } static Pos mysqlAmericanDate(Pos cur, Pos end, Date & date) { - cur = readNumber2(cur, end, date.month); + Int32 month; + cur = readNumber2(cur, end, month); cur = assertChar(cur, end, '/'); + date.setMonth(month); - cur = readNumber2(cur, end, date.day); + Int32 day; + cur = readNumber2(cur, end, day); cur = assertChar(cur, end, '/'); + date.appendDayOfMonth(day); - cur = readNumber2(cur, end, date.year); - cur = assertChar(cur, end, '/'); - - date.week_date_format = false; - date.day_of_year_format = false; - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; + Int32 year; + cur = readNumber2(cur, end, year); + date.setYear(year); return cur; } - static Pos mysqlDayOfMonthSpacePadded(Pos cur, Pos end, Date & date) { ensureSpace(cur, end, 2, "mysqlDayOfMonthSpacePadded requires size >= 2"); - date.day = *cur == ' ' ? 0 : (*cur - '0'); + Int32 day_of_month = *cur == ' ' ? 0 : (*cur - '0'); ++cur; - date.day = 10 * date.day + (*cur - '0'); + day_of_month = 10 * day_of_month + (*cur - '0'); ++cur; - date.week_date_format = false; - date.day_of_year_format = false; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } + date.appendDayOfMonth(day_of_month); return cur; } static Pos mysqlISO8601Date(Pos cur, Pos end, Date & date) { - cur = readNumber4(cur, end, date.year); + Int32 year; + cur = readNumber4(cur, end, year); cur = assertChar(cur, end, '-'); - cur = readNumber2(cur, end, date.month); + date.setYear(year); + + Int32 month; + cur = readNumber2(cur, end, month); cur = assertChar(cur, end, '-'); - cur = readNumber2(cur, end, date.day); + date.setMonth(month); - date.week_date_format = false; - date.day_of_year_format = false; - - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; + Int32 day; + cur = readNumber2(cur, end, day); + date.appendDayOfMonth(day); return cur; } static Pos mysqlISO8601Year2(Pos cur, Pos end, Date & date) { - cur = readNumber2(cur, end, date.year); - date.year += 2000; - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; + Int32 year2; + cur = readNumber2(cur, end, year2); + date.setYear2(year2); return cur; } static Pos mysqlISO8601Year4(Pos cur, Pos end, Date & date) { - cur = readNumber4(cur, end, date.year); - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; + Int32 year; + cur = readNumber4(cur, end, year); + date.setYear(year); return cur; } static Pos mysqlDayOfYear(Pos cur, Pos end, Date & date) { - cur = readNumber3(cur, end, date.day_of_year); - - date.day_of_year_values.push_back(date.day_of_year); - date.day_of_year_format = true; - date.week_date_format = false; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } + Int32 day_of_year; + cur = readNumber3(cur, end, day_of_year); + date.appendDayOfYear(day_of_year); return cur; } @@ -697,75 +840,72 @@ namespace { ensureSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1"); - date.day_of_week = *cur - '0'; - date.week_date_format = true; - date.day_of_year_format = false; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } + date.setDayOfWeek(*cur - '0'); + ++cur; return cur; } static Pos mysqlISO8601Week(Pos cur, Pos end, Date & date) { - cur = readNumber2(cur, end, date.week); - date.week_date_format = true; - date.day_of_year_format = false; - if (date.has_year) - { - date.has_year = true; - date.year = 2000; - } + Int32 week; + cur = readNumber2(cur, end, week); + date.setWeek(week); return cur; } static Pos mysqlDayOfWeek0To6(Pos cur, Pos end, Date & date) { - cur = mysqlDayOfWeek(cur, end, date); - if (date.day_of_week == 0) - date.day_of_week = 7; + ensureSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1"); + Int32 day_of_week = *cur - '0'; + if (day_of_week == 0) + day_of_week = 7; + + date.setDayOfWeek(day_of_week); + ++cur; return cur; } static Pos mysqlDayOfWeekTextLong(Pos cur, Pos end, Date & date) { - mysqlDayOfWeekTextShort(cur, end, date); - auto expect_text = weekdaysFull[date.day_of_week - 1]; + ensureSpace(cur, end, 3, "jodaDayOfWeekText requires the first part size >= 3"); + String text1(cur, 3); + Poco::toLowerInPlace(text1); + auto it = dayOfWeekMap.find(text1); + if (it == dayOfWeekMap.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown day of week text: {}", text1); + cur += 3; - ensureSpace(cur, end, expect_text.size(), "mysqlDayOfWeekTextLong requires size >= " + std::to_string(expect_text.size())); - std::string_view text(cur, expect_text.size()); - if (text != expect_text) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown full day of week text {}", expect_text); + size_t left_size = it->second.first.size(); + ensureSpace(cur, end, left_size, "jodaDayOfWeekText requires the second parg size >= " + std::to_string(left_size)); + String text2(cur, left_size); + Poco::toLowerInPlace(text2); + if (text2 != it->second.first) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown day of week text: {}", text1 + text2); + cur += left_size; - cur += expect_text.size(); + date.setDayOfWeek(it->second.second); return cur; } static Pos mysqlYear2(Pos cur, Pos end, Date & date) { - cur = readNumber2(cur, end, date.year); - date.year += 2000; - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; + Int32 year2; + cur = readNumber2(cur, end, year2); + date.setYear2(year2); return cur; } static Pos mysqlYear4(Pos cur, Pos end, Date & date) { - cur = readNumber4(cur, end, date.year); - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; + Int32 year; + cur = readNumber4(cur, end, year); + date.setYear(year); return cur; } static Pos mysqlTimezoneOffset(Pos cur, Pos end, Date & date) { - /// TODO figure out what timezone_id mean ensureSpace(cur, end, 1, "Parse mysqlTimezoneOffset failed"); Int32 sign = 1; if (*cur == '-') @@ -784,10 +924,9 @@ namespace static Pos mysqlMinute(Pos cur, Pos end, Date & date) { - cur = readNumber2(cur, end, date.minute); - if (date.minute < 0 || date.minute > 59) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for minute must be in the range [0, 59]", date.minute); - + Int32 minute; + cur = readNumber2(cur, end, minute); + date.setMinute(minute); return cur; } @@ -795,83 +934,80 @@ namespace { ensureSpace(cur, end, 2, "mysqlAMPM requires size >= 2"); - std::string text(cur, 2); - Poco::toUpperInPlace(text); - if (text == "PM") - date.is_am = false; - else if (text == "AM") - date.is_am = true; - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Text should be AM or PM, but {} provided", text); - + String text(cur, 2); + date.setAMPM(text); cur += 2; return cur; } static Pos mysqlHHMM12(Pos cur, Pos end, Date & date) { - cur = readNumber2(cur, end, date.hour); - date.is_clock_hour = false; - date.is_hour_of_half_day = true; - + Int32 hour; + cur = readNumber2(cur, end, hour); cur = assertChar(cur, end, ':'); - cur = readNumber2(cur, end, date.minute); + date.setHour(hour, true, true); + + Int32 minute; + cur = readNumber2(cur, end, minute); cur = assertChar(cur, end, ' '); + date.setMinute(minute); + cur = mysqlAMPM(cur, end, date); return cur; } static Pos mysqlHHMM24(Pos cur, Pos end, Date & date) { - cur = readNumber2(cur, end, date.hour); - date.is_clock_hour = false; - date.is_hour_of_half_day = false; - + Int32 hour; + cur = readNumber2(cur, end, hour); cur = assertChar(cur, end, ':'); - cur = readNumber2(cur, end, date.minute); + date.setHour(hour, false, false); + + Int32 minute; + cur = readNumber2(cur, end, minute); + date.setMinute(minute); return cur; } static Pos mysqlSecond(Pos cur, Pos end, Date & date) { - cur = readNumber2(cur, end, date.second); - if (date.second < 0 || date.second > 59) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for second must be in the range [0,59]", date.second); + Int32 second; + cur = readNumber2(cur, end, second); + date.setSecond(second); return cur; } static Pos mysqlISO8601Time(Pos cur, Pos end, Date & date) { - cur = readNumber2(cur, end, date.hour); + Int32 hour; + cur = readNumber2(cur, end, hour); cur = assertChar(cur, end, ':'); - cur = readNumber2(cur, end, date.minute); - cur = assertChar(cur, end, ':'); - cur = readNumber2(cur, end, date.second); + date.setHour(hour, false, false); - date.is_clock_hour = false; - date.is_hour_of_half_day = false; + Int32 minute; + cur = readNumber2(cur, end, minute); + cur = assertChar(cur, end, ':'); + date.setMinute(minute); + + Int32 second; + cur = readNumber2(cur, end, second); + date.setSecond(second); return cur; } static Pos mysqlHour12(Pos cur, Pos end, Date & date) { - cur = readNumber2(cur, end, date.hour); - if (date.hour < 1 || date.hour > 12) - - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for mysql hour12 must be in the range [1,12]", date.hour); - date.is_hour_of_half_day = true; - date.is_clock_hour = false; + Int32 hour; + cur = readNumber2(cur, end, hour); + date.setHour(hour, true, true); return cur; } static Pos mysqlHour24(Pos cur, Pos end, Date & date) { - cur = readNumber2(cur, end, date.hour); - if (date.hour < 0 || date.hour > 23) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for mysql hour24 must be in the range [0,23]", date.hour); - - date.is_hour_of_half_day = false; - date.is_clock_hour = false; + Int32 hour; + cur = readNumber2(cur, end, hour); + date.setHour(hour, false, false); return cur; } @@ -953,97 +1089,52 @@ namespace { ensureSpace(cur, end, 2, "jodaEra requires size >= 2"); - String text(cur, 2); - Poco::toLowerInPlace(text); - if (text == "ad") - date.is_ad = true; - else if (text == "bc") - date.is_ad = false; - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown era {}", std::string(cur, 2)); - + String era(cur, 2); + date.setEra(era); cur += 2; return cur; } static Pos jodaCenturyOfEra(int repetitions, Pos cur, Pos end, Date & date) { - Int32 number; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, repetitions, number); - - if (number < 0 || number > 2922789) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for century of era must be in the range [0, 2922789]", number); - - date.century_format = true; - date.year = 100 * number; - date.has_year = true; + Int32 century; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, repetitions, century); + date.setCentrury(century); return cur; } static Pos jodaYearOfEra(int repetitions, Pos cur, Pos end, Date & date) { - Int32 number; - cur = readNumberWithVariableLength(cur, end, false, false, true, repetitions, repetitions, number); - - date.century_format = false; - date.is_year_of_era = true; - if (number > 292278993 || number < 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for year of era must be in the range [1, 292278993]", number); - - date.has_year = true; - date.year = number; + Int32 year_of_era; + cur = readNumberWithVariableLength(cur, end, false, false, true, repetitions, repetitions, year_of_era); + date.setYear(year_of_era, true); return cur; } static Pos jodaWeekYear(int repetitions, Pos cur, Pos end, Date & date) { - Int32 number; - cur = readNumberWithVariableLength(cur, end, true, true, true, repetitions, repetitions, number); - if (number < -292275054 || number > 292278993) - throw Exception( - ErrorCodes::LOGICAL_ERROR, "Value {} for week year must be in the range [-292275054,292278993]", number); - - date.year = number; - date.week_date_format = true; - date.day_of_year_format = false; - date.century_format = false; - date.has_year = true; + Int32 week_year; + cur = readNumberWithVariableLength(cur, end, true, true, true, repetitions, repetitions, week_year); + date.setYear(week_year, false, true); return cur; } static Pos jodaWeekOfWeekYear(int repetitions, Pos cur, Pos end, Date & date) { - Int32 number; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), number); - if (number < 1 || number > 52) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for week of week year must be in the range [1, 52]", number); - - date.week = number; - date.week_date_format = true; - date.day_of_year_format = false; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } + Int32 week; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), week); + date.setWeek(week); return cur; } static Pos jodaDayOfWeek1Based(int repetitions, Pos cur, Pos end, Date & date) { - Int32 number; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, repetitions, number); - if (number < 1 || number > 7) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for day of week 1-based must be in the range [1, 7]", number); + Int32 day_of_week; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, repetitions, day_of_week); + if (day_of_week < 1 || day_of_week > 7) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for day of week 1-based must be in the range [1, 7]", day_of_week); - date.day_of_week = number; - date.week_date_format = true; - date.day_of_year_format = false; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } + date.setDayOfWeek(day_of_week); return cur; } @@ -1056,16 +1147,17 @@ namespace auto it = dayOfWeekMap.find(text1); if (it == dayOfWeekMap.end()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown day of week text: {}", text1); - - date.day_of_week = it->second.second; cur += 3; - if (cur + it->second.first.size() <= end) + date.setDayOfWeek(it->second.second); + + size_t left_size = it->second.first.size(); + if (cur + left_size <= end) { - String text2(cur, it->second.first.size()); + String text2(cur, left_size); Poco::toLowerInPlace(text2); if (text2 == it->second.first) { - cur += it->second.first.size(); + cur += left_size; return cur; } } @@ -1074,79 +1166,47 @@ namespace static Pos jodaYear(int repetitions, Pos cur, Pos end, Date & date) { - Int32 number; - cur = readNumberWithVariableLength(cur, end, true, true, true, repetitions, repetitions, number); - if (number > 292278994 || number < -292275055) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for year must be in the range [-292275055,292278994]", number); - - date.century_format = false; - date.is_year_of_era = false; - date.has_year = true; - date.year = number; + Int32 year; + cur = readNumberWithVariableLength(cur, end, true, true, true, repetitions, repetitions, year); + date.setYear(year); return cur; } static Pos jodaDayOfYear(int repetitions, Pos cur, Pos end, Date & date) { - Int32 number; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 3), number); - if (number < 1 || number > 366) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for day of year must be in the range [1, 366]", number); - - date.day_of_year_values.push_back(number); - date.day_of_year = number; - date.day_of_year_format = true; - date.week_date_format = false; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } + Int32 day_of_year; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 3), day_of_year); + date.appendDayOfYear(day_of_year); return cur; } static Pos jodaMonthOfYear(int repetitions, Pos cur, Pos end, Date & date) { - Int32 number; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, 2, number); - if (number < 1 || number > 12) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for month of year must be in the range [1, 12]", number); - - date.month = number; - date.week_date_format = false; - date.day_of_year_format = false; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } + Int32 month; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, 2, month); + date.setMonth(month); return cur; } static Pos jodaMonthOfYearText(int, Pos cur, Pos end, Date & date) { ensureSpace(cur, end, 3, "jodaMonthOfYearText requires size >= 3"); - String text1(cur, 3); Poco::toLowerInPlace(text1); auto it = monthMap.find(text1); if (it == monthMap.end()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown month of year text: {}", text1); - - date.month = it->second.second; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } cur += 3; - if (cur + it->second.first.size() <= end) + date.setMonth(it->second.second); + + size_t left_size = it->second.first.size(); + if (cur + left_size <= end) { - String text2(cur, it->second.first.size()); + String text2(cur, left_size); Poco::toLowerInPlace(text2); if (text2 == it->second.first) { - cur += it->second.first.size(); + cur += left_size; return cur; } } @@ -1155,20 +1215,9 @@ namespace static Pos jodaDayOfMonth(int repetitions, Pos cur, Pos end, Date & date) { - Int32 number; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), number); - if (number < 1 || number > 31) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for day of month must be in the range [1, 31]", number); - - date.day_of_month_values.push_back(number); - date.day = number; - date.week_date_format = false; - date.day_of_year_format = false; - if (!date.has_year) - { - date.has_year = true; - date.year = 2000; - } + Int32 day_of_month; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), day_of_month); + date.appendDayOfMonth(day_of_month); return cur; } @@ -1177,89 +1226,56 @@ namespace ensureSpace(cur, end, 2, "jodaHalfDayOfDay requires size >= 2"); String text(cur, 2); - Poco::toLowerInPlace(text); - if (text == "am") - date.is_am = true; - else if (text == "pm") - date.is_am = false; - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown half day of day: {}", text); - + date.setAMPM(text); cur += 2; return cur; } static Pos jodaHourOfHalfDay(int repetitions, Pos cur, Pos end, Date & date) { - Int32 number; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), number); - if (number > 11 || number < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for hour of half day must be in the range [0, 11]", number); - - date.is_clock_hour = false; - date.is_hour_of_half_day = true; - date.hour = number; + Int32 hour; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), hour); + date.setHour(hour, true, false); return cur; } static Pos jodaClockHourOfHalfDay(int repetitions, Pos cur, Pos end, Date & date) { - Int32 number; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), number); - if (number > 12 || number < 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for clock hour of half day must be in the range [1, 12]", number); - - date.is_clock_hour = true; - date.is_hour_of_half_day = true; - date.hour = number % 12; + Int32 hour; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), hour); + date.setHour(hour, true, true); return cur; } static Pos jodaHourOfDay(int repetitions, Pos cur, Pos end, Date & date) { - Int32 number; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), number); - if (number > 23 || number < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for hour of day must be in the range [0, 23]", number); - - date.is_clock_hour = false; - date.is_hour_of_half_day = false; - date.hour = number; + Int32 hour; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), hour); + date.setHour(hour, false, false); return cur; } static Pos jodaClockHourOfDay(int repetitions, Pos cur, Pos end, Date & date) { - Int32 number; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), number); - if (number > 24 || number < 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for clock hour of day must be in the range [1, 24]", number); - - date.is_clock_hour = true; - date.is_hour_of_half_day = false; - date.hour = number % 24; + Int32 hour; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), hour); + date.setHour(hour, false, true); return cur; } static Pos jodaMinuteOfHour(int repetitions, Pos cur, Pos end, Date & date) { - Int32 number; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), number); - if (number > 59 || number < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for minute of hour must be in the range [0, 59]", number); - - date.minute = number; + Int32 minute; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), minute); + date.setMinute(minute); return cur; } static Pos jodaSecondOfMinute(int repetitions, Pos cur, Pos end, Date & date) { - Int32 number; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), number); - if (number > 59 || number < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for second of minute must be in the range [0, 59]", number); - - date.second = number; + Int32 second; + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), second); + date.setSecond(second); return cur; } }; From 5f649542a324612280da60a1de529158411201ec Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Tue, 28 Feb 2023 10:08:57 -0800 Subject: [PATCH 089/559] Fix murmurhash for s390x --- contrib/murmurhash/src/MurmurHash2.cpp | 52 +++++++++++++++---- contrib/murmurhash/src/MurmurHash3.cpp | 30 +++++++++-- .../0_stateless/00678_murmurhash.reference | 4 +- .../queries/0_stateless/00678_murmurhash.sql | 5 +- 4 files changed, 74 insertions(+), 17 deletions(-) diff --git a/contrib/murmurhash/src/MurmurHash2.cpp b/contrib/murmurhash/src/MurmurHash2.cpp index 1c4469b0a02..0bd0a352dc4 100644 --- a/contrib/murmurhash/src/MurmurHash2.cpp +++ b/contrib/murmurhash/src/MurmurHash2.cpp @@ -31,6 +31,40 @@ #define BIG_CONSTANT(x) (x##LLU) #endif // !defined(_MSC_VER) +// +//----------------------------------------------------------------------------- +// Block read - on little-endian machines this is a single load, +// while on big-endian or unknown machines the byte accesses should +// still get optimized into the most efficient instruction. +static inline uint32_t getblock ( const uint32_t * p ) +{ +#if defined(__BYTE_ORDER__) && (__BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__) + return *p; +#else + const uint8_t *c = (const uint8_t *)p; + return (uint32_t)c[0] | + (uint32_t)c[1] << 8 | + (uint32_t)c[2] << 16 | + (uint32_t)c[3] << 24; +#endif +} + +static inline uint64_t getblock ( const uint64_t * p ) +{ +#if defined(__BYTE_ORDER__) && (__BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__) + return *p; +#else + const uint8_t *c = (const uint8_t *)p; + return (uint64_t)c[0] | + (uint64_t)c[1] << 8 | + (uint64_t)c[2] << 16 | + (uint64_t)c[3] << 24 | + (uint64_t)c[4] << 32 | + (uint64_t)c[5] << 40 | + (uint64_t)c[6] << 48 | + (uint64_t)c[7] << 56; +#endif +} //----------------------------------------------------------------------------- @@ -52,7 +86,7 @@ uint32_t MurmurHash2 ( const void * key, size_t len, uint32_t seed ) while(len >= 4) { - uint32_t k = *(uint32_t*)data; + uint32_t k = getblock((const uint32_t *)data); k *= m; k ^= k >> r; @@ -105,7 +139,7 @@ uint64_t MurmurHash64A ( const void * key, size_t len, uint64_t seed ) while(data != end) { - uint64_t k = *data++; + uint64_t k = getblock(data++); k *= m; k ^= k >> r; @@ -151,12 +185,12 @@ uint64_t MurmurHash64B ( const void * key, size_t len, uint64_t seed ) while(len >= 8) { - uint32_t k1 = *data++; + uint32_t k1 = getblock(data++); k1 *= m; k1 ^= k1 >> r; k1 *= m; h1 *= m; h1 ^= k1; len -= 4; - uint32_t k2 = *data++; + uint32_t k2 = getblock(data++); k2 *= m; k2 ^= k2 >> r; k2 *= m; h2 *= m; h2 ^= k2; len -= 4; @@ -164,7 +198,7 @@ uint64_t MurmurHash64B ( const void * key, size_t len, uint64_t seed ) if(len >= 4) { - uint32_t k1 = *data++; + uint32_t k1 = getblock(data++); k1 *= m; k1 ^= k1 >> r; k1 *= m; h1 *= m; h1 ^= k1; len -= 4; @@ -215,7 +249,7 @@ uint32_t MurmurHash2A ( const void * key, size_t len, uint32_t seed ) while(len >= 4) { - uint32_t k = *(uint32_t*)data; + uint32_t k = getblock((const uint32_t *)data); mmix(h,k); @@ -278,7 +312,7 @@ public: while(len >= 4) { - uint32_t k = *(uint32_t*)data; + uint32_t k = getblock((const uint32_t *)data); mmix(m_hash,k); @@ -427,7 +461,7 @@ uint32_t MurmurHashAligned2 ( const void * key, size_t len, uint32_t seed ) while(len >= 4) { - d = *(uint32_t *)data; + d = getblock((const uint32_t *)data); t = (t >> sr) | (d << sl); uint32_t k = t; @@ -492,7 +526,7 @@ uint32_t MurmurHashAligned2 ( const void * key, size_t len, uint32_t seed ) { while(len >= 4) { - uint32_t k = *(uint32_t *)data; + uint32_t k = getblock((const uint32_t *)data); MIX(h,k,m); diff --git a/contrib/murmurhash/src/MurmurHash3.cpp b/contrib/murmurhash/src/MurmurHash3.cpp index cf5158e97ad..6573c470be3 100644 --- a/contrib/murmurhash/src/MurmurHash3.cpp +++ b/contrib/murmurhash/src/MurmurHash3.cpp @@ -55,14 +55,32 @@ inline uint64_t rotl64 ( uint64_t x, int8_t r ) FORCE_INLINE uint32_t getblock32 ( const uint32_t * p, int i ) { - uint32_t res; - memcpy(&res, p + i, sizeof(res)); - return res; +#if defined(__BYTE_ORDER__) && (__BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__) + return p[i]; +#else + const uint8_t *c = (const uint8_t *)&p[i]; + return (uint32_t)c[0] | + (uint32_t)c[1] << 8 | + (uint32_t)c[2] << 16 | + (uint32_t)c[3] << 24; +#endif } FORCE_INLINE uint64_t getblock64 ( const uint64_t * p, int i ) { +#if defined(__BYTE_ORDER__) && (__BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__) return p[i]; +#else + const uint8_t *c = (const uint8_t *)&p[i]; + return (uint64_t)c[0] | + (uint64_t)c[1] << 8 | + (uint64_t)c[2] << 16 | + (uint64_t)c[3] << 24 | + (uint64_t)c[4] << 32 | + (uint64_t)c[5] << 40 | + (uint64_t)c[6] << 48 | + (uint64_t)c[7] << 56; +#endif } //----------------------------------------------------------------------------- @@ -329,9 +347,13 @@ void MurmurHash3_x64_128 ( const void * key, const size_t len, h1 += h2; h2 += h1; - +#if defined(__BYTE_ORDER__) && (__BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__) ((uint64_t*)out)[0] = h1; ((uint64_t*)out)[1] = h2; +#else + ((uint64_t*)out)[0] = h2; + ((uint64_t*)out)[1] = h1; +#endif } //----------------------------------------------------------------------------- diff --git a/tests/queries/0_stateless/00678_murmurhash.reference b/tests/queries/0_stateless/00678_murmurhash.reference index 988c022f1bf..fb4a00ba046 100644 --- a/tests/queries/0_stateless/00678_murmurhash.reference +++ b/tests/queries/0_stateless/00678_murmurhash.reference @@ -25,5 +25,5 @@ 9631199822919835226 4334672815104069193 4334672815104069193 -6145F501578671E2877DBA2BE487AF7E -16FE7483905CCE7A85670E43E4678877 +1 +1 diff --git a/tests/queries/0_stateless/00678_murmurhash.sql b/tests/queries/0_stateless/00678_murmurhash.sql index 705c62480a0..cff3cc9996f 100644 --- a/tests/queries/0_stateless/00678_murmurhash.sql +++ b/tests/queries/0_stateless/00678_murmurhash.sql @@ -32,5 +32,6 @@ SELECT gccMurmurHash('foo'); SELECT gccMurmurHash('\x01'); SELECT gccMurmurHash(1); -SELECT hex(murmurHash3_128('foo')); -SELECT hex(murmurHash3_128('\x01')); +SELECT hex(murmurHash3_128('foo')) = hex(reverse(unhex('6145F501578671E2877DBA2BE487AF7E'))) or hex(murmurHash3_128('foo')) = '6145F501578671E2877DBA2BE487AF7E'; + +SELECT hex(murmurHash3_128('\x01')) = hex(reverse(unhex('16FE7483905CCE7A85670E43E4678877'))) or hex(murmurHash3_128('\x01')) = '16FE7483905CCE7A85670E43E4678877'; From 573c674dc2eeff10742c59f0863775aba223c056 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 1 Mar 2023 11:02:41 +0800 Subject: [PATCH 090/559] refactor some functions --- src/Functions/parseDateTime.cpp | 51 +++++++++---------- .../02668_parse_datetime.reference | 10 ++-- .../0_stateless/02668_parse_datetime.sql | 10 ++-- 3 files changed, 34 insertions(+), 37 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 20f9838ad2f..8c846323788 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -18,9 +18,9 @@ namespace { using Pos = const char *; - constexpr std::string_view weekdaysShort[] = {"sun", "mon", "tue", "wed", "thu", "fri", "sat"}; - // constexpr std::string_view weekdaysFull[] = {"sunday", "monday", "tuesday", "wednesday", "thursday", "friday", "saturday"}; - constexpr std::string_view monthsShort[] = {"jan", "feb", "mar", "apr", "may", "jun", "jul", "aug", "sep", "oct", "nov", "dec"}; + constexpr Int32 minYear = 1970; + constexpr Int32 maxYear = 2106; + const std::unordered_map> dayOfWeekMap{ {"mon", {"day", 1}}, {"tue", {"sday", 2}}, @@ -46,11 +46,19 @@ namespace {"dec", {"ember", 12}}, }; + /// key: month, value: total days of current month if current year is leap year. constexpr Int32 leapDays[] = {0, 31, 29, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}; + + /// key: month, value: total days of current month if current year is not leap year. constexpr Int32 normalDays[] = {0, 31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}; + /// key: month, value: cumulative days from Januray to current month(inclusive) if current year is leap year. constexpr Int32 cumulativeLeapDays[] = {0, 31, 60, 91, 121, 152, 182, 213, 244, 274, 305, 335, 366}; + + /// key: month, value: cumulative days from Januray to current month(inclusive) if current year is not leap year. constexpr Int32 cumulativeDays[] = {0, 31, 59, 90, 120, 151, 181, 212, 243, 273, 304, 334, 365}; + + /// key: year, value: cumulative days from epoch(1970-01-01) to the first day of current year(exclusive). constexpr Int32 cumulativeYearDays[] = {0, 365, 730, 1096, 1461, 1826, 2191, 2557, 2922, 3287, 3652, 4018, 4383, 4748, 5113, 5479, 5844, 6209, 6574, 6940, 7305, 7670, 8035, 8401, 8766, 9131, 9496, 9862, 10227, 10592, 10957, 11323, 11688, 12053, 12418, 12784, @@ -62,9 +70,6 @@ namespace 46021, 46387, 46752, 47117, 47482, 47847, 48212, 48577, 48942, 49308, 49673}; - constexpr Int32 minYear = 1970; - constexpr Int32 maxYear = 2106; - Int64 numLiteralChars(const char * cur, const char * end) { bool found = false; @@ -115,7 +120,6 @@ namespace Int32 hour = 0; Int32 minute = 0; Int32 second = 0; - // Int32 microsecond = 0; bool is_am = true; // AM -> true, PM -> false std::optional time_zone_offset; @@ -280,9 +284,11 @@ namespace void setHour(Int32 hour_, bool is_hour_of_half_day_ = false, bool is_clock_hour_ = false) { + std::cout << "set hour:" << hour_ << ",is_hour_of_half_day_:" << is_hour_of_half_day_ << ",is_clock_hour_:" << is_clock_hour_ + << std::endl; Int32 max_hour; Int32 min_hour; - Int32 new_hour = hour; + Int32 new_hour = hour_; if (!is_hour_of_half_day_ && !is_clock_hour_) { max_hour = 23; @@ -706,15 +712,10 @@ namespace String text(cur, 3); Poco::toLowerInPlace(text); - Int32 i = 0; - for (; i < 7; ++i) - if (text == weekdaysShort[i]) - break; - - if (i == 7) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown short week text {}", text); - - date.setDayOfWeek(i+1); + auto it = dayOfWeekMap.find(text); + if (it == dayOfWeekMap.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown day of week short text {}", text); + date.setDayOfWeek(it->second.second); cur += 3; return cur; } @@ -723,17 +724,13 @@ namespace { ensureSpace(cur, end, 3, "Parsing MonthOfYearTextShort requires size >= 3"); - String str(cur, 3); - Poco::toLowerInPlace(str); + String text(cur, 3); + Poco::toLowerInPlace(text); + auto it = monthMap.find(text); + if (it == monthMap.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown month of year short text {}", text); - Int32 i = 0; - for (; i < 12; ++i) - if (str == monthsShort[i]) - break; - if (i == 12) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because unknown short month text"); - - date.setMonth(i+1); + date.setMonth(it->second.second); cur += 3; return cur; } diff --git a/tests/queries/0_stateless/02668_parse_datetime.reference b/tests/queries/0_stateless/02668_parse_datetime.reference index 24df3a8bd76..f0678585a8e 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.reference +++ b/tests/queries/0_stateless/02668_parse_datetime.reference @@ -108,7 +108,7 @@ select parseDateTime('1234567', '%k', 'UTC'); -- { serverError LOGICAL_ERROR } -- hour of half day select parseDateTime('07', '%h', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); 1 -select parseDateTime('12', '%h', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); +select parseDateTime('12', '%h', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); 1 select parseDateTime('01', '%h', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); 1 @@ -119,7 +119,7 @@ select parseDateTime('13', '%h', 'UTC'); -- { serverError LOGICAL_ERROR } select parseDateTime('123456789', '%h', 'UTC'); -- { serverError LOGICAL_ERROR } select parseDateTime('07', '%I', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); 1 -select parseDateTime('12', '%I', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); +select parseDateTime('12', '%I', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); 1 select parseDateTime('01', '%I', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); 1 @@ -130,7 +130,7 @@ select parseDateTime('13', '%I', 'UTC'); -- { serverError LOGICAL_ERROR } select parseDateTime('123456789', '%I', 'UTC'); -- { serverError LOGICAL_ERROR } select parseDateTime('07', '%l', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); 1 -select parseDateTime('12', '%l', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); +select parseDateTime('12', '%l', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); 1 select parseDateTime('01', '%l', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); 1 @@ -164,9 +164,9 @@ select parseDateTime('06 PM', '%h %p', 'UTC') = toDateTime('1970-01-01 18:00:00' 1 select parseDateTime('06 AM', '%h %p', 'UTC') = toDateTime('1970-01-01 06:00:00', 'UTC'); 1 -select parseDateTime('12 PM', '%h %p', 'UTC') = toDateTime('1970-01-02 00:00:00', 'UTC'); +select parseDateTime('12 PM', '%h %p', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); 1 -select parseDateTime('12 AM', '%h %p', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); +select parseDateTime('12 AM', '%h %p', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); 1 -- minute select parseDateTime('08', '%i', 'UTC') = toDateTime('1970-01-01 00:08:00', 'UTC'); diff --git a/tests/queries/0_stateless/02668_parse_datetime.sql b/tests/queries/0_stateless/02668_parse_datetime.sql index ac8e68c4f93..4a6d3711c8c 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.sql +++ b/tests/queries/0_stateless/02668_parse_datetime.sql @@ -77,21 +77,21 @@ select parseDateTime('1234567', '%k', 'UTC'); -- { serverError LOGICAL_ERROR } -- hour of half day select parseDateTime('07', '%h', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); -select parseDateTime('12', '%h', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); +select parseDateTime('12', '%h', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); select parseDateTime('01', '%h', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); select parseDateTime('10', '%h', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); select parseDateTime('00', '%h', 'UTC'); -- { serverError LOGICAL_ERROR } select parseDateTime('13', '%h', 'UTC'); -- { serverError LOGICAL_ERROR } select parseDateTime('123456789', '%h', 'UTC'); -- { serverError LOGICAL_ERROR } select parseDateTime('07', '%I', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); -select parseDateTime('12', '%I', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); +select parseDateTime('12', '%I', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); select parseDateTime('01', '%I', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); select parseDateTime('10', '%I', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); select parseDateTime('00', '%I', 'UTC'); -- { serverError LOGICAL_ERROR } select parseDateTime('13', '%I', 'UTC'); -- { serverError LOGICAL_ERROR } select parseDateTime('123456789', '%I', 'UTC'); -- { serverError LOGICAL_ERROR } select parseDateTime('07', '%l', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); -select parseDateTime('12', '%l', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); +select parseDateTime('12', '%l', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); select parseDateTime('01', '%l', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); select parseDateTime('10', '%l', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); select parseDateTime('00', '%l', 'UTC'); -- { serverError LOGICAL_ERROR } @@ -111,8 +111,8 @@ select parseDateTime('01 PM', '%h %p', 'UTC') = toDateTime('1970-01-01 13:00:00' select parseDateTime('01 AM', '%h %p', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); select parseDateTime('06 PM', '%h %p', 'UTC') = toDateTime('1970-01-01 18:00:00', 'UTC'); select parseDateTime('06 AM', '%h %p', 'UTC') = toDateTime('1970-01-01 06:00:00', 'UTC'); -select parseDateTime('12 PM', '%h %p', 'UTC') = toDateTime('1970-01-02 00:00:00', 'UTC'); -select parseDateTime('12 AM', '%h %p', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); +select parseDateTime('12 PM', '%h %p', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); +select parseDateTime('12 AM', '%h %p', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -- minute select parseDateTime('08', '%i', 'UTC') = toDateTime('1970-01-01 00:08:00', 'UTC'); From d47452a1e5e7b67c7a275b540e1847be91931030 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 1 Mar 2023 11:25:38 +0800 Subject: [PATCH 091/559] remove useless code --- src/Functions/parseDateTime.cpp | 65 --------------------------------- 1 file changed, 65 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 8c846323788..be732e5a36b 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -69,7 +69,6 @@ namespace 39447, 39812, 40177, 40543, 40908, 41273, 41638, 42004, 42369, 42734, 43099, 43465, 43830, 44195, 44560, 44926, 45291, 45656, 46021, 46387, 46752, 47117, 47482, 47847, 48212, 48577, 48942, 49308, 49673}; - Int64 numLiteralChars(const char * cur, const char * end) { bool found = false; @@ -284,8 +283,6 @@ namespace void setHour(Int32 hour_, bool is_hour_of_half_day_ = false, bool is_clock_hour_ = false) { - std::cout << "set hour:" << hour_ << ",is_hour_of_half_day_:" << is_hour_of_half_day_ << ",is_clock_hour_:" << is_clock_hour_ - << std::endl; Int32 max_hour; Int32 min_hour; Int32 new_hour = hour_; @@ -444,7 +441,6 @@ namespace return res; } - Int64 checkAndGetDateTime(const DateLUTImpl & time_zone) { /// Era is BC and year of era is provided @@ -477,27 +473,20 @@ namespace else { days_since_epoch = daysSinceEpochFromDate(year, month, day); - std::cout << "year:" << year << "month:" << month << "day:" << day << std::endl; } - std::cout << "days_since_epoch:" << days_since_epoch << std::endl; Int64 seconds_since_epoch = days_since_epoch * 86400 + hour * 3600 + minute * 60 + second; - std::cout << "seconds_since_epoch:" << seconds_since_epoch << std::endl; /// Time zone is not specified, use local time zone if (!time_zone_offset) *time_zone_offset = time_zone.getOffsetAtStartOfEpoch(); - // std::cout << "timezonename:" << time_zone.getTimeZone() << std::endl; - // std::cout << "time_zone_offset:" << *time_zone_offset << time_zone.getOffsetAtStartOfEpoch() << std::endl; - // std::cout << "before timestamp:" << seconds_since_epoch << std::endl; /// Time zone is specified in format string. if (seconds_since_epoch >= *time_zone_offset) seconds_since_epoch -= *time_zone_offset; else throw Exception(ErrorCodes::LOGICAL_ERROR, "Seconds since epoch is negative"); - std::cout << "after adjustment:" << seconds_since_epoch << std::endl; return seconds_since_epoch; } }; @@ -576,7 +565,6 @@ namespace String format = getFormat(arguments); const auto * time_zone = getTimeZone(arguments).first; - std::cout << "timezonename:" << getTimeZone(arguments).second << std::endl; std::vector instructions; parseFormat(format, instructions); @@ -591,11 +579,7 @@ namespace Pos end = str_ref.data + str_ref.size; Date date; for (const auto & instruction : instructions) - { cur = instruction.perform(cur, end, date); - std::cout << "instruction:" << instruction.toString() << std::endl; - std::cout << "date:" << date.toString() << std::endl; - } // Ensure all input was consumed. if (cur < end) @@ -1581,108 +1565,60 @@ namespace { case 'G': instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaEra, repetitions)); - // reserve_size += repetitions <= 3 ? 2 : 13; break; case 'C': instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaCenturyOfEra, repetitions)); - /// Year range [1900, 2299] - // reserve_size += std::max(repetitions, 2); break; case 'Y': instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaYearOfEra, repetitions)); - /// Year range [1900, 2299] - // reserve_size += repetitions == 2 ? 2 : std::max(repetitions, 4); break; case 'x': - instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaWeekYear, repetitions)); - /// weekyear range [1900, 2299] - // reserve_size += std::max(repetitions, 4); break; case 'w': instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaWeekOfWeekYear, repetitions)); - /// Week of weekyear range [1, 52] - // reserve_size += std::max(repetitions, 2); break; case 'e': instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaDayOfWeek1Based, repetitions)); - /// Day of week range [1, 7] - // reserve_size += std::max(repetitions, 1); break; case 'E': instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaDayOfWeekText, repetitions)); - /// Maximum length of short name is 3, maximum length of full name is 9. - // reserve_size += repetitions <= 3 ? 3 : 9; break; case 'y': instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaYear, repetitions)); - /// Year range [1900, 2299] - // reserve_size += repetitions == 2 ? 2 : std::max(repetitions, 4); break; case 'D': instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaDayOfYear, repetitions)); - /// Day of year range [1, 366] - // reserve_size += std::max(repetitions, 3); break; case 'M': if (repetitions <= 2) - { instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaMonthOfYear, repetitions)); - /// Month of year range [1, 12] - // reserve_size += 2; - } else - { instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaMonthOfYearText, repetitions)); - /// Maximum length of short name is 3, maximum length of full name is 9. - // reserve_size += repetitions <= 3 ? 3 : 9; - } break; case 'd': instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaDayOfMonth, repetitions)); - /// Day of month range [1, 3] - // reserve_size += std::max(repetitions, 3); break; case 'a': - /// Default half day of day is "AM" instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaHalfDayOfDay, repetitions)); - // reserve_size += 2; break; case 'K': - /// Default hour of half day is 0 instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaHourOfHalfDay, repetitions)); - /// Hour of half day range [0, 11] - // reserve_size += std::max(repetitions, 2); break; case 'h': - /// Default clock hour of half day is 12 instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaClockHourOfHalfDay, repetitions)); - /// Clock hour of half day range [1, 12] - // reserve_size += std::max(repetitions, 2); break; case 'H': - /// Default hour of day is 0 instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaHourOfDay, repetitions)); - /// Hour of day range [0, 23] - // reserve_size += std::max(repetitions, 2); break; case 'k': - /// Default clock hour of day is 24 instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaClockHourOfDay, repetitions)); - /// Clock hour of day range [1, 24] - // reserve_size += std::max(repetitions, 2); break; case 'm': - /// Default minute of hour is 0 instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaMinuteOfHour, repetitions)); - /// Minute of hour range [0, 59] - // reserve_size += std::max(repetitions, 2); break; case 's': - /// Default second of minute is 0 instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaSecondOfMinute, repetitions)); - /// Second of minute range [0, 59] - // reserve_size += std::max(repetitions, 2); break; case 'S': throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for fractional seconds"); @@ -1698,7 +1634,6 @@ namespace ErrorCodes::NOT_IMPLEMENTED, "format is not supported for {}", String(cur_token, repetitions)); instructions.emplace_back(String(cur_token, pos - cur_token)); - // reserve_size += pos - cur_token; break; } } From 86744585fd15b43eaa9e4ae650dc08a0d3cf44f5 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Wed, 1 Mar 2023 06:30:07 +0100 Subject: [PATCH 092/559] Disable path check in SQLite storage for clickhouse-local --- src/Databases/SQLite/SQLiteUtils.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Databases/SQLite/SQLiteUtils.cpp b/src/Databases/SQLite/SQLiteUtils.cpp index 132ea49ff25..152370050f1 100644 --- a/src/Databases/SQLite/SQLiteUtils.cpp +++ b/src/Databases/SQLite/SQLiteUtils.cpp @@ -24,7 +24,7 @@ void processSQLiteError(const String & message, bool throw_on_error) LOG_ERROR(&Poco::Logger::get("SQLiteEngine"), fmt::runtime(message)); } -String validateSQLiteDatabasePath(const String & path, const String & user_files_path, bool throw_on_error) +String validateSQLiteDatabasePath(const String & path, const String & user_files_path, bool need_check, bool throw_on_error) { if (fs::path(path).is_relative()) return fs::absolute(fs::path(user_files_path) / path).lexically_normal(); @@ -32,7 +32,7 @@ String validateSQLiteDatabasePath(const String & path, const String & user_files String absolute_path = fs::absolute(path).lexically_normal(); String absolute_user_files_path = fs::absolute(user_files_path).lexically_normal(); - if (!absolute_path.starts_with(absolute_user_files_path)) + if (need_check && !absolute_path.starts_with(absolute_user_files_path)) { processSQLiteError(fmt::format("SQLite database file path '{}' must be inside 'user_files' directory", path), throw_on_error); return ""; @@ -42,8 +42,11 @@ String validateSQLiteDatabasePath(const String & path, const String & user_files SQLitePtr openSQLiteDB(const String & path, ContextPtr context, bool throw_on_error) { + // If run in Local mode, no need for path checking. + bool need_check = context->getApplicationType() != Context::ApplicationType::LOCAL; + auto user_files_path = context->getUserFilesPath(); - auto database_path = validateSQLiteDatabasePath(path, user_files_path, throw_on_error); + auto database_path = validateSQLiteDatabasePath(path, user_files_path, need_check, throw_on_error); /// For attach database there is no throw mode. if (database_path.empty()) From d7b83608c598ac4a48838a2dc5065904c40d0ce7 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 1 Mar 2023 17:15:29 +0800 Subject: [PATCH 093/559] finish performance profile --- src/Functions/parseDateTime.cpp | 261 +++++++++++++++++--------------- 1 file changed, 142 insertions(+), 119 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index be732e5a36b..b21fd8ffe77 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -102,6 +102,7 @@ namespace Int32 year = 1970; Int32 month = 1; Int32 day = 1; + std::vector day_of_month_values; bool is_ad = true; // AD -> true, BC -> false. Int32 week = 1; // Week of year based on ISO week date, e.g: 27 @@ -109,6 +110,7 @@ namespace bool week_date_format = false; Int32 day_of_year = 1; + std::vector day_of_year_values; bool day_of_year_format = false; bool century_format = false; @@ -119,14 +121,43 @@ namespace Int32 hour = 0; Int32 minute = 0; Int32 second = 0; - bool is_am = true; // AM -> true, PM -> false - std::optional time_zone_offset; + bool is_am = true; // AM -> true, PM -> false bool is_clock_hour = false; // Whether most recent hour specifier is clockhour bool is_hour_of_half_day = false; // Whether most recent hour specifier is of half day. - std::vector day_of_month_values; - std::vector day_of_year_values; + std::optional time_zone_offset; + + void reset() + { + year = 1970; + month = 1; + day = 1; + day_of_month_values.clear(); + is_ad = true; + + week = 1; + day_of_week = 1; + week_date_format = false; + + day_of_year = 1; + day_of_year_values.clear(); + day_of_year_format = false; + + century_format = false; + + is_year_of_era = false; // Year of era cannot be zero or negative. + has_year = false; // Whether year was explicitly specified. + + hour = 0; + minute = 0; + second = 0; + is_am = true; // AM -> true, PM -> false + is_clock_hour = false; // Whether most recent hour specifier is clockhour + is_hour_of_half_day = false; // Whether most recent hour specifier is of half day. + + time_zone_offset.reset(); + } void setCentrury(Int32 century) { @@ -168,7 +199,7 @@ namespace } } - void appendDayOfMonth(Int32 day_of_month) + ALWAYS_INLINE void appendDayOfMonth(Int32 day_of_month) { if (day_of_month < 1 || day_of_month > 31) throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for day of month must be in the range [1, 31]", day_of_month); @@ -184,7 +215,7 @@ namespace } } - void appendDayOfYear(Int32 day_of_year_) + ALWAYS_INLINE void appendDayOfYear(Int32 day_of_year_) { if (day_of_year_ < 1 || day_of_year_ > 366) throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for day of year must be in the range [1, 366]", day_of_year_); @@ -270,7 +301,7 @@ namespace throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown era {}", text); } - void setAMPM(String & text) + ALWAYS_INLINE void setAMPM(String & text) { Poco::toLowerInPlace(text); if (text == "am") @@ -281,7 +312,7 @@ namespace throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown half day of day: {}", text); } - void setHour(Int32 hour_, bool is_hour_of_half_day_ = false, bool is_clock_hour_ = false) + ALWAYS_INLINE void setHour(Int32 hour_, bool is_hour_of_half_day_ = false, bool is_clock_hour_ = false) { Int32 max_hour; Int32 min_hour; @@ -348,47 +379,16 @@ namespace static bool isDateValid(Int32 year_, Int32 month_, Int32 day_) { - if (month_ < 1 || month_ > 12) - return false; - - if (year_ < minYear || year_ > maxYear) - return false; - + /// The range of month[1, 12] and day[1, 31] already checked before bool leap = isLeapYear(year_); - if (day_ < 1) - return false; - - if (leap && day_ > leapDays[month_]) - return false; - - if (!leap && day_ > normalDays[month_]) - return false; - return true; + return (year_ >= minYear && year_ <= maxYear) && ((leap && day_ <= leapDays[month_]) || (!leap && day_ <= normalDays[month_])); } static bool isDayOfYearValid(Int32 year_, Int32 day_of_year_) { - if (year_ < minYear || year_ > maxYear) - return false; - - if (day_of_year_ < 1 || day_of_year_ > 365 + (isLeapYear(year_) ? 1 : 0)) - return false; - - return true; - } - - static bool isWeekDateValid(Int32 week_year_, Int32 week_of_year_, Int32 day_of_week_) - { - if (day_of_week_ < 1 || day_of_week_ > 7) - return false; - - if (week_of_year_ < 1 || week_of_year_ > 52) - return false; - - if (week_year_ < minYear || week_year_ > maxYear) - return false; - - return true; + /// The range of day_of_year[1, 366] already checked before + bool leap = isLeapYear(year_); + return (year_ >= minYear && year_ <= maxYear) && (day_of_year_ <= 365 + (leap ? 1 : 0)); } static Int32 extractISODayOfTheWeek(Int32 days_since_epoch) @@ -396,31 +396,27 @@ namespace if (days_since_epoch < 0) { // negative date: start off at 4 and cycle downwards - return (7 - ((-int64_t(days_since_epoch) + 3) % 7)); + return (7 - ((-days_since_epoch + 3) % 7)); } else { // positive date: start off at 4 and cycle upwards - return ((int64_t(days_since_epoch) + 3) % 7) + 1; + return ((days_since_epoch + 3) % 7) + 1; } } - static Int32 daysSinceEpochFromWeekDate(int32_t week_year_, int32_t week_of_year_, int32_t day_of_week_) + static ALWAYS_INLINE Int32 daysSinceEpochFromWeekDate(int32_t week_year_, int32_t week_of_year_, int32_t day_of_week_) { - if (!isWeekDateValid(week_year_, week_of_year_, day_of_week_)) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Invalid week date, week year:{} week of year:{} day of week:{}", - week_year_, - week_of_year_, - day_of_week_); + /// The range of week_of_year[1, 53], day_of_week[1, 7] already checked before + if (week_year_ < minYear || week_year_ > maxYear) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid week year {}", week_year_); Int32 days_since_epoch_of_jan_fourth = daysSinceEpochFromDate(week_year_, 1, 4); Int32 first_day_of_week_year = extractISODayOfTheWeek(days_since_epoch_of_jan_fourth); return days_since_epoch_of_jan_fourth - (first_day_of_week_year - 1) + 7 * (week_of_year_ - 1) + day_of_week_ - 1; } - static Int32 daysSinceEpochFromDayOfYear(Int32 year_, Int32 day_of_year_) + static ALWAYS_INLINE Int32 daysSinceEpochFromDayOfYear(Int32 year_, Int32 day_of_year_) { if (!isDayOfYearValid(year_, day_of_year_)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of year, year:{} day of year:{}", year_, day_of_year_); @@ -430,7 +426,7 @@ namespace return res; } - static Int32 daysSinceEpochFromDate(Int32 year_, Int32 month_, Int32 day_) + static ALWAYS_INLINE Int32 daysSinceEpochFromDate(Int32 year_, Int32 month_, Int32 day_) { if (!isDateValid(year_, month_, day_)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid date, year:{} month:{} day:{}", year_, month_, day_); @@ -471,9 +467,7 @@ namespace else if (day_of_year_format) days_since_epoch = daysSinceEpochFromDayOfYear(year, day_of_year); else - { days_since_epoch = daysSinceEpochFromDate(year, month, day); - } Int64 seconds_since_epoch = days_since_epoch * 86400 + hour * 3600 + minute * 60 + second; @@ -572,12 +566,15 @@ namespace auto col_res = ColumnDateTime::create(); col_res->reserve(input_rows_count); auto & data_res = col_res->getData(); + Date date; for (size_t i = 0; i < input_rows_count; ++i) { + date.reset(); + StringRef str_ref = col_str->getDataAt(i); Pos cur = str_ref.data; Pos end = str_ref.data + str_ref.size; - Date date; + // Date date; for (const auto & instruction : instructions) cur = instruction.perform(cur, end, date); @@ -631,7 +628,7 @@ namespace return func(cur, end, date); else { - ensureSpace(cur, end, literal.size(), "requires size >= " + std::to_string(literal.size())); + ensureSpace(cur, end, literal.size(), "required literal size not matched"); if (std::string_view(cur, literal.size()) != literal) throw Exception( ErrorCodes::LOGICAL_ERROR, "Expect literal {} but {} provided", literal, std::string_view(cur, literal.size())); @@ -640,48 +637,62 @@ namespace } } - template + template static Pos readNumber2(Pos cur, Pos end, T & res) { - ensureSpace(cur, end, 2, "readNumber2 requires size >= 2"); - res = (*cur - '0') * 10; - ++cur; - res += *cur - '0'; - ++cur; - return cur; - } + if constexpr (check_space) + ensureSpace(cur, end, 2, "readNumber2 requires size >= 2"); - template - static Pos readNumber3(Pos cur, Pos end, T & res) - { - cur = readNumber2(cur, end, res); - - ensureSpace(cur, end, 1, "readNumber3 requires size >= 3"); + res = (*cur - '0'); + ++cur; res = res * 10 + (*cur - '0'); ++cur; return cur; } - template - static Pos readNumber4(Pos cur, Pos end, T & res) + template + static Pos readNumber3(Pos cur, Pos end, T & res) { - cur = readNumber2(cur, end, res); + if constexpr (check_space) + ensureSpace(cur, end, 3, "readNumber4 requires size >= 3"); - T tmp; - cur = readNumber2(cur, end, tmp); - res = res * 100 + tmp; + res = (*cur - '0'); + ++cur; + res = res * 10 + (*cur - '0'); + ++cur; + res = res * 10 + (*cur - '0'); + ++cur; return cur; } - static ALWAYS_INLINE void ensureSpace(Pos cur, Pos end, size_t len, const String & msg) + template + static Pos readNumber4(Pos cur, Pos end, T & res) + { + if constexpr (check_space) + ensureSpace(cur, end, 4, "readNumber4 requires size >= 4"); + + res = (*cur - '0'); + ++cur; + res = res * 10 + (*cur - '0'); + ++cur; + res = res * 10 + (*cur - '0'); + ++cur; + res = res * 10 + (*cur - '0'); + ++cur; + return cur; + } + + static void ensureSpace(Pos cur, Pos end, size_t len, const String & msg) { if (cur > end || cur + len > end) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because {}", msg); } - static ALWAYS_INLINE Pos assertChar(Pos cur, Pos end, char ch) + template + static Pos assertChar(Pos cur, Pos end, char ch) { - ensureSpace(cur, end, 1, "assertChar requires size >= 1"); + if constexpr (check_space) + ensureSpace(cur, end, 1, "assertChar requires size >= 1"); if (*cur != ch) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expect char {}, but {} provided", String(ch, 1), String(*cur, 1)); @@ -745,18 +756,20 @@ namespace static Pos mysqlAmericanDate(Pos cur, Pos end, Date & date) { + ensureSpace(cur, end, 8, "mysqlAmericanDate requires size >= 8"); + Int32 month; - cur = readNumber2(cur, end, month); - cur = assertChar(cur, end, '/'); + cur = readNumber2(cur, end, month); + cur = assertChar(cur, end, '/'); date.setMonth(month); Int32 day; - cur = readNumber2(cur, end, day); - cur = assertChar(cur, end, '/'); + cur = readNumber2(cur, end, day); + cur = assertChar(cur, end, '/'); date.appendDayOfMonth(day); Int32 year; - cur = readNumber2(cur, end, year); + cur = readNumber2(cur, end, year); date.setYear(year); return cur; } @@ -777,18 +790,19 @@ namespace static Pos mysqlISO8601Date(Pos cur, Pos end, Date & date) { + ensureSpace(cur, end, 10, "mysqlISO8601Date requires size >= 10"); + Int32 year; - cur = readNumber4(cur, end, year); - cur = assertChar(cur, end, '-'); - date.setYear(year); - Int32 month; - cur = readNumber2(cur, end, month); - cur = assertChar(cur, end, '-'); - date.setMonth(month); - Int32 day; - cur = readNumber2(cur, end, day); + cur = readNumber4(cur, end, year); + cur = assertChar(cur, end, '-'); + cur = readNumber2(cur, end, month); + cur = assertChar(cur, end, '-'); + cur = readNumber2(cur, end, day); + + date.setYear(year); + date.setMonth(month); date.appendDayOfMonth(day); return cur; } @@ -820,7 +834,6 @@ namespace static Pos mysqlDayOfWeek(Pos cur, Pos end, Date & date) { ensureSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1"); - date.setDayOfWeek(*cur - '0'); ++cur; return cur; @@ -849,7 +862,7 @@ namespace static Pos mysqlDayOfWeekTextLong(Pos cur, Pos end, Date & date) { - ensureSpace(cur, end, 3, "jodaDayOfWeekText requires the first part size >= 3"); + ensureSpace(cur, end, 6, "jodaDayOfWeekText requires the size >= 6"); String text1(cur, 3); Poco::toLowerInPlace(text1); auto it = dayOfWeekMap.find(text1); @@ -887,17 +900,22 @@ namespace static Pos mysqlTimezoneOffset(Pos cur, Pos end, Date & date) { - ensureSpace(cur, end, 1, "Parse mysqlTimezoneOffset failed"); - Int32 sign = 1; + ensureSpace(cur, end, 5, "mysqlTimezoneOffset requires size >= 5"); + + Int32 sign; if (*cur == '-') sign = -1; + else if (*cur == '+') + sign = 1; + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown sign time zone offset: {}", std::string_view(cur, 1)); ++cur; Int32 hour; - cur = readNumber2(cur, end, hour); + cur = readNumber2(cur, end, hour); Int32 minute; - cur = readNumber2(cur, end, minute); + cur = readNumber2(cur, end, minute); *date.time_zone_offset = sign * (hour * 3600 + minute * 60); return cur; @@ -923,14 +941,16 @@ namespace static Pos mysqlHHMM12(Pos cur, Pos end, Date & date) { + ensureSpace(cur, end, 8, "mysqlHHMM12 requires size >= 8"); + Int32 hour; - cur = readNumber2(cur, end, hour); - cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, hour); + cur = assertChar(cur, end, ':'); date.setHour(hour, true, true); Int32 minute; - cur = readNumber2(cur, end, minute); - cur = assertChar(cur, end, ' '); + cur = readNumber2(cur, end, minute); + cur = assertChar(cur, end, ' '); date.setMinute(minute); cur = mysqlAMPM(cur, end, date); @@ -939,13 +959,15 @@ namespace static Pos mysqlHHMM24(Pos cur, Pos end, Date & date) { + ensureSpace(cur, end, 5, "mysqlHHMM24 requires size >= 5"); + Int32 hour; - cur = readNumber2(cur, end, hour); - cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, hour); + cur = assertChar(cur, end, ':'); date.setHour(hour, false, false); Int32 minute; - cur = readNumber2(cur, end, minute); + cur = readNumber2(cur, end, minute); date.setMinute(minute); return cur; } @@ -960,18 +982,19 @@ namespace static Pos mysqlISO8601Time(Pos cur, Pos end, Date & date) { + ensureSpace(cur, end, 8, "mysqlISO8601Time requires size >= 8"); + Int32 hour; - cur = readNumber2(cur, end, hour); - cur = assertChar(cur, end, ':'); - date.setHour(hour, false, false); - Int32 minute; - cur = readNumber2(cur, end, minute); - cur = assertChar(cur, end, ':'); - date.setMinute(minute); - Int32 second; - cur = readNumber2(cur, end, second); + cur = readNumber2(cur, end, hour); + cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, minute); + cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, second); + + date.setHour(hour, false, false); + date.setMinute(minute); date.setSecond(second); return cur; } From e85c4f1d3ad18f03671afbdf12d613a0c1e1864f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 1 Mar 2023 20:24:50 +0800 Subject: [PATCH 094/559] fix code styles and typos --- src/Functions/parseDateTime.cpp | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index b21fd8ffe77..7994bdb70bc 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -14,6 +14,16 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NOT_IMPLEMENTED; + extern const int BAD_ARGUMENTS; +} + namespace { using Pos = const char *; @@ -52,10 +62,10 @@ namespace /// key: month, value: total days of current month if current year is not leap year. constexpr Int32 normalDays[] = {0, 31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}; - /// key: month, value: cumulative days from Januray to current month(inclusive) if current year is leap year. + /// key: month, value: cumulative days from January to current month(inclusive) if current year is leap year. constexpr Int32 cumulativeLeapDays[] = {0, 31, 60, 91, 121, 152, 182, 213, 244, 274, 305, 335, 366}; - /// key: month, value: cumulative days from Januray to current month(inclusive) if current year is not leap year. + /// key: month, value: cumulative days from January to current month(inclusive) if current year is not leap year. constexpr Int32 cumulativeDays[] = {0, 31, 59, 90, 120, 151, 181, 212, 243, 273, 304, 334, 365}; /// key: year, value: cumulative days from epoch(1970-01-01) to the first day of current year(exclusive). From 4716273349c03204d0de25031753cbae5d8838ee Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 1 Mar 2023 13:37:05 +0100 Subject: [PATCH 095/559] Fix test --- src/Storages/NamedCollectionsHelpers.cpp | 32 ++++++++++++++++------ src/Storages/NamedCollectionsHelpers.h | 2 +- src/TableFunctions/TableFunctionRemote.cpp | 12 ++++++-- 3 files changed, 34 insertions(+), 12 deletions(-) diff --git a/src/Storages/NamedCollectionsHelpers.cpp b/src/Storages/NamedCollectionsHelpers.cpp index 54d15e1e40c..81801c68344 100644 --- a/src/Storages/NamedCollectionsHelpers.cpp +++ b/src/Storages/NamedCollectionsHelpers.cpp @@ -30,7 +30,7 @@ namespace return NamedCollectionFactory::instance().tryGet(collection_name); } - std::optional> getKeyValueFromAST(ASTPtr ast) + std::optional>> getKeyValueFromAST(ASTPtr ast, bool) { const auto * function = ast->as(); if (!function || function->name != "equals") @@ -42,20 +42,27 @@ namespace if (function_args.size() != 2) return std::nullopt; - auto literal_key = evaluateConstantExpressionOrIdentifierAsLiteral( - function_args[0], Context::getGlobalContextInstance()); + auto context = Context::getGlobalContextInstance(); + auto literal_key = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[0], context); auto key = checkAndGetLiteralArgument(literal_key, "key"); - auto literal_value = evaluateConstantExpressionOrIdentifierAsLiteral( - function_args[1], Context::getGlobalContextInstance()); - auto value = literal_value->as()->value; + ASTPtr literal_value; + try + { + literal_value = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[1], context); + } + catch (...) + { + return std::pair{key, function_args[1]}; + } + auto value = literal_value->as()->value; return std::pair{key, value}; } } -MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts, bool throw_unknown_collection) +MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts, bool throw_unknown_collection, std::vector> * non_convertible) { if (asts.empty()) return nullptr; @@ -73,14 +80,21 @@ MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts, bool thr for (auto * it = std::next(asts.begin()); it != asts.end(); ++it) { - auto value_override = getKeyValueFromAST(*it); + auto value_override = getKeyValueFromAST(*it, non_convertible != nullptr); + if (!value_override && !(*it)->as()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value argument or function"); if (!value_override) continue; + if (const ASTPtr * value = std::get_if(&value_override->second)) + { + non_convertible->emplace_back(value_override->first, *value); + continue; + } + const auto & [key, value] = *value_override; - collection_copy->setOrUpdate(key, toString(value)); + collection_copy->setOrUpdate(key, toString(std::get(value_override->second))); } return collection_copy; diff --git a/src/Storages/NamedCollectionsHelpers.h b/src/Storages/NamedCollectionsHelpers.h index 619adfc54b6..40d83ff9a12 100644 --- a/src/Storages/NamedCollectionsHelpers.h +++ b/src/Storages/NamedCollectionsHelpers.h @@ -18,7 +18,7 @@ namespace DB /// Helper function to get named collection for table engine. /// Table engines have collection name as first argument of ast and other arguments are key-value overrides. -MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts, bool throw_unknown_collection = true); +MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts, bool throw_unknown_collection = true, std::vector> * non_convertible = nullptr); /// Helper function to get named collection for dictionary source. /// Dictionaries have collection name as name argument of dict configuration and other arguments are overrides. MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 4952aa16efa..55c61b8a82a 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -51,19 +51,27 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr */ size_t max_args = is_cluster_function ? 4 : 6; NamedCollectionPtr named_collection; - if (!is_cluster_function && (named_collection = tryGetNamedCollectionWithOverrides(args))) + std::vector> non_convertible; + if (!is_cluster_function && (named_collection = tryGetNamedCollectionWithOverrides(args, false, &non_convertible))) { validateNamedCollection>( *named_collection, {"addresses_expr", "host", "hostname", "table"}, {"username", "user", "password", "sharding_key", "port", "database", "db"}); + if (!non_convertible.empty()) + { + if (non_convertible.size() != 1 || (non_convertible[0].first != "database" && non_convertible[0].first != "db")) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected argument representation for {}", non_convertible[0].first); + remote_table_function_ptr = non_convertible[0].second; + } + else + database = named_collection->getAnyOrDefault({"db", "database"}, "default"); cluster_description = named_collection->getOrDefault("addresses_expr", ""); if (cluster_description.empty() && named_collection->hasAny({"host", "hostname"})) cluster_description = named_collection->has("port") ? named_collection->getAny({"host", "hostname"}) + ':' + toString(named_collection->get("port")) : named_collection->getAny({"host", "hostname"}); - database = named_collection->getAnyOrDefault({"db", "database"}, "default"); table = named_collection->get("table"); username = named_collection->getAnyOrDefault({"username", "user"}, "default"); password = named_collection->getOrDefault("password", ""); From 80d017629a6a9f6cb8a0c76d00607ebf9b68ccc2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 1 Mar 2023 17:00:10 +0100 Subject: [PATCH 096/559] Fix test --- src/Databases/DatabaseFactory.cpp | 4 ++-- src/Storages/StorageMySQL.cpp | 2 +- src/Storages/StoragePostgreSQL.cpp | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 97ec0de9552..57281ca93d4 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -317,7 +317,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args)) { - configuration = StoragePostgreSQL::processNamedCollectionResult(*named_collection); + configuration = StoragePostgreSQL::processNamedCollectionResult(*named_collection, false); use_table_cache = named_collection->getOrDefault("use_tables_cache", 0); } else @@ -380,7 +380,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args)) { - configuration = StoragePostgreSQL::processNamedCollectionResult(*named_collection); + configuration = StoragePostgreSQL::processNamedCollectionResult(*named_collection, false); } else { diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index fdeea044dee..7d619c518cf 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -246,7 +246,7 @@ StorageMySQL::Configuration StorageMySQL::processNamedCollectionResult( for (const auto & setting : mysql_settings) optional_arguments.insert(setting.getName()); - ValidateKeysMultiset required_arguments = {"user", "username", "password", "database", "db", "table"}; + ValidateKeysMultiset required_arguments = {"user", "username", "password", "database", "db"}; if (require_table) required_arguments.insert("table"); validateNamedCollection>(named_collection, required_arguments, optional_arguments); diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 1bfc056f316..c208ad8ef5d 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -390,7 +390,7 @@ SinkToStoragePtr StoragePostgreSQL::write( StoragePostgreSQL::Configuration StoragePostgreSQL::processNamedCollectionResult(const NamedCollection & named_collection, bool require_table) { StoragePostgreSQL::Configuration configuration; - ValidateKeysMultiset required_arguments = {"user", "username", "password", "database", "db", "table"}; + ValidateKeysMultiset required_arguments = {"user", "username", "password", "database", "db"}; if (require_table) required_arguments.insert("table"); From b5a24b34fdfb6796647ad00509242fb7256ab9d1 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 2 Mar 2023 16:58:34 +0800 Subject: [PATCH 097/559] fix fast tests --- src/Functions/parseDateTime.cpp | 9 +++++++-- .../02415_all_new_functions_must_be_documented.reference | 2 ++ 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 7994bdb70bc..59b18e0f9c7 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -483,7 +483,9 @@ namespace /// Time zone is not specified, use local time zone if (!time_zone_offset) - *time_zone_offset = time_zone.getOffsetAtStartOfEpoch(); + *time_zone_offset = time_zone.timezoneOffset(seconds_since_epoch); + + std::cout << "time_zone:" << time_zone.getTimeZone() << ",offset:" << *time_zone_offset << std::endl; /// Time zone is specified in format string. if (seconds_since_epoch >= *time_zone_offset) @@ -584,9 +586,12 @@ namespace StringRef str_ref = col_str->getDataAt(i); Pos cur = str_ref.data; Pos end = str_ref.data + str_ref.size; - // Date date; for (const auto & instruction : instructions) + { + std::cout << "instruction:" << instruction.toString() << std::endl; cur = instruction.perform(cur, end, date); + std::cout << "date:" << date.toString() << std::endl; + } // Ensure all input was consumed. if (cur < end) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index e41249af54c..d146fc73800 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -512,6 +512,7 @@ nullIf nullIn nullInIgnoreSet or +parseDateTime parseDateTime32BestEffort parseDateTime32BestEffortOrNull parseDateTime32BestEffortOrZero @@ -527,6 +528,7 @@ parseDateTimeBestEffortOrZero parseDateTimeBestEffortUS parseDateTimeBestEffortUSOrNull parseDateTimeBestEffortUSOrZero +parseDateTimeInJodaSyntax parseTimeDelta partitionId path From 09a3bad3e0157060ef14592885d77fb37be950e2 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 2 Mar 2023 17:01:16 +0800 Subject: [PATCH 098/559] remove std::cout --- src/Functions/parseDateTime.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 59b18e0f9c7..028ec44cf90 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -484,8 +484,7 @@ namespace /// Time zone is not specified, use local time zone if (!time_zone_offset) *time_zone_offset = time_zone.timezoneOffset(seconds_since_epoch); - - std::cout << "time_zone:" << time_zone.getTimeZone() << ",offset:" << *time_zone_offset << std::endl; + // std::cout << "time_zone:" << time_zone.getTimeZone() << ",offset:" << *time_zone_offset << std::endl; /// Time zone is specified in format string. if (seconds_since_epoch >= *time_zone_offset) @@ -588,9 +587,9 @@ namespace Pos end = str_ref.data + str_ref.size; for (const auto & instruction : instructions) { - std::cout << "instruction:" << instruction.toString() << std::endl; + // std::cout << "instruction:" << instruction.toString() << std::endl; cur = instruction.perform(cur, end, date); - std::cout << "date:" << date.toString() << std::endl; + // std::cout << "date:" << date.toString() << std::endl; } // Ensure all input was consumed. From cd7cd0526bf693c87765410facb04439e852fb68 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 2 Mar 2023 19:04:33 +0100 Subject: [PATCH 099/559] Fix tests --- src/Databases/DatabaseFactory.cpp | 4 +-- src/Storages/MySQL/MySQLSettings.cpp | 20 +++++++++++--- src/Storages/MySQL/MySQLSettings.h | 2 +- src/Storages/NamedCollectionsHelpers.cpp | 16 +++++++----- src/Storages/NamedCollectionsHelpers.h | 26 +++++++++++++++---- src/Storages/StorageExternalDistributed.cpp | 2 +- src/Storages/StorageMySQL.cpp | 1 + src/TableFunctions/TableFunctionRemote.cpp | 19 +++++++++----- .../configs/named_collections.xml | 13 +++++++--- .../test_mask_sensitive_info/test.py | 8 +++--- .../configs/named_collections.xml | 2 -- .../configs/named_collections.xml | 1 - tests/integration/test_storage_mysql/test.py | 2 +- 13 files changed, 81 insertions(+), 35 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 57281ca93d4..cbd9f84df60 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -221,8 +221,8 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String { if (engine_name == "MySQL") { - mysql_settings->loadFromQueryContext(context); - mysql_settings->loadFromQuery(*engine_define); /// higher priority + mysql_settings->loadFromQuery(*engine_define); + mysql_settings->loadFromQueryContext(context, *engine_define); /// Will override only if not changed. auto mysql_pool = createMySQLPoolWithFailover(configuration, *mysql_settings); diff --git a/src/Storages/MySQL/MySQLSettings.cpp b/src/Storages/MySQL/MySQLSettings.cpp index b3bc11482f4..67942114182 100644 --- a/src/Storages/MySQL/MySQLSettings.cpp +++ b/src/Storages/MySQL/MySQLSettings.cpp @@ -4,6 +4,8 @@ #include #include #include +#include +#include namespace DB @@ -44,15 +46,27 @@ void MySQLSettings::loadFromQuery(ASTStorage & storage_def) } } -void MySQLSettings::loadFromQueryContext(ContextPtr context) +void MySQLSettings::loadFromQueryContext(ContextPtr context, ASTStorage & storage_def) { if (!context->hasQueryContext()) return; const Settings & settings = context->getQueryContext()->getSettingsRef(); - if (settings.mysql_datatypes_support_level.value != mysql_datatypes_support_level.value) - set("mysql_datatypes_support_level", settings.mysql_datatypes_support_level.toString()); + /// Setting from SETTING clause have bigger priority. + if (!mysql_datatypes_support_level.changed + && settings.mysql_datatypes_support_level.value != mysql_datatypes_support_level.value) + { + static constexpr auto setting_name = "mysql_datatypes_support_level"; + set(setting_name, settings.mysql_datatypes_support_level.toString()); + auto & changes = storage_def.settings->changes; + if (changes.end() == std::find_if( + changes.begin(), changes.end(), + [](const SettingChange & c) { return c.name == setting_name; })) + { + changes.push_back(SettingChange{setting_name, settings.mysql_datatypes_support_level.toString()}); + } + } } } diff --git a/src/Storages/MySQL/MySQLSettings.h b/src/Storages/MySQL/MySQLSettings.h index 40771d0aacb..850ac432aa1 100644 --- a/src/Storages/MySQL/MySQLSettings.h +++ b/src/Storages/MySQL/MySQLSettings.h @@ -37,7 +37,7 @@ struct MySQLSettings : public MySQLBaseSettings { void loadFromQuery(ASTStorage & storage_def); void loadFromQuery(const ASTSetQuery & settings_def); - void loadFromQueryContext(ContextPtr context); + void loadFromQueryContext(ContextPtr context, ASTStorage & storage_def); }; diff --git a/src/Storages/NamedCollectionsHelpers.cpp b/src/Storages/NamedCollectionsHelpers.cpp index 81801c68344..0cca2e4b9df 100644 --- a/src/Storages/NamedCollectionsHelpers.cpp +++ b/src/Storages/NamedCollectionsHelpers.cpp @@ -30,7 +30,7 @@ namespace return NamedCollectionFactory::instance().tryGet(collection_name); } - std::optional>> getKeyValueFromAST(ASTPtr ast, bool) + std::optional>> getKeyValueFromAST(ASTPtr ast, bool fallback_to_ast_value) { const auto * function = ast->as(); if (!function || function->name != "equals") @@ -53,7 +53,9 @@ namespace } catch (...) { - return std::pair{key, function_args[1]}; + if (fallback_to_ast_value) + return std::pair{key, function_args[1]}; + throw; } auto value = literal_value->as()->value; @@ -62,7 +64,8 @@ namespace } -MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts, bool throw_unknown_collection, std::vector> * non_convertible) +MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides( + ASTs asts, bool throw_unknown_collection, std::vector> * complex_args) { if (asts.empty()) return nullptr; @@ -80,7 +83,7 @@ MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts, bool thr for (auto * it = std::next(asts.begin()); it != asts.end(); ++it) { - auto value_override = getKeyValueFromAST(*it, non_convertible != nullptr); + auto value_override = getKeyValueFromAST(*it, complex_args != nullptr); if (!value_override && !(*it)->as()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value argument or function"); @@ -89,7 +92,7 @@ MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts, bool thr if (const ASTPtr * value = std::get_if(&value_override->second)) { - non_convertible->emplace_back(value_override->first, *value); + complex_args->emplace_back(value_override->first, *value); continue; } @@ -100,7 +103,8 @@ MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts, bool thr return collection_copy; } -MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) +MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides( + const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) { auto collection_name = config.getString(config_prefix + ".name", ""); if (collection_name.empty()) diff --git a/src/Storages/NamedCollectionsHelpers.h b/src/Storages/NamedCollectionsHelpers.h index 40d83ff9a12..8c6c1fb4e24 100644 --- a/src/Storages/NamedCollectionsHelpers.h +++ b/src/Storages/NamedCollectionsHelpers.h @@ -62,15 +62,29 @@ template struct NamedCollectionValidateKey bool operator<(const auto & other) const { - if (*this == other) - return false; - return value < other.value; + std::string_view canonical_self = value; + std::string_view canonical_other = other.value; + for (const auto & equal : EqualKeys::equal_keys) + { + if ((equal.first == value) || (equal.second == value)) + canonical_self = std::max(equal.first, equal.second); + if ((equal.first == other.value) || (equal.second == other.value)) + canonical_other = std::max(equal.first, equal.second); + } + + return canonical_self < canonical_other; } }; -template using ValidateKeysMultiset = std::multiset>; -using ValidateKeysSet = std::multiset; +template +std::ostream & operator << (std::ostream & ostr, const NamedCollectionValidateKey & key) +{ + ostr << key.value; + return ostr; +} +template using ValidateKeysMultiset = std::multiset, std::less>>; +using ValidateKeysSet = std::multiset; template void validateNamedCollection( @@ -91,7 +105,9 @@ void validateNamedCollection( } if (optional_keys.contains(key)) + { continue; + } auto match = std::find_if( optional_regex_keys.begin(), optional_regex_keys.end(), diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp index a5c22d0807d..45ca659a8fe 100644 --- a/src/Storages/StorageExternalDistributed.cpp +++ b/src/Storages/StorageExternalDistributed.cpp @@ -146,7 +146,7 @@ void registerStorageExternalDistributed(StorageFactory & factory) { auto current_configuration{configuration}; current_configuration.addresses = parseRemoteDescriptionForExternalDatabase(shard_address, max_addresses, 3306); - auto pool = createMySQLPoolWithFailover(configuration, mysql_settings); + auto pool = createMySQLPoolWithFailover(current_configuration, mysql_settings); shards.insert(std::make_shared( args.table_id, std::move(pool), configuration.database, configuration.table, /* replace_query = */ false, /* on_duplicate_clause = */ "", diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 7d619c518cf..61c715bdaeb 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -297,6 +297,7 @@ StorageMySQL::Configuration StorageMySQL::getConfiguration(ASTs engine_args, Con const auto & host_port = checkAndGetLiteralArgument(engine_args[0], "host:port"); size_t max_addresses = context_->getSettingsRef().glob_expansion_max_elements; + configuration.addresses_expr = host_port; configuration.addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306); configuration.database = checkAndGetLiteralArgument(engine_args[1], "database"); configuration.table = checkAndGetLiteralArgument(engine_args[2], "table"); diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 55c61b8a82a..eed9e07b532 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -51,18 +51,25 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr */ size_t max_args = is_cluster_function ? 4 : 6; NamedCollectionPtr named_collection; - std::vector> non_convertible; - if (!is_cluster_function && (named_collection = tryGetNamedCollectionWithOverrides(args, false, &non_convertible))) + std::vector> complex_args; + if (!is_cluster_function && (named_collection = tryGetNamedCollectionWithOverrides(args, false, &complex_args))) { validateNamedCollection>( *named_collection, {"addresses_expr", "host", "hostname", "table"}, {"username", "user", "password", "sharding_key", "port", "database", "db"}); - if (!non_convertible.empty()) + + if (!complex_args.empty()) { - if (non_convertible.size() != 1 || (non_convertible[0].first != "database" && non_convertible[0].first != "db")) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected argument representation for {}", non_convertible[0].first); - remote_table_function_ptr = non_convertible[0].second; + for (const auto & [arg_name, arg_ast] : complex_args) + { + if (arg_name == "database" || arg_name == "db") + remote_table_function_ptr = arg_ast; + else if (arg_name == "sharding_key") + sharding_key = arg_ast; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected argument representation for {}", arg_name); + } } else database = named_collection->getAnyOrDefault({"db", "database"}, "default"); diff --git a/tests/integration/test_mask_sensitive_info/configs/named_collections.xml b/tests/integration/test_mask_sensitive_info/configs/named_collections.xml index ee923a90171..a4b58f6f812 100644 --- a/tests/integration/test_mask_sensitive_info/configs/named_collections.xml +++ b/tests/integration/test_mask_sensitive_info/configs/named_collections.xml @@ -2,9 +2,16 @@ - - - + + user + pass + + + + + + + diff --git a/tests/integration/test_mask_sensitive_info/test.py b/tests/integration/test_mask_sensitive_info/test.py index 3f71b047213..5df40c38041 100644 --- a/tests/integration/test_mask_sensitive_info/test.py +++ b/tests/integration/test_mask_sensitive_info/test.py @@ -126,7 +126,7 @@ def test_create_table(): f"MySQL(named_collection_2, database = 'mysql_db', host = 'mysql57', port = 3306, password = '{password}', table = 'mysql_table', user = 'mysql_user')", f"MySQL(named_collection_3, database = 'mysql_db', host = 'mysql57', port = 3306, table = 'mysql_table')", f"PostgreSQL(named_collection_4, host = 'postgres1', port = 5432, database = 'postgres_db', table = 'postgres_table', user = 'postgres_user', password = '{password}')", - f"MongoDB(named_collection_5, host = 'mongo1', port = 5432, database = 'mongo_db', collection = 'mongo_col', user = 'mongo_user', password = '{password}')", + f"MongoDB(named_collection_5, host = 'mongo1', port = 5432, db = 'mongo_db', collection = 'mongo_col', user = 'mongo_user', password = '{password}')", f"S3(named_collection_6, url = 'http://minio1:9001/root/data/test8.csv', access_key_id = 'minio', secret_access_key = '{password}', format = 'CSV')", ] @@ -163,7 +163,7 @@ def test_create_table(): "CREATE TABLE table9 (`x` int) ENGINE = MySQL(named_collection_2, database = 'mysql_db', host = 'mysql57', port = 3306, password = '[HIDDEN]', table = 'mysql_table', user = 'mysql_user')", "CREATE TABLE table10 (x int) ENGINE = MySQL(named_collection_3, database = 'mysql_db', host = 'mysql57', port = 3306, table = 'mysql_table')", "CREATE TABLE table11 (`x` int) ENGINE = PostgreSQL(named_collection_4, host = 'postgres1', port = 5432, database = 'postgres_db', table = 'postgres_table', user = 'postgres_user', password = '[HIDDEN]')", - "CREATE TABLE table12 (`x` int) ENGINE = MongoDB(named_collection_5, host = 'mongo1', port = 5432, database = 'mongo_db', collection = 'mongo_col', user = 'mongo_user', password = '[HIDDEN]'", + "CREATE TABLE table12 (`x` int) ENGINE = MongoDB(named_collection_5, host = 'mongo1', port = 5432, db = 'mongo_db', collection = 'mongo_col', user = 'mongo_user', password = '[HIDDEN]'", "CREATE TABLE table13 (`x` int) ENGINE = S3(named_collection_6, url = 'http://minio1:9001/root/data/test8.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]', format = 'CSV')", ], must_not_contain=[password], @@ -233,7 +233,7 @@ def test_table_functions(): f"remoteSecure('127.{{2..11}}', 'default', 'remote_table', 'remote_user', rand())", f"mysql(named_collection_1, host = 'mysql57', port = 3306, database = 'mysql_db', table = 'mysql_table', user = 'mysql_user', password = '{password}')", f"postgresql(named_collection_2, password = '{password}', host = 'postgres1', port = 5432, database = 'postgres_db', table = 'postgres_table', user = 'postgres_user')", - f"s3(named_collection_3, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '{password}')", + f"s3(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '{password}')", f"remote(named_collection_4, addresses_expr = '127.{{2..11}}', database = 'default', table = 'remote_table', user = 'remote_user', password = '{password}', sharding_key = rand())", f"remoteSecure(named_collection_5, addresses_expr = '127.{{2..11}}', database = 'default', table = 'remote_table', user = 'remote_user', password = '{password}')", ] @@ -286,7 +286,7 @@ def test_table_functions(): "CREATE TABLE tablefunc24 (x int) AS remoteSecure('127.{2..11}', 'default', 'remote_table', 'remote_user', rand())", "CREATE TABLE tablefunc25 (`x` int) AS mysql(named_collection_1, host = 'mysql57', port = 3306, database = 'mysql_db', table = 'mysql_table', user = 'mysql_user', password = '[HIDDEN]')", "CREATE TABLE tablefunc26 (`x` int) AS postgresql(named_collection_2, password = '[HIDDEN]', host = 'postgres1', port = 5432, database = 'postgres_db', table = 'postgres_table', user = 'postgres_user')", - "CREATE TABLE tablefunc27 (`x` int) AS s3(named_collection_3, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]')", + "CREATE TABLE tablefunc27 (`x` int) AS s3(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]')", "CREATE TABLE tablefunc28 (`x` int) AS remote(named_collection_4, addresses_expr = '127.{2..11}', database = 'default', table = 'remote_table', user = 'remote_user', password = '[HIDDEN]', sharding_key = rand())", "CREATE TABLE tablefunc29 (`x` int) AS remoteSecure(named_collection_5, addresses_expr = '127.{2..11}', database = 'default', table = 'remote_table', user = 'remote_user', password = '[HIDDEN]')", ], diff --git a/tests/integration/test_mysql_database_engine/configs/named_collections.xml b/tests/integration/test_mysql_database_engine/configs/named_collections.xml index fd18dfa6202..3b65536f20f 100644 --- a/tests/integration/test_mysql_database_engine/configs/named_collections.xml +++ b/tests/integration/test_mysql_database_engine/configs/named_collections.xml @@ -6,7 +6,6 @@ mysql57 3306 test_database - test_table
postgres @@ -19,7 +18,6 @@ mysql57 1111 clickhouse - test_table
diff --git a/tests/integration/test_storage_mysql/configs/named_collections.xml b/tests/integration/test_storage_mysql/configs/named_collections.xml index 4d3fbf6085c..04117f32d4b 100644 --- a/tests/integration/test_storage_mysql/configs/named_collections.xml +++ b/tests/integration/test_storage_mysql/configs/named_collections.xml @@ -36,7 +36,6 @@ mysql57 3306 clickhouse - test_settings
1 20123001 20123002 diff --git a/tests/integration/test_storage_mysql/test.py b/tests/integration/test_storage_mysql/test.py index 50f0c5519b5..0c9369a8efa 100644 --- a/tests/integration/test_storage_mysql/test.py +++ b/tests/integration/test_storage_mysql/test.py @@ -765,7 +765,7 @@ def test_settings(started_cluster): rw_timeout = 20123001 connect_timeout = 20123002 - node1.query(f"SELECT * FROM mysql(mysql_with_settings)") + node1.query(f"SELECT * FROM mysql(mysql_with_settings, table='test_settings')") assert node1.contains_in_log( f"with settings: connect_timeout={connect_timeout}, read_write_timeout={rw_timeout}" ) From 6feee40ee9191363692a7f9426184688cfc28dcb Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 3 Mar 2023 10:42:35 +0800 Subject: [PATCH 100/559] change as request --- src/Functions/parseDateTime.cpp | 263 ++++++++++++++++---------------- 1 file changed, 135 insertions(+), 128 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 028ec44cf90..b0b931c5243 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -107,7 +107,7 @@ namespace return found ? count : -1; } - struct Date + struct DateTime { Int32 year = 1970; Int32 month = 1; @@ -162,6 +162,7 @@ namespace hour = 0; minute = 0; second = 0; + is_am = true; // AM -> true, PM -> false is_clock_hour = false; // Whether most recent hour specifier is clockhour is_hour_of_half_day = false; // Whether most recent hour specifier is of half day. @@ -169,7 +170,7 @@ namespace time_zone_offset.reset(); } - void setCentrury(Int32 century) + void setCentury(Int32 century) { if (century < 19 || century > 21) throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for century must be in the range [19, 21]", century); @@ -507,7 +508,7 @@ namespace /// _FUNC_(str[, format, timezone]) - template + template class FunctionParseDateTimeImpl : public IFunction { public: @@ -535,21 +536,21 @@ namespace if (!isString(arguments[0].type)) throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of first argument of function {} when arguments size is 1. Should be string", + "Illegal type {} of first argument of function {} when arguments size is 1. Should be String", arguments[0].type->getName(), getName()); if (arguments.size() > 1 && !isString(arguments[1].type)) throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of second argument of function {} when arguments size is 1. Should be string", + "Illegal type {} of second argument of function {} when arguments size is 1. Should be String", arguments[0].type->getName(), getName()); if (arguments.size() > 2 && !isString(arguments[2].type)) throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of third argument of function {} when arguments size is 1. Should be string", + "Illegal type {} of third argument of function {} when arguments size is 1. Should be String", arguments[0].type->getName(), getName()); @@ -577,7 +578,7 @@ namespace auto col_res = ColumnDateTime::create(); col_res->reserve(input_rows_count); auto & data_res = col_res->getData(); - Date date; + DateTime date; for (size_t i = 0; i < input_rows_count; ++i) { date.reset(); @@ -612,10 +613,16 @@ namespace class Action { private: + enum class NeedCheckSpace + { + Yes, + No + }; + using Func = std::conditional_t< parse_syntax == ParseDateTimeTraits::ParseSyntax::MySQL, - Pos (*)(Pos, Pos, Date &), - std::function>; + Pos (*)(Pos, Pos, DateTime &), + std::function>; Func func{}; std::string func_name; @@ -636,13 +643,13 @@ namespace return "literal:" + literal; } - Pos perform(Pos cur, Pos end, Date & date) const + Pos perform(Pos cur, Pos end, DateTime & date) const { if (func) return func(cur, end, date); else { - ensureSpace(cur, end, literal.size(), "required literal size not matched"); + checkSpace(cur, end, literal.size(), "required literal size not matched"); if (std::string_view(cur, literal.size()) != literal) throw Exception( ErrorCodes::LOGICAL_ERROR, "Expect literal {} but {} provided", literal, std::string_view(cur, literal.size())); @@ -651,11 +658,11 @@ namespace } } - template + template static Pos readNumber2(Pos cur, Pos end, T & res) { - if constexpr (check_space) - ensureSpace(cur, end, 2, "readNumber2 requires size >= 2"); + if constexpr (need_check_space == NeedCheckSpace::Yes) + checkSpace(cur, end, 2, "readNumber2 requires size >= 2"); res = (*cur - '0'); ++cur; @@ -664,11 +671,11 @@ namespace return cur; } - template + template static Pos readNumber3(Pos cur, Pos end, T & res) { - if constexpr (check_space) - ensureSpace(cur, end, 3, "readNumber4 requires size >= 3"); + if constexpr (need_check_space == NeedCheckSpace::Yes) + checkSpace(cur, end, 3, "readNumber4 requires size >= 3"); res = (*cur - '0'); ++cur; @@ -679,11 +686,11 @@ namespace return cur; } - template + template static Pos readNumber4(Pos cur, Pos end, T & res) { - if constexpr (check_space) - ensureSpace(cur, end, 4, "readNumber4 requires size >= 4"); + if constexpr (need_check_space == NeedCheckSpace::Yes) + checkSpace(cur, end, 4, "readNumber4 requires size >= 4"); res = (*cur - '0'); ++cur; @@ -696,17 +703,17 @@ namespace return cur; } - static void ensureSpace(Pos cur, Pos end, size_t len, const String & msg) + static void checkSpace(Pos cur, Pos end, size_t len, const String & msg) { if (cur > end || cur + len > end) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because {}", msg); } - template + template static Pos assertChar(Pos cur, Pos end, char ch) { - if constexpr (check_space) - ensureSpace(cur, end, 1, "assertChar requires size >= 1"); + if constexpr (need_check_space == NeedCheckSpace::Yes) + checkSpace(cur, end, 1, "assertChar requires size >= 1"); if (*cur != ch) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expect char {}, but {} provided", String(ch, 1), String(*cur, 1)); @@ -715,9 +722,9 @@ namespace return cur; } - static Pos mysqlDayOfWeekTextShort(Pos cur, Pos end, Date & date) + static Pos mysqlDayOfWeekTextShort(Pos cur, Pos end, DateTime & date) { - ensureSpace(cur, end, 3, "Parsing DayOfWeekTextShort requires size >= 3"); + checkSpace(cur, end, 3, "Parsing DayOfWeekTextShort requires size >= 3"); String text(cur, 3); Poco::toLowerInPlace(text); @@ -729,9 +736,9 @@ namespace return cur; } - static Pos mysqlMonthOfYearTextShort(Pos cur, Pos end, Date & date) + static Pos mysqlMonthOfYearTextShort(Pos cur, Pos end, DateTime & date) { - ensureSpace(cur, end, 3, "Parsing MonthOfYearTextShort requires size >= 3"); + checkSpace(cur, end, 3, "Parsing MonthOfYearTextShort requires size >= 3"); String text(cur, 3); Poco::toLowerInPlace(text); @@ -744,53 +751,53 @@ namespace return cur; } - static Pos mysqlMonth(Pos cur, Pos end, Date & date) + static Pos mysqlMonth(Pos cur, Pos end, DateTime & date) { Int32 month; - cur = readNumber2(cur, end, month); + cur = readNumber2(cur, end, month); date.setMonth(month); return cur; } - static Pos mysqlCentury(Pos cur, Pos end, Date & date) + static Pos mysqlCentury(Pos cur, Pos end, DateTime & date) { Int32 century; - cur = readNumber2(cur, end, century); - date.setCentrury(century); + cur = readNumber2(cur, end, century); + date.setCentury(century); return cur; } - static Pos mysqlDayOfMonth(Pos cur, Pos end, Date & date) + static Pos mysqlDayOfMonth(Pos cur, Pos end, DateTime & date) { Int32 day_of_month; - cur = readNumber2(cur, end, day_of_month); + cur = readNumber2(cur, end, day_of_month); date.appendDayOfMonth(day_of_month); return cur; } - static Pos mysqlAmericanDate(Pos cur, Pos end, Date & date) + static Pos mysqlAmericanDate(Pos cur, Pos end, DateTime & date) { - ensureSpace(cur, end, 8, "mysqlAmericanDate requires size >= 8"); + checkSpace(cur, end, 8, "mysqlAmericanDate requires size >= 8"); Int32 month; - cur = readNumber2(cur, end, month); - cur = assertChar(cur, end, '/'); + cur = readNumber2(cur, end, month); + cur = assertChar(cur, end, '/'); date.setMonth(month); Int32 day; - cur = readNumber2(cur, end, day); - cur = assertChar(cur, end, '/'); + cur = readNumber2(cur, end, day); + cur = assertChar(cur, end, '/'); date.appendDayOfMonth(day); Int32 year; - cur = readNumber2(cur, end, year); + cur = readNumber2(cur, end, year); date.setYear(year); return cur; } - static Pos mysqlDayOfMonthSpacePadded(Pos cur, Pos end, Date & date) + static Pos mysqlDayOfMonthSpacePadded(Pos cur, Pos end, DateTime & date) { - ensureSpace(cur, end, 2, "mysqlDayOfMonthSpacePadded requires size >= 2"); + checkSpace(cur, end, 2, "mysqlDayOfMonthSpacePadded requires size >= 2"); Int32 day_of_month = *cur == ' ' ? 0 : (*cur - '0'); ++cur; @@ -802,18 +809,18 @@ namespace return cur; } - static Pos mysqlISO8601Date(Pos cur, Pos end, Date & date) + static Pos mysqlISO8601Date(Pos cur, Pos end, DateTime & date) { - ensureSpace(cur, end, 10, "mysqlISO8601Date requires size >= 10"); + checkSpace(cur, end, 10, "mysqlISO8601Date requires size >= 10"); Int32 year; Int32 month; Int32 day; - cur = readNumber4(cur, end, year); - cur = assertChar(cur, end, '-'); - cur = readNumber2(cur, end, month); - cur = assertChar(cur, end, '-'); - cur = readNumber2(cur, end, day); + cur = readNumber4(cur, end, year); + cur = assertChar(cur, end, '-'); + cur = readNumber2(cur, end, month); + cur = assertChar(cur, end, '-'); + cur = readNumber2(cur, end, day); date.setYear(year); date.setMonth(month); @@ -821,49 +828,49 @@ namespace return cur; } - static Pos mysqlISO8601Year2(Pos cur, Pos end, Date & date) + static Pos mysqlISO8601Year2(Pos cur, Pos end, DateTime & date) { Int32 year2; - cur = readNumber2(cur, end, year2); + cur = readNumber2(cur, end, year2); date.setYear2(year2); return cur; } - static Pos mysqlISO8601Year4(Pos cur, Pos end, Date & date) + static Pos mysqlISO8601Year4(Pos cur, Pos end, DateTime & date) { Int32 year; - cur = readNumber4(cur, end, year); + cur = readNumber4(cur, end, year); date.setYear(year); return cur; } - static Pos mysqlDayOfYear(Pos cur, Pos end, Date & date) + static Pos mysqlDayOfYear(Pos cur, Pos end, DateTime & date) { Int32 day_of_year; - cur = readNumber3(cur, end, day_of_year); + cur = readNumber3(cur, end, day_of_year); date.appendDayOfYear(day_of_year); return cur; } - static Pos mysqlDayOfWeek(Pos cur, Pos end, Date & date) + static Pos mysqlDayOfWeek(Pos cur, Pos end, DateTime & date) { - ensureSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1"); + checkSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1"); date.setDayOfWeek(*cur - '0'); ++cur; return cur; } - static Pos mysqlISO8601Week(Pos cur, Pos end, Date & date) + static Pos mysqlISO8601Week(Pos cur, Pos end, DateTime & date) { Int32 week; - cur = readNumber2(cur, end, week); + cur = readNumber2(cur, end, week); date.setWeek(week); return cur; } - static Pos mysqlDayOfWeek0To6(Pos cur, Pos end, Date & date) + static Pos mysqlDayOfWeek0To6(Pos cur, Pos end, DateTime & date) { - ensureSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1"); + checkSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1"); Int32 day_of_week = *cur - '0'; if (day_of_week == 0) @@ -874,9 +881,9 @@ namespace return cur; } - static Pos mysqlDayOfWeekTextLong(Pos cur, Pos end, Date & date) + static Pos mysqlDayOfWeekTextLong(Pos cur, Pos end, DateTime & date) { - ensureSpace(cur, end, 6, "jodaDayOfWeekText requires the size >= 6"); + checkSpace(cur, end, 6, "jodaDayOfWeekText requires the size >= 6"); String text1(cur, 3); Poco::toLowerInPlace(text1); auto it = dayOfWeekMap.find(text1); @@ -885,7 +892,7 @@ namespace cur += 3; size_t left_size = it->second.first.size(); - ensureSpace(cur, end, left_size, "jodaDayOfWeekText requires the second parg size >= " + std::to_string(left_size)); + checkSpace(cur, end, left_size, "jodaDayOfWeekText requires the second parg size >= " + std::to_string(left_size)); String text2(cur, left_size); Poco::toLowerInPlace(text2); if (text2 != it->second.first) @@ -896,25 +903,25 @@ namespace return cur; } - static Pos mysqlYear2(Pos cur, Pos end, Date & date) + static Pos mysqlYear2(Pos cur, Pos end, DateTime & date) { Int32 year2; - cur = readNumber2(cur, end, year2); + cur = readNumber2(cur, end, year2); date.setYear2(year2); return cur; } - static Pos mysqlYear4(Pos cur, Pos end, Date & date) + static Pos mysqlYear4(Pos cur, Pos end, DateTime & date) { Int32 year; - cur = readNumber4(cur, end, year); + cur = readNumber4(cur, end, year); date.setYear(year); return cur; } - static Pos mysqlTimezoneOffset(Pos cur, Pos end, Date & date) + static Pos mysqlTimezoneOffset(Pos cur, Pos end, DateTime & date) { - ensureSpace(cur, end, 5, "mysqlTimezoneOffset requires size >= 5"); + checkSpace(cur, end, 5, "mysqlTimezoneOffset requires size >= 5"); Int32 sign; if (*cur == '-') @@ -926,26 +933,26 @@ namespace ++cur; Int32 hour; - cur = readNumber2(cur, end, hour); + cur = readNumber2(cur, end, hour); Int32 minute; - cur = readNumber2(cur, end, minute); + cur = readNumber2(cur, end, minute); *date.time_zone_offset = sign * (hour * 3600 + minute * 60); return cur; } - static Pos mysqlMinute(Pos cur, Pos end, Date & date) + static Pos mysqlMinute(Pos cur, Pos end, DateTime & date) { Int32 minute; - cur = readNumber2(cur, end, minute); + cur = readNumber2(cur, end, minute); date.setMinute(minute); return cur; } - static Pos mysqlAMPM(Pos cur, Pos end, Date & date) + static Pos mysqlAMPM(Pos cur, Pos end, DateTime & date) { - ensureSpace(cur, end, 2, "mysqlAMPM requires size >= 2"); + checkSpace(cur, end, 2, "mysqlAMPM requires size >= 2"); String text(cur, 2); date.setAMPM(text); @@ -953,59 +960,59 @@ namespace return cur; } - static Pos mysqlHHMM12(Pos cur, Pos end, Date & date) + static Pos mysqlHHMM12(Pos cur, Pos end, DateTime & date) { - ensureSpace(cur, end, 8, "mysqlHHMM12 requires size >= 8"); + checkSpace(cur, end, 8, "mysqlHHMM12 requires size >= 8"); Int32 hour; - cur = readNumber2(cur, end, hour); - cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, hour); + cur = assertChar(cur, end, ':'); date.setHour(hour, true, true); Int32 minute; - cur = readNumber2(cur, end, minute); - cur = assertChar(cur, end, ' '); + cur = readNumber2(cur, end, minute); + cur = assertChar(cur, end, ' '); date.setMinute(minute); cur = mysqlAMPM(cur, end, date); return cur; } - static Pos mysqlHHMM24(Pos cur, Pos end, Date & date) + static Pos mysqlHHMM24(Pos cur, Pos end, DateTime & date) { - ensureSpace(cur, end, 5, "mysqlHHMM24 requires size >= 5"); + checkSpace(cur, end, 5, "mysqlHHMM24 requires size >= 5"); Int32 hour; - cur = readNumber2(cur, end, hour); - cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, hour); + cur = assertChar(cur, end, ':'); date.setHour(hour, false, false); Int32 minute; - cur = readNumber2(cur, end, minute); + cur = readNumber2(cur, end, minute); date.setMinute(minute); return cur; } - static Pos mysqlSecond(Pos cur, Pos end, Date & date) + static Pos mysqlSecond(Pos cur, Pos end, DateTime & date) { Int32 second; - cur = readNumber2(cur, end, second); + cur = readNumber2(cur, end, second); date.setSecond(second); return cur; } - static Pos mysqlISO8601Time(Pos cur, Pos end, Date & date) + static Pos mysqlISO8601Time(Pos cur, Pos end, DateTime & date) { - ensureSpace(cur, end, 8, "mysqlISO8601Time requires size >= 8"); + checkSpace(cur, end, 8, "mysqlISO8601Time requires size >= 8"); Int32 hour; Int32 minute; Int32 second; - cur = readNumber2(cur, end, hour); - cur = assertChar(cur, end, ':'); - cur = readNumber2(cur, end, minute); - cur = assertChar(cur, end, ':'); - cur = readNumber2(cur, end, second); + cur = readNumber2(cur, end, hour); + cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, minute); + cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, second); date.setHour(hour, false, false); date.setMinute(minute); @@ -1013,18 +1020,18 @@ namespace return cur; } - static Pos mysqlHour12(Pos cur, Pos end, Date & date) + static Pos mysqlHour12(Pos cur, Pos end, DateTime & date) { Int32 hour; - cur = readNumber2(cur, end, hour); + cur = readNumber2(cur, end, hour); date.setHour(hour, true, true); return cur; } - static Pos mysqlHour24(Pos cur, Pos end, Date & date) + static Pos mysqlHour24(Pos cur, Pos end, DateTime & date) { Int32 hour; - cur = readNumber2(cur, end, hour); + cur = readNumber2(cur, end, hour); date.setHour(hour, false, false); return cur; } @@ -1103,9 +1110,9 @@ namespace return cur; } - static Pos jodaEra(int, Pos cur, Pos end, Date & date) + static Pos jodaEra(int, Pos cur, Pos end, DateTime & date) { - ensureSpace(cur, end, 2, "jodaEra requires size >= 2"); + checkSpace(cur, end, 2, "jodaEra requires size >= 2"); String era(cur, 2); date.setEra(era); @@ -1113,15 +1120,15 @@ namespace return cur; } - static Pos jodaCenturyOfEra(int repetitions, Pos cur, Pos end, Date & date) + static Pos jodaCenturyOfEra(int repetitions, Pos cur, Pos end, DateTime & date) { Int32 century; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, repetitions, century); - date.setCentrury(century); + date.setCentury(century); return cur; } - static Pos jodaYearOfEra(int repetitions, Pos cur, Pos end, Date & date) + static Pos jodaYearOfEra(int repetitions, Pos cur, Pos end, DateTime & date) { Int32 year_of_era; cur = readNumberWithVariableLength(cur, end, false, false, true, repetitions, repetitions, year_of_era); @@ -1129,7 +1136,7 @@ namespace return cur; } - static Pos jodaWeekYear(int repetitions, Pos cur, Pos end, Date & date) + static Pos jodaWeekYear(int repetitions, Pos cur, Pos end, DateTime & date) { Int32 week_year; cur = readNumberWithVariableLength(cur, end, true, true, true, repetitions, repetitions, week_year); @@ -1137,7 +1144,7 @@ namespace return cur; } - static Pos jodaWeekOfWeekYear(int repetitions, Pos cur, Pos end, Date & date) + static Pos jodaWeekOfWeekYear(int repetitions, Pos cur, Pos end, DateTime & date) { Int32 week; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), week); @@ -1145,7 +1152,7 @@ namespace return cur; } - static Pos jodaDayOfWeek1Based(int repetitions, Pos cur, Pos end, Date & date) + static Pos jodaDayOfWeek1Based(int repetitions, Pos cur, Pos end, DateTime & date) { Int32 day_of_week; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, repetitions, day_of_week); @@ -1156,9 +1163,9 @@ namespace return cur; } - static Pos jodaDayOfWeekText(size_t /*min_represent_digits*/, Pos cur, Pos end, Date & date) + static Pos jodaDayOfWeekText(size_t /*min_represent_digits*/, Pos cur, Pos end, DateTime & date) { - ensureSpace(cur, end, 3, "jodaDayOfWeekText requires size >= 3"); + checkSpace(cur, end, 3, "jodaDayOfWeekText requires size >= 3"); String text1(cur, 3); Poco::toLowerInPlace(text1); @@ -1182,7 +1189,7 @@ namespace return cur; } - static Pos jodaYear(int repetitions, Pos cur, Pos end, Date & date) + static Pos jodaYear(int repetitions, Pos cur, Pos end, DateTime & date) { Int32 year; cur = readNumberWithVariableLength(cur, end, true, true, true, repetitions, repetitions, year); @@ -1190,7 +1197,7 @@ namespace return cur; } - static Pos jodaDayOfYear(int repetitions, Pos cur, Pos end, Date & date) + static Pos jodaDayOfYear(int repetitions, Pos cur, Pos end, DateTime & date) { Int32 day_of_year; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 3), day_of_year); @@ -1198,7 +1205,7 @@ namespace return cur; } - static Pos jodaMonthOfYear(int repetitions, Pos cur, Pos end, Date & date) + static Pos jodaMonthOfYear(int repetitions, Pos cur, Pos end, DateTime & date) { Int32 month; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, 2, month); @@ -1206,9 +1213,9 @@ namespace return cur; } - static Pos jodaMonthOfYearText(int, Pos cur, Pos end, Date & date) + static Pos jodaMonthOfYearText(int, Pos cur, Pos end, DateTime & date) { - ensureSpace(cur, end, 3, "jodaMonthOfYearText requires size >= 3"); + checkSpace(cur, end, 3, "jodaMonthOfYearText requires size >= 3"); String text1(cur, 3); Poco::toLowerInPlace(text1); auto it = monthMap.find(text1); @@ -1231,7 +1238,7 @@ namespace return cur; } - static Pos jodaDayOfMonth(int repetitions, Pos cur, Pos end, Date & date) + static Pos jodaDayOfMonth(int repetitions, Pos cur, Pos end, DateTime & date) { Int32 day_of_month; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), day_of_month); @@ -1239,9 +1246,9 @@ namespace return cur; } - static Pos jodaHalfDayOfDay(int, Pos cur, Pos end, Date & date) + static Pos jodaHalfDayOfDay(int, Pos cur, Pos end, DateTime & date) { - ensureSpace(cur, end, 2, "jodaHalfDayOfDay requires size >= 2"); + checkSpace(cur, end, 2, "jodaHalfDayOfDay requires size >= 2"); String text(cur, 2); date.setAMPM(text); @@ -1249,7 +1256,7 @@ namespace return cur; } - static Pos jodaHourOfHalfDay(int repetitions, Pos cur, Pos end, Date & date) + static Pos jodaHourOfHalfDay(int repetitions, Pos cur, Pos end, DateTime & date) { Int32 hour; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), hour); @@ -1257,7 +1264,7 @@ namespace return cur; } - static Pos jodaClockHourOfHalfDay(int repetitions, Pos cur, Pos end, Date & date) + static Pos jodaClockHourOfHalfDay(int repetitions, Pos cur, Pos end, DateTime & date) { Int32 hour; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), hour); @@ -1265,7 +1272,7 @@ namespace return cur; } - static Pos jodaHourOfDay(int repetitions, Pos cur, Pos end, Date & date) + static Pos jodaHourOfDay(int repetitions, Pos cur, Pos end, DateTime & date) { Int32 hour; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), hour); @@ -1273,7 +1280,7 @@ namespace return cur; } - static Pos jodaClockHourOfDay(int repetitions, Pos cur, Pos end, Date & date) + static Pos jodaClockHourOfDay(int repetitions, Pos cur, Pos end, DateTime & date) { Int32 hour; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), hour); @@ -1281,7 +1288,7 @@ namespace return cur; } - static Pos jodaMinuteOfHour(int repetitions, Pos cur, Pos end, Date & date) + static Pos jodaMinuteOfHour(int repetitions, Pos cur, Pos end, DateTime & date) { Int32 minute; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), minute); @@ -1289,7 +1296,7 @@ namespace return cur; } - static Pos jodaSecondOfMinute(int repetitions, Pos cur, Pos end, Date & date) + static Pos jodaSecondOfMinute(int repetitions, Pos cur, Pos end, DateTime & date) { Int32 second; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), second); @@ -1679,7 +1686,7 @@ namespace } - ALWAYS_INLINE String getFormat(const ColumnsWithTypeAndName & arguments) const + String getFormat(const ColumnsWithTypeAndName & arguments) const { if (arguments.size() < 2) { @@ -1699,7 +1706,7 @@ namespace return format_column->getValue(); } - ALWAYS_INLINE std::pair getTimeZone(const ColumnsWithTypeAndName & arguments) const + std::pair getTimeZone(const ColumnsWithTypeAndName & arguments) const { if (arguments.size() < 3) return {&DateLUT::instance(), ""}; From 0fa3fb359286054a0c2c97c7b68ffd1e0180f6fe Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 3 Mar 2023 14:04:32 +0100 Subject: [PATCH 101/559] Fix show grants for user which has all grants --- src/Access/AccessRights.cpp | 23 +++++++++++++------ src/Access/Common/AccessRightsElement.cpp | 5 ++++ src/Access/Common/AccessRightsElement.h | 16 ++++++++++--- .../integration/test_grant_and_revoke/test.py | 3 +++ .../test_named_collections/test.py | 16 ++++++++++--- 5 files changed, 50 insertions(+), 13 deletions(-) diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index 37597552a41..cfa14e6c88b 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -124,20 +124,29 @@ namespace const auto & element = sorted[i]; if (element.access_flags) { - auto per_parameter = element.access_flags.splitIntoParameterTypes(); - if (per_parameter.size() == 1) + const bool all_granted = sorted.size() == 1 && element.access_flags.contains(AccessFlags::allFlags()); + if (all_granted) { /// Easy case: one Element is converted to one AccessRightsElement. res.emplace_back(element.getResult()); } else { - /// Difficult case: one element is converted into multiple AccessRightsElements. - for (const auto & [_, parameter_flags] : per_parameter) + auto per_parameter = element.access_flags.splitIntoParameterTypes(); + if (per_parameter.size() == 1) { - auto current_element{element}; - current_element.access_flags = parameter_flags; - res.emplace_back(current_element.getResult()); + /// Easy case: one Element is converted to one AccessRightsElement. + res.emplace_back(element.getResult()); + } + else + { + /// Difficult case: one element is converted into multiple AccessRightsElements. + for (const auto & [_, parameter_flags] : per_parameter) + { + auto current_element{element}; + current_element.access_flags = parameter_flags; + res.emplace_back(current_element.getResult()); + } } } } diff --git a/src/Access/Common/AccessRightsElement.cpp b/src/Access/Common/AccessRightsElement.cpp index 81cebd68b4c..e11d43634ec 100644 --- a/src/Access/Common/AccessRightsElement.cpp +++ b/src/Access/Common/AccessRightsElement.cpp @@ -233,6 +233,11 @@ bool AccessRightsElements::sameDatabaseAndTableAndParameter() const return (size() < 2) || std::all_of(std::next(begin()), end(), [this](const AccessRightsElement & e) { return e.sameDatabaseAndTableAndParameter(front()); }); } +bool AccessRightsElements::sameDatabaseAndTable() const +{ + return (size() < 2) || std::all_of(std::next(begin()), end(), [this](const AccessRightsElement & e) { return e.sameDatabaseAndTable(front()); }); +} + bool AccessRightsElements::sameOptions() const { return (size() < 2) || std::all_of(std::next(begin()), end(), [this](const AccessRightsElement & e) { return e.sameOptions(front()); }); diff --git a/src/Access/Common/AccessRightsElement.h b/src/Access/Common/AccessRightsElement.h index 96850f0880e..ba625fc43df 100644 --- a/src/Access/Common/AccessRightsElement.h +++ b/src/Access/Common/AccessRightsElement.h @@ -55,13 +55,22 @@ struct AccessRightsElement bool sameDatabaseAndTableAndParameter(const AccessRightsElement & other) const { - return (database == other.database) && (any_database == other.any_database) - && (table == other.table) && (any_table == other.any_table) - && (parameter == other.parameter) && (any_parameter == other.any_parameter) + return sameDatabaseAndTable(other) && sameParameter(other); + } + + bool sameParameter(const AccessRightsElement & other) const + { + return (parameter == other.parameter) && (any_parameter == other.any_parameter) && (access_flags.getParameterType() == other.access_flags.getParameterType()) && (isGlobalWithParameter() == other.isGlobalWithParameter()); } + bool sameDatabaseAndTable(const AccessRightsElement & other) const + { + return (database == other.database) && (any_database == other.any_database) + && (table == other.table) && (any_table == other.any_table); + } + bool sameOptions(const AccessRightsElement & other) const { return (grant_option == other.grant_option) && (is_partial_revoke == other.is_partial_revoke); @@ -92,6 +101,7 @@ public: bool empty() const; bool sameDatabaseAndTableAndParameter() const; + bool sameDatabaseAndTable() const; bool sameOptions() const; /// Resets flags which cannot be granted. diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test.py index 8d48f7449e4..4d89e6255d3 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test.py @@ -402,6 +402,9 @@ def test_introspection(): assert instance.query("SHOW GRANTS FOR B") == TSV( ["GRANT CREATE ON *.* TO B WITH GRANT OPTION"] ) + assert instance.query("SHOW GRANTS FOR default") == TSV( + ["GRANT ALL ON *.* TO default WITH GRANT OPTION"] + ) assert instance.query("SHOW GRANTS FOR A,B") == TSV( [ "GRANT SELECT ON test.table TO A", diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index 1f27826d213..5574c77b886 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -100,7 +100,9 @@ def test_default_access(cluster): ) node.restart_clickhouse() assert ( - node.query("select collection['key1'] from system.named_collections").strip() + node.query( + "select collection['key1'] from system.named_collections where name = 'collection1'" + ).strip() == "value1" ) replace_in_users_config( @@ -111,7 +113,9 @@ def test_default_access(cluster): ) node.restart_clickhouse() assert ( - node.query("select collection['key1'] from system.named_collections").strip() + node.query( + "select collection['key1'] from system.named_collections where name = 'collection1'" + ).strip() == "[HIDDEN]" ) replace_in_users_config( @@ -122,13 +126,19 @@ def test_default_access(cluster): ) node.restart_clickhouse() assert ( - node.query("select collection['key1'] from system.named_collections").strip() + node.query( + "select collection['key1'] from system.named_collections where name = 'collection1'" + ).strip() == "value1" ) def test_granular_access_show_query(cluster): node = cluster.instances["node"] + assert ( + "GRANT ALL ON *.* TO default WITH GRANT OPTION" + == node.query("SHOW GRANTS FOR default").strip() + ) # includes named collections control assert 1 == int(node.query("SELECT count() FROM system.named_collections")) assert ( "collection1" == node.query("SELECT name FROM system.named_collections").strip() From f78da967b3f1b39bb628b6777be3539d2047fc34 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 3 Mar 2023 16:32:23 +0100 Subject: [PATCH 102/559] Fix test --- src/Storages/NamedCollectionsHelpers.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/NamedCollectionsHelpers.h b/src/Storages/NamedCollectionsHelpers.h index 8c6c1fb4e24..86e215bccf5 100644 --- a/src/Storages/NamedCollectionsHelpers.h +++ b/src/Storages/NamedCollectionsHelpers.h @@ -67,9 +67,9 @@ template struct NamedCollectionValidateKey for (const auto & equal : EqualKeys::equal_keys) { if ((equal.first == value) || (equal.second == value)) - canonical_self = std::max(equal.first, equal.second); + canonical_self = std::max(canonical_self, std::max(equal.first, equal.second)); if ((equal.first == other.value) || (equal.second == other.value)) - canonical_other = std::max(equal.first, equal.second); + canonical_other = std::max(canonical_other, std::max(equal.first, equal.second)); } return canonical_self < canonical_other; From c1e80683c444512a446e4579b000f512996237d5 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 3 Mar 2023 17:57:10 +0100 Subject: [PATCH 103/559] Refine OptimizeRegularExpression Function --- src/Common/OptimizedRegularExpression.cpp | 107 +++++++++++++++------- src/Common/OptimizedRegularExpression.h | 3 +- src/Common/tests/gtest_optimize_re.cpp | 31 +++++++ 3 files changed, 107 insertions(+), 34 deletions(-) create mode 100644 src/Common/tests/gtest_optimize_re.cpp diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index 7d96feba1f3..ff87cd7ef86 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -47,6 +47,25 @@ void OptimizedRegularExpressionImpl::analyze( Substrings trivial_substrings(1); Substring * last_substring = &trivial_substrings.back(); + std::string bracket_string; + bool appending_bracket_string = false; + + auto finish_last_substring = [&]() + { + if (depth != 0) + return; + /// combine last substr and bracket string + last_substring->first += bracket_string; + bracket_string = ""; + /// if we can still append, no need to finish it. e.g. abc(de)fg should capture abcdefg + if (!last_substring->first.empty() && !appending_bracket_string) + { + trivial_substrings.resize(trivial_substrings.size() + 1); + last_substring = &trivial_substrings.back(); + } + appending_bracket_string = false; + }; + bool in_curly_braces = false; bool in_square_braces = false; @@ -83,15 +102,21 @@ void OptimizedRegularExpressionImpl::analyze( last_substring->second = pos - begin; last_substring->first.push_back(*pos); } + else if (depth == 1 && appending_bracket_string) + { + bracket_string += *pos; + } break; default: /// all other escape sequences are not supported is_trivial = false; - if (!last_substring->first.empty()) - { - trivial_substrings.resize(trivial_substrings.size() + 1); - last_substring = &trivial_substrings.back(); - } + appending_bracket_string = false; + //if (!last_substring->first.empty()) + //{ + // trivial_substrings.resize(trivial_substrings.size() + 1); + // last_substring = &trivial_substrings.back(); + //} + finish_last_substring(); break; } @@ -102,8 +127,13 @@ void OptimizedRegularExpressionImpl::analyze( case '|': if (depth == 0) has_alternative_on_depth_0 = true; + if (depth == 1) + { + appending_bracket_string = false; + bracket_string = ""; + } is_trivial = false; - if (!in_square_braces && !last_substring->first.empty()) + if (!in_square_braces && !last_substring->first.empty() && depth == 0) { trivial_substrings.resize(trivial_substrings.size() + 1); last_substring = &trivial_substrings.back(); @@ -116,11 +146,10 @@ void OptimizedRegularExpressionImpl::analyze( { ++depth; is_trivial = false; - if (!last_substring->first.empty()) - { - trivial_substrings.resize(trivial_substrings.size() + 1); - last_substring = &trivial_substrings.back(); - } + /// we dont change the value of appending_bracket_string when depth > 1 + /// e.g. (de(fg)) should capture defg + if (depth == 1) + appending_bracket_string = true; /// Check for case-insensitive flag. if (pos + 1 < end && pos[1] == '?') @@ -143,6 +172,10 @@ void OptimizedRegularExpressionImpl::analyze( break; } } + if (pos + 2 < end && pos[1] == '?' && pos[2] == ':') + { + pos += 2; + } } ++pos; break; @@ -151,11 +184,8 @@ void OptimizedRegularExpressionImpl::analyze( in_square_braces = true; ++depth; is_trivial = false; - if (!last_substring->first.empty()) - { - trivial_substrings.resize(trivial_substrings.size() + 1); - last_substring = &trivial_substrings.back(); - } + appending_bracket_string = false; + finish_last_substring(); ++pos; break; @@ -166,11 +196,12 @@ void OptimizedRegularExpressionImpl::analyze( in_square_braces = false; --depth; is_trivial = false; - if (!last_substring->first.empty()) - { - trivial_substrings.resize(trivial_substrings.size() + 1); - last_substring = &trivial_substrings.back(); - } + finish_last_substring(); + //if (!last_substring->first.empty()) + //{ + // trivial_substrings.resize(trivial_substrings.size() + 1); + // last_substring = &trivial_substrings.back(); + //} ++pos; break; @@ -179,22 +210,21 @@ void OptimizedRegularExpressionImpl::analyze( { --depth; is_trivial = false; - if (!last_substring->first.empty()) + if (pos + 1 < end && (pos[1] == '?' || pos[1] == '*')) { - trivial_substrings.resize(trivial_substrings.size() + 1); - last_substring = &trivial_substrings.back(); + /// TODO: (abc(def)?) should remain the abc part. + bracket_string = ""; + appending_bracket_string = false; } + finish_last_substring(); } ++pos; break; case '^': case '$': case '.': case '+': is_trivial = false; - if (!last_substring->first.empty() && !in_square_braces) - { - trivial_substrings.resize(trivial_substrings.size() + 1); - last_substring = &trivial_substrings.back(); - } + appending_bracket_string = false; + finish_last_substring(); ++pos; break; @@ -206,12 +236,20 @@ void OptimizedRegularExpressionImpl::analyze( [[fallthrough]]; case '*': is_trivial = false; - if (!last_substring->first.empty() && !in_square_braces) + if (depth == 0 && !last_substring->first.empty() && !in_square_braces) { last_substring->first.resize(last_substring->first.size() - 1); - trivial_substrings.resize(trivial_substrings.size() + 1); - last_substring = &trivial_substrings.back(); } + if (depth >= 1 && appending_bracket_string) + { + /// ab(*cd) should be ab + appending_bracket_string = false; + if (!bracket_string.empty()) + { + bracket_string.resize(bracket_string.size() - 1); + } + } + finish_last_substring(); ++pos; break; @@ -232,11 +270,16 @@ void OptimizedRegularExpressionImpl::analyze( last_substring->second = pos - begin; last_substring->first.push_back(*pos); } + else if (depth >= 1 && appending_bracket_string) + bracket_string += *pos; ++pos; break; } } + appending_bracket_string = false; + finish_last_substring(); + if (last_substring && last_substring->first.empty()) trivial_substrings.pop_back(); diff --git a/src/Common/OptimizedRegularExpression.h b/src/Common/OptimizedRegularExpression.h index d8ed1e205c8..d8b54520bf3 100644 --- a/src/Common/OptimizedRegularExpression.h +++ b/src/Common/OptimizedRegularExpression.h @@ -95,6 +95,7 @@ public: out_required_substring_is_prefix = required_substring_is_prefix; } + static void analyze(std::string_view regexp_, std::string & required_substring, bool & is_trivial, bool & required_substring_is_prefix); private: bool is_trivial; bool required_substring_is_prefix; @@ -104,8 +105,6 @@ private: std::optional case_insensitive_substring_searcher; std::unique_ptr re2; unsigned number_of_subpatterns; - - static void analyze(std::string_view regexp_, std::string & required_substring, bool & is_trivial, bool & required_substring_is_prefix); }; using OptimizedRegularExpression = OptimizedRegularExpressionImpl; diff --git a/src/Common/tests/gtest_optimize_re.cpp b/src/Common/tests/gtest_optimize_re.cpp new file mode 100644 index 00000000000..e68f699ee80 --- /dev/null +++ b/src/Common/tests/gtest_optimize_re.cpp @@ -0,0 +1,31 @@ +#include + +#include + +TEST(OptimizeRE, analyze) +{ + auto test_f = [](const std::string & regexp, const std::string & answer) + { + std::string required; + bool is_trivial; + bool is_prefix; + OptimizedRegularExpression::analyze(regexp, required, is_trivial, is_prefix); + EXPECT_EQ(required, answer); + }; + test_f("abc", "abc"); + test_f("abc(de)fg", "abcdefg"); + test_f("abc(de|xyz)fg", "abc"); + test_f("abc(de?f|xyz)fg", "abc"); + test_f("abc|fg", ""); + test_f("(abc)", "abc"); + test_f("(abc|fg)", ""); + test_f("abc(abc|fg)xyzz", "xyzz"); + test_f("abc[k]xyzz", "xyzz"); + /// actually the best answer should be xyzz + test_f("(abc[k]xyzz)", "abc"); + test_f("abc((de)fg(hi))jk", "abcdefghijk"); + test_f("abc((de)fghi+zzz)jk", "abcdefghi"); + test_f("abc((de)fg(hi))?jk", "abc"); + test_f("abc((de)fghi?zzz)jk", "abcdefgh"); + test_f("abc(*cd)jk", "abc"); +} From d7cb43a20b9939db6c7aa328936f26b2235dfb53 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sun, 5 Mar 2023 16:24:05 +0800 Subject: [PATCH 104/559] change as requested --- src/Functions/formatDateTime.cpp | 199 ++++++++++++----------------- src/Functions/parseDateTime.cpp | 211 +++++++++++++------------------ 2 files changed, 171 insertions(+), 239 deletions(-) diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index 2172aa4c4fe..43ef47adadb 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include @@ -54,55 +55,19 @@ struct FormatDateTimeTraits }; -template struct ActionValueTypeMap {}; -template <> struct ActionValueTypeMap { using ActionValueType = UInt32; }; -template <> struct ActionValueTypeMap { using ActionValueType = UInt32; }; -template <> struct ActionValueTypeMap { using ActionValueType = UInt32; }; -template <> struct ActionValueTypeMap { using ActionValueType = UInt32; }; -template <> struct ActionValueTypeMap { using ActionValueType = UInt32; }; -template <> struct ActionValueTypeMap { using ActionValueType = UInt32; }; -template <> struct ActionValueTypeMap { using ActionValueType = UInt32; }; -template <> struct ActionValueTypeMap { using ActionValueType = UInt32; }; -template <> struct ActionValueTypeMap { using ActionValueType = UInt16; }; -template <> struct ActionValueTypeMap { using ActionValueType = Int32; }; -template <> struct ActionValueTypeMap { using ActionValueType = UInt32; }; -template <> struct ActionValueTypeMap { using ActionValueType = Int64; }; - -/// Counts the number of literal characters in Joda format string until the next closing literal -/// sequence single quote. Returns -1 if no literal single quote was found. -/// In Joda format string(https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html) -/// literal content must be quoted with single quote. and two single quote means literal with one single quote. -/// For example: -/// Format string: "'aaaa'", unescaped literal: "aaaa"; -/// Format string: "'aa''aa'", unescaped literal: "aa'aa"; -/// Format string: "'aaa''aa" is not valid because of missing of end single quote. -Int64 numLiteralChars(const char * cur, const char * end) -{ - bool found = false; - Int64 count = 0; - while (cur < end) - { - if (*cur == '\'') - { - if (cur + 1 < end && *(cur + 1) == '\'') - { - count += 2; - cur += 2; - } - else - { - found = true; - break; - } - } - else - { - ++count; - ++cur; - } - } - return found ? count : -1; -} +template struct InstructionValueTypeMap {}; +template <> struct InstructionValueTypeMap { using InstructionValueType = UInt32; }; +template <> struct InstructionValueTypeMap { using InstructionValueType = UInt32; }; +template <> struct InstructionValueTypeMap { using InstructionValueType = UInt32; }; +template <> struct InstructionValueTypeMap { using InstructionValueType = UInt32; }; +template <> struct InstructionValueTypeMap { using InstructionValueType = UInt32; }; +template <> struct InstructionValueTypeMap { using InstructionValueType = UInt32; }; +template <> struct InstructionValueTypeMap { using InstructionValueType = UInt32; }; +template <> struct InstructionValueTypeMap { using InstructionValueType = UInt32; }; +template <> struct InstructionValueTypeMap { using InstructionValueType = UInt16; }; +template <> struct InstructionValueTypeMap { using InstructionValueType = Int32; }; +template <> struct InstructionValueTypeMap { using InstructionValueType = UInt32; }; +template <> struct InstructionValueTypeMap { using InstructionValueType = Int64; }; /// Cast value from integer to string, making sure digits number in result string is no less than total_digits by padding leading '0'. String padValue(UInt32 val, size_t min_digits) @@ -184,7 +149,7 @@ private: } template - class Action + class Instruction { public: /// Using std::function will cause performance degradation in MySQL format by 0.45x. @@ -201,8 +166,8 @@ private: /// extra_shift is only used in MySQL format syntax. It is always 0 in Joda format syntax. size_t extra_shift = 0; - /// Action for appending date/time related number in specified format. - explicit Action(Func && func_) : func(std::move(func_)) {} + /// Instruction for appending date/time related number in specified format. + explicit Instruction(Func && func_) : func(std::move(func_)) {} void perform(char *& dest, Time source, UInt64 fractional_second, UInt32 scale, const DateLUTImpl & timezone) { @@ -825,8 +790,8 @@ public: if constexpr (std::is_same_v) scale = times->getScale(); - using T = typename ActionValueTypeMap::ActionValueType; - std::vector> instructions; + using T = typename InstructionValueTypeMap::InstructionValueType; + std::vector> instructions; String out_template; auto result_size = parseFormat(format, instructions, scale, out_template); @@ -898,7 +863,7 @@ public: } template - size_t parseFormat(const String & format, std::vector> & instructions, UInt32 scale, String & out_template) const + size_t parseFormat(const String & format, std::vector> & instructions, UInt32 scale, String & out_template) const { if constexpr (format_syntax == FormatDateTimeTraits::FormatSyntax::MySQL) return parseMySQLFormat(format, instructions, scale, out_template); @@ -913,12 +878,12 @@ public: } template - size_t parseMySQLFormat(const String & format, std::vector> & instructions, UInt32 scale, String & out_template) const + size_t parseMySQLFormat(const String & format, std::vector> & instructions, UInt32 scale, String & out_template) const { auto add_extra_shift = [&](size_t amount) { if (instructions.empty()) - instructions.emplace_back(&Action::mysqlNoop); + instructions.emplace_back(&Instruction::mysqlNoop); instructions.back().extra_shift += amount; }; @@ -953,43 +918,43 @@ public: { // Abbreviated weekday [Mon...Sun] case 'a': - instructions.emplace_back(&Action::mysqlDayOfWeekTextShort); + instructions.emplace_back(&Instruction::mysqlDayOfWeekTextShort); out_template += "Mon"; break; // Abbreviated month [Jan...Dec] case 'b': - instructions.emplace_back(&Action::mysqlMonthOfYearTextShort); + instructions.emplace_back(&Instruction::mysqlMonthOfYearTextShort); out_template += "Jan"; break; // Month as a decimal number (01-12) case 'c': - instructions.emplace_back(&Action::mysqlMonth); + instructions.emplace_back(&Instruction::mysqlMonth); out_template += "00"; break; // Year, divided by 100, zero-padded case 'C': - instructions.emplace_back(&Action::mysqlCentury); + instructions.emplace_back(&Instruction::mysqlCentury); out_template += "00"; break; // Day of month, zero-padded (01-31) case 'd': - instructions.emplace_back(&Action::mysqlDayOfMonth); + instructions.emplace_back(&Instruction::mysqlDayOfMonth); out_template += "00"; break; // Short MM/DD/YY date, equivalent to %m/%d/%y case 'D': - instructions.emplace_back(&Action::mysqlAmericanDate); + instructions.emplace_back(&Instruction::mysqlAmericanDate); out_template += "00/00/00"; break; // Day of month, space-padded ( 1-31) 23 case 'e': - instructions.emplace_back(&Action::mysqlDayOfMonthSpacePadded); + instructions.emplace_back(&Instruction::mysqlDayOfMonthSpacePadded); out_template += " 0"; break; @@ -997,86 +962,86 @@ public: case 'f': { /// If the time data type has no fractional part, then we print '0' as the fractional part. - instructions.emplace_back(&Action::mysqlFractionalSecond); + instructions.emplace_back(&Instruction::mysqlFractionalSecond); out_template += String(std::max(1, scale), '0'); break; } // Short YYYY-MM-DD date, equivalent to %Y-%m-%d 2001-08-23 case 'F': - instructions.emplace_back(&Action::mysqlISO8601Date); + instructions.emplace_back(&Instruction::mysqlISO8601Date); out_template += "0000-00-00"; break; // Last two digits of year of ISO 8601 week number (see %G) case 'g': - instructions.emplace_back(&Action::mysqlISO8601Year2); + instructions.emplace_back(&Instruction::mysqlISO8601Year2); out_template += "00"; break; // Year of ISO 8601 week number (see %V) case 'G': - instructions.emplace_back(&Action::mysqlISO8601Year4); + instructions.emplace_back(&Instruction::mysqlISO8601Year4); out_template += "0000"; break; // Day of the year (001-366) 235 case 'j': - instructions.emplace_back(&Action::mysqlDayOfYear); + instructions.emplace_back(&Instruction::mysqlDayOfYear); out_template += "000"; break; // Month as a decimal number (01-12) case 'm': - instructions.emplace_back(&Action::mysqlMonth); + instructions.emplace_back(&Instruction::mysqlMonth); out_template += "00"; break; // ISO 8601 weekday as number with Monday as 1 (1-7) case 'u': - instructions.emplace_back(&Action::mysqlDayOfWeek); + instructions.emplace_back(&Instruction::mysqlDayOfWeek); out_template += "0"; break; // ISO 8601 week number (01-53) case 'V': - instructions.emplace_back(&Action::mysqlISO8601Week); + instructions.emplace_back(&Instruction::mysqlISO8601Week); out_template += "00"; break; // Weekday as a decimal number with Sunday as 0 (0-6) 4 case 'w': - instructions.emplace_back(&Action::mysqlDayOfWeek0To6); + instructions.emplace_back(&Instruction::mysqlDayOfWeek0To6); out_template += "0"; break; // Full weekday [Monday...Sunday] case 'W': - instructions.emplace_back(&Action::mysqlDayOfWeekTextLong); + instructions.emplace_back(&Instruction::mysqlDayOfWeekTextLong); out_template += "Monday"; break; // Two digits year case 'y': - instructions.emplace_back(&Action::mysqlYear2); + instructions.emplace_back(&Instruction::mysqlYear2); out_template += "00"; break; // Four digits year case 'Y': - instructions.emplace_back(&Action::mysqlYear4); + instructions.emplace_back(&Instruction::mysqlYear4); out_template += "0000"; break; // Quarter (1-4) case 'Q': - instructions.template emplace_back(&Action::mysqlQuarter); + instructions.template emplace_back(&Instruction::mysqlQuarter); out_template += "0"; break; // Offset from UTC timezone as +hhmm or -hhmm case 'z': - instructions.emplace_back(&Action::mysqlTimezoneOffset); + instructions.emplace_back(&Instruction::mysqlTimezoneOffset); out_template += "+0000"; break; @@ -1084,79 +1049,79 @@ public: // Minute (00-59) case 'M': - add_instruction_or_extra_shift(&Action::mysqlMinute, 2); + add_instruction_or_extra_shift(&Instruction::mysqlMinute, 2); out_template += "00"; break; // AM or PM case 'p': - add_instruction_or_extra_shift(&Action::mysqlAMPM, 2); + add_instruction_or_extra_shift(&Instruction::mysqlAMPM, 2); out_template += "AM"; break; // 12-hour HH:MM time, equivalent to %h:%i %p 2:55 PM case 'r': - add_instruction_or_extra_shift(&Action::mysqlHHMM12, 8); + add_instruction_or_extra_shift(&Instruction::mysqlHHMM12, 8); out_template += "12:00 AM"; break; // 24-hour HH:MM time, equivalent to %H:%i 14:55 case 'R': - add_instruction_or_extra_shift(&Action::mysqlHHMM24, 5); + add_instruction_or_extra_shift(&Instruction::mysqlHHMM24, 5); out_template += "00:00"; break; // Seconds case 's': - add_instruction_or_extra_shift(&Action::mysqlSecond, 2); + add_instruction_or_extra_shift(&Instruction::mysqlSecond, 2); out_template += "00"; break; // Seconds case 'S': - add_instruction_or_extra_shift(&Action::mysqlSecond, 2); + add_instruction_or_extra_shift(&Instruction::mysqlSecond, 2); out_template += "00"; break; // ISO 8601 time format (HH:MM:SS), equivalent to %H:%i:%S 14:55:02 case 'T': - add_instruction_or_extra_shift(&Action::mysqlISO8601Time, 8); + add_instruction_or_extra_shift(&Instruction::mysqlISO8601Time, 8); out_template += "00:00:00"; break; // Hour in 12h format (01-12) case 'h': - add_instruction_or_extra_shift(&Action::mysqlHour12, 2); + add_instruction_or_extra_shift(&Instruction::mysqlHour12, 2); out_template += "12"; break; // Hour in 24h format (00-23) case 'H': - add_instruction_or_extra_shift(&Action::mysqlHour24, 2); + add_instruction_or_extra_shift(&Instruction::mysqlHour24, 2); out_template += "00"; break; // Minute of hour range [0, 59] case 'i': - add_instruction_or_extra_shift(&Action::mysqlMinute, 2); + add_instruction_or_extra_shift(&Instruction::mysqlMinute, 2); out_template += "00"; break; // Hour in 12h format (01-12) case 'I': - add_instruction_or_extra_shift(&Action::mysqlHour12, 2); + add_instruction_or_extra_shift(&Instruction::mysqlHour12, 2); out_template += "12"; break; // Hour in 24h format (00-23) case 'k': - add_instruction_or_extra_shift(&Action::mysqlHour24, 2); + add_instruction_or_extra_shift(&Instruction::mysqlHour24, 2); out_template += "00"; break; // Hour in 12h format (01-12) case 'l': - add_instruction_or_extra_shift(&Action::mysqlHour12, 2); + add_instruction_or_extra_shift(&Instruction::mysqlHour12, 2); out_template += "12"; break; @@ -1209,7 +1174,7 @@ public: } template - size_t parseJodaFormat(const String & format, std::vector> & instructions, UInt32, String &) const + size_t parseJodaFormat(const String & format, std::vector> & instructions, UInt32, String &) const { /// If the argument was DateTime, add instruction for printing. If it was date, just append default literal auto add_instruction = [&](auto && func [[maybe_unused]], const String & default_literal [[maybe_unused]]) @@ -1217,7 +1182,7 @@ public: if constexpr (std::is_same_v || std::is_same_v) instructions.emplace_back(func); else - instructions.emplace_back(std::bind_front(&Action::template jodaLiteral, default_literal)); + instructions.emplace_back(std::bind_front(&Instruction::template jodaLiteral, default_literal)); }; size_t reserve_size = 0; @@ -1235,7 +1200,7 @@ public: if (pos + 1 < end && *(pos + 1) == '\'') { std::string_view literal(cur_token, 1); - instructions.emplace_back(std::bind_front(&Action::template jodaLiteral, literal)); + instructions.emplace_back(std::bind_front(&Instruction::template jodaLiteral, literal)); ++reserve_size; pos += 2; } @@ -1251,7 +1216,7 @@ public: { std::string_view literal(cur_token + i, 1); instructions.emplace_back( - std::bind_front(&Action::template jodaLiteral, literal)); + std::bind_front(&Instruction::template jodaLiteral, literal)); ++reserve_size; if (*(cur_token + i) == '\'') i += 1; @@ -1272,115 +1237,115 @@ public: switch (*cur_token) { case 'G': - instructions.emplace_back(std::bind_front(&Action::jodaEra, repetitions)); + instructions.emplace_back(std::bind_front(&Instruction::jodaEra, repetitions)); reserve_size += repetitions <= 3 ? 2 : 13; break; case 'C': - instructions.emplace_back(std::bind_front(&Action::jodaCenturyOfEra, repetitions)); + instructions.emplace_back(std::bind_front(&Instruction::jodaCenturyOfEra, repetitions)); /// Year range [1900, 2299] reserve_size += std::max(repetitions, 2); break; case 'Y': - instructions.emplace_back(std::bind_front(&Action::jodaYearOfEra, repetitions)); + instructions.emplace_back(std::bind_front(&Instruction::jodaYearOfEra, repetitions)); /// Year range [1900, 2299] reserve_size += repetitions == 2 ? 2 : std::max(repetitions, 4); break; case 'x': - instructions.emplace_back(std::bind_front(&Action::jodaWeekYear, repetitions)); + instructions.emplace_back(std::bind_front(&Instruction::jodaWeekYear, repetitions)); /// weekyear range [1900, 2299] reserve_size += std::max(repetitions, 4); break; case 'w': - instructions.emplace_back(std::bind_front(&Action::jodaWeekOfWeekYear, repetitions)); + instructions.emplace_back(std::bind_front(&Instruction::jodaWeekOfWeekYear, repetitions)); /// Week of weekyear range [1, 52] reserve_size += std::max(repetitions, 2); break; case 'e': - instructions.emplace_back(std::bind_front(&Action::jodaDayOfWeek1Based, repetitions)); + instructions.emplace_back(std::bind_front(&Instruction::jodaDayOfWeek1Based, repetitions)); /// Day of week range [1, 7] reserve_size += std::max(repetitions, 1); break; case 'E': - instructions.emplace_back(std::bind_front(&Action::jodaDayOfWeekText, repetitions)); + instructions.emplace_back(std::bind_front(&Instruction::jodaDayOfWeekText, repetitions)); /// Maximum length of short name is 3, maximum length of full name is 9. reserve_size += repetitions <= 3 ? 3 : 9; break; case 'y': - instructions.emplace_back(std::bind_front(&Action::jodaYear, repetitions)); + instructions.emplace_back(std::bind_front(&Instruction::jodaYear, repetitions)); /// Year range [1900, 2299] reserve_size += repetitions == 2 ? 2 : std::max(repetitions, 4); break; case 'D': - instructions.emplace_back(std::bind_front(&Action::jodaDayOfYear, repetitions)); + instructions.emplace_back(std::bind_front(&Instruction::jodaDayOfYear, repetitions)); /// Day of year range [1, 366] reserve_size += std::max(repetitions, 3); break; case 'M': if (repetitions <= 2) { - instructions.emplace_back(std::bind_front(&Action::jodaMonthOfYear, repetitions)); + instructions.emplace_back(std::bind_front(&Instruction::jodaMonthOfYear, repetitions)); /// Month of year range [1, 12] reserve_size += 2; } else { - instructions.emplace_back(std::bind_front(&Action::jodaMonthOfYearText, repetitions)); + instructions.emplace_back(std::bind_front(&Instruction::jodaMonthOfYearText, repetitions)); /// Maximum length of short name is 3, maximum length of full name is 9. reserve_size += repetitions <= 3 ? 3 : 9; } break; case 'd': - instructions.emplace_back(std::bind_front(&Action::jodaDayOfMonth, repetitions)); + instructions.emplace_back(std::bind_front(&Instruction::jodaDayOfMonth, repetitions)); /// Day of month range [1, 3] reserve_size += std::max(repetitions, 3); break; case 'a': /// Default half day of day is "AM" - add_instruction(std::bind_front(&Action::jodaHalfDayOfDay, repetitions), "AM"); + add_instruction(std::bind_front(&Instruction::jodaHalfDayOfDay, repetitions), "AM"); reserve_size += 2; break; case 'K': /// Default hour of half day is 0 add_instruction( - std::bind_front(&Action::jodaHourOfHalfDay, repetitions), padValue(0, repetitions)); + std::bind_front(&Instruction::jodaHourOfHalfDay, repetitions), padValue(0, repetitions)); /// Hour of half day range [0, 11] reserve_size += std::max(repetitions, 2); break; case 'h': /// Default clock hour of half day is 12 add_instruction( - std::bind_front(&Action::jodaClockHourOfHalfDay, repetitions), + std::bind_front(&Instruction::jodaClockHourOfHalfDay, repetitions), padValue(12, repetitions)); /// Clock hour of half day range [1, 12] reserve_size += std::max(repetitions, 2); break; case 'H': /// Default hour of day is 0 - add_instruction(std::bind_front(&Action::jodaHourOfDay, repetitions), padValue(0, repetitions)); + add_instruction(std::bind_front(&Instruction::jodaHourOfDay, repetitions), padValue(0, repetitions)); /// Hour of day range [0, 23] reserve_size += std::max(repetitions, 2); break; case 'k': /// Default clock hour of day is 24 - add_instruction(std::bind_front(&Action::jodaClockHourOfDay, repetitions), padValue(24, repetitions)); + add_instruction(std::bind_front(&Instruction::jodaClockHourOfDay, repetitions), padValue(24, repetitions)); /// Clock hour of day range [1, 24] reserve_size += std::max(repetitions, 2); break; case 'm': /// Default minute of hour is 0 - add_instruction(std::bind_front(&Action::jodaMinuteOfHour, repetitions), padValue(0, repetitions)); + add_instruction(std::bind_front(&Instruction::jodaMinuteOfHour, repetitions), padValue(0, repetitions)); /// Minute of hour range [0, 59] reserve_size += std::max(repetitions, 2); break; case 's': /// Default second of minute is 0 - add_instruction(std::bind_front(&Action::jodaSecondOfMinute, repetitions), padValue(0, repetitions)); + add_instruction(std::bind_front(&Instruction::jodaSecondOfMinute, repetitions), padValue(0, repetitions)); /// Second of minute range [0, 59] reserve_size += std::max(repetitions, 2); break; case 'S': /// Default fraction of second is 0 - instructions.emplace_back(std::bind_front(&Action::jodaFractionOfSecond, repetitions)); + instructions.emplace_back(std::bind_front(&Instruction::jodaFractionOfSecond, repetitions)); /// 'S' repetitions range [0, 9] reserve_size += repetitions <= 9 ? repetitions : 9; break; @@ -1388,7 +1353,7 @@ public: if (repetitions <= 3) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Short name time zone is not yet supported"); - instructions.emplace_back(std::bind_front(&Action::jodaTimezone, repetitions)); + instructions.emplace_back(std::bind_front(&Instruction::jodaTimezone, repetitions)); /// Longest length of full name of time zone is 32. reserve_size += 32; break; @@ -1399,7 +1364,7 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for {}", String(cur_token, repetitions)); std::string_view literal(cur_token, pos - cur_token); - instructions.emplace_back(std::bind_front(&Action::template jodaLiteral, literal)); + instructions.emplace_back(std::bind_front(&Instruction::template jodaLiteral, literal)); reserve_size += pos - cur_token; break; } diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index b0b931c5243..c8a558c831a 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -8,9 +8,11 @@ #include #include #include +#include #include #include +#include namespace DB { @@ -22,6 +24,8 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NOT_IMPLEMENTED; extern const int BAD_ARGUMENTS; + extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; + extern const int CANNOT_PARSE_TEXT; } namespace @@ -79,41 +83,12 @@ namespace 39447, 39812, 40177, 40543, 40908, 41273, 41638, 42004, 42369, 42734, 43099, 43465, 43830, 44195, 44560, 44926, 45291, 45656, 46021, 46387, 46752, 47117, 47482, 47847, 48212, 48577, 48942, 49308, 49673}; - Int64 numLiteralChars(const char * cur, const char * end) - { - bool found = false; - Int64 count = 0; - while (cur < end) - { - if (*cur == '\'') - { - if (cur + 1 < end && *(cur + 1) == '\'') - { - count += 2; - cur += 2; - } - else - { - found = true; - break; - } - } - else - { - ++count; - ++cur; - } - } - return found ? count : -1; - } - struct DateTime { Int32 year = 1970; Int32 month = 1; Int32 day = 1; std::vector day_of_month_values; - bool is_ad = true; // AD -> true, BC -> false. Int32 week = 1; // Week of year based on ISO week date, e.g: 27 Int32 day_of_week = 1; // Day of week, Monday:1, Tuesday:2, ..., Sunday:7 @@ -144,7 +119,6 @@ namespace month = 1; day = 1; day_of_month_values.clear(); - is_ad = true; week = 1; day_of_week = 1; @@ -301,20 +275,18 @@ namespace second = second_; } - void setEra(String & text) + void setEra(String & text) // NOLINT { - Poco::toLowerInPlace(text); - if (text == "ad") - is_ad = true; - else if (text == "bc") - is_ad = false; - else + boost::to_lower(text); + if (text == "bc") + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Era BC exceeds the range of DateTime"); + else if (text != "ad") throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown era {}", text); } ALWAYS_INLINE void setAMPM(String & text) { - Poco::toLowerInPlace(text); + boost::to_lower(text); if (text == "am") is_am = true; else if (text == "pm") @@ -450,10 +422,6 @@ namespace Int64 checkAndGetDateTime(const DateLUTImpl & time_zone) { - /// Era is BC and year of era is provided - if (is_year_of_era && !is_ad) - year = -1 * (year - 1); - if (is_hour_of_half_day && !is_am) hour += 12; @@ -572,7 +540,7 @@ namespace String format = getFormat(arguments); const auto * time_zone = getTimeZone(arguments).first; - std::vector instructions; + std::vector instructions; parseFormat(format, instructions); auto col_res = ColumnDateTime::create(); @@ -610,7 +578,7 @@ namespace private: - class Action + class Instruction { private: enum class NeedCheckSpace @@ -629,10 +597,10 @@ namespace std::string literal; public: - explicit Action(Func && func_, const char * func_name_) : func(std::move(func_)), func_name(func_name_) { } + explicit Instruction(Func && func_, const char * func_name_) : func(std::move(func_)), func_name(func_name_) { } - explicit Action(const String & literal_) : literal(literal_) { } - explicit Action(String && literal_) : literal(std::move(literal_)) { } + explicit Instruction(const String & literal_) : literal(literal_) { } + explicit Instruction(String && literal_) : literal(std::move(literal_)) { } /// For debug [[maybe_unused]] String toString() const @@ -705,7 +673,7 @@ namespace static void checkSpace(Pos cur, Pos end, size_t len, const String & msg) { - if (cur > end || cur + len > end) + if (cur > end || cur + len > end) [[unlikely]] throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because {}", msg); } @@ -727,10 +695,10 @@ namespace checkSpace(cur, end, 3, "Parsing DayOfWeekTextShort requires size >= 3"); String text(cur, 3); - Poco::toLowerInPlace(text); + boost::to_lower(text); auto it = dayOfWeekMap.find(text); if (it == dayOfWeekMap.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown day of week short text {}", text); + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Unknown day of week short text {}", text); date.setDayOfWeek(it->second.second); cur += 3; return cur; @@ -741,10 +709,10 @@ namespace checkSpace(cur, end, 3, "Parsing MonthOfYearTextShort requires size >= 3"); String text(cur, 3); - Poco::toLowerInPlace(text); + boost::to_lower(text); auto it = monthMap.find(text); if (it == monthMap.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown month of year short text {}", text); + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Unknown month of year short text {}", text); date.setMonth(it->second.second); cur += 3; @@ -885,18 +853,18 @@ namespace { checkSpace(cur, end, 6, "jodaDayOfWeekText requires the size >= 6"); String text1(cur, 3); - Poco::toLowerInPlace(text1); + boost::to_lower(text1); auto it = dayOfWeekMap.find(text1); if (it == dayOfWeekMap.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown day of week text: {}", text1); + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Unknown day of week text: {}", text1); cur += 3; size_t left_size = it->second.first.size(); checkSpace(cur, end, left_size, "jodaDayOfWeekText requires the second parg size >= " + std::to_string(left_size)); String text2(cur, left_size); - Poco::toLowerInPlace(text2); + boost::to_lower(text2); if (text2 != it->second.first) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown day of week text: {}", text1 + text2); + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Unknown day of week text: {}", text1 + text2); cur += left_size; date.setDayOfWeek(it->second.second); @@ -1168,10 +1136,10 @@ namespace checkSpace(cur, end, 3, "jodaDayOfWeekText requires size >= 3"); String text1(cur, 3); - Poco::toLowerInPlace(text1); + boost::to_lower(text1); auto it = dayOfWeekMap.find(text1); if (it == dayOfWeekMap.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown day of week text: {}", text1); + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Unknown day of week text: {}", text1); cur += 3; date.setDayOfWeek(it->second.second); @@ -1179,7 +1147,7 @@ namespace if (cur + left_size <= end) { String text2(cur, left_size); - Poco::toLowerInPlace(text2); + boost::to_lower(text2); if (text2 == it->second.first) { cur += left_size; @@ -1217,10 +1185,10 @@ namespace { checkSpace(cur, end, 3, "jodaMonthOfYearText requires size >= 3"); String text1(cur, 3); - Poco::toLowerInPlace(text1); + boost::to_lower(text1); auto it = monthMap.find(text1); if (it == monthMap.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown month of year text: {}", text1); + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Unknown month of year text: {}", text1); cur += 3; date.setMonth(it->second.second); @@ -1228,7 +1196,7 @@ namespace if (cur + left_size <= end) { String text2(cur, left_size); - Poco::toLowerInPlace(text2); + boost::to_lower(text2); if (text2 == it->second.first) { cur += left_size; @@ -1306,7 +1274,7 @@ namespace }; - ALWAYS_INLINE void parseFormat(const String & format, std::vector & instructions) const + ALWAYS_INLINE void parseFormat(const String & format, std::vector & instructions) const { if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::MySQL) parseMysqlFormat(format, instructions); @@ -1320,7 +1288,7 @@ namespace getName()); } - ALWAYS_INLINE void parseMysqlFormat(const String & format, std::vector & instructions) const + ALWAYS_INLINE void parseMysqlFormat(const String & format, std::vector & instructions) const { #define ACTION_ARGS(func) &(func), #func @@ -1328,13 +1296,13 @@ namespace Pos end = pos + format.size(); while (true) { - Pos percent_pos = find_first_symbols<'%'>(pos, end); - if (percent_pos < end) + Pos next_percent_pos = find_first_symbols<'%'>(pos, end); + if (next_percent_pos < end) { - if (pos < percent_pos) - instructions.emplace_back(String(pos, percent_pos - pos)); + if (pos < next_percent_pos) + instructions.emplace_back(String(pos, next_percent_pos - pos)); - pos = percent_pos + 1; + pos = next_percent_pos + 1; if (pos >= end) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Sign '%' is the last in format, if you need it, use '%%'"); @@ -1342,37 +1310,37 @@ namespace { // Abbreviated weekday [Mon...Sun] case 'a': - instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfWeekTextShort)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlDayOfWeekTextShort)); break; // Abbreviated month [Jan...Dec] case 'b': - instructions.emplace_back(ACTION_ARGS(Action::mysqlMonthOfYearTextShort)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlMonthOfYearTextShort)); break; // Month as a decimal number (01-12) case 'c': - instructions.emplace_back(ACTION_ARGS(Action::mysqlMonth)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlMonth)); break; // Year, divided by 100, zero-padded case 'C': - instructions.emplace_back(ACTION_ARGS(Action::mysqlCentury)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlCentury)); break; // Day of month, zero-padded (01-31) case 'd': - instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfMonth)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlDayOfMonth)); break; // Short MM/DD/YY date, equivalent to %m/%d/%y case 'D': - instructions.emplace_back(ACTION_ARGS(Action::mysqlAmericanDate)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlAmericanDate)); break; // Day of month, space-padded ( 1-31) 23 case 'e': - instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfMonthSpacePadded)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlDayOfMonthSpacePadded)); break; // Fractional seconds @@ -1381,57 +1349,57 @@ namespace // Short YYYY-MM-DD date, equivalent to %Y-%m-%d 2001-08-23 case 'F': - instructions.emplace_back(ACTION_ARGS(Action::mysqlISO8601Date)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlISO8601Date)); break; // Last two digits of year of ISO 8601 week number (see %G) case 'g': - instructions.emplace_back(ACTION_ARGS(Action::mysqlISO8601Year2)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlISO8601Year2)); break; // Year of ISO 8601 week number (see %V) case 'G': - instructions.emplace_back(ACTION_ARGS(Action::mysqlISO8601Year4)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlISO8601Year4)); break; // Day of the year (001-366) 235 case 'j': - instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfYear)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlDayOfYear)); break; // Month as a decimal number (01-12) case 'm': - instructions.emplace_back(ACTION_ARGS(Action::mysqlMonth)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlMonth)); break; // ISO 8601 weekday as number with Monday as 1 (1-7) case 'u': - instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfWeek)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlDayOfWeek)); break; // ISO 8601 week number (01-53) case 'V': - instructions.emplace_back(ACTION_ARGS(Action::mysqlISO8601Week)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlISO8601Week)); break; // Weekday as a decimal number with Sunday as 0 (0-6) 4 case 'w': - instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfWeek0To6)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlDayOfWeek0To6)); break; // Full weekday [Monday...Sunday] case 'W': - instructions.emplace_back(ACTION_ARGS(Action::mysqlDayOfWeekTextLong)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlDayOfWeekTextLong)); break; // Two digits year case 'y': - instructions.emplace_back(ACTION_ARGS(Action::mysqlYear2)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlYear2)); break; // Four digits year case 'Y': - instructions.emplace_back(ACTION_ARGS(Action::mysqlYear4)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlYear4)); break; // Quarter (1-4) @@ -1441,74 +1409,74 @@ namespace // Offset from UTC timezone as +hhmm or -hhmm case 'z': - instructions.emplace_back(ACTION_ARGS(Action::mysqlTimezoneOffset)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlTimezoneOffset)); break; /// Time components. If the argument is Date, not a DateTime, then this components will have default value. // Minute (00-59) case 'M': - instructions.emplace_back(ACTION_ARGS(Action::mysqlMinute)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlMinute)); break; // AM or PM case 'p': - instructions.emplace_back(ACTION_ARGS(Action::mysqlAMPM)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlAMPM)); break; // 12-hour HH:MM time, equivalent to %h:%i %p 2:55 PM case 'r': - instructions.emplace_back(ACTION_ARGS(Action::mysqlHHMM12)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlHHMM12)); break; // 24-hour HH:MM time, equivalent to %H:%i 14:55 case 'R': - instructions.emplace_back(ACTION_ARGS(Action::mysqlHHMM24)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlHHMM24)); break; // Seconds case 's': - instructions.emplace_back(ACTION_ARGS(Action::mysqlSecond)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlSecond)); break; // Seconds case 'S': - instructions.emplace_back(ACTION_ARGS(Action::mysqlSecond)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlSecond)); break; // ISO 8601 time format (HH:MM:SS), equivalent to %H:%i:%S 14:55:02 case 'T': - instructions.emplace_back(ACTION_ARGS(Action::mysqlISO8601Time)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlISO8601Time)); break; // Hour in 12h format (01-12) case 'h': - instructions.emplace_back(ACTION_ARGS(Action::mysqlHour12)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlHour12)); break; // Hour in 24h format (00-23) case 'H': - instructions.emplace_back(ACTION_ARGS(Action::mysqlHour24)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlHour24)); break; // Minute of hour range [0, 59] case 'i': - instructions.emplace_back(ACTION_ARGS(Action::mysqlMinute)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlMinute)); break; // Hour in 12h format (01-12) case 'I': - instructions.emplace_back(ACTION_ARGS(Action::mysqlHour12)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlHour12)); break; // Hour in 24h format (00-23) case 'k': - instructions.emplace_back(ACTION_ARGS(Action::mysqlHour24)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlHour24)); break; // Hour in 12h format (01-12) case 'l': - instructions.emplace_back(ACTION_ARGS(Action::mysqlHour12)); + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlHour12)); break; case 't': @@ -1555,7 +1523,7 @@ namespace #undef ACTION_ARGS } - void parseJodaFormat(const String & format, std::vector & instructions) const + void parseJodaFormat(const String & format, std::vector & instructions) const { #define ACTION_ARGS_WITH_BIND(func, arg) std::bind_front(&(func), (arg)), #func @@ -1574,7 +1542,6 @@ namespace if (pos + 1 < end && *(pos + 1) == '\'') { instructions.emplace_back(String(cur_token, 1)); - // ++reserve_size; pos += 2; } else @@ -1608,61 +1575,61 @@ namespace switch (*cur_token) { case 'G': - instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaEra, repetitions)); + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaEra, repetitions)); break; case 'C': - instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaCenturyOfEra, repetitions)); + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaCenturyOfEra, repetitions)); break; case 'Y': - instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaYearOfEra, repetitions)); + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaYearOfEra, repetitions)); break; case 'x': - instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaWeekYear, repetitions)); + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaWeekYear, repetitions)); break; case 'w': - instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaWeekOfWeekYear, repetitions)); + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaWeekOfWeekYear, repetitions)); break; case 'e': - instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaDayOfWeek1Based, repetitions)); + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaDayOfWeek1Based, repetitions)); break; case 'E': - instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaDayOfWeekText, repetitions)); + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaDayOfWeekText, repetitions)); break; case 'y': - instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaYear, repetitions)); + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaYear, repetitions)); break; case 'D': - instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaDayOfYear, repetitions)); + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaDayOfYear, repetitions)); break; case 'M': if (repetitions <= 2) - instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaMonthOfYear, repetitions)); + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaMonthOfYear, repetitions)); else - instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaMonthOfYearText, repetitions)); + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaMonthOfYearText, repetitions)); break; case 'd': - instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaDayOfMonth, repetitions)); + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaDayOfMonth, repetitions)); break; case 'a': - instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaHalfDayOfDay, repetitions)); + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaHalfDayOfDay, repetitions)); break; case 'K': - instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaHourOfHalfDay, repetitions)); + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaHourOfHalfDay, repetitions)); break; case 'h': - instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaClockHourOfHalfDay, repetitions)); + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaClockHourOfHalfDay, repetitions)); break; case 'H': - instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaHourOfDay, repetitions)); + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaHourOfDay, repetitions)); break; case 'k': - instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaClockHourOfDay, repetitions)); + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaClockHourOfDay, repetitions)); break; case 'm': - instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaMinuteOfHour, repetitions)); + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaMinuteOfHour, repetitions)); break; case 's': - instructions.emplace_back(ACTION_ARGS_WITH_BIND(Action::jodaSecondOfMinute, repetitions)); + instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaSecondOfMinute, repetitions)); break; case 'S': throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for fractional seconds"); From 8f2d75cef851072cc32d1955fef0a0e8f5ec9a78 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 5 Mar 2023 12:50:29 +0100 Subject: [PATCH 105/559] Fix tests --- src/Databases/DatabaseFactory.cpp | 9 +++++---- src/Storages/Kafka/StorageKafka.cpp | 2 +- src/Storages/MeiliSearch/StorageMeiliSearch.cpp | 2 +- src/Storages/NATS/StorageNATS.cpp | 2 +- src/Storages/NamedCollectionsHelpers.cpp | 12 +++++++----- src/Storages/NamedCollectionsHelpers.h | 5 ++++- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 2 +- src/Storages/StorageMongoDB.cpp | 2 +- src/Storages/StorageMySQL.cpp | 7 +++---- src/Storages/StoragePostgreSQL.cpp | 6 +++--- src/Storages/StorageS3.cpp | 2 +- src/Storages/StorageURL.cpp | 2 +- src/TableFunctions/TableFunctionRemote.cpp | 2 +- src/TableFunctions/TableFunctionS3.cpp | 2 +- src/TableFunctions/TableFunctionURL.cpp | 2 +- tests/integration/test_mask_sensitive_info/test.py | 8 ++++---- tests/integration/test_storage_postgresql/test.py | 2 +- 17 files changed, 37 insertions(+), 32 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index cbd9f84df60..47cf0c6b6d0 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -186,7 +186,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String ASTs & arguments = engine->arguments->children; auto mysql_settings = std::make_unique(); - if (auto named_collection = tryGetNamedCollectionWithOverrides(arguments)) + if (auto named_collection = tryGetNamedCollectionWithOverrides(arguments, context)) { configuration = StorageMySQL::processNamedCollectionResult(*named_collection, *mysql_settings, false); } @@ -222,7 +222,8 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String if (engine_name == "MySQL") { mysql_settings->loadFromQuery(*engine_define); - mysql_settings->loadFromQueryContext(context, *engine_define); /// Will override only if not changed. + if (engine_define->settings) + mysql_settings->loadFromQueryContext(context, *engine_define); /// Will override only if not changed. auto mysql_pool = createMySQLPoolWithFailover(configuration, *mysql_settings); @@ -315,7 +316,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String auto use_table_cache = false; StoragePostgreSQL::Configuration configuration; - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args)) + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, context)) { configuration = StoragePostgreSQL::processNamedCollectionResult(*named_collection, false); use_table_cache = named_collection->getOrDefault("use_tables_cache", 0); @@ -378,7 +379,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String ASTs & engine_args = engine->arguments->children; StoragePostgreSQL::Configuration configuration; - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args)) + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, context)) { configuration = StoragePostgreSQL::processNamedCollectionResult(*named_collection, false); } diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 86410447ee1..dedad6f44d7 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -838,7 +838,7 @@ void registerStorageKafka(StorageFactory & factory) auto kafka_settings = std::make_unique(); String collection_name; - if (auto named_collection = tryGetNamedCollectionWithOverrides(args.engine_args)) + if (auto named_collection = tryGetNamedCollectionWithOverrides(args.engine_args, args.getLocalContext())) { for (const auto & setting : kafka_settings->all()) { diff --git a/src/Storages/MeiliSearch/StorageMeiliSearch.cpp b/src/Storages/MeiliSearch/StorageMeiliSearch.cpp index 56dad2a0d13..62a6c471070 100644 --- a/src/Storages/MeiliSearch/StorageMeiliSearch.cpp +++ b/src/Storages/MeiliSearch/StorageMeiliSearch.cpp @@ -129,7 +129,7 @@ SinkToStoragePtr StorageMeiliSearch::write(const ASTPtr & /*query*/, const Stora MeiliSearchConfiguration StorageMeiliSearch::getConfiguration(ASTs engine_args, ContextPtr context) { - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args)) + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, context)) { validateNamedCollection(*named_collection, {"url", "index"}, {"key"}); diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index feb44fe92e4..aa4ec77b0d8 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -711,7 +711,7 @@ void registerStorageNATS(StorageFactory & factory) auto creator_fn = [](const StorageFactory::Arguments & args) { auto nats_settings = std::make_unique(); - if (auto named_collection = tryGetNamedCollectionWithOverrides(args.engine_args)) + if (auto named_collection = tryGetNamedCollectionWithOverrides(args.engine_args, args.getLocalContext())) { for (const auto & setting : nats_settings->all()) { diff --git a/src/Storages/NamedCollectionsHelpers.cpp b/src/Storages/NamedCollectionsHelpers.cpp index 0cca2e4b9df..c6e928d9412 100644 --- a/src/Storages/NamedCollectionsHelpers.cpp +++ b/src/Storages/NamedCollectionsHelpers.cpp @@ -30,7 +30,7 @@ namespace return NamedCollectionFactory::instance().tryGet(collection_name); } - std::optional>> getKeyValueFromAST(ASTPtr ast, bool fallback_to_ast_value) + std::optional>> getKeyValueFromAST(ASTPtr ast, bool fallback_to_ast_value, ContextPtr context) { const auto * function = ast->as(); if (!function || function->name != "equals") @@ -42,14 +42,16 @@ namespace if (function_args.size() != 2) return std::nullopt; - auto context = Context::getGlobalContextInstance(); auto literal_key = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[0], context); auto key = checkAndGetLiteralArgument(literal_key, "key"); ASTPtr literal_value; try { - literal_value = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[1], context); + if (key == "database" || key == "db") + literal_value = evaluateConstantExpressionForDatabaseName(function_args[1], context); + else + literal_value = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[1], context); } catch (...) { @@ -65,7 +67,7 @@ namespace MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides( - ASTs asts, bool throw_unknown_collection, std::vector> * complex_args) + ASTs asts, ContextPtr context, bool throw_unknown_collection, std::vector> * complex_args) { if (asts.empty()) return nullptr; @@ -83,7 +85,7 @@ MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides( for (auto * it = std::next(asts.begin()); it != asts.end(); ++it) { - auto value_override = getKeyValueFromAST(*it, complex_args != nullptr); + auto value_override = getKeyValueFromAST(*it, complex_args != nullptr, context); if (!value_override && !(*it)->as()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value argument or function"); diff --git a/src/Storages/NamedCollectionsHelpers.h b/src/Storages/NamedCollectionsHelpers.h index 86e215bccf5..0510ed7e298 100644 --- a/src/Storages/NamedCollectionsHelpers.h +++ b/src/Storages/NamedCollectionsHelpers.h @@ -18,7 +18,7 @@ namespace DB /// Helper function to get named collection for table engine. /// Table engines have collection name as first argument of ast and other arguments are key-value overrides. -MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts, bool throw_unknown_collection = true, std::vector> * non_convertible = nullptr); +MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts, ContextPtr context, bool throw_unknown_collection = true, std::vector> * non_convertible = nullptr); /// Helper function to get named collection for dictionary source. /// Dictionaries have collection name as name argument of dict configuration and other arguments are overrides. MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); @@ -109,6 +109,9 @@ void validateNamedCollection( continue; } + if (required_keys.contains(key)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Duplicate key {} in named collection", key); + auto match = std::find_if( optional_regex_keys.begin(), optional_regex_keys.end(), [&](const std::regex & regex) { return std::regex_search(key, regex); }) diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index c5ea6f810ef..b7fb2c6df64 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -1195,7 +1195,7 @@ void registerStorageRabbitMQ(StorageFactory & factory) { auto rabbitmq_settings = std::make_unique(); - if (auto named_collection = tryGetNamedCollectionWithOverrides(args.engine_args)) + if (auto named_collection = tryGetNamedCollectionWithOverrides(args.engine_args, args.getLocalContext())) { for (const auto & setting : rabbitmq_settings->all()) { diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 57aa81efe0a..0bad4ab8bf0 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -176,7 +176,7 @@ StorageMongoDB::Configuration StorageMongoDB::getConfiguration(ASTs engine_args, { Configuration configuration; - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args)) + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, context)) { validateNamedCollection( *named_collection, diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 61c715bdaeb..7e133538e41 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -280,7 +280,7 @@ StorageMySQL::Configuration StorageMySQL::processNamedCollectionResult( StorageMySQL::Configuration StorageMySQL::getConfiguration(ASTs engine_args, ContextPtr context_, MySQLSettings & storage_settings) { StorageMySQL::Configuration configuration; - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args)) + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, context_)) { configuration = StorageMySQL::processNamedCollectionResult(*named_collection, storage_settings); } @@ -294,11 +294,10 @@ StorageMySQL::Configuration StorageMySQL::getConfiguration(ASTs engine_args, Con for (auto & engine_arg : engine_args) engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context_); - const auto & host_port = checkAndGetLiteralArgument(engine_args[0], "host:port"); + configuration.addresses_expr = checkAndGetLiteralArgument(engine_args[0], "host:port"); size_t max_addresses = context_->getSettingsRef().glob_expansion_max_elements; - configuration.addresses_expr = host_port; - configuration.addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306); + configuration.addresses = parseRemoteDescriptionForExternalDatabase(configuration.addresses_expr, max_addresses, 3306); configuration.database = checkAndGetLiteralArgument(engine_args[1], "database"); configuration.table = checkAndGetLiteralArgument(engine_args[2], "table"); configuration.username = checkAndGetLiteralArgument(engine_args[3], "username"); diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index c208ad8ef5d..cf87d23bf94 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -419,7 +419,7 @@ StoragePostgreSQL::Configuration StoragePostgreSQL::processNamedCollectionResult StoragePostgreSQL::Configuration StoragePostgreSQL::getConfiguration(ASTs engine_args, ContextPtr context) { StoragePostgreSQL::Configuration configuration; - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args)) + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, context)) { configuration = StoragePostgreSQL::processNamedCollectionResult(*named_collection); } @@ -438,10 +438,10 @@ StoragePostgreSQL::Configuration StoragePostgreSQL::getConfiguration(ASTs engine for (auto & engine_arg : engine_args) engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context); - const auto & host_port = checkAndGetLiteralArgument(engine_args[0], "host:port"); + configuration.addresses_expr = checkAndGetLiteralArgument(engine_args[0], "host:port"); size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements; - configuration.addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 5432); + configuration.addresses = parseRemoteDescriptionForExternalDatabase(configuration.addresses_expr, max_addresses, 5432); if (configuration.addresses.size() == 1) { configuration.host = configuration.addresses[0].first; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index ed290c38c1f..d605eaf2d13 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1294,7 +1294,7 @@ StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, Context { StorageS3::Configuration configuration; - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args)) + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) { processNamedCollectionResult(configuration, *named_collection); } diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 152dda8f360..691254867ac 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -1091,7 +1091,7 @@ StorageURL::Configuration StorageURL::getConfiguration(ASTs & args, ContextPtr l { StorageURL::Configuration configuration; - if (auto named_collection = tryGetNamedCollectionWithOverrides(args)) + if (auto named_collection = tryGetNamedCollectionWithOverrides(args, local_context)) { StorageURL::processNamedCollectionResult(configuration, *named_collection); collectHeaders(args, configuration.headers, local_context); diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index eed9e07b532..1ee51bcb040 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -52,7 +52,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr size_t max_args = is_cluster_function ? 4 : 6; NamedCollectionPtr named_collection; std::vector> complex_args; - if (!is_cluster_function && (named_collection = tryGetNamedCollectionWithOverrides(args, false, &complex_args))) + if (!is_cluster_function && (named_collection = tryGetNamedCollectionWithOverrides(args, context, false, &complex_args))) { validateNamedCollection>( *named_collection, diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 1994787f831..f082b192ee0 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -32,7 +32,7 @@ namespace ErrorCodes void TableFunctionS3::parseArgumentsImpl( const String & error_message, ASTs & args, ContextPtr context, StorageS3::Configuration & s3_configuration, bool get_format_from_file) { - if (auto named_collection = tryGetNamedCollectionWithOverrides(args)) + if (auto named_collection = tryGetNamedCollectionWithOverrides(args, context)) { StorageS3::processNamedCollectionResult(s3_configuration, *named_collection); } diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index 5de6c6b4ccc..468f949203d 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -36,7 +36,7 @@ void TableFunctionURL::parseArguments(const ASTPtr & ast, ContextPtr context) auto & url_function_args = assert_cast(args[0].get())->children; - if (auto named_collection = tryGetNamedCollectionWithOverrides(url_function_args)) + if (auto named_collection = tryGetNamedCollectionWithOverrides(url_function_args, context)) { StorageURL::processNamedCollectionResult(configuration, *named_collection); diff --git a/tests/integration/test_mask_sensitive_info/test.py b/tests/integration/test_mask_sensitive_info/test.py index 5df40c38041..92232f7e6a8 100644 --- a/tests/integration/test_mask_sensitive_info/test.py +++ b/tests/integration/test_mask_sensitive_info/test.py @@ -234,8 +234,8 @@ def test_table_functions(): f"mysql(named_collection_1, host = 'mysql57', port = 3306, database = 'mysql_db', table = 'mysql_table', user = 'mysql_user', password = '{password}')", f"postgresql(named_collection_2, password = '{password}', host = 'postgres1', port = 5432, database = 'postgres_db', table = 'postgres_table', user = 'postgres_user')", f"s3(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '{password}')", - f"remote(named_collection_4, addresses_expr = '127.{{2..11}}', database = 'default', table = 'remote_table', user = 'remote_user', password = '{password}', sharding_key = rand())", - f"remoteSecure(named_collection_5, addresses_expr = '127.{{2..11}}', database = 'default', table = 'remote_table', user = 'remote_user', password = '{password}')", + f"remote(named_collection_6, addresses_expr = '127.{{2..11}}', database = 'default', table = 'remote_table', user = 'remote_user', password = '{password}', sharding_key = rand())", + f"remoteSecure(named_collection_6, addresses_expr = '127.{{2..11}}', database = 'default', table = 'remote_table', user = 'remote_user', password = '{password}')", ] for i, table_function in enumerate(table_functions): @@ -287,8 +287,8 @@ def test_table_functions(): "CREATE TABLE tablefunc25 (`x` int) AS mysql(named_collection_1, host = 'mysql57', port = 3306, database = 'mysql_db', table = 'mysql_table', user = 'mysql_user', password = '[HIDDEN]')", "CREATE TABLE tablefunc26 (`x` int) AS postgresql(named_collection_2, password = '[HIDDEN]', host = 'postgres1', port = 5432, database = 'postgres_db', table = 'postgres_table', user = 'postgres_user')", "CREATE TABLE tablefunc27 (`x` int) AS s3(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]')", - "CREATE TABLE tablefunc28 (`x` int) AS remote(named_collection_4, addresses_expr = '127.{2..11}', database = 'default', table = 'remote_table', user = 'remote_user', password = '[HIDDEN]', sharding_key = rand())", - "CREATE TABLE tablefunc29 (`x` int) AS remoteSecure(named_collection_5, addresses_expr = '127.{2..11}', database = 'default', table = 'remote_table', user = 'remote_user', password = '[HIDDEN]')", + "CREATE TABLE tablefunc28 (`x` int) AS remote(named_collection_6, addresses_expr = '127.{2..11}', database = 'default', table = 'remote_table', user = 'remote_user', password = '[HIDDEN]', sharding_key = rand())", + "CREATE TABLE tablefunc29 (`x` int) AS remoteSecure(named_collection_6, addresses_expr = '127.{2..11}', database = 'default', table = 'remote_table', user = 'remote_user', password = '[HIDDEN]')", ], must_not_contain=[password], ) diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index a3ebbe97451..3b7aae1ccdc 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -382,7 +382,7 @@ def test_postgres_distributed(started_cluster): """ CREATE TABLE test_shards2 (id UInt32, name String, age UInt32, money UInt32) - ENGINE = ExternalDistributed('PostgreSQL', postgres4, description='postgres{1|2}:5432,postgres{3|4}:5432'); """ + ENGINE = ExternalDistributed('PostgreSQL', postgres4, addresses_expr='postgres{1|2}:5432,postgres{3|4}:5432'); """ ) result = node2.query("SELECT DISTINCT(name) FROM test_shards2 ORDER BY name") From d9e75e5c0a39ff7d9eac2f7e8f10b09c1720bb1b Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 5 Mar 2023 13:13:32 +0100 Subject: [PATCH 106/559] Fix test --- tests/integration/test_named_collections/test.py | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index 5574c77b886..af5aab38264 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -410,6 +410,16 @@ def test_config_reload(cluster): ).strip() ) + replace_in_server_config(node, "value2", "value1") + node.query("SYSTEM RELOAD CONFIG") + + assert ( + "value1" + == node.query( + "select collection['key1'] from system.named_collections where name = 'collection1'" + ).strip() + ) + def test_sql_commands(cluster): node = cluster.instances["node"] From c06af1f1e774e9e7b8f792e3df3abb5605392b39 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 5 Mar 2023 22:12:51 +0100 Subject: [PATCH 107/559] Fix clang-tidy --- src/Databases/DatabaseFactory.cpp | 4 +-- src/Storages/Kafka/StorageKafka.cpp | 39 ++++++++++++-------------- src/Storages/MySQL/MySQLSettings.cpp | 12 ++++++-- src/Storages/NamedCollectionsHelpers.h | 3 +- 4 files changed, 31 insertions(+), 27 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 47cf0c6b6d0..89a799349bf 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -221,9 +221,9 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String { if (engine_name == "MySQL") { - mysql_settings->loadFromQuery(*engine_define); + mysql_settings->loadFromQueryContext(context, *engine_define); if (engine_define->settings) - mysql_settings->loadFromQueryContext(context, *engine_define); /// Will override only if not changed. + mysql_settings->loadFromQuery(*engine_define); auto mysql_pool = createMySQLPoolWithFailover(configuration, *mysql_settings); diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index dedad6f44d7..6de87e18855 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -914,27 +914,24 @@ void registerStorageKafka(StorageFactory & factory) * - Do intermediate commits when the batch consumed and handled */ - if (has_settings) - { - /* 0 = raw, 1 = evaluateConstantExpressionAsLiteral, 2=evaluateConstantExpressionOrIdentifierAsLiteral */ - CHECK_KAFKA_STORAGE_ARGUMENT(1, kafka_broker_list, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(2, kafka_topic_list, 1) - CHECK_KAFKA_STORAGE_ARGUMENT(3, kafka_group_name, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(4, kafka_format, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(5, kafka_row_delimiter, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(6, kafka_schema, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(7, kafka_num_consumers, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(8, kafka_max_block_size, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(9, kafka_skip_broken_messages, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(10, kafka_commit_every_batch, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(11, kafka_client_id, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(12, kafka_poll_timeout_ms, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(13, kafka_flush_interval_ms, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(14, kafka_thread_per_consumer, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(15, kafka_handle_error_mode, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(16, kafka_commit_on_select, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(17, kafka_max_rows_per_message, 0) - } + /* 0 = raw, 1 = evaluateConstantExpressionAsLiteral, 2=evaluateConstantExpressionOrIdentifierAsLiteral */ + CHECK_KAFKA_STORAGE_ARGUMENT(1, kafka_broker_list, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(2, kafka_topic_list, 1) + CHECK_KAFKA_STORAGE_ARGUMENT(3, kafka_group_name, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(4, kafka_format, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(5, kafka_row_delimiter, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(6, kafka_schema, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(7, kafka_num_consumers, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(8, kafka_max_block_size, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(9, kafka_skip_broken_messages, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(10, kafka_commit_every_batch, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(11, kafka_client_id, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(12, kafka_poll_timeout_ms, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(13, kafka_flush_interval_ms, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(14, kafka_thread_per_consumer, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(15, kafka_handle_error_mode, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(16, kafka_commit_on_select, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(17, kafka_max_rows_per_message, 0) #undef CHECK_KAFKA_STORAGE_ARGUMENT diff --git a/src/Storages/MySQL/MySQLSettings.cpp b/src/Storages/MySQL/MySQLSettings.cpp index 67942114182..fd53174f4f6 100644 --- a/src/Storages/MySQL/MySQLSettings.cpp +++ b/src/Storages/MySQL/MySQLSettings.cpp @@ -53,12 +53,18 @@ void MySQLSettings::loadFromQueryContext(ContextPtr context, ASTStorage & storag const Settings & settings = context->getQueryContext()->getSettingsRef(); - /// Setting from SETTING clause have bigger priority. - if (!mysql_datatypes_support_level.changed - && settings.mysql_datatypes_support_level.value != mysql_datatypes_support_level.value) + if (settings.mysql_datatypes_support_level.value != mysql_datatypes_support_level.value) { static constexpr auto setting_name = "mysql_datatypes_support_level"; set(setting_name, settings.mysql_datatypes_support_level.toString()); + + if (!storage_def.settings) + { + auto settings_ast = std::make_shared(); + settings_ast->is_standalone = false; + storage_def.set(storage_def.settings, settings_ast); + } + auto & changes = storage_def.settings->changes; if (changes.end() == std::find_if( changes.begin(), changes.end(), diff --git a/src/Storages/NamedCollectionsHelpers.h b/src/Storages/NamedCollectionsHelpers.h index 0510ed7e298..1473a3fbe48 100644 --- a/src/Storages/NamedCollectionsHelpers.h +++ b/src/Storages/NamedCollectionsHelpers.h @@ -18,7 +18,8 @@ namespace DB /// Helper function to get named collection for table engine. /// Table engines have collection name as first argument of ast and other arguments are key-value overrides. -MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts, ContextPtr context, bool throw_unknown_collection = true, std::vector> * non_convertible = nullptr); +MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides( + ASTs asts, ContextPtr context, bool throw_unknown_collection = true, std::vector> * complex_args = nullptr); /// Helper function to get named collection for dictionary source. /// Dictionaries have collection name as name argument of dict configuration and other arguments are overrides. MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); From 4fc1f131d3805c3aa192d562c411015fa48cfd84 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 6 Mar 2023 12:39:02 +0800 Subject: [PATCH 108/559] change as requested --- src/Functions/parseDateTime.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index c8a558c831a..c10698ad31c 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -556,15 +556,13 @@ namespace Pos end = str_ref.data + str_ref.size; for (const auto & instruction : instructions) { - // std::cout << "instruction:" << instruction.toString() << std::endl; cur = instruction.perform(cur, end, date); - // std::cout << "date:" << date.toString() << std::endl; } // Ensure all input was consumed. if (cur < end) throw Exception( - ErrorCodes::LOGICAL_ERROR, + ErrorCodes::CANNOT_PARSE_TEXT, "Invalid format input {} is malformed at {}", str_ref.toView(), std::string_view(cur, end - cur)); From 89a2329e62d9c9c9c8752858eee6b1e1616035ab Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 6 Mar 2023 17:08:55 +0800 Subject: [PATCH 109/559] change as requested --- src/Functions/formatDateTime.cpp | 17 +- src/Functions/numLiteralChars.h | 44 ++++ src/Functions/parseDateTime.cpp | 412 ++++++++++++++++++------------- 3 files changed, 286 insertions(+), 187 deletions(-) create mode 100644 src/Functions/numLiteralChars.h diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index 43ef47adadb..3e98c5d35c2 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -865,16 +865,14 @@ public: template size_t parseFormat(const String & format, std::vector> & instructions, UInt32 scale, String & out_template) const { + static_assert( + format_syntax == FormatDateTimeTraits::FormatSyntax::MySQL || format_syntax == FormatDateTimeTraits::FormatSyntax::Joda, + "format syntax must be one of MySQL or Joda"); + if constexpr (format_syntax == FormatDateTimeTraits::FormatSyntax::MySQL) return parseMySQLFormat(format, instructions, scale, out_template); - else if constexpr (format_syntax == FormatDateTimeTraits::FormatSyntax::Joda) - return parseJodaFormat(format, instructions, scale, out_template); else - throw Exception( - ErrorCodes::NOT_IMPLEMENTED, - "Unknown datetime format style {} in function {}", - magic_enum::enum_name(format_syntax), - getName()); + return parseJodaFormat(format, instructions, scale, out_template); } template @@ -896,7 +894,7 @@ public: }; const char * pos = format.data(); - const char * const end = pos + format.size(); + const char * const end = format.data() + format.size(); while (true) { @@ -1187,8 +1185,7 @@ public: size_t reserve_size = 0; const char * pos = format.data(); - const char * end = pos + format.size(); - + const char * end = format.data() + format.size(); while (pos < end) { const char * cur_token = pos; diff --git a/src/Functions/numLiteralChars.h b/src/Functions/numLiteralChars.h new file mode 100644 index 00000000000..ba7a0fbf193 --- /dev/null +++ b/src/Functions/numLiteralChars.h @@ -0,0 +1,44 @@ +#pragma once + +#include + +namespace DB +{ + +/// Counts the number of literal characters in Joda format string until the next closing literal +/// sequence single quote. Returns -1 if no literal single quote was found. +/// In Joda format string(https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html) +/// literal content must be quoted with single quote. and two single quote means literal with one single quote. +/// For example: +/// Format string: "'aaaa'", unescaped literal: "aaaa"; +/// Format string: "'aa''aa'", unescaped literal: "aa'aa"; +/// Format string: "'aaa''aa" is not valid because of missing of end single quote. +inline Int64 numLiteralChars(const char * cur, const char * end) +{ + bool found = false; + Int64 count = 0; + while (cur < end) + { + if (*cur == '\'') + { + if (cur + 1 < end && *(cur + 1) == '\'') + { + count += 2; + cur += 2; + } + else + { + found = true; + break; + } + } + else + { + ++count; + ++cur; + } + } + return found ? count : -1; +} + +} diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index c10698ad31c..e6826541df1 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -26,6 +26,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; extern const int CANNOT_PARSE_TEXT; + extern const int NOT_ENOUGH_SPACE; } namespace @@ -522,7 +523,7 @@ namespace arguments[0].type->getName(), getName()); - String time_zone_name = getTimeZone(arguments).second; + String time_zone_name = getTimeZone(arguments).getTimeZone(); return std::make_shared(time_zone_name); } @@ -538,10 +539,8 @@ namespace getName()); String format = getFormat(arguments); - const auto * time_zone = getTimeZone(arguments).first; - - std::vector instructions; - parseFormat(format, instructions); + const auto & time_zone = getTimeZone(arguments); + std::vector instructions = parseFormat(format); auto col_res = ColumnDateTime::create(); col_res->reserve(input_rows_count); @@ -567,7 +566,7 @@ namespace str_ref.toView(), std::string_view(cur, end - cur)); - Int64 time = date.checkAndGetDateTime(*time_zone); + Int64 time = date.checkAndGetDateTime(time_zone); data_res.push_back(static_cast(time)); } @@ -587,48 +586,56 @@ namespace using Func = std::conditional_t< parse_syntax == ParseDateTimeTraits::ParseSyntax::MySQL, - Pos (*)(Pos, Pos, DateTime &), - std::function>; + Pos (*)(Pos, Pos, const String &, DateTime &), + std::function>; Func func{}; std::string func_name; - - std::string literal; + std::string literal; /// Only used when current instruction parses literal + std::string flag; /// Parsed flags in MySQL or Joda format string public: - explicit Instruction(Func && func_, const char * func_name_) : func(std::move(func_)), func_name(func_name_) { } + explicit Instruction(Func && func_, const char * func_name_, const std::string_view & flag_) + : func(std::move(func_)), func_name(func_name_), flag(flag_) + { + } - explicit Instruction(const String & literal_) : literal(literal_) { } - explicit Instruction(String && literal_) : literal(std::move(literal_)) { } + explicit Instruction(const String & literal_) : literal(literal_), flag("LITERAL") { } + explicit Instruction(String && literal_) : literal(std::move(literal_)), flag("LITERAL") { } /// For debug [[maybe_unused]] String toString() const { if (func) - return "func:" + func_name; + return "func:" + func_name + ",flag:" + flag; else - return "literal:" + literal; + return "literal:" + literal + ",flag:" + flag; } Pos perform(Pos cur, Pos end, DateTime & date) const { if (func) - return func(cur, end, date); + return func(cur, end, flag, date); else { - checkSpace(cur, end, literal.size(), "required literal size not matched"); + checkSpace(cur, end, literal.size(), "required literal size not matched", flag); if (std::string_view(cur, literal.size()) != literal) throw Exception( - ErrorCodes::LOGICAL_ERROR, "Expect literal {} but {} provided", literal, std::string_view(cur, literal.size())); + ErrorCodes::LOGICAL_ERROR, + "Unable to parse flag {} from {} because literal {} is expected but {} provided", + flag, + std::string_view(cur, end - cur), + literal, + std::string_view(cur, literal.size())); cur += literal.size(); return cur; } } template - static Pos readNumber2(Pos cur, Pos end, T & res) + static Pos readNumber2(Pos cur, Pos end, [[maybe_unused]] const String & flag, T & res) { if constexpr (need_check_space == NeedCheckSpace::Yes) - checkSpace(cur, end, 2, "readNumber2 requires size >= 2"); + checkSpace(cur, end, 2, "readNumber2 requires size >= 2", flag); res = (*cur - '0'); ++cur; @@ -638,10 +645,10 @@ namespace } template - static Pos readNumber3(Pos cur, Pos end, T & res) + static Pos readNumber3(Pos cur, Pos end, [[maybe_unused]] const String & flag, T & res) { if constexpr (need_check_space == NeedCheckSpace::Yes) - checkSpace(cur, end, 3, "readNumber4 requires size >= 3"); + checkSpace(cur, end, 3, "readNumber4 requires size >= 3", flag); res = (*cur - '0'); ++cur; @@ -653,10 +660,10 @@ namespace } template - static Pos readNumber4(Pos cur, Pos end, T & res) + static Pos readNumber4(Pos cur, Pos end, [[maybe_unused]] const String & flag, T & res) { if constexpr (need_check_space == NeedCheckSpace::Yes) - checkSpace(cur, end, 4, "readNumber4 requires size >= 4"); + checkSpace(cur, end, 4, "readNumber4 requires size >= 4", flag); res = (*cur - '0'); ++cur; @@ -669,101 +676,122 @@ namespace return cur; } - static void checkSpace(Pos cur, Pos end, size_t len, const String & msg) + static void checkSpace(Pos cur, Pos end, size_t len, const String & msg, const String & flag) { if (cur > end || cur + len > end) [[unlikely]] - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to parse because {}", msg); + throw Exception( + ErrorCodes::NOT_ENOUGH_SPACE, + "Unable to parse flag {} from {} because {}", + flag, + std::string_view(cur, end - cur), + msg); } template - static Pos assertChar(Pos cur, Pos end, char ch) + static Pos assertChar(Pos cur, Pos end, char ch, [[maybe_unused]] const String & flag) { if constexpr (need_check_space == NeedCheckSpace::Yes) - checkSpace(cur, end, 1, "assertChar requires size >= 1"); + checkSpace(cur, end, 1, "assertChar requires size >= 1", flag); if (*cur != ch) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expect char {}, but {} provided", String(ch, 1), String(*cur, 1)); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Unable to parse flag {} from {} because char {} is expected but {} provided", + flag, + std::string_view(cur, end - cur), + String(ch, 1), + String(*cur, 1)); ++cur; return cur; } - static Pos mysqlDayOfWeekTextShort(Pos cur, Pos end, DateTime & date) + static Pos mysqlDayOfWeekTextShort(Pos cur, Pos end, const String & flag, DateTime & date) { - checkSpace(cur, end, 3, "Parsing DayOfWeekTextShort requires size >= 3"); + checkSpace(cur, end, 3, "mysqlDayOfWeekTextShort requires size >= 3", flag); String text(cur, 3); boost::to_lower(text); auto it = dayOfWeekMap.find(text); if (it == dayOfWeekMap.end()) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Unknown day of week short text {}", text); + throw Exception( + ErrorCodes::CANNOT_PARSE_TEXT, + "Unable to parse flag {} from {} because of unknown day of week short text {} ", + flag, + std::string_view(cur, end - cur), + text); date.setDayOfWeek(it->second.second); cur += 3; return cur; } - static Pos mysqlMonthOfYearTextShort(Pos cur, Pos end, DateTime & date) + static Pos mysqlMonthOfYearTextShort(Pos cur, Pos end, const String & flag, DateTime & date) { - checkSpace(cur, end, 3, "Parsing MonthOfYearTextShort requires size >= 3"); + checkSpace(cur, end, 3, "mysqlMonthOfYearTextShort requires size >= 3", flag); String text(cur, 3); boost::to_lower(text); auto it = monthMap.find(text); if (it == monthMap.end()) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Unknown month of year short text {}", text); + throw Exception( + ErrorCodes::CANNOT_PARSE_TEXT, + "Unable to parse flag {} from {} because of unknown month of year short text {}", + flag, + std::string_view(cur, end - cur), + text); date.setMonth(it->second.second); cur += 3; return cur; } - static Pos mysqlMonth(Pos cur, Pos end, DateTime & date) + static Pos mysqlMonth(Pos cur, Pos end, const String & flag, DateTime & date) { Int32 month; - cur = readNumber2(cur, end, month); + cur = readNumber2(cur, end, flag, month); date.setMonth(month); return cur; } - static Pos mysqlCentury(Pos cur, Pos end, DateTime & date) + static Pos mysqlCentury(Pos cur, Pos end, const String & flag, DateTime & date) { Int32 century; - cur = readNumber2(cur, end, century); + cur = readNumber2(cur, end, flag, century); date.setCentury(century); return cur; } - static Pos mysqlDayOfMonth(Pos cur, Pos end, DateTime & date) + static Pos mysqlDayOfMonth(Pos cur, Pos end, const String & flag, DateTime & date) { Int32 day_of_month; - cur = readNumber2(cur, end, day_of_month); + cur = readNumber2(cur, end, flag, day_of_month); date.appendDayOfMonth(day_of_month); return cur; } - static Pos mysqlAmericanDate(Pos cur, Pos end, DateTime & date) + static Pos mysqlAmericanDate(Pos cur, Pos end, const String & flag, DateTime & date) { - checkSpace(cur, end, 8, "mysqlAmericanDate requires size >= 8"); + checkSpace(cur, end, 8, "mysqlAmericanDate requires size >= 8", flag); Int32 month; - cur = readNumber2(cur, end, month); - cur = assertChar(cur, end, '/'); + cur = readNumber2(cur, end, flag, month); + cur = assertChar(cur, end, '/', flag); date.setMonth(month); Int32 day; - cur = readNumber2(cur, end, day); - cur = assertChar(cur, end, '/'); + cur = readNumber2(cur, end, flag, day); + cur = assertChar(cur, end, '/', flag); date.appendDayOfMonth(day); Int32 year; - cur = readNumber2(cur, end, year); + cur = readNumber2(cur, end, flag, year); date.setYear(year); return cur; } - static Pos mysqlDayOfMonthSpacePadded(Pos cur, Pos end, DateTime & date) + static Pos mysqlDayOfMonthSpacePadded(Pos cur, Pos end, const String & flag, DateTime & date) { - checkSpace(cur, end, 2, "mysqlDayOfMonthSpacePadded requires size >= 2"); + checkSpace(cur, end, 2, "mysqlDayOfMonthSpacePadded requires size >= 2", flag); Int32 day_of_month = *cur == ' ' ? 0 : (*cur - '0'); ++cur; @@ -775,18 +803,18 @@ namespace return cur; } - static Pos mysqlISO8601Date(Pos cur, Pos end, DateTime & date) + static Pos mysqlISO8601Date(Pos cur, Pos end, const String & flag, DateTime & date) { - checkSpace(cur, end, 10, "mysqlISO8601Date requires size >= 10"); + checkSpace(cur, end, 10, "mysqlISO8601Date requires size >= 10", flag); Int32 year; Int32 month; Int32 day; - cur = readNumber4(cur, end, year); - cur = assertChar(cur, end, '-'); - cur = readNumber2(cur, end, month); - cur = assertChar(cur, end, '-'); - cur = readNumber2(cur, end, day); + cur = readNumber4(cur, end, flag, year); + cur = assertChar(cur, end, '-', flag); + cur = readNumber2(cur, end, flag, month); + cur = assertChar(cur, end, '-', flag); + cur = readNumber2(cur, end, flag, day); date.setYear(year); date.setMonth(month); @@ -794,49 +822,49 @@ namespace return cur; } - static Pos mysqlISO8601Year2(Pos cur, Pos end, DateTime & date) + static Pos mysqlISO8601Year2(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 year2; - cur = readNumber2(cur, end, year2); + cur = readNumber2(cur, end, flag, year2); date.setYear2(year2); return cur; } - static Pos mysqlISO8601Year4(Pos cur, Pos end, DateTime & date) + static Pos mysqlISO8601Year4(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 year; - cur = readNumber4(cur, end, year); + cur = readNumber4(cur, end, flag, year); date.setYear(year); return cur; } - static Pos mysqlDayOfYear(Pos cur, Pos end, DateTime & date) + static Pos mysqlDayOfYear(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 day_of_year; - cur = readNumber3(cur, end, day_of_year); + cur = readNumber3(cur, end, flag, day_of_year); date.appendDayOfYear(day_of_year); return cur; } - static Pos mysqlDayOfWeek(Pos cur, Pos end, DateTime & date) + static Pos mysqlDayOfWeek(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { - checkSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1"); + checkSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1", flag); date.setDayOfWeek(*cur - '0'); ++cur; return cur; } - static Pos mysqlISO8601Week(Pos cur, Pos end, DateTime & date) + static Pos mysqlISO8601Week(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 week; - cur = readNumber2(cur, end, week); + cur = readNumber2(cur, end, flag, week); date.setWeek(week); return cur; } - static Pos mysqlDayOfWeek0To6(Pos cur, Pos end, DateTime & date) + static Pos mysqlDayOfWeek0To6(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { - checkSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1"); + checkSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1", flag); Int32 day_of_week = *cur - '0'; if (day_of_week == 0) @@ -847,47 +875,57 @@ namespace return cur; } - static Pos mysqlDayOfWeekTextLong(Pos cur, Pos end, DateTime & date) + static Pos mysqlDayOfWeekTextLong(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { - checkSpace(cur, end, 6, "jodaDayOfWeekText requires the size >= 6"); + checkSpace(cur, end, 6, "jodaDayOfWeekText requires the size >= 6", flag); String text1(cur, 3); boost::to_lower(text1); auto it = dayOfWeekMap.find(text1); if (it == dayOfWeekMap.end()) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Unknown day of week text: {}", text1); + throw Exception( + ErrorCodes::CANNOT_PARSE_TEXT, + "Unable to parse first part of flag {} from {} because of unknown day of week text: {}", + flag, + std::string_view(cur, end - cur), + text1); cur += 3; size_t left_size = it->second.first.size(); - checkSpace(cur, end, left_size, "jodaDayOfWeekText requires the second parg size >= " + std::to_string(left_size)); + checkSpace(cur, end, left_size, "jodaDayOfWeekText requires the second parg size >= " + std::to_string(left_size), flag); String text2(cur, left_size); boost::to_lower(text2); if (text2 != it->second.first) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Unknown day of week text: {}", text1 + text2); + throw Exception( + ErrorCodes::CANNOT_PARSE_TEXT, + "Unable to parse second part of flag {} from {} because of unknown day of week text: {}", + flag, + std::string_view(cur, end - cur), + text1 + text2); cur += left_size; date.setDayOfWeek(it->second.second); return cur; } - static Pos mysqlYear2(Pos cur, Pos end, DateTime & date) + static Pos mysqlYear2(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 year2; - cur = readNumber2(cur, end, year2); + cur = readNumber2(cur, end, flag, year2); date.setYear2(year2); return cur; } - static Pos mysqlYear4(Pos cur, Pos end, DateTime & date) + static Pos mysqlYear4(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 year; - cur = readNumber4(cur, end, year); + cur = readNumber4(cur, end, flag, year); date.setYear(year); return cur; } - static Pos mysqlTimezoneOffset(Pos cur, Pos end, DateTime & date) + static Pos mysqlTimezoneOffset(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { - checkSpace(cur, end, 5, "mysqlTimezoneOffset requires size >= 5"); + checkSpace(cur, end, 5, "mysqlTimezoneOffset requires size >= 5", flag); Int32 sign; if (*cur == '-') @@ -895,30 +933,35 @@ namespace else if (*cur == '+') sign = 1; else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown sign time zone offset: {}", std::string_view(cur, 1)); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Unable to parse flag {} from {} because of unknown sign time zone offset: {}", + flag, + std::string_view(cur, end - cur), + std::string_view(cur, 1)); ++cur; Int32 hour; - cur = readNumber2(cur, end, hour); + cur = readNumber2(cur, end, flag, hour); Int32 minute; - cur = readNumber2(cur, end, minute); + cur = readNumber2(cur, end, flag, minute); *date.time_zone_offset = sign * (hour * 3600 + minute * 60); return cur; } - static Pos mysqlMinute(Pos cur, Pos end, DateTime & date) + static Pos mysqlMinute(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 minute; - cur = readNumber2(cur, end, minute); + cur = readNumber2(cur, end, flag, minute); date.setMinute(minute); return cur; } - static Pos mysqlAMPM(Pos cur, Pos end, DateTime & date) + static Pos mysqlAMPM(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { - checkSpace(cur, end, 2, "mysqlAMPM requires size >= 2"); + checkSpace(cur, end, 2, "mysqlAMPM requires size >= 2", flag); String text(cur, 2); date.setAMPM(text); @@ -926,59 +969,59 @@ namespace return cur; } - static Pos mysqlHHMM12(Pos cur, Pos end, DateTime & date) + static Pos mysqlHHMM12(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { - checkSpace(cur, end, 8, "mysqlHHMM12 requires size >= 8"); + checkSpace(cur, end, 8, "mysqlHHMM12 requires size >= 8", flag); Int32 hour; - cur = readNumber2(cur, end, hour); - cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, flag, hour); + cur = assertChar(cur, end, ':', flag); date.setHour(hour, true, true); Int32 minute; - cur = readNumber2(cur, end, minute); - cur = assertChar(cur, end, ' '); + cur = readNumber2(cur, end, flag, minute); + cur = assertChar(cur, end, ' ', flag); date.setMinute(minute); - cur = mysqlAMPM(cur, end, date); + cur = mysqlAMPM(cur, end, flag, date); return cur; } - static Pos mysqlHHMM24(Pos cur, Pos end, DateTime & date) + static Pos mysqlHHMM24(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { - checkSpace(cur, end, 5, "mysqlHHMM24 requires size >= 5"); + checkSpace(cur, end, 5, "mysqlHHMM24 requires size >= 5", flag); Int32 hour; - cur = readNumber2(cur, end, hour); - cur = assertChar(cur, end, ':'); + cur = readNumber2(cur, end, flag, hour); + cur = assertChar(cur, end, ':', flag); date.setHour(hour, false, false); Int32 minute; - cur = readNumber2(cur, end, minute); + cur = readNumber2(cur, end, flag, minute); date.setMinute(minute); return cur; } - static Pos mysqlSecond(Pos cur, Pos end, DateTime & date) + static Pos mysqlSecond(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 second; - cur = readNumber2(cur, end, second); + cur = readNumber2(cur, end, flag, second); date.setSecond(second); return cur; } - static Pos mysqlISO8601Time(Pos cur, Pos end, DateTime & date) + static Pos mysqlISO8601Time(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { - checkSpace(cur, end, 8, "mysqlISO8601Time requires size >= 8"); + checkSpace(cur, end, 8, "mysqlISO8601Time requires size >= 8", flag); Int32 hour; Int32 minute; Int32 second; - cur = readNumber2(cur, end, hour); - cur = assertChar(cur, end, ':'); - cur = readNumber2(cur, end, minute); - cur = assertChar(cur, end, ':'); - cur = readNumber2(cur, end, second); + cur = readNumber2(cur, end, flag, hour); + cur = assertChar(cur, end, ':', flag); + cur = readNumber2(cur, end, flag, minute); + cur = assertChar(cur, end, ':', flag); + cur = readNumber2(cur, end, flag, second); date.setHour(hour, false, false); date.setMinute(minute); @@ -986,18 +1029,18 @@ namespace return cur; } - static Pos mysqlHour12(Pos cur, Pos end, DateTime & date) + static Pos mysqlHour12(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 hour; - cur = readNumber2(cur, end, hour); + cur = readNumber2(cur, end, flag, hour); date.setHour(hour, true, true); return cur; } - static Pos mysqlHour24(Pos cur, Pos end, DateTime & date) + static Pos mysqlHour24(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 hour; - cur = readNumber2(cur, end, hour); + cur = readNumber2(cur, end, flag, hour); date.setHour(hour, false, false); return cur; } @@ -1010,6 +1053,7 @@ namespace bool is_year, int repetitions, int max_digits_consume, + const String & flag, Int32 & number) { bool negative = false; @@ -1068,7 +1112,11 @@ namespace /// Need to have read at least one digit. if (cur <= start) - throw Exception(ErrorCodes::LOGICAL_ERROR, "read number from {} failed", String(cur, end - cur)); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Unable to parse flag {} from {} because read number failed", + flag, + std::string_view(cur, end - cur)); if (negative) number *= -1; @@ -1076,9 +1124,9 @@ namespace return cur; } - static Pos jodaEra(int, Pos cur, Pos end, DateTime & date) + static Pos jodaEra(int, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { - checkSpace(cur, end, 2, "jodaEra requires size >= 2"); + checkSpace(cur, end, 2, "jodaEra requires size >= 2", flag); String era(cur, 2); date.setEra(era); @@ -1086,58 +1134,61 @@ namespace return cur; } - static Pos jodaCenturyOfEra(int repetitions, Pos cur, Pos end, DateTime & date) + static Pos jodaCenturyOfEra(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 century; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, repetitions, century); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, repetitions, flag, century); date.setCentury(century); return cur; } - static Pos jodaYearOfEra(int repetitions, Pos cur, Pos end, DateTime & date) + static Pos jodaYearOfEra(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 year_of_era; - cur = readNumberWithVariableLength(cur, end, false, false, true, repetitions, repetitions, year_of_era); + cur = readNumberWithVariableLength(cur, end, false, false, true, repetitions, repetitions, flag, year_of_era); date.setYear(year_of_era, true); return cur; } - static Pos jodaWeekYear(int repetitions, Pos cur, Pos end, DateTime & date) + static Pos jodaWeekYear(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 week_year; - cur = readNumberWithVariableLength(cur, end, true, true, true, repetitions, repetitions, week_year); + cur = readNumberWithVariableLength(cur, end, true, true, true, repetitions, repetitions, flag, week_year); date.setYear(week_year, false, true); return cur; } - static Pos jodaWeekOfWeekYear(int repetitions, Pos cur, Pos end, DateTime & date) + static Pos jodaWeekOfWeekYear(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 week; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), week); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, week); date.setWeek(week); return cur; } - static Pos jodaDayOfWeek1Based(int repetitions, Pos cur, Pos end, DateTime & date) + static Pos jodaDayOfWeek1Based(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 day_of_week; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, repetitions, day_of_week); - if (day_of_week < 1 || day_of_week > 7) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for day of week 1-based must be in the range [1, 7]", day_of_week); - + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, repetitions, flag, day_of_week); date.setDayOfWeek(day_of_week); return cur; } - static Pos jodaDayOfWeekText(size_t /*min_represent_digits*/, Pos cur, Pos end, DateTime & date) + static Pos + jodaDayOfWeekText(size_t /*min_represent_digits*/, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { - checkSpace(cur, end, 3, "jodaDayOfWeekText requires size >= 3"); + checkSpace(cur, end, 3, "jodaDayOfWeekText requires size >= 3", flag); String text1(cur, 3); boost::to_lower(text1); auto it = dayOfWeekMap.find(text1); if (it == dayOfWeekMap.end()) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Unknown day of week text: {}", text1); + throw Exception( + ErrorCodes::CANNOT_PARSE_TEXT, + "Unable to parse flag {} from {} because of unknown day of week text: {}", + flag, + std::string_view(cur, end - cur), + text1); cur += 3; date.setDayOfWeek(it->second.second); @@ -1155,38 +1206,43 @@ namespace return cur; } - static Pos jodaYear(int repetitions, Pos cur, Pos end, DateTime & date) + static Pos jodaYear(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 year; - cur = readNumberWithVariableLength(cur, end, true, true, true, repetitions, repetitions, year); + cur = readNumberWithVariableLength(cur, end, true, true, true, repetitions, repetitions, flag, year); date.setYear(year); return cur; } - static Pos jodaDayOfYear(int repetitions, Pos cur, Pos end, DateTime & date) + static Pos jodaDayOfYear(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 day_of_year; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 3), day_of_year); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 3), flag, day_of_year); date.appendDayOfYear(day_of_year); return cur; } - static Pos jodaMonthOfYear(int repetitions, Pos cur, Pos end, DateTime & date) + static Pos jodaMonthOfYear(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 month; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, 2, month); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, 2, flag, month); date.setMonth(month); return cur; } - static Pos jodaMonthOfYearText(int, Pos cur, Pos end, DateTime & date) + static Pos jodaMonthOfYearText(int, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { - checkSpace(cur, end, 3, "jodaMonthOfYearText requires size >= 3"); + checkSpace(cur, end, 3, "jodaMonthOfYearText requires size >= 3", flag); String text1(cur, 3); boost::to_lower(text1); auto it = monthMap.find(text1); if (it == monthMap.end()) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Unknown month of year text: {}", text1); + throw Exception( + ErrorCodes::CANNOT_PARSE_TEXT, + "Unable to parse flag {} from {} because of unknown month of year text: {}", + flag, + std::string_view(cur, end - cur), + text1); cur += 3; date.setMonth(it->second.second); @@ -1204,17 +1260,18 @@ namespace return cur; } - static Pos jodaDayOfMonth(int repetitions, Pos cur, Pos end, DateTime & date) + static Pos jodaDayOfMonth(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 day_of_month; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), day_of_month); + cur = readNumberWithVariableLength( + cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, day_of_month); date.appendDayOfMonth(day_of_month); return cur; } - static Pos jodaHalfDayOfDay(int, Pos cur, Pos end, DateTime & date) + static Pos jodaHalfDayOfDay(int, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { - checkSpace(cur, end, 2, "jodaHalfDayOfDay requires size >= 2"); + checkSpace(cur, end, 2, "jodaHalfDayOfDay requires size >= 2", flag); String text(cur, 2); date.setAMPM(text); @@ -1222,79 +1279,79 @@ namespace return cur; } - static Pos jodaHourOfHalfDay(int repetitions, Pos cur, Pos end, DateTime & date) + static Pos jodaHourOfHalfDay(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 hour; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), hour); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, hour); date.setHour(hour, true, false); return cur; } - static Pos jodaClockHourOfHalfDay(int repetitions, Pos cur, Pos end, DateTime & date) + static Pos jodaClockHourOfHalfDay(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 hour; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), hour); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, hour); date.setHour(hour, true, true); return cur; } - static Pos jodaHourOfDay(int repetitions, Pos cur, Pos end, DateTime & date) + static Pos jodaHourOfDay(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 hour; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), hour); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, hour); date.setHour(hour, false, false); return cur; } - static Pos jodaClockHourOfDay(int repetitions, Pos cur, Pos end, DateTime & date) + static Pos jodaClockHourOfDay(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 hour; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), hour); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, hour); date.setHour(hour, false, true); return cur; } - static Pos jodaMinuteOfHour(int repetitions, Pos cur, Pos end, DateTime & date) + static Pos jodaMinuteOfHour(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 minute; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), minute); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, minute); date.setMinute(minute); return cur; } - static Pos jodaSecondOfMinute(int repetitions, Pos cur, Pos end, DateTime & date) + static Pos jodaSecondOfMinute(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) { Int32 second; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), second); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, second); date.setSecond(second); return cur; } }; - - ALWAYS_INLINE void parseFormat(const String & format, std::vector & instructions) const + std::vector parseFormat(const String & format) const { + static_assert( + parse_syntax == ParseDateTimeTraits::ParseSyntax::MySQL || parse_syntax == ParseDateTimeTraits::ParseSyntax::Joda, + "parse syntax must be one of MySQL or Joda"); + if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::MySQL) - parseMysqlFormat(format, instructions); - else if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::Joda) - parseJodaFormat(format, instructions); + return parseMysqlFormat(format); else - throw Exception( - ErrorCodes::NOT_IMPLEMENTED, - "Unknown datetime format style {} in function {}", - magic_enum::enum_name(parse_syntax), - getName()); + return parseJodaFormat(format); } - ALWAYS_INLINE void parseMysqlFormat(const String & format, std::vector & instructions) const + std::vector parseMysqlFormat(const String & format) const { -#define ACTION_ARGS(func) &(func), #func +#define ACTION_ARGS(func) &(func), #func, std::string_view(pos - 1, 2) Pos pos = format.data(); - Pos end = pos + format.size(); + Pos end = format.data() + format.size(); + + std::vector instructions; while (true) { Pos next_percent_pos = find_first_symbols<'%'>(pos, end); + if (next_percent_pos < end) { if (pos < next_percent_pos) @@ -1518,17 +1575,18 @@ namespace break; } } + return instructions; #undef ACTION_ARGS } - void parseJodaFormat(const String & format, std::vector & instructions) const + std::vector parseJodaFormat(const String & format) const { -#define ACTION_ARGS_WITH_BIND(func, arg) std::bind_front(&(func), (arg)), #func +#define ACTION_ARGS_WITH_BIND(func, arg) std::bind_front(&(func), (arg)), #func, std::string_view(cur_token, repetitions) - // size_t reserve_size = 0; const char * pos = format.data(); - const char * end = pos + format.size(); + const char * end = format.data() + format.size(); + std::vector instructions; while (pos < end) { const char * cur_token = pos; @@ -1553,7 +1611,6 @@ namespace for (Int64 i = 1; i <= count; i++) { instructions.emplace_back(String(cur_token + i, 1)); - // ++reserve_size; if (*(cur_token + i) == '\'') i += 1; } @@ -1647,6 +1704,7 @@ namespace } } } + return instructions; #undef ACTION_ARGS_WITH_BIND } @@ -1671,10 +1729,10 @@ namespace return format_column->getValue(); } - std::pair getTimeZone(const ColumnsWithTypeAndName & arguments) const + const DateLUTImpl & getTimeZone(const ColumnsWithTypeAndName & arguments) const { if (arguments.size() < 3) - return {&DateLUT::instance(), ""}; + return DateLUT::instance(); const auto * col = checkAndGetColumnConst(arguments[2].column.get()); if (!col) @@ -1687,7 +1745,7 @@ namespace String time_zone = col->getValue(); if (time_zone.empty()) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Provided time zone must be non-empty and be a valid time zone"); - return {&DateLUT::instance(time_zone), time_zone}; + return DateLUT::instance(time_zone); } }; From 3979ad13489f3ebcd01e060d6f3bc2aa1e21b76e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 6 Mar 2023 17:35:08 +0800 Subject: [PATCH 110/559] remove usedless vectors --- src/Functions/parseDateTime.cpp | 33 ++++++--------------------------- 1 file changed, 6 insertions(+), 27 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index e6826541df1..206b7e985be 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -89,14 +89,12 @@ namespace Int32 year = 1970; Int32 month = 1; Int32 day = 1; - std::vector day_of_month_values; Int32 week = 1; // Week of year based on ISO week date, e.g: 27 Int32 day_of_week = 1; // Day of week, Monday:1, Tuesday:2, ..., Sunday:7 bool week_date_format = false; Int32 day_of_year = 1; - std::vector day_of_year_values; bool day_of_year_format = false; bool century_format = false; @@ -119,14 +117,12 @@ namespace year = 1970; month = 1; day = 1; - day_of_month_values.clear(); week = 1; day_of_week = 1; week_date_format = false; day_of_year = 1; - day_of_year_values.clear(); day_of_year_format = false; century_format = false; @@ -185,12 +181,11 @@ namespace } } - ALWAYS_INLINE void appendDayOfMonth(Int32 day_of_month) + void setDayOfMonth(Int32 day_of_month) { if (day_of_month < 1 || day_of_month > 31) throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for day of month must be in the range [1, 31]", day_of_month); - day_of_month_values.push_back(day_of_month); day = day_of_month; week_date_format = false; day_of_year_format = false; @@ -206,7 +201,6 @@ namespace if (day_of_year_ < 1 || day_of_year_ > 366) throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for day of year must be in the range [1, 366]", day_of_year_); - day_of_year_values.push_back(day_of_year_); day_of_year = day_of_year_; day_of_year_format = true; week_date_format = false; @@ -426,20 +420,6 @@ namespace if (is_hour_of_half_day && !is_am) hour += 12; - /// Ensure all day of year values are valid for ending year value - for (const auto d : day_of_month_values) - { - if (!isDateValid(year, month, d)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of month, year:{} month:{} day:{}", year, month, d); - } - - // Ensure all day of year values are valid for ending year value - for (const auto d : day_of_year_values) - { - if (!isDayOfYearValid(year, d)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of year, year:{} day of year:{}", year, d); - } - // Convert the parsed date/time into a timestamp. Int32 days_since_epoch; if (week_date_format) @@ -454,7 +434,6 @@ namespace /// Time zone is not specified, use local time zone if (!time_zone_offset) *time_zone_offset = time_zone.timezoneOffset(seconds_since_epoch); - // std::cout << "time_zone:" << time_zone.getTimeZone() << ",offset:" << *time_zone_offset << std::endl; /// Time zone is specified in format string. if (seconds_since_epoch >= *time_zone_offset) @@ -765,7 +744,7 @@ namespace { Int32 day_of_month; cur = readNumber2(cur, end, flag, day_of_month); - date.appendDayOfMonth(day_of_month); + date.setDayOfMonth(day_of_month); return cur; } @@ -781,7 +760,7 @@ namespace Int32 day; cur = readNumber2(cur, end, flag, day); cur = assertChar(cur, end, '/', flag); - date.appendDayOfMonth(day); + date.setDayOfMonth(day); Int32 year; cur = readNumber2(cur, end, flag, year); @@ -799,7 +778,7 @@ namespace day_of_month = 10 * day_of_month + (*cur - '0'); ++cur; - date.appendDayOfMonth(day_of_month); + date.setDayOfMonth(day_of_month); return cur; } @@ -818,7 +797,7 @@ namespace date.setYear(year); date.setMonth(month); - date.appendDayOfMonth(day); + date.setDayOfMonth(day); return cur; } @@ -1265,7 +1244,7 @@ namespace Int32 day_of_month; cur = readNumberWithVariableLength( cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, day_of_month); - date.appendDayOfMonth(day_of_month); + date.setDayOfMonth(day_of_month); return cur; } From 8567e3976b28dc724773f8ddcd8ba2c80fca6ed7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 6 Mar 2023 12:30:25 +0100 Subject: [PATCH 111/559] Fix kafka --- src/Storages/Kafka/StorageKafka.cpp | 38 ++++++++++++++++------------- 1 file changed, 21 insertions(+), 17 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 6de87e18855..61495c966cb 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -915,23 +915,27 @@ void registerStorageKafka(StorageFactory & factory) */ /* 0 = raw, 1 = evaluateConstantExpressionAsLiteral, 2=evaluateConstantExpressionOrIdentifierAsLiteral */ - CHECK_KAFKA_STORAGE_ARGUMENT(1, kafka_broker_list, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(2, kafka_topic_list, 1) - CHECK_KAFKA_STORAGE_ARGUMENT(3, kafka_group_name, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(4, kafka_format, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(5, kafka_row_delimiter, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(6, kafka_schema, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(7, kafka_num_consumers, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(8, kafka_max_block_size, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(9, kafka_skip_broken_messages, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(10, kafka_commit_every_batch, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(11, kafka_client_id, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(12, kafka_poll_timeout_ms, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(13, kafka_flush_interval_ms, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(14, kafka_thread_per_consumer, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(15, kafka_handle_error_mode, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(16, kafka_commit_on_select, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(17, kafka_max_rows_per_message, 0) + /// In case of named collection we already validated the arguments. + if (collection_name.empty()) + { + CHECK_KAFKA_STORAGE_ARGUMENT(1, kafka_broker_list, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(2, kafka_topic_list, 1) + CHECK_KAFKA_STORAGE_ARGUMENT(3, kafka_group_name, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(4, kafka_format, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(5, kafka_row_delimiter, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(6, kafka_schema, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(7, kafka_num_consumers, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(8, kafka_max_block_size, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(9, kafka_skip_broken_messages, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(10, kafka_commit_every_batch, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(11, kafka_client_id, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(12, kafka_poll_timeout_ms, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(13, kafka_flush_interval_ms, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(14, kafka_thread_per_consumer, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(15, kafka_handle_error_mode, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(16, kafka_commit_on_select, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(17, kafka_max_rows_per_message, 0) + } #undef CHECK_KAFKA_STORAGE_ARGUMENT From 420108a7a05f3cbc5d4230b2fcf1dad0168a9070 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 6 Mar 2023 19:10:36 +0100 Subject: [PATCH 112/559] support alternatives --- src/Common/OptimizedRegularExpression.cpp | 197 ++++++++++++---------- src/Common/OptimizedRegularExpression.h | 8 +- src/Common/tests/gtest_optimize_re.cpp | 26 ++- 3 files changed, 131 insertions(+), 100 deletions(-) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index ff87cd7ef86..b1fc9a2174c 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -1,3 +1,5 @@ +#include + #include #include #include @@ -16,11 +18,13 @@ namespace DB template -void OptimizedRegularExpressionImpl::analyze( +const char * OptimizedRegularExpressionImpl::analyze( std::string_view regexp, + const char * pos, std::string & required_substring, bool & is_trivial, - bool & required_substring_is_prefix) + bool & required_substring_is_prefix, + std::vector & alternatives) { /** The expression is trivial if all the metacharacters in it are escaped. * The non-alternative string is @@ -30,9 +34,9 @@ void OptimizedRegularExpressionImpl::analyze( * and also avoid substrings of the form `http://` or `www` and some other * (this is the hack for typical use case in web analytics applications). */ - const char * begin = regexp.data(); - const char * pos = begin; + const char * begin = pos; const char * end = regexp.data() + regexp.size(); + bool first_call = begin == regexp.data(); int depth = 0; is_trivial = true; required_substring_is_prefix = false; @@ -47,23 +51,46 @@ void OptimizedRegularExpressionImpl::analyze( Substrings trivial_substrings(1); Substring * last_substring = &trivial_substrings.back(); - std::string bracket_string; - bool appending_bracket_string = false; - - auto finish_last_substring = [&]() + auto finish_non_trivial_char = [&]() { if (depth != 0) return; - /// combine last substr and bracket string - last_substring->first += bracket_string; - bracket_string = ""; - /// if we can still append, no need to finish it. e.g. abc(de)fg should capture abcdefg - if (!last_substring->first.empty() && !appending_bracket_string) + + if (!last_substring->first.empty()) { trivial_substrings.resize(trivial_substrings.size() + 1); last_substring = &trivial_substrings.back(); } - appending_bracket_string = false; + }; + + + auto finish_group = [&](std::string group_required_string, bool group_is_trivial, bool group_is_prefix, std::vector & group_alternatives) + { + if (alternatives.empty() && !group_alternatives.empty()) + { + /// Check if group alternatives has empty strings + bool has_empty_str = false; + for (const std::string & alter : group_alternatives) + has_empty_str |= alter.empty(); + if (!has_empty_str) + alternatives = std::move(group_alternatives); + } + + if (group_is_prefix) + last_substring->first += group_required_string; + else + { + finish_non_trivial_char(); + last_substring->first = group_required_string; + } + /// if we can still append, no need to finish it. e.g. abc(de)fg should capture abcdefg + if (!last_substring->first.empty() && !group_is_trivial) + { + trivial_substrings.resize(trivial_substrings.size() + 1); + last_substring = &trivial_substrings.back(); + } + if (!group_is_trivial) + is_trivial = false; }; bool in_curly_braces = false; @@ -92,31 +119,19 @@ void OptimizedRegularExpressionImpl::analyze( case '$': case '.': case '[': + case ']': case '?': case '*': case '+': + case '-': case '{': - if (depth == 0 && !in_curly_braces && !in_square_braces) - { - if (last_substring->first.empty()) - last_substring->second = pos - begin; - last_substring->first.push_back(*pos); - } - else if (depth == 1 && appending_bracket_string) - { - bracket_string += *pos; - } - break; + case '}': + case '/': + goto ordinary; default: /// all other escape sequences are not supported is_trivial = false; - appending_bracket_string = false; - //if (!last_substring->first.empty()) - //{ - // trivial_substrings.resize(trivial_substrings.size() + 1); - // last_substring = &trivial_substrings.back(); - //} - finish_last_substring(); + finish_non_trivial_char(); break; } @@ -125,32 +140,18 @@ void OptimizedRegularExpressionImpl::analyze( } case '|': - if (depth == 0) - has_alternative_on_depth_0 = true; - if (depth == 1) - { - appending_bracket_string = false; - bracket_string = ""; - } is_trivial = false; - if (!in_square_braces && !last_substring->first.empty() && depth == 0) - { - trivial_substrings.resize(trivial_substrings.size() + 1); - last_substring = &trivial_substrings.back(); - } ++pos; + if (depth == 0) + { + has_alternative_on_depth_0 = true; + goto finish; + } break; case '(': if (!in_square_braces) { - ++depth; - is_trivial = false; - /// we dont change the value of appending_bracket_string when depth > 1 - /// e.g. (de(fg)) should capture defg - if (depth == 1) - appending_bracket_string = true; - /// Check for case-insensitive flag. if (pos + 1 < end && pos[1] == '?') { @@ -176,6 +177,23 @@ void OptimizedRegularExpressionImpl::analyze( { pos += 2; } + std::string group_required_substr; + bool group_is_trival; + bool group_is_prefix; + std::vector group_alters; + pos = analyze(regexp, pos + 1, group_required_substr, group_is_trival, group_is_prefix, group_alters); + /// pos should be ')', if not, then it is not a valid regular expression + if (pos == end) + return pos; + + /// For ()? ()* (){0,1}, we can just ignore the whole group. + if ((pos + 1 < end && (pos[1] == '?' || pos[1] == '*')) || + (pos + 2 < end && pos[1] == '{' && pos[2] == '0')) + { + finish_non_trivial_char(); + } + else + finish_group(group_required_substr, group_is_trival, group_is_prefix, group_alters); } ++pos; break; @@ -184,8 +202,7 @@ void OptimizedRegularExpressionImpl::analyze( in_square_braces = true; ++depth; is_trivial = false; - appending_bracket_string = false; - finish_last_substring(); + finish_non_trivial_char(); ++pos; break; @@ -193,38 +210,25 @@ void OptimizedRegularExpressionImpl::analyze( if (!in_square_braces) goto ordinary; - in_square_braces = false; --depth; + if (depth == 0) + in_square_braces = false; is_trivial = false; - finish_last_substring(); - //if (!last_substring->first.empty()) - //{ - // trivial_substrings.resize(trivial_substrings.size() + 1); - // last_substring = &trivial_substrings.back(); - //} + finish_non_trivial_char(); ++pos; break; case ')': if (!in_square_braces) { - --depth; - is_trivial = false; - if (pos + 1 < end && (pos[1] == '?' || pos[1] == '*')) - { - /// TODO: (abc(def)?) should remain the abc part. - bracket_string = ""; - appending_bracket_string = false; - } - finish_last_substring(); + goto finish; } ++pos; break; case '^': case '$': case '.': case '+': is_trivial = false; - appending_bracket_string = false; - finish_last_substring(); + finish_non_trivial_char(); ++pos; break; @@ -240,16 +244,7 @@ void OptimizedRegularExpressionImpl::analyze( { last_substring->first.resize(last_substring->first.size() - 1); } - if (depth >= 1 && appending_bracket_string) - { - /// ab(*cd) should be ab - appending_bracket_string = false; - if (!bracket_string.empty()) - { - bracket_string.resize(bracket_string.size() - 1); - } - } - finish_last_substring(); + finish_non_trivial_char(); ++pos; break; @@ -270,22 +265,19 @@ void OptimizedRegularExpressionImpl::analyze( last_substring->second = pos - begin; last_substring->first.push_back(*pos); } - else if (depth >= 1 && appending_bracket_string) - bracket_string += *pos; ++pos; break; } } - - appending_bracket_string = false; - finish_last_substring(); - +finish: if (last_substring && last_substring->first.empty()) trivial_substrings.pop_back(); if (!is_trivial) { - if (!has_alternative_on_depth_0 && !has_case_insensitive_flag) + /// we calculate required substring even though has_alternative_on_depth_0. + /// we will clear the required substring after putting it to alternatives. + if (!has_case_insensitive_flag) { /// We choose the non-alternative substring of the maximum length for first search. @@ -305,7 +297,7 @@ void OptimizedRegularExpressionImpl::analyze( } } - if (max_length >= MIN_LENGTH_FOR_STRSTR) + if (max_length >= MIN_LENGTH_FOR_STRSTR || !first_call) { required_substring = candidate_it->first; required_substring_is_prefix = candidate_it->second == 0; @@ -318,6 +310,30 @@ void OptimizedRegularExpressionImpl::analyze( required_substring_is_prefix = trivial_substrings.front().second == 0; } + /// if it is xxx|xxx|xxx, we should call the next xxx|xxx recursively and collect the result. + if (has_alternative_on_depth_0) + { + if (alternatives.empty()) + alternatives.push_back(required_substring); + std::vector next_alternatives; + /// this two vals are useless, xxx|xxx cannot be trivial nor prefix. + bool next_is_trivial; + bool next_is_prefix; + pos = analyze(regexp, pos, required_substring, next_is_trivial, next_is_prefix, next_alternatives); + /// For xxx|xxx|xxx, we only conbine the alternatives and return a empty required_substring. + if (next_alternatives.empty()) + { + alternatives.push_back(required_substring); + } + else + { + alternatives.insert(alternatives.end(), next_alternatives.begin(), next_alternatives.end()); + } + required_substring.clear(); + } + + return pos; + /* std::cerr << "regexp: " << regexp << ", is_trivial: " << is_trivial @@ -330,7 +346,8 @@ void OptimizedRegularExpressionImpl::analyze( template OptimizedRegularExpressionImpl::OptimizedRegularExpressionImpl(const std::string & regexp_, int options) { - analyze(regexp_, required_substring, is_trivial, required_substring_is_prefix); + std::vector alternatives; /// this vector collects patterns in (xx|xx|xx). for now it's not used. + analyze(regexp_, regexp_.data(), required_substring, is_trivial, required_substring_is_prefix, alternatives); /// Just three following options are supported if (options & (~(RE_CASELESS | RE_NO_CAPTURE | RE_DOT_NL))) diff --git a/src/Common/OptimizedRegularExpression.h b/src/Common/OptimizedRegularExpression.h index d8b54520bf3..a19ce22deab 100644 --- a/src/Common/OptimizedRegularExpression.h +++ b/src/Common/OptimizedRegularExpression.h @@ -95,7 +95,13 @@ public: out_required_substring_is_prefix = required_substring_is_prefix; } - static void analyze(std::string_view regexp_, std::string & required_substring, bool & is_trivial, bool & required_substring_is_prefix); + static const char * analyze( + std::string_view regexp_, + const char * pos, + std::string & required_substring, + bool & is_trivial, + bool & required_substring_is_prefix, + std::vector & alternatives); private: bool is_trivial; bool required_substring_is_prefix; diff --git a/src/Common/tests/gtest_optimize_re.cpp b/src/Common/tests/gtest_optimize_re.cpp index e68f699ee80..ebd6a28900b 100644 --- a/src/Common/tests/gtest_optimize_re.cpp +++ b/src/Common/tests/gtest_optimize_re.cpp @@ -4,28 +4,36 @@ TEST(OptimizeRE, analyze) { - auto test_f = [](const std::string & regexp, const std::string & answer) + auto test_f = [](const std::string & regexp, const std::string & answer, std::vector expect_alternatives = {}) { std::string required; bool is_trivial; bool is_prefix; - OptimizedRegularExpression::analyze(regexp, required, is_trivial, is_prefix); + std::vector alternatives; + OptimizedRegularExpression::analyze(regexp, regexp.data(), required, is_trivial, is_prefix, alternatives); + std::cerr << regexp << std::endl; EXPECT_EQ(required, answer); + EXPECT_EQ(alternatives, expect_alternatives); }; test_f("abc", "abc"); test_f("abc(de)fg", "abcdefg"); - test_f("abc(de|xyz)fg", "abc"); - test_f("abc(de?f|xyz)fg", "abc"); - test_f("abc|fg", ""); + test_f("abc(de|xyz)fg", "abc", {"de", "xyz"}); + test_f("abc(de?f|xyz)fg", "abc", {"d", "xyz"}); + test_f("abc|fgk|xyz", "", {"abc","fgk", "xyz"}); test_f("(abc)", "abc"); - test_f("(abc|fg)", ""); - test_f("abc(abc|fg)xyzz", "xyzz"); + test_f("(abc|fgk)", "", {"abc","fgk"}); + test_f("abc(abc|fg)xyzz", "xyzz", {"abc","fg"}); test_f("abc[k]xyzz", "xyzz"); - /// actually the best answer should be xyzz - test_f("(abc[k]xyzz)", "abc"); + test_f("(abc[k]xyzz)", "xyzz"); test_f("abc((de)fg(hi))jk", "abcdefghijk"); test_f("abc((de)fghi+zzz)jk", "abcdefghi"); test_f("abc((de)fg(hi))?jk", "abc"); test_f("abc((de)fghi?zzz)jk", "abcdefgh"); test_f("abc(*cd)jk", "abc"); + test_f(R"(abc(de|xyz|(\{xx\}))fg)", "abc", {"de", "xyz", "{xx}"}); + test_f("abc(abc|fg)?xyzz", "xyzz"); + test_f("abc(abc|fg){0,1}xyzz", "xyzz"); + test_f("abc(abc|fg)xyzz|bcdd?k|bc(f|g|h?)z", "", {"abc", "fg", "bcd", "bc"}); + test_f("abc(abc|fg)xyzz|bc(dd?x|kk?y|(f))k|bc(f|g|h?)z", "", {"abc", "fg", "d", "k", "f", "bc"}); + test_f("((?:abc|efg|xyz)/[a-zA-Z0-9]{1-50})(/?[^ ]*|)", "", {"abc", "efg", "xyz"}); } From 1ab4ef0ffa1ac372cf9997646c796a329e5d10bc Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 7 Mar 2023 11:45:32 +0800 Subject: [PATCH 113/559] add some comments --- src/Functions/parseDateTime.cpp | 65 +++++++++++++++++++-------------- 1 file changed, 37 insertions(+), 28 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 206b7e985be..450db1b2ce4 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -86,31 +86,38 @@ namespace struct DateTime { - Int32 year = 1970; - Int32 month = 1; - Int32 day = 1; + /// If both week_date_format and week_date_format is false, date is composed of year, month and day + Int32 year = 1970; /// year, range [1970, 2106] + Int32 month = 1; /// month of year, range [1, 12] + Int32 day = 1; /// day of month, range [1, 31] - Int32 week = 1; // Week of year based on ISO week date, e.g: 27 - Int32 day_of_week = 1; // Day of week, Monday:1, Tuesday:2, ..., Sunday:7 - bool week_date_format = false; + Int32 week = 1; /// ISO week of year, range [1, 53] + Int32 day_of_week = 1; /// day of week, range [1, 7], 1 represents Monday, 2 represents Tuesday... + bool week_date_format + = false; /// If true, date is composed of week year(reuse year), week of year(use week) and day of week(use day_of_week) - Int32 day_of_year = 1; - bool day_of_year_format = false; + Int32 day_of_year = 1; /// day of year, range [1, 366] + bool day_of_year_format = false; /// If true, date is composed of year(reuse year), day of year(use day_of_year) - bool century_format = false; + bool century_format = false; /// If true, year is calculated from century, range [19, 21] - bool is_year_of_era = false; // Year of era cannot be zero or negative. - bool has_year = false; // Whether year was explicitly specified. + bool is_year_of_era = false; /// If true, year is calculated from era and year of era, the latter cannot be zero or negative. + bool has_year = false; /// Whether year was explicitly specified. + /// If is_clock_hour = true, is_hour_of_half_day = true, hour's range is [1, 12] + /// If is_clock_hour = true, is_hour_of_half_day = false, hour's range is [1, 24] + /// If is_clock_hour = false, is_hour_of_half_day = true, hour's range is [0, 11] + /// If is_clock_hour = false, is_hour_of_half_day = false, hour's range is [0, 23] Int32 hour = 0; - Int32 minute = 0; - Int32 second = 0; + Int32 minute = 0; /// range [0, 59] + Int32 second = 0; /// range [0, 59] - bool is_am = true; // AM -> true, PM -> false - bool is_clock_hour = false; // Whether most recent hour specifier is clockhour - bool is_hour_of_half_day = false; // Whether most recent hour specifier is of half day. + bool is_am = true; /// AM -> true, PM -> false + bool is_clock_hour = false; /// Whether the hour is clockhour + bool is_hour_of_half_day = false; /// Whether the hour is of half day - std::optional time_zone_offset; + bool has_time_zone_offset = false; /// If true, time zone offset if explicitly specified. + Int32 time_zone_offset = 0; /// Offset in seconds between current timezone to UTC. void reset() { @@ -127,18 +134,19 @@ namespace century_format = false; - is_year_of_era = false; // Year of era cannot be zero or negative. - has_year = false; // Whether year was explicitly specified. + is_year_of_era = false; + has_year = false; hour = 0; minute = 0; second = 0; - is_am = true; // AM -> true, PM -> false - is_clock_hour = false; // Whether most recent hour specifier is clockhour - is_hour_of_half_day = false; // Whether most recent hour specifier is of half day. + is_am = true; + is_clock_hour = false; + is_hour_of_half_day = false; - time_zone_offset.reset(); + has_time_zone_offset = false; + time_zone_offset = 0; } void setCentury(Int32 century) @@ -432,12 +440,12 @@ namespace Int64 seconds_since_epoch = days_since_epoch * 86400 + hour * 3600 + minute * 60 + second; /// Time zone is not specified, use local time zone - if (!time_zone_offset) - *time_zone_offset = time_zone.timezoneOffset(seconds_since_epoch); + if (has_time_zone_offset) + time_zone_offset = static_cast(time_zone.timezoneOffset(seconds_since_epoch)); /// Time zone is specified in format string. - if (seconds_since_epoch >= *time_zone_offset) - seconds_since_epoch -= *time_zone_offset; + if (seconds_since_epoch >= time_zone_offset) + seconds_since_epoch -= time_zone_offset; else throw Exception(ErrorCodes::LOGICAL_ERROR, "Seconds since epoch is negative"); @@ -926,7 +934,8 @@ namespace Int32 minute; cur = readNumber2(cur, end, flag, minute); - *date.time_zone_offset = sign * (hour * 3600 + minute * 60); + date.has_time_zone_offset = true; + date.time_zone_offset = sign * (hour * 3600 + minute * 60); return cur; } From 6eafdc57a3471d2a43a6f009cbe0ee054fbb6f19 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 7 Mar 2023 14:35:50 +0800 Subject: [PATCH 114/559] fix failed uts --- src/Functions/parseDateTime.cpp | 72 ++--- .../02668_parse_datetime.reference | 86 +++--- .../0_stateless/02668_parse_datetime.sql | 84 +++--- ...68_parse_datetime_in_joda_syntax.reference | 261 +++++------------- .../02668_parse_datetime_in_joda_syntax.sql | 112 ++++---- 5 files changed, 253 insertions(+), 362 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 450db1b2ce4..6f32ccf716d 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -25,7 +25,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int BAD_ARGUMENTS; extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; - extern const int CANNOT_PARSE_TEXT; + extern const int CANNOT_PARSE_DATETIME; extern const int NOT_ENOUGH_SPACE; } @@ -117,7 +117,7 @@ namespace bool is_hour_of_half_day = false; /// Whether the hour is of half day bool has_time_zone_offset = false; /// If true, time zone offset if explicitly specified. - Int32 time_zone_offset = 0; /// Offset in seconds between current timezone to UTC. + Int64 time_zone_offset = 0; /// Offset in seconds between current timezone to UTC. void reset() { @@ -152,7 +152,7 @@ namespace void setCentury(Int32 century) { if (century < 19 || century > 21) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for century must be in the range [19, 21]", century); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Value {} for century must be in the range [19, 21]", century); century_format = true; year = 100 * century; @@ -162,7 +162,7 @@ namespace void setDayOfWeek(Int32 day_of_week_) { if (day_of_week_ < 1 || day_of_week_ > 7) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for day of week must be in the range [1, 7]", day_of_week_); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Value {} for day of week must be in the range [1, 7]", day_of_week_); day_of_week = day_of_week_; week_date_format = true; @@ -177,7 +177,7 @@ namespace void setMonth(Int32 month_) { if (month_ < 1 || month_ > 12) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for month of year must be in the range [1, 12]", month_); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Value {} for month of year must be in the range [1, 12]", month_); month = month_; week_date_format = false; @@ -192,7 +192,7 @@ namespace void setDayOfMonth(Int32 day_of_month) { if (day_of_month < 1 || day_of_month > 31) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for day of month must be in the range [1, 31]", day_of_month); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Value {} for day of month must be in the range [1, 31]", day_of_month); day = day_of_month; week_date_format = false; @@ -204,10 +204,10 @@ namespace } } - ALWAYS_INLINE void appendDayOfYear(Int32 day_of_year_) + ALWAYS_INLINE void setDayOfYear(Int32 day_of_year_) { if (day_of_year_ < 1 || day_of_year_ > 366) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for day of year must be in the range [1, 366]", day_of_year_); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Value {} for day of year must be in the range [1, 366]", day_of_year_); day_of_year = day_of_year_; day_of_year_format = true; @@ -226,7 +226,7 @@ namespace else if (year_ >= 0 && year_ < 70) year_ += 2000; else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for year2 must be in the range [0, 99]", year_); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Value {} for year2 must be in the range [0, 99]", year_); setYear(year_, is_year_of_era_, is_week_year); } @@ -234,7 +234,7 @@ namespace void setYear(Int32 year_, bool is_year_of_era_ = false, bool is_week_year = false) { if (year_ < minYear || year_ > maxYear) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for year must be in the range [{}, {}]", year_, minYear, maxYear); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Value {} for year must be in the range [{}, {}]", year_, minYear, maxYear); year = year_; century_format = false; @@ -250,7 +250,7 @@ namespace void setWeek(Int32 week_) { if (week_ < 1 || week_ > 53) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for week of week year must be in the range [1, 53]", week_); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Value {} for week of week year must be in the range [1, 53]", week_); week = week_; week_date_format = true; @@ -265,7 +265,7 @@ namespace void setMinute(Int32 minute_) { if (minute_ < 0 || minute_ > 59) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for minute must be in the range [0, 59]", minute_); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Value {} for minute must be in the range [0, 59]", minute_); minute = minute_; } @@ -273,7 +273,7 @@ namespace void setSecond(Int32 second_) { if (second_ < 0 || second_ > 59) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Value {} for second must be in the range [0, 59]", second_); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Value {} for second must be in the range [0, 59]", second_); second = second_; } @@ -282,9 +282,9 @@ namespace { boost::to_lower(text); if (text == "bc") - throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Era BC exceeds the range of DateTime"); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Era BC exceeds the range of DateTime"); else if (text != "ad") - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown era {}", text); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Unknown era {}", text); } ALWAYS_INLINE void setAMPM(String & text) @@ -295,7 +295,7 @@ namespace else if (text == "pm") is_am = false; else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown half day of day: {}", text); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Unknown half day of day: {}", text); } ALWAYS_INLINE void setHour(Int32 hour_, bool is_hour_of_half_day_ = false, bool is_clock_hour_ = false) @@ -328,7 +328,7 @@ namespace if (hour_ < min_hour || hour_ > max_hour) throw Exception( - ErrorCodes::LOGICAL_ERROR, + ErrorCodes::CANNOT_PARSE_DATETIME, "Value {} for hour must be in the range [{}, {}] if_hour_of_half_day={} and is_clock_hour={}", hour, max_hour, @@ -395,7 +395,7 @@ namespace { /// The range of week_of_year[1, 53], day_of_week[1, 7] already checked before if (week_year_ < minYear || week_year_ > maxYear) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid week year {}", week_year_); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Invalid week year {}", week_year_); Int32 days_since_epoch_of_jan_fourth = daysSinceEpochFromDate(week_year_, 1, 4); Int32 first_day_of_week_year = extractISODayOfTheWeek(days_since_epoch_of_jan_fourth); @@ -405,7 +405,7 @@ namespace static ALWAYS_INLINE Int32 daysSinceEpochFromDayOfYear(Int32 year_, Int32 day_of_year_) { if (!isDayOfYearValid(year_, day_of_year_)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of year, year:{} day of year:{}", year_, day_of_year_); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Invalid day of year, year:{} day of year:{}", year_, day_of_year_); Int32 res = daysSinceEpochFromDate(year_, 1, 1); res += day_of_year_ - 1; @@ -415,7 +415,7 @@ namespace static ALWAYS_INLINE Int32 daysSinceEpochFromDate(Int32 year_, Int32 month_, Int32 day_) { if (!isDateValid(year_, month_, day_)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid date, year:{} month:{} day:{}", year_, month_, day_); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Invalid date, year:{} month:{} day:{}", year_, month_, day_); Int32 res = cumulativeYearDays[year_ - 1970]; res += isLeapYear(year_) ? cumulativeLeapDays[month_ - 1] : cumulativeDays[month_ - 1]; @@ -440,14 +440,14 @@ namespace Int64 seconds_since_epoch = days_since_epoch * 86400 + hour * 3600 + minute * 60 + second; /// Time zone is not specified, use local time zone - if (has_time_zone_offset) - time_zone_offset = static_cast(time_zone.timezoneOffset(seconds_since_epoch)); + if (!has_time_zone_offset) + time_zone_offset = time_zone.timezoneOffset(seconds_since_epoch); /// Time zone is specified in format string. if (seconds_since_epoch >= time_zone_offset) seconds_since_epoch -= time_zone_offset; else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Seconds since epoch is negative"); + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Seconds since epoch is negative"); return seconds_since_epoch; } @@ -548,7 +548,7 @@ namespace // Ensure all input was consumed. if (cur < end) throw Exception( - ErrorCodes::CANNOT_PARSE_TEXT, + ErrorCodes::CANNOT_PARSE_DATETIME, "Invalid format input {} is malformed at {}", str_ref.toView(), std::string_view(cur, end - cur)); @@ -607,7 +607,7 @@ namespace checkSpace(cur, end, literal.size(), "required literal size not matched", flag); if (std::string_view(cur, literal.size()) != literal) throw Exception( - ErrorCodes::LOGICAL_ERROR, + ErrorCodes::CANNOT_PARSE_DATETIME, "Unable to parse flag {} from {} because literal {} is expected but {} provided", flag, std::string_view(cur, end - cur), @@ -682,7 +682,7 @@ namespace if (*cur != ch) throw Exception( - ErrorCodes::LOGICAL_ERROR, + ErrorCodes::CANNOT_PARSE_DATETIME, "Unable to parse flag {} from {} because char {} is expected but {} provided", flag, std::string_view(cur, end - cur), @@ -702,7 +702,7 @@ namespace auto it = dayOfWeekMap.find(text); if (it == dayOfWeekMap.end()) throw Exception( - ErrorCodes::CANNOT_PARSE_TEXT, + ErrorCodes::CANNOT_PARSE_DATETIME, "Unable to parse flag {} from {} because of unknown day of week short text {} ", flag, std::string_view(cur, end - cur), @@ -721,7 +721,7 @@ namespace auto it = monthMap.find(text); if (it == monthMap.end()) throw Exception( - ErrorCodes::CANNOT_PARSE_TEXT, + ErrorCodes::CANNOT_PARSE_DATETIME, "Unable to parse flag {} from {} because of unknown month of year short text {}", flag, std::string_view(cur, end - cur), @@ -829,7 +829,7 @@ namespace { Int32 day_of_year; cur = readNumber3(cur, end, flag, day_of_year); - date.appendDayOfYear(day_of_year); + date.setDayOfYear(day_of_year); return cur; } @@ -870,7 +870,7 @@ namespace auto it = dayOfWeekMap.find(text1); if (it == dayOfWeekMap.end()) throw Exception( - ErrorCodes::CANNOT_PARSE_TEXT, + ErrorCodes::CANNOT_PARSE_DATETIME, "Unable to parse first part of flag {} from {} because of unknown day of week text: {}", flag, std::string_view(cur, end - cur), @@ -883,7 +883,7 @@ namespace boost::to_lower(text2); if (text2 != it->second.first) throw Exception( - ErrorCodes::CANNOT_PARSE_TEXT, + ErrorCodes::CANNOT_PARSE_DATETIME, "Unable to parse second part of flag {} from {} because of unknown day of week text: {}", flag, std::string_view(cur, end - cur), @@ -921,7 +921,7 @@ namespace sign = 1; else throw Exception( - ErrorCodes::LOGICAL_ERROR, + ErrorCodes::CANNOT_PARSE_DATETIME, "Unable to parse flag {} from {} because of unknown sign time zone offset: {}", flag, std::string_view(cur, end - cur), @@ -1101,7 +1101,7 @@ namespace /// Need to have read at least one digit. if (cur <= start) throw Exception( - ErrorCodes::LOGICAL_ERROR, + ErrorCodes::CANNOT_PARSE_DATETIME, "Unable to parse flag {} from {} because read number failed", flag, std::string_view(cur, end - cur)); @@ -1172,7 +1172,7 @@ namespace auto it = dayOfWeekMap.find(text1); if (it == dayOfWeekMap.end()) throw Exception( - ErrorCodes::CANNOT_PARSE_TEXT, + ErrorCodes::CANNOT_PARSE_DATETIME, "Unable to parse flag {} from {} because of unknown day of week text: {}", flag, std::string_view(cur, end - cur), @@ -1206,7 +1206,7 @@ namespace { Int32 day_of_year; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 3), flag, day_of_year); - date.appendDayOfYear(day_of_year); + date.setDayOfYear(day_of_year); return cur; } @@ -1226,7 +1226,7 @@ namespace auto it = monthMap.find(text1); if (it == monthMap.end()) throw Exception( - ErrorCodes::CANNOT_PARSE_TEXT, + ErrorCodes::CANNOT_PARSE_DATETIME, "Unable to parse flag {} from {} because of unknown month of year text: {}", flag, std::string_view(cur, end - cur), diff --git a/tests/queries/0_stateless/02668_parse_datetime.reference b/tests/queries/0_stateless/02668_parse_datetime.reference index f0678585a8e..b2a42e01585 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.reference +++ b/tests/queries/0_stateless/02668_parse_datetime.reference @@ -9,23 +9,23 @@ select parseDateTime('07', '%m') = toDateTime('2000-07-01'); 1 select parseDateTime('11-', '%m-') = toDateTime('2000-11-01'); 1 -select parseDateTime('00', '%m'); -- { serverError LOGICAL_ERROR } -select parseDateTime('13', '%m'); -- { serverError LOGICAL_ERROR } -select parseDateTime('12345', '%m'); -- { serverError LOGICAL_ERROR } +select parseDateTime('00', '%m'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('13', '%m'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('12345', '%m'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime('02', '%c') = toDateTime('2000-02-01'); 1 select parseDateTime('07', '%c') = toDateTime('2000-07-01'); 1 select parseDateTime('11-', '%c-') = toDateTime('2000-11-01'); 1 -select parseDateTime('00', '%c'); -- { serverError LOGICAL_ERROR } -select parseDateTime('13', '%c'); -- { serverError LOGICAL_ERROR } -select parseDateTime('12345', '%c'); -- { serverError LOGICAL_ERROR } +select parseDateTime('00', '%c'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('13', '%c'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('12345', '%c'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime('jun', '%b') = toDateTime('2000-06-01'); 1 select parseDateTime('JUN', '%b') = toDateTime('2000-06-01'); 1 -select parseDateTime('abc', '%b'); -- { serverError LOGICAL_ERROR } +select parseDateTime('abc', '%b'); -- { serverError CANNOT_PARSE_DATETIME } -- day of month select parseDateTime('07', '%d') = toDateTime('2000-01-07'); 1 @@ -33,19 +33,21 @@ select parseDateTime('01', '%d') = toDateTime('2000-01-01'); 1 select parseDateTime('/11', '/%d') = toDateTime('2000-01-11'); 1 -select parseDateTime('00', '%d'); -- { serverError LOGICAL_ERROR } -select parseDateTime('32', '%d'); -- { serverError LOGICAL_ERROR } -select parseDateTime('12345', '%d'); -- { serverError LOGICAL_ERROR } -select parseDateTime('02-31', '%m-%d'); -- { serverError LOGICAL_ERROR } -select parseDateTime('04-31', '%m-%d'); -- { serverError LOGICAL_ERROR } --- Ensure all days of month are checked against final selected month -select parseDateTime('01 31 20 02', '%m %d %d %m'); -- { serverError LOGICAL_ERROR } -select parseDateTime('02 31 20 04', '%m %d %d %m'); -- { serverError LOGICAL_ERROR } +select parseDateTime('00', '%d'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('32', '%d'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('12345', '%d'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('02-31', '%m-%d'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('04-31', '%m-%d'); -- { serverError CANNOT_PARSE_DATETIME } +-- The last one is chosen if multiple months of year if supplied +select parseDateTime('01 31 20 02', '%m %d %d %m') = toDateTime('2000-02-20'); +1 +select parseDateTime('02 31 20 04', '%m %d %d %m') = toDateTime('2000-04-20'); +1 select parseDateTime('02 31 01', '%m %d %m') = toDateTime('2000-01-31'); 1 select parseDateTime('2000-02-29', '%Y-%m-%d') = toDateTime('2000-02-29'); 1 -select parseDateTime('2001-02-29', '%Y-%m-%d'); -- { serverError LOGICAL_ERROR } +select parseDateTime('2001-02-29', '%Y-%m-%d'); -- { serverError CANNOT_PARSE_DATETIME } -- day of year select parseDateTime('001', '%j') = toDateTime('2000-01-01'); 1 @@ -75,11 +77,11 @@ select parseDateTime('1980 060', '%Y %j') = toDateTime('1980-02-29'); 1 select parseDateTime('1980 366', '%Y %j') = toDateTime('1980-12-31'); 1 -select parseDateTime('1981 366', '%Y %j'); -- { serverError LOGICAL_ERROR } -select parseDateTime('367', '%j'); -- { serverError LOGICAL_ERROR } -select parseDateTime('000', '%j'); -- { serverError LOGICAL_ERROR } --- Ensure all days of year are checked against final selected year -select parseDateTime('2000 366 2001', '%Y %j %Y'); -- { serverError LOGICAL_ERROR } +select parseDateTime('1981 366', '%Y %j'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('367', '%j'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('000', '%j'); -- { serverError CANNOT_PARSE_DATETIME } +-- The last one is chosen if multiple day of years are supplied. +select parseDateTime('2000 366 2001', '%Y %j %Y'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime('2001 366 2000', '%Y %j %Y') = toDateTime('2000-12-31'); 1 -- hour of day @@ -91,9 +93,9 @@ select parseDateTime('00', '%H', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC 1 select parseDateTime('10', '%H', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); 1 -select parseDateTime('24', '%H', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('-1', '%H', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('1234567', '%H', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('24', '%H', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('-1', '%H', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('1234567', '%H', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime('07', '%k', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); 1 select parseDateTime('23', '%k', 'UTC') = toDateTime('1970-01-01 23:00:00', 'UTC'); @@ -102,9 +104,9 @@ select parseDateTime('00', '%k', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC 1 select parseDateTime('10', '%k', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); 1 -select parseDateTime('24', '%k', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('-1', '%k', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('1234567', '%k', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('24', '%k', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('-1', '%k', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('1234567', '%k', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- hour of half day select parseDateTime('07', '%h', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); 1 @@ -114,9 +116,9 @@ select parseDateTime('01', '%h', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC 1 select parseDateTime('10', '%h', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); 1 -select parseDateTime('00', '%h', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('13', '%h', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('123456789', '%h', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('00', '%h', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('13', '%h', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('123456789', '%h', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime('07', '%I', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); 1 select parseDateTime('12', '%I', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); @@ -125,9 +127,9 @@ select parseDateTime('01', '%I', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC 1 select parseDateTime('10', '%I', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); 1 -select parseDateTime('00', '%I', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('13', '%I', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('123456789', '%I', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('00', '%I', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('13', '%I', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('123456789', '%I', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime('07', '%l', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); 1 select parseDateTime('12', '%l', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); @@ -136,9 +138,9 @@ select parseDateTime('01', '%l', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC 1 select parseDateTime('10', '%l', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); 1 -select parseDateTime('00', '%l', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('13', '%l', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('123456789', '%l', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('00', '%l', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('13', '%l', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('123456789', '%l', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- half of day select parseDateTime('07 PM', '%H %p', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); 1 @@ -175,9 +177,9 @@ select parseDateTime('59', '%i', 'UTC') = toDateTime('1970-01-01 00:59:00', 'UTC 1 select parseDateTime('00/', '%i/', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); 1 -select parseDateTime('60', '%i', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('-1', '%i', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('123456789', '%i', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('60', '%i', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('-1', '%i', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('123456789', '%i', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- second select parseDateTime('09', '%s', 'UTC') = toDateTime('1970-01-01 00:00:09', 'UTC'); 1 @@ -185,9 +187,9 @@ select parseDateTime('58', '%s', 'UTC') = toDateTime('1970-01-01 00:00:58', 'UTC 1 select parseDateTime('00/', '%s/', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); 1 -select parseDateTime('60', '%s', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('-1', '%s', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('123456789', '%s', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('60', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('-1', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('123456789', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- mixed YMD format select parseDateTime('2021-01-04+23:00:00', '%Y-%m-%d+%H:%i:%s') = toDateTime('2021-01-04 23:00:00'); 1 diff --git a/tests/queries/0_stateless/02668_parse_datetime.sql b/tests/queries/0_stateless/02668_parse_datetime.sql index 4a6d3711c8c..8ff3f5a03f2 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.sql +++ b/tests/queries/0_stateless/02668_parse_datetime.sql @@ -6,36 +6,36 @@ select parseDateTime('2020', '%Y') = toDateTime('2020-01-01'); select parseDateTime('02', '%m') = toDateTime('2000-02-01'); select parseDateTime('07', '%m') = toDateTime('2000-07-01'); select parseDateTime('11-', '%m-') = toDateTime('2000-11-01'); -select parseDateTime('00', '%m'); -- { serverError LOGICAL_ERROR } -select parseDateTime('13', '%m'); -- { serverError LOGICAL_ERROR } -select parseDateTime('12345', '%m'); -- { serverError LOGICAL_ERROR } +select parseDateTime('00', '%m'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('13', '%m'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('12345', '%m'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime('02', '%c') = toDateTime('2000-02-01'); select parseDateTime('07', '%c') = toDateTime('2000-07-01'); select parseDateTime('11-', '%c-') = toDateTime('2000-11-01'); -select parseDateTime('00', '%c'); -- { serverError LOGICAL_ERROR } -select parseDateTime('13', '%c'); -- { serverError LOGICAL_ERROR } -select parseDateTime('12345', '%c'); -- { serverError LOGICAL_ERROR } +select parseDateTime('00', '%c'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('13', '%c'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('12345', '%c'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime('jun', '%b') = toDateTime('2000-06-01'); select parseDateTime('JUN', '%b') = toDateTime('2000-06-01'); -select parseDateTime('abc', '%b'); -- { serverError LOGICAL_ERROR } +select parseDateTime('abc', '%b'); -- { serverError CANNOT_PARSE_DATETIME } -- day of month select parseDateTime('07', '%d') = toDateTime('2000-01-07'); select parseDateTime('01', '%d') = toDateTime('2000-01-01'); select parseDateTime('/11', '/%d') = toDateTime('2000-01-11'); -select parseDateTime('00', '%d'); -- { serverError LOGICAL_ERROR } -select parseDateTime('32', '%d'); -- { serverError LOGICAL_ERROR } -select parseDateTime('12345', '%d'); -- { serverError LOGICAL_ERROR } -select parseDateTime('02-31', '%m-%d'); -- { serverError LOGICAL_ERROR } -select parseDateTime('04-31', '%m-%d'); -- { serverError LOGICAL_ERROR } --- Ensure all days of month are checked against final selected month -select parseDateTime('01 31 20 02', '%m %d %d %m'); -- { serverError LOGICAL_ERROR } -select parseDateTime('02 31 20 04', '%m %d %d %m'); -- { serverError LOGICAL_ERROR } +select parseDateTime('00', '%d'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('32', '%d'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('12345', '%d'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('02-31', '%m-%d'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('04-31', '%m-%d'); -- { serverError CANNOT_PARSE_DATETIME } +-- The last one is chosen if multiple months of year if supplied +select parseDateTime('01 31 20 02', '%m %d %d %m') = toDateTime('2000-02-20'); +select parseDateTime('02 31 20 04', '%m %d %d %m') = toDateTime('2000-04-20'); select parseDateTime('02 31 01', '%m %d %m') = toDateTime('2000-01-31'); select parseDateTime('2000-02-29', '%Y-%m-%d') = toDateTime('2000-02-29'); -select parseDateTime('2001-02-29', '%Y-%m-%d'); -- { serverError LOGICAL_ERROR } +select parseDateTime('2001-02-29', '%Y-%m-%d'); -- { serverError CANNOT_PARSE_DATETIME } -- day of year select parseDateTime('001', '%j') = toDateTime('2000-01-01'); @@ -52,11 +52,11 @@ select parseDateTime('1980 /031/', '%Y /%j/') = toDateTime('1980-01-31'); select parseDateTime('1980 032', '%Y %j') = toDateTime('1980-02-01'); select parseDateTime('1980 060', '%Y %j') = toDateTime('1980-02-29'); select parseDateTime('1980 366', '%Y %j') = toDateTime('1980-12-31'); -select parseDateTime('1981 366', '%Y %j'); -- { serverError LOGICAL_ERROR } -select parseDateTime('367', '%j'); -- { serverError LOGICAL_ERROR } -select parseDateTime('000', '%j'); -- { serverError LOGICAL_ERROR } --- Ensure all days of year are checked against final selected year -select parseDateTime('2000 366 2001', '%Y %j %Y'); -- { serverError LOGICAL_ERROR } +select parseDateTime('1981 366', '%Y %j'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('367', '%j'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('000', '%j'); -- { serverError CANNOT_PARSE_DATETIME } +-- The last one is chosen if multiple day of years are supplied. +select parseDateTime('2000 366 2001', '%Y %j %Y'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime('2001 366 2000', '%Y %j %Y') = toDateTime('2000-12-31'); -- hour of day @@ -64,39 +64,39 @@ select parseDateTime('07', '%H', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC select parseDateTime('23', '%H', 'UTC') = toDateTime('1970-01-01 23:00:00', 'UTC'); select parseDateTime('00', '%H', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); select parseDateTime('10', '%H', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); -select parseDateTime('24', '%H', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('-1', '%H', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('1234567', '%H', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('24', '%H', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('-1', '%H', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('1234567', '%H', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime('07', '%k', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); select parseDateTime('23', '%k', 'UTC') = toDateTime('1970-01-01 23:00:00', 'UTC'); select parseDateTime('00', '%k', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); select parseDateTime('10', '%k', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); -select parseDateTime('24', '%k', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('-1', '%k', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('1234567', '%k', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('24', '%k', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('-1', '%k', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('1234567', '%k', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- hour of half day select parseDateTime('07', '%h', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); select parseDateTime('12', '%h', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); select parseDateTime('01', '%h', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); select parseDateTime('10', '%h', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); -select parseDateTime('00', '%h', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('13', '%h', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('123456789', '%h', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('00', '%h', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('13', '%h', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('123456789', '%h', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime('07', '%I', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); select parseDateTime('12', '%I', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); select parseDateTime('01', '%I', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); select parseDateTime('10', '%I', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); -select parseDateTime('00', '%I', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('13', '%I', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('123456789', '%I', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('00', '%I', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('13', '%I', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('123456789', '%I', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime('07', '%l', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); select parseDateTime('12', '%l', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); select parseDateTime('01', '%l', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); select parseDateTime('10', '%l', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); -select parseDateTime('00', '%l', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('13', '%l', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('123456789', '%l', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('00', '%l', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('13', '%l', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('123456789', '%l', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- half of day select parseDateTime('07 PM', '%H %p', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); @@ -118,17 +118,17 @@ select parseDateTime('12 AM', '%h %p', 'UTC') = toDateTime('1970-01-01 00:00:00' select parseDateTime('08', '%i', 'UTC') = toDateTime('1970-01-01 00:08:00', 'UTC'); select parseDateTime('59', '%i', 'UTC') = toDateTime('1970-01-01 00:59:00', 'UTC'); select parseDateTime('00/', '%i/', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -select parseDateTime('60', '%i', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('-1', '%i', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('123456789', '%i', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('60', '%i', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('-1', '%i', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('123456789', '%i', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- second select parseDateTime('09', '%s', 'UTC') = toDateTime('1970-01-01 00:00:09', 'UTC'); select parseDateTime('58', '%s', 'UTC') = toDateTime('1970-01-01 00:00:58', 'UTC'); select parseDateTime('00/', '%s/', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -select parseDateTime('60', '%s', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('-1', '%s', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTime('123456789', '%s', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTime('60', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('-1', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTime('123456789', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- mixed YMD format select parseDateTime('2021-01-04+23:00:00', '%Y-%m-%d+%H:%i:%s') = toDateTime('2021-01-04 23:00:00'); diff --git a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference index 5d338022a26..99426a8c8d8 100644 --- a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference +++ b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference @@ -1,344 +1,233 @@ -- { echoOn } -- empty select parseDateTimeInJodaSyntax(' ', ' ', 'UTC') = toDateTime('1970-01-01', 'UTC'); -1 + -- era select parseDateTimeInJodaSyntax('AD 1999', 'G YYYY') = toDateTime('1999-01-01'); -1 select parseDateTimeInJodaSyntax('ad 1999', 'G YYYY') = toDateTime('1999-01-01'); -1 select parseDateTimeInJodaSyntax('Ad 1999', 'G YYYY') = toDateTime('1999-01-01'); -1 select parseDateTimeInJodaSyntax('AD 1999', 'G YYYY') = toDateTime('1999-01-01'); -1 select parseDateTimeInJodaSyntax('AD 1999', 'G yyyy') = toDateTime('1999-01-01'); -1 select parseDateTimeInJodaSyntax('AD 1999 2000', 'G YYYY yyyy') = toDateTime('2000-01-01'); -1 select parseDateTimeInJodaSyntax('AD 1999 2000', 'G yyyy YYYY') = toDateTime('2000-01-01'); -1 -select parseDateTimeInJodaSyntax('AD 1999', 'G Y'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('AD 1999', 'G YY'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('AD 1999', 'G YYY'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('BC', 'G'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('AB', 'G'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('AD 1999', 'G Y'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('AD 1999', 'G YY'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('AD 1999', 'G YYY'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('BC', 'G'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('AB', 'G'); -- { serverError CANNOT_PARSE_DATETIME } + -- year of era select parseDateTimeInJodaSyntax('2106', 'YYYY', 'UTC') = toDateTime('2106-01-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('1970', 'YYYY', 'UTC') = toDateTime('1970-01-01', 'UTC'); -1 -select parseDateTimeInJodaSyntax('1969', 'YYYY', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('2107', 'YYYY', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('+1999', 'YYYY', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('1969', 'YYYY', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('2107', 'YYYY', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('+1999', 'YYYY', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } + select parseDateTimeInJodaSyntax('12', 'YY', 'UTC') = toDateTime('2012-01-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('69', 'YY', 'UTC') = toDateTime('2069-01-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('70', 'YY', 'UTC') = toDateTime('1970-01-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('99', 'YY', 'UTC') = toDateTime('1999-01-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('01', 'YY', 'UTC') = toDateTime('2001-01-01', 'UTC'); -1 -select parseDateTimeInJodaSyntax('1', 'YY', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('1', 'YY', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } + select parseDateTimeInJodaSyntax('99 98 97', 'YY YY YY', 'UTC') = toDateTime('1997-01-01', 'UTC'); -1 + -- year select parseDateTimeInJodaSyntax('12', 'yy', 'UTC') = toDateTime('2012-01-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('69', 'yy', 'UTC') = toDateTime('2069-01-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('70', 'yy', 'UTC') = toDateTime('1970-01-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('99', 'yy', 'UTC') = toDateTime('1999-01-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('+99', 'yy', 'UTC') = toDateTime('1999-01-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('+99 02', 'yy MM', 'UTC') = toDateTime('1999-02-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('10 +10', 'MM yy', 'UTC') = toDateTime('2010-10-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('10+2001', 'MMyyyy', 'UTC') = toDateTime('2001-10-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('+200110', 'yyyyMM', 'UTC') = toDateTime('2001-10-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('1970', 'yyyy', 'UTC') = toDateTime('1970-01-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('2106', 'yyyy', 'UTC') = toDateTime('2106-01-01', 'UTC'); -1 -select parseDateTimeInJodaSyntax('1969', 'yyyy', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('2107', 'yyyy', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('1969', 'yyyy', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('2107', 'yyyy', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } + -- week year select parseDateTimeInJodaSyntax('2106', 'xxxx', 'UTC') = toDateTime('2106-01-04', 'UTC'); -1 select parseDateTimeInJodaSyntax('1971', 'xxxx', 'UTC') = toDateTime('1971-01-04', 'UTC'); -1 select parseDateTimeInJodaSyntax('2025', 'xxxx', 'UTC') = toDateTime('2024-12-30', 'UTC'); -1 select parseDateTimeInJodaSyntax('12', 'xx', 'UTC') = toDateTime('2012-01-02', 'UTC'); -1 select parseDateTimeInJodaSyntax('69', 'xx', 'UTC') = toDateTime('2068-12-31', 'UTC'); -1 select parseDateTimeInJodaSyntax('99', 'xx', 'UTC') = toDateTime('1999-01-04', 'UTC'); -1 select parseDateTimeInJodaSyntax('01', 'xx', 'UTC') = toDateTime('2001-01-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('+10', 'xx', 'UTC') = toDateTime('2010-01-04', 'UTC'); -1 select parseDateTimeInJodaSyntax('+99 01', 'xx ww', 'UTC') = toDateTime('1999-01-04', 'UTC'); -1 select parseDateTimeInJodaSyntax('+99 02', 'xx ww', 'UTC') = toDateTime('1999-01-11', 'UTC'); -1 select parseDateTimeInJodaSyntax('10 +10', 'ww xx', 'UTC') = toDateTime('2010-03-08', 'UTC'); -1 select parseDateTimeInJodaSyntax('2+10', 'wwxx', 'UTC') = toDateTime('2010-01-11', 'UTC'); -1 select parseDateTimeInJodaSyntax('+102', 'xxM', 'UTC') = toDateTime('2010-02-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('+20102', 'xxxxM', 'UTC') = toDateTime('2010-02-01', 'UTC'); -1 -select parseDateTimeInJodaSyntax('1970', 'xxxx', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('1969', 'xxxx', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('2107', 'xxxx', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('1970', 'xxxx', 'UTC'); -- { serverError VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE } +select parseDateTimeInJodaSyntax('1969', 'xxxx', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('2107', 'xxxx', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } + -- century of era select parseDateTimeInJodaSyntax('20', 'CC', 'UTC') = toDateTime('2000-01-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('21', 'CC', 'UTC') = toDateTime('2100-01-01', 'UTC'); -1 -select parseDateTimeInJodaSyntax('19', 'CC', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('22', 'CC', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('19', 'CC', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('22', 'CC', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } + -- month select parseDateTimeInJodaSyntax('1', 'M', 'UTC') = toDateTime('2000-01-01', 'UTC'); -1 select parseDateTimeInJodaSyntax(' 7', ' MM', 'UTC') = toDateTime('2000-07-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('11', 'M', 'UTC') = toDateTime('2000-11-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('10-', 'M-', 'UTC') = toDateTime('2000-10-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('-12-', '-M-', 'UTC') = toDateTime('2000-12-01', 'UTC'); -1 -select parseDateTimeInJodaSyntax('0', 'M', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('13', 'M', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('12345', 'M', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('0', 'M', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('13', 'M', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('12345', 'M', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } --- Ensure MMM and MMMM specifiers consume both short- and long-form month names select parseDateTimeInJodaSyntax('Aug', 'MMM', 'UTC') = toDateTime('2000-08-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('AuG', 'MMM', 'UTC') = toDateTime('2000-08-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('august', 'MMM', 'UTC') = toDateTime('2000-08-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('Aug', 'MMMM', 'UTC') = toDateTime('2000-08-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('AuG', 'MMMM', 'UTC') = toDateTime('2000-08-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('august', 'MMMM', 'UTC') = toDateTime('2000-08-01', 'UTC'); -1 --- invalid month names -select parseDateTimeInJodaSyntax('Decembr', 'MMM', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('Decembr', 'MMMM', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('Decemberary', 'MMM', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('Decemberary', 'MMMM', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('asdf', 'MMM', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('asdf', 'MMMM', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('Decembr', 'MMM', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('Decembr', 'MMMM', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('Decemberary', 'MMM', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('Decemberary', 'MMMM', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('asdf', 'MMM', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('asdf', 'MMMM', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } + -- day of month select parseDateTimeInJodaSyntax('1', 'd', 'UTC') = toDateTime('2000-01-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('7 ', 'dd ', 'UTC') = toDateTime('2000-01-07', 'UTC'); -1 select parseDateTimeInJodaSyntax('/11', '/dd', 'UTC') = toDateTime('2000-01-11', 'UTC'); -1 select parseDateTimeInJodaSyntax('/31/', '/d/', 'UTC') = toDateTime('2000-01-31', 'UTC'); -1 -select parseDateTimeInJodaSyntax('0', 'd', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('32', 'd', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('12345', 'd', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('02-31', 'M-d', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('04-31', 'M-d', 'UTC'); -- { serverError LOGICAL_ERROR } ---- Ensure all days of month are checked against final selected month +select parseDateTimeInJodaSyntax('0', 'd', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('32', 'd', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('12345', 'd', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('02-31', 'M-d', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('04-31', 'M-d', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +-- The last one is chosen if multiple day of months are supplied. select parseDateTimeInJodaSyntax('2 31 1', 'M d M', 'UTC') = toDateTime('2000-01-31', 'UTC'); -1 -select parseDateTimeInJodaSyntax('1 31 20 2', 'M d d M', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('2 31 20 4', 'M d d M', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('1 31 20 2', 'M d d M', 'UTC') = toDateTime('2000-02-20', 'UTC'); +select parseDateTimeInJodaSyntax('2 31 20 4', 'M d d M', 'UTC') = toDateTime('2000-04-20', 'UTC'); --- Leap year select parseDateTimeInJodaSyntax('2020-02-29', 'YYYY-M-d', 'UTC') = toDateTime('2020-02-29', 'UTC'); -1 -select parseDateTimeInJodaSyntax('2001-02-29', 'YYYY-M-d', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('2001-02-29', 'YYYY-M-d', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } + -- day of year select parseDateTimeInJodaSyntax('1', 'D', 'UTC') = toDateTime('2000-01-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('7 ', 'DD ', 'UTC') = toDateTime('2000-01-07', 'UTC'); -1 select parseDateTimeInJodaSyntax('/11', '/DD', 'UTC') = toDateTime('2000-01-11', 'UTC'); -1 select parseDateTimeInJodaSyntax('/31/', '/DDD/', 'UTC') = toDateTime('2000-01-31', 'UTC'); -1 select parseDateTimeInJodaSyntax('32', 'D', 'UTC') = toDateTime('2000-02-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('60', 'D', 'UTC') = toDateTime('2000-02-29', 'UTC'); -1 select parseDateTimeInJodaSyntax('365', 'D', 'UTC') = toDateTime('2000-12-30', 'UTC'); -1 select parseDateTimeInJodaSyntax('366', 'D', 'UTC') = toDateTime('2000-12-31', 'UTC'); -1 select parseDateTimeInJodaSyntax('1999 1', 'yyyy D', 'UTC') = toDateTime('1999-01-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('1999 7 ', 'yyyy DD ', 'UTC') = toDateTime('1999-01-07', 'UTC'); -1 select parseDateTimeInJodaSyntax('1999 /11', 'yyyy /DD', 'UTC') = toDateTime('1999-01-11', 'UTC'); -1 select parseDateTimeInJodaSyntax('1999 /31/', 'yyyy /DD/', 'UTC') = toDateTime('1999-01-31', 'UTC'); -1 select parseDateTimeInJodaSyntax('1999 32', 'yyyy D', 'UTC') = toDateTime('1999-02-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('1999 60', 'yyyy D', 'UTC') = toDateTime('1999-03-01', 'UTC'); -1 select parseDateTimeInJodaSyntax('1999 365', 'yyyy D', 'UTC') = toDateTime('1999-12-31', 'UTC'); -1 -select parseDateTimeInJodaSyntax('1999 366', 'yyyy D', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('1999 366', 'yyyy D', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } --- Ensure all days of year are checked against final selected year select parseDateTimeInJodaSyntax('2001 366 2000', 'yyyy D yyyy', 'UTC') = toDateTime('2000-12-31', 'UTC'); -1 -select parseDateTimeInJodaSyntax('2000 366 2001', 'yyyy D yyyy', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('0', 'D', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('367', 'D', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('2000 366 2001', 'yyyy D yyyy', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('0', 'D', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('367', 'D', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } + -- hour of day select parseDateTimeInJodaSyntax('7', 'H', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('23', 'HH', 'UTC') = toDateTime('1970-01-01 23:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('0', 'HHH', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('10', 'HHHHHHHH', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); -1 --- invalid hour od day -select parseDateTimeInJodaSyntax('24', 'H', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('-1', 'H', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('123456789', 'H', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('24', 'H', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('-1', 'H', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('123456789', 'H', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } + -- clock hour of day select parseDateTimeInJodaSyntax('7', 'k', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('24', 'kk', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('1', 'kkk', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('10', 'kkkkkkkk', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); -1 -- invalid clock hour of day -select parseDateTimeInJodaSyntax('25', 'k', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('0', 'k', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('123456789', 'k', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('25', 'k', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('0', 'k', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('123456789', 'k', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } + -- hour of half day select parseDateTimeInJodaSyntax('7', 'K', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('11', 'KK', 'UTC') = toDateTime('1970-01-01 11:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('0', 'KKK', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('10', 'KKKKKKKK', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); -1 -- invalid hour of half day -select parseDateTimeInJodaSyntax('12', 'K', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('-1', 'K', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('123456789', 'K', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('12', 'K', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('-1', 'K', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('123456789', 'K', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } + -- clock hour of half day select parseDateTimeInJodaSyntax('7', 'h', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('12', 'hh', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('1', 'hhh', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('10', 'hhhhhhhh', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); -1 -- invalid clock hour of half day -select parseDateTimeInJodaSyntax('13', 'h', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('0', 'h', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('123456789', 'h', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('13', 'h', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('0', 'h', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('123456789', 'h', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } + -- half of day --- Half of day has no effect if hour or clockhour of day is provided hour of day tests select parseDateTimeInJodaSyntax('7 PM', 'H a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('7 AM', 'H a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('7 pm', 'H a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('7 am', 'H a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('0 PM', 'H a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('0 AM', 'H a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('0 pm', 'H a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('0 am', 'H a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('7 PM', 'k a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('7 AM', 'k a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('7 pm', 'k a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('7 am', 'k a', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('24 PM', 'k a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('24 AM', 'k a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('24 pm', 'k a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('24 am', 'k a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -1 -- Half of day has effect if hour or clockhour of halfday is provided select parseDateTimeInJodaSyntax('0 PM', 'K a', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('0 AM', 'K a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('6 PM', 'K a', 'UTC') = toDateTime('1970-01-01 18:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('6 AM', 'K a', 'UTC') = toDateTime('1970-01-01 06:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('11 PM', 'K a', 'UTC') = toDateTime('1970-01-01 23:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('11 AM', 'K a', 'UTC') = toDateTime('1970-01-01 11:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('1 PM', 'h a', 'UTC') = toDateTime('1970-01-01 13:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('1 AM', 'h a', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('6 PM', 'h a', 'UTC') = toDateTime('1970-01-01 18:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('6 AM', 'h a', 'UTC') = toDateTime('1970-01-01 06:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('12 PM', 'h a', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('12 AM', 'h a', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -1 -- time gives precendent to most recent time specifier select parseDateTimeInJodaSyntax('0 1 AM', 'H h a', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('12 1 PM', 'H h a', 'UTC') = toDateTime('1970-01-01 13:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('1 AM 0', 'h a H', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('1 AM 12', 'h a H', 'UTC') = toDateTime('1970-01-01 12:00:00', 'UTC'); -1 + -- minute select parseDateTimeInJodaSyntax('8', 'm', 'UTC') = toDateTime('1970-01-01 00:08:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('59', 'mm', 'UTC') = toDateTime('1970-01-01 00:59:00', 'UTC'); -1 select parseDateTimeInJodaSyntax('0/', 'mmm/', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -1 -select parseDateTimeInJodaSyntax('60', 'm', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('-1', 'm', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('123456789', 'm', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('60', 'm', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('-1', 'm', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('123456789', 'm', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } + -- second select parseDateTimeInJodaSyntax('9', 's', 'UTC') = toDateTime('1970-01-01 00:00:09', 'UTC'); -1 select parseDateTimeInJodaSyntax('58', 'ss', 'UTC') = toDateTime('1970-01-01 00:00:58', 'UTC'); -1 select parseDateTimeInJodaSyntax('0/', 's/', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -1 -select parseDateTimeInJodaSyntax('60', 's', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('-1', 's', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('123456789', 's', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('60', 's', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('-1', 's', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('123456789', 's', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } + +-- { echoOff } diff --git a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql index 2f2b0d26b2c..99426a8c8d8 100644 --- a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql +++ b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql @@ -10,25 +10,25 @@ select parseDateTimeInJodaSyntax('AD 1999', 'G YYYY') = toDateTime('1999-01-01') select parseDateTimeInJodaSyntax('AD 1999', 'G yyyy') = toDateTime('1999-01-01'); select parseDateTimeInJodaSyntax('AD 1999 2000', 'G YYYY yyyy') = toDateTime('2000-01-01'); select parseDateTimeInJodaSyntax('AD 1999 2000', 'G yyyy YYYY') = toDateTime('2000-01-01'); -select parseDateTimeInJodaSyntax('AD 1999', 'G Y'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('AD 1999', 'G YY'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('AD 1999', 'G YYY'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('BC', 'G'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('AB', 'G'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('AD 1999', 'G Y'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('AD 1999', 'G YY'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('AD 1999', 'G YYY'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('BC', 'G'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('AB', 'G'); -- { serverError CANNOT_PARSE_DATETIME } -- year of era select parseDateTimeInJodaSyntax('2106', 'YYYY', 'UTC') = toDateTime('2106-01-01', 'UTC'); select parseDateTimeInJodaSyntax('1970', 'YYYY', 'UTC') = toDateTime('1970-01-01', 'UTC'); -select parseDateTimeInJodaSyntax('1969', 'YYYY', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('2107', 'YYYY', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('+1999', 'YYYY', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('1969', 'YYYY', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('2107', 'YYYY', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('+1999', 'YYYY', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTimeInJodaSyntax('12', 'YY', 'UTC') = toDateTime('2012-01-01', 'UTC'); select parseDateTimeInJodaSyntax('69', 'YY', 'UTC') = toDateTime('2069-01-01', 'UTC'); select parseDateTimeInJodaSyntax('70', 'YY', 'UTC') = toDateTime('1970-01-01', 'UTC'); select parseDateTimeInJodaSyntax('99', 'YY', 'UTC') = toDateTime('1999-01-01', 'UTC'); select parseDateTimeInJodaSyntax('01', 'YY', 'UTC') = toDateTime('2001-01-01', 'UTC'); -select parseDateTimeInJodaSyntax('1', 'YY', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('1', 'YY', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTimeInJodaSyntax('99 98 97', 'YY YY YY', 'UTC') = toDateTime('1997-01-01', 'UTC'); @@ -44,8 +44,8 @@ select parseDateTimeInJodaSyntax('10+2001', 'MMyyyy', 'UTC') = toDateTime('2001- select parseDateTimeInJodaSyntax('+200110', 'yyyyMM', 'UTC') = toDateTime('2001-10-01', 'UTC'); select parseDateTimeInJodaSyntax('1970', 'yyyy', 'UTC') = toDateTime('1970-01-01', 'UTC'); select parseDateTimeInJodaSyntax('2106', 'yyyy', 'UTC') = toDateTime('2106-01-01', 'UTC'); -select parseDateTimeInJodaSyntax('1969', 'yyyy', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('2107', 'yyyy', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('1969', 'yyyy', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('2107', 'yyyy', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- week year select parseDateTimeInJodaSyntax('2106', 'xxxx', 'UTC') = toDateTime('2106-01-04', 'UTC'); @@ -62,15 +62,15 @@ select parseDateTimeInJodaSyntax('10 +10', 'ww xx', 'UTC') = toDateTime('2010-03 select parseDateTimeInJodaSyntax('2+10', 'wwxx', 'UTC') = toDateTime('2010-01-11', 'UTC'); select parseDateTimeInJodaSyntax('+102', 'xxM', 'UTC') = toDateTime('2010-02-01', 'UTC'); select parseDateTimeInJodaSyntax('+20102', 'xxxxM', 'UTC') = toDateTime('2010-02-01', 'UTC'); -select parseDateTimeInJodaSyntax('1970', 'xxxx', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('1969', 'xxxx', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('2107', 'xxxx', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('1970', 'xxxx', 'UTC'); -- { serverError VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE } +select parseDateTimeInJodaSyntax('1969', 'xxxx', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('2107', 'xxxx', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- century of era select parseDateTimeInJodaSyntax('20', 'CC', 'UTC') = toDateTime('2000-01-01', 'UTC'); select parseDateTimeInJodaSyntax('21', 'CC', 'UTC') = toDateTime('2100-01-01', 'UTC'); -select parseDateTimeInJodaSyntax('19', 'CC', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('22', 'CC', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('19', 'CC', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('22', 'CC', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- month select parseDateTimeInJodaSyntax('1', 'M', 'UTC') = toDateTime('2000-01-01', 'UTC'); @@ -78,9 +78,9 @@ select parseDateTimeInJodaSyntax(' 7', ' MM', 'UTC') = toDateTime('2000-07-01', select parseDateTimeInJodaSyntax('11', 'M', 'UTC') = toDateTime('2000-11-01', 'UTC'); select parseDateTimeInJodaSyntax('10-', 'M-', 'UTC') = toDateTime('2000-10-01', 'UTC'); select parseDateTimeInJodaSyntax('-12-', '-M-', 'UTC') = toDateTime('2000-12-01', 'UTC'); -select parseDateTimeInJodaSyntax('0', 'M', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('13', 'M', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('12345', 'M', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('0', 'M', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('13', 'M', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('12345', 'M', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } --- Ensure MMM and MMMM specifiers consume both short- and long-form month names select parseDateTimeInJodaSyntax('Aug', 'MMM', 'UTC') = toDateTime('2000-08-01', 'UTC'); select parseDateTimeInJodaSyntax('AuG', 'MMM', 'UTC') = toDateTime('2000-08-01', 'UTC'); @@ -89,30 +89,30 @@ select parseDateTimeInJodaSyntax('Aug', 'MMMM', 'UTC') = toDateTime('2000-08-01' select parseDateTimeInJodaSyntax('AuG', 'MMMM', 'UTC') = toDateTime('2000-08-01', 'UTC'); select parseDateTimeInJodaSyntax('august', 'MMMM', 'UTC') = toDateTime('2000-08-01', 'UTC'); --- invalid month names -select parseDateTimeInJodaSyntax('Decembr', 'MMM', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('Decembr', 'MMMM', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('Decemberary', 'MMM', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('Decemberary', 'MMMM', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('asdf', 'MMM', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('asdf', 'MMMM', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('Decembr', 'MMM', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('Decembr', 'MMMM', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('Decemberary', 'MMM', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('Decemberary', 'MMMM', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('asdf', 'MMM', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('asdf', 'MMMM', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- day of month select parseDateTimeInJodaSyntax('1', 'd', 'UTC') = toDateTime('2000-01-01', 'UTC'); select parseDateTimeInJodaSyntax('7 ', 'dd ', 'UTC') = toDateTime('2000-01-07', 'UTC'); select parseDateTimeInJodaSyntax('/11', '/dd', 'UTC') = toDateTime('2000-01-11', 'UTC'); select parseDateTimeInJodaSyntax('/31/', '/d/', 'UTC') = toDateTime('2000-01-31', 'UTC'); -select parseDateTimeInJodaSyntax('0', 'd', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('32', 'd', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('12345', 'd', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('02-31', 'M-d', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('04-31', 'M-d', 'UTC'); -- { serverError LOGICAL_ERROR } ---- Ensure all days of month are checked against final selected month +select parseDateTimeInJodaSyntax('0', 'd', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('32', 'd', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('12345', 'd', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('02-31', 'M-d', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('04-31', 'M-d', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +-- The last one is chosen if multiple day of months are supplied. select parseDateTimeInJodaSyntax('2 31 1', 'M d M', 'UTC') = toDateTime('2000-01-31', 'UTC'); -select parseDateTimeInJodaSyntax('1 31 20 2', 'M d d M', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('2 31 20 4', 'M d d M', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('1 31 20 2', 'M d d M', 'UTC') = toDateTime('2000-02-20', 'UTC'); +select parseDateTimeInJodaSyntax('2 31 20 4', 'M d d M', 'UTC') = toDateTime('2000-04-20', 'UTC'); --- Leap year select parseDateTimeInJodaSyntax('2020-02-29', 'YYYY-M-d', 'UTC') = toDateTime('2020-02-29', 'UTC'); -select parseDateTimeInJodaSyntax('2001-02-29', 'YYYY-M-d', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('2001-02-29', 'YYYY-M-d', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- day of year select parseDateTimeInJodaSyntax('1', 'D', 'UTC') = toDateTime('2000-01-01', 'UTC'); @@ -130,12 +130,12 @@ select parseDateTimeInJodaSyntax('1999 /31/', 'yyyy /DD/', 'UTC') = toDateTime(' select parseDateTimeInJodaSyntax('1999 32', 'yyyy D', 'UTC') = toDateTime('1999-02-01', 'UTC'); select parseDateTimeInJodaSyntax('1999 60', 'yyyy D', 'UTC') = toDateTime('1999-03-01', 'UTC'); select parseDateTimeInJodaSyntax('1999 365', 'yyyy D', 'UTC') = toDateTime('1999-12-31', 'UTC'); -select parseDateTimeInJodaSyntax('1999 366', 'yyyy D', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('1999 366', 'yyyy D', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } --- Ensure all days of year are checked against final selected year select parseDateTimeInJodaSyntax('2001 366 2000', 'yyyy D yyyy', 'UTC') = toDateTime('2000-12-31', 'UTC'); -select parseDateTimeInJodaSyntax('2000 366 2001', 'yyyy D yyyy', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('0', 'D', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('367', 'D', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('2000 366 2001', 'yyyy D yyyy', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('0', 'D', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('367', 'D', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- hour of day select parseDateTimeInJodaSyntax('7', 'H', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); @@ -143,9 +143,9 @@ select parseDateTimeInJodaSyntax('23', 'HH', 'UTC') = toDateTime('1970-01-01 23: select parseDateTimeInJodaSyntax('0', 'HHH', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); select parseDateTimeInJodaSyntax('10', 'HHHHHHHH', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); --- invalid hour od day -select parseDateTimeInJodaSyntax('24', 'H', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('-1', 'H', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('123456789', 'H', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('24', 'H', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('-1', 'H', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('123456789', 'H', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- clock hour of day select parseDateTimeInJodaSyntax('7', 'k', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); @@ -153,9 +153,9 @@ select parseDateTimeInJodaSyntax('24', 'kk', 'UTC') = toDateTime('1970-01-01 00: select parseDateTimeInJodaSyntax('1', 'kkk', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); select parseDateTimeInJodaSyntax('10', 'kkkkkkkk', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); -- invalid clock hour of day -select parseDateTimeInJodaSyntax('25', 'k', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('0', 'k', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('123456789', 'k', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('25', 'k', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('0', 'k', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('123456789', 'k', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- hour of half day select parseDateTimeInJodaSyntax('7', 'K', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); @@ -163,9 +163,9 @@ select parseDateTimeInJodaSyntax('11', 'KK', 'UTC') = toDateTime('1970-01-01 11: select parseDateTimeInJodaSyntax('0', 'KKK', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); select parseDateTimeInJodaSyntax('10', 'KKKKKKKK', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); -- invalid hour of half day -select parseDateTimeInJodaSyntax('12', 'K', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('-1', 'K', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('123456789', 'K', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('12', 'K', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('-1', 'K', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('123456789', 'K', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- clock hour of half day select parseDateTimeInJodaSyntax('7', 'h', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); @@ -173,9 +173,9 @@ select parseDateTimeInJodaSyntax('12', 'hh', 'UTC') = toDateTime('1970-01-01 00: select parseDateTimeInJodaSyntax('1', 'hhh', 'UTC') = toDateTime('1970-01-01 01:00:00', 'UTC'); select parseDateTimeInJodaSyntax('10', 'hhhhhhhh', 'UTC') = toDateTime('1970-01-01 10:00:00', 'UTC'); -- invalid clock hour of half day -select parseDateTimeInJodaSyntax('13', 'h', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('0', 'h', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('123456789', 'h', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('13', 'h', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('0', 'h', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('123456789', 'h', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- half of day --- Half of day has no effect if hour or clockhour of day is provided hour of day tests @@ -218,16 +218,16 @@ select parseDateTimeInJodaSyntax('1 AM 12', 'h a H', 'UTC') = toDateTime('1970-0 select parseDateTimeInJodaSyntax('8', 'm', 'UTC') = toDateTime('1970-01-01 00:08:00', 'UTC'); select parseDateTimeInJodaSyntax('59', 'mm', 'UTC') = toDateTime('1970-01-01 00:59:00', 'UTC'); select parseDateTimeInJodaSyntax('0/', 'mmm/', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -select parseDateTimeInJodaSyntax('60', 'm', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('-1', 'm', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('123456789', 'm', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('60', 'm', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('-1', 'm', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('123456789', 'm', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- second select parseDateTimeInJodaSyntax('9', 's', 'UTC') = toDateTime('1970-01-01 00:00:09', 'UTC'); select parseDateTimeInJodaSyntax('58', 'ss', 'UTC') = toDateTime('1970-01-01 00:00:58', 'UTC'); select parseDateTimeInJodaSyntax('0/', 's/', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -select parseDateTimeInJodaSyntax('60', 's', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('-1', 's', 'UTC'); -- { serverError LOGICAL_ERROR } -select parseDateTimeInJodaSyntax('123456789', 's', 'UTC'); -- { serverError LOGICAL_ERROR } +select parseDateTimeInJodaSyntax('60', 's', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('-1', 's', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +select parseDateTimeInJodaSyntax('123456789', 's', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- { echoOff } From b303fb563068e2e5552c1902d8cf1bd7a9c1fe98 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 7 Mar 2023 15:30:38 +0800 Subject: [PATCH 115/559] finish docs --- contrib/libunwind | 2 +- .../functions/date-time-functions.md | 4 +- .../functions/type-conversion-functions.md | 63 +++++++++++++++++++ src/Functions/parseDateTime.cpp | 2 +- 4 files changed, 67 insertions(+), 4 deletions(-) diff --git a/contrib/libunwind b/contrib/libunwind index 5022f30f3e0..e48aa13f67d 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit 5022f30f3e092a54a7c101c335ce5e08769db366 +Subproject commit e48aa13f67dc722511b5af33a32ba9b7748176b5 diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index f6af8abcbaf..cd36fb53830 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1199,7 +1199,7 @@ SELECT timeSlots(toDateTime64('1980-12-12 21:01:02.1234', 4, 'UTC'), toDecimal64 └───────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` -## formatDateTime +## formatDateTime {#date_time_functions-formatDateTime} Formats a Time according to the given Format string. Format is a constant expression, so you cannot have multiple formats for a single result column. @@ -1295,7 +1295,7 @@ Result: - [formatDateTimeInJodaSyntax](##formatDateTimeInJodaSyntax) -## formatDateTimeInJodaSyntax +## formatDateTimeInJodaSyntax {#date_time_functions-formatDateTimeInJodaSyntax} Similar to formatDateTime, except that it formats datetime in Joda style instead of MySQL style. Refer to https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html. diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index e587e56b20e..41c779dcbbb 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1347,6 +1347,69 @@ Same as for [parseDateTime64BestEffort](#parsedatetime64besteffort), except that Same as for [parseDateTime64BestEffort](#parsedatetime64besteffort), except that this function prefers US date format (`MM/DD/YYYY` etc.) in case of ambiguity and returns zero date or zero date time when it encounters a date format that cannot be processed. +## parseDateTime +Parse [DateTime](/docs/en/sql-reference/data-types/datetime.md) from string according to a specified MySQL format string, refer to https://dev.mysql.com/doc/refman/8.0/en/date-and-time-functions.html#function_date-format. It is nearly an opposite operation of function [formatDateTime](/docs/en/sql-reference/functions/date-time-functions.md#date_time_functions-formatDateTime). + +Alias: `TO_UNIX_TIMESTAMP`. + + +**Syntax** + +``` sql +parseDateTime(str, format[, timezone]) +``` + +**Returned value(s)** + +Returns DateTime values parsed from input string according to the determined MySQL style format. + + +**Supported replacement fields** + +Most of replacement fields used in [formatDateTime](/docs/en/sql-reference/functions/date-time-functions.md#date_time_functions-formatDateTime) is supported, except: +- %f: fractional second +- %Q: Quarter (1-4) + +**Example** + +``` sql +SELECT parseDateTime('2021-01-04+23:00:00', '%Y-%m-%d+%H:%i:%s') + +┌─parseDateTime('2021-01-04+23:00:00', '%Y-%m-%d+%H:%i:%s')─┐ +│ 2021-01-04 23:00:00 │ +└───────────────────────────────────────────────────────────┘ +``` + +## parseDateTimeInJodaSyntax + +Similar to parseDateTime, except that it parses string to [DateTime](/docs/en/sql-reference/data-types/datetime.md) in Joda style instead of MySQL style. Refer to https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html. It is nearly an opposite operation of function [formatDateTimeInJodaSyntax](/docs/en/sql-reference/functions/date-time-functions.md#date_time_functions-formatDateTimeInJodaSyntax) + +**Syntax** + +``` sql +parseDateTimeInJodaSyntax(str, format[, timezone]) +``` + +**Returned value(s)** + +Returns DateTime values parsed from input string according to the determined Joda style format. + +**Supported replacement fields** + +Most of replacement fields used in [formatDateTimeInJoda](/docs/en/sql-reference/functions/date-time-functions.md#date_time_functions-formatDateTime) is supported, except: +- S: fraction of second +- z: time zone +- Z: time zone offset/id + +**Example** + +``` sql +SELECT parseDateTimeInJodaSyntax('2023-02-24 14:53:31', 'yyyy-MM-dd HH:mm:ss', 'Europe/Minsk') + +┌─parseDateTimeInJodaSyntax('2023-02-24 14:53:31', 'yyyy-MM-dd HH:mm:ss', 'Europe/Minsk')─┐ +│ 2023-02-24 14:53:31 │ +└─────────────────────────────────────────────────────────────────────────────────────────┘ +``` ## toLowCardinality diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 6f32ccf716d..edf25d3e131 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -1756,7 +1756,7 @@ namespace REGISTER_FUNCTION(ParseDateTime) { factory.registerFunction(); - factory.registerAlias("TO_UNIXTIME", "parseDateTime"); + factory.registerAlias("TO_UNIX_TIMESTAMP", "parseDateTime"); factory.registerFunction(); } From ea73f68ab448853df738bbc3df1abfd518e45071 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 7 Mar 2023 15:50:11 +0800 Subject: [PATCH 116/559] rename symbols --- src/Functions/parseDateTime.cpp | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index edf25d3e131..2283f09893f 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -532,17 +532,19 @@ namespace auto col_res = ColumnDateTime::create(); col_res->reserve(input_rows_count); auto & data_res = col_res->getData(); - DateTime date; + + /// Make datetime fit in a cache line. + alignas(64) DateTime datetime; for (size_t i = 0; i < input_rows_count; ++i) { - date.reset(); + datetime.reset(); StringRef str_ref = col_str->getDataAt(i); Pos cur = str_ref.data; Pos end = str_ref.data + str_ref.size; for (const auto & instruction : instructions) { - cur = instruction.perform(cur, end, date); + cur = instruction.perform(cur, end, datetime); } // Ensure all input was consumed. @@ -553,7 +555,7 @@ namespace str_ref.toView(), std::string_view(cur, end - cur)); - Int64 time = date.checkAndGetDateTime(time_zone); + Int64 time = datetime.checkAndGetDateTime(time_zone); data_res.push_back(static_cast(time)); } From 5723e1f3a3e594d7b6d4bb061e51a0cd757a9678 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 7 Mar 2023 16:24:00 +0800 Subject: [PATCH 117/559] fix code style --- src/Functions/parseDateTime.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 2283f09893f..58d9d8f317b 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -18,7 +18,6 @@ namespace DB { namespace ErrorCodes { - extern const int LOGICAL_ERROR; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; From 8186473cc28c730b1f150f62dac971ec0923e2fb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 7 Mar 2023 14:28:28 +0000 Subject: [PATCH 118/559] Remove some commented code. --- src/Processors/QueryPlan/AggregatingStep.h | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index 8597d09cf0e..3d128d788ac 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -117,13 +117,8 @@ public: DataStreams input_streams_, Aggregator::Params params_, bool final_, - //size_t max_block_size_, size_t merge_threads_, size_t temporary_data_merge_threads_ - //bool group_by_use_nulls_, - //SortDescription group_by_sort_description_, - //bool should_produce_results_in_order_of_bucket_number_ - //bool memory_bound_merging_of_aggregation_results_enabled_ ); String getName() const override { return "AggregatingProjection"; } @@ -132,16 +127,9 @@ public: private: Aggregator::Params params; bool final; - //size_t max_block_size; size_t merge_threads; size_t temporary_data_merge_threads; - //bool storage_has_evenly_distributed_read; - //bool group_by_use_nulls; - //SortDescription group_by_sort_description; - //bool should_produce_results_in_order_of_bucket_number; - //bool memory_bound_merging_of_aggregation_results_enabled; - Processors aggregating; }; From a170a909a4832fcc548566625e8a38379803163f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 10 Mar 2023 10:06:32 +0000 Subject: [PATCH 119/559] Add expiration window for S3 credentials --- src/Backups/BackupIO_S3.cpp | 5 ++- src/Coordination/KeeperSnapshotManagerS3.cpp | 4 ++- src/Disks/ObjectStorages/S3/diskSettings.cpp | 4 ++- src/IO/S3/Client.cpp | 6 ++-- src/IO/S3/Client.h | 3 +- src/IO/S3/Credentials.cpp | 32 ++++++++++++++------ src/IO/S3/Credentials.h | 15 +++++++-- src/IO/S3Common.cpp | 8 ++++- src/IO/S3Common.h | 1 + src/Storages/StorageS3.cpp | 3 +- 10 files changed, 61 insertions(+), 20 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 2f315e8d488..2f6c76dcdf6 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include @@ -67,7 +68,9 @@ namespace settings.auth_settings.use_environment_credentials.value_or( context->getConfigRef().getBool("s3.use_environment_credentials", false)), settings.auth_settings.use_insecure_imds_request.value_or( - context->getConfigRef().getBool("s3.use_insecure_imds_request", false))); + context->getConfigRef().getBool("s3.use_insecure_imds_request", false)), + settings.auth_settings.expiration_window_seconds.value_or( + context->getConfigRef().getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS))); } Aws::Vector listObjects(S3::Client & client, const S3::URI & s3_uri, const String & file_name) diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index 7b47324a890..cabeb13e2f8 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -7,6 +7,7 @@ #include #include +#include #include #include #include @@ -103,7 +104,8 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo auth_settings.server_side_encryption_customer_key_base64, std::move(headers), auth_settings.use_environment_credentials.value_or(false), - auth_settings.use_insecure_imds_request.value_or(false)); + auth_settings.use_insecure_imds_request.value_or(false), + auth_settings.expiration_window_seconds.value_or(S3::DEFAULT_EXPIRATION_WINDOW_SECONDS)); auto new_client = std::make_shared(std::move(new_uri), std::move(auth_settings), std::move(client)); diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index e0e4735f519..1c3bb857798 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include @@ -152,7 +153,8 @@ std::unique_ptr getClient( config.getString(config_prefix + ".server_side_encryption_customer_key_base64", ""), {}, config.getBool(config_prefix + ".use_environment_credentials", config.getBool("s3.use_environment_credentials", false)), - config.getBool(config_prefix + ".use_insecure_imds_request", config.getBool("s3.use_insecure_imds_request", false))); + config.getBool(config_prefix + ".use_insecure_imds_request", config.getBool("s3.use_insecure_imds_request", false)), + config.getBool(config_prefix + ".expiration_window_seconds", config.getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS))); } } diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 5c0539ee486..a21f83a6444 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -564,7 +564,8 @@ std::unique_ptr ClientFactory::create( // NOLINT const String & server_side_encryption_customer_key_base64, HTTPHeaderEntries headers, bool use_environment_credentials, - bool use_insecure_imds_request) + bool use_insecure_imds_request, + uint64_t expiration_window_seconds) { PocoHTTPClientConfiguration client_configuration = cfg_; client_configuration.updateSchemeAndRegion(); @@ -592,7 +593,8 @@ std::unique_ptr ClientFactory::create( // NOLINT client_configuration, std::move(credentials), use_environment_credentials, - use_insecure_imds_request); + use_insecure_imds_request, + expiration_window_seconds); client_configuration.retryStrategy = std::make_shared(std::move(client_configuration.retryStrategy)); return Client::create( diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index 18ba62d1006..f095c5b31e7 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -223,7 +223,8 @@ public: const String & server_side_encryption_customer_key_base64, HTTPHeaderEntries headers, bool use_environment_credentials, - bool use_insecure_imds_request); + bool use_insecure_imds_request, + uint64_t expiration_window_seconds); PocoHTTPClientConfiguration createClientConfiguration( const String & force_region, diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 4b9fa59ea2a..f6675961ddc 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -21,6 +21,21 @@ namespace DB::S3 { +namespace +{ + +bool areCredentialsEmptyOrExpired(const Aws::Auth::AWSCredentials & credentials, uint64_t expiration_window_seconds) +{ + if (credentials.IsEmpty()) + return true; + + const Aws::Utils::DateTime now = Aws::Utils::DateTime::Now(); + return now >= credentials.GetExpiration() - std::chrono::seconds(expiration_window_seconds); +} + + +} + AWSEC2MetadataClient::AWSEC2MetadataClient(const Aws::Client::ClientConfiguration & client_configuration, const char * endpoint_) : Aws::Internal::AWSHttpResourceClient(client_configuration) , endpoint(endpoint_) @@ -270,8 +285,10 @@ void AWSInstanceProfileCredentialsProvider::refreshIfExpired() Reload(); } -AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider::AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider(DB::S3::PocoHTTPClientConfiguration & aws_client_configuration) +AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider::AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider( + DB::S3::PocoHTTPClientConfiguration & aws_client_configuration, uint64_t expiration_window_seconds_) : logger(&Poco::Logger::get("AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider")) + , expiration_window_seconds(expiration_window_seconds_) { // check environment variables String tmp_region = Aws::Environment::GetEnv("AWS_DEFAULT_REGION"); @@ -388,16 +405,12 @@ void AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider::Reload() void AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider::refreshIfExpired() { Aws::Utils::Threading::ReaderLockGuard guard(m_reloadLock); - if (!credentials.IsExpiredOrEmpty()) - { + if (!areCredentialsEmptyOrExpired(credentials, expiration_window_seconds)) return; - } guard.UpgradeToWriterLock(); - if (!credentials.IsExpiredOrEmpty()) // double-checked lock to avoid refreshing twice - { + if (!areCredentialsEmptyOrExpired(credentials, expiration_window_seconds)) // double-checked lock to avoid refreshing twice return; - } Reload(); } @@ -406,7 +419,8 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( const DB::S3::PocoHTTPClientConfiguration & configuration, const Aws::Auth::AWSCredentials & credentials, bool use_environment_credentials, - bool use_insecure_imds_request) + bool use_insecure_imds_request, + uint64_t expiration_window_seconds) { auto * logger = &Poco::Logger::get("S3CredentialsProviderChain"); @@ -439,7 +453,7 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( configuration.for_disk_s3, configuration.get_request_throttler, configuration.put_request_throttler); - AddProvider(std::make_shared(aws_client_configuration)); + AddProvider(std::make_shared(aws_client_configuration, expiration_window_seconds)); } AddProvider(std::make_shared()); diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index f786810726d..d6214c5e2fa 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -17,6 +17,8 @@ namespace DB::S3 { +inline static constexpr uint64_t DEFAULT_EXPIRATION_WINDOW_SECONDS = 120; + class AWSEC2MetadataClient : public Aws::Internal::AWSHttpResourceClient { static constexpr char EC2_SECURITY_CREDENTIALS_RESOURCE[] = "/latest/meta-data/iam/security-credentials"; @@ -97,9 +99,11 @@ class AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider : public Aws::Auth::AWS /// See STSAssumeRoleWebIdentityCredentialsProvider. public: - explicit AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider(DB::S3::PocoHTTPClientConfiguration & aws_client_configuration); + explicit AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider( + DB::S3::PocoHTTPClientConfiguration & aws_client_configuration, uint64_t expiration_window_seconds_); Aws::Auth::AWSCredentials GetAWSCredentials() override; + protected: void Reload() override; @@ -114,14 +118,19 @@ private: Aws::String token; bool initialized = false; Poco::Logger * logger; + uint64_t expiration_window_seconds; }; class S3CredentialsProviderChain : public Aws::Auth::AWSCredentialsProviderChain { public: - S3CredentialsProviderChain(const DB::S3::PocoHTTPClientConfiguration & configuration, const Aws::Auth::AWSCredentials & credentials, bool use_environment_credentials, bool use_insecure_imds_request); + S3CredentialsProviderChain( + const DB::S3::PocoHTTPClientConfiguration & configuration, + const Aws::Auth::AWSCredentials & credentials, + bool use_environment_credentials, + bool use_insecure_imds_request, + uint64_t expiration_window_seconds); }; - } #endif diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index aa8de07c3f4..4acc31ca472 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -85,6 +85,10 @@ AuthSettings AuthSettings::loadFromConfig(const std::string & config_elem, const if (config.has(config_elem + ".use_insecure_imds_request")) use_insecure_imds_request = config.getBool(config_elem + ".use_insecure_imds_request"); + std::optional expiration_window_seconds; + if (config.has(config_elem + ".expiration_window_seconds")) + expiration_window_seconds = config.getUInt64(config_elem + ".expiration_window_seconds"); + HTTPHeaderEntries headers; Poco::Util::AbstractConfiguration::Keys subconfig_keys; config.keys(config_elem, subconfig_keys); @@ -107,7 +111,8 @@ AuthSettings AuthSettings::loadFromConfig(const std::string & config_elem, const std::move(server_side_encryption_customer_key_base64), std::move(headers), use_environment_credentials, - use_insecure_imds_request + use_insecure_imds_request, + expiration_window_seconds }; } @@ -127,6 +132,7 @@ void AuthSettings::updateFrom(const AuthSettings & from) server_side_encryption_customer_key_base64 = from.server_side_encryption_customer_key_base64; use_environment_credentials = from.use_environment_credentials; use_insecure_imds_request = from.use_insecure_imds_request; + expiration_window_seconds = from.expiration_window_seconds; } } diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index 7f277176632..ff948c065f8 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -84,6 +84,7 @@ struct AuthSettings std::optional use_environment_credentials; std::optional use_insecure_imds_request; + std::optional expiration_window_seconds; bool operator==(const AuthSettings & other) const = default; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index ed290c38c1f..baf18844b55 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1266,7 +1266,8 @@ void StorageS3::updateConfiguration(ContextPtr ctx, StorageS3::Configuration & u upd.auth_settings.server_side_encryption_customer_key_base64, std::move(headers), upd.auth_settings.use_environment_credentials.value_or(ctx->getConfigRef().getBool("s3.use_environment_credentials", false)), - upd.auth_settings.use_insecure_imds_request.value_or(ctx->getConfigRef().getBool("s3.use_insecure_imds_request", false))); + upd.auth_settings.use_insecure_imds_request.value_or(ctx->getConfigRef().getBool("s3.use_insecure_imds_request", false)), + upd.auth_settings.expiration_window_seconds.value_or(ctx->getConfigRef().getUInt64("s3.expiration_window_seconds", 120))); } void StorageS3::processNamedCollectionResult(StorageS3::Configuration & configuration, const NamedCollection & collection) From 55c07ea16ef535614828f14b255a57db64a22335 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 10 Mar 2023 10:12:01 +0000 Subject: [PATCH 120/559] Update docs --- docs/en/engines/table-engines/integrations/s3.md | 2 ++ docs/en/engines/table-engines/mergetree-family/mergetree.md | 1 + 2 files changed, 3 insertions(+) diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 723425429a5..dd843945e10 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -150,6 +150,7 @@ The following settings can be specified in configuration file for given endpoint - `use_environment_credentials` — If set to `true`, S3 client will try to obtain credentials from environment variables and [Amazon EC2](https://en.wikipedia.org/wiki/Amazon_Elastic_Compute_Cloud) metadata for given endpoint. Optional, default value is `false`. - `region` — Specifies S3 region name. Optional. - `use_insecure_imds_request` — If set to `true`, S3 client will use insecure IMDS request while obtaining credentials from Amazon EC2 metadata. Optional, default value is `false`. +- `expiration_window_seconds` — Grace period for checking if expiration-based credentials have expired. Optional, default value is `120`. - `header` — Adds specified HTTP header to a request to given endpoint. Optional, can be specified multiple times. - `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set. Optional. - `max_single_read_retries` — The maximum number of attempts during single read. Default value is `4`. Optional. @@ -166,6 +167,7 @@ The following settings can be specified in configuration file for given endpoint + diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index fc8060077b0..64bbe6cbb50 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -960,6 +960,7 @@ Optional parameters: - `support_batch_delete` — This controls the check to see if batch deletes are supported. Set this to `false` when using Google Cloud Storage (GCS) as GCS does not support batch deletes and preventing the checks will prevent error messages in the logs. - `use_environment_credentials` — Reads AWS credentials from the Environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY and AWS_SESSION_TOKEN if they exist. Default value is `false`. - `use_insecure_imds_request` — If set to `true`, S3 client will use insecure IMDS request while obtaining credentials from Amazon EC2 metadata. Default value is `false`. +- `expiration_window_seconds` — Grace period for checking if expiration-based credentials have expired. Optional, default value is `120`. - `proxy` — Proxy configuration for S3 endpoint. Each `uri` element inside `proxy` block should contain a proxy URL. - `connect_timeout_ms` — Socket connect timeout in milliseconds. Default value is `10 seconds`. - `request_timeout_ms` — Request timeout in milliseconds. Default value is `5 seconds`. From 39a11854862d914033ba3f9271cd0c57da75ae11 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 10 Mar 2023 15:30:29 +0100 Subject: [PATCH 121/559] fix test --- src/Common/OptimizedRegularExpression.cpp | 1 + src/Common/tests/gtest_optimize_re.cpp | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index b1fc9a2174c..efa33f27ed4 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -150,6 +150,7 @@ const char * OptimizedRegularExpressionImpl::analyze( break; case '(': + is_trivial = false; if (!in_square_braces) { /// Check for case-insensitive flag. diff --git a/src/Common/tests/gtest_optimize_re.cpp b/src/Common/tests/gtest_optimize_re.cpp index ebd6a28900b..c90b5086b1c 100644 --- a/src/Common/tests/gtest_optimize_re.cpp +++ b/src/Common/tests/gtest_optimize_re.cpp @@ -25,7 +25,7 @@ TEST(OptimizeRE, analyze) test_f("abc(abc|fg)xyzz", "xyzz", {"abc","fg"}); test_f("abc[k]xyzz", "xyzz"); test_f("(abc[k]xyzz)", "xyzz"); - test_f("abc((de)fg(hi))jk", "abcdefghijk"); + test_f("abc((de)fg(hi))jk", "abcdefghi"); test_f("abc((de)fghi+zzz)jk", "abcdefghi"); test_f("abc((de)fg(hi))?jk", "abc"); test_f("abc((de)fghi?zzz)jk", "abcdefgh"); From b6c91d9ace8dc532710a13434523808939c2de8e Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 10 Mar 2023 15:48:18 +0000 Subject: [PATCH 122/559] Support Decimals and Date32 in Avro format --- docs/en/interfaces/formats.md | 37 +++++----- docs/en/interfaces/schema-inference.md | 5 ++ .../Formats/Impl/AvroRowInputFormat.cpp | 74 ++++++++++++++++++- .../Formats/Impl/AvroRowInputFormat.h | 12 +-- .../Formats/Impl/AvroRowOutputFormat.cpp | 51 ++++++++++++- .../Formats/Impl/AvroRowOutputFormat.h | 4 +- 6 files changed, 153 insertions(+), 30 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index db2e773a685..23581d7bb62 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1808,23 +1808,26 @@ ClickHouse Avro format supports reading and writing [Avro data files](https://av The table below shows supported data types and how they match ClickHouse [data types](/docs/en/sql-reference/data-types/index.md) in `INSERT` and `SELECT` queries. -| Avro data type `INSERT` | ClickHouse data type | Avro data type `SELECT` | -|---------------------------------------------|-----------------------------------------------------------------------------------------------------------------|-------------------------------------------------| -| `boolean`, `int`, `long`, `float`, `double` | [Int(8\ | 16\ |32)](/docs/en/sql-reference/data-types/int-uint.md), [UInt(8\|16\|32)](/docs/en/sql-reference/data-types/int-uint.md) | `int` | -| `boolean`, `int`, `long`, `float`, `double` | [Int64](/docs/en/sql-reference/data-types/int-uint.md), [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | `long` | -| `boolean`, `int`, `long`, `float`, `double` | [Float32](/docs/en/sql-reference/data-types/float.md) | `float` | -| `boolean`, `int`, `long`, `float`, `double` | [Float64](/docs/en/sql-reference/data-types/float.md) | `double` | -| `bytes`, `string`, `fixed`, `enum` | [String](/docs/en/sql-reference/data-types/string.md) | `bytes` or `string` \* | -| `bytes`, `string`, `fixed` | [FixedString(N)](/docs/en/sql-reference/data-types/fixedstring.md) | `fixed(N)` | -| `enum` | [Enum(8\ | 16)](/docs/en/sql-reference/data-types/enum.md) | `enum` | -| `array(T)` | [Array(T)](/docs/en/sql-reference/data-types/array.md) | `array(T)` | -| `union(null, T)`, `union(T, null)` | [Nullable(T)](/docs/en/sql-reference/data-types/date.md) | `union(null, T)` | -| `null` | [Nullable(Nothing)](/docs/en/sql-reference/data-types/special-data-types/nothing.md) | `null` | -| `int (date)` \** | [Date](/docs/en/sql-reference/data-types/date.md) | `int (date)` \** | -| `long (timestamp-millis)` \** | [DateTime64(3)](/docs/en/sql-reference/data-types/datetime.md) | `long (timestamp-millis)` \* | -| `long (timestamp-micros)` \** | [DateTime64(6)](/docs/en/sql-reference/data-types/datetime.md) | `long (timestamp-micros)` \* | -| `int` | [IPv4](/docs/en/sql-reference/data-types/domains/ipv4.md) | `int` | -| `fixed(16)` | [IPv6](/docs/en/sql-reference/data-types/domains/ipv6.md) | `fixed(16)` | +| Avro data type `INSERT` | ClickHouse data type | Avro data type `SELECT` | +|---------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------|-------------------------------| +| `boolean`, `int`, `long`, `float`, `double` | [Int(8\16\32)](/docs/en/sql-reference/data-types/int-uint.md), [UInt(8\16\32)](/docs/en/sql-reference/data-types/int-uint.md) | `int` | +| `boolean`, `int`, `long`, `float`, `double` | [Int64](/docs/en/sql-reference/data-types/int-uint.md), [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | `long` | +| `boolean`, `int`, `long`, `float`, `double` | [Float32](/docs/en/sql-reference/data-types/float.md) | `float` | +| `boolean`, `int`, `long`, `float`, `double` | [Float64](/docs/en/sql-reference/data-types/float.md) | `double` | +| `bytes`, `string`, `fixed`, `enum` | [String](/docs/en/sql-reference/data-types/string.md) | `bytes` or `string` \* | +| `bytes`, `string`, `fixed` | [FixedString(N)](/docs/en/sql-reference/data-types/fixedstring.md) | `fixed(N)` | +| `enum` | [Enum(8\16)](/docs/en/sql-reference/data-types/enum.md) | `enum` | +| `array(T)` | [Array(T)](/docs/en/sql-reference/data-types/array.md) | `array(T)` | +| `union(null, T)`, `union(T, null)` | [Nullable(T)](/docs/en/sql-reference/data-types/date.md) | `union(null, T)` | +| `null` | [Nullable(Nothing)](/docs/en/sql-reference/data-types/special-data-types/nothing.md) | `null` | +| `int (date)` \** | [Date](/docs/en/sql-reference/data-types/date.md), [Date32](docs/en/sql-reference/data-types/date32.md) | `int (date)` \** | +| `long (timestamp-millis)` \** | [DateTime64(3)](/docs/en/sql-reference/data-types/datetime.md) | `long (timestamp-millis)` \** | +| `long (timestamp-micros)` \** | [DateTime64(6)](/docs/en/sql-reference/data-types/datetime.md) | `long (timestamp-micros)` \** | +| `int` | [IPv4](/docs/en/sql-reference/data-types/domains/ipv4.md) | `int` | +| `fixed(16)` | [IPv6](/docs/en/sql-reference/data-types/domains/ipv6.md) | `fixed(16)` | +| `bytes (decimal)` \** | [Decimal(P, S)](/docs/en/sql-reference/data-types/decimal.mdd) | `bytes (decimal)` \** | +| `string (uuid)` \** | [UUID](/docs/en/sql-reference/data-types/uuid.mdd) | `string (uuid)` \** | + \* `bytes` is default, controlled by [output_format_avro_string_column_pattern](/docs/en/operations/settings/settings-formats.md/#output_format_avro_string_column_pattern) \** [Avro logical types](https://avro.apache.org/docs/current/spec.html#Logical+Types) diff --git a/docs/en/interfaces/schema-inference.md b/docs/en/interfaces/schema-inference.md index e028b4a6d96..4b9d37bcc7d 100644 --- a/docs/en/interfaces/schema-inference.md +++ b/docs/en/interfaces/schema-inference.md @@ -1473,6 +1473,7 @@ In Avro format ClickHouse reads its schema from the data and converts it to Clic |------------------------------------|--------------------------------------------------------------------------------| | `boolean` | [Bool](../sql-reference/data-types/boolean.md) | | `int` | [Int32](../sql-reference/data-types/int-uint.md) | +| `int (date)` \* | [Date32](../sql-reference/data-types/date32.md) | | `long` | [Int64](../sql-reference/data-types/int-uint.md) | | `float` | [Float32](../sql-reference/data-types/float.md) | | `double` | [Float64](../sql-reference/data-types/float.md) | @@ -1482,6 +1483,10 @@ In Avro format ClickHouse reads its schema from the data and converts it to Clic | `array(T)` | [Array(T)](../sql-reference/data-types/array.md) | | `union(null, T)`, `union(T, null)` | [Nullable(T)](../sql-reference/data-types/date.md) | | `null` | [Nullable(Nothing)](../sql-reference/data-types/special-data-types/nothing.md) | +| `string (uuid)` \* | [UUID](../sql-reference/data-types/uuid.md) | +| `binary (decimal)` \* | [Decimal(P, S)](../sql-reference/data-typesdecimal.md) | + +\* [Avro logical types](https://avro.apache.org/docs/current/spec.html#Logical+Types) Other Avro types are not supported. diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index cb851c4a1e9..9873107208e 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -11,11 +11,14 @@ #include #include #include +#include #include #include #include +#include +#include #include #include #include @@ -124,6 +127,7 @@ static void insertNumber(IColumn & column, WhichDataType type, T value) case TypeIndex::Int16: assert_cast(column).insertValue(static_cast(value)); break; + case TypeIndex::Date32: [[fallthrough]]; case TypeIndex::Int32: assert_cast(column).insertValue(static_cast(value)); break; @@ -153,6 +157,39 @@ static void insertNumber(IColumn & column, WhichDataType type, T value) } } +template +static AvroDeserializer::DeserializeFn createDecimalDeserializeFn(const avro::NodePtr & root_node, const DataTypePtr & target_type) +{ + auto logical_type = root_node->logicalType(); + const auto & decimal_type = assert_cast(*target_type); + if (decimal_type.getScale() != static_cast(logical_type.scale()) || decimal_type.getPrecision() != static_cast(logical_type.precision())) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot insert Avro decimal with scale {} and precision {} to ClickHouse Decimal with scale {} and precision {}", + logical_type.scale(), + logical_type.precision(), + decimal_type.getScale(), + decimal_type.getPrecision()); + + return [tmp = std::string(), target_type](IColumn & column, avro::Decoder & decoder) mutable + { + decoder.decodeString(tmp); + if (tmp.size() != sizeof(typename DecimalType::FieldType)) + throw ParsingException( + ErrorCodes::CANNOT_PARSE_UUID, + "Cannot parse type {}, expected binary data with size {}, got {}", + target_type->getName(), + sizeof(typename DecimalType::FieldType), + tmp.size()); + + typename DecimalType::FieldType field; + ReadBufferFromString buf(tmp); + readBinaryBigEndian(field.value, buf); + assert_cast(column).insertValue(field); + return true; + }; +} + static std::string nodeToJson(avro::NodePtr root_node) { std::ostringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM @@ -169,7 +206,7 @@ static std::string nodeName(avro::NodePtr node) return avro::toString(node->type()); } -AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::NodePtr root_node, DataTypePtr target_type) +AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(const avro::NodePtr & root_node, const DataTypePtr & target_type) { if (target_type->lowCardinality()) { @@ -214,6 +251,14 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node return true; }; } + if (target.isDecimal32()) + return createDecimalDeserializeFn(root_node, target_type); + if (target.isDecimal64()) + return createDecimalDeserializeFn(root_node, target_type); + if (target.isDecimal128()) + return createDecimalDeserializeFn(root_node, target_type); + if (target.isDecimal256()) + return createDecimalDeserializeFn(root_node, target_type); break; case avro::AVRO_INT: if (target_type->isValueRepresentedByNumber()) @@ -526,7 +571,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node target_type->getName(), avro::toString(root_node->type()), nodeToJson(root_node)); } -AvroDeserializer::SkipFn AvroDeserializer::createSkipFn(avro::NodePtr root_node) +AvroDeserializer::SkipFn AvroDeserializer::createSkipFn(const avro::NodePtr & root_node) { switch (root_node->type()) { @@ -1042,19 +1087,40 @@ DataTypePtr AvroSchemaReader::avroNodeToDataType(avro::NodePtr node) switch (node->type()) { case avro::Type::AVRO_INT: + { + if (node->logicalType().type() == avro::LogicalType::DATE) + return {std::make_shared()}; + return {std::make_shared()}; + } case avro::Type::AVRO_LONG: + { + auto logical_type = node->logicalType(); + if (logical_type.type() == avro::LogicalType::TIMESTAMP_MILLIS) + return {std::make_shared(3)}; + else if (logical_type.type() == avro::LogicalType::TIMESTAMP_MICROS) + return {std::make_shared(6)}; + return std::make_shared(); + } case avro::Type::AVRO_BOOL: return DataTypeFactory::instance().get("Bool"); case avro::Type::AVRO_FLOAT: return std::make_shared(); case avro::Type::AVRO_DOUBLE: return std::make_shared(); - case avro::Type::AVRO_STRING: - return std::make_shared(); + case avro::Type::AVRO_STRING: [[fallthrough]]; case avro::Type::AVRO_BYTES: + { + auto logical_type = node->logicalType(); + if (logical_type.type() == avro::LogicalType::UUID) + return std::make_shared(); + + if (logical_type.type() == avro::LogicalType::DECIMAL) + return createDecimal(logical_type.precision(), logical_type.scale()); + return std::make_shared(); + } case avro::Type::AVRO_ENUM: { if (node->names() < 128) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.h b/src/Processors/Formats/Impl/AvroRowInputFormat.h index 25589880c14..ccadb431fa2 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.h +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.h @@ -51,13 +51,13 @@ public: AvroDeserializer(const Block & header, avro::ValidSchema schema, bool allow_missing_fields, bool null_as_default_); void deserializeRow(MutableColumns & columns, avro::Decoder & decoder, RowReadExtension & ext) const; -private: using DeserializeFn = std::function; using DeserializeNestedFn = std::function; +private: using SkipFn = std::function; - DeserializeFn createDeserializeFn(avro::NodePtr root_node, DataTypePtr target_type); - SkipFn createSkipFn(avro::NodePtr root_node); + DeserializeFn createDeserializeFn(const avro::NodePtr & root_node, const DataTypePtr & target_type); + SkipFn createSkipFn(const avro::NodePtr & root_node); struct Action { @@ -86,14 +86,14 @@ private: : type(Skip) , skip_fn(skip_fn_) {} - Action(std::vector nested_column_indexes_, std::vector nested_deserializers_) + Action(const std::vector & nested_column_indexes_, const std::vector & nested_deserializers_) : type(Nested) , nested_column_indexes(nested_column_indexes_) , nested_deserializers(nested_deserializers_) {} - static Action recordAction(std::vector field_actions) { return Action(Type::Record, field_actions); } + static Action recordAction(const std::vector & field_actions) { return Action(Type::Record, field_actions); } - static Action unionAction(std::vector branch_actions) { return Action(Type::Union, branch_actions); } + static Action unionAction(const std::vector & branch_actions) { return Action(Type::Union, branch_actions); } void execute(MutableColumns & columns, avro::Decoder & decoder, RowReadExtension & ext) const diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp index 1ca56a1c5cc..f1b42147cd6 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -88,8 +89,30 @@ private: WriteBuffer & out; }; +namespace +{ -AvroSerializer::SchemaWithSerializeFn AvroSerializer::createSchemaWithSerializeFn(DataTypePtr data_type, size_t & type_name_increment, const String & column_name) +template +AvroSerializer::SchemaWithSerializeFn createDecimalSchemaWithSerializeFn(const DataTypePtr & data_type) +{ + auto schema = avro::BytesSchema(); + const auto & provided_type = assert_cast(*data_type); + auto logical_type = avro::LogicalType(avro::LogicalType::DECIMAL); + logical_type.setScale(provided_type.getScale()); + logical_type.setPrecision(provided_type.getPrecision()); + schema.root()->setLogicalType(logical_type); + return {schema, [](const IColumn & column, size_t row_num, avro::Encoder & encoder) + { + const auto & col = assert_cast(column); + WriteBufferFromOwnString buf; + writeBinaryBigEndian(col.getElement(row_num).value, buf); + encoder.encodeBytes(reinterpret_cast(buf.str().data()), buf.str().size()); + }}; +} + +} + +AvroSerializer::SchemaWithSerializeFn AvroSerializer::createSchemaWithSerializeFn(const DataTypePtr & data_type, size_t & type_name_increment, const String & column_name) { ++type_name_increment; @@ -167,6 +190,16 @@ AvroSerializer::SchemaWithSerializeFn AvroSerializer::createSchemaWithSerializeF encoder.encodeInt(date); }}; } + case TypeIndex::Date32: + { + auto schema = avro::IntSchema(); + schema.root()->setLogicalType(avro::LogicalType(avro::LogicalType::DATE)); + return {schema, [](const IColumn & column, size_t row_num, avro::Encoder & encoder) + { + Int32 date = assert_cast(column).getElement(row_num); + encoder.encodeInt(date); + }}; + } case TypeIndex::DateTime64: { auto schema = avro::LongSchema(); @@ -185,6 +218,22 @@ AvroSerializer::SchemaWithSerializeFn AvroSerializer::createSchemaWithSerializeF encoder.encodeLong(col.getElement(row_num)); }}; } + case TypeIndex::Decimal32: + { + return createDecimalSchemaWithSerializeFn(data_type); + } + case TypeIndex::Decimal64: + { + return createDecimalSchemaWithSerializeFn(data_type); + } + case TypeIndex::Decimal128: + { + return createDecimalSchemaWithSerializeFn(data_type); + } + case TypeIndex::Decimal256: + { + return createDecimalSchemaWithSerializeFn(data_type); + } case TypeIndex::String: if (traits->isStringAsString(column_name)) return {avro::StringSchema(), [](const IColumn & column, size_t row_num, avro::Encoder & encoder) diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.h b/src/Processors/Formats/Impl/AvroRowOutputFormat.h index d7b15a95d26..fdcf97e14ba 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.h +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.h @@ -27,7 +27,6 @@ public: const avro::ValidSchema & getSchema() const { return valid_schema; } void serializeRow(const Columns & columns, size_t row_num, avro::Encoder & encoder); -private: using SerializeFn = std::function; struct SchemaWithSerializeFn { @@ -35,8 +34,9 @@ private: SerializeFn serialize; }; +private: /// Type names for different complex types (e.g. enums, fixed strings) must be unique. We use simple incremental number to give them different names. - SchemaWithSerializeFn createSchemaWithSerializeFn(DataTypePtr data_type, size_t & type_name_increment, const String & column_name); + SchemaWithSerializeFn createSchemaWithSerializeFn(const DataTypePtr & data_type, size_t & type_name_increment, const String & column_name); std::vector serialize_fns; avro::ValidSchema valid_schema; From e73026672fc961ebc823d7f35a4f52505647af15 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 10 Mar 2023 15:48:33 +0000 Subject: [PATCH 123/559] Add tests --- .../02588_avro_date32_and_decimals.reference | 5 +++++ .../0_stateless/02588_avro_date32_and_decimals.sh | 14 ++++++++++++++ 2 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/02588_avro_date32_and_decimals.reference create mode 100755 tests/queries/0_stateless/02588_avro_date32_and_decimals.sh diff --git a/tests/queries/0_stateless/02588_avro_date32_and_decimals.reference b/tests/queries/0_stateless/02588_avro_date32_and_decimals.reference new file mode 100644 index 00000000000..3a2d4fcf4f7 --- /dev/null +++ b/tests/queries/0_stateless/02588_avro_date32_and_decimals.reference @@ -0,0 +1,5 @@ +Date32 1942-08-16 +Decimal(9, 4) 4242.4242 +Decimal(18, 14) 4242.4242 +Decimal(38, 34) 4242.4242 +Decimal(76, 64) 4242.4242 diff --git a/tests/queries/0_stateless/02588_avro_date32_and_decimals.sh b/tests/queries/0_stateless/02588_avro_date32_and_decimals.sh new file mode 100755 index 00000000000..57363b85f0e --- /dev/null +++ b/tests/queries/0_stateless/02588_avro_date32_and_decimals.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "select toInt32(-10000)::Date32 as d format Avro" | $CLICKHOUSE_LOCAL --input-format=Avro -q "select toTypeName(d), d from table" + +$CLICKHOUSE_LOCAL -q "select 4242.4242::Decimal32(4) as d format Avro" | $CLICKHOUSE_LOCAL --input-format=Avro -q "select toTypeName(d), d from table" +$CLICKHOUSE_LOCAL -q "select 4242.4242::Decimal64(14) as d format Avro" | $CLICKHOUSE_LOCAL --input-format=Avro -q "select toTypeName(d), d from table" +$CLICKHOUSE_LOCAL -q "select 4242.4242::Decimal128(34) as d format Avro" | $CLICKHOUSE_LOCAL --input-format=Avro -q "select toTypeName(d), d from table" +$CLICKHOUSE_LOCAL -q "select 4242.4242::Decimal256(64) as d format Avro" | $CLICKHOUSE_LOCAL --input-format=Avro -q "select toTypeName(d), d from table" + From 6544e7c3eebb4dfe5ec70d536eb5c5b7ef2114e2 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 10 Mar 2023 17:11:27 +0100 Subject: [PATCH 124/559] Fix typos --- docs/en/interfaces/formats.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 23581d7bb62..ecca6eeb9af 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1825,8 +1825,8 @@ The table below shows supported data types and how they match ClickHouse [data t | `long (timestamp-micros)` \** | [DateTime64(6)](/docs/en/sql-reference/data-types/datetime.md) | `long (timestamp-micros)` \** | | `int` | [IPv4](/docs/en/sql-reference/data-types/domains/ipv4.md) | `int` | | `fixed(16)` | [IPv6](/docs/en/sql-reference/data-types/domains/ipv6.md) | `fixed(16)` | -| `bytes (decimal)` \** | [Decimal(P, S)](/docs/en/sql-reference/data-types/decimal.mdd) | `bytes (decimal)` \** | -| `string (uuid)` \** | [UUID](/docs/en/sql-reference/data-types/uuid.mdd) | `string (uuid)` \** | +| `bytes (decimal)` \** | [Decimal(P, S)](/docs/en/sql-reference/data-types/decimal.md) | `bytes (decimal)` \** | +| `string (uuid)` \** | [UUID](/docs/en/sql-reference/data-types/uuid.md) | `string (uuid)` \** | \* `bytes` is default, controlled by [output_format_avro_string_column_pattern](/docs/en/operations/settings/settings-formats.md/#output_format_avro_string_column_pattern) From a5628a19055c09448ba31147d8b6234e53830158 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 10 Mar 2023 17:11:48 +0100 Subject: [PATCH 125/559] Fix typo --- docs/en/interfaces/schema-inference.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/schema-inference.md b/docs/en/interfaces/schema-inference.md index 4b9d37bcc7d..c448d0aee47 100644 --- a/docs/en/interfaces/schema-inference.md +++ b/docs/en/interfaces/schema-inference.md @@ -1484,7 +1484,7 @@ In Avro format ClickHouse reads its schema from the data and converts it to Clic | `union(null, T)`, `union(T, null)` | [Nullable(T)](../sql-reference/data-types/date.md) | | `null` | [Nullable(Nothing)](../sql-reference/data-types/special-data-types/nothing.md) | | `string (uuid)` \* | [UUID](../sql-reference/data-types/uuid.md) | -| `binary (decimal)` \* | [Decimal(P, S)](../sql-reference/data-typesdecimal.md) | +| `binary (decimal)` \* | [Decimal(P, S)](../sql-reference/data-types/decimal.md) | \* [Avro logical types](https://avro.apache.org/docs/current/spec.html#Logical+Types) From 6ba79c2737134eb0d50ae294a909b3b7dc900ab8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sun, 12 Mar 2023 16:05:24 +0000 Subject: [PATCH 126/559] better support of identifiers from compound expressions --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 13 ++-- src/Analyzer/Utils.cpp | 24 -------- src/Analyzer/Utils.h | 8 --- src/DataTypes/IDataType.cpp | 14 ++--- src/DataTypes/IDataType.h | 14 ++--- src/Functions/getSubcolumn.cpp | 59 +++++++++++++++++++ ...77_analyzer_compound_expressions.reference | 17 ++++++ .../02677_analyzer_compound_expressions.sql | 44 ++++++++++++++ 8 files changed, 143 insertions(+), 50 deletions(-) create mode 100644 src/Functions/getSubcolumn.cpp create mode 100644 tests/queries/0_stateless/02677_analyzer_compound_expressions.reference create mode 100644 tests/queries/0_stateless/02677_analyzer_compound_expressions.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 34c03a9ffb6..b12fd19a961 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2378,7 +2378,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromCompoundExpression(const auto expression_type = compound_expression->getResultType(); - if (!nestedIdentifierCanBeResolved(expression_type, nested_path)) + if (!expression_type->hasSubcolumn(nested_path.getFullName())) { std::unordered_set valid_identifiers; collectCompoundExpressionValidIdentifiersForTypoCorrection(expression_identifier, @@ -2405,10 +2405,15 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromCompoundExpression(const getHintsErrorMessageSuffix(hints)); } - auto tuple_element_result = wrapExpressionNodeInTupleElement(compound_expression, nested_path); - resolveFunction(tuple_element_result, scope); + QueryTreeNodePtr get_subcolumn_function = std::make_shared("getSubcolumn"); + auto & get_subcolumn_function_arguments_nodes = get_subcolumn_function->as()->getArguments().getNodes(); - return tuple_element_result; + get_subcolumn_function_arguments_nodes.reserve(2); + get_subcolumn_function_arguments_nodes.push_back(compound_expression); + get_subcolumn_function_arguments_nodes.push_back(std::make_shared(nested_path.getFullName())); + + resolveFunction(get_subcolumn_function, scope); + return get_subcolumn_function; } /** Resolve identifier from expression arguments. diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index c5a5c042cbc..7eb20b3368c 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -380,30 +380,6 @@ QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & join_tree_nod return result; } -bool nestedIdentifierCanBeResolved(const DataTypePtr & compound_type, IdentifierView nested_identifier) -{ - const IDataType * current_type = compound_type.get(); - - for (const auto & identifier_part : nested_identifier) - { - while (const DataTypeArray * array = checkAndGetDataType(current_type)) - current_type = array->getNestedType().get(); - - const DataTypeTuple * tuple = checkAndGetDataType(current_type); - - if (!tuple) - return false; - - auto position = tuple->tryGetPositionByName(identifier_part); - if (!position) - return false; - - current_type = tuple->getElements()[*position].get(); - } - - return true; -} - namespace { diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index 3e2d95c6012..59ca89c6b14 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -45,14 +45,6 @@ QueryTreeNodePtr extractLeftTableExpression(const QueryTreeNodePtr & join_tree_n */ QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & join_tree_node); -/** Returns true if nested identifier can be resolved from compound type. - * Compound type can be tuple or array of tuples. - * - * Example: Compound type: Tuple(nested_path Tuple(nested_path_2 UInt64)). Nested identifier: nested_path_1.nested_path_2. - * Result: true. - */ -bool nestedIdentifierCanBeResolved(const DataTypePtr & compound_type, IdentifierView nested_identifier); - /** Assert that there are no function nodes with specified function name in node children. * Do not visit subqueries. */ diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index e0612fbbf36..4ffe82039b2 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -102,7 +102,7 @@ void IDataType::forEachSubcolumn( template Ptr IDataType::getForSubcolumn( - const String & subcolumn_name, + std::string_view subcolumn_name, const SubstreamData & data, Ptr SubstreamData::*member, bool throw_if_null) const @@ -120,36 +120,36 @@ Ptr IDataType::getForSubcolumn( return res; } -bool IDataType::hasSubcolumn(const String & subcolumn_name) const +bool IDataType::hasSubcolumn(std::string_view subcolumn_name) const { return tryGetSubcolumnType(subcolumn_name) != nullptr; } -DataTypePtr IDataType::tryGetSubcolumnType(const String & subcolumn_name) const +DataTypePtr IDataType::tryGetSubcolumnType(std::string_view subcolumn_name) const { auto data = SubstreamData(getDefaultSerialization()).withType(getPtr()); return getForSubcolumn(subcolumn_name, data, &SubstreamData::type, false); } -DataTypePtr IDataType::getSubcolumnType(const String & subcolumn_name) const +DataTypePtr IDataType::getSubcolumnType(std::string_view subcolumn_name) const { auto data = SubstreamData(getDefaultSerialization()).withType(getPtr()); return getForSubcolumn(subcolumn_name, data, &SubstreamData::type, true); } -ColumnPtr IDataType::tryGetSubcolumn(const String & subcolumn_name, const ColumnPtr & column) const +ColumnPtr IDataType::tryGetSubcolumn(std::string_view subcolumn_name, const ColumnPtr & column) const { auto data = SubstreamData(getDefaultSerialization()).withColumn(column); return getForSubcolumn(subcolumn_name, data, &SubstreamData::column, false); } -ColumnPtr IDataType::getSubcolumn(const String & subcolumn_name, const ColumnPtr & column) const +ColumnPtr IDataType::getSubcolumn(std::string_view subcolumn_name, const ColumnPtr & column) const { auto data = SubstreamData(getDefaultSerialization()).withColumn(column); return getForSubcolumn(subcolumn_name, data, &SubstreamData::column, true); } -SerializationPtr IDataType::getSubcolumnSerialization(const String & subcolumn_name, const SerializationPtr & serialization) const +SerializationPtr IDataType::getSubcolumnSerialization(std::string_view subcolumn_name, const SerializationPtr & serialization) const { auto data = SubstreamData(serialization); return getForSubcolumn(subcolumn_name, data, &SubstreamData::serialization, true); diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index e5bdbeca69e..2ffc30bb8ae 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -79,15 +79,15 @@ public: /// Data type id. It's used for runtime type checks. virtual TypeIndex getTypeId() const = 0; - bool hasSubcolumn(const String & subcolumn_name) const; + bool hasSubcolumn(std::string_view subcolumn_name) const; - DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const; - DataTypePtr getSubcolumnType(const String & subcolumn_name) const; + DataTypePtr tryGetSubcolumnType(std::string_view subcolumn_name) const; + DataTypePtr getSubcolumnType(std::string_view subcolumn_name) const; - ColumnPtr tryGetSubcolumn(const String & subcolumn_name, const ColumnPtr & column) const; - ColumnPtr getSubcolumn(const String & subcolumn_name, const ColumnPtr & column) const; + ColumnPtr tryGetSubcolumn(std::string_view subcolumn_name, const ColumnPtr & column) const; + ColumnPtr getSubcolumn(std::string_view subcolumn_name, const ColumnPtr & column) const; - SerializationPtr getSubcolumnSerialization(const String & subcolumn_name, const SerializationPtr & serialization) const; + SerializationPtr getSubcolumnSerialization(std::string_view subcolumn_name, const SerializationPtr & serialization) const; using SubstreamData = ISerialization::SubstreamData; using SubstreamPath = ISerialization::SubstreamPath; @@ -315,7 +315,7 @@ public: private: template Ptr getForSubcolumn( - const String & subcolumn_name, + std::string_view subcolumn_name, const SubstreamData & data, Ptr SubstreamData::*member, bool throw_if_null) const; diff --git a/src/Functions/getSubcolumn.cpp b/src/Functions/getSubcolumn.cpp new file mode 100644 index 00000000000..268e5af79be --- /dev/null +++ b/src/Functions/getSubcolumn.cpp @@ -0,0 +1,59 @@ +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + +namespace +{ + +class FunctionGetSubcolumn : public IFunction +{ +public: + static constexpr auto name = "getSubcolumn"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 2; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + auto subcolumn_name = getSubcolumnName(arguments); + return arguments[0].type->getSubcolumnType(subcolumn_name); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + { + auto subcolumn_name = getSubcolumnName(arguments); + return arguments[0].type->getSubcolumn(subcolumn_name, arguments[0].column); + } + +private: + std::string_view getSubcolumnName(const ColumnsWithTypeAndName & arguments) const + { + const auto * column = arguments[1].column.get(); + if (!isString(arguments[1].type) || !column || !checkAndGetColumnConstStringOrFixedString(column)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "The second argument of function {} should be a constant string with the name of a subcolumn", name); + + return column->getDataAt(0).toView(); + } +}; + +} + +REGISTER_FUNCTION(GetSubcolumn) +{ + factory.registerFunction(); +} + +} diff --git a/tests/queries/0_stateless/02677_analyzer_compound_expressions.reference b/tests/queries/0_stateless/02677_analyzer_compound_expressions.reference new file mode 100644 index 00000000000..b630a020fb9 --- /dev/null +++ b/tests/queries/0_stateless/02677_analyzer_compound_expressions.reference @@ -0,0 +1,17 @@ +a b +a b +1 a b +1 a b +3 +3 +[0,0,1] +[0,0,1] +3 [2,0,1] +3 [2,0,1] +['foo','bar'] [1,2] +['foo','bar'] [1,2] +['foo','bar'] [(1,2),(3,4)] [1,3] [2,4] +['foo','bar'] [(1,2),(3,4)] [1,3] [2,4] +['foo','bar'] [(1,2),(3,4)] [1,3] [2,4] +['foo','bar'] [(1,2),(3,4)] [1,3] [2,4] +3 diff --git a/tests/queries/0_stateless/02677_analyzer_compound_expressions.sql b/tests/queries/0_stateless/02677_analyzer_compound_expressions.sql new file mode 100644 index 00000000000..50df99bef57 --- /dev/null +++ b/tests/queries/0_stateless/02677_analyzer_compound_expressions.sql @@ -0,0 +1,44 @@ +SET allow_experimental_analyzer = 1; + +WITH ('a', 'b')::Tuple(c1 String, c2 String) AS t +SELECT t.c1, t.c2; + +WITH materialize(('a', 'b')::Tuple(c1 String, c2 String)) AS t +SELECT t.c1, t.c2; + +WITH (1, ('a', 'b'))::Tuple(c1 UInt64, t1 Tuple(c1 String, c2 String)) AS t +SELECT t.c1, t.t1.c1, t.t1.c2; + +WITH materialize((1, ('a', 'b'))::Tuple(c1 UInt64, t1 Tuple(c1 String, c2 String))) AS t +SELECT t.c1, t.t1.c1, t.t1.c2; + +WITH [1, 2, 3] AS arr SELECT arr.size0; +WITH materialize([1, 2, 3]) AS arr SELECT arr.size0; + +WITH [1, 2, NULL] AS arr SELECT arr.null; +WITH materialize([1, 2, NULL]) AS arr SELECT arr.null; + +WITH [[1, 2], [], [3]] AS arr SELECT arr.size0, arr.size1; +WITH materialize([[1, 2], [], [3]]) AS arr SELECT arr.size0, arr.size1; + +WITH map('foo', 1, 'bar', 2) AS m SELECT m.keys, m.values; +WITH materialize(map('foo', 1, 'bar', 2)) AS m SELECT m.keys, m.values; + +WITH map('foo', (1, 2), 'bar', (3, 4))::Map(String, Tuple(a UInt64, b UInt64)) AS m +SELECT m.keys, m.values, m.values.a, m.values.b; + +WITH materialize(map('foo', (1, 2), 'bar', (3, 4))::Map(String, Tuple(a UInt64, b UInt64))) AS m +SELECT m.keys, m.values, m.values.a, m.values.b; + +WITH map('foo', (1, 2), 'bar', (3, 4))::Map(String, Tuple(a UInt64, b UInt64)) AS m +SELECT m.keys, m.values, m.values.*; + +WITH materialize(map('foo', (1, 2), 'bar', (3, 4))::Map(String, Tuple(a UInt64, b UInt64))) AS m +SELECT m.keys, m.values, m.values.*; + +WITH [1, 2, 3] AS arr SELECT arr.*; -- { serverError UNSUPPORTED_METHOD } +WITH map('foo', 1, 'bar', 2) AS m SELECT m.*; -- { serverError UNSUPPORTED_METHOD } + +SELECT getSubcolumn([1, 2, 3], 'size0'); +SELECT getSubcolumn([1, 2, 3], materialize('size0')); -- { serverError ILLEGAL_COLUMN } +SELECT getSubcolumn([1, 2, 3], 'aaa'); -- { serverError ILLEGAL_COLUMN } From f33b6516862a53b5b18c0a9876b0a1af355bb45f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 13 Mar 2023 04:51:50 +0100 Subject: [PATCH 127/559] Add fuzzer for data formats --- base/base/coverage.cpp | 2 + src/CMakeLists.txt | 1 + src/Core/ColumnWithTypeAndName.h | 3 - src/Core/NamesAndTypes.h | 1 - src/Formats/CMakeLists.txt | 3 + src/Formats/FormatFactory.cpp | 16 ++-- src/Formats/FormatFactory.h | 1 + src/Formats/fuzzers/CMakeLists.txt | 2 + src/Formats/fuzzers/format_fuzzer.cpp | 87 +++++++++++++++++++ .../fuzzers/execute_query_fuzzer.cpp | 1 - 10 files changed, 102 insertions(+), 15 deletions(-) create mode 100644 src/Formats/CMakeLists.txt create mode 100644 src/Formats/fuzzers/CMakeLists.txt create mode 100644 src/Formats/fuzzers/format_fuzzer.cpp diff --git a/base/base/coverage.cpp b/base/base/coverage.cpp index 043f97f9593..1027638be3d 100644 --- a/base/base/coverage.cpp +++ b/base/base/coverage.cpp @@ -2,6 +2,8 @@ #if WITH_COVERAGE +#pragma GCC diagnostic ignored "-Wreserved-identifier" + # include # include diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 6c5142813c5..edf638c5350 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -75,6 +75,7 @@ add_subdirectory (Coordination) add_subdirectory (Bridge) add_subdirectory (Daemon) add_subdirectory (Loggers) +add_subdirectory (Formats) set(dbms_headers) diff --git a/src/Core/ColumnWithTypeAndName.h b/src/Core/ColumnWithTypeAndName.h index 592ad39d55a..15807066e62 100644 --- a/src/Core/ColumnWithTypeAndName.h +++ b/src/Core/ColumnWithTypeAndName.h @@ -14,8 +14,6 @@ class WriteBuffer; * Column data could be nullptr - to represent just 'header' of column. * Name could be either name from a table or some temporary generated name during expression evaluation. */ -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wnull-dereference" struct ColumnWithTypeAndName { ColumnPtr column; @@ -37,6 +35,5 @@ struct ColumnWithTypeAndName void dumpStructure(WriteBuffer & out) const; String dumpStructure() const; }; -#pragma GCC diagnostic pop } diff --git a/src/Core/NamesAndTypes.h b/src/Core/NamesAndTypes.h index 78535a751c3..6cada7c8a69 100644 --- a/src/Core/NamesAndTypes.h +++ b/src/Core/NamesAndTypes.h @@ -83,7 +83,6 @@ public: template NamesAndTypesList(Iterator begin, Iterator end) : std::list(begin, end) {} - void readText(ReadBuffer & buf); void writeText(WriteBuffer & buf) const; diff --git a/src/Formats/CMakeLists.txt b/src/Formats/CMakeLists.txt new file mode 100644 index 00000000000..316043093c0 --- /dev/null +++ b/src/Formats/CMakeLists.txt @@ -0,0 +1,3 @@ +if (ENABLE_FUZZING) + add_subdirectory(fuzzers) +endif() diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index a951a7fdd92..83663876b55 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -226,9 +226,7 @@ InputFormatPtr FormatFactory::getInput( ? *_format_settings : getFormatSettings(context); if (!getCreators(name).input_creator) - { throw Exception(ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT, "Format {} is not suitable for input", name); - } const Settings & settings = context->getSettingsRef(); const auto & file_segmentation_engine = getCreators(name).file_segmentation_engine; @@ -270,21 +268,19 @@ InputFormatPtr FormatFactory::getInput( ParallelParsingInputFormat::Params params{ buf, sample, parser_creator, file_segmentation_engine, name, settings.max_threads, settings.min_chunk_bytes_for_parallel_parsing, max_block_size, context->getApplicationType() == Context::ApplicationType::SERVER}; + auto format = std::make_shared(params); if (!settings.input_format_record_errors_file_path.toString().empty()) - { format->setErrorsLogger(std::make_shared(context)); - } return format; } - - - auto format = getInputFormat(name, buf, sample, context, max_block_size, format_settings); - if (!settings.input_format_record_errors_file_path.toString().empty()) + else { - format->setErrorsLogger(std::make_shared(context)); + auto format = getInputFormat(name, buf, sample, context, max_block_size, format_settings); + if (!settings.input_format_record_errors_file_path.toString().empty()) + format->setErrorsLogger(std::make_shared(context)); + return format; } - return format; } InputFormatPtr FormatFactory::getInputFormat( diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 398548e4b22..6697a22984c 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -15,6 +15,7 @@ #include #include + namespace DB { diff --git a/src/Formats/fuzzers/CMakeLists.txt b/src/Formats/fuzzers/CMakeLists.txt new file mode 100644 index 00000000000..984823f3360 --- /dev/null +++ b/src/Formats/fuzzers/CMakeLists.txt @@ -0,0 +1,2 @@ +clickhouse_add_executable(format_fuzzer format_fuzzer.cpp ${SRCS}) +target_link_libraries(format_fuzzer PRIVATE dbms clickhouse_aggregate_functions ${LIB_FUZZING_ENGINE}) diff --git a/src/Formats/fuzzers/format_fuzzer.cpp b/src/Formats/fuzzers/format_fuzzer.cpp new file mode 100644 index 00000000000..8c0c01c7c0a --- /dev/null +++ b/src/Formats/fuzzers/format_fuzzer.cpp @@ -0,0 +1,87 @@ +#include + +#include +#include + +#include +#include + +#include +#include + +#include +#include + +#include +#include + +#include + + +extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) +try +{ + using namespace DB; + + static SharedContextHolder shared_context; + static ContextMutablePtr context; + + auto initialize = [&]() mutable + { + shared_context = Context::createShared(); + context = Context::createGlobal(shared_context.get()); + context->makeGlobalContext(); + context->setApplicationType(Context::ApplicationType::LOCAL); + + registerAggregateFunctions(); + registerFormats(); + + return true; + }; + + static bool initialized = initialize(); + (void) initialized; + + /// The input format is as follows: + /// - format name on the first line, + /// - table structure on the second line, + /// - the data for the rest of the input. + + DB::ReadBufferFromMemory in(data, size); + + String format; + readStringUntilNewlineInto(format, in); + assertChar('\n', in); + + String structure; + readStringUntilNewlineInto(structure, in); + assertChar('\n', in); + + ColumnsDescription description; + parseColumnsListFromString(structure, context); + auto columns_info = description.getOrdinary(); + + Block header; + for (auto & info : columns_info) + { + ColumnWithTypeAndName column; + column.name = info.name; + column.type = info.type; + column.column = column.type->createColumn(); + header.insert(std::move(column)); + } + + InputFormatPtr input_format = context->getInputFormat(format, in, header, 13 /* small block size */); + + QueryPipeline pipeline(Pipe(std::move(input_format))); + PullingPipelineExecutor executor(pipeline); + Block res; + while (executor.pull(res)) + ; + + return 0; +} +catch (...) +{ + return 1; +} diff --git a/src/Interpreters/fuzzers/execute_query_fuzzer.cpp b/src/Interpreters/fuzzers/execute_query_fuzzer.cpp index 30db25668cf..284e780ed1f 100644 --- a/src/Interpreters/fuzzers/execute_query_fuzzer.cpp +++ b/src/Interpreters/fuzzers/execute_query_fuzzer.cpp @@ -1,4 +1,3 @@ -#include #include #include #include "Processors/Executors/PullingPipelineExecutor.h" From 43b938d303efed21c9053ad6275cf23cd3c1f645 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 13 Mar 2023 05:21:48 +0100 Subject: [PATCH 128/559] Update the fuzzer --- src/Formats/fuzzers/format_fuzzer.cpp | 28 +++++++++++++++++-- .../parseColumnsListForTableFunction.h | 2 +- 2 files changed, 26 insertions(+), 4 deletions(-) diff --git a/src/Formats/fuzzers/format_fuzzer.cpp b/src/Formats/fuzzers/format_fuzzer.cpp index 8c0c01c7c0a..aabe8ed966a 100644 --- a/src/Formats/fuzzers/format_fuzzer.cpp +++ b/src/Formats/fuzzers/format_fuzzer.cpp @@ -47,6 +47,29 @@ try /// - table structure on the second line, /// - the data for the rest of the input. + /** The corpus was generated as follows: + + i=0; find ../../../../tests/queries -name '*.sql' | + xargs -I{} bash -c "tr '\n' ' ' <{}; echo" | + rg -o -i 'CREATE TABLE\s+\w+\s+\(.+?\) ENGINE' | + sed -r -e 's/CREATE TABLE\s+\w+\s+\((.+?)\) ENGINE/\1/i' | sort | uniq | + while read line; do + i=$((i+1)); + clickhouse-local --query "SELECT name FROM system.formats ORDER BY rand() LIMIT 1" >> $i; + echo "$line" >> $i; + echo $RANDOM >> $i; + echo $i; + done + */ + + /// Compile the code as follows: + /// mkdir build_asan_fuzz + /// cd build_asan_fuzz + /// CC=clang CXX=clang++ cmake -D SANITIZE=address -D ENABLE_FUZZING=1 -D WITH_COVERAGE=1 .. + /// + /// The fuzzer can be run as follows: + /// ../../../build_asan_fuzz/src/Formats/fuzzers/format_fuzzer corpus -jobs=64 + DB::ReadBufferFromMemory in(data, size); String format; @@ -57,12 +80,11 @@ try readStringUntilNewlineInto(structure, in); assertChar('\n', in); - ColumnsDescription description; - parseColumnsListFromString(structure, context); + ColumnsDescription description = parseColumnsListFromString(structure, context); auto columns_info = description.getOrdinary(); Block header; - for (auto & info : columns_info) + for (const auto & info : columns_info) { ColumnWithTypeAndName column; column.name = info.name; diff --git a/src/Interpreters/parseColumnsListForTableFunction.h b/src/Interpreters/parseColumnsListForTableFunction.h index 212c378d3b5..14119ab55da 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.h +++ b/src/Interpreters/parseColumnsListForTableFunction.h @@ -31,7 +31,7 @@ struct DataTypeValidationSettings void validateDataType(const DataTypePtr & type, const DataTypeValidationSettings & settings); /// Parses a common argument for table functions such as table structure given in string -ColumnsDescription parseColumnsListFromString(const std::string & structure, const ContextPtr & context); +[[nodiscard]] ColumnsDescription parseColumnsListFromString(const std::string & structure, const ContextPtr & context); bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescription & columns, const ContextPtr & context, String & error); From 02f7ef472398f976e913fcd71369c40ab9e2ab86 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 13 Mar 2023 05:28:06 +0100 Subject: [PATCH 129/559] Update comment --- src/Formats/fuzzers/format_fuzzer.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Formats/fuzzers/format_fuzzer.cpp b/src/Formats/fuzzers/format_fuzzer.cpp index aabe8ed966a..5e66fef0de8 100644 --- a/src/Formats/fuzzers/format_fuzzer.cpp +++ b/src/Formats/fuzzers/format_fuzzer.cpp @@ -67,6 +67,9 @@ try /// cd build_asan_fuzz /// CC=clang CXX=clang++ cmake -D SANITIZE=address -D ENABLE_FUZZING=1 -D WITH_COVERAGE=1 .. /// + /// The corpus is located here: + /// https://github.com/ClickHouse/fuzz-corpus/tree/main/format_fuzzer + /// /// The fuzzer can be run as follows: /// ../../../build_asan_fuzz/src/Formats/fuzzers/format_fuzzer corpus -jobs=64 From 96b4cccac42dcf03ebbba933efafb074698c5d1d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 13 Mar 2023 05:39:35 +0100 Subject: [PATCH 130/559] Add a test --- tests/queries/0_stateless/02680_default_star.reference | 0 tests/queries/0_stateless/02680_default_star.sql | 6 ++++++ 2 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/02680_default_star.reference create mode 100644 tests/queries/0_stateless/02680_default_star.sql diff --git a/tests/queries/0_stateless/02680_default_star.reference b/tests/queries/0_stateless/02680_default_star.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02680_default_star.sql b/tests/queries/0_stateless/02680_default_star.sql new file mode 100644 index 00000000000..d560bd01e41 --- /dev/null +++ b/tests/queries/0_stateless/02680_default_star.sql @@ -0,0 +1,6 @@ +-- These queries yield syntax error, not logical error. + +CREATE TEMPORARY TABLE test (ad DEFAULT *); -- { clientError SYNTAX_ERROR } +CREATE TEMPORARY TABLE test (ad INT DEFAULT *); -- { clientError SYNTAX_ERROR } +CREATE TEMPORARY TABLE test (ad DEFAULT * NOT NULL); -- { clientError SYNTAX_ERROR } +CREATE TEMPORARY TABLE test (ad DEFAULT t.* NOT NULL); -- { clientError SYNTAX_ERROR } From 2e3aa985353562bd28c98f5e06458bd2d978283e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 13 Mar 2023 06:46:19 +0100 Subject: [PATCH 131/559] Add a test, fix logical errors --- .../AggregateFunctionGroupArray.h | 2 ++ .../AggregateFunctionGroupUniqArray.h | 1 - src/AggregateFunctions/AggregateFunctionMap.h | 8 ++------ .../AggregateFunctionSequenceNextNode.h | 5 ++++- src/AggregateFunctions/QuantileExact.h | 5 +++++ src/IO/ReadHelpers.h | 17 +++++++++++------ src/Parsers/ParserCreateQuery.h | 4 ++++ .../02681_group_array_too_large_size.reference | 0 .../02681_group_array_too_large_size.sql | 8 ++++++++ 9 files changed, 36 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/02681_group_array_too_large_size.reference create mode 100644 tests/queries/0_stateless/02681_group_array_too_large_size.sql diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.h b/src/AggregateFunctions/AggregateFunctionGroupArray.h index eaffb04e2a9..5a799dc3641 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.h +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.h @@ -366,6 +366,8 @@ struct GroupArrayNodeBase { UInt64 size; readVarUInt(size, buf); + if (unlikely(size > AGGREGATE_FUNCTION_GROUP_ARRAY_MAX_ARRAY_SIZE)) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size"); Node * node = reinterpret_cast(arena->alignedAlloc(sizeof(Node) + size, alignof(Node))); node->size = size; diff --git a/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h b/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h index f8e426363d8..bc7ccb08267 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h +++ b/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h @@ -181,7 +181,6 @@ public: auto & set = this->data(place).value; size_t size; readVarUInt(size, buf); - //TODO: set.reserve(size); for (size_t i = 0; i < size; ++i) set.insert(readStringBinaryInto(*arena, buf)); diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h index 91530698bf4..55f6611974e 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.h +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -61,15 +61,11 @@ struct AggregateFunctionMapCombinatorData static void writeKey(String key, WriteBuffer & buf) { - writeVarUInt(key.size(), buf); - writeString(key, buf); + writeStringBinary(key, buf); } static void readKey(String & key, ReadBuffer & buf) { - UInt64 size; - readVarUInt(size, buf); - key.resize(size); - buf.readStrict(key.data(), size); + readStringBinary(key, buf); } }; diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 76610772b22..4fd7db4160e 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -86,7 +86,7 @@ struct NodeBase { UInt64 size; readVarUInt(size, buf); - if unlikely (size > max_node_size_deserialize) + if (unlikely(size > max_node_size_deserialize)) throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large node state size"); Node * node = reinterpret_cast(arena->alignedAlloc(sizeof(Node) + size, alignof(Node))); @@ -323,6 +323,9 @@ public: if (unlikely(size == 0)) return; + if (unlikely(size > max_node_size_deserialize)) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size"); + auto & value = data(place).value; value.resize(size, arena); diff --git a/src/AggregateFunctions/QuantileExact.h b/src/AggregateFunctions/QuantileExact.h index b7af17b52bf..c67621a99ce 100644 --- a/src/AggregateFunctions/QuantileExact.h +++ b/src/AggregateFunctions/QuantileExact.h @@ -8,6 +8,8 @@ #include #include +#define QUANTILE_EXACT_MAX_ARRAY_SIZE 1'000'000'000 + namespace DB { @@ -17,6 +19,7 @@ namespace ErrorCodes { extern const int NOT_IMPLEMENTED; extern const int BAD_ARGUMENTS; + extern const int TOO_LARGE_ARRAY_SIZE; } @@ -54,6 +57,8 @@ struct QuantileExactBase { size_t size = 0; readVarUInt(size, buf); + if (unlikely(size > QUANTILE_EXACT_MAX_ARRAY_SIZE)) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size"); array.resize(size); buf.readStrict(reinterpret_cast(array.data()), size * sizeof(array[0])); } diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index f8931a7f622..cac42c198b1 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -61,6 +61,8 @@ namespace ErrorCodes extern const int CANNOT_READ_ARRAY_FROM_TEXT; extern const int CANNOT_PARSE_NUMBER; extern const int INCORRECT_DATA; + extern const int TOO_LARGE_STRING_SIZE; + extern const int TOO_LARGE_ARRAY_SIZE; } /// Helper functions for formatted input. @@ -128,13 +130,13 @@ inline void readFloatBinary(T & x, ReadBuffer & buf) readPODBinary(x, buf); } -inline void readStringBinary(std::string & s, ReadBuffer & buf, size_t MAX_STRING_SIZE = DEFAULT_MAX_STRING_SIZE) +inline void readStringBinary(std::string & s, ReadBuffer & buf, size_t max_string_size = DEFAULT_MAX_STRING_SIZE) { size_t size = 0; readVarUInt(size, buf); - if (size > MAX_STRING_SIZE) - throw Poco::Exception("Too large string size."); + if (size > max_string_size) + throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large string size."); s.resize(size); buf.readStrict(s.data(), size); @@ -146,6 +148,9 @@ inline StringRef readStringBinaryInto(Arena & arena, ReadBuffer & buf) size_t size = 0; readVarUInt(size, buf); + if (unlikely(size > DEFAULT_MAX_STRING_SIZE)) + throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large string size."); + char * data = arena.alloc(size); buf.readStrict(data, size); @@ -154,13 +159,13 @@ inline StringRef readStringBinaryInto(Arena & arena, ReadBuffer & buf) template -void readVectorBinary(std::vector & v, ReadBuffer & buf, size_t MAX_VECTOR_SIZE = DEFAULT_MAX_STRING_SIZE) +void readVectorBinary(std::vector & v, ReadBuffer & buf) { size_t size = 0; readVarUInt(size, buf); - if (size > MAX_VECTOR_SIZE) - throw Poco::Exception("Too large vector size."); + if (size > DEFAULT_MAX_STRING_SIZE) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size."); v.resize(size); for (size_t i = 0; i < size; ++i) diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index ef87988aab2..2489b108004 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -265,6 +265,10 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E } } + /// This will rule out unusual expressions like *, t.* that cannot appear in DEFAULT + if (default_expression && !dynamic_cast(default_expression.get())) + return false; + if (require_type && !type && !default_expression) return false; /// reject column name without type diff --git a/tests/queries/0_stateless/02681_group_array_too_large_size.reference b/tests/queries/0_stateless/02681_group_array_too_large_size.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02681_group_array_too_large_size.sql b/tests/queries/0_stateless/02681_group_array_too_large_size.sql new file mode 100644 index 00000000000..7b09f9b468e --- /dev/null +++ b/tests/queries/0_stateless/02681_group_array_too_large_size.sql @@ -0,0 +1,8 @@ +-- This query throw high-level exception instead of low-level "too large size passed to allocator": + +SELECT * FROM format(CSV, 'entitypArray AggregateFunction(groupArray, String)', +'295TMiews.viewN""""""TabSeparated +d St"" + + +r'); -- { serverError TOO_LARGE_ARRAY_SIZE } From 38d5635bf2d6fde2a094bd92407a6ef87304a912 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 13 Mar 2023 07:40:56 +0100 Subject: [PATCH 132/559] Fix too large size in quantiles --- src/AggregateFunctions/ReservoirSampler.h | 9 ++++++++- .../ReservoirSamplerDeterministic.h | 5 +++++ src/AggregateFunctions/fuzzers/CMakeLists.txt | 0 ...ate_function_state_deserialization_fuzzer.cpp | 0 .../02682_quantiles_too_large_size.reference | 0 .../02682_quantiles_too_large_size.sql | Bin 0 -> 1067 bytes 6 files changed, 13 insertions(+), 1 deletion(-) create mode 100644 src/AggregateFunctions/fuzzers/CMakeLists.txt create mode 100644 src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp create mode 100644 tests/queries/0_stateless/02682_quantiles_too_large_size.reference create mode 100644 tests/queries/0_stateless/02682_quantiles_too_large_size.sql diff --git a/src/AggregateFunctions/ReservoirSampler.h b/src/AggregateFunctions/ReservoirSampler.h index b59f75b377e..ef0e7c6566e 100644 --- a/src/AggregateFunctions/ReservoirSampler.h +++ b/src/AggregateFunctions/ReservoirSampler.h @@ -24,6 +24,7 @@ struct Settings; namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int TOO_LARGE_ARRAY_SIZE; } } @@ -208,7 +209,13 @@ public: { DB::readIntBinary(sample_count, buf); DB::readIntBinary(total_values, buf); - samples.resize(std::min(total_values, sample_count)); + + size_t size = std::min(total_values, sample_count); + static constexpr size_t MAX_RESERVOIR_SIZE = 1_GiB; + if (unlikely(size > MAX_RESERVOIR_SIZE)) + throw DB::Exception(DB::ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size"); + + samples.resize(size); std::string rng_string; DB::readStringBinary(rng_string, buf); diff --git a/src/AggregateFunctions/ReservoirSamplerDeterministic.h b/src/AggregateFunctions/ReservoirSamplerDeterministic.h index 17e4ce0e494..5e1d23ed2c2 100644 --- a/src/AggregateFunctions/ReservoirSamplerDeterministic.h +++ b/src/AggregateFunctions/ReservoirSamplerDeterministic.h @@ -22,6 +22,7 @@ struct Settings; namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int TOO_LARGE_ARRAY_SIZE; } } @@ -163,6 +164,10 @@ public: if (size > total_values) size = total_values; + static constexpr size_t MAX_RESERVOIR_SIZE = 1_GiB; + if (unlikely(size > MAX_RESERVOIR_SIZE)) + throw DB::Exception(DB::ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size"); + samples.resize(size); for (size_t i = 0; i < size; ++i) DB::readPODBinary(samples[i], buf); diff --git a/src/AggregateFunctions/fuzzers/CMakeLists.txt b/src/AggregateFunctions/fuzzers/CMakeLists.txt new file mode 100644 index 00000000000..e69de29bb2d diff --git a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02682_quantiles_too_large_size.reference b/tests/queries/0_stateless/02682_quantiles_too_large_size.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02682_quantiles_too_large_size.sql b/tests/queries/0_stateless/02682_quantiles_too_large_size.sql new file mode 100644 index 0000000000000000000000000000000000000000..fff98f667c792cffb9c3cef22c129d9e1b88fda4 GIT binary patch literal 1067 zcmb_aO>fgc5RDL*tdt)Br=dbhQrVT`I$v-ykVKK%2H8dkF3~ogI;(EH?(RBig7^vi zQ2qdCHi=ugO1J=X+1WSm%|6YJZ9_C?3hr{0YtVBkRxoGD+~&AM4tPJqOf!)JOO-|% zgSY`MRz>+l3jv(U1VOP4LPnLs7_igXWp1T&Ti(Rx}14AyTK6i=RFZC;2 z#Q|MCb|$pY6Sq^M#)0+o`VW;tU05S#61&E=TB~-8#zeQ*?U8!(E!Z}E1BG&lav-IU Y5KSi23qKeQrhX9kU#8*s>j1u!- Date: Mon, 13 Mar 2023 07:42:58 +0100 Subject: [PATCH 133/559] Update comment --- src/Formats/fuzzers/format_fuzzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/fuzzers/format_fuzzer.cpp b/src/Formats/fuzzers/format_fuzzer.cpp index 5e66fef0de8..81aaac88691 100644 --- a/src/Formats/fuzzers/format_fuzzer.cpp +++ b/src/Formats/fuzzers/format_fuzzer.cpp @@ -71,7 +71,7 @@ try /// https://github.com/ClickHouse/fuzz-corpus/tree/main/format_fuzzer /// /// The fuzzer can be run as follows: - /// ../../../build_asan_fuzz/src/Formats/fuzzers/format_fuzzer corpus -jobs=64 + /// ../../../build_asan_fuzz/src/Formats/fuzzers/format_fuzzer corpus -jobs=64 -rss_limit_mb=8192 DB::ReadBufferFromMemory in(data, size); From 7ab74dd862425a7db95f32adb029ae72879279c9 Mon Sep 17 00:00:00 2001 From: kolechenkov <60389882+kolechenkov@users.noreply.github.com> Date: Mon, 13 Mar 2023 10:27:48 +0300 Subject: [PATCH 134/559] Update gui.md Added a missed bracket --- docs/ru/interfaces/third-party/gui.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/interfaces/third-party/gui.md b/docs/ru/interfaces/third-party/gui.md index e7190362dc4..d06eada165b 100644 --- a/docs/ru/interfaces/third-party/gui.md +++ b/docs/ru/interfaces/third-party/gui.md @@ -187,7 +187,7 @@ sidebar_label: "Визуальные интерфейсы от сторонни DataLens [доступен бесплатно](https://cloud.yandex.ru/docs/datalens/pricing), в том числе и для коммерческого использования. -- [Знакомство с DataLens]((https://youtu.be/57ngi_6BINE). +- [Знакомство с DataLens]((https://youtu.be/57ngi_6BINE)). - [Чат сообщества DataLens](https://t.me/YandexDataLens) - [Документация DataLens](https://cloud.yandex.ru/docs/datalens/). - [Сценарий по визуализации данных из ClickHouse](https://cloud.yandex.ru/docs/solutions/datalens/data-from-ch-visualization). From c300f5e68fa130626d66f00d5c7a47f36d8b3617 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 12 Mar 2023 13:48:16 +0000 Subject: [PATCH 135/559] Cleanup: max_size --> max_size_in_bytes To disambiguate the threshold from the maximum number of cache entries. --- src/Common/CacheBase.h | 10 +++++----- src/Common/LRUCachePolicy.h | 31 ++++++++++++++++--------------- src/Common/SLRUCachePolicy.h | 30 +++++++++++++++--------------- 3 files changed, 36 insertions(+), 35 deletions(-) diff --git a/src/Common/CacheBase.h b/src/Common/CacheBase.h index 8145bdf95b5..fe0f2408f05 100644 --- a/src/Common/CacheBase.h +++ b/src/Common/CacheBase.h @@ -27,7 +27,7 @@ namespace ErrorCodes /// (default policy evicts entries which are not used for a long time). /// WeightFunction is a functor that takes Mapped as a parameter and returns "weight" (approximate size) /// of that value. -/// Cache starts to evict entries when their total weight exceeds max_size. +/// Cache starts to evict entries when their total weight exceeds max_size_in_bytes. /// Value weight should not change after insertion. template , typename WeightFunction = TrivialWeightFunction> class CacheBase @@ -37,7 +37,7 @@ public: using Mapped = TMapped; using MappedPtr = std::shared_ptr; - explicit CacheBase(size_t max_size, size_t max_elements_size = 0, String cache_policy_name = "", double size_ratio = 0.5) + explicit CacheBase(size_t max_size_in_bytes, size_t max_elements_size = 0, String cache_policy_name = "", double size_ratio = 0.5) { auto on_weight_loss_function = [&](size_t weight_loss) { onRemoveOverflowWeightLoss(weight_loss); }; @@ -47,12 +47,12 @@ public: if (cache_policy_name == "LRU") { using LRUPolicy = LRUCachePolicy; - cache_policy = std::make_unique(max_size, max_elements_size, on_weight_loss_function); + cache_policy = std::make_unique(max_size_in_bytes, max_elements_size, on_weight_loss_function); } else if (cache_policy_name == "SLRU") { using SLRUPolicy = SLRUCachePolicy; - cache_policy = std::make_unique(max_size, max_elements_size, size_ratio, on_weight_loss_function); + cache_policy = std::make_unique(max_size_in_bytes, max_elements_size, size_ratio, on_weight_loss_function); } else { @@ -175,7 +175,7 @@ public: } size_t maxSize() const - TSA_NO_THREAD_SAFETY_ANALYSIS // disabled because max_size of cache_policy is a constant parameter + TSA_NO_THREAD_SAFETY_ANALYSIS // disabled because max_size_in_bytes of cache_policy is a constant parameter { return cache_policy->maxSize(); } diff --git a/src/Common/LRUCachePolicy.h b/src/Common/LRUCachePolicy.h index 3c069eb276b..0202d0ae509 100644 --- a/src/Common/LRUCachePolicy.h +++ b/src/Common/LRUCachePolicy.h @@ -12,7 +12,7 @@ namespace DB /// Cache policy LRU evicts entries which are not used for a long time. /// WeightFunction is a functor that takes Mapped as a parameter and returns "weight" (approximate size) /// of that value. -/// Cache starts to evict entries when their total weight exceeds max_size. +/// Cache starts to evict entries when their total weight exceeds max_size_in_bytes. /// Value weight should not change after insertion. /// To work with the thread-safe implementation of this class use a class "CacheBase" with first parameter "LRU" /// and next parameters in the same order as in the constructor of the current class. @@ -27,18 +27,19 @@ public: using Base = ICachePolicy; using typename Base::OnWeightLossFunction; - /** Initialize LRUCachePolicy with max_size and max_elements_size. + /** Initialize LRUCachePolicy with max_size_in_bytes and max_elements_size. * max_elements_size == 0 means no elements size restrictions. */ - explicit LRUCachePolicy(size_t max_size_, size_t max_elements_size_ = 0, OnWeightLossFunction on_weight_loss_function_ = {}) - : max_size(std::max(static_cast(1), max_size_)), max_elements_size(max_elements_size_) + explicit LRUCachePolicy(size_t max_size_in_bytes_, size_t max_elements_size_ = 0, OnWeightLossFunction on_weight_loss_function_ = {}) + : max_size_in_bytes(std::max(static_cast(1), max_size_in_bytes_)) + , max_elements_size(max_elements_size_) { Base::on_weight_loss_function = on_weight_loss_function_; } size_t weight(std::lock_guard & /* cache_lock */) const override { - return current_size; + return current_size_in_bytes; } size_t count(std::lock_guard & /* cache_lock */) const override @@ -48,14 +49,14 @@ public: size_t maxSize() const override { - return max_size; + return max_size_in_bytes; } void reset(std::lock_guard & /* cache_lock */) override { queue.clear(); cells.clear(); - current_size = 0; + current_size_in_bytes = 0; } void remove(const Key & key, std::lock_guard & /* cache_lock */) override @@ -64,7 +65,7 @@ public: if (it == cells.end()) return; auto & cell = it->second; - current_size -= cell.size; + current_size_in_bytes -= cell.size; queue.erase(cell.queue_iterator); cells.erase(it); } @@ -107,13 +108,13 @@ public: } else { - current_size -= cell.size; + current_size_in_bytes -= cell.size; queue.splice(queue.end(), queue, cell.queue_iterator); } cell.value = mapped; cell.size = cell.value ? weight_function(*cell.value) : 0; - current_size += cell.size; + current_size_in_bytes += cell.size; removeOverflow(); } @@ -136,8 +137,8 @@ protected: Cells cells; /// Total weight of values. - size_t current_size = 0; - const size_t max_size; + size_t current_size_in_bytes = 0; + const size_t max_size_in_bytes; const size_t max_elements_size; WeightFunction weight_function; @@ -147,7 +148,7 @@ protected: size_t current_weight_lost = 0; size_t queue_size = cells.size(); - while ((current_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size)) && (queue_size > 0)) + while ((current_size_in_bytes > max_size_in_bytes || (max_elements_size != 0 && queue_size > max_elements_size)) && (queue_size > 0)) { const Key & key = queue.front(); @@ -160,7 +161,7 @@ protected: const auto & cell = it->second; - current_size -= cell.size; + current_size_in_bytes -= cell.size; current_weight_lost += cell.size; cells.erase(it); @@ -170,7 +171,7 @@ protected: Base::on_weight_loss_function(current_weight_lost); - if (current_size > (1ull << 63)) + if (current_size_in_bytes > (1ull << 63)) { LOG_ERROR(&Poco::Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it."); abort(); diff --git a/src/Common/SLRUCachePolicy.h b/src/Common/SLRUCachePolicy.h index e1d72aa630a..da268708729 100644 --- a/src/Common/SLRUCachePolicy.h +++ b/src/Common/SLRUCachePolicy.h @@ -14,7 +14,7 @@ namespace DB /// this policy protects entries which were used more then once from a sequential scan. /// WeightFunction is a functor that takes Mapped as a parameter and returns "weight" (approximate size) /// of that value. -/// Cache starts to evict entries when their total weight exceeds max_size. +/// Cache starts to evict entries when their total weight exceeds max_size_in_bytes. /// Value weight should not change after insertion. /// To work with the thread-safe implementation of this class use a class "CacheBase" with first parameter "SLRU" /// and next parameters in the same order as in the constructor of the current class. @@ -29,14 +29,14 @@ public: using Base = ICachePolicy; using typename Base::OnWeightLossFunction; - /** Initialize SLRUCachePolicy with max_size and max_protected_size. + /** Initialize SLRUCachePolicy with max_size_in_bytes and max_protected_size. * max_protected_size shows how many of the most frequently used entries will not be evicted after a sequential scan. * max_protected_size == 0 means that the default protected size is equal to half of the total max size. */ /// TODO: construct from special struct with cache policy parameters (also with max_protected_size). - SLRUCachePolicy(size_t max_size_, size_t max_elements_size_ = 0, double size_ratio = 0.5, OnWeightLossFunction on_weight_loss_function_ = {}) - : max_protected_size(static_cast(max_size_ * std::min(1.0, size_ratio))) - , max_size(max_size_) + SLRUCachePolicy(size_t max_size_in_bytes_, size_t max_elements_size_ = 0, double size_ratio = 0.5, OnWeightLossFunction on_weight_loss_function_ = {}) + : max_protected_size(static_cast(max_size_in_bytes_ * std::min(1.0, size_ratio))) + , max_size_in_bytes(max_size_in_bytes_) , max_elements_size(max_elements_size_) { Base::on_weight_loss_function = on_weight_loss_function_; @@ -44,7 +44,7 @@ public: size_t weight(std::lock_guard & /* cache_lock */) const override { - return current_size; + return current_size_in_bytes; } size_t count(std::lock_guard & /* cache_lock */) const override @@ -54,7 +54,7 @@ public: size_t maxSize() const override { - return max_size; + return max_size_in_bytes; } void reset(std::lock_guard & /* cache_lock */) override @@ -62,7 +62,7 @@ public: cells.clear(); probationary_queue.clear(); protected_queue.clear(); - current_size = 0; + current_size_in_bytes = 0; current_protected_size = 0; } @@ -72,7 +72,7 @@ public: if (it == cells.end()) return; auto & cell = it->second; - current_size -= cell.size; + current_size_in_bytes -= cell.size; if (cell.is_protected) { current_protected_size -= cell.size; @@ -129,7 +129,7 @@ public: } else { - current_size -= cell.size; + current_size_in_bytes -= cell.size; if (cell.is_protected) { current_protected_size -= cell.size; @@ -144,11 +144,11 @@ public: cell.value = mapped; cell.size = cell.value ? weight_function(*cell.value) : 0; - current_size += cell.size; + current_size_in_bytes += cell.size; current_protected_size += cell.is_protected ? cell.size : 0; removeOverflow(protected_queue, max_protected_size, current_protected_size, /*is_protected=*/true); - removeOverflow(probationary_queue, max_size, current_size, /*is_protected=*/false); + removeOverflow(probationary_queue, max_size_in_bytes, current_size_in_bytes, /*is_protected=*/false); } protected: @@ -171,9 +171,9 @@ protected: Cells cells; size_t current_protected_size = 0; - size_t current_size = 0; + size_t current_size_in_bytes = 0; const size_t max_protected_size; - const size_t max_size; + const size_t max_size_in_bytes; const size_t max_elements_size; WeightFunction weight_function; @@ -240,7 +240,7 @@ protected: Base::on_weight_loss_function(current_weight_lost); } - if (current_size > (1ull << 63)) + if (current_size_in_bytes > (1ull << 63)) { LOG_ERROR(&Poco::Logger::get("SLRUCache"), "SLRUCache became inconsistent. There must be a bug in it."); abort(); From 7d5bc0d8c9aadbf8e6a291cb013a2d0820b887e5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 12 Mar 2023 13:50:48 +0000 Subject: [PATCH 136/559] Cleanup: TrivialWeightFunction --> EqualWeightFunction A true "trival" weight function would return .size() of the cache entry. --- src/Common/CacheBase.h | 2 +- src/Common/ICachePolicy.h | 5 +++-- src/Common/LRUCachePolicy.h | 2 +- src/Common/SLRUCachePolicy.h | 2 +- 4 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Common/CacheBase.h b/src/Common/CacheBase.h index fe0f2408f05..ba8609ae0d0 100644 --- a/src/Common/CacheBase.h +++ b/src/Common/CacheBase.h @@ -29,7 +29,7 @@ namespace ErrorCodes /// of that value. /// Cache starts to evict entries when their total weight exceeds max_size_in_bytes. /// Value weight should not change after insertion. -template , typename WeightFunction = TrivialWeightFunction> +template , typename WeightFunction = EqualWeightFunction> class CacheBase { public: diff --git a/src/Common/ICachePolicy.h b/src/Common/ICachePolicy.h index 4e5916f125e..56e27cf39bd 100644 --- a/src/Common/ICachePolicy.h +++ b/src/Common/ICachePolicy.h @@ -6,8 +6,9 @@ namespace DB { + template -struct TrivialWeightFunction +struct EqualWeightFunction { size_t operator()(const T &) const { @@ -15,7 +16,7 @@ struct TrivialWeightFunction } }; -template , typename WeightFunction = TrivialWeightFunction> +template , typename WeightFunction = EqualWeightFunction> class ICachePolicy { public: diff --git a/src/Common/LRUCachePolicy.h b/src/Common/LRUCachePolicy.h index 0202d0ae509..60587294b51 100644 --- a/src/Common/LRUCachePolicy.h +++ b/src/Common/LRUCachePolicy.h @@ -16,7 +16,7 @@ namespace DB /// Value weight should not change after insertion. /// To work with the thread-safe implementation of this class use a class "CacheBase" with first parameter "LRU" /// and next parameters in the same order as in the constructor of the current class. -template , typename WeightFunction = TrivialWeightFunction> +template , typename WeightFunction = EqualWeightFunction> class LRUCachePolicy : public ICachePolicy { public: diff --git a/src/Common/SLRUCachePolicy.h b/src/Common/SLRUCachePolicy.h index da268708729..debac3824bf 100644 --- a/src/Common/SLRUCachePolicy.h +++ b/src/Common/SLRUCachePolicy.h @@ -18,7 +18,7 @@ namespace DB /// Value weight should not change after insertion. /// To work with the thread-safe implementation of this class use a class "CacheBase" with first parameter "SLRU" /// and next parameters in the same order as in the constructor of the current class. -template , typename WeightFunction = TrivialWeightFunction> +template , typename WeightFunction = EqualWeightFunction> class SLRUCachePolicy : public ICachePolicy { public: From 39898fbe64ad235a952993881b9e10013def3f29 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 12 Mar 2023 13:57:10 +0000 Subject: [PATCH 137/559] Cleanup: max_elements_size --> max_entries To disambiguate the maximum number of entries from the maximum byte size of an entry. --- src/Common/CacheBase.h | 8 +++----- src/Common/LRUCachePolicy.h | 12 ++++++------ src/Common/SLRUCachePolicy.h | 12 ++++++------ 3 files changed, 15 insertions(+), 17 deletions(-) diff --git a/src/Common/CacheBase.h b/src/Common/CacheBase.h index ba8609ae0d0..237bf664f6d 100644 --- a/src/Common/CacheBase.h +++ b/src/Common/CacheBase.h @@ -37,7 +37,7 @@ public: using Mapped = TMapped; using MappedPtr = std::shared_ptr; - explicit CacheBase(size_t max_size_in_bytes, size_t max_elements_size = 0, String cache_policy_name = "", double size_ratio = 0.5) + explicit CacheBase(size_t max_size_in_bytes, size_t max_entries = 0, String cache_policy_name = "", double size_ratio = 0.5) { auto on_weight_loss_function = [&](size_t weight_loss) { onRemoveOverflowWeightLoss(weight_loss); }; @@ -47,17 +47,15 @@ public: if (cache_policy_name == "LRU") { using LRUPolicy = LRUCachePolicy; - cache_policy = std::make_unique(max_size_in_bytes, max_elements_size, on_weight_loss_function); + cache_policy = std::make_unique(max_size_in_bytes, max_entries, on_weight_loss_function); } else if (cache_policy_name == "SLRU") { using SLRUPolicy = SLRUCachePolicy; - cache_policy = std::make_unique(max_size_in_bytes, max_elements_size, size_ratio, on_weight_loss_function); + cache_policy = std::make_unique(max_size_in_bytes, max_entries, size_ratio, on_weight_loss_function); } else - { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Undeclared cache policy name: {}", cache_policy_name); - } } MappedPtr get(const Key & key) diff --git a/src/Common/LRUCachePolicy.h b/src/Common/LRUCachePolicy.h index 60587294b51..be473310445 100644 --- a/src/Common/LRUCachePolicy.h +++ b/src/Common/LRUCachePolicy.h @@ -27,12 +27,12 @@ public: using Base = ICachePolicy; using typename Base::OnWeightLossFunction; - /** Initialize LRUCachePolicy with max_size_in_bytes and max_elements_size. - * max_elements_size == 0 means no elements size restrictions. + /** Initialize LRUCachePolicy with max_size_in_bytes and max_entries. + * max_entries == 0 means no elements size restrictions. */ - explicit LRUCachePolicy(size_t max_size_in_bytes_, size_t max_elements_size_ = 0, OnWeightLossFunction on_weight_loss_function_ = {}) + explicit LRUCachePolicy(size_t max_size_in_bytes_, size_t max_entries_ = 0, OnWeightLossFunction on_weight_loss_function_ = {}) : max_size_in_bytes(std::max(static_cast(1), max_size_in_bytes_)) - , max_elements_size(max_elements_size_) + , max_entries(max_entries_) { Base::on_weight_loss_function = on_weight_loss_function_; } @@ -139,7 +139,7 @@ protected: /// Total weight of values. size_t current_size_in_bytes = 0; const size_t max_size_in_bytes; - const size_t max_elements_size; + const size_t max_entries; WeightFunction weight_function; @@ -148,7 +148,7 @@ protected: size_t current_weight_lost = 0; size_t queue_size = cells.size(); - while ((current_size_in_bytes > max_size_in_bytes || (max_elements_size != 0 && queue_size > max_elements_size)) && (queue_size > 0)) + while ((current_size_in_bytes > max_size_in_bytes || (max_entries != 0 && queue_size > max_entries)) && (queue_size > 0)) { const Key & key = queue.front(); diff --git a/src/Common/SLRUCachePolicy.h b/src/Common/SLRUCachePolicy.h index debac3824bf..cd97a39af06 100644 --- a/src/Common/SLRUCachePolicy.h +++ b/src/Common/SLRUCachePolicy.h @@ -34,10 +34,10 @@ public: * max_protected_size == 0 means that the default protected size is equal to half of the total max size. */ /// TODO: construct from special struct with cache policy parameters (also with max_protected_size). - SLRUCachePolicy(size_t max_size_in_bytes_, size_t max_elements_size_ = 0, double size_ratio = 0.5, OnWeightLossFunction on_weight_loss_function_ = {}) + explicit SLRUCachePolicy(size_t max_size_in_bytes_, size_t max_entries_ = 0, double size_ratio = 0.5, OnWeightLossFunction on_weight_loss_function_ = {}) : max_protected_size(static_cast(max_size_in_bytes_ * std::min(1.0, size_ratio))) , max_size_in_bytes(max_size_in_bytes_) - , max_elements_size(max_elements_size_) + , max_entries(max_entries_) { Base::on_weight_loss_function = on_weight_loss_function_; } @@ -174,7 +174,7 @@ protected: size_t current_size_in_bytes = 0; const size_t max_protected_size; const size_t max_size_in_bytes; - const size_t max_elements_size; + const size_t max_entries; WeightFunction weight_function; @@ -188,11 +188,11 @@ protected: { /// Check if after remove all elements from probationary part there will be no more than max elements /// in protected queue and weight of all protected elements will be less then max protected weight. - /// It's not possible to check only cells.size() > max_elements_size + /// It's not possible to check only cells.size() > max_entries /// because protected elements move to probationary part and still remain in cache. need_remove = [&]() { - return ((max_elements_size != 0 && cells.size() - probationary_queue.size() > max_elements_size) + return ((max_entries != 0 && cells.size() - probationary_queue.size() > max_entries) || (current_weight_size > max_weight_size)) && (queue_size > 0); }; } @@ -200,7 +200,7 @@ protected: { need_remove = [&]() { - return ((max_elements_size != 0 && cells.size() > max_elements_size) + return ((max_entries != 0 && cells.size() > max_entries) || (current_weight_size > max_weight_size)) && (queue_size > 0); }; } From 9f6cb98c61718fa7c4c65b2195b36a5143e81c47 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 13 Mar 2023 07:02:52 +0000 Subject: [PATCH 138/559] Cleanup: Remove default parameters from (S)LRUCachePolicy Allows to add new mandatory parameters without accidentally breaking something. --- src/Common/LRUCachePolicy.h | 2 +- src/Common/SLRUCachePolicy.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/LRUCachePolicy.h b/src/Common/LRUCachePolicy.h index be473310445..1db7c5f2333 100644 --- a/src/Common/LRUCachePolicy.h +++ b/src/Common/LRUCachePolicy.h @@ -30,7 +30,7 @@ public: /** Initialize LRUCachePolicy with max_size_in_bytes and max_entries. * max_entries == 0 means no elements size restrictions. */ - explicit LRUCachePolicy(size_t max_size_in_bytes_, size_t max_entries_ = 0, OnWeightLossFunction on_weight_loss_function_ = {}) + LRUCachePolicy(size_t max_size_in_bytes_, size_t max_entries_, OnWeightLossFunction on_weight_loss_function_) : max_size_in_bytes(std::max(static_cast(1), max_size_in_bytes_)) , max_entries(max_entries_) { diff --git a/src/Common/SLRUCachePolicy.h b/src/Common/SLRUCachePolicy.h index cd97a39af06..04e2b8dc4c9 100644 --- a/src/Common/SLRUCachePolicy.h +++ b/src/Common/SLRUCachePolicy.h @@ -34,7 +34,7 @@ public: * max_protected_size == 0 means that the default protected size is equal to half of the total max size. */ /// TODO: construct from special struct with cache policy parameters (also with max_protected_size). - explicit SLRUCachePolicy(size_t max_size_in_bytes_, size_t max_entries_ = 0, double size_ratio = 0.5, OnWeightLossFunction on_weight_loss_function_ = {}) + SLRUCachePolicy(size_t max_size_in_bytes_, size_t max_entries_, double size_ratio, OnWeightLossFunction on_weight_loss_function_) : max_protected_size(static_cast(max_size_in_bytes_ * std::min(1.0, size_ratio))) , max_size_in_bytes(max_size_in_bytes_) , max_entries(max_entries_) From 614810e471d91f0de6015179e9f9d0168edfeef6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 13 Mar 2023 07:18:33 +0000 Subject: [PATCH 139/559] Cleanup: Fix file name typo --- src/Common/tests/{gtest_slru_cahce.cpp => gtest_slru_cache.cpp} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename src/Common/tests/{gtest_slru_cahce.cpp => gtest_slru_cache.cpp} (100%) diff --git a/src/Common/tests/gtest_slru_cahce.cpp b/src/Common/tests/gtest_slru_cache.cpp similarity index 100% rename from src/Common/tests/gtest_slru_cahce.cpp rename to src/Common/tests/gtest_slru_cache.cpp From d165a15c58014ee64b32c2d1c6e693518fc365bb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 13 Mar 2023 07:28:02 +0000 Subject: [PATCH 140/559] Cleanup: Simplify some typedefs --- src/Common/ICachePolicy.h | 4 +--- src/Common/LRUCachePolicy.h | 9 +++------ src/Common/SLRUCachePolicy.h | 9 +++------ 3 files changed, 7 insertions(+), 15 deletions(-) diff --git a/src/Common/ICachePolicy.h b/src/Common/ICachePolicy.h index 56e27cf39bd..1cdf379dcfc 100644 --- a/src/Common/ICachePolicy.h +++ b/src/Common/ICachePolicy.h @@ -16,12 +16,10 @@ struct EqualWeightFunction } }; -template , typename WeightFunction = EqualWeightFunction> +template , typename WeightFunction = EqualWeightFunction> class ICachePolicy { public: - using Key = TKey; - using Mapped = TMapped; using MappedPtr = std::shared_ptr; using OnWeightLossFunction = std::function; diff --git a/src/Common/LRUCachePolicy.h b/src/Common/LRUCachePolicy.h index 1db7c5f2333..40625e4de79 100644 --- a/src/Common/LRUCachePolicy.h +++ b/src/Common/LRUCachePolicy.h @@ -16,15 +16,12 @@ namespace DB /// Value weight should not change after insertion. /// To work with the thread-safe implementation of this class use a class "CacheBase" with first parameter "LRU" /// and next parameters in the same order as in the constructor of the current class. -template , typename WeightFunction = EqualWeightFunction> -class LRUCachePolicy : public ICachePolicy +template , typename WeightFunction = EqualWeightFunction> +class LRUCachePolicy : public ICachePolicy { public: - using Key = TKey; - using Mapped = TMapped; using MappedPtr = std::shared_ptr; - - using Base = ICachePolicy; + using Base = ICachePolicy; using typename Base::OnWeightLossFunction; /** Initialize LRUCachePolicy with max_size_in_bytes and max_entries. diff --git a/src/Common/SLRUCachePolicy.h b/src/Common/SLRUCachePolicy.h index 04e2b8dc4c9..25ac8d75599 100644 --- a/src/Common/SLRUCachePolicy.h +++ b/src/Common/SLRUCachePolicy.h @@ -18,15 +18,12 @@ namespace DB /// Value weight should not change after insertion. /// To work with the thread-safe implementation of this class use a class "CacheBase" with first parameter "SLRU" /// and next parameters in the same order as in the constructor of the current class. -template , typename WeightFunction = EqualWeightFunction> -class SLRUCachePolicy : public ICachePolicy +template , typename WeightFunction = EqualWeightFunction> +class SLRUCachePolicy : public ICachePolicy { public: - using Key = TKey; - using Mapped = TMapped; using MappedPtr = std::shared_ptr; - - using Base = ICachePolicy; + using Base = ICachePolicy; using typename Base::OnWeightLossFunction; /** Initialize SLRUCachePolicy with max_size_in_bytes and max_protected_size. From 475415e42146cdeda774dddf53ea47b2583de773 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 13 Mar 2023 07:33:41 +0000 Subject: [PATCH 141/559] Cleanup: Move on_weight_loss_function into concrete cache policies Makes ICachePolicy a pure abstract interface --- src/Common/ICachePolicy.h | 3 --- src/Common/LRUCachePolicy.h | 6 ++++-- src/Common/SLRUCachePolicy.h | 12 ++++++------ 3 files changed, 10 insertions(+), 11 deletions(-) diff --git a/src/Common/ICachePolicy.h b/src/Common/ICachePolicy.h index 1cdf379dcfc..208cf480576 100644 --- a/src/Common/ICachePolicy.h +++ b/src/Common/ICachePolicy.h @@ -33,9 +33,6 @@ public: virtual void set(const Key & key, const MappedPtr & mapped, std::lock_guard & /* cache_lock */) = 0; virtual ~ICachePolicy() = default; - -protected: - OnWeightLossFunction on_weight_loss_function = [](size_t) {}; }; } diff --git a/src/Common/LRUCachePolicy.h b/src/Common/LRUCachePolicy.h index 40625e4de79..f191f0b7d6b 100644 --- a/src/Common/LRUCachePolicy.h +++ b/src/Common/LRUCachePolicy.h @@ -21,6 +21,7 @@ class LRUCachePolicy : public ICachePolicy; + using Base = ICachePolicy; using typename Base::OnWeightLossFunction; @@ -30,8 +31,8 @@ public: LRUCachePolicy(size_t max_size_in_bytes_, size_t max_entries_, OnWeightLossFunction on_weight_loss_function_) : max_size_in_bytes(std::max(static_cast(1), max_size_in_bytes_)) , max_entries(max_entries_) + , on_weight_loss_function(on_weight_loss_function_) { - Base::on_weight_loss_function = on_weight_loss_function_; } size_t weight(std::lock_guard & /* cache_lock */) const override @@ -139,6 +140,7 @@ protected: const size_t max_entries; WeightFunction weight_function; + OnWeightLossFunction on_weight_loss_function; void removeOverflow() { @@ -166,7 +168,7 @@ protected: --queue_size; } - Base::on_weight_loss_function(current_weight_lost); + on_weight_loss_function(current_weight_lost); if (current_size_in_bytes > (1ull << 63)) { diff --git a/src/Common/SLRUCachePolicy.h b/src/Common/SLRUCachePolicy.h index 25ac8d75599..80d622eba3e 100644 --- a/src/Common/SLRUCachePolicy.h +++ b/src/Common/SLRUCachePolicy.h @@ -23,6 +23,7 @@ class SLRUCachePolicy : public ICachePolicy; + using Base = ICachePolicy; using typename Base::OnWeightLossFunction; @@ -35,9 +36,9 @@ public: : max_protected_size(static_cast(max_size_in_bytes_ * std::min(1.0, size_ratio))) , max_size_in_bytes(max_size_in_bytes_) , max_entries(max_entries_) - { - Base::on_weight_loss_function = on_weight_loss_function_; - } + , on_weight_loss_function(on_weight_loss_function_) + { + } size_t weight(std::lock_guard & /* cache_lock */) const override { @@ -174,6 +175,7 @@ protected: const size_t max_entries; WeightFunction weight_function; + OnWeightLossFunction on_weight_loss_function; void removeOverflow(SLRUQueue & queue, const size_t max_weight_size, size_t & current_weight_size, bool is_protected) { @@ -233,9 +235,7 @@ protected: } if (!is_protected) - { - Base::on_weight_loss_function(current_weight_lost); - } + on_weight_loss_function(current_weight_lost); if (current_size_in_bytes > (1ull << 63)) { From 04ba321ebd3fa3c8e1c5f773638cdf6582035189 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 13 Mar 2023 10:28:40 +0000 Subject: [PATCH 142/559] Fix unit test build --- src/IO/S3/Client.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index f095c5b31e7..dd9eda33d92 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -11,6 +11,7 @@ #include #include #include +#include #include #include @@ -224,7 +225,7 @@ public: HTTPHeaderEntries headers, bool use_environment_credentials, bool use_insecure_imds_request, - uint64_t expiration_window_seconds); + uint64_t expiration_window_seconds = DEFAULT_EXPIRATION_WINDOW_SECONDS); PocoHTTPClientConfiguration createClientConfiguration( const String & force_region, From 657aa654462137f4c4c90d095501eb301ee4ee80 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 13 Mar 2023 09:06:58 +0000 Subject: [PATCH 143/559] Cleanup: Untangle CacheBase's constructors This prepares for the custom TTLCachePolicy which users should not be able to configure explicitly. --- programs/local/LocalServer.cpp | 4 ++-- programs/server/Server.cpp | 4 ++-- src/Common/CacheBase.h | 22 +++++++++++++++++----- src/Common/tests/gtest_lru_cache.cpp | 12 ++++++------ src/Common/tests/gtest_slru_cache.cpp | 22 +++++++++++----------- src/IO/UncompressedCache.h | 7 +++++-- src/Interpreters/Context.cpp | 8 ++++---- src/Interpreters/Context.h | 4 ++-- src/Storages/MarkCache.h | 7 +++++-- 9 files changed, 54 insertions(+), 36 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 8e092bdf8e4..5768e744f94 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -600,13 +600,13 @@ void LocalServer::processConfig() String uncompressed_cache_policy = config().getString("uncompressed_cache_policy", ""); size_t uncompressed_cache_size = config().getUInt64("uncompressed_cache_size", 0); if (uncompressed_cache_size) - global_context->setUncompressedCache(uncompressed_cache_size, uncompressed_cache_policy); + global_context->setUncompressedCache(uncompressed_cache_policy, uncompressed_cache_size); /// Size of cache for marks (index of MergeTree family of tables). String mark_cache_policy = config().getString("mark_cache_policy", ""); size_t mark_cache_size = config().getUInt64("mark_cache_size", 5368709120); if (mark_cache_size) - global_context->setMarkCache(mark_cache_size, mark_cache_policy); + global_context->setMarkCache(mark_cache_policy, mark_cache_size); /// Size of cache for uncompressed blocks of MergeTree indices. Zero means disabled. size_t index_uncompressed_cache_size = config().getUInt64("index_uncompressed_cache_size", 0); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 57d361886d2..02aa88620aa 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1459,7 +1459,7 @@ try LOG_INFO(log, "Uncompressed cache size was lowered to {} because the system has low amount of memory", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); } - global_context->setUncompressedCache(uncompressed_cache_size, uncompressed_cache_policy); + global_context->setUncompressedCache(uncompressed_cache_policy, uncompressed_cache_size); /// Load global settings from default_profile and system_profile. global_context->setDefaultProfiles(config()); @@ -1484,7 +1484,7 @@ try LOG_INFO(log, "Mark cache size was lowered to {} because the system has low amount of memory", formatReadableSizeWithBinarySuffix(mark_cache_size)); } - global_context->setMarkCache(mark_cache_size, mark_cache_policy); + global_context->setMarkCache(mark_cache_policy, mark_cache_size); if (server_settings.index_uncompressed_cache_size) global_context->setIndexUncompressedCache(server_settings.index_uncompressed_cache_size); diff --git a/src/Common/CacheBase.h b/src/Common/CacheBase.h index 237bf664f6d..eb537c22834 100644 --- a/src/Common/CacheBase.h +++ b/src/Common/CacheBase.h @@ -37,12 +37,21 @@ public: using Mapped = TMapped; using MappedPtr = std::shared_ptr; - explicit CacheBase(size_t max_size_in_bytes, size_t max_entries = 0, String cache_policy_name = "", double size_ratio = 0.5) + /// Use this ctor if you don't care about the internal cache policy. + explicit CacheBase(size_t max_size_in_bytes, size_t max_entries = 0, double size_ratio = 0.5) + : CacheBase("SLRU", max_size_in_bytes, max_entries, size_ratio) + { + } + + /// Use this ctor if you want the user to configure the cache policy via some setting. Supports only general-purpose policies LRU and SLRU. + explicit CacheBase(std::string_view cache_policy_name, size_t max_size_in_bytes, size_t max_entries = 0, double size_ratio = 0.5) { auto on_weight_loss_function = [&](size_t weight_loss) { onRemoveOverflowWeightLoss(weight_loss); }; + static constexpr std::string_view default_cache_policy = "SLRU"; + if (cache_policy_name.empty()) - cache_policy_name = default_cache_policy_name; + cache_policy_name = default_cache_policy; if (cache_policy_name == "LRU") { @@ -55,9 +64,14 @@ public: cache_policy = std::make_unique(max_size_in_bytes, max_entries, size_ratio, on_weight_loss_function); } else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Undeclared cache policy name: {}", cache_policy_name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown cache policy name: {}", cache_policy_name); } + /// Use this ctor to provide an arbitrary cache policy. + explicit CacheBase(std::unique_ptr> cache_policy_) + : cache_policy(std::move(cache_policy_)) + {} + MappedPtr get(const Key & key) { std::lock_guard lock(mutex); @@ -188,8 +202,6 @@ private: std::unique_ptr cache_policy TSA_GUARDED_BY(mutex); - inline static const String default_cache_policy_name = "SLRU"; - std::atomic hits{0}; std::atomic misses{0}; diff --git a/src/Common/tests/gtest_lru_cache.cpp b/src/Common/tests/gtest_lru_cache.cpp index f74d1eb9464..9a2cb354bd5 100644 --- a/src/Common/tests/gtest_lru_cache.cpp +++ b/src/Common/tests/gtest_lru_cache.cpp @@ -6,7 +6,7 @@ TEST(LRUCache, set) { using SimpleCacheBase = DB::CacheBase; - auto lru_cache = SimpleCacheBase(/*max_size*/ 10, /*max_elements_size*/ 10, "LRU"); + auto lru_cache = SimpleCacheBase("LRU", /*max_size*/ 10, /*max_elements_size*/ 10); lru_cache.set(1, std::make_shared(2)); lru_cache.set(2, std::make_shared(3)); @@ -19,7 +19,7 @@ TEST(LRUCache, set) TEST(LRUCache, update) { using SimpleCacheBase = DB::CacheBase; - auto lru_cache = SimpleCacheBase(/*max_size*/ 10, /*max_elements_size*/ 10, "LRU"); + auto lru_cache = SimpleCacheBase("LRU", /*max_size*/ 10, /*max_elements_size*/ 10); lru_cache.set(1, std::make_shared(2)); lru_cache.set(1, std::make_shared(3)); auto val = lru_cache.get(1); @@ -30,7 +30,7 @@ TEST(LRUCache, update) TEST(LRUCache, get) { using SimpleCacheBase = DB::CacheBase; - auto lru_cache = SimpleCacheBase(/*max_size*/ 10, /*max_elements_size*/ 10, "LRU"); + auto lru_cache = SimpleCacheBase("LRU", /*max_size*/ 10, /*max_elements_size*/ 10); lru_cache.set(1, std::make_shared(2)); lru_cache.set(2, std::make_shared(3)); SimpleCacheBase::MappedPtr value = lru_cache.get(1); @@ -50,7 +50,7 @@ struct ValueWeight TEST(LRUCache, evictOnSize) { using SimpleCacheBase = DB::CacheBase; - auto lru_cache = SimpleCacheBase(/*max_size*/ 20, /*max_elements_size*/ 3, "LRU"); + auto lru_cache = SimpleCacheBase("LRU", /*max_size*/ 20, /*max_elements_size*/ 3); lru_cache.set(1, std::make_shared(2)); lru_cache.set(2, std::make_shared(3)); lru_cache.set(3, std::make_shared(4)); @@ -66,7 +66,7 @@ TEST(LRUCache, evictOnSize) TEST(LRUCache, evictOnWeight) { using SimpleCacheBase = DB::CacheBase, ValueWeight>; - auto lru_cache = SimpleCacheBase(/*max_size*/ 10, /*max_elements_size*/ 10, "LRU"); + auto lru_cache = SimpleCacheBase("LRU", /*max_size*/ 10, /*max_elements_size*/ 10); lru_cache.set(1, std::make_shared(2)); lru_cache.set(2, std::make_shared(3)); lru_cache.set(3, std::make_shared(4)); @@ -87,7 +87,7 @@ TEST(LRUCache, evictOnWeight) TEST(LRUCache, getOrSet) { using SimpleCacheBase = DB::CacheBase, ValueWeight>; - auto lru_cache = SimpleCacheBase(/*max_size*/ 10, /*max_elements_size*/ 10, "LRU"); + auto lru_cache = SimpleCacheBase("LRU", /*max_size*/ 10, /*max_elements_size*/ 10); size_t x = 10; auto load_func = [&] { return std::make_shared(x); }; auto [value, loaded] = lru_cache.getOrSet(1, load_func); diff --git a/src/Common/tests/gtest_slru_cache.cpp b/src/Common/tests/gtest_slru_cache.cpp index 66df0dbec77..f7ae9f9b16e 100644 --- a/src/Common/tests/gtest_slru_cache.cpp +++ b/src/Common/tests/gtest_slru_cache.cpp @@ -6,7 +6,7 @@ TEST(SLRUCache, set) { using SimpleCacheBase = DB::CacheBase; - auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/0, "SLRU", /*size_ratio*/0.5); + auto slru_cache = SimpleCacheBase("SLRU", /*max_size=*/10, /*max_elements_size=*/0, /*size_ratio*/0.5); slru_cache.set(1, std::make_shared(2)); slru_cache.set(2, std::make_shared(3)); @@ -19,7 +19,7 @@ TEST(SLRUCache, set) TEST(SLRUCache, update) { using SimpleCacheBase = DB::CacheBase; - auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/0, "SLRU", /*size_ratio*/0.5); + auto slru_cache = SimpleCacheBase("SLRU", /*max_size=*/10, /*max_elements_size=*/0, /*size_ratio*/0.5); slru_cache.set(1, std::make_shared(2)); slru_cache.set(1, std::make_shared(3)); @@ -31,7 +31,7 @@ TEST(SLRUCache, update) TEST(SLRUCache, get) { using SimpleCacheBase = DB::CacheBase; - auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/0, "SLRU", /*size_ratio*/0.5); + auto slru_cache = SimpleCacheBase("SLRU", /*max_size=*/10, /*max_elements_size=*/0, /*size_ratio*/0.5); slru_cache.set(1, std::make_shared(2)); slru_cache.set(2, std::make_shared(3)); @@ -47,7 +47,7 @@ TEST(SLRUCache, get) TEST(SLRUCache, remove) { using SimpleCacheBase = DB::CacheBase; - auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/0, "SLRU", /*size_ratio*/0.5); + auto slru_cache = SimpleCacheBase("SLRU", /*max_size=*/10, /*max_elements_size=*/0, /*size_ratio*/0.5); slru_cache.set(1, std::make_shared(2)); slru_cache.set(2, std::make_shared(3)); @@ -63,7 +63,7 @@ TEST(SLRUCache, remove) TEST(SLRUCache, removeFromProtected) { using SimpleCacheBase = DB::CacheBase; - auto slru_cache = SimpleCacheBase(/*max_size=*/2, /*max_elements_size=*/0, "SLRU", /*size_ratio*/0.5); + auto slru_cache = SimpleCacheBase("SLRU", /*max_size=*/2, /*max_elements_size=*/0, /*size_ratio*/0.5); slru_cache.set(1, std::make_shared(2)); slru_cache.set(1, std::make_shared(3)); @@ -96,7 +96,7 @@ TEST(SLRUCache, removeFromProtected) TEST(SLRUCache, reset) { using SimpleCacheBase = DB::CacheBase; - auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/0, "SLRU", /*size_ratio*/0.5); + auto slru_cache = SimpleCacheBase("SLRU", /*max_size=*/10, /*max_elements_size=*/0, /*size_ratio*/0.5); slru_cache.set(1, std::make_shared(2)); slru_cache.set(2, std::make_shared(3)); @@ -119,7 +119,7 @@ struct ValueWeight TEST(SLRUCache, evictOnElements) { using SimpleCacheBase = DB::CacheBase, ValueWeight>; - auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/1, "SLRU", /*size_ratio*/0.5); + auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/1, /*size_ratio*/0.5); slru_cache.set(1, std::make_shared(2)); slru_cache.set(2, std::make_shared(3)); @@ -140,7 +140,7 @@ TEST(SLRUCache, evictOnElements) TEST(SLRUCache, evictOnWeight) { using SimpleCacheBase = DB::CacheBase, ValueWeight>; - auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/0, "SLRU", /*size_ratio*/0.5); + auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/0, /*size_ratio*/0.5); slru_cache.set(1, std::make_shared(2)); slru_cache.set(2, std::make_shared(3)); slru_cache.set(3, std::make_shared(4)); @@ -161,7 +161,7 @@ TEST(SLRUCache, evictOnWeight) TEST(SLRUCache, evictFromProtectedPart) { using SimpleCacheBase = DB::CacheBase, ValueWeight>; - auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/0, "SLRU", /*size_ratio*/0.5); + auto slru_cache = SimpleCacheBase("SLRU", /*max_size=*/10, /*max_elements_size=*/0, /*size_ratio*/0.5); slru_cache.set(1, std::make_shared(2)); slru_cache.set(1, std::make_shared(2)); @@ -177,7 +177,7 @@ TEST(SLRUCache, evictFromProtectedPart) TEST(SLRUCache, evictStreamProtected) { using SimpleCacheBase = DB::CacheBase, ValueWeight>; - auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/0, "SLRU", /*size_ratio*/0.5); + auto slru_cache = SimpleCacheBase("SLRU", /*max_size=*/10, /*max_elements_size=*/0, /*size_ratio*/0.5); slru_cache.set(1, std::make_shared(2)); slru_cache.set(1, std::make_shared(2)); @@ -201,7 +201,7 @@ TEST(SLRUCache, evictStreamProtected) TEST(SLRUCache, getOrSet) { using SimpleCacheBase = DB::CacheBase, ValueWeight>; - auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/0, "SLRU", /*size_ratio*/0.5); + auto slru_cache = SimpleCacheBase("SLRU", /*max_size=*/10, /*max_elements_size=*/0, /*size_ratio*/0.5); size_t x = 5; auto load_func = [&] { return std::make_shared(x); }; auto [value, loaded] = slru_cache.getOrSet(1, load_func); diff --git a/src/IO/UncompressedCache.h b/src/IO/UncompressedCache.h index 3d1c907d364..2e654b27ed7 100644 --- a/src/IO/UncompressedCache.h +++ b/src/IO/UncompressedCache.h @@ -42,8 +42,11 @@ private: using Base = CacheBase; public: - explicit UncompressedCache(size_t max_size_in_bytes, const String & uncompressed_cache_policy = "") - : Base(max_size_in_bytes, 0, uncompressed_cache_policy) {} + explicit UncompressedCache(size_t max_size_in_bytes) + : Base(max_size_in_bytes) {} + + UncompressedCache(const String & uncompressed_cache_policy, size_t max_size_in_bytes) + : Base(uncompressed_cache_policy, max_size_in_bytes) {} /// Calculate key from path to file and offset. static UInt128 hash(const String & path_to_file, size_t offset) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index cf1d5203bf7..0a2348d8749 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1946,14 +1946,14 @@ QueryStatusPtr Context::getProcessListElement() const } -void Context::setUncompressedCache(size_t max_size_in_bytes, const String & uncompressed_cache_policy) +void Context::setUncompressedCache(const String & uncompressed_cache_policy, size_t max_size_in_bytes) { auto lock = getLock(); if (shared->uncompressed_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Uncompressed cache has been already created."); - shared->uncompressed_cache = std::make_shared(max_size_in_bytes, uncompressed_cache_policy); + shared->uncompressed_cache = std::make_shared(uncompressed_cache_policy, max_size_in_bytes); } @@ -1972,14 +1972,14 @@ void Context::dropUncompressedCache() const } -void Context::setMarkCache(size_t cache_size_in_bytes, const String & mark_cache_policy) +void Context::setMarkCache(const String & mark_cache_policy, size_t cache_size_in_bytes) { auto lock = getLock(); if (shared->mark_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mark cache has been already created."); - shared->mark_cache = std::make_shared(cache_size_in_bytes, mark_cache_policy); + shared->mark_cache = std::make_shared(mark_cache_policy, cache_size_in_bytes); } MarkCachePtr Context::getMarkCache() const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 67594a41459..bbfbd4defdc 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -861,12 +861,12 @@ public: void setSystemZooKeeperLogAfterInitializationIfNeeded(); /// Create a cache of uncompressed blocks of specified size. This can be done only once. - void setUncompressedCache(size_t max_size_in_bytes, const String & uncompressed_cache_policy); + void setUncompressedCache(const String & uncompressed_cache_policy, size_t max_size_in_bytes); std::shared_ptr getUncompressedCache() const; void dropUncompressedCache() const; /// Create a cache of marks of specified size. This can be done only once. - void setMarkCache(size_t cache_size_in_bytes, const String & mark_cache_policy); + void setMarkCache(const String & mark_cache_policy, size_t cache_size_in_bytes); std::shared_ptr getMarkCache() const; void dropMarkCache() const; ThreadPool & getLoadMarksThreadpool() const; diff --git a/src/Storages/MarkCache.h b/src/Storages/MarkCache.h index 9095bf6bb35..ba521073928 100644 --- a/src/Storages/MarkCache.h +++ b/src/Storages/MarkCache.h @@ -40,8 +40,11 @@ private: using Base = CacheBase; public: - explicit MarkCache(size_t max_size_in_bytes, const String & mark_cache_policy = "") - : Base(max_size_in_bytes, 0, mark_cache_policy) {} + explicit MarkCache(size_t max_size_in_bytes) + : Base(max_size_in_bytes) {} + + MarkCache(const String & mark_cache_policy, size_t max_size_in_bytes) + : Base(mark_cache_policy, max_size_in_bytes) {} /// Calculate key from path to file and offset. static UInt128 hash(const String & path_to_file) From 1d8004756a8aed483c5e47f5924c601ae8b675c6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 13 Mar 2023 12:10:22 +0000 Subject: [PATCH 144/559] Cleanup: Reuse aliases instead of redeclaring it --- src/Common/CacheBase.h | 9 ++++----- src/Common/ICachePolicy.h | 4 +++- src/Common/LRUCachePolicy.h | 3 +-- src/Common/SLRUCachePolicy.h | 3 +-- 4 files changed, 9 insertions(+), 10 deletions(-) diff --git a/src/Common/CacheBase.h b/src/Common/CacheBase.h index eb537c22834..204e21a1e89 100644 --- a/src/Common/CacheBase.h +++ b/src/Common/CacheBase.h @@ -33,9 +33,10 @@ template ; + using CachePolicy = ICachePolicy; + using Key = typename CachePolicy::Key; + using Mapped = typename CachePolicy::Mapped; + using MappedPtr = typename CachePolicy::MappedPtr; /// Use this ctor if you don't care about the internal cache policy. explicit CacheBase(size_t max_size_in_bytes, size_t max_entries = 0, double size_ratio = 0.5) @@ -198,8 +199,6 @@ protected: mutable std::mutex mutex; private: - using CachePolicy = ICachePolicy; - std::unique_ptr cache_policy TSA_GUARDED_BY(mutex); std::atomic hits{0}; diff --git a/src/Common/ICachePolicy.h b/src/Common/ICachePolicy.h index 208cf480576..a9dff7a99c1 100644 --- a/src/Common/ICachePolicy.h +++ b/src/Common/ICachePolicy.h @@ -16,10 +16,12 @@ struct EqualWeightFunction } }; -template , typename WeightFunction = EqualWeightFunction> +template , typename WeightFunction = EqualWeightFunction> class ICachePolicy { public: + using Key = TKey; + using Mapped = TMapped; using MappedPtr = std::shared_ptr; using OnWeightLossFunction = std::function; diff --git a/src/Common/LRUCachePolicy.h b/src/Common/LRUCachePolicy.h index f191f0b7d6b..19cd94be1d3 100644 --- a/src/Common/LRUCachePolicy.h +++ b/src/Common/LRUCachePolicy.h @@ -20,9 +20,8 @@ template , class LRUCachePolicy : public ICachePolicy { public: - using MappedPtr = std::shared_ptr; - using Base = ICachePolicy; + using typename Base::MappedPtr; using typename Base::OnWeightLossFunction; /** Initialize LRUCachePolicy with max_size_in_bytes and max_entries. diff --git a/src/Common/SLRUCachePolicy.h b/src/Common/SLRUCachePolicy.h index 80d622eba3e..fed958d3aff 100644 --- a/src/Common/SLRUCachePolicy.h +++ b/src/Common/SLRUCachePolicy.h @@ -22,9 +22,8 @@ template , class SLRUCachePolicy : public ICachePolicy { public: - using MappedPtr = std::shared_ptr; - using Base = ICachePolicy; + using typename Base::MappedPtr; using typename Base::OnWeightLossFunction; /** Initialize SLRUCachePolicy with max_size_in_bytes and max_protected_size. From de8d0040a4452e72c7d8d00444ce787000e0ce21 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 13 Mar 2023 18:34:47 +0100 Subject: [PATCH 145/559] refine code --- src/Common/OptimizedRegularExpression.cpp | 170 ++++++++++++++++------ src/Common/OptimizedRegularExpression.h | 3 +- src/Common/tests/gtest_optimize_re.cpp | 23 +-- 3 files changed, 139 insertions(+), 57 deletions(-) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index efa33f27ed4..4d0cdae1b99 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -16,15 +16,42 @@ namespace DB } } +namespace +{ -template -const char * OptimizedRegularExpressionImpl::analyze( +struct Literal +{ + std::string literal; + bool prefix; /// this literal string is the prefix of the whole string. + bool suffix; /// this literal string is the suffic of the whole string. + void clear() + { + literal.clear(); + prefix = false; + suffix = false; + } +}; + +using Literals = std::vector; + +size_t shortest_alter_length(const Literals & literals) +{ + if (literals.empty()) return 0; + size_t shortest = ~(0); + for (const auto & lit : literals) + { + if (shortest > lit.literal.size()) + shortest = lit.literal.size(); + } + return shortest; +} + +const char * analyzeImpl( std::string_view regexp, const char * pos, - std::string & required_substring, + Literal & required_substring, bool & is_trivial, - bool & required_substring_is_prefix, - std::vector & alternatives) + Literals & global_alters) { /** The expression is trivial if all the metacharacters in it are escaped. * The non-alternative string is @@ -39,7 +66,7 @@ const char * OptimizedRegularExpressionImpl::analyze( bool first_call = begin == regexp.data(); int depth = 0; is_trivial = true; - required_substring_is_prefix = false; + ///required_substring_is_prefix = false; required_substring.clear(); bool has_alternative_on_depth_0 = false; bool has_case_insensitive_flag = false; @@ -51,46 +78,79 @@ const char * OptimizedRegularExpressionImpl::analyze( Substrings trivial_substrings(1); Substring * last_substring = &trivial_substrings.back(); - auto finish_non_trivial_char = [&]() + Literals cur_alters; + + auto finish_cur_alters = [&]() + { + if (cur_alters.empty()) + return; + + if (global_alters.empty()) + { + global_alters = std::move(cur_alters); + return; + } + if (shortest_alter_length(global_alters) > shortest_alter_length(cur_alters)) + { + cur_alters.clear(); + } + else + { + global_alters.clear(); + global_alters = std::move(cur_alters); + } + }; + + auto finish_non_trivial_char = [&](bool create_new_substr = true) { if (depth != 0) return; - if (!last_substring->first.empty()) + for (auto & alter : cur_alters) + { + if (alter.suffix) + { + alter.literal += last_substring->first; + } + } + + finish_cur_alters(); + + if (!last_substring->first.empty() && create_new_substr) { trivial_substrings.resize(trivial_substrings.size() + 1); last_substring = &trivial_substrings.back(); } }; - - auto finish_group = [&](std::string group_required_string, bool group_is_trivial, bool group_is_prefix, std::vector & group_alternatives) + /// Resolve the string or alters in a group (xxxxx) + auto finish_group = [&](Literal & group_required_string, Literals & group_alters) { - if (alternatives.empty() && !group_alternatives.empty()) + for (auto & alter : group_alters) { - /// Check if group alternatives has empty strings - bool has_empty_str = false; - for (const std::string & alter : group_alternatives) - has_empty_str |= alter.empty(); - if (!has_empty_str) - alternatives = std::move(group_alternatives); + if (alter.prefix) + { + alter.literal = last_substring->first + alter.literal; + } } - if (group_is_prefix) - last_substring->first += group_required_string; + if (group_required_string.prefix) + last_substring->first += group_required_string.literal; else { finish_non_trivial_char(); - last_substring->first = group_required_string; + last_substring->first = std::move(group_required_string).literal; } /// if we can still append, no need to finish it. e.g. abc(de)fg should capture abcdefg - if (!last_substring->first.empty() && !group_is_trivial) + if (!last_substring->first.empty() && !group_required_string.suffix) { trivial_substrings.resize(trivial_substrings.size() + 1); last_substring = &trivial_substrings.back(); } - if (!group_is_trivial) - is_trivial = false; + + /// assign group alters to current alters. + finish_cur_alters(); + cur_alters = std::move(group_alters); }; bool in_curly_braces = false; @@ -178,23 +238,24 @@ const char * OptimizedRegularExpressionImpl::analyze( { pos += 2; } - std::string group_required_substr; - bool group_is_trival; - bool group_is_prefix; - std::vector group_alters; - pos = analyze(regexp, pos + 1, group_required_substr, group_is_trival, group_is_prefix, group_alters); + Literal group_required_substr; + bool group_is_trival = true; + Literals group_alters; + pos = analyzeImpl(regexp, pos + 1, group_required_substr, group_is_trival, group_alters); /// pos should be ')', if not, then it is not a valid regular expression if (pos == end) return pos; - /// For ()? ()* (){0,1}, we can just ignore the whole group. + /// For ()? or ()* or (){0,1}, we can just ignore the whole group. if ((pos + 1 < end && (pos[1] == '?' || pos[1] == '*')) || (pos + 2 < end && pos[1] == '{' && pos[2] == '0')) { finish_non_trivial_char(); } else - finish_group(group_required_substr, group_is_trival, group_is_prefix, group_alters); + { + finish_group(group_required_substr, group_alters); + } } ++pos; break; @@ -271,8 +332,8 @@ const char * OptimizedRegularExpressionImpl::analyze( } } finish: - if (last_substring && last_substring->first.empty()) - trivial_substrings.pop_back(); + + finish_non_trivial_char(false); if (!is_trivial) { @@ -300,35 +361,37 @@ finish: if (max_length >= MIN_LENGTH_FOR_STRSTR || !first_call) { - required_substring = candidate_it->first; - required_substring_is_prefix = candidate_it->second == 0; + required_substring.literal = candidate_it->first; + required_substring.prefix = candidate_it->second == 0; + required_substring.suffix = candidate_it + 1 == trivial_substrings.end(); } } } else if (!trivial_substrings.empty()) { - required_substring = trivial_substrings.front().first; - required_substring_is_prefix = trivial_substrings.front().second == 0; + required_substring.literal = trivial_substrings.front().first; + required_substring.prefix = trivial_substrings.front().second == 0; + required_substring.suffix = true; } /// if it is xxx|xxx|xxx, we should call the next xxx|xxx recursively and collect the result. if (has_alternative_on_depth_0) { - if (alternatives.empty()) - alternatives.push_back(required_substring); - std::vector next_alternatives; + /// compare the quality of required substring and alternatives and choose the better one. + if (shortest_alter_length(global_alters) < required_substring.literal.size()) + global_alters = {required_substring}; + Literals next_alternatives; /// this two vals are useless, xxx|xxx cannot be trivial nor prefix. - bool next_is_trivial; - bool next_is_prefix; - pos = analyze(regexp, pos, required_substring, next_is_trivial, next_is_prefix, next_alternatives); + bool next_is_trivial = true; + pos = analyzeImpl(regexp, pos, required_substring, next_is_trivial, next_alternatives); /// For xxx|xxx|xxx, we only conbine the alternatives and return a empty required_substring. if (next_alternatives.empty()) { - alternatives.push_back(required_substring); + global_alters.push_back(required_substring); } else { - alternatives.insert(alternatives.end(), next_alternatives.begin(), next_alternatives.end()); + global_alters.insert(global_alters.end(), next_alternatives.begin(), next_alternatives.end()); } required_substring.clear(); } @@ -342,13 +405,30 @@ finish: << ", required_substring_is_prefix: " << required_substring_is_prefix << std::endl;*/ } +} +template +void OptimizedRegularExpressionImpl::analyze( + std::string_view regexp_, + std::string & required_substring, + bool & is_trivial, + bool & required_substring_is_prefix, + std::vector & alternatives) +{ + Literals alter_literals; + Literal required_lit; + analyzeImpl(regexp_, regexp_.data(), required_lit, is_trivial, alter_literals); + required_substring = std::move(required_lit.literal); + required_substring_is_prefix = required_lit.prefix; + for (auto & lit : alter_literals) + alternatives.push_back(std::move(lit.literal)); +} template OptimizedRegularExpressionImpl::OptimizedRegularExpressionImpl(const std::string & regexp_, int options) { std::vector alternatives; /// this vector collects patterns in (xx|xx|xx). for now it's not used. - analyze(regexp_, regexp_.data(), required_substring, is_trivial, required_substring_is_prefix, alternatives); + analyze(regexp_, required_substring, is_trivial, required_substring_is_prefix, alternatives); /// Just three following options are supported if (options & (~(RE_CASELESS | RE_NO_CAPTURE | RE_DOT_NL))) diff --git a/src/Common/OptimizedRegularExpression.h b/src/Common/OptimizedRegularExpression.h index a19ce22deab..566bedc5549 100644 --- a/src/Common/OptimizedRegularExpression.h +++ b/src/Common/OptimizedRegularExpression.h @@ -95,9 +95,8 @@ public: out_required_substring_is_prefix = required_substring_is_prefix; } - static const char * analyze( + static void analyze( std::string_view regexp_, - const char * pos, std::string & required_substring, bool & is_trivial, bool & required_substring_is_prefix, diff --git a/src/Common/tests/gtest_optimize_re.cpp b/src/Common/tests/gtest_optimize_re.cpp index c90b5086b1c..9de652da365 100644 --- a/src/Common/tests/gtest_optimize_re.cpp +++ b/src/Common/tests/gtest_optimize_re.cpp @@ -10,30 +10,33 @@ TEST(OptimizeRE, analyze) bool is_trivial; bool is_prefix; std::vector alternatives; - OptimizedRegularExpression::analyze(regexp, regexp.data(), required, is_trivial, is_prefix, alternatives); + OptimizedRegularExpression::analyze(regexp, required, is_trivial, is_prefix, alternatives); std::cerr << regexp << std::endl; EXPECT_EQ(required, answer); EXPECT_EQ(alternatives, expect_alternatives); }; test_f("abc", "abc"); test_f("abc(de)fg", "abcdefg"); - test_f("abc(de|xyz)fg", "abc", {"de", "xyz"}); - test_f("abc(de?f|xyz)fg", "abc", {"d", "xyz"}); + test_f("abc(de|xyz)fg", "abc", {"abcdefg", "abcxyzfg"}); + test_f("abc(de?f|xyz)fg", "abc", {"abcd", "abcxyzfg"}); test_f("abc|fgk|xyz", "", {"abc","fgk", "xyz"}); test_f("(abc)", "abc"); test_f("(abc|fgk)", "", {"abc","fgk"}); - test_f("abc(abc|fg)xyzz", "xyzz", {"abc","fg"}); + test_f("(abc|fgk)(e|f|zkh|)", "", {"abc","fgk"}); + test_f("abc(abc|fg)xyzz", "xyzz", {"abcabcxyzz","abcfgxyzz"}); test_f("abc[k]xyzz", "xyzz"); test_f("(abc[k]xyzz)", "xyzz"); - test_f("abc((de)fg(hi))jk", "abcdefghi"); + test_f("abc((de)fg(hi))jk", "abcdefghijk"); + test_f("abc((?:de)fg(?:hi))jk", "abcdefghijk"); test_f("abc((de)fghi+zzz)jk", "abcdefghi"); test_f("abc((de)fg(hi))?jk", "abc"); test_f("abc((de)fghi?zzz)jk", "abcdefgh"); - test_f("abc(*cd)jk", "abc"); - test_f(R"(abc(de|xyz|(\{xx\}))fg)", "abc", {"de", "xyz", "{xx}"}); + test_f("abc(*cd)jk", "cdjk"); + test_f(R"(abc(de|xyz|(\{xx\}))fg)", "abc", {"abcdefg", "abcxyzfg", "abc{xx}fg"}); test_f("abc(abc|fg)?xyzz", "xyzz"); test_f("abc(abc|fg){0,1}xyzz", "xyzz"); - test_f("abc(abc|fg)xyzz|bcdd?k|bc(f|g|h?)z", "", {"abc", "fg", "bcd", "bc"}); - test_f("abc(abc|fg)xyzz|bc(dd?x|kk?y|(f))k|bc(f|g|h?)z", "", {"abc", "fg", "d", "k", "f", "bc"}); - test_f("((?:abc|efg|xyz)/[a-zA-Z0-9]{1-50})(/?[^ ]*|)", "", {"abc", "efg", "xyz"}); + test_f("abc(abc|fg)xyzz|bcdd?k|bc(f|g|h?)z", "", {"abcabcxyzz", "abcfgxyzz", "bcd", "bcfz", "bcgz", "bcz"}); + test_f("abc(abc|fg)xyzz|bc(dd?x|kk?y|(f))k|bc(f|g|h?)z", "", {"abcabcxyzz", "abcfgxyzz", "bcd", "bck", "bcfk", "bcfz", "bcgz", "bcz"}); + test_f("((?:abc|efg|xyz)/[a-zA-Z0-9]{1-50})(/?[^ ]*|)", "", {"abc/", "efg/", "xyz/"}); + test_f(R"([Bb]ai[Dd]u[Ss]pider(?:-[A-Za-z]{1,30})(?:-[A-Za-z]{1,30}|)|bingbot|\bYeti(?:-[a-z]{1,30}|)|Catchpoint(?: bot|)|[Cc]harlotte|Daumoa(?:-feedfetcher|)|(?:[a-zA-Z]{1,30}-|)Googlebot(?:-[a-zA-Z]{1,30}|))", "", {"pider-", "bingbot", "Yeti-", "Yeti", "Catchpoint bot", "Catchpoint", "harlotte", "Daumoa-feedfetcher", "Daumoa", "Googlebot-", "Googlebot"}); } From 0bf9c78fb48a378b91326390fa490a5327ce5004 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 13 Mar 2023 18:54:43 +0100 Subject: [PATCH 146/559] Add two more bugs --- .../0_stateless/02683_native_too_large_size.sql | Bin 0 -> 5701 bytes tests/queries/0_stateless/02684_bson.sql | Bin 0 -> 8635 bytes 2 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/queries/0_stateless/02683_native_too_large_size.sql create mode 100644 tests/queries/0_stateless/02684_bson.sql diff --git a/tests/queries/0_stateless/02683_native_too_large_size.sql b/tests/queries/0_stateless/02683_native_too_large_size.sql new file mode 100644 index 0000000000000000000000000000000000000000..57bbfed27df6cd990d614a07b0434d811ace6497 GIT binary patch literal 5701 zcmb_fdu$X%7{B&tuXlaWf`BF(4=q%nSbJA0)L<-kSGx83*xr?*L3DdJy<26|d$7KyyVj7Z_RK0&agvrPlP&B62D{*-k_`{M0$Va_jmyxP2tM`ZC zsG8b?r!q7dlVKjpNUF-Z?T)c8f^AK4BkJpon#1h{ zg8%_!)&&(y5Kz<1+5m*`U{erH&>@g%8w$tBdZSJ(?r}P<7pO2Zs^MAn%Snyvj)~iW zoJyOkXu-o^#DcfsDW^N7`ypcEr4m`mB?*|*>;jt2iqv@+j$-o{I zBuvxV!#xN^|Xyn4B2{C__SE13)SclSbCNQx(Xs3KA8WhSREvS@lRM5K2o(1)90I zA-wWN$E*cGjmpXpGZNG=99J?>N?MXGsI9|0Ibpi8txu7+P)Cq`+KpuF11ZCAz;?@f zg4&%)8hhDl)Rs&HRSyDylLbSBp`z(V?XqU>{(f0Yx8tqBKv)-o8N4T0%tj&hXRd|E znSvR`zys`4P!bSlFeBsP7fu?g)m<+ffSLlV?9&T|p$4q{wLy=VE{V25{LwuE6o7b1 zyR2$B!$g6SvNrfS1jljyib_gq6uZeBzaiHrjwVcT3?^hLW9*A*($)mYHm0yW9`A`D zWaWt%P?9)6aTX`@Bx4MB#q5F*R1}cV>_VKHMxdgCm?oh-s>t+~+WelIYo}7>8jRb4xyQo7`iXsNoA%zmUM!oJUYxvSU)xyj(BpMaZSYmvdr2L z9F9vVQ;#>o`cTxGpK)@r{XrHD??}GN(7^y^0JXKU&ttHF5*!)LO^M~J8vKh#9(wU& z+qYjm9oUjt{OhyZUb&;amxrm{Ylm033o5rk2O^MYU@^{_s z&c|z{EFnxr5De7d!Ck3bg)wD}Yw5_DL)zYQgV3HAvQOi^zV75rTOIPeD5uL&vqD3k247I#`rvHy4HN#iJmvcl^<1Uqh$GUWsIfG0Aa+EBX#*#k=-o#a&-yMK`8=_7Pmq`Nxe} zpj}(cX-g?F%n$HlM^Ub5#~f_2TJf6n(T?Fe!dag(N&*Q6Hiv~z__F5rU>B(`deqZL z$`1G1AyW6zWJV|qHO!Hka*BKwKP}9#(ne$HgmU#TcN()b9PB>-(TB4sRdxbwSkJ-M z6$SlO+8DEAuT@YrrOpnm*W}Q2o|a3kFP}xL=|WnnnZLXy`!39`(nit?vz?!V%NN!{ z4Qs)q{Ds+A6n7T##n2oaSeCs8d^^cVOIA z9Nkyit|x}t-PL~WY|31lF+DJHufu$HrQxnSj?Y(u zc+D$OP8GcyIaKA?iVe!4^bsY!Eje@^tRvSZr^2{DITU5=B9%X=%DKB0jhT%rcJ8kG zI(zB=dV7qk8g2dfw&2AR12^x=_kMDo4^E!H050#S{NcvQO;>MRnSS$f=|=z9;jyCQ ghaW$h-uG6`f;G>Dnf)Ac^CL~qo?O({F1yY5AAV15IRF3v literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/02684_bson.sql b/tests/queries/0_stateless/02684_bson.sql new file mode 100644 index 0000000000000000000000000000000000000000..4370eb47f415871b19e7fc71def12995e86398e6 GIT binary patch literal 8635 zcmeI1--}yC6vwC3AQq9^p!LN_k8Ef z+_^L7%;)U1mM`YB^~~PPY_(j3@8Mi>3K(JtgNg)rVf zJ(fA@cl~_mFLkI;CiahI>Ol-fE#^Vk8p|AEteNl9FxJh|s0XVwlk;OL9>y|K6Qa;TU`OJA5Cwd9n?PJy~;bm3ntG7JM*h)Z? zK_S8lKY3w zBKn_jJ{8qL8Bz;6UCKKm1RkY4f-srTZ|`rK+`4H>G>HuA?=`J^Rs0sswtXJa*eq@q zR$jeWk5Gt;LAatj6`~y52s_MgT4kQahar<2W-DhG!r(Y=gviq}4`2`cj+=mUcJ~TV zxzkj+YzIvoCWS}_0foatd*oOc%T9AaY_fkbh}(0J8rlikvaAC+p<}klU_S>fJe(A| zQ;j~Y9}8iF!3m~LJop8NMr2=CWlJm$sXcwFbd|-_o?jRk^FEFt_M^N;L&{xcE{I|{ zVTv?t*#tk#*#?|nML%qo$sT zHD4jap~3@DLbK zKmWb%D(NqVCSJLQzYVXXnSzg$4ijAA9u?KOvjB2-SljE$z_{q@<5Uw`7GL zEy?*_RZ<>B%Xrro1eSKn5`R((#Dnya+(v(48E?<5iSmer-+&u7pvTl!J*H&?ES*8N6hs0U zU}ImzwSLyH*0w>l6w(UR(h&co98{y|0jT8@ET+Y@R%AA7L+-0UGztkoqonaHt(1nd zJS)h)bdt?v&zVbjWB{76JAfG<5X<^mK^aYhWMwKKlViEU$DYNp+)K~G80`d#rIhLc zF@cjvCW&rPB$jb1lIPY`g2i$cL}^H3lsv;>X+wx>R-$bfjAnwsC=K9mvcb?7Wrw{e zxNtS(l}&2C2r3bCQYC4*!;-ovz`vNyio2+yq^XN5Ha~Zl*MnW@oYG)*psPmFEgFR? z4k1Prrv-3(d^K*BY+!3`M6I>;=#GMzwSs`GvJ_&`@-$$jnRf6>9=rdpPcczy0#n$>Uc>*Kp*=Azk4Jw#ao)ajPO-MX4*dK=}8T zu*rsD3f(v<)IHnXcJA!UpE@b@id&%m^2H0co$~H{^2$=^eCFS!(A%wD3${T03BpDf zs4qpYZ*Knl$MWC;bqqXhF(wa3S)fjcz-!0x+QISh7%noR@%#$jJq4ToJ(IJ0B7R6U zM;+Gn;V+tClm2HF?KfiDHj$j#$%_A!bK>i~Yg*CX0&f^t(dK~4*Q=ua;H}-midJlW ze+~RUtY{xUeKjlE7q=f;8CJCFHc6Z>5M}Zj42!^FMJv}N1ptRX9>m3wwBd}bm##)d z+kWHyop&5~Gw0wqQPEx)&!712=!Ivm498zjs8reVzUX$b;lukP@xMZHW3Me;#VZEV rb$DMS|Gy5mV0d3duHSvp$ Date: Mon, 13 Mar 2023 21:50:57 +0100 Subject: [PATCH 147/559] Update docs/ru/interfaces/third-party/gui.md --- docs/ru/interfaces/third-party/gui.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/interfaces/third-party/gui.md b/docs/ru/interfaces/third-party/gui.md index d06eada165b..34d2f0e371a 100644 --- a/docs/ru/interfaces/third-party/gui.md +++ b/docs/ru/interfaces/third-party/gui.md @@ -187,7 +187,7 @@ sidebar_label: "Визуальные интерфейсы от сторонни DataLens [доступен бесплатно](https://cloud.yandex.ru/docs/datalens/pricing), в том числе и для коммерческого использования. -- [Знакомство с DataLens]((https://youtu.be/57ngi_6BINE)). +- [Знакомство с DataLens](https://youtu.be/57ngi_6BINE). - [Чат сообщества DataLens](https://t.me/YandexDataLens) - [Документация DataLens](https://cloud.yandex.ru/docs/datalens/). - [Сценарий по визуализации данных из ClickHouse](https://cloud.yandex.ru/docs/solutions/datalens/data-from-ch-visualization). From f331b9b39890b7d19c658f20da219636b10d44bf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 13 Mar 2023 23:49:28 +0100 Subject: [PATCH 148/559] Fix errors and add tests --- .../AggregateFunctionGroupBitmapData.h | 8 ++++++++ .../AggregateFunctionHistogram.h | 3 +++ src/Core/Settings.h | 1 + .../Serializations/SerializationArray.cpp | 15 +++++++++++++++ src/DataTypes/Serializations/SerializationMap.cpp | 7 +++++++ .../Serializations/SerializationString.cpp | 10 ++++++++++ src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + 8 files changed, 46 insertions(+) diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index 801526432ae..6a4c48f0fff 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -20,6 +20,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int TOO_LARGE_ARRAY_SIZE; +} + enum BitmapKind { Small = 0, @@ -113,6 +118,9 @@ public: { size_t size; readVarUInt(size, in); + static constexpr size_t max_size = 1_GiB; + if (size > max_size) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size in groupBitmap."); std::unique_ptr buf(new char[size]); in.readStrict(buf.get(), size); rb = std::make_shared(RoaringBitmap::read(buf.get())); diff --git a/src/AggregateFunctions/AggregateFunctionHistogram.h b/src/AggregateFunctions/AggregateFunctionHistogram.h index 35e5f241ec9..62ed071856a 100644 --- a/src/AggregateFunctions/AggregateFunctionHistogram.h +++ b/src/AggregateFunctions/AggregateFunctionHistogram.h @@ -292,6 +292,9 @@ public: readVarUInt(size, buf); if (size > max_bins * 2) throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too many bins"); + static constexpr size_t max_size = 1_GiB; + if (size > max_size) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size in histogram."); buf.readStrict(reinterpret_cast(points), size * sizeof(WeightedValue)); } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 881cbe42a02..70e039f42fc 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -843,6 +843,7 @@ class IColumn; /** 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) \ M(UInt64, format_binary_max_string_size, 1_GiB, "The maximum allowed size for String in RowBinary format. It prevents allocating large amount of memory in case of corrupted data. 0 means there is no limit", 0) \ + M(UInt64, format_binary_max_array_size, 1_GiB, "The maximum allowed size for Array in RowBinary format. It prevents allocating large amount of memory in case of corrupted data. 0 means there is no limit", 0) \ M(URI, format_avro_schema_registry_url, "", "For AvroConfluent format: Confluent Schema Registry URL.", 0) \ \ M(Bool, output_format_json_quote_64bit_integers, true, "Controls quoting of 64-bit integers in JSON output format.", 0) \ diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index 73b232690c7..5ed85cd1d1a 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -43,6 +43,14 @@ void SerializationArray::deserializeBinary(Field & field, ReadBuffer & istr, con { size_t size; readVarUInt(size, istr); + if (settings.max_binary_array_size && size > settings.max_binary_array_size) + throw Exception( + ErrorCodes::TOO_LARGE_ARRAY_SIZE, + "Too large array size: {}. The maximum is: {}. To increase the maximum, use setting " + "format_binary_max_array_size", + size, + settings.max_binary_array_size); + field = Array(); Array & arr = field.get(); arr.reserve(size); @@ -75,6 +83,13 @@ void SerializationArray::deserializeBinary(IColumn & column, ReadBuffer & istr, size_t size; readVarUInt(size, istr); + if (settings.max_binary_array_size && size > settings.max_binary_array_size) + throw Exception( + ErrorCodes::TOO_LARGE_ARRAY_SIZE, + "Too large array size: {}. The maximum is: {}. To increase the maximum, use setting " + "format_binary_max_array_size", + size, + settings.max_binary_array_size); IColumn & nested_column = column_array.getData(); diff --git a/src/DataTypes/Serializations/SerializationMap.cpp b/src/DataTypes/Serializations/SerializationMap.cpp index 34da0f11cae..7381f586093 100644 --- a/src/DataTypes/Serializations/SerializationMap.cpp +++ b/src/DataTypes/Serializations/SerializationMap.cpp @@ -54,6 +54,13 @@ void SerializationMap::deserializeBinary(Field & field, ReadBuffer & istr, const { size_t size; readVarUInt(size, istr); + if (settings.max_binary_array_size && size > settings.max_binary_array_size) + throw Exception( + ErrorCodes::TOO_LARGE_ARRAY_SIZE, + "Too large map size: {}. The maximum is: {}. To increase the maximum, use setting " + "format_binary_max_array_size", + size, + settings.max_binary_array_size); field = Map(); Map & map = field.get(); map.reserve(size); diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index 96608fbbc04..76e71322b95 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -14,6 +14,8 @@ #include #include +#include + #ifdef __SSE2__ #include #endif @@ -158,6 +160,14 @@ static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnSt UInt64 size; readVarUInt(size, istr); + static constexpr size_t max_string_size = 16_GiB; /// Arbitrary value to prevent logical errors and overflows, but large enough. + if (size > max_string_size) + throw Exception( + ErrorCodes::TOO_LARGE_STRING_SIZE, + "Too large string size: {}. The maximum is: {}.", + size, + max_string_size); + offset += size + 1; offsets.push_back(offset); diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 9db85ced580..cadef4b9e99 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -193,6 +193,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.bson.output_string_as_string = settings.output_format_bson_string_as_string; format_settings.bson.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_bson_skip_fields_with_unsupported_types_in_schema_inference; format_settings.max_binary_string_size = settings.format_binary_max_string_size; + format_settings.max_binary_array_size = settings.format_binary_max_array_size; format_settings.native.allow_types_conversion = settings.input_format_native_allow_types_conversion; format_settings.max_parser_depth = context->getSettingsRef().max_parser_depth; format_settings.client_protocol_version = context->getClientProtocolVersion(); diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index ef6be805bea..84bc4e53326 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -82,6 +82,7 @@ struct FormatSettings Float32 input_allow_errors_ratio = 0; UInt64 max_binary_string_size = 0; + UInt64 max_binary_array_size = 0; UInt64 client_protocol_version = 0; UInt64 max_parser_depth = DBMS_DEFAULT_MAX_PARSER_DEPTH; From b218d523f6039b93c3b083e1fe47847ed379f5af Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 14 Mar 2023 01:22:27 +0100 Subject: [PATCH 149/559] Add a bug --- .../02683_native_too_large_size.reference | 0 .../02683_native_too_large_size.sql | Bin 5701 -> 5742 bytes .../queries/0_stateless/02684_bson.reference | 0 tests/queries/0_stateless/02684_bson.sql | Bin 8635 -> 8725 bytes tests/queries/0_stateless/02685_bson2.sql | Bin 0 -> 21295 bytes 5 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/queries/0_stateless/02683_native_too_large_size.reference create mode 100644 tests/queries/0_stateless/02684_bson.reference create mode 100644 tests/queries/0_stateless/02685_bson2.sql diff --git a/tests/queries/0_stateless/02683_native_too_large_size.reference b/tests/queries/0_stateless/02683_native_too_large_size.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02683_native_too_large_size.sql b/tests/queries/0_stateless/02683_native_too_large_size.sql index 57bbfed27df6cd990d614a07b0434d811ace6497..e8752477f2d0697d35ad86be1cd1b755e2ae993f 100644 GIT binary patch delta 50 zcmX@A^G;`ji(5yjUT%B0067Br`uxAu%bxv_v5@52{2V xC9x#Yii>N*E=6Sppyq0Y;?$zD)FRiSqWmHSPd{h>pdeT0ka!oz5J!btE&v|qB8>n5 delta 10 RcmbR0vfFv1#V$ogE&v#U1E&B0 diff --git a/tests/queries/0_stateless/02685_bson2.sql b/tests/queries/0_stateless/02685_bson2.sql new file mode 100644 index 0000000000000000000000000000000000000000..55334f212b18fc66992f2e62e89115cad3af2eff GIT binary patch literal 21295 zcmeI4du$v>9mltnBf>PWIne|pDz?n&rfpF91F2Pk-^}bh z_IyF{C9WN3?+<(DJCB{6`OSQG_BXqoom$4z3bx~*f@dyhUfHqdT@4*7psHusmZs0x zHBU1vKZ#b5n&q-f)500+)8&wBn>E}! z(?-#gPRHaCWR%Lh6-DwC^eIn143}IU)o-yKR$rnGA~JcUCv-=zxICDdfLhPn z&a5ztNk>0m6!Lb>DIn2Pd89jqa@r~x))IB1?C7qr-l5`LeY%W0^qw3M>_3_qTmJ{8OQmX~5t28|Ipmnw1m2Gv7tExaoqcIk{M=Cq9EC^F(p)_( z*Mwd|$+GUa$m5NhgqHVD%u2|(S;LxTnKEw5p0`X}FA8PM_IATaF7U1#p0bRCv?K61 z9d``q1KpGd;E3qGj5}@?{jn?x4NN8(mly$p$%3Uw*_KB#jTjyu{nLh59)Y3359h>oALCVIAxBn54A+AhCX1|+7~s3{OpctaZk1}#O}zfTPA=S8;BwwB7`o|?i#m0v zKv=+9U}SP~BJDV0p2#Ds2nGtXIKEC?$JAIpC?svm0>lJGn3}4#<)XZ&JLiZ>*Oxqu z04l*^Xw~W14Ge5N*~*dFT=}IDbC_@pYIE4gh%)XJ2_V?T5;0i=Kw1USS^-te7r1L& zPxtc3nS+8!AF+a1TvD%8b)$rlR39}|Y#{PEiKU*J9Hko156UPi8nue7u=4~ouy~9! zsi;Zhh-|}-U76HNJRSzZo@7>BpHD6}K5(PKh?IjYbs`RT0IjY2Hz-QfzYkCf+jqva zYpWkq#hrkHe-UyYKn@j^-vuaFfD+{cP1*I&(>LKXDv|IlfO5^dA?OeLOiI~=V;TRI zhkx?X6Dnh;vgb-D3{wcwozm|9;y~}IXIWN4e_XVpj_2ur&MOj(z(tY+CE~XL4XK>@ zG}jX12V`Cstg29?pgm0oQZS7`H(>=UEy($Ll~A6Fg7I<~6PSb}g7|SFC@Kse!fl!! zhj?n0w~0s*5gJMNdYtd<4;~+frWItTh1GMNs*LJ1BmtZV5lJ9KC#n)5!_TDwRie4_ z_2>E(HOO)WS4s_%oEA8IUnhF6xQ;Ff3gdz;iWUf$lMuTit+D@FIQwE@!W(P3S1BzvK z>_eFbIg%;F$EKDqu{*0TdG{QU_uMW+F03ScFevVyAfUTonnhnAAe? zHA-SCshqXdQJ>QoxOHh6Zhe`xJrK4NJ|Wq>8YsDxK#-wk&jY5&!kSE4>(m8Wk@yu3KG) z`sJ60fBVZX#1A${SF81pAzfh#7RZ%naiNBET}O27Cc`MCD=dF$ODv17Cy$17g(WX- zxsa}~pruq=mvl9+{p&rqeeJy?&C&IT+d{g6afEbbce|Iv+BKvrf4JNPU2nfXq-#i5 z-D9iEbxqgEH%@+jaPi1=b9C+79?~_WE16?2pC`j1T`z>L&xUlx?pk_Z60Tj>-P-lA z?04Ptx0gPhZp$t-N7ubCF3<1!ox_=r2)}DPm>BcB9=!L3y<3WXk;Z=4*I|Tlf1&~x zpb)=nCpOZu=T!${u`=w$C>o|^%sx zHI$sRn{k!Jvth{{D8!X~j7*x0J85x?i3X%j+Kr$!bke54(G+yjVizQ_5P9G zHKp|bJuwzKY5fgSbo7Q!T5#gBWi)9Lm=Zx*2`BAmqb>I7vo9gZNqbyI@wd-xo*n%6 zBh}_m{M1$cP`ThOP#b^xKKJ$q&YmBN@rDJ0;t||>K8j(xtXtN1RCL3{EH1#laMLcu z65TzqdD!r1VG(bc8|-eYg8M%adk=ywgLAsqSv4#GxUo`{?iz&YL8kO?pwg>JY20XJ zW7RpgQbi%fnxpi3+e&rmPg_VzK<1ub^>L{eL#U1yZazHs5vK_3LVz z^QFUPY%ut@me~BguUt$vZ~4@Qsc;1yuAo!l=zQpRX86I7oFPn$Dx)Oc6Iug73cWt{-b86e#X0jod;q5j0WIN=tC*3Wr(CbGb zmoLQ0+;TkRa_D4UCtJAYo|wDh=_l?wDRKF8GMAtK#mC=#MBQ?8b6h^9Esx7zDs7$; zYvw-zRN0#OlkU9>3xE07fyQg*4oE4jnfa4#GST@@wmtX=gd0oG((iT{3;NdhaGUE4 z`q8clPqE=`aAN_j6prytLSUIp9$_OlCx7kdTqcLh Date: Tue, 14 Mar 2023 02:31:04 +0100 Subject: [PATCH 150/559] Fix style --- src/DataTypes/Serializations/SerializationMap.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/DataTypes/Serializations/SerializationMap.cpp b/src/DataTypes/Serializations/SerializationMap.cpp index 7381f586093..a176d4c0719 100644 --- a/src/DataTypes/Serializations/SerializationMap.cpp +++ b/src/DataTypes/Serializations/SerializationMap.cpp @@ -20,6 +20,7 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_READ_MAP_FROM_TEXT; + extern const int TOO_LARGE_ARRAY_SIZE; } SerializationMap::SerializationMap(const SerializationPtr & key_, const SerializationPtr & value_, const SerializationPtr & nested_) From e03618357e8b19b5025c70d9ad4d01a7ff674a73 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 13 Mar 2023 14:42:06 +0000 Subject: [PATCH 151/559] Cleanup: Acquire lock in CacheBase::maxSize() The upcoming TTLPolicy will likely have a runtime-configurable cache size. --- src/Common/CacheBase.h | 4 ++-- src/Common/ICachePolicy.h | 2 +- src/Common/LRUCachePolicy.h | 2 +- src/Common/SLRUCachePolicy.h | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Common/CacheBase.h b/src/Common/CacheBase.h index 204e21a1e89..89aa78b390c 100644 --- a/src/Common/CacheBase.h +++ b/src/Common/CacheBase.h @@ -188,9 +188,9 @@ public: } size_t maxSize() const - TSA_NO_THREAD_SAFETY_ANALYSIS // disabled because max_size_in_bytes of cache_policy is a constant parameter { - return cache_policy->maxSize(); + std::lock_guard lock(mutex); + return cache_policy->maxSize(lock); } virtual ~CacheBase() = default; diff --git a/src/Common/ICachePolicy.h b/src/Common/ICachePolicy.h index a9dff7a99c1..f14e5b30c4f 100644 --- a/src/Common/ICachePolicy.h +++ b/src/Common/ICachePolicy.h @@ -27,7 +27,7 @@ public: virtual size_t weight(std::lock_guard & /* cache_lock */) const = 0; virtual size_t count(std::lock_guard & /* cache_lock */) const = 0; - virtual size_t maxSize() const = 0; + virtual size_t maxSize(std::lock_guard& /* cache_lock */) const = 0; virtual void reset(std::lock_guard & /* cache_lock */) = 0; virtual void remove(const Key & key, std::lock_guard & /* cache_lock */) = 0; diff --git a/src/Common/LRUCachePolicy.h b/src/Common/LRUCachePolicy.h index 19cd94be1d3..e308d016587 100644 --- a/src/Common/LRUCachePolicy.h +++ b/src/Common/LRUCachePolicy.h @@ -44,7 +44,7 @@ public: return cells.size(); } - size_t maxSize() const override + size_t maxSize(std::lock_guard & /* cache_lock */) const override { return max_size_in_bytes; } diff --git a/src/Common/SLRUCachePolicy.h b/src/Common/SLRUCachePolicy.h index fed958d3aff..a568d7f9c7f 100644 --- a/src/Common/SLRUCachePolicy.h +++ b/src/Common/SLRUCachePolicy.h @@ -49,7 +49,7 @@ public: return cells.size(); } - size_t maxSize() const override + size_t maxSize(std::lock_guard & /* cache_lock */) const override { return max_size_in_bytes; } From f1450923cbb3c836c833c1d3825ffae968894769 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 13 Mar 2023 15:28:03 +0000 Subject: [PATCH 152/559] Extend CacheBase with dump() method --- src/Common/CacheBase.h | 7 +++++++ src/Common/ICachePolicy.h | 8 ++++++++ src/Common/LRUCachePolicy.h | 9 +++++++++ src/Common/SLRUCachePolicy.h | 9 +++++++++ 4 files changed, 33 insertions(+) diff --git a/src/Common/CacheBase.h b/src/Common/CacheBase.h index 89aa78b390c..f38c3943f09 100644 --- a/src/Common/CacheBase.h +++ b/src/Common/CacheBase.h @@ -37,6 +37,7 @@ public: using Key = typename CachePolicy::Key; using Mapped = typename CachePolicy::Mapped; using MappedPtr = typename CachePolicy::MappedPtr; + using KeyMapped = typename CachePolicy::KeyMapped; /// Use this ctor if you don't care about the internal cache policy. explicit CacheBase(size_t max_size_in_bytes, size_t max_entries = 0, double size_ratio = 0.5) @@ -160,6 +161,12 @@ public: out_misses = misses; } + std::vector dump() const + { + std::lock_guard lock(mutex); + return cache_policy->dump(); + } + void reset() { std::lock_guard lock(mutex); diff --git a/src/Common/ICachePolicy.h b/src/Common/ICachePolicy.h index f14e5b30c4f..c9d5bb282f6 100644 --- a/src/Common/ICachePolicy.h +++ b/src/Common/ICachePolicy.h @@ -25,6 +25,12 @@ public: using MappedPtr = std::shared_ptr; using OnWeightLossFunction = std::function; + struct KeyMapped + { + Key key; + MappedPtr mapped; + }; + virtual size_t weight(std::lock_guard & /* cache_lock */) const = 0; virtual size_t count(std::lock_guard & /* cache_lock */) const = 0; virtual size_t maxSize(std::lock_guard& /* cache_lock */) const = 0; @@ -34,6 +40,8 @@ public: virtual MappedPtr get(const Key & key, std::lock_guard & /* cache_lock */) = 0; virtual void set(const Key & key, const MappedPtr & mapped, std::lock_guard & /* cache_lock */) = 0; + virtual std::vector dump() const = 0; + virtual ~ICachePolicy() = default; }; diff --git a/src/Common/LRUCachePolicy.h b/src/Common/LRUCachePolicy.h index e308d016587..d08434eb1bf 100644 --- a/src/Common/LRUCachePolicy.h +++ b/src/Common/LRUCachePolicy.h @@ -22,6 +22,7 @@ class LRUCachePolicy : public ICachePolicy; using typename Base::MappedPtr; + using typename Base::KeyMapped; using typename Base::OnWeightLossFunction; /** Initialize LRUCachePolicy with max_size_in_bytes and max_entries. @@ -116,6 +117,14 @@ public: removeOverflow(); } + std::vector dump() const override + { + std::vector res; + for (const auto & [key, cell] : cells) + res.push_back({key, cell.value}); + return res; + } + protected: using LRUQueue = std::list; using LRUQueueIterator = typename LRUQueue::iterator; diff --git a/src/Common/SLRUCachePolicy.h b/src/Common/SLRUCachePolicy.h index a568d7f9c7f..42ee9916805 100644 --- a/src/Common/SLRUCachePolicy.h +++ b/src/Common/SLRUCachePolicy.h @@ -24,6 +24,7 @@ class SLRUCachePolicy : public ICachePolicy; using typename Base::MappedPtr; + using typename Base::KeyMapped; using typename Base::OnWeightLossFunction; /** Initialize SLRUCachePolicy with max_size_in_bytes and max_protected_size. @@ -148,6 +149,14 @@ public: removeOverflow(probationary_queue, max_size_in_bytes, current_size_in_bytes, /*is_protected=*/false); } + std::vector dump() const override + { + std::vector res; + for (const auto & [key, cell] : cells) + res.push_back({key, cell.value}); + return res; + } + protected: using SLRUQueue = std::list; using SLRUQueueIterator = typename SLRUQueue::iterator; From e1fb25a004e44b9549f881206fef0e88a1b77328 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 13 Mar 2023 15:38:36 +0000 Subject: [PATCH 153/559] Extend CacheBase with getWithKey() method --- src/Common/CacheBase.h | 10 ++++++++++ src/Common/ICachePolicy.h | 4 ++++ src/Common/LRUCachePolicy.h | 18 +++++++++++++++--- src/Common/SLRUCachePolicy.h | 27 ++++++++++++++++++++++----- 4 files changed, 51 insertions(+), 8 deletions(-) diff --git a/src/Common/CacheBase.h b/src/Common/CacheBase.h index f38c3943f09..d9013973081 100644 --- a/src/Common/CacheBase.h +++ b/src/Common/CacheBase.h @@ -82,7 +82,17 @@ public: ++hits; else ++misses; + return res; + } + std::optional getWithKey(const Key & key) + { + std::lock_guard lock(mutex); + auto res = cache_policy->getWithKey(key, lock); + if (res.has_value()) + ++hits; + else + ++misses; return res; } diff --git a/src/Common/ICachePolicy.h b/src/Common/ICachePolicy.h index c9d5bb282f6..7f31af06296 100644 --- a/src/Common/ICachePolicy.h +++ b/src/Common/ICachePolicy.h @@ -37,7 +37,11 @@ public: virtual void reset(std::lock_guard & /* cache_lock */) = 0; virtual void remove(const Key & key, std::lock_guard & /* cache_lock */) = 0; + /// HashFunction usually hashes the entire key and the found key will be equal the provided key. In such cases, use get(). It is also + /// possible to store other, non-hashed data in the key. In that case, the found key is potentially different from the provided key. + /// Then use getWithKey() to also return the found key including it's non-hashed data. virtual MappedPtr get(const Key & key, std::lock_guard & /* cache_lock */) = 0; + virtual std::optional getWithKey(const Key &, std::lock_guard & /*cache_lock*/) = 0; virtual void set(const Key & key, const MappedPtr & mapped, std::lock_guard & /* cache_lock */) = 0; virtual std::vector dump() const = 0; diff --git a/src/Common/LRUCachePolicy.h b/src/Common/LRUCachePolicy.h index d08434eb1bf..0e1ef42474d 100644 --- a/src/Common/LRUCachePolicy.h +++ b/src/Common/LRUCachePolicy.h @@ -72,9 +72,7 @@ public: { auto it = cells.find(key); if (it == cells.end()) - { - return MappedPtr(); - } + return {}; Cell & cell = it->second; @@ -84,6 +82,20 @@ public: return cell.value; } + std::optional getWithKey(const Key & key, std::lock_guard & /*cache_lock*/) override + { + auto it = cells.find(key); + if (it == cells.end()) + return std::nullopt; + + Cell & cell = it->second; + + /// Move the key to the end of the queue. The iterator remains valid. + queue.splice(queue.end(), queue, cell.queue_iterator); + + return std::make_optional({it->first, cell.value}); + } + void set(const Key & key, const MappedPtr & mapped, std::lock_guard & /* cache_lock */) override { auto [it, inserted] = cells.emplace(std::piecewise_construct, diff --git a/src/Common/SLRUCachePolicy.h b/src/Common/SLRUCachePolicy.h index 42ee9916805..c16a9daaf8c 100644 --- a/src/Common/SLRUCachePolicy.h +++ b/src/Common/SLRUCachePolicy.h @@ -84,16 +84,12 @@ public: { auto it = cells.find(key); if (it == cells.end()) - { - return MappedPtr(); - } + return {}; Cell & cell = it->second; if (cell.is_protected) - { protected_queue.splice(protected_queue.end(), protected_queue, cell.queue_iterator); - } else { cell.is_protected = true; @@ -105,6 +101,27 @@ public: return cell.value; } + std::optional getWithKey(const Key & key, std::lock_guard & /*cache_lock*/) override + { + auto it = cells.find(key); + if (it == cells.end()) + return std::nullopt; + + Cell & cell = it->second; + + if (cell.is_protected) + protected_queue.splice(protected_queue.end(), protected_queue, cell.queue_iterator); + else + { + cell.is_protected = true; + current_protected_size += cell.size; + protected_queue.splice(protected_queue.end(), probationary_queue, cell.queue_iterator); + removeOverflow(protected_queue, max_protected_size, current_protected_size, /*is_protected=*/true); + } + + return std::make_optional({it->first, cell.value}); + } + void set(const Key & key, const MappedPtr & mapped, std::lock_guard & /* cache_lock */) override { auto [it, inserted] = cells.emplace(std::piecewise_construct, From b98579559393ae46f9379ea00e762246d2669161 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 13 Mar 2023 15:39:42 +0000 Subject: [PATCH 154/559] Cleanup: Group ICachePolicy methods --- src/Common/ICachePolicy.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Common/ICachePolicy.h b/src/Common/ICachePolicy.h index 7f31af06296..342ae58cd1c 100644 --- a/src/Common/ICachePolicy.h +++ b/src/Common/ICachePolicy.h @@ -35,15 +35,17 @@ public: virtual size_t count(std::lock_guard & /* cache_lock */) const = 0; virtual size_t maxSize(std::lock_guard& /* cache_lock */) const = 0; - virtual void reset(std::lock_guard & /* cache_lock */) = 0; - virtual void remove(const Key & key, std::lock_guard & /* cache_lock */) = 0; /// HashFunction usually hashes the entire key and the found key will be equal the provided key. In such cases, use get(). It is also /// possible to store other, non-hashed data in the key. In that case, the found key is potentially different from the provided key. /// Then use getWithKey() to also return the found key including it's non-hashed data. virtual MappedPtr get(const Key & key, std::lock_guard & /* cache_lock */) = 0; virtual std::optional getWithKey(const Key &, std::lock_guard & /*cache_lock*/) = 0; + virtual void set(const Key & key, const MappedPtr & mapped, std::lock_guard & /* cache_lock */) = 0; + virtual void remove(const Key & key, std::lock_guard & /* cache_lock */) = 0; + + virtual void reset(std::lock_guard & /* cache_lock */) = 0; virtual std::vector dump() const = 0; virtual ~ICachePolicy() = default; From eed365c8c8dc94cb25047fbc385fb6631ecbd4a9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 14 Mar 2023 09:11:36 +0000 Subject: [PATCH 155/559] Cleanup: Slightly more naming consistency (probably rolls back earlier changes) --- src/Common/CacheBase.h | 10 +++++----- src/Common/LRUCachePolicy.h | 12 ++++++------ src/Common/SLRUCachePolicy.h | 12 ++++++------ 3 files changed, 17 insertions(+), 17 deletions(-) diff --git a/src/Common/CacheBase.h b/src/Common/CacheBase.h index d9013973081..e1b933923b6 100644 --- a/src/Common/CacheBase.h +++ b/src/Common/CacheBase.h @@ -40,13 +40,13 @@ public: using KeyMapped = typename CachePolicy::KeyMapped; /// Use this ctor if you don't care about the internal cache policy. - explicit CacheBase(size_t max_size_in_bytes, size_t max_entries = 0, double size_ratio = 0.5) - : CacheBase("SLRU", max_size_in_bytes, max_entries, size_ratio) + explicit CacheBase(size_t max_size_in_bytes, size_t max_count = 0, double size_ratio = 0.5) + : CacheBase("SLRU", max_size_in_bytes, max_count, size_ratio) { } /// Use this ctor if you want the user to configure the cache policy via some setting. Supports only general-purpose policies LRU and SLRU. - explicit CacheBase(std::string_view cache_policy_name, size_t max_size_in_bytes, size_t max_entries = 0, double size_ratio = 0.5) + explicit CacheBase(std::string_view cache_policy_name, size_t max_size_in_bytes, size_t max_count = 0, double size_ratio = 0.5) { auto on_weight_loss_function = [&](size_t weight_loss) { onRemoveOverflowWeightLoss(weight_loss); }; @@ -58,12 +58,12 @@ public: if (cache_policy_name == "LRU") { using LRUPolicy = LRUCachePolicy; - cache_policy = std::make_unique(max_size_in_bytes, max_entries, on_weight_loss_function); + cache_policy = std::make_unique(max_size_in_bytes, max_count, on_weight_loss_function); } else if (cache_policy_name == "SLRU") { using SLRUPolicy = SLRUCachePolicy; - cache_policy = std::make_unique(max_size_in_bytes, max_entries, size_ratio, on_weight_loss_function); + cache_policy = std::make_unique(max_size_in_bytes, max_count, size_ratio, on_weight_loss_function); } else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown cache policy name: {}", cache_policy_name); diff --git a/src/Common/LRUCachePolicy.h b/src/Common/LRUCachePolicy.h index 0e1ef42474d..3128ac3001b 100644 --- a/src/Common/LRUCachePolicy.h +++ b/src/Common/LRUCachePolicy.h @@ -25,12 +25,12 @@ public: using typename Base::KeyMapped; using typename Base::OnWeightLossFunction; - /** Initialize LRUCachePolicy with max_size_in_bytes and max_entries. - * max_entries == 0 means no elements size restrictions. + /** Initialize LRUCachePolicy with max_size_in_bytes and max_count. + * max_count == 0 means no elements size restrictions. */ - LRUCachePolicy(size_t max_size_in_bytes_, size_t max_entries_, OnWeightLossFunction on_weight_loss_function_) + LRUCachePolicy(size_t max_size_in_bytes_, size_t max_count_, OnWeightLossFunction on_weight_loss_function_) : max_size_in_bytes(std::max(static_cast(1), max_size_in_bytes_)) - , max_entries(max_entries_) + , max_count(max_count_) , on_weight_loss_function(on_weight_loss_function_) { } @@ -157,7 +157,7 @@ protected: /// Total weight of values. size_t current_size_in_bytes = 0; const size_t max_size_in_bytes; - const size_t max_entries; + const size_t max_count; WeightFunction weight_function; OnWeightLossFunction on_weight_loss_function; @@ -167,7 +167,7 @@ protected: size_t current_weight_lost = 0; size_t queue_size = cells.size(); - while ((current_size_in_bytes > max_size_in_bytes || (max_entries != 0 && queue_size > max_entries)) && (queue_size > 0)) + while ((current_size_in_bytes > max_size_in_bytes || (max_count != 0 && queue_size > max_count)) && (queue_size > 0)) { const Key & key = queue.front(); diff --git a/src/Common/SLRUCachePolicy.h b/src/Common/SLRUCachePolicy.h index c16a9daaf8c..5325c7f4094 100644 --- a/src/Common/SLRUCachePolicy.h +++ b/src/Common/SLRUCachePolicy.h @@ -32,10 +32,10 @@ public: * max_protected_size == 0 means that the default protected size is equal to half of the total max size. */ /// TODO: construct from special struct with cache policy parameters (also with max_protected_size). - SLRUCachePolicy(size_t max_size_in_bytes_, size_t max_entries_, double size_ratio, OnWeightLossFunction on_weight_loss_function_) + SLRUCachePolicy(size_t max_size_in_bytes_, size_t max_count_, double size_ratio, OnWeightLossFunction on_weight_loss_function_) : max_protected_size(static_cast(max_size_in_bytes_ * std::min(1.0, size_ratio))) , max_size_in_bytes(max_size_in_bytes_) - , max_entries(max_entries_) + , max_count(max_count_) , on_weight_loss_function(on_weight_loss_function_) { } @@ -197,7 +197,7 @@ protected: size_t current_size_in_bytes = 0; const size_t max_protected_size; const size_t max_size_in_bytes; - const size_t max_entries; + const size_t max_count; WeightFunction weight_function; OnWeightLossFunction on_weight_loss_function; @@ -212,11 +212,11 @@ protected: { /// Check if after remove all elements from probationary part there will be no more than max elements /// in protected queue and weight of all protected elements will be less then max protected weight. - /// It's not possible to check only cells.size() > max_entries + /// It's not possible to check only cells.size() > max_count /// because protected elements move to probationary part and still remain in cache. need_remove = [&]() { - return ((max_entries != 0 && cells.size() - probationary_queue.size() > max_entries) + return ((max_count != 0 && cells.size() - probationary_queue.size() > max_count) || (current_weight_size > max_weight_size)) && (queue_size > 0); }; } @@ -224,7 +224,7 @@ protected: { need_remove = [&]() { - return ((max_entries != 0 && cells.size() > max_entries) + return ((max_count != 0 && cells.size() > max_count) || (current_weight_size > max_weight_size)) && (queue_size > 0); }; } From 73afae2d3ff7a768e123b9e4de4105b6fa9d450e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 13 Mar 2023 09:34:03 +0000 Subject: [PATCH 156/559] Introduce TTLCachePolicy --- src/Common/CacheBase.h | 16 +- src/Common/ICachePolicy.h | 10 ++ src/Common/LRUCachePolicy.h | 2 +- src/Common/SLRUCachePolicy.h | 2 +- src/Common/TTLCachePolicy.h | 148 ++++++++++++++++++ src/Interpreters/Cache/QueryCache.cpp | 133 +++++++--------- src/Interpreters/Cache/QueryCache.h | 65 ++++---- .../System/StorageSystemQueryCache.cpp | 8 +- 8 files changed, 262 insertions(+), 122 deletions(-) create mode 100644 src/Common/TTLCachePolicy.h diff --git a/src/Common/CacheBase.h b/src/Common/CacheBase.h index e1b933923b6..4ae313d7ecf 100644 --- a/src/Common/CacheBase.h +++ b/src/Common/CacheBase.h @@ -32,8 +32,10 @@ namespace ErrorCodes template , typename WeightFunction = EqualWeightFunction> class CacheBase { -public: +private: using CachePolicy = ICachePolicy; + +public: using Key = typename CachePolicy::Key; using Mapped = typename CachePolicy::Mapped; using MappedPtr = typename CachePolicy::MappedPtr; @@ -210,6 +212,18 @@ public: return cache_policy->maxSize(lock); } + void setMaxCount(size_t max_count) + { + std::lock_guard lock(mutex); + return cache_policy->setMaxCount(max_count, lock); + } + + void setMaxSize(size_t max_size_in_bytes) + { + std::lock_guard lock(mutex); + return cache_policy->setMaxSize(max_size_in_bytes, lock); + } + virtual ~CacheBase() = default; protected: diff --git a/src/Common/ICachePolicy.h b/src/Common/ICachePolicy.h index 342ae58cd1c..dca82095af1 100644 --- a/src/Common/ICachePolicy.h +++ b/src/Common/ICachePolicy.h @@ -1,5 +1,7 @@ #pragma once +#include + #include #include #include @@ -7,6 +9,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + template struct EqualWeightFunction { @@ -35,6 +42,9 @@ public: virtual size_t count(std::lock_guard & /* cache_lock */) const = 0; virtual size_t maxSize(std::lock_guard& /* cache_lock */) const = 0; + virtual void setMaxCount(size_t /*max_count*/, std::lock_guard & /* cache_lock */) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for cache policy"); } + virtual void setMaxSize(size_t /*max_size_in_bytes*/, std::lock_guard & /* cache_lock */) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for cache policy"); } + /// HashFunction usually hashes the entire key and the found key will be equal the provided key. In such cases, use get(). It is also /// possible to store other, non-hashed data in the key. In that case, the found key is potentially different from the provided key. /// Then use getWithKey() to also return the found key including it's non-hashed data. diff --git a/src/Common/LRUCachePolicy.h b/src/Common/LRUCachePolicy.h index 3128ac3001b..56974b18115 100644 --- a/src/Common/LRUCachePolicy.h +++ b/src/Common/LRUCachePolicy.h @@ -137,7 +137,7 @@ public: return res; } -protected: +private: using LRUQueue = std::list; using LRUQueueIterator = typename LRUQueue::iterator; diff --git a/src/Common/SLRUCachePolicy.h b/src/Common/SLRUCachePolicy.h index 5325c7f4094..e36bca83c61 100644 --- a/src/Common/SLRUCachePolicy.h +++ b/src/Common/SLRUCachePolicy.h @@ -174,7 +174,7 @@ public: return res; } -protected: +private: using SLRUQueue = std::list; using SLRUQueueIterator = typename SLRUQueue::iterator; diff --git a/src/Common/TTLCachePolicy.h b/src/Common/TTLCachePolicy.h new file mode 100644 index 00000000000..fc6367b807f --- /dev/null +++ b/src/Common/TTLCachePolicy.h @@ -0,0 +1,148 @@ +#pragma once + +#include + +#include + +namespace DB +{ + +/// TTLCachePolicy evicts entries for which IsStaleFunction returns true. +/// The cache size (in bytes and number of entries) can be changed at runtime. It is expected to set both sizes explicitly after construction. +template +class TTLCachePolicy : public ICachePolicy +{ +public: + using Base = ICachePolicy; + using typename Base::MappedPtr; + using typename Base::KeyMapped; + using typename Base::OnWeightLossFunction; + + TTLCachePolicy() + : max_size_in_bytes(0) + , max_count(0) + { + } + + size_t weight(std::lock_guard & /* cache_lock */) const override + { + return size_in_bytes; + } + + size_t count(std::lock_guard & /* cache_lock */) const override + { + return cache.size(); + } + + size_t maxSize(std::lock_guard & /* cache_lock */) const override + { + return max_size_in_bytes; + } + + void setMaxCount(size_t max_count_, std::lock_guard & /* cache_lock */) override + { + /// lazy behavior: the cache only shrinks upon the next insert + max_count = max_count_; + } + + void setMaxSize(size_t max_size_in_bytes_, std::lock_guard & /* cache_lock */) override + { + /// lazy behavior: the cache only shrinks upon the next insert + max_size_in_bytes = max_size_in_bytes_; + } + + void reset(std::lock_guard & /* cache_lock */) override + { + cache.clear(); + } + + void remove(const Key & key, std::lock_guard & /* cache_lock */) override + { + auto it = cache.find(key); + if (it == cache.end()) + return; + size_in_bytes -= weight_function(*it->second); + cache.erase(it); + } + + MappedPtr get(const Key & key, std::lock_guard & /* cache_lock */) override + { + auto it = cache.find(key); + if (it == cache.end()) + return {}; + return it->second; + } + + std::optional getWithKey(const Key & key, std::lock_guard & /* cache_lock */) override + { + auto it = cache.find(key); + if (it == cache.end()) + return std::nullopt; + return std::make_optional({it->first, it->second}); + } + + /// Evicts on a best-effort basis. If there are too many non-stale entries, the new entry may not be cached at all! + void set(const Key & key, const MappedPtr & mapped, std::lock_guard & /* cache_lock */) override + { + chassert(mapped.get()); + + const size_t entry_size_in_bytes = weight_function(*mapped); + + auto sufficient_space_in_cache = [&]() + { + return (size_in_bytes + entry_size_in_bytes <= max_size_in_bytes) && (cache.size() + 1 <= max_count); + }; + + if (!sufficient_space_in_cache()) + { + /// Remove stale entries + for (auto it = cache.begin(); it != cache.end();) + if (is_stale_function(it->first)) + { + size_in_bytes -= weight_function(*it->second); + it = cache.erase(it); + } + else + ++it; + } + + if (sufficient_space_in_cache()) + { + /// Insert or replace key + if (auto it = cache.find(key); it != cache.end()) + { + size_in_bytes -= weight_function(*it->second); + cache.erase(it); // stupid bug: (*) doesn't replace existing entries (likely due to custom hash function), need to erase explicitly + } + + cache[key] = std::move(mapped); // (*) + size_in_bytes += entry_size_in_bytes; + } + } + + std::vector dump() const override + { + std::vector res; + for (const auto & [key, mapped] : cache) + res.push_back({key, mapped}); + return res; + } + +private: + using Cache = std::unordered_map; + Cache cache; + + /// TODO To speed up removal of stale entries, we could also add another container sorted on expiry times which maps keys to iterators + /// into the cache. To insert an entry, add it to the cache + add the iterator to the sorted container. To remove stale entries, do a + /// binary search on the sorted container and erase all left of the found key. + + size_t size_in_bytes = 0; + size_t max_size_in_bytes; + size_t max_count; + + WeightFunction weight_function; + IsStaleFunction is_stale_function; + /// TODO support OnWeightLossFunction callback +}; + +} diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index b0c8766e505..ce2373a8af9 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include /// chassert @@ -152,43 +153,32 @@ size_t QueryCache::KeyHasher::operator()(const Key & key) const return res; } -size_t QueryCache::QueryResult::sizeInBytes() const +size_t QueryCache::QueryResultWeight::operator()(const QueryResult & chunks) const { size_t res = 0; - for (const auto & chunk : *chunks) + for (const auto & chunk : chunks) res += chunk.allocatedBytes(); return res; -}; +} -namespace -{ - -auto is_stale = [](const QueryCache::Key & key) +bool QueryCache::IsStale::operator()(const Key & key) const { return (key.expires_at < std::chrono::system_clock::now()); }; -} - -QueryCache::Writer::Writer(std::mutex & mutex_, Cache & cache_, const Key & key_, - size_t & cache_size_in_bytes_, size_t max_cache_size_in_bytes_, - size_t max_cache_entries_, +QueryCache::Writer::Writer(Cache & cache_, const Key & key_, size_t max_entry_size_in_bytes_, size_t max_entry_size_in_rows_, std::chrono::milliseconds min_query_runtime_) - : mutex(mutex_) - , cache(cache_) + : cache(cache_) , key(key_) - , cache_size_in_bytes(cache_size_in_bytes_) - , max_cache_size_in_bytes(max_cache_size_in_bytes_) - , max_cache_entries(max_cache_entries_) , max_entry_size_in_bytes(max_entry_size_in_bytes_) , max_entry_size_in_rows(max_entry_size_in_rows_) , min_query_runtime(min_query_runtime_) { - if (auto it = cache.find(key); it != cache.end() && !is_stale(it->first)) + if (auto entry = cache.getWithKey(key); entry.has_value() && !IsStale()(entry->key)) { skip_insert = true; /// Key already contained in cache and did not expire yet --> don't replace it - LOG_TRACE(&Poco::Logger::get("QueryResultCache"), "Skipped insert (non-stale entry found), query: {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (non-stale entry found), query: {}", key.queryStringFromAst()); } } @@ -197,18 +187,20 @@ void QueryCache::Writer::buffer(Chunk && partial_query_result) if (skip_insert) return; - auto & chunks = query_result.chunks; + std::lock_guard lock(mutex); - chunks->emplace_back(std::move(partial_query_result)); + auto & chunks = *query_result; - new_entry_size_in_bytes += chunks->back().allocatedBytes(); - new_entry_size_in_rows += chunks->back().getNumRows(); + chunks.emplace_back(std::move(partial_query_result)); + + new_entry_size_in_bytes += chunks.back().allocatedBytes(); + new_entry_size_in_rows += chunks.back().getNumRows(); if ((new_entry_size_in_bytes > max_entry_size_in_bytes) || (new_entry_size_in_rows > max_entry_size_in_rows)) { - chunks->clear(); /// eagerly free some space + chunks.clear(); /// eagerly free some space skip_insert = true; - LOG_TRACE(&Poco::Logger::get("QueryResultCache"), "Skipped insert (query result too big), new_entry_size_in_bytes: {} ({}), new_entry_size_in_rows: {} ({}), query: {}", new_entry_size_in_bytes, max_entry_size_in_bytes, new_entry_size_in_rows, max_entry_size_in_rows, key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (query result too big), new_entry_size_in_bytes: {} ({}), new_entry_size_in_rows: {} ({}), query: {}", new_entry_size_in_bytes, max_entry_size_in_bytes, new_entry_size_in_rows, max_entry_size_in_rows, key.queryStringFromAst()); } } @@ -217,81 +209,47 @@ void QueryCache::Writer::finalizeWrite() if (skip_insert) return; - if (std::chrono::duration_cast(std::chrono::system_clock::now() - query_start_time) < min_query_runtime) - { - LOG_TRACE(&Poco::Logger::get("QueryResultCache"), "Skipped insert (query not expensive enough), query: {}", key.queryStringFromAst()); - return; - } - std::lock_guard lock(mutex); - if (auto it = cache.find(key); it != cache.end() && !is_stale(it->first)) + if (std::chrono::duration_cast(std::chrono::system_clock::now() - query_start_time) < min_query_runtime) { - /// same check as in ctor because a parallel Writer could have inserted the current key in the meantime - LOG_TRACE(&Poco::Logger::get("QueryResultCache"), "Skipped insert (non-stale entry found), query: {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (query not expensive enough), query: {}", key.queryStringFromAst()); return; } - auto sufficient_space_in_cache = [this]() TSA_REQUIRES(mutex) + if (auto entry = cache.getWithKey(key); entry.has_value() && !IsStale()(entry->key)) { - return (cache_size_in_bytes + new_entry_size_in_bytes <= max_cache_size_in_bytes) && (cache.size() + 1 <= max_cache_entries); - }; - - if (!sufficient_space_in_cache()) - { - size_t removed_items = 0; - /// Remove stale entries - for (auto it = cache.begin(); it != cache.end();) - if (is_stale(it->first)) - { - cache_size_in_bytes -= it->second.sizeInBytes(); - it = cache.erase(it); - ++removed_items; - } - else - ++it; - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Removed {} stale entries", removed_items); + /// same check as in ctor because a parallel Writer could have inserted the current key in the meantime + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (non-stale entry found), query: {}", key.queryStringFromAst()); + return; } - if (!sufficient_space_in_cache()) - LOG_TRACE(&Poco::Logger::get("QueryResultCache"), "Skipped insert (cache has insufficient space), query: {}", key.queryStringFromAst()); - else - { - //// Insert or replace key - cache_size_in_bytes += query_result.sizeInBytes(); - if (auto it = cache.find(key); it != cache.end()) - cache_size_in_bytes -= it->second.sizeInBytes(); // key replacement - - cache[key] = std::move(query_result); - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Stored result of query {}", key.queryStringFromAst()); - } + cache.set(key, query_result); } -QueryCache::Reader::Reader(const Cache & cache_, const Key & key, size_t & cache_size_in_bytes_, const std::lock_guard &) +QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guard &) { - auto it = cache_.find(key); + auto entry = cache_.getWithKey(key); - if (it == cache_.end()) + if (!entry.has_value()) { LOG_TRACE(&Poco::Logger::get("QueryCache"), "No entry found for query {}", key.queryStringFromAst()); return; } - if (it->first.username.has_value() && it->first.username != key.username) + if (entry->key.username.has_value() && entry->key.username != key.username) { LOG_TRACE(&Poco::Logger::get("QueryCache"), "Inaccessible entry found for query {}", key.queryStringFromAst()); return; } - if (is_stale(it->first)) + if (IsStale()(entry->key)) { - cache_size_in_bytes_ -= it->second.sizeInBytes(); - const_cast(cache_).erase(it); - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Stale entry found and removed for query {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Stale entry found for query {}", key.queryStringFromAst()); return; } - pipe = Pipe(std::make_shared(it->first.header, it->second.chunks)); + pipe = Pipe(std::make_shared(entry->key.header, entry->mapped)); LOG_TRACE(&Poco::Logger::get("QueryCache"), "Entry found for query {}", key.queryStringFromAst()); } @@ -316,19 +274,19 @@ Pipe && QueryCache::Reader::getPipe() QueryCache::Reader QueryCache::createReader(const Key & key) { std::lock_guard lock(mutex); - return Reader(cache, key, cache_size_in_bytes, lock); + return Reader(cache, key, lock); } QueryCache::Writer QueryCache::createWriter(const Key & key, std::chrono::milliseconds min_query_runtime) { std::lock_guard lock(mutex); - return Writer(mutex, cache, key, cache_size_in_bytes, max_cache_size_in_bytes, max_cache_entries, max_cache_entry_size_in_bytes, max_cache_entry_size_in_rows, min_query_runtime); + return Writer(cache, key, max_entry_size_in_bytes, max_entry_size_in_rows, min_query_runtime); } void QueryCache::reset() { + cache.reset(); std::lock_guard lock(mutex); - cache.clear(); times_executed.clear(); cache_size_in_bytes = 0; } @@ -344,13 +302,28 @@ size_t QueryCache::recordQueryRun(const Key & key) return times; } +std::vector QueryCache::dump() const +{ + return cache.dump(); +} + +QueryCache::QueryCache() + : cache(std::make_unique>()) +{ +} + void QueryCache::updateConfiguration(const Poco::Util::AbstractConfiguration & config) { std::lock_guard lock(mutex); - max_cache_size_in_bytes = config.getUInt64("query_cache.size", 1_GiB); - max_cache_entries = config.getUInt64("query_cache.max_entries", 1024); - max_cache_entry_size_in_bytes = config.getUInt64("query_cache.max_entry_size", 1_MiB); - max_cache_entry_size_in_rows = config.getUInt64("query_cache.max_entry_rows", 30'000'000); + + size_t max_size_in_bytes = config.getUInt64("query_cache.size", 1_GiB); + cache.setMaxSize(max_size_in_bytes); + + size_t max_entries = config.getUInt64("query_cache.max_entries", 1024); + cache.setMaxCount(max_entries); + + max_entry_size_in_bytes = config.getUInt64("query_cache.max_entry_size", 1_MiB); + max_entry_size_in_rows = config.getUInt64("query_cache.max_entry_rows", 30'000'000); } } diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index 66477d77dcb..763e797ac07 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -41,7 +42,7 @@ public: /// Result metadata for constructing the pipe. const Block header; - /// Std::nullopt means that the associated entry can be read by other users. In general, sharing is a bad idea: First, it is + /// std::nullopt means that the associated entry can be read by other users. In general, sharing is a bad idea: First, it is /// unlikely that different users pose the same queries. Second, sharing potentially breaches security. E.g. User A should not be /// able to bypass row policies on some table by running the same queries as user B for whom no row policies exist. const std::optional username; @@ -57,15 +58,7 @@ public: String queryStringFromAst() const; }; - struct QueryResult - { - std::shared_ptr chunks = std::make_shared(); - size_t sizeInBytes() const; - - /// Notes: 1. For performance reasons, we cache the original result chunks as-is (no concatenation during cache insert or lookup). - /// 2. Ref-counting (shared_ptr) ensures that eviction of an entry does not affect queries which still read from the cache. - /// (this can also be achieved by copying the chunks during lookup but that would be under the cache lock --> too slow) - }; + using QueryResult = Chunks; private: struct KeyHasher @@ -73,8 +66,18 @@ private: size_t operator()(const Key & key) const; }; + struct QueryResultWeight + { + size_t operator()(const QueryResult & chunks) const; + }; + + struct IsStale + { + bool operator()(const Key & key) const; + }; + /// query --> query result - using Cache = std::unordered_map; + using Cache = CacheBase; /// query --> query execution count using TimesExecuted = std::unordered_map; @@ -97,24 +100,19 @@ public: void buffer(Chunk && partial_query_result); void finalizeWrite(); private: - std::mutex & mutex; - Cache & cache TSA_GUARDED_BY(mutex); + std::mutex mutex; + Cache & cache; const Key key; - size_t & cache_size_in_bytes TSA_GUARDED_BY(mutex); - const size_t max_cache_size_in_bytes; - const size_t max_cache_entries; - size_t new_entry_size_in_bytes = 0; + size_t new_entry_size_in_bytes TSA_GUARDED_BY(mutex) = 0; const size_t max_entry_size_in_bytes; - size_t new_entry_size_in_rows = 0; + size_t new_entry_size_in_rows TSA_GUARDED_BY(mutex) = 0; const size_t max_entry_size_in_rows; const std::chrono::time_point query_start_time = std::chrono::system_clock::now(); /// Writer construction and finalizeWrite() coincide with query start/end const std::chrono::milliseconds min_query_runtime; - QueryResult query_result; + std::shared_ptr query_result TSA_GUARDED_BY(mutex) = std::make_shared(); std::atomic skip_insert = false; - Writer(std::mutex & mutex_, Cache & cache_, const Key & key_, - size_t & cache_size_in_bytes_, size_t max_cache_size_in_bytes_, - size_t max_cache_entries_, + Writer(Cache & cache_, const Key & key_, size_t max_entry_size_in_bytes_, size_t max_entry_size_in_rows_, std::chrono::milliseconds min_query_runtime_); @@ -128,11 +126,13 @@ public: bool hasCacheEntryForKey() const; Pipe && getPipe(); /// must be called only if hasCacheEntryForKey() returns true private: - Reader(const Cache & cache_, const Key & key, size_t & cache_size_in_bytes_, const std::lock_guard &); + Reader(Cache & cache_, const Key & key, const std::lock_guard &); Pipe pipe; friend class QueryCache; /// for createReader() }; + QueryCache(); + void updateConfiguration(const Poco::Util::AbstractConfiguration & config); Reader createReader(const Key & key); @@ -143,23 +143,18 @@ public: /// Record new execution of query represented by key. Returns number of executions so far. size_t recordQueryRun(const Key & key); + /// For debugging and system tables + std::vector dump() const; + private: - /// Implementation note: The query result implements a custom caching mechanism and doesn't make use of CacheBase, unlike many other - /// internal caches in ClickHouse. The main reason is that we don't need standard CacheBase (S)LRU eviction as the expiry times - /// associated with cache entries provide a "natural" eviction criterion. As a future TODO, we could make an expiry-based eviction - /// policy and use that with CacheBase (e.g. see #23706) - /// TODO To speed up removal of stale entries, we could also add another container sorted on expiry times which maps keys to iterators - /// into the cache. To insert an entry, add it to the cache + add the iterator to the sorted container. To remove stale entries, do a - /// binary search on the sorted container and erase all left of the found key. + Cache cache; + mutable std::mutex mutex; - Cache cache TSA_GUARDED_BY(mutex); TimesExecuted times_executed TSA_GUARDED_BY(mutex); /// Cache configuration - size_t max_cache_size_in_bytes TSA_GUARDED_BY(mutex) = 0; - size_t max_cache_entries TSA_GUARDED_BY(mutex) = 0; - size_t max_cache_entry_size_in_bytes TSA_GUARDED_BY(mutex) = 0; - size_t max_cache_entry_size_in_rows TSA_GUARDED_BY(mutex) = 0; + size_t max_entry_size_in_bytes TSA_GUARDED_BY(mutex) = 0; + size_t max_entry_size_in_rows TSA_GUARDED_BY(mutex) = 0; size_t cache_size_in_bytes TSA_GUARDED_BY(mutex) = 0; /// Updated in each cache insert/delete diff --git a/src/Storages/System/StorageSystemQueryCache.cpp b/src/Storages/System/StorageSystemQueryCache.cpp index 2de8e4594b9..2cbcc773ad6 100644 --- a/src/Storages/System/StorageSystemQueryCache.cpp +++ b/src/Storages/System/StorageSystemQueryCache.cpp @@ -33,11 +33,11 @@ void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr if (!query_cache) return; + std::vector content = query_cache->dump(); + const String & username = context->getUserName(); - std::lock_guard lock(query_cache->mutex); - - for (const auto & [key, result] : query_cache->cache) + for (const auto & [key, query_result] : content) { /// Showing other user's queries is considered a security risk if (key.username.has_value() && key.username != username) @@ -48,7 +48,7 @@ void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr res_columns[2]->insert(std::chrono::system_clock::to_time_t(key.expires_at)); res_columns[3]->insert(key.expires_at < std::chrono::system_clock::now()); res_columns[4]->insert(!key.username.has_value()); - res_columns[5]->insert(result.sizeInBytes()); + res_columns[5]->insert(QueryCache::QueryResultWeight()(*query_result)); } } From e8f42e5fb3f8da8d792cd8d3274293cf7c59b09f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Mar 2023 11:26:35 +0000 Subject: [PATCH 157/559] Fix aliases in analyzeAggregateProjection. Remove setting from test. --- src/Interpreters/ActionsDAG.cpp | 8 +++++--- .../02521_cannot_find_column_in_projection.sql | 2 -- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 174dbe8611e..9270a6bb951 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -795,9 +795,11 @@ ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_mapaddInput(new_input->result_name, new_input->result_type); - if (!rename.empty() && new_input->result_name != rename) - node = &dag->addAlias(*node, rename); + bool should_rename = !rename.empty() && new_input->result_name != rename; + const auto & input_name = should_rename ? rename : new_input->result_name; + node = &dag->addInput(input_name, new_input->result_type); + if (should_rename) + node = &dag->addAlias(*node, new_input->result_name); } stack.pop_back(); diff --git a/tests/queries/0_stateless/02521_cannot_find_column_in_projection.sql b/tests/queries/0_stateless/02521_cannot_find_column_in_projection.sql index 255c6f56ab3..6ee8ec07178 100644 --- a/tests/queries/0_stateless/02521_cannot_find_column_in_projection.sql +++ b/tests/queries/0_stateless/02521_cannot_find_column_in_projection.sql @@ -1,5 +1,3 @@ -SET allow_experimental_analyzer = 1; - drop table if exists test; create table test(day Date, id UInt32) engine=MergeTree partition by day order by tuple(); insert into test select toDate('2023-01-05') AS day, number from numbers(10); From a32d7a00bc14c57e3669cd29d3a7b6347cee5a0e Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Tue, 14 Mar 2023 06:44:05 -0700 Subject: [PATCH 158/559] Fix SipHash integer hashing and byte order issue in GenerateRandom for s390x --- src/Common/SipHash.h | 11 ++++- src/Storages/StorageGenerateRandom.cpp | 65 ++++++++++++++++---------- src/Storages/StorageGenerateRandom.h | 2 +- 3 files changed, 52 insertions(+), 26 deletions(-) diff --git a/src/Common/SipHash.h b/src/Common/SipHash.h index 404f4390aec..8f2a5e1a48f 100644 --- a/src/Common/SipHash.h +++ b/src/Common/SipHash.h @@ -164,7 +164,16 @@ public: template ALWAYS_INLINE void update(const T & x) { - update(reinterpret_cast(&x), sizeof(x)); /// NOLINT + if constexpr (std::endian::native == std::endian::big) + { + T rev_x = x; + char *start = reinterpret_cast(&rev_x); + char *end = start + sizeof(T); + std::reverse(start, end); + update(reinterpret_cast(&rev_x), sizeof(rev_x)); /// NOLINT + } + else + update(reinterpret_cast(&x), sizeof(x)); /// NOLINT } ALWAYS_INLINE void update(const std::string & x) diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index 601306bd1bf..03f3aff412f 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -50,16 +50,33 @@ namespace ErrorCodes namespace { -void fillBufferWithRandomData(char * __restrict data, size_t size, pcg64 & rng) +void fillBufferWithRandomData(char * __restrict data, size_t limit, size_t size_of_type, pcg64 & rng, [[maybe_unused]] bool flip_bytes = false) { + size_t size = limit * size_of_type; char * __restrict end = data + size; while (data < end) { /// The loop can be further optimized. UInt64 number = rng(); +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + unalignedStoreLE(data, number); +#else unalignedStore(data, number); +#endif data += sizeof(UInt64); /// We assume that data has at least 7-byte padding (see PaddedPODArray) } +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + if (flip_bytes) + { + data = end - size; + while (data < end) + { + char * rev_end = data + size_of_type; + std::reverse(data, rev_end); + data += size_of_type; + } + } +#endif } @@ -216,7 +233,7 @@ ColumnPtr fillColumnWithRandomData( { auto column = ColumnUInt8::create(); column->getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit * sizeof(UInt8), rng); + fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit, sizeof(UInt8), rng); return column; } case TypeIndex::UInt16: [[fallthrough]]; @@ -224,7 +241,7 @@ ColumnPtr fillColumnWithRandomData( { auto column = ColumnUInt16::create(); column->getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit * sizeof(UInt16), rng); + fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit, sizeof(UInt16), rng, true); return column; } case TypeIndex::Date32: @@ -242,28 +259,28 @@ ColumnPtr fillColumnWithRandomData( { auto column = ColumnUInt32::create(); column->getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit * sizeof(UInt32), rng); + fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit, sizeof(UInt32), rng, true); return column; } case TypeIndex::UInt64: { auto column = ColumnUInt64::create(); column->getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit * sizeof(UInt64), rng); + fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit, sizeof(UInt64), rng, true); return column; } case TypeIndex::UInt128: { auto column = ColumnUInt128::create(); column->getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit * sizeof(UInt128), rng); + fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit, sizeof(UInt128), rng, true); return column; } case TypeIndex::UInt256: { auto column = ColumnUInt256::create(); column->getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit * sizeof(UInt256), rng); + fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit, sizeof(UInt256), rng); return column; } case TypeIndex::UUID: @@ -271,63 +288,63 @@ ColumnPtr fillColumnWithRandomData( auto column = ColumnUUID::create(); column->getData().resize(limit); /// NOTE This is slightly incorrect as random UUIDs should have fixed version 4. - fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit * sizeof(UUID), rng); + fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit, sizeof(UUID), rng); return column; } case TypeIndex::Int8: { auto column = ColumnInt8::create(); column->getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit * sizeof(Int8), rng); + fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit, sizeof(Int8), rng); return column; } case TypeIndex::Int16: { auto column = ColumnInt16::create(); column->getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit * sizeof(Int16), rng); + fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit, sizeof(Int16), rng, true); return column; } case TypeIndex::Int32: { auto column = ColumnInt32::create(); column->getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit * sizeof(Int32), rng); + fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit, sizeof(Int32), rng, true); return column; } case TypeIndex::Int64: { auto column = ColumnInt64::create(); column->getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit * sizeof(Int64), rng); + fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit, sizeof(Int64), rng, true); return column; } case TypeIndex::Int128: { auto column = ColumnInt128::create(); column->getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit * sizeof(Int128), rng); + fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit, sizeof(Int128), rng, true); return column; } case TypeIndex::Int256: { auto column = ColumnInt256::create(); column->getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit * sizeof(Int256), rng); + fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit, sizeof(Int256), rng, true); return column; } case TypeIndex::Float32: { auto column = ColumnFloat32::create(); column->getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit * sizeof(Float32), rng); + fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit, sizeof(Float32), rng, true); return column; } case TypeIndex::Float64: { auto column = ColumnFloat64::create(); column->getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit * sizeof(Float64), rng); + fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit, sizeof(Float64), rng, true); return column; } case TypeIndex::Decimal32: @@ -335,7 +352,7 @@ ColumnPtr fillColumnWithRandomData( auto column = type->createColumn(); auto & column_concrete = typeid_cast &>(*column); column_concrete.getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column_concrete.getData().data()), limit * sizeof(Decimal32), rng); + fillBufferWithRandomData(reinterpret_cast(column_concrete.getData().data()), limit, sizeof(Decimal32), rng, true); return column; } case TypeIndex::Decimal64: /// TODO Decimal may be generated out of range. @@ -343,7 +360,7 @@ ColumnPtr fillColumnWithRandomData( auto column = type->createColumn(); auto & column_concrete = typeid_cast &>(*column); column_concrete.getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column_concrete.getData().data()), limit * sizeof(Decimal64), rng); + fillBufferWithRandomData(reinterpret_cast(column_concrete.getData().data()), limit, sizeof(Decimal64), rng, true); return column; } case TypeIndex::Decimal128: @@ -351,7 +368,7 @@ ColumnPtr fillColumnWithRandomData( auto column = type->createColumn(); auto & column_concrete = typeid_cast &>(*column); column_concrete.getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column_concrete.getData().data()), limit * sizeof(Decimal128), rng); + fillBufferWithRandomData(reinterpret_cast(column_concrete.getData().data()), limit, sizeof(Decimal128), rng, true); return column; } case TypeIndex::Decimal256: @@ -359,7 +376,7 @@ ColumnPtr fillColumnWithRandomData( auto column = type->createColumn(); auto & column_concrete = typeid_cast &>(*column); column_concrete.getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column_concrete.getData().data()), limit * sizeof(Decimal256), rng); + fillBufferWithRandomData(reinterpret_cast(column_concrete.getData().data()), limit, sizeof(Decimal256), rng, true); return column; } case TypeIndex::FixedString: @@ -367,7 +384,7 @@ ColumnPtr fillColumnWithRandomData( size_t n = typeid_cast(*type).getN(); auto column = ColumnFixedString::create(n); column->getChars().resize(limit * n); - fillBufferWithRandomData(reinterpret_cast(column->getChars().data()), limit * n, rng); + fillBufferWithRandomData(reinterpret_cast(column->getChars().data()), limit, n, rng); return column; } case TypeIndex::DateTime64: @@ -401,14 +418,14 @@ ColumnPtr fillColumnWithRandomData( { auto column = ColumnIPv4::create(); column->getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit * sizeof(IPv4), rng); + fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit, sizeof(IPv4), rng); return column; } case TypeIndex::IPv6: { auto column = ColumnIPv6::create(); column->getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit * sizeof(IPv6), rng); + fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit, sizeof(IPv6), rng); return column; } @@ -473,7 +490,7 @@ StorageGenerateRandom::StorageGenerateRandom( const String & comment, UInt64 max_array_length_, UInt64 max_string_length_, - std::optional random_seed_) + const std::optional & random_seed_) : IStorage(table_id_), max_array_length(max_array_length_), max_string_length(max_string_length_) { static constexpr size_t MAX_ARRAY_SIZE = 1 << 30; diff --git a/src/Storages/StorageGenerateRandom.h b/src/Storages/StorageGenerateRandom.h index 6b050c07e52..e59e2250c82 100644 --- a/src/Storages/StorageGenerateRandom.h +++ b/src/Storages/StorageGenerateRandom.h @@ -17,7 +17,7 @@ public: const String & comment, UInt64 max_array_length, UInt64 max_string_length, - std::optional random_seed); + const std::optional & random_seed); std::string getName() const override { return "GenerateRandom"; } From 01be209e431a4921de7629c54ab069e3a92fc779 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 14 Mar 2023 17:44:02 +0100 Subject: [PATCH 159/559] fix test --- src/Common/OptimizedRegularExpression.cpp | 4 +++- src/Common/tests/gtest_optimize_re.cpp | 10 ++++++---- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index 4d0cdae1b99..22cd0958c42 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -359,7 +360,7 @@ finish: } } - if (max_length >= MIN_LENGTH_FOR_STRSTR || !first_call) + if (max_length >= MIN_LENGTH_FOR_STRSTR || (!first_call && max_length > 0)) { required_substring.literal = candidate_it->first; required_substring.prefix = candidate_it->second == 0; @@ -430,6 +431,7 @@ OptimizedRegularExpressionImpl::OptimizedRegularExpressionImpl(cons std::vector alternatives; /// this vector collects patterns in (xx|xx|xx). for now it's not used. analyze(regexp_, required_substring, is_trivial, required_substring_is_prefix, alternatives); + /// Just three following options are supported if (options & (~(RE_CASELESS | RE_NO_CAPTURE | RE_DOT_NL))) throw DB::Exception(DB::ErrorCodes::CANNOT_COMPILE_REGEXP, "OptimizedRegularExpression: Unsupported option."); diff --git a/src/Common/tests/gtest_optimize_re.cpp b/src/Common/tests/gtest_optimize_re.cpp index 9de652da365..eb837a2f344 100644 --- a/src/Common/tests/gtest_optimize_re.cpp +++ b/src/Common/tests/gtest_optimize_re.cpp @@ -4,7 +4,7 @@ TEST(OptimizeRE, analyze) { - auto test_f = [](const std::string & regexp, const std::string & answer, std::vector expect_alternatives = {}) + auto test_f = [](const std::string & regexp, const std::string & answer, std::vector expect_alternatives = {}, bool trival_expected = false) { std::string required; bool is_trivial; @@ -14,8 +14,10 @@ TEST(OptimizeRE, analyze) std::cerr << regexp << std::endl; EXPECT_EQ(required, answer); EXPECT_EQ(alternatives, expect_alternatives); + EXPECT_EQ(is_trivial, trival_expected); }; - test_f("abc", "abc"); + test_f("abc", "abc", {}, true); + test_f("c([^k]*)de", ""); test_f("abc(de)fg", "abcdefg"); test_f("abc(de|xyz)fg", "abc", {"abcdefg", "abcxyzfg"}); test_f("abc(de?f|xyz)fg", "abc", {"abcd", "abcxyzfg"}); @@ -35,8 +37,8 @@ TEST(OptimizeRE, analyze) test_f(R"(abc(de|xyz|(\{xx\}))fg)", "abc", {"abcdefg", "abcxyzfg", "abc{xx}fg"}); test_f("abc(abc|fg)?xyzz", "xyzz"); test_f("abc(abc|fg){0,1}xyzz", "xyzz"); - test_f("abc(abc|fg)xyzz|bcdd?k|bc(f|g|h?)z", "", {"abcabcxyzz", "abcfgxyzz", "bcd", "bcfz", "bcgz", "bcz"}); - test_f("abc(abc|fg)xyzz|bc(dd?x|kk?y|(f))k|bc(f|g|h?)z", "", {"abcabcxyzz", "abcfgxyzz", "bcd", "bck", "bcfk", "bcfz", "bcgz", "bcz"}); + test_f("abc(abc|fg)xyzz|bcdd?k|bc(f|g|h?)z", "", {"abcabcxyzz", "abcfgxyzz", "bcd", "bcfz", "bcgz", ""}); + test_f("abc(abc|fg)xyzz|bc(dd?x|kk?y|(f))k|bc(f|g|h?)z", "", {"abcabcxyzz", "abcfgxyzz", "bcd", "bck", "bcfk", "bcfz", "bcgz", ""}); test_f("((?:abc|efg|xyz)/[a-zA-Z0-9]{1-50})(/?[^ ]*|)", "", {"abc/", "efg/", "xyz/"}); test_f(R"([Bb]ai[Dd]u[Ss]pider(?:-[A-Za-z]{1,30})(?:-[A-Za-z]{1,30}|)|bingbot|\bYeti(?:-[a-z]{1,30}|)|Catchpoint(?: bot|)|[Cc]harlotte|Daumoa(?:-feedfetcher|)|(?:[a-zA-Z]{1,30}-|)Googlebot(?:-[a-zA-Z]{1,30}|))", "", {"pider-", "bingbot", "Yeti-", "Yeti", "Catchpoint bot", "Catchpoint", "harlotte", "Daumoa-feedfetcher", "Daumoa", "Googlebot-", "Googlebot"}); } From 076d33bb03e13e6dfae1c111ba591196b651ab77 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 14 Mar 2023 18:15:42 +0100 Subject: [PATCH 160/559] refine a little bit --- src/Common/OptimizedRegularExpression.cpp | 2 +- src/Common/tests/gtest_optimize_re.cpp | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index 22cd0958c42..c2354f14bcb 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -386,7 +386,7 @@ finish: bool next_is_trivial = true; pos = analyzeImpl(regexp, pos, required_substring, next_is_trivial, next_alternatives); /// For xxx|xxx|xxx, we only conbine the alternatives and return a empty required_substring. - if (next_alternatives.empty()) + if (next_alternatives.empty() || shortest_alter_length(next_alternatives) < required_substring.literal.size()) { global_alters.push_back(required_substring); } diff --git a/src/Common/tests/gtest_optimize_re.cpp b/src/Common/tests/gtest_optimize_re.cpp index eb837a2f344..088993cfa84 100644 --- a/src/Common/tests/gtest_optimize_re.cpp +++ b/src/Common/tests/gtest_optimize_re.cpp @@ -37,8 +37,10 @@ TEST(OptimizeRE, analyze) test_f(R"(abc(de|xyz|(\{xx\}))fg)", "abc", {"abcdefg", "abcxyzfg", "abc{xx}fg"}); test_f("abc(abc|fg)?xyzz", "xyzz"); test_f("abc(abc|fg){0,1}xyzz", "xyzz"); - test_f("abc(abc|fg)xyzz|bcdd?k|bc(f|g|h?)z", "", {"abcabcxyzz", "abcfgxyzz", "bcd", "bcfz", "bcgz", ""}); - test_f("abc(abc|fg)xyzz|bc(dd?x|kk?y|(f))k|bc(f|g|h?)z", "", {"abcabcxyzz", "abcfgxyzz", "bcd", "bck", "bcfk", "bcfz", "bcgz", ""}); + test_f("abc(abc|fg)xyzz|bcdd?k|bc(f|g|h?)z", "", {"abcabcxyzz", "abcfgxyzz", "bcd", "bc"}); + test_f("abc(abc|fg)xyzz|bc(dd?x|kk?y|(f))k|bc(f|g|h?)z", "", {"abcabcxyzz", "abcfgxyzz", "bcd", "bck", "bcfk", "bc"}); test_f("((?:abc|efg|xyz)/[a-zA-Z0-9]{1-50})(/?[^ ]*|)", "", {"abc/", "efg/", "xyz/"}); test_f(R"([Bb]ai[Dd]u[Ss]pider(?:-[A-Za-z]{1,30})(?:-[A-Za-z]{1,30}|)|bingbot|\bYeti(?:-[a-z]{1,30}|)|Catchpoint(?: bot|)|[Cc]harlotte|Daumoa(?:-feedfetcher|)|(?:[a-zA-Z]{1,30}-|)Googlebot(?:-[a-zA-Z]{1,30}|))", "", {"pider-", "bingbot", "Yeti-", "Yeti", "Catchpoint bot", "Catchpoint", "harlotte", "Daumoa-feedfetcher", "Daumoa", "Googlebot-", "Googlebot"}); + test_f("abc|(:?xx|yy|zz|x?)def", "", {"abc", "def"}); + test_f("abc|(:?xx|yy|zz|x?){1,2}def", "", {"abc", "def"}); } From 8db4964ebc38005ac356344e3d137089a3e833d9 Mon Sep 17 00:00:00 2001 From: SmitaRKulkarni <64093672+SmitaRKulkarni@users.noreply.github.com> Date: Tue, 14 Mar 2023 20:23:43 +0100 Subject: [PATCH 161/559] Revert "Revert "Backup_Restore_concurrency_check_node"" --- src/Backups/BackupCoordinationRemote.cpp | 9 ++++-- .../test_disallow_concurrency.py | 31 ++++++++++--------- 2 files changed, 22 insertions(+), 18 deletions(-) diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 8e43676f59c..48440ed66fb 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -750,9 +750,12 @@ bool BackupCoordinationRemote::hasConcurrentBackups(const std::atomic &) if (existing_backup_uuid == toString(backup_uuid)) continue; - const auto status = zk->get(root_zookeeper_path + "/" + existing_backup_path + "/stage"); - if (status != Stage::COMPLETED) - return true; + String status; + if (zk->tryGet(root_zookeeper_path + "/" + existing_backup_path + "/stage", status)) + { + if (status != Stage::COMPLETED) + return true; + } } zk->createIfNotExists(backup_stage_path, ""); diff --git a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py index 43c8adda65a..27e95096f5a 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py @@ -9,7 +9,8 @@ from helpers.test_tools import TSV, assert_eq_with_retry cluster = ClickHouseCluster(__file__) -num_nodes = 10 +num_nodes = 4 +ddl_task_timeout = 640 def generate_cluster_def(): @@ -85,7 +86,7 @@ def drop_after_test(): node0.query( "DROP TABLE IF EXISTS tbl ON CLUSTER 'cluster' NO DELAY", settings={ - "distributed_ddl_task_timeout": 360, + "distributed_ddl_task_timeout": ddl_task_timeout, }, ) @@ -107,7 +108,7 @@ def create_and_fill_table(): "ORDER BY x" ) for i in range(num_nodes): - nodes[i].query(f"INSERT INTO tbl SELECT number FROM numbers(40000000)") + nodes[i].query(f"INSERT INTO tbl SELECT number FROM numbers(80000000)") # All the tests have concurrent backup/restores with same backup names @@ -145,7 +146,7 @@ def test_concurrent_backups_on_same_node(): nodes[0].query( f"DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY", settings={ - "distributed_ddl_task_timeout": 360, + "distributed_ddl_task_timeout": ddl_task_timeout, }, ) nodes[0].query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}") @@ -202,7 +203,7 @@ def test_concurrent_restores_on_same_node(): nodes[0].query( f"DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY", settings={ - "distributed_ddl_task_timeout": 360, + "distributed_ddl_task_timeout": ddl_task_timeout, }, ) restore_id = ( @@ -226,44 +227,44 @@ def test_concurrent_restores_on_different_node(): backup_name = new_backup_name() id = ( - nodes[0] + nodes[1] .query(f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name} ASYNC") .split("\t")[0] ) assert_eq_with_retry( - nodes[0], + nodes[1], f"SELECT status FROM system.backups WHERE status == 'CREATING_BACKUP' AND id = '{id}'", "CREATING_BACKUP", ) assert_eq_with_retry( - nodes[0], + nodes[1], f"SELECT status FROM system.backups WHERE status == 'BACKUP_CREATED' AND id = '{id}'", "BACKUP_CREATED", ) - nodes[0].query( + nodes[1].query( f"DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY", settings={ - "distributed_ddl_task_timeout": 360, + "distributed_ddl_task_timeout": ddl_task_timeout, }, ) restore_id = ( - nodes[0] + nodes[1] .query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name} ASYNC") .split("\t")[0] ) assert_eq_with_retry( - nodes[0], - f"SELECT status FROM system.backups WHERE status == 'RESTORING'", + nodes[1], + f"SELECT status FROM system.backups WHERE status == 'RESTORING' AND id == '{restore_id}'", "RESTORING", ) - assert "Concurrent restores not supported" in nodes[1].query_and_get_error( + assert "Concurrent restores not supported" in nodes[0].query_and_get_error( f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" ) assert_eq_with_retry( - nodes[0], + nodes[1], f"SELECT status FROM system.backups WHERE status == 'RESTORED' AND id == '{restore_id}'", "RESTORED", ) From e863e49a498865782e6363fabe24b39191735607 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 14 Mar 2023 20:26:19 +0100 Subject: [PATCH 162/559] Renamed cluster configuration xml to cluster_for_disallow_concurrency_test.xml for disallow_concurreny of backup and restore test --- .../test_backup_restore_on_cluster/test_disallow_concurrency.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py index 27e95096f5a..6d0b62e79f2 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py @@ -16,7 +16,7 @@ ddl_task_timeout = 640 def generate_cluster_def(): path = os.path.join( os.path.dirname(os.path.realpath(__file__)), - "./_gen/cluster_for_concurrency_test.xml", + "./_gen/cluster_for_disallow_concurrency_test.xml", ) os.makedirs(os.path.dirname(path), exist_ok=True) with open(path, "w") as f: From 8cd36d2a360be5a8fc52ffb5877ea611e982ffe0 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 14 Mar 2023 20:30:44 +0100 Subject: [PATCH 163/559] Removed generated file cluster_for_concurrency_test.xml --- .../_gen/cluster_for_concurrency_test.xml | 48 ------------------- 1 file changed, 48 deletions(-) delete mode 100644 tests/integration/test_backup_restore_on_cluster/_gen/cluster_for_concurrency_test.xml diff --git a/tests/integration/test_backup_restore_on_cluster/_gen/cluster_for_concurrency_test.xml b/tests/integration/test_backup_restore_on_cluster/_gen/cluster_for_concurrency_test.xml deleted file mode 100644 index 08684e34e45..00000000000 --- a/tests/integration/test_backup_restore_on_cluster/_gen/cluster_for_concurrency_test.xml +++ /dev/null @@ -1,48 +0,0 @@ - - - - - - node0 - 9000 - - - node1 - 9000 - - - node2 - 9000 - - - node3 - 9000 - - - node4 - 9000 - - - node5 - 9000 - - - node6 - 9000 - - - node7 - 9000 - - - node8 - 9000 - - - node9 - 9000 - - - - - \ No newline at end of file From eddda2eb7381eeb45568cb8a7b1eacc01095d9a4 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Tue, 14 Mar 2023 22:10:08 +0000 Subject: [PATCH 164/559] Add SSL authentication to the native protocol --- src/Server/TCPHandler.cpp | 16 ++++ .../configs/ssl_config.xml | 2 +- .../test_ssl_cert_authentication/test.py | 89 +++++++++++++++++++ 3 files changed, 106 insertions(+), 1 deletion(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 27b3712ffd3..201220c3953 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -10,6 +10,8 @@ #include #include #include +#include +#include #include #include #include @@ -1224,6 +1226,20 @@ void TCPHandler::receiveHello() session = makeSession(); auto & client_info = session->getClientInfo(); + + /// Authentification with SSL user certificate + if (dynamic_cast(socket().impl())) + { + Poco::Net::SecureStreamSocket secure_socket(socket()); + if (secure_socket.havePeerCertificate()) + { + session->authenticate( + SSLCertificateCredentials{user, secure_socket.peerCertificate().commonName()}, + getClientAddress(client_info)); + return; + } + } + session->authenticate(user, password, getClientAddress(client_info)); } diff --git a/tests/integration/test_ssl_cert_authentication/configs/ssl_config.xml b/tests/integration/test_ssl_cert_authentication/configs/ssl_config.xml index 163449872be..ed3b2b595db 100644 --- a/tests/integration/test_ssl_cert_authentication/configs/ssl_config.xml +++ b/tests/integration/test_ssl_cert_authentication/configs/ssl_config.xml @@ -9,7 +9,7 @@ You have to configure certificate to enable this interface. See the openSSL section below. --> - + 9440 diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index 0246b835fd5..7c62ca0d8b6 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -1,9 +1,12 @@ import pytest +from helpers.client import Client from helpers.cluster import ClickHouseCluster from helpers.ssl_context import WrapSSLContextWithSNI import urllib.request, urllib.parse import ssl import os.path +from os import remove + # The test cluster is configured with certificate for that host name, see 'server-ext.cnf'. # The client have to verify server certificate against that name. Client uses SNI @@ -66,6 +69,54 @@ def execute_query_https( return response.decode("utf-8") +config = """ + + + none + + {certificateFile} + {privateKeyFile} + {caConfig} + + + AcceptCertificateHandler + + + +""" + + +def execute_query_native(node, query, user, cert_name): + + config_path = f"{SCRIPT_DIR}/configs/client.xml" + + formatted = config.format( + certificateFile=f"{SCRIPT_DIR}/certs/{cert_name}-cert.pem", + privateKeyFile=f"{SCRIPT_DIR}/certs/{cert_name}-key.pem", + caConfig=f"{SCRIPT_DIR}/certs/ca-cert.pem", + ) + + file = open(config_path, "w") + file.write(formatted) + file.close() + + client = Client( + node.ip_address, + 9440, + command=cluster.client_bin_path, + secure=True, + config=config_path, + ) + + try: + result = client.query(query, user=user) + remove(config_path) + return result + except: + remove(config_path) + raise + + def test_https(): assert ( execute_query_https("SELECT currentUser()", user="john", cert_name="client1") @@ -81,6 +132,27 @@ def test_https(): ) +def test_native(): + assert ( + execute_query_native( + instance, "SELECT currentUser()", user="john", cert_name="client1" + ) + == "john\n" + ) + assert ( + execute_query_native( + instance, "SELECT currentUser()", user="lucy", cert_name="client2" + ) + == "lucy\n" + ) + assert ( + execute_query_native( + instance, "SELECT currentUser()", user="lucy", cert_name="client3" + ) + == "lucy\n" + ) + + def test_https_wrong_cert(): # Wrong certificate: different user's certificate with pytest.raises(Exception) as err: @@ -107,6 +179,23 @@ def test_https_wrong_cert(): ) +def test_native_wrong_cert(): + # Wrong certificate: different user's certificate + with pytest.raises(Exception) as err: + execute_query_native( + instance, "SELECT currentUser()", user="john", cert_name="client2" + ) + assert "AUTHENTICATION_FAILED" in str(err.value) + + # Wrong certificate: self-signed certificate. + # In this case clickhouse-client itself will throw an error + with pytest.raises(Exception) as err: + execute_query_native( + instance, "SELECT currentUser()", user="john", cert_name="wrong" + ) + assert "UNKNOWN_CA" in str(err.value) + + def test_https_non_ssl_auth(): # Users with non-SSL authentication are allowed, in this case we can skip sending a client certificate at all (because "verificationMode" is set to "relaxed"). # assert execute_query_https("SELECT currentUser()", user="peter", enable_ssl_auth=False) == "peter\n" From f3908a9e84c51d0b43a20da920885815e2a134cb Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Tue, 14 Mar 2023 22:49:10 +0000 Subject: [PATCH 165/559] Fix style --- src/Server/TCPHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 201220c3953..ab1fb5fca1e 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1227,7 +1227,7 @@ void TCPHandler::receiveHello() session = makeSession(); auto & client_info = session->getClientInfo(); - /// Authentification with SSL user certificate + /// Authentication with SSL user certificate if (dynamic_cast(socket().impl())) { Poco::Net::SecureStreamSocket secure_socket(socket()); From 398978e0bc05e6866d2318e464e2ac919a6b3c86 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 15 Mar 2023 00:09:29 +0000 Subject: [PATCH 166/559] Fix build --- src/Server/TCPHandler.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index ab1fb5fca1e..6cc79533f78 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -10,8 +10,6 @@ #include #include #include -#include -#include #include #include #include @@ -51,6 +49,11 @@ #include #include +#if USE_SSL +# include +# include +#endif + #include "Core/Protocol.h" #include "Storages/MergeTree/RequestResponse.h" #include "TCPHandler.h" @@ -1227,6 +1230,7 @@ void TCPHandler::receiveHello() session = makeSession(); auto & client_info = session->getClientInfo(); +#if USE_SSL /// Authentication with SSL user certificate if (dynamic_cast(socket().impl())) { @@ -1239,6 +1243,7 @@ void TCPHandler::receiveHello() return; } } +#endif session->authenticate(user, password, getClientAddress(client_info)); } From 1393e9ba035c932487d82b3d0b52085b017b1954 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 15 Mar 2023 01:09:54 +0000 Subject: [PATCH 167/559] Add test --- .../0_stateless/02676_sqlite_local_path_check.reference | 1 + .../queries/0_stateless/02676_sqlite_local_path_check.sh | 8 ++++++++ 2 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/02676_sqlite_local_path_check.reference create mode 100755 tests/queries/0_stateless/02676_sqlite_local_path_check.sh diff --git a/tests/queries/0_stateless/02676_sqlite_local_path_check.reference b/tests/queries/0_stateless/02676_sqlite_local_path_check.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/02676_sqlite_local_path_check.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/02676_sqlite_local_path_check.sh b/tests/queries/0_stateless/02676_sqlite_local_path_check.sh new file mode 100755 index 00000000000..0cb78db0432 --- /dev/null +++ b/tests/queries/0_stateless/02676_sqlite_local_path_check.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_LOCAL} -q "SELECT * FROM sqlite('/nonexistent', 'table')" 2>&1 | grep -c "PATH_ACCESS_DENIED"; From 14a7c5a0199c4a406ade680306e8e5a40a9dd2a9 Mon Sep 17 00:00:00 2001 From: alexX512 Date: Wed, 15 Mar 2023 06:06:55 +0000 Subject: [PATCH 168/559] Add stop_reading_on_first_cancel setting --- src/Client/ClientBase.cpp | 37 +++-- src/Client/ClientBase.h | 2 +- src/Core/Settings.h | 1 + src/Processors/Executors/ExecutingGraph.cpp | 10 +- src/Processors/Executors/ExecutingGraph.h | 2 +- src/Processors/Executors/PipelineExecutor.cpp | 20 ++- src/Processors/Executors/PipelineExecutor.h | 6 +- .../PullingAsyncPipelineExecutor.cpp | 41 ++++-- .../Executors/PullingAsyncPipelineExecutor.h | 10 +- src/Server/TCPHandler.cpp | 132 +++++++++++------- src/Server/TCPHandler.h | 16 ++- 11 files changed, 186 insertions(+), 91 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 53eb5080130..cc74c18c870 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -173,7 +173,7 @@ static void incrementProfileEventsBlock(Block & dst, const Block & src) auto & dst_column_host_name = typeid_cast(*mutable_columns[name_pos["host_name"]]); auto & dst_array_current_time = typeid_cast(*mutable_columns[name_pos["current_time"]]).getData(); - auto & dst_array_type = typeid_cast(*mutable_columns[name_pos["type"]]).getData(); + auto & dst_array_type = typeid_cast(*mutable_columns[name_pos["type"]]).getData(); auto & dst_column_name = typeid_cast(*mutable_columns[name_pos["name"]]); auto & dst_array_value = typeid_cast(*mutable_columns[name_pos["value"]]).getData(); @@ -264,21 +264,31 @@ static void incrementProfileEventsBlock(Block & dst, const Block & src) } -std::atomic_flag exit_on_signal; +std::atomic exit_after_signals = 0; class QueryInterruptHandler : private boost::noncopyable { public: - static void start() { exit_on_signal.clear(); } + /// Store how much interrupt signals can be before stopping the query + /// by default stop after the first interrupt signal. + static void start(Int32 signals_before_stop = 1) { exit_after_signals.store(signals_before_stop); } + + /// Set value not greater then 0 to mark the query as stopped. + static void stop() { return exit_after_signals.store(0); } + /// Return true if the query was stopped. - static bool stop() { return exit_on_signal.test_and_set(); } - static bool cancelled() { return exit_on_signal.test(); } + /// Query was stoped if it received at least "signals_before_stop" interrupt signals. + static bool try_stop() { return exit_after_signals.fetch_sub(1) <= 0; } + static bool cancelled() { return exit_after_signals.load() <= 0; } + + /// Return how much interrupt signals remain before stop. + static Int32 cancelled_status() { return exit_after_signals.load(); } }; /// This signal handler is set only for SIGINT. void interruptSignalHandler(int signum) { - if (QueryInterruptHandler::stop()) + if (QueryInterruptHandler::try_stop()) safeExit(128 + signum); } @@ -853,12 +863,15 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa } } + const auto & settings = global_context->getSettingsRef(); + const Int32 signals_before_stop = settings.stop_reading_on_first_cancel ? 2 : 1; + int retries_left = 10; while (retries_left) { try { - QueryInterruptHandler::start(); + QueryInterruptHandler::start(signals_before_stop); SCOPE_EXIT({ QueryInterruptHandler::stop(); }); connection->sendQuery( @@ -875,7 +888,7 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa if (send_external_tables) sendExternalTables(parsed_query); - receiveResult(parsed_query); + receiveResult(parsed_query, signals_before_stop); break; } @@ -900,7 +913,7 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa /// Receives and processes packets coming from server. /// Also checks if query execution should be cancelled. -void ClientBase::receiveResult(ASTPtr parsed_query) +void ClientBase::receiveResult(ASTPtr parsed_query, Int32 signals_before_stop) { // TODO: get the poll_interval from commandline. const auto receive_timeout = connection_parameters.timeouts.receive_timeout; @@ -924,7 +937,11 @@ void ClientBase::receiveResult(ASTPtr parsed_query) /// to avoid losing sync. if (!cancelled) { - if (QueryInterruptHandler::cancelled()) + if (QueryInterruptHandler::cancelled_status() == signals_before_stop - 1) + { + connection->sendCancel(); + } + else if (QueryInterruptHandler::cancelled()) { cancelQuery(); } diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 52e15a1a075..e9e539d251e 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -131,7 +131,7 @@ protected: private: - void receiveResult(ASTPtr parsed_query); + void receiveResult(ASTPtr parsed_query, Int32 signals_before_stop); bool receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled_); void receiveLogsAndProfileEvents(ASTPtr parsed_query); bool receiveSampleBlock(Block & out, ColumnsDescription & columns_description, ASTPtr parsed_query); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index aeb6b5b1130..558fc95b446 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -277,6 +277,7 @@ class IColumn; \ M(Bool, final, false, "Query with the FINAL modifier by default. If the engine does not support final, it does not have any effect. On queries with multiple tables final is applied only on those that support it. It also works on distributed tables", 0) \ \ + M(Bool, stop_reading_on_first_cancel, false, "If setting is enabled, pipeline needs to return a result on a partially read table before the query is completely cancelled.", 0) \ /** Settings for testing hedged requests */ \ M(Milliseconds, sleep_in_send_tables_status_ms, 0, "Time to sleep in sending tables status response in TCPHandler", 0) \ M(Milliseconds, sleep_in_send_data_ms, 0, "Time to sleep in sending data in TCPHandler", 0) \ diff --git a/src/Processors/Executors/ExecutingGraph.cpp b/src/Processors/Executors/ExecutingGraph.cpp index eb06d45c81b..7af9fcda3b9 100644 --- a/src/Processors/Executors/ExecutingGraph.cpp +++ b/src/Processors/Executors/ExecutingGraph.cpp @@ -402,7 +402,7 @@ bool ExecutingGraph::updateNode(uint64_t pid, Queue & queue, Queue & async_queue return true; } -void ExecutingGraph::cancel(bool hard_cancel) +void ExecutingGraph::cancel(bool cancel_all_processors) { std::exception_ptr exception_ptr; @@ -413,7 +413,10 @@ void ExecutingGraph::cancel(bool hard_cancel) { try { - if (hard_cancel || source_processors.at(proc)) + /// Stop all processors in the general case, but in a specific case + /// where the pipeline needs to return a result on a partially read table, + /// stop only the processors that read from the source + if (cancel_all_processors || source_processors.at(proc)) { IProcessor * processor = processors->at(proc).get(); processor->cancel(); @@ -432,7 +435,8 @@ void ExecutingGraph::cancel(bool hard_cancel) tryLogCurrentException("ExecutingGraph"); } } - cancelled = true; + if (cancel_all_processors) + cancelled = true; } if (exception_ptr) diff --git a/src/Processors/Executors/ExecutingGraph.h b/src/Processors/Executors/ExecutingGraph.h index 971c1f0e128..69acc7fb31b 100644 --- a/src/Processors/Executors/ExecutingGraph.h +++ b/src/Processors/Executors/ExecutingGraph.h @@ -138,7 +138,7 @@ public: /// If processor wants to be expanded, lock will be upgraded to get write access to pipeline. bool updateNode(uint64_t pid, Queue & queue, Queue & async_queue); - void cancel(bool hard_cancel = false); + void cancel(bool cancel_all_processors = true); private: /// Add single edge to edges list. Check processor is known. diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index 923987417bd..fb9cfb5c509 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -67,12 +67,20 @@ const Processors & PipelineExecutor::getProcessors() const return graph->getProcessors(); } -void PipelineExecutor::cancel(bool hard_cancel) +void PipelineExecutor::cancel() { cancelled = true; - if (hard_cancel) - finish(); - graph->cancel(hard_cancel); + finish(); + graph->cancel(); +} + +void PipelineExecutor::cancelReading() +{ + if (!cancelled_reading) + { + cancelled_reading = true; + graph->cancel(/*cancel_all_processors*/ false); + } } void PipelineExecutor::finish() @@ -148,7 +156,7 @@ bool PipelineExecutor::checkTimeLimitSoft() // We call cancel here so that all processors are notified and tasks waken up // so that the "break" is faster and doesn't wait for long events if (!continuing) - cancel(/*hard_cancel*/ true); + cancel(); return continuing; } @@ -229,7 +237,7 @@ void PipelineExecutor::executeStepImpl(size_t thread_num, std::atomic_bool * yie break; if (!context.executeTask()) - cancel(/*hard_cancel*/ true); + cancel(); if (tasks.isFinished()) break; diff --git a/src/Processors/Executors/PipelineExecutor.h b/src/Processors/Executors/PipelineExecutor.h index a45a5b6a830..147e5b2744a 100644 --- a/src/Processors/Executors/PipelineExecutor.h +++ b/src/Processors/Executors/PipelineExecutor.h @@ -48,7 +48,10 @@ public: const Processors & getProcessors() const; /// Cancel execution. May be called from another thread. - void cancel(bool hard_cancel = false); + void cancel(); + + /// Cancel processors which only read data from source. May be called from another thread. + void cancelReading(); /// Checks the query time limits (cancelled or timeout). Throws on cancellation or when time limit is reached and the query uses "break" bool checkTimeLimit(); @@ -78,6 +81,7 @@ private: bool trace_processors = false; std::atomic_bool cancelled = false; + std::atomic_bool cancelled_reading = false; Poco::Logger * log = &Poco::Logger::get("PipelineExecutor"); diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index bc6ec198592..ae63b4242c6 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -174,15 +174,42 @@ bool PullingAsyncPipelineExecutor::pull(Block & block, uint64_t milliseconds) } void PullingAsyncPipelineExecutor::cancel() +{ + /// Cancel execution if it wasn't finished. + cancelWithExceptionHandling([&]() + { + data->executor->cancel(); + }); + + /// The following code is needed to rethrow exception from PipelineExecutor. + /// It could have been thrown from pull(), but we will not likely call it again. + + /// Join thread here to wait for possible exception. + if (data->thread.joinable()) + data->thread.join(); + + /// Rethrow exception to not swallow it in destructor. + data->rethrowExceptionIfHas(); +} + +void PullingAsyncPipelineExecutor::cancelReading() +{ + /// Stop reading from source if pipeline wasn't finished. + cancelWithExceptionHandling([&]() + { + data->executor->cancelReading(); + }); +} + +void PullingAsyncPipelineExecutor::cancelWithExceptionHandling(CancelFunc && cancel_func) { if (!data) return; - /// Cancel execution if it wasn't finished. try { if (!data->is_finished && data->executor) - data->executor->cancel(/*hard_cancel*/ true); + cancel_func(); } catch (...) { @@ -194,16 +221,6 @@ void PullingAsyncPipelineExecutor::cancel() data->has_exception = true; } } - - /// The following code is needed to rethrow exception from PipelineExecutor. - /// It could have been thrown from pull(), but we will not likely call it again. - - /// Join thread here to wait for possible exception. - if (data->thread.joinable()) - data->thread.join(); - - /// Rethrow exception to not swallow it in destructor. - data->rethrowExceptionIfHas(); } Chunk PullingAsyncPipelineExecutor::getTotals() diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.h b/src/Processors/Executors/PullingAsyncPipelineExecutor.h index 7e45246ffd6..dce88b48262 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.h +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.h @@ -32,9 +32,12 @@ public: bool pull(Chunk & chunk, uint64_t milliseconds = 0); bool pull(Block & block, uint64_t milliseconds = 0); - /// Stop execution. It is not necessary, but helps to stop execution before executor is destroyed. + /// Stop execution of all processors. It is not necessary, but helps to stop execution before executor is destroyed. void cancel(); + /// Stop processors which only read data from source. + void cancelReading(); + /// Get totals and extremes. Returns empty chunk if doesn't have any. Chunk getTotals(); Chunk getExtremes(); @@ -49,6 +52,11 @@ public: /// Internal executor data. struct Data; +private: + using CancelFunc = std::function; + + void cancelWithExceptionHandling(CancelFunc && cancel_func); + private: QueryPipeline & pipeline; std::shared_ptr lazy_format; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 6349c559c26..98e857a0059 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -112,6 +112,21 @@ namespace ErrorCodes extern const int QUERY_WAS_CANCELLED; } +std::string QueryState::cancellationStatusToName(CancellationStatus status) +{ + switch (status) + { + case CancellationStatus::FULLY_CANCELLED: + return "FULLY_CANCELLED"; + case CancellationStatus::READ_CANCELLED: + return "READ_CANCELLED"; + case CancellationStatus::NOT_CANCELLED: + return "NOT_CANCELLED"; + } + + UNREACHABLE(); +} + TCPHandler::TCPHandler(IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_, bool parse_proxy_protocol_, std::string server_display_name_) : Poco::Net::TCPServerConnection(socket_) , server(server_) @@ -369,7 +384,7 @@ void TCPHandler::runImpl() std::lock_guard lock(task_callback_mutex); - if (state.is_cancelled) + if (state.cancellation_status == CancellationStatus::FULLY_CANCELLED) return {}; sendReadTaskRequestAssumeLocked(); @@ -385,7 +400,7 @@ void TCPHandler::runImpl() CurrentMetrics::Increment callback_metric_increment(CurrentMetrics::MergeTreeAllRangesAnnouncementsSent); std::lock_guard lock(task_callback_mutex); - if (state.is_cancelled) + if (state.cancellation_status == CancellationStatus::FULLY_CANCELLED) return; sendMergeTreeAllRangesAnnounecementAssumeLocked(announcement); @@ -399,7 +414,7 @@ void TCPHandler::runImpl() CurrentMetrics::Increment callback_metric_increment(CurrentMetrics::MergeTreeReadTaskRequestsSent); std::lock_guard lock(task_callback_mutex); - if (state.is_cancelled) + if (state.cancellation_status == CancellationStatus::FULLY_CANCELLED) return std::nullopt; sendMergeTreeReadTaskRequestAssumeLocked(std::move(request)); @@ -417,7 +432,7 @@ void TCPHandler::runImpl() auto finish_or_cancel = [this]() { - if (state.is_cancelled) + if (state.cancellation_status == CancellationStatus::FULLY_CANCELLED) state.io.onCancelOrConnectionLoss(); else state.io.onFinish(); @@ -447,7 +462,7 @@ void TCPHandler::runImpl() { std::scoped_lock lock(task_callback_mutex, fatal_error_mutex); - if (isQueryCancelled()) + if (getQueryCancellationStatus() == CancellationStatus::FULLY_CANCELLED) return true; sendProgress(); @@ -666,7 +681,7 @@ bool TCPHandler::readDataNext() { LOG_INFO(log, "Client has dropped the connection, cancel the query."); state.is_connection_closed = true; - state.is_cancelled = true; + state.cancellation_status = CancellationStatus::FULLY_CANCELLED; break; } @@ -711,7 +726,7 @@ void TCPHandler::readData() while (readDataNext()) ; - if (state.is_cancelled) + if (state.cancellation_status == CancellationStatus::FULLY_CANCELLED) throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled"); } @@ -724,7 +739,7 @@ void TCPHandler::skipData() while (readDataNext()) ; - if (state.is_cancelled) + if (state.cancellation_status == CancellationStatus::FULLY_CANCELLED) throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled"); } @@ -762,7 +777,7 @@ void TCPHandler::processInsertQuery() while (readDataNext()) executor.push(std::move(state.block_for_insert)); - if (state.is_cancelled) + if (state.cancellation_status == CancellationStatus::FULLY_CANCELLED) executor.cancel(); else executor.finish(); @@ -816,7 +831,8 @@ void TCPHandler::processOrdinaryQueryWithProcessors() { std::unique_lock lock(task_callback_mutex); - if (isQueryCancelled()) + auto cancellation_status = getQueryCancellationStatus(); + if (cancellation_status == CancellationStatus::FULLY_CANCELLED) { /// Several callback like callback for parallel reading could be called from inside the pipeline /// and we have to unlock the mutex from our side to prevent deadlock. @@ -825,6 +841,10 @@ void TCPHandler::processOrdinaryQueryWithProcessors() executor.cancel(); break; } + else if (cancellation_status == CancellationStatus::READ_CANCELLED) + { + executor.cancelReading(); + } if (after_send_progress.elapsed() / 1000 >= interactive_delay) { @@ -855,7 +875,7 @@ void TCPHandler::processOrdinaryQueryWithProcessors() * because we have not read all the data yet, * and there could be ongoing calculations in other threads at the same time. */ - if (!isQueryCancelled()) + if (getQueryCancellationStatus() != CancellationStatus::FULLY_CANCELLED) { sendTotals(executor.getTotalsBlock()); sendExtremes(executor.getExtremesBlock()); @@ -1325,15 +1345,7 @@ bool TCPHandler::receivePacket() case Protocol::Client::Cancel: { - /// For testing connection collector. - if (unlikely(sleep_in_receive_cancel.totalMilliseconds())) - { - std::chrono::milliseconds ms(sleep_in_receive_cancel.totalMilliseconds()); - std::this_thread::sleep_for(ms); - } - - LOG_INFO(log, "Received 'Cancel' packet from the client, canceling the query"); - state.is_cancelled = true; + decreaseCancellationStatus("Received 'Cancel' packet from the client, canceling the query."); return false; } @@ -1376,14 +1388,7 @@ String TCPHandler::receiveReadTaskResponseAssumeLocked() { if (packet_type == Protocol::Client::Cancel) { - LOG_INFO(log, "Received 'Cancel' packet from the client, canceling the read task"); - state.is_cancelled = true; - /// For testing connection collector. - if (unlikely(sleep_in_receive_cancel.totalMilliseconds())) - { - std::chrono::milliseconds ms(sleep_in_receive_cancel.totalMilliseconds()); - std::this_thread::sleep_for(ms); - } + decreaseCancellationStatus("Received 'Cancel' packet from the client, canceling the read task."); return {}; } else @@ -1410,14 +1415,7 @@ std::optional TCPHandler::receivePartitionMergeTreeReadTas { if (packet_type == Protocol::Client::Cancel) { - LOG_INFO(log, "Received 'Cancel' packet from the client, canceling the MergeTree read task"); - state.is_cancelled = true; - /// For testing connection collector. - if (unlikely(sleep_in_receive_cancel.totalMilliseconds())) - { - std::chrono::milliseconds ms(sleep_in_receive_cancel.totalMilliseconds()); - std::this_thread::sleep_for(ms); - } + decreaseCancellationStatus("Received 'Cancel' packet from the client, canceling the MergeTree read task."); return std::nullopt; } else @@ -1792,13 +1790,47 @@ void TCPHandler::initProfileEventsBlockOutput(const Block & block) } -bool TCPHandler::isQueryCancelled() +void TCPHandler::decreaseCancellationStatus(const std::string& log_message) { - if (state.is_cancelled || state.sent_all_data) - return true; + auto prev_status = QueryState::cancellationStatusToName(state.cancellation_status); + + bool stop_reading_on_first_cancel = false; + if (query_context) + { + const auto & settings = query_context->getSettingsRef(); + stop_reading_on_first_cancel = settings.stop_reading_on_first_cancel; + } + + if (stop_reading_on_first_cancel && state.cancellation_status == CancellationStatus::NOT_CANCELLED) + { + state.cancellation_status = CancellationStatus::READ_CANCELLED; + } + else + { + state.cancellation_status = CancellationStatus::FULLY_CANCELLED; + } + + auto current_status = QueryState::cancellationStatusToName(state.cancellation_status); + LOG_INFO(log, "Change cancellation status from {} to {}. Log message: {}", prev_status, current_status, log_message); + + /// For testing connection collector. + { + if (unlikely(sleep_in_receive_cancel.totalMilliseconds())) + { + std::chrono::milliseconds ms(sleep_in_receive_cancel.totalMilliseconds()); + std::this_thread::sleep_for(ms); + } + } +} + + +QueryState::CancellationStatus TCPHandler::getQueryCancellationStatus() +{ + if (state.cancellation_status == CancellationStatus::FULLY_CANCELLED || state.sent_all_data) + return CancellationStatus::FULLY_CANCELLED; if (after_check_cancelled.elapsed() / 1000 < interactive_delay) - return false; + return state.cancellation_status; after_check_cancelled.restart(); @@ -1808,9 +1840,9 @@ bool TCPHandler::isQueryCancelled() if (in->eof()) { LOG_INFO(log, "Client has dropped the connection, cancel the query."); - state.is_cancelled = true; + state.cancellation_status = CancellationStatus::FULLY_CANCELLED; state.is_connection_closed = true; - return true; + return CancellationStatus::FULLY_CANCELLED; } UInt64 packet_type = 0; @@ -1821,25 +1853,17 @@ bool TCPHandler::isQueryCancelled() case Protocol::Client::Cancel: if (state.empty()) throw NetException(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, "Unexpected packet Cancel received from client"); - LOG_INFO(log, "Query was cancelled."); - state.is_cancelled = true; - /// For testing connection collector. - { - if (unlikely(sleep_in_receive_cancel.totalMilliseconds())) - { - std::chrono::milliseconds ms(sleep_in_receive_cancel.totalMilliseconds()); - std::this_thread::sleep_for(ms); - } - } - return true; + decreaseCancellationStatus("Query was cancelled."); + + return state.cancellation_status; default: throw NetException(ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT, "Unknown packet from client {}", toString(packet_type)); } } - return false; + return state.cancellation_status; } diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index f06b0b060b3..873d0b74dbd 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -75,8 +75,17 @@ struct QueryState /// Streams of blocks, that are processing the query. BlockIO io; + enum class CancellationStatus: UInt8 + { + FULLY_CANCELLED, + READ_CANCELLED, + NOT_CANCELLED + }; + + static std::string cancellationStatusToName(CancellationStatus status); + /// Is request cancelled - bool is_cancelled = false; + CancellationStatus cancellation_status = CancellationStatus::NOT_CANCELLED; bool is_connection_closed = false; /// empty or not bool is_empty = true; @@ -269,7 +278,10 @@ private: void initLogsBlockOutput(const Block & block); void initProfileEventsBlockOutput(const Block & block); - bool isQueryCancelled(); + using CancellationStatus = typename QueryState::CancellationStatus; + + void decreaseCancellationStatus(const std::string& log_message); + CancellationStatus getQueryCancellationStatus(); /// This function is called from different threads. void updateProgress(const Progress & value); From 8590e4b4596cd98381088b19b1162819ae365e20 Mon Sep 17 00:00:00 2001 From: alexX512 Date: Wed, 15 Mar 2023 10:50:01 +0000 Subject: [PATCH 169/559] Add tests fr stop_reading_on_first_cancel setting --- src/Client/ClientBase.cpp | 10 ++++++---- src/Client/ClientBase.h | 2 +- .../25340_stop_reading_on_first_cancel.reference | 1 + .../0_stateless/25340_stop_reading_on_first_cancel.sh | 11 +++++++++++ 4 files changed, 19 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/25340_stop_reading_on_first_cancel.reference create mode 100755 tests/queries/0_stateless/25340_stop_reading_on_first_cancel.sh diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index cc74c18c870..f02e8401384 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -173,7 +173,7 @@ static void incrementProfileEventsBlock(Block & dst, const Block & src) auto & dst_column_host_name = typeid_cast(*mutable_columns[name_pos["host_name"]]); auto & dst_array_current_time = typeid_cast(*mutable_columns[name_pos["current_time"]]).getData(); - auto & dst_array_type = typeid_cast(*mutable_columns[name_pos["type"]]).getData(); + auto & dst_array_type = typeid_cast(*mutable_columns[name_pos["type"]]).getData(); auto & dst_column_name = typeid_cast(*mutable_columns[name_pos["name"]]); auto & dst_array_value = typeid_cast(*mutable_columns[name_pos["value"]]).getData(); @@ -888,7 +888,7 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa if (send_external_tables) sendExternalTables(parsed_query); - receiveResult(parsed_query, signals_before_stop); + receiveResult(parsed_query, signals_before_stop, settings.stop_reading_on_first_cancel); break; } @@ -913,7 +913,7 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa /// Receives and processes packets coming from server. /// Also checks if query execution should be cancelled. -void ClientBase::receiveResult(ASTPtr parsed_query, Int32 signals_before_stop) +void ClientBase::receiveResult(ASTPtr parsed_query, Int32 signals_before_stop, bool stop_reading_on_first_cancel) { // TODO: get the poll_interval from commandline. const auto receive_timeout = connection_parameters.timeouts.receive_timeout; @@ -937,9 +937,11 @@ void ClientBase::receiveResult(ASTPtr parsed_query, Int32 signals_before_stop) /// to avoid losing sync. if (!cancelled) { - if (QueryInterruptHandler::cancelled_status() == signals_before_stop - 1) + if (stop_reading_on_first_cancel && QueryInterruptHandler::cancelled_status() == signals_before_stop - 1) { connection->sendCancel(); + /// First cancel reading request was sent. Next requests will only be with a full cancel + stop_reading_on_first_cancel = false; } else if (QueryInterruptHandler::cancelled()) { diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index e9e539d251e..faf3fa8653a 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -131,7 +131,7 @@ protected: private: - void receiveResult(ASTPtr parsed_query, Int32 signals_before_stop); + void receiveResult(ASTPtr parsed_query, Int32 signals_before_stop, bool stop_reading_on_first_cancel); bool receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled_); void receiveLogsAndProfileEvents(ASTPtr parsed_query); bool receiveSampleBlock(Block & out, ColumnsDescription & columns_description, ASTPtr parsed_query); diff --git a/tests/queries/0_stateless/25340_stop_reading_on_first_cancel.reference b/tests/queries/0_stateless/25340_stop_reading_on_first_cancel.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/25340_stop_reading_on_first_cancel.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/25340_stop_reading_on_first_cancel.sh b/tests/queries/0_stateless/25340_stop_reading_on_first_cancel.sh new file mode 100755 index 00000000000..947cbd402fe --- /dev/null +++ b/tests/queries/0_stateless/25340_stop_reading_on_first_cancel.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -n --query="SELECT sum(number * 0) FROM numbers(1000000000) SETTINGS stop_reading_on_first_cancel=true;" & +pid=$! +sleep 1 +kill -INT $pid +wait $pid \ No newline at end of file From b2aa9324ac5f6ed0a46bba0ec66d7033d98d0d82 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 15 Mar 2023 11:03:42 +0000 Subject: [PATCH 170/559] Un-friend system view (no longer necessary) --- src/Interpreters/Cache/QueryCache.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index 763e797ac07..4c91d7f1ae7 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -157,8 +157,6 @@ private: size_t max_entry_size_in_rows TSA_GUARDED_BY(mutex) = 0; size_t cache_size_in_bytes TSA_GUARDED_BY(mutex) = 0; /// Updated in each cache insert/delete - - friend class StorageSystemQueryCache; }; using QueryCachePtr = std::shared_ptr; From 1d06d343dc102d9c064503b9e85a35111104c436 Mon Sep 17 00:00:00 2001 From: alexX512 Date: Wed, 15 Mar 2023 11:15:50 +0000 Subject: [PATCH 171/559] Add last blank line in the new test --- tests/queries/0_stateless/25340_stop_reading_on_first_cancel.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/25340_stop_reading_on_first_cancel.sh b/tests/queries/0_stateless/25340_stop_reading_on_first_cancel.sh index 947cbd402fe..52111cdd8f3 100755 --- a/tests/queries/0_stateless/25340_stop_reading_on_first_cancel.sh +++ b/tests/queries/0_stateless/25340_stop_reading_on_first_cancel.sh @@ -8,4 +8,4 @@ $CLICKHOUSE_CLIENT -n --query="SELECT sum(number * 0) FROM numbers(1000000000) S pid=$! sleep 1 kill -INT $pid -wait $pid \ No newline at end of file +wait $pid From 26e15b288bc760f5ff6d3cb71e178278e4f5bd88 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 15 Mar 2023 11:37:25 +0000 Subject: [PATCH 172/559] Check for duplicate column in BSONEachRow format --- src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp | 3 +++ .../queries/0_stateless/02590_bson_duplicate_column.reference | 0 tests/queries/0_stateless/02590_bson_duplicate_column.sql | 1 + 3 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/02590_bson_duplicate_column.reference create mode 100644 tests/queries/0_stateless/02590_bson_duplicate_column.sql diff --git a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp index 02fe58094ae..f768d3b568f 100644 --- a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp @@ -786,6 +786,9 @@ bool BSONEachRowRowInputFormat::readRow(MutableColumns & columns, RowReadExtensi } else { + if (seen_columns[index]) + throw Exception(ErrorCodes::INCORRECT_DATA, "Duplicate field found while parsing BSONNEachRow format: {}", name); + seen_columns[index] = true; read_columns[index] = readField(*columns[index], types[index], BSONType(type)); } diff --git a/tests/queries/0_stateless/02590_bson_duplicate_column.reference b/tests/queries/0_stateless/02590_bson_duplicate_column.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02590_bson_duplicate_column.sql b/tests/queries/0_stateless/02590_bson_duplicate_column.sql new file mode 100644 index 00000000000..decc6b27e8b --- /dev/null +++ b/tests/queries/0_stateless/02590_bson_duplicate_column.sql @@ -0,0 +1 @@ +select * from format(BSONEachRow, x'130000001078002a0000001078002a00000000'); -- {serverError INCORRECT_DATA} From aa5127c2fdd7e5a0dbae8003f8e57c166a8bc4db Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 15 Mar 2023 08:33:45 +0100 Subject: [PATCH 173/559] Add sanity checks for writing number in variable length format And just to double check: # var_uint 9223372036854775807 ffffffffffffffff7f ffffffffffffffff7f ffffffffffffffff7f x: 9223372036854775807, y: 9223372036854775807 # var_uint 9223372036854775808 808080808080808080 808080808080808080 808080808080808080 x: 9223372036854775808, y: 0 Signed-off-by: Azat Khuzhin --- src/IO/VarInt.h | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/src/IO/VarInt.h b/src/IO/VarInt.h index 0869051034a..d026192cb7d 100644 --- a/src/IO/VarInt.h +++ b/src/IO/VarInt.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -14,7 +15,19 @@ namespace ErrorCodes } -/** Write UInt64 in variable length format (base128) NOTE Only up to 2^63 - 1 are supported. */ +/** Variable-Length Quantity (VLQ) Base-128 compression + * + * NOTE: Due to historical reasons, only up to 1<<63-1 are supported, which + * cannot be changed without breaking the backward compatibility. + * Also some drivers may support full 1<<64 range (i.e. python - + * clickhouse-driver), while others has the same limitations as ClickHouse + * (i.e. Rust - clickhouse-rs). + * So implementing VLQ for the whole 1<<64 range will require different set of + * helpers. + */ +constexpr size_t VAR_UINT_MAX = (1ULL<<63) - 1; + +/** Write UInt64 in variable length format (base128) */ void writeVarUInt(UInt64 x, std::ostream & ostr); void writeVarUInt(UInt64 x, WriteBuffer & ostr); char * writeVarUInt(UInt64 x, char * ostr); @@ -186,6 +199,7 @@ inline const char * readVarUInt(UInt64 & x, const char * istr, size_t size) inline void writeVarUInt(UInt64 x, WriteBuffer & ostr) { + chassert(x <= VAR_UINT_MAX); for (size_t i = 0; i < 9; ++i) { uint8_t byte = x & 0x7F; @@ -205,6 +219,7 @@ inline void writeVarUInt(UInt64 x, WriteBuffer & ostr) inline void writeVarUInt(UInt64 x, std::ostream & ostr) { + chassert(x <= VAR_UINT_MAX); for (size_t i = 0; i < 9; ++i) { uint8_t byte = x & 0x7F; @@ -222,6 +237,7 @@ inline void writeVarUInt(UInt64 x, std::ostream & ostr) inline char * writeVarUInt(UInt64 x, char * ostr) { + chassert(x <= VAR_UINT_MAX); for (size_t i = 0; i < 9; ++i) { uint8_t byte = x & 0x7F; From 727ad9289bd4b5b03d127c8faa877d82b76438f4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 15 Mar 2023 13:10:35 +0100 Subject: [PATCH 174/559] Use VAR_UINT_MAX for unknown_packet_in_send_data Signed-off-by: Azat Khuzhin --- src/Server/TCPHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index b240c99fc7f..2a5e2c302b3 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1824,7 +1824,7 @@ void TCPHandler::sendData(const Block & block) { --unknown_packet_in_send_data; if (unknown_packet_in_send_data == 0) - writeVarUInt(UInt64(-1), *out); + writeVarUInt(VAR_UINT_MAX, *out); } writeVarUInt(Protocol::Server::Data, *out); From cbbdb5aabc252102068caf6c0de264a52427ef4e Mon Sep 17 00:00:00 2001 From: alexX512 Date: Wed, 15 Mar 2023 13:05:38 +0000 Subject: [PATCH 175/559] Fix typos --- src/Client/ClientBase.cpp | 4 ++-- src/Processors/Executors/ExecutingGraph.cpp | 4 ++-- src/Processors/Executors/PullingAsyncPipelineExecutor.cpp | 2 +- src/Server/TCPHandler.cpp | 8 +------- ...rence => 25341_stop_reading_on_first_cancel.reference} | 0 ...st_cancel.sh => 25341_stop_reading_on_first_cancel.sh} | 0 6 files changed, 6 insertions(+), 12 deletions(-) rename tests/queries/0_stateless/{25340_stop_reading_on_first_cancel.reference => 25341_stop_reading_on_first_cancel.reference} (100%) rename tests/queries/0_stateless/{25340_stop_reading_on_first_cancel.sh => 25341_stop_reading_on_first_cancel.sh} (100%) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 635fec012e1..6a2961d69da 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -277,7 +277,7 @@ public: static void stop() { return exit_after_signals.store(0); } /// Return true if the query was stopped. - /// Query was stoped if it received at least "signals_before_stop" interrupt signals. + /// Query was stopped if it received at least "signals_before_stop" interrupt signals. static bool try_stop() { return exit_after_signals.fetch_sub(1) <= 0; } static bool cancelled() { return exit_after_signals.load() <= 0; } @@ -865,7 +865,7 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa const auto & settings = global_context->getSettingsRef(); const Int32 signals_before_stop = settings.stop_reading_on_first_cancel ? 2 : 1; - + int retries_left = 10; while (retries_left) { diff --git a/src/Processors/Executors/ExecutingGraph.cpp b/src/Processors/Executors/ExecutingGraph.cpp index 7af9fcda3b9..37d58d01bee 100644 --- a/src/Processors/Executors/ExecutingGraph.cpp +++ b/src/Processors/Executors/ExecutingGraph.cpp @@ -413,8 +413,8 @@ void ExecutingGraph::cancel(bool cancel_all_processors) { try { - /// Stop all processors in the general case, but in a specific case - /// where the pipeline needs to return a result on a partially read table, + /// Stop all processors in the general case, but in a specific case + /// where the pipeline needs to return a result on a partially read table, /// stop only the processors that read from the source if (cancel_all_processors || source_processors.at(proc)) { diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index ae63b4242c6..af68802a640 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -180,7 +180,7 @@ void PullingAsyncPipelineExecutor::cancel() { data->executor->cancel(); }); - + /// The following code is needed to rethrow exception from PipelineExecutor. /// It could have been thrown from pull(), but we will not likely call it again. diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index bc42f7c6a19..ed1ee69c852 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1337,14 +1337,8 @@ bool TCPHandler::receivePacket() return false; case Protocol::Client::Cancel: -<<<<<<< HEAD { decreaseCancellationStatus("Received 'Cancel' packet from the client, canceling the query."); - -======= - LOG_INFO(log, "Received 'Cancel' packet from the client, canceling the query"); - state.is_cancelled = true; ->>>>>>> 00c05f7f8b8f11bcb9c11e54c9bec6b77932933f return false; case Protocol::Client::Hello: @@ -1802,7 +1796,7 @@ void TCPHandler::decreaseCancellationStatus(const std::string& log_message) const auto & settings = query_context->getSettingsRef(); stop_reading_on_first_cancel = settings.stop_reading_on_first_cancel; } - + if (stop_reading_on_first_cancel && state.cancellation_status == CancellationStatus::NOT_CANCELLED) { state.cancellation_status = CancellationStatus::READ_CANCELLED; diff --git a/tests/queries/0_stateless/25340_stop_reading_on_first_cancel.reference b/tests/queries/0_stateless/25341_stop_reading_on_first_cancel.reference similarity index 100% rename from tests/queries/0_stateless/25340_stop_reading_on_first_cancel.reference rename to tests/queries/0_stateless/25341_stop_reading_on_first_cancel.reference diff --git a/tests/queries/0_stateless/25340_stop_reading_on_first_cancel.sh b/tests/queries/0_stateless/25341_stop_reading_on_first_cancel.sh similarity index 100% rename from tests/queries/0_stateless/25340_stop_reading_on_first_cancel.sh rename to tests/queries/0_stateless/25341_stop_reading_on_first_cancel.sh From c4ac4c596fb6d628840a1e7a65323bd69106b593 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 15 Mar 2023 13:17:13 +0000 Subject: [PATCH 176/559] Fix comments --- src/Processors/Formats/Impl/AvroRowInputFormat.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 9873107208e..383b1b08a73 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -173,13 +173,14 @@ static AvroDeserializer::DeserializeFn createDecimalDeserializeFn(const avro::No return [tmp = std::string(), target_type](IColumn & column, avro::Decoder & decoder) mutable { + static constexpr size_t field_type_size = sizeof(typename DecimalType::FieldType); decoder.decodeString(tmp); - if (tmp.size() != sizeof(typename DecimalType::FieldType)) + if (tmp.size() != field_type_size) throw ParsingException( ErrorCodes::CANNOT_PARSE_UUID, "Cannot parse type {}, expected binary data with size {}, got {}", target_type->getName(), - sizeof(typename DecimalType::FieldType), + field_type_size, tmp.size()); typename DecimalType::FieldType field; @@ -1098,7 +1099,7 @@ DataTypePtr AvroSchemaReader::avroNodeToDataType(avro::NodePtr node) auto logical_type = node->logicalType(); if (logical_type.type() == avro::LogicalType::TIMESTAMP_MILLIS) return {std::make_shared(3)}; - else if (logical_type.type() == avro::LogicalType::TIMESTAMP_MICROS) + if (logical_type.type() == avro::LogicalType::TIMESTAMP_MICROS) return {std::make_shared(6)}; return std::make_shared(); From 0088c18192225e0cfbfdc2043a30b70daf9af58e Mon Sep 17 00:00:00 2001 From: alexX512 Date: Wed, 15 Mar 2023 13:37:47 +0000 Subject: [PATCH 177/559] Fix fasttest error --- .../Executors/PullingAsyncPipelineExecutor.h | 1 + src/Server/TCPHandler.cpp | 12 ------------ 2 files changed, 1 insertion(+), 12 deletions(-) diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.h b/src/Processors/Executors/PullingAsyncPipelineExecutor.h index dce88b48262..361bcc0155c 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.h +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.h @@ -1,4 +1,5 @@ #pragma once +#include #include namespace DB diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index ed1ee69c852..b018d7de580 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1356,13 +1356,11 @@ bool TCPHandler::receivePacket() } } - void TCPHandler::receiveIgnoredPartUUIDs() { readVectorBinary(state.part_uuids_to_ignore.emplace(), *in); } - void TCPHandler::receiveUnexpectedIgnoredPartUUIDs() { std::vector skip_part_uuids; @@ -1370,7 +1368,6 @@ void TCPHandler::receiveUnexpectedIgnoredPartUUIDs() throw NetException(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, "Unexpected packet IgnoredPartUUIDs received from client"); } - String TCPHandler::receiveReadTaskResponseAssumeLocked() { UInt64 packet_type = 0; @@ -1397,7 +1394,6 @@ String TCPHandler::receiveReadTaskResponseAssumeLocked() return response; } - std::optional TCPHandler::receivePartitionMergeTreeReadTaskResponseAssumeLocked() { UInt64 packet_type = 0; @@ -1420,7 +1416,6 @@ std::optional TCPHandler::receivePartitionMergeTreeReadTas return response; } - void TCPHandler::receiveClusterNameAndSalt() { readStringBinary(cluster, *in); @@ -1676,7 +1671,6 @@ bool TCPHandler::receiveData(bool scalar) return true; } - bool TCPHandler::receiveUnexpectedData(bool throw_exception) { String skip_external_table_name; @@ -1725,7 +1719,6 @@ void TCPHandler::initBlockInput() } } - void TCPHandler::initBlockOutput(const Block & block) { if (!state.block_out) @@ -1771,7 +1764,6 @@ void TCPHandler::initLogsBlockOutput(const Block & block) } } - void TCPHandler::initProfileEventsBlockOutput(const Block & block) { if (!state.profile_events_block_out) @@ -1785,7 +1777,6 @@ void TCPHandler::initProfileEventsBlockOutput(const Block & block) } } - void TCPHandler::decreaseCancellationStatus(const std::string& log_message) { auto prev_status = QueryState::cancellationStatusToName(state.cancellation_status); @@ -1819,7 +1810,6 @@ void TCPHandler::decreaseCancellationStatus(const std::string& log_message) } } - QueryState::CancellationStatus TCPHandler::getQueryCancellationStatus() { if (state.cancellation_status == CancellationStatus::FULLY_CANCELLED || state.sent_all_data) @@ -1862,7 +1852,6 @@ QueryState::CancellationStatus TCPHandler::getQueryCancellationStatus() return state.cancellation_status; } - void TCPHandler::sendData(const Block & block) { initBlockOutput(block); @@ -1919,7 +1908,6 @@ void TCPHandler::sendData(const Block & block) } } - void TCPHandler::sendLogData(const Block & block) { initLogsBlockOutput(block); From c102adb72842f8685744b00ab779c9044bc74955 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 14 Mar 2023 18:46:26 +0100 Subject: [PATCH 178/559] Analyzer added PREWHERE support --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 8 +- src/Analyzer/ValidationUtils.cpp | 46 +- src/Analyzer/ValidationUtils.h | 7 +- src/Interpreters/ActionsDAG.cpp | 113 +++- src/Interpreters/ActionsDAG.h | 10 +- src/Interpreters/InterpreterSelectQuery.cpp | 6 +- src/Planner/CollectTableExpressionData.cpp | 165 +++++- src/Planner/CollectTableExpressionData.h | 9 +- src/Planner/Planner.cpp | 68 +-- src/Planner/PlannerActionsVisitor.cpp | 529 ++++++++++-------- src/Planner/PlannerActionsVisitor.h | 29 +- src/Planner/PlannerJoinTree.cpp | 164 ++++-- src/Planner/TableExpressionData.h | 28 + src/Planner/Utils.cpp | 4 +- .../QueryPlan/Optimizations/Optimizations.h | 1 + .../Optimizations/optimizePrewhere.cpp | 387 +++++++++++++ .../Optimizations/optimizeReadInOrder.cpp | 2 +- .../QueryPlan/Optimizations/optimizeTree.cpp | 1 + .../QueryPlan/ReadFromMergeTree.cpp | 11 + src/Processors/QueryPlan/ReadFromMergeTree.h | 8 +- .../MergeTreeBaseSelectProcessor.cpp | 43 +- .../MergeTreeSplitPrewhereIntoReadSteps.cpp | 8 +- .../MergeTree/MergeTreeWhereOptimizer.cpp | 406 ++++++++------ .../MergeTree/MergeTreeWhereOptimizer.h | 71 ++- src/Storages/MergeTree/RPNBuilder.cpp | 55 +- src/Storages/MergeTree/RPNBuilder.h | 6 + 26 files changed, 1571 insertions(+), 614 deletions(-) create mode 100644 src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 38575965973..5353dde2078 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6846,13 +6846,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.isGroupByAll()) expandGroupByAll(query_node_typed); - if (query_node_typed.hasPrewhere()) - assertNoFunctionNodes(query_node_typed.getPrewhere(), - "arrayJoin", - ErrorCodes::ILLEGAL_PREWHERE, - "ARRAY JOIN", - "in PREWHERE"); - + validateFilters(query_node); validateAggregates(query_node, { .group_by_use_nulls = scope.group_by_use_nulls }); for (const auto & column : projection_columns) diff --git a/src/Analyzer/ValidationUtils.cpp b/src/Analyzer/ValidationUtils.cpp index 58e6f26c03a..af35632ab81 100644 --- a/src/Analyzer/ValidationUtils.cpp +++ b/src/Analyzer/ValidationUtils.cpp @@ -17,8 +17,50 @@ namespace ErrorCodes extern const int NOT_AN_AGGREGATE; extern const int NOT_IMPLEMENTED; extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER; + extern const int ILLEGAL_PREWHERE; } +namespace +{ + +void validateFilter(const QueryTreeNodePtr & filter_node, std::string_view exception_place_message, const QueryTreeNodePtr & query_node) +{ + auto filter_node_result_type = filter_node->getResultType(); + if (!filter_node_result_type->canBeUsedInBooleanContext()) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, + "Invalid type for filter in {}: {}. In query {}", + exception_place_message, + filter_node_result_type->getName(), + query_node->formatASTForErrorMessage()); +} + +} + +void validateFilters(const QueryTreeNodePtr & query_node) +{ + const auto & query_node_typed = query_node->as(); + if (query_node_typed.hasPrewhere()) + { + validateFilter(query_node_typed.getPrewhere(), "PREWHERE", query_node); + + assertNoFunctionNodes(query_node_typed.getPrewhere(), + "arrayJoin", + ErrorCodes::ILLEGAL_PREWHERE, + "ARRAY JOIN", + "in PREWHERE"); + } + + if (query_node_typed.hasWhere()) + validateFilter(query_node_typed.getWhere(), "WHERE", query_node); + + if (query_node_typed.hasHaving()) + validateFilter(query_node_typed.getHaving(), "HAVING", query_node); +} + +namespace +{ + class ValidateGroupByColumnsVisitor : public ConstInDepthQueryTreeVisitor { public: @@ -106,7 +148,9 @@ private: const QueryTreeNodePtr & query_node; }; -void validateAggregates(const QueryTreeNodePtr & query_node, ValidationParams params) +} + +void validateAggregates(const QueryTreeNodePtr & query_node, AggregatesValidationParams params) { const auto & query_node_typed = query_node->as(); auto join_tree_node_type = query_node_typed.getJoinTree()->getNodeType(); diff --git a/src/Analyzer/ValidationUtils.h b/src/Analyzer/ValidationUtils.h index c15a3531c8d..3041447af17 100644 --- a/src/Analyzer/ValidationUtils.h +++ b/src/Analyzer/ValidationUtils.h @@ -5,7 +5,10 @@ namespace DB { -struct ValidationParams +/// Validate PREWHERE, WHERE, HAVING in query node +void validateFilters(const QueryTreeNodePtr & query_node); + +struct AggregatesValidationParams { bool group_by_use_nulls = false; }; @@ -20,7 +23,7 @@ struct ValidationParams * PROJECTION. * 5. Throws exception if there is GROUPING SETS or ROLLUP or CUBE or WITH TOTALS without aggregation. */ -void validateAggregates(const QueryTreeNodePtr & query_node, ValidationParams params); +void validateAggregates(const QueryTreeNodePtr & query_node, AggregatesValidationParams params); /** Assert that there are no function nodes with specified function name in node children. * Do not visit subqueries. diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 46b5a93b28c..1964f6fd8b3 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -188,9 +189,9 @@ const ActionsDAG::Node & ActionsDAG::addArrayJoin(const Node & child, std::strin } const ActionsDAG::Node & ActionsDAG::addFunction( - const FunctionOverloadResolverPtr & function, - NodeRawConstPtrs children, - std::string result_name) + const FunctionOverloadResolverPtr & function, + NodeRawConstPtrs children, + std::string result_name) { auto [arguments, all_const] = getFunctionArguments(children); @@ -1364,6 +1365,83 @@ void ActionsDAG::mergeInplace(ActionsDAG && second) first.projected_output = second.projected_output; } +void ActionsDAG::mergeNodes(ActionsDAG && second) +{ + std::unordered_map node_name_to_node; + for (auto & node : nodes) + node_name_to_node.emplace(node.result_name, &node); + + struct Frame + { + ActionsDAG::Node * node = nullptr; + bool visited_children = false; + }; + + std::unordered_map const_node_to_node; + for (auto & node : second.nodes) + const_node_to_node.emplace(&node, &node); + + std::vector nodes_to_process; + nodes_to_process.reserve(second.getOutputs().size()); + for (auto & node : second.getOutputs()) + nodes_to_process.push_back({const_node_to_node.at(node), false /*visited_children*/}); + + std::unordered_set nodes_to_move_from_second_dag; + + while (!nodes_to_process.empty()) + { + auto & node_to_process = nodes_to_process.back(); + auto * node = node_to_process.node; + + auto node_it = node_name_to_node.find(node->result_name); + if (node_it != node_name_to_node.end()) + { + nodes_to_process.pop_back(); + continue; + } + + if (!node_to_process.visited_children) + { + node_to_process.visited_children = true; + + for (auto & child : node->children) + nodes_to_process.push_back({const_node_to_node.at(child), false /*visited_children*/}); + + /// If node has children process them first + if (!node->children.empty()) + continue; + } + + for (auto & child : node->children) + child = node_name_to_node.at(child->result_name); + + node_name_to_node.emplace(node->result_name, node); + nodes_to_move_from_second_dag.insert(node); + + nodes_to_process.pop_back(); + } + + if (nodes_to_move_from_second_dag.empty()) + return; + + auto second_nodes_end = second.nodes.end(); + for (auto second_node_it = second.nodes.begin(); second_node_it != second_nodes_end;) + { + if (!nodes_to_move_from_second_dag.contains(&(*second_node_it))) + { + ++second_node_it; + continue; + } + + auto node_to_move_it = second_node_it; + ++second_node_it; + nodes.splice(nodes.end(), second.nodes, node_to_move_it); + + if (node_to_move_it->type == ActionType::INPUT) + inputs.push_back(&(*node_to_move_it)); + } +} + ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split_nodes) const { /// Split DAG into two parts. @@ -2193,7 +2271,8 @@ bool ActionsDAG::isSortingPreserved( ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( const NodeRawConstPtrs & filter_nodes, const std::unordered_map & node_name_to_input_node_column, - const ContextPtr & context) + const ContextPtr & context, + bool single_output_condition_node) { if (filter_nodes.empty()) return nullptr; @@ -2281,13 +2360,35 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( NodeRawConstPtrs function_children; function_children.reserve(node->children.size()); + FunctionOverloadResolverPtr function_overload_resolver; + + if (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())) + { + auto index_hint_filter_dag = buildFilterActionsDAG(index_hint->getActions()->getOutputs(), + node_name_to_input_node_column, + context, + false /*single_output_condition_node*/); + + auto index_hint_function_clone = std::make_shared(); + index_hint_function_clone->setActions(std::move(index_hint_filter_dag)); + function_overload_resolver = std::make_shared(std::move(index_hint_function_clone)); + } + } + } + for (const auto & child : node->children) function_children.push_back(node_to_result_node.find(child)->second); 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( - node->function_base, + function_base, std::move(function_children), std::move(arguments), {}, @@ -2307,7 +2408,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( for (const auto & node : filter_nodes) result_dag_outputs.push_back(node_to_result_node.find(node)->second); - if (result_dag_outputs.size() > 1) + if (result_dag_outputs.size() > 1 && single_output_condition_node) { auto function_builder = FunctionFactory::instance().get("and", context); result_dag_outputs = { &result_dag->addFunction(function_builder, result_dag_outputs, {}) }; diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 5f0005dae37..4b63a350d7d 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -290,6 +290,9 @@ public: /// So that pointers to nodes are kept valid. void mergeInplace(ActionsDAG && second); + /// Merge current nodes with specified dag nodes + void mergeNodes(ActionsDAG && second); + using SplitResult = std::pair; /// Split ActionsDAG into two DAGs, where first part contains all nodes from split_nodes and their children. @@ -344,15 +347,18 @@ public: * Additionally during dag construction if node has name that exists in node_name_to_input_column map argument * in final dag this node is represented as INPUT node with specified column. * - * Result dag has only single output node: + * If single_output_condition_node = true, result dag has single output node: * 1. If there is single filter node, result dag output will contain this node. * 2. If there are multiple filter nodes, result dag output will contain single `and` function node * and children of this node will be filter nodes. + * + * If single_output_condition_node = false, result dag has multiple output nodes. */ static ActionsDAGPtr buildFilterActionsDAG( const NodeRawConstPtrs & filter_nodes, const std::unordered_map & node_name_to_input_node_column, - const ContextPtr & context); + const ContextPtr & context, + bool single_output_condition_node = true); private: NodeRawConstPtrs getParents(const Node * target) const; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 79b073e30f3..6802b0911b9 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -636,14 +636,14 @@ InterpreterSelectQuery::InterpreterSelectQuery( Names queried_columns = syntax_analyzer_result->requiredSourceColumns(); const auto & supported_prewhere_columns = storage->supportedPrewhereColumns(); - MergeTreeWhereOptimizer{ - current_info, - context, + MergeTreeWhereOptimizer where_optimizer{ std::move(column_compressed_sizes), metadata_snapshot, queried_columns, supported_prewhere_columns, log}; + + where_optimizer.optimize(current_info, context); } } diff --git a/src/Planner/CollectTableExpressionData.cpp b/src/Planner/CollectTableExpressionData.cpp index 0b820e849f3..b00709ba651 100644 --- a/src/Planner/CollectTableExpressionData.cpp +++ b/src/Planner/CollectTableExpressionData.cpp @@ -10,6 +10,7 @@ #include #include +#include namespace DB { @@ -17,6 +18,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int ILLEGAL_PREWHERE; } namespace @@ -78,23 +80,128 @@ public: static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) { - return !(child_node->getNodeType() == QueryTreeNodeType::QUERY || child_node->getNodeType() == QueryTreeNodeType::UNION); + auto child_node_type = child_node->getNodeType(); + return !(child_node_type == QueryTreeNodeType::QUERY || child_node_type == QueryTreeNodeType::UNION); } private: PlannerContext & planner_context; }; +class CollectPrewhereTableExpressionVisitor : public ConstInDepthQueryTreeVisitor +{ +public: + explicit CollectPrewhereTableExpressionVisitor(const QueryTreeNodePtr & query_node_) + : query_node(query_node_) + {} + + const QueryTreeNodePtr & getPrewhereTableExpression() const + { + return table_expression; + } + + void visitImpl(const QueryTreeNodePtr & node) + { + auto * column_node = node->as(); + if (!column_node) + return; + + auto column_source = column_node->getColumnSourceOrNull(); + if (!column_source) + throw Exception(ErrorCodes::ILLEGAL_PREWHERE, + "Invalid column {} in PREWHERE. In query {}", + column_node->formatASTForErrorMessage(), + query_node->formatASTForErrorMessage()); + + auto * table_column_source = column_source->as(); + auto * table_function_column_source = column_source->as(); + + if (!table_column_source && !table_function_column_source) + throw Exception(ErrorCodes::ILLEGAL_PREWHERE, + "Invalid column {} in PREWHERE. Expected column source to be table or table function. Actual {}. In query {}", + column_node->formatASTForErrorMessage(), + column_source->formatASTForErrorMessage(), + query_node->formatASTForErrorMessage()); + + if (table_expression && table_expression.get() != column_source.get()) + throw Exception(ErrorCodes::ILLEGAL_PREWHERE, + "Invalid column {} in PREWHERE. Expected columns from single table or table function {}. Actual {}. In query {}", + column_node->formatASTForErrorMessage(), + table_expression->formatASTForErrorMessage(), + column_source->formatASTForErrorMessage(), + query_node->formatASTForErrorMessage()); + + if (!table_expression) + { + const auto & storage = table_column_source ? table_column_source->getStorage() : table_function_column_source->getStorage(); + if (!storage->supportsPrewhere()) + throw Exception(ErrorCodes::ILLEGAL_PREWHERE, + "Storage {} (table {}) does not support PREWHERE", + storage->getName(), + storage->getStorageID().getNameForLogs()); + + table_expression = std::move(column_source); + table_supported_prewhere_columns = storage->supportedPrewhereColumns(); + } + + if (table_supported_prewhere_columns && !table_supported_prewhere_columns->contains(column_node->getColumnName())) + throw Exception(ErrorCodes::ILLEGAL_PREWHERE, + "Table expression {} does not support column {} in PREWHERE. In query {}", + table_expression->formatASTForErrorMessage(), + column_node->formatASTForErrorMessage(), + query_node->formatASTForErrorMessage()); + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) + { + auto child_node_type = child_node->getNodeType(); + return !(child_node_type == QueryTreeNodeType::QUERY || child_node_type == QueryTreeNodeType::UNION); + } + +private: + QueryTreeNodePtr query_node; + QueryTreeNodePtr table_expression; + std::optional table_supported_prewhere_columns; +}; + +void checkStorageSupportPrewhere(const QueryTreeNodePtr & table_expression) +{ + if (auto * table_node = table_expression->as()) + { + auto storage = table_node->getStorage(); + if (!storage->supportsPrewhere()) + throw Exception(ErrorCodes::ILLEGAL_PREWHERE, + "Storage {} (table {}) does not support PREWHERE", + storage->getName(), + storage->getStorageID().getNameForLogs()); + } + else if (auto * table_function_node = table_expression->as()) + { + auto storage = table_function_node->getStorage(); + if (!storage->supportsPrewhere()) + throw Exception(ErrorCodes::ILLEGAL_PREWHERE, + "Table function storage {} (table {}) does not support PREWHERE", + storage->getName(), + storage->getStorageID().getNameForLogs()); + } + else + { + throw Exception(ErrorCodes::ILLEGAL_PREWHERE, + "Subquery {} does not support PREWHERE", + table_expression->formatASTForErrorMessage()); + } } -void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContext & planner_context) +} + +void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr & planner_context) { auto & query_node_typed = query_node->as(); auto table_expressions_nodes = extractTableExpressions(query_node_typed.getJoinTree()); for (auto & table_expression_node : table_expressions_nodes) { - auto & table_expression_data = planner_context.getOrCreateTableExpressionData(table_expression_node); + auto & table_expression_data = planner_context->getOrCreateTableExpressionData(table_expression_node); if (auto * table_node = table_expression_node->as()) { @@ -108,8 +215,56 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContext & } } - CollectSourceColumnsVisitor collect_source_columns_visitor(planner_context); - collect_source_columns_visitor.visit(query_node); + CollectSourceColumnsVisitor collect_source_columns_visitor(*planner_context); + for (auto & node : query_node_typed.getChildren()) + { + if (!node || node == query_node_typed.getPrewhere()) + continue; + + collect_source_columns_visitor.visit(node); + } + + if (query_node_typed.hasPrewhere()) + { + CollectPrewhereTableExpressionVisitor collect_prewhere_table_expression_visitor(query_node); + collect_prewhere_table_expression_visitor.visit(query_node_typed.getPrewhere()); + + auto prewhere_table_expression = collect_prewhere_table_expression_visitor.getPrewhereTableExpression(); + if (!prewhere_table_expression) + { + prewhere_table_expression = table_expressions_nodes[0]; + checkStorageSupportPrewhere(prewhere_table_expression); + } + + auto & table_expression_data = planner_context->getOrCreateTableExpressionData(prewhere_table_expression); + const auto & column_names = table_expression_data.getColumnNames(); + NameSet required_column_names_without_prewhere(column_names.begin(), column_names.end()); + + collect_source_columns_visitor.visit(query_node_typed.getPrewhere()); + + auto prewhere_actions_dag = std::make_shared(); + + PlannerActionsVisitor visitor(planner_context, false /*use_column_identifier_as_action_node_name*/); + auto expression_nodes = visitor.visit(prewhere_actions_dag, query_node_typed.getPrewhere()); + 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[0]); + + 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); + + table_expression_data.setPrewhereFilterActions(std::move(prewhere_actions_dag)); + } +} + +void collectSourceColumns(QueryTreeNodePtr & expression_node, PlannerContextPtr & planner_context) +{ + CollectSourceColumnsVisitor collect_source_columns_visitor(*planner_context); + collect_source_columns_visitor.visit(expression_node); } } diff --git a/src/Planner/CollectTableExpressionData.h b/src/Planner/CollectTableExpressionData.h index f4e2d579dca..ed3f0ff7a47 100644 --- a/src/Planner/CollectTableExpressionData.h +++ b/src/Planner/CollectTableExpressionData.h @@ -12,6 +12,13 @@ namespace DB * * ALIAS table column nodes are registered in table expression data and replaced in query tree with inner alias expression. */ -void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContext & planner_context); +void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr & planner_context); + +/** Collect source columns for expression node. + * Collected source columns are registered in planner context. + * + * ALIAS table column nodes are registered in table expression data and replaced in query tree with inner alias expression. + */ +void collectSourceColumns(QueryTreeNodePtr & expression_node, PlannerContextPtr & planner_context); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 2242bf92e6b..d7f95a1598f 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -84,21 +84,8 @@ namespace ErrorCodes /** ClickHouse query planner. * - * TODO: Support JOIN with JOIN engine. - * TODO: Support VIEWs. - * TODO: JOIN drop unnecessary columns after ON, USING section - * TODO: Support RBAC. Support RBAC for ALIAS columns - * TODO: Support PREWHERE - * TODO: Support DISTINCT - * TODO: Support trivial count optimization - * TODO: Support projections - * TODO: Support read in order optimization - * TODO: UNION storage limits - * TODO: Support max streams - * TODO: Support ORDER BY read in order optimization - * TODO: Support GROUP BY read in order optimization - * TODO: Support Key Condition. Support indexes for IN function. - * TODO: Better support for quota and limits. + * TODO: Support projections. + * TODO: Support indexes for IN function. */ namespace @@ -135,37 +122,6 @@ void checkStoragesSupportTransactions(const PlannerContextPtr & planner_context) } } -void checkStorageSupportPrewhere(const QueryTreeNodePtr & query_node) -{ - auto & query_node_typed = query_node->as(); - auto table_expression = extractLeftTableExpression(query_node_typed.getJoinTree()); - - if (auto * table_node = table_expression->as()) - { - auto storage = table_node->getStorage(); - if (!storage->supportsPrewhere()) - throw Exception(ErrorCodes::ILLEGAL_PREWHERE, - "Storage {} (table {}) does not support PREWHERE", - storage->getName(), - storage->getStorageID().getNameForLogs()); - } - else if (auto * table_function_node = table_expression->as()) - { - auto storage = table_function_node->getStorage(); - if (!storage->supportsPrewhere()) - throw Exception(ErrorCodes::ILLEGAL_PREWHERE, - "Table function storage {} (table {}) does not support PREWHERE", - storage->getName(), - storage->getStorageID().getNameForLogs()); - } - else - { - throw Exception(ErrorCodes::ILLEGAL_PREWHERE, - "Subquery {} does not support PREWHERE", - query_node->formatASTForErrorMessage()); - } -} - /// Extend lifetime of query context, storages, and table locks void extendQueryContextAndStoragesLifetime(QueryPlan & query_plan, const PlannerContextPtr & planner_context) { @@ -1140,18 +1096,6 @@ void Planner::buildPlanForQueryNode() auto & query_node = query_tree->as(); const auto & query_context = planner_context->getQueryContext(); - if (query_node.hasPrewhere()) - { - checkStorageSupportPrewhere(query_tree); - - if (query_node.hasWhere()) - query_node.getWhere() = mergeConditionNodes({query_node.getPrewhere(), query_node.getWhere()}, query_context); - else - query_node.getWhere() = query_node.getPrewhere(); - - query_node.getPrewhere() = {}; - } - if (query_node.hasWhere()) { auto condition_constant = tryExtractConstantFromConditionNode(query_node.getWhere()); @@ -1185,8 +1129,8 @@ void Planner::buildPlanForQueryNode() } checkStoragesSupportTransactions(planner_context); - collectTableExpressionData(query_tree, *planner_context); collectSets(query_tree, *planner_context); + collectTableExpressionData(query_tree, planner_context); auto top_level_identifiers = collectTopLevelColumnIdentifiers(query_tree, planner_context); auto join_tree_query_plan = buildJoinTreeQueryPlan(query_tree, @@ -1215,6 +1159,12 @@ void Planner::buildPlanForQueryNode() std::vector result_actions_to_execute; + for (auto & [_, table_expression_data] : planner_context->getTableExpressionNodeToData()) + { + if (table_expression_data.getPrewhereFilterActions()) + result_actions_to_execute.push_back(table_expression_data.getPrewhereFilterActions()); + } + if (query_processing_info.isIntermediateStage()) { addPreliminarySortOrDistinctOrLimitStepsIfNeeded(query_plan, diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index ac524a011a9..429815ad23f 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -44,6 +44,264 @@ namespace ErrorCodes namespace { +class ActionNodeNameHelper +{ +public: + ActionNodeNameHelper(QueryTreeNodeToName & node_to_name_, + const PlannerContext & planner_context_, + bool use_column_identifier_as_action_node_name_) + : node_to_name(node_to_name_) + , planner_context(planner_context_) + , use_column_identifier_as_action_node_name(use_column_identifier_as_action_node_name_) + { + } + + String calculateActionNodeName(const QueryTreeNodePtr & node) + { + auto it = node_to_name.find(node); + if (it != node_to_name.end()) + return it->second; + + String result; + auto node_type = node->getNodeType(); + + switch (node_type) + { + case QueryTreeNodeType::COLUMN: + { + const ColumnIdentifier * column_identifier = nullptr; + if (use_column_identifier_as_action_node_name) + column_identifier = planner_context.getColumnNodeIdentifierOrNull(node); + + if (column_identifier) + { + result = *column_identifier; + } + else + { + const auto & column_node = node->as(); + result = column_node.getColumnName(); + } + + break; + } + case QueryTreeNodeType::CONSTANT: + { + const auto & constant_node = node->as(); + result = calculateConstantActionNodeName(constant_node.getValue(), constant_node.getResultType()); + break; + } + case QueryTreeNodeType::FUNCTION: + { + const auto & function_node = node->as(); + String in_function_second_argument_node_name; + + if (isNameOfInFunction(function_node.getFunctionName())) + { + const auto & in_second_argument_node = function_node.getArguments().getNodes().at(1); + in_function_second_argument_node_name = planner_context.createSetKey(in_second_argument_node); + } + + WriteBufferFromOwnString buffer; + buffer << function_node.getFunctionName(); + + const auto & function_parameters_nodes = function_node.getParameters().getNodes(); + + if (!function_parameters_nodes.empty()) + { + buffer << '('; + + size_t function_parameters_nodes_size = function_parameters_nodes.size(); + for (size_t i = 0; i < function_parameters_nodes_size; ++i) + { + const auto & function_parameter_node = function_parameters_nodes[i]; + buffer << calculateActionNodeName(function_parameter_node); + + if (i + 1 != function_parameters_nodes_size) + buffer << ", "; + } + + buffer << ')'; + } + + const auto & function_arguments_nodes = function_node.getArguments().getNodes(); + String function_argument_name; + + buffer << '('; + + size_t function_arguments_nodes_size = function_arguments_nodes.size(); + for (size_t i = 0; i < function_arguments_nodes_size; ++i) + { + if (i == 1 && !in_function_second_argument_node_name.empty()) + { + function_argument_name = in_function_second_argument_node_name; + } + else + { + const auto & function_argument_node = function_arguments_nodes[i]; + function_argument_name = calculateActionNodeName(function_argument_node); + } + + buffer << function_argument_name; + + if (i + 1 != function_arguments_nodes_size) + buffer << ", "; + } + + buffer << ')'; + + if (function_node.isWindowFunction()) + { + buffer << " OVER ("; + buffer << calculateWindowNodeActionName(function_node.getWindowNode()); + buffer << ')'; + } + + result = buffer.str(); + break; + } + case QueryTreeNodeType::LAMBDA: + { + auto lambda_hash = node->getTreeHash(); + result = "__lambda_" + toString(lambda_hash.first) + '_' + toString(lambda_hash.second); + break; + } + default: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid action query tree node {}", node->formatASTForErrorMessage()); + } + } + + node_to_name.emplace(node, result); + + return result; + } + + static String calculateConstantActionNodeName(const Field & constant_literal, const DataTypePtr & constant_type) + { + auto constant_name = applyVisitor(FieldVisitorToString(), constant_literal); + return constant_name + "_" + constant_type->getName(); + } + + static String calculateConstantActionNodeName(const Field & constant_literal) + { + return calculateConstantActionNodeName(constant_literal, applyVisitor(FieldToDataType(), constant_literal)); + } + + String calculateWindowNodeActionName(const QueryTreeNodePtr & node) + { + auto & window_node = node->as(); + WriteBufferFromOwnString buffer; + + if (window_node.hasPartitionBy()) + { + buffer << "PARTITION BY "; + + auto & partition_by_nodes = window_node.getPartitionBy().getNodes(); + size_t partition_by_nodes_size = partition_by_nodes.size(); + + for (size_t i = 0; i < partition_by_nodes_size; ++i) + { + auto & partition_by_node = partition_by_nodes[i]; + buffer << calculateActionNodeName(partition_by_node); + if (i + 1 != partition_by_nodes_size) + buffer << ", "; + } + } + + if (window_node.hasOrderBy()) + { + if (window_node.hasPartitionBy()) + buffer << ' '; + + buffer << "ORDER BY "; + + auto & order_by_nodes = window_node.getOrderBy().getNodes(); + size_t order_by_nodes_size = order_by_nodes.size(); + + for (size_t i = 0; i < order_by_nodes_size; ++i) + { + auto & sort_node = order_by_nodes[i]->as(); + buffer << calculateActionNodeName(sort_node.getExpression()); + + auto sort_direction = sort_node.getSortDirection(); + buffer << (sort_direction == SortDirection::ASCENDING ? " ASC" : " DESC"); + + auto nulls_sort_direction = sort_node.getNullsSortDirection(); + + if (nulls_sort_direction) + buffer << " NULLS " << (nulls_sort_direction == sort_direction ? "LAST" : "FIRST"); + + if (auto collator = sort_node.getCollator()) + buffer << " COLLATE " << collator->getLocale(); + + if (sort_node.withFill()) + { + buffer << " WITH FILL"; + + if (sort_node.hasFillFrom()) + buffer << " FROM " << calculateActionNodeName(sort_node.getFillFrom()); + + if (sort_node.hasFillTo()) + buffer << " TO " << calculateActionNodeName(sort_node.getFillTo()); + + if (sort_node.hasFillStep()) + buffer << " STEP " << calculateActionNodeName(sort_node.getFillStep()); + } + + if (i + 1 != order_by_nodes_size) + buffer << ", "; + } + } + + auto & window_frame = window_node.getWindowFrame(); + if (!window_frame.is_default) + { + if (window_node.hasPartitionBy() || window_node.hasOrderBy()) + buffer << ' '; + + buffer << window_frame.type << " BETWEEN "; + if (window_frame.begin_type == WindowFrame::BoundaryType::Current) + { + buffer << "CURRENT ROW"; + } + else if (window_frame.begin_type == WindowFrame::BoundaryType::Unbounded) + { + buffer << "UNBOUNDED"; + buffer << " " << (window_frame.begin_preceding ? "PRECEDING" : "FOLLOWING"); + } + else + { + buffer << calculateActionNodeName(window_node.getFrameBeginOffsetNode()); + buffer << " " << (window_frame.begin_preceding ? "PRECEDING" : "FOLLOWING"); + } + + buffer << " AND "; + + if (window_frame.end_type == WindowFrame::BoundaryType::Current) + { + buffer << "CURRENT ROW"; + } + else if (window_frame.end_type == WindowFrame::BoundaryType::Unbounded) + { + buffer << "UNBOUNDED"; + buffer << " " << (window_frame.end_preceding ? "PRECEDING" : "FOLLOWING"); + } + else + { + buffer << calculateActionNodeName(window_node.getFrameEndOffsetNode()); + buffer << " " << (window_frame.end_preceding ? "PRECEDING" : "FOLLOWING"); + } + } + + return buffer.str(); + } +private: + std::unordered_map & node_to_name; + const PlannerContext & planner_context; + bool use_column_identifier_as_action_node_name = true; +}; + class ActionsScopeNode { public: @@ -165,7 +423,9 @@ private: class PlannerActionsVisitorImpl { public: - PlannerActionsVisitorImpl(ActionsDAGPtr actions_dag, const PlannerContextPtr & planner_context_); + PlannerActionsVisitorImpl(ActionsDAGPtr actions_dag, + const PlannerContextPtr & planner_context_, + bool use_column_identifier_as_action_node_name_); ActionsDAG::NodeRawConstPtrs visit(QueryTreeNodePtr expression_node); @@ -189,10 +449,14 @@ private: std::vector actions_stack; std::unordered_map node_to_node_name; const PlannerContextPtr planner_context; + ActionNodeNameHelper action_node_name_helper; }; -PlannerActionsVisitorImpl::PlannerActionsVisitorImpl(ActionsDAGPtr actions_dag, const PlannerContextPtr & planner_context_) +PlannerActionsVisitorImpl::PlannerActionsVisitorImpl(ActionsDAGPtr actions_dag, + const PlannerContextPtr & planner_context_, + 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_) { actions_stack.emplace_back(std::move(actions_dag), nullptr); } @@ -236,7 +500,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitColumn(const QueryTreeNodePtr & node) { - auto column_node_name = calculateActionNodeName(node, *planner_context, node_to_node_name); + auto column_node_name = action_node_name_helper.calculateActionNodeName(node); const auto & column_node = node->as(); Int64 actions_stack_size = static_cast(actions_stack.size() - 1); @@ -386,7 +650,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitIndexHintFunction(const QueryTreeNodePtr & node) { const auto & function_node = node->as(); - auto function_node_name = calculateActionNodeName(node, *planner_context, node_to_node_name); + auto function_node_name = action_node_name_helper.calculateActionNodeName(node); auto index_hint_actions_dag = std::make_shared(); auto & index_hint_actions_dag_outputs = index_hint_actions_dag->getOutputs(); @@ -428,7 +692,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi if (isNameOfInFunction(function_node.getFunctionName())) in_function_second_argument_node_name_with_level = makeSetForInFunction(node); - auto function_node_name = calculateActionNodeName(node, *planner_context, node_to_node_name); + auto function_node_name = action_node_name_helper.calculateActionNodeName(node); /* Aggregate functions, window functions, and GROUP BY expressions were already analyzed in the previous steps. * If we have already visited some expression, we don't need to revisit it or its arguments again. @@ -516,266 +780,57 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi } -PlannerActionsVisitor::PlannerActionsVisitor(const PlannerContextPtr & planner_context_) +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_) {} ActionsDAG::NodeRawConstPtrs PlannerActionsVisitor::visit(ActionsDAGPtr actions_dag, QueryTreeNodePtr expression_node) { - PlannerActionsVisitorImpl actions_visitor_impl(actions_dag, planner_context); + PlannerActionsVisitorImpl actions_visitor_impl(actions_dag, planner_context, use_column_identifier_as_action_node_name); return actions_visitor_impl.visit(expression_node); } -String calculateActionNodeName(const QueryTreeNodePtr & node, const PlannerContext & planner_context, QueryTreeNodeToName & node_to_name) +String calculateActionNodeName(const QueryTreeNodePtr & node, + const PlannerContext & planner_context, + QueryTreeNodeToName & node_to_name, + bool use_column_identifier_as_action_node_name) { - auto it = node_to_name.find(node); - if (it != node_to_name.end()) - return it->second; - - String result; - auto node_type = node->getNodeType(); - - switch (node_type) - { - case QueryTreeNodeType::COLUMN: - { - const auto * column_identifier = planner_context.getColumnNodeIdentifierOrNull(node); - - if (column_identifier) - { - result = *column_identifier; - } - else - { - const auto & column_node = node->as(); - result = column_node.getColumnName(); - } - - break; - } - case QueryTreeNodeType::CONSTANT: - { - const auto & constant_node = node->as(); - result = calculateConstantActionNodeName(constant_node.getValue(), constant_node.getResultType()); - break; - } - case QueryTreeNodeType::FUNCTION: - { - const auto & function_node = node->as(); - String in_function_second_argument_node_name; - - if (isNameOfInFunction(function_node.getFunctionName())) - { - const auto & in_second_argument_node = function_node.getArguments().getNodes().at(1); - in_function_second_argument_node_name = planner_context.createSetKey(in_second_argument_node); - } - - WriteBufferFromOwnString buffer; - buffer << function_node.getFunctionName(); - - const auto & function_parameters_nodes = function_node.getParameters().getNodes(); - - if (!function_parameters_nodes.empty()) - { - buffer << '('; - - size_t function_parameters_nodes_size = function_parameters_nodes.size(); - for (size_t i = 0; i < function_parameters_nodes_size; ++i) - { - const auto & function_parameter_node = function_parameters_nodes[i]; - buffer << calculateActionNodeName(function_parameter_node, planner_context, node_to_name); - - if (i + 1 != function_parameters_nodes_size) - buffer << ", "; - } - - buffer << ')'; - } - - const auto & function_arguments_nodes = function_node.getArguments().getNodes(); - String function_argument_name; - - buffer << '('; - - size_t function_arguments_nodes_size = function_arguments_nodes.size(); - for (size_t i = 0; i < function_arguments_nodes_size; ++i) - { - if (i == 1 && !in_function_second_argument_node_name.empty()) - { - function_argument_name = in_function_second_argument_node_name; - } - else - { - const auto & function_argument_node = function_arguments_nodes[i]; - function_argument_name = calculateActionNodeName(function_argument_node, planner_context, node_to_name); - } - - buffer << function_argument_name; - - if (i + 1 != function_arguments_nodes_size) - buffer << ", "; - } - - buffer << ')'; - - if (function_node.isWindowFunction()) - { - buffer << " OVER ("; - buffer << calculateWindowNodeActionName(function_node.getWindowNode(), planner_context, node_to_name); - buffer << ')'; - } - - result = buffer.str(); - break; - } - case QueryTreeNodeType::LAMBDA: - { - auto lambda_hash = node->getTreeHash(); - - result = "__lambda_" + toString(lambda_hash.first) + '_' + toString(lambda_hash.second); - break; - } - default: - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid action query tree node {}", node->formatASTForErrorMessage()); - } - } - - node_to_name.emplace(node, result); - - return result; + ActionNodeNameHelper helper(node_to_name, planner_context, use_column_identifier_as_action_node_name); + return helper.calculateActionNodeName(node); } -String calculateActionNodeName(const QueryTreeNodePtr & node, const PlannerContext & planner_context) +String calculateActionNodeName(const QueryTreeNodePtr & node, const PlannerContext & planner_context, bool use_column_identifier_as_action_node_name) { QueryTreeNodeToName empty_map; - return calculateActionNodeName(node, planner_context, empty_map); + ActionNodeNameHelper helper(empty_map, planner_context, use_column_identifier_as_action_node_name); + return helper.calculateActionNodeName(node); } String calculateConstantActionNodeName(const Field & constant_literal, const DataTypePtr & constant_type) { - auto constant_name = applyVisitor(FieldVisitorToString(), constant_literal); - return constant_name + "_" + constant_type->getName(); + return ActionNodeNameHelper::calculateConstantActionNodeName(constant_literal, constant_type); } String calculateConstantActionNodeName(const Field & constant_literal) { - return calculateConstantActionNodeName(constant_literal, applyVisitor(FieldToDataType(), constant_literal)); + return ActionNodeNameHelper::calculateConstantActionNodeName(constant_literal); } -String calculateWindowNodeActionName(const QueryTreeNodePtr & node, const PlannerContext & planner_context, QueryTreeNodeToName & node_to_name) +String calculateWindowNodeActionName(const QueryTreeNodePtr & node, + const PlannerContext & planner_context, + QueryTreeNodeToName & node_to_name, + bool use_column_identifier_as_action_node_name) { - auto & window_node = node->as(); - WriteBufferFromOwnString buffer; - - if (window_node.hasPartitionBy()) - { - buffer << "PARTITION BY "; - - auto & partition_by_nodes = window_node.getPartitionBy().getNodes(); - size_t partition_by_nodes_size = partition_by_nodes.size(); - - for (size_t i = 0; i < partition_by_nodes_size; ++i) - { - auto & partition_by_node = partition_by_nodes[i]; - buffer << calculateActionNodeName(partition_by_node, planner_context, node_to_name); - if (i + 1 != partition_by_nodes_size) - buffer << ", "; - } - } - - if (window_node.hasOrderBy()) - { - if (window_node.hasPartitionBy()) - buffer << ' '; - - buffer << "ORDER BY "; - - auto & order_by_nodes = window_node.getOrderBy().getNodes(); - size_t order_by_nodes_size = order_by_nodes.size(); - - for (size_t i = 0; i < order_by_nodes_size; ++i) - { - auto & sort_node = order_by_nodes[i]->as(); - buffer << calculateActionNodeName(sort_node.getExpression(), planner_context, node_to_name); - - auto sort_direction = sort_node.getSortDirection(); - buffer << (sort_direction == SortDirection::ASCENDING ? " ASC" : " DESC"); - - auto nulls_sort_direction = sort_node.getNullsSortDirection(); - - if (nulls_sort_direction) - buffer << " NULLS " << (nulls_sort_direction == sort_direction ? "LAST" : "FIRST"); - - if (auto collator = sort_node.getCollator()) - buffer << " COLLATE " << collator->getLocale(); - - if (sort_node.withFill()) - { - buffer << " WITH FILL"; - - if (sort_node.hasFillFrom()) - buffer << " FROM " << calculateActionNodeName(sort_node.getFillFrom(), planner_context, node_to_name); - - if (sort_node.hasFillTo()) - buffer << " TO " << calculateActionNodeName(sort_node.getFillTo(), planner_context, node_to_name); - - if (sort_node.hasFillStep()) - buffer << " STEP " << calculateActionNodeName(sort_node.getFillStep(), planner_context, node_to_name); - } - - if (i + 1 != order_by_nodes_size) - buffer << ", "; - } - } - - auto & window_frame = window_node.getWindowFrame(); - if (!window_frame.is_default) - { - if (window_node.hasPartitionBy() || window_node.hasOrderBy()) - buffer << ' '; - - buffer << window_frame.type << " BETWEEN "; - if (window_frame.begin_type == WindowFrame::BoundaryType::Current) - { - buffer << "CURRENT ROW"; - } - else if (window_frame.begin_type == WindowFrame::BoundaryType::Unbounded) - { - buffer << "UNBOUNDED"; - buffer << " " << (window_frame.begin_preceding ? "PRECEDING" : "FOLLOWING"); - } - else - { - buffer << calculateActionNodeName(window_node.getFrameBeginOffsetNode(), planner_context, node_to_name); - buffer << " " << (window_frame.begin_preceding ? "PRECEDING" : "FOLLOWING"); - } - - buffer << " AND "; - - if (window_frame.end_type == WindowFrame::BoundaryType::Current) - { - buffer << "CURRENT ROW"; - } - else if (window_frame.end_type == WindowFrame::BoundaryType::Unbounded) - { - buffer << "UNBOUNDED"; - buffer << " " << (window_frame.end_preceding ? "PRECEDING" : "FOLLOWING"); - } - else - { - buffer << calculateActionNodeName(window_node.getFrameEndOffsetNode(), planner_context, node_to_name); - buffer << " " << (window_frame.end_preceding ? "PRECEDING" : "FOLLOWING"); - } - } - - return buffer.str(); + ActionNodeNameHelper helper(node_to_name, planner_context, use_column_identifier_as_action_node_name); + return helper.calculateWindowNodeActionName(node); } -String calculateWindowNodeActionName(const QueryTreeNodePtr & node, const PlannerContext & planner_context) +String calculateWindowNodeActionName(const QueryTreeNodePtr & node, const PlannerContext & planner_context, bool use_column_identifier_as_action_node_name) { QueryTreeNodeToName empty_map; - return calculateWindowNodeActionName(node, planner_context, empty_map); + ActionNodeNameHelper helper(empty_map, planner_context, use_column_identifier_as_action_node_name); + return helper.calculateWindowNodeActionName(node); } } diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h index 405031daa40..2a1c166bfc7 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -23,7 +23,7 @@ using PlannerContextPtr = std::shared_ptr; * Preconditions: * 1. Table expression data for table expression nodes is collected in planner context. * For column node, that has column table expression source, identifier for column name in table expression data - * is used as action dag node name. + * is used as action dag node name, if use_column_identifier_as_action_node_name = true. * 2. Sets for IN functions are already collected in planner context. * * During actions build, there is special handling for following functions: @@ -33,7 +33,7 @@ using PlannerContextPtr = std::shared_ptr; class PlannerActionsVisitor { public: - explicit PlannerActionsVisitor(const PlannerContextPtr & planner_context_); + 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. @@ -43,21 +43,27 @@ public: private: const PlannerContextPtr planner_context; + bool use_column_identifier_as_action_node_name = true; }; /** Calculate query tree expression node action dag name and add them into node to name map. * If node exists in map, name from map is used. * - * For column node column node identifier from planner context is used. + * For column node column node identifier from planner context is used, if use_column_identifier_as_action_node_name = true. */ using QueryTreeNodeToName = std::unordered_map; -String calculateActionNodeName(const QueryTreeNodePtr & node, const PlannerContext & planner_context, QueryTreeNodeToName & node_to_name); +String calculateActionNodeName(const QueryTreeNodePtr & node, + const PlannerContext & planner_context, + QueryTreeNodeToName & node_to_name, + bool use_column_identifier_as_action_node_name = true); /** Calculate query tree expression node action dag name. * - * For column node column node identifier from planner context is used. + * For column node column node identifier from planner context is used, if use_column_identifier_as_action_node_name = true. */ -String calculateActionNodeName(const QueryTreeNodePtr & node, const PlannerContext & planner_context); +String calculateActionNodeName(const QueryTreeNodePtr & node, + const PlannerContext & planner_context, + bool use_column_identifier_as_action_node_name = true); /// Calculate action node name for constant String calculateConstantActionNodeName(const Field & constant_literal, const DataTypePtr & constant_type); @@ -67,12 +73,19 @@ String calculateConstantActionNodeName(const Field & constant_literal); /** Calculate action node name for window node. * Window node action name can only be part of window function action name. + * For column node column node identifier from planner context is used, if use_column_identifier_as_action_node_name = true. */ -String calculateWindowNodeActionName(const QueryTreeNodePtr & node, const PlannerContext & planner_context, QueryTreeNodeToName & node_to_name); +String calculateWindowNodeActionName(const QueryTreeNodePtr & node, + const PlannerContext & planner_context, + QueryTreeNodeToName & node_to_name, + bool use_column_identifier_as_action_node_name = true); /** Calculate action node name for window node. * Window node action name can only be part of window function action name. + * For column node column node identifier from planner context is used, if use_column_identifier_as_action_node_name = true. */ -String calculateWindowNodeActionName(const QueryTreeNodePtr & node, const PlannerContext & planner_context); +String calculateWindowNodeActionName(const QueryTreeNodePtr & node, + const PlannerContext & planner_context, + bool use_column_identifier_as_action_node_name = true); } diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index a48cceebfb6..b88fa4f751a 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include @@ -22,13 +23,15 @@ #include #include #include -#include +#include +#include #include #include #include #include #include +#include #include #include #include @@ -44,6 +47,8 @@ #include #include #include +#include +#include #include #include @@ -62,6 +67,7 @@ namespace ErrorCodes extern const int PARAMETER_OUT_OF_BOUND; extern const int TOO_MANY_COLUMNS; extern const int UNSUPPORTED_METHOD; + extern const int BAD_ARGUMENTS; } namespace @@ -163,19 +169,39 @@ bool applyTrivialCountIfPossible( if (!settings.optimize_trivial_count_query) return false; - /// can't apply if FINAL - if (table_node.getTableExpressionModifiers().has_value() && table_node.getTableExpressionModifiers()->hasFinal()) - return false; - - auto & main_query_node = query_tree->as(); - if (main_query_node.hasGroupBy()) - return false; - const auto & storage = table_node.getStorage(); - if (!storage || storage->hasLightweightDeletedMask()) + auto storage_id = storage->getStorageID(); + auto row_policy_filter = query_context->getRowPolicyFilter(storage_id.getDatabaseName(), + storage_id.getTableName(), + RowPolicyFilterType::SELECT_FILTER); + if (row_policy_filter) + return {}; + + /** Transaction check here is necessary because + * MergeTree maintains total count for all parts in Active state and it simply returns that number for trivial select count() from table query. + * But if we have current transaction, then we should return number of rows in current snapshot (that may include parts in Outdated state), + * so we have to use totalRowsByPartitionPredicate() instead of totalRows even for trivial query + * See https://github.com/ClickHouse/ClickHouse/pull/24258/files#r828182031 + */ + if (query_context->getCurrentTransaction()) return false; - if (settings.max_parallel_replicas > 1 || settings.allow_experimental_query_deduplication + /// can't apply if FINAL + if (table_node.getTableExpressionModifiers().has_value() && + (table_node.getTableExpressionModifiers()->hasFinal() || table_node.getTableExpressionModifiers()->hasSampleSizeRatio() || + table_node.getTableExpressionModifiers()->hasSampleOffsetRatio())) + return false; + + // TODO: It's possible to optimize count() given only partition predicates + auto & main_query_node = query_tree->as(); + if (main_query_node.hasGroupBy() || main_query_node.hasPrewhere() || main_query_node.hasWhere()) + return false; + + if (storage->hasLightweightDeletedMask()) + return false; + + if (settings.max_parallel_replicas > 1 || + settings.allow_experimental_query_deduplication || settings.empty_result_for_aggregation_by_empty_set) return false; @@ -189,31 +215,12 @@ bool applyTrivialCountIfPossible( if (!count_func) return false; - /// get number of rows - std::optional num_rows{}; - /// Transaction check here is necessary because - /// MergeTree maintains total count for all parts in Active state and it simply returns that number for trivial select count() from table query. - /// But if we have current transaction, then we should return number of rows in current snapshot (that may include parts in Outdated state), - /// so we have to use totalRowsByPartitionPredicate() instead of totalRows even for trivial query - /// See https://github.com/ClickHouse/ClickHouse/pull/24258/files#r828182031 - if (!main_query_node.hasPrewhere() && !main_query_node.hasWhere() && !query_context->getCurrentTransaction()) - { - num_rows = storage->totalRows(settings); - } - // TODO: - // else // It's possible to optimize count() given only partition predicates - // { - // SelectQueryInfo temp_query_info; - // temp_query_info.query = query_ptr; - // temp_query_info.syntax_analyzer_result = syntax_analyzer_result; - // temp_query_info.prepared_sets = query_analyzer->getPreparedSets(); - // num_rows = storage->totalRowsByPartitionPredicate(temp_query_info, context); - // } - + /// Get number of rows + std::optional num_rows = storage->totalRows(settings); if (!num_rows) return false; - /// set aggregation state + /// Set aggregation state const AggregateFunctionCount & agg_count = *count_func; std::vector state(agg_count.sizeOfData()); AggregateDataPtr place = state.data(); @@ -307,6 +314,70 @@ void prepareBuildQueryPlanForTableExpression(const QueryTreeNodePtr & table_expr settings.max_columns_to_read); } +void updatePrewhereOutputsIfNeeded(SelectQueryInfo & table_expression_query_info, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot) +{ + if (!table_expression_query_info.prewhere_info) + return; + + auto & prewhere_actions = table_expression_query_info.prewhere_info->prewhere_actions; + + NameSet required_columns; + if (column_names.size() == 1) + required_columns.insert(column_names[0]); + + auto & table_expression_modifiers = table_expression_query_info.table_expression_modifiers; + if (table_expression_modifiers) + { + if (table_expression_modifiers->hasSampleSizeRatio() || + table_expression_query_info.planner_context->getQueryContext()->getSettingsRef().parallel_replicas_count > 1) + { + /// We evaluate sampling for Merge lazily so we need to get all the columns + if (storage_snapshot->storage.getName() == "Merge") + { + const auto columns = storage_snapshot->getMetadataForQuery()->getColumns().getAll(); + for (const auto & column : columns) + required_columns.insert(column.name); + } + else + { + auto columns_required_for_sampling = storage_snapshot->getMetadataForQuery()->getColumnsRequiredForSampling(); + required_columns.insert(columns_required_for_sampling.begin(), columns_required_for_sampling.end()); + } + } + + if (table_expression_modifiers->hasFinal()) + { + auto columns_required_for_final = storage_snapshot->getMetadataForQuery()->getColumnsRequiredForFinal(); + required_columns.insert(columns_required_for_final.begin(), columns_required_for_final.end()); + } + } + + std::unordered_set required_output_nodes; + + for (const auto * input : prewhere_actions->getInputs()) + { + if (required_columns.contains(input->result_name)) + required_output_nodes.insert(input); + } + + if (required_output_nodes.empty()) + return; + + auto & prewhere_outputs = prewhere_actions->getOutputs(); + for (const auto & output : prewhere_outputs) + { + auto required_output_node_it = required_output_nodes.find(output); + if (required_output_node_it == required_output_nodes.end()) + continue; + + required_output_nodes.erase(required_output_node_it); + } + + prewhere_outputs.insert(prewhere_outputs.end(), required_output_nodes.begin(), required_output_nodes.end()); +} + JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, const SelectQueryInfo & select_query_info, const SelectQueryOptions & select_query_options, @@ -428,9 +499,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres else table_expression_query_info.table_expression_modifiers = table_function_node->getTableExpressionModifiers(); - auto columns_names = table_expression_data.getColumnNames(); - - bool need_rewrite_query_with_final = storage->needRewriteQueryWithFinal(columns_names); + bool need_rewrite_query_with_final = storage->needRewriteQueryWithFinal(table_expression_data.getColumnNames()); if (need_rewrite_query_with_final) { if (table_expression_query_info.table_expression_modifiers) @@ -452,8 +521,11 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } /// Apply trivial_count optimization if possible - bool is_trivial_count_applied = !select_query_options.only_analyze && is_single_table_expression && table_node && select_query_info.has_aggregates - && applyTrivialCountIfPossible(query_plan, *table_node, select_query_info.query_tree, planner_context->getQueryContext(), columns_names); + bool is_trivial_count_applied = !select_query_options.only_analyze && + is_single_table_expression && + table_node && + select_query_info.has_aggregates && + applyTrivialCountIfPossible(query_plan, *table_node, select_query_info.query_tree, planner_context->getQueryContext(), table_expression_data.getColumnNames()); if (is_trivial_count_applied) { @@ -463,6 +535,20 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres { if (!select_query_options.only_analyze) { + const auto & prewhere_actions = table_expression_data.getPrewhereFilterActions(); + + if (prewhere_actions) + { + table_expression_query_info.prewhere_info = std::make_shared(); + table_expression_query_info.prewhere_info->prewhere_actions = prewhere_actions; + table_expression_query_info.prewhere_info->prewhere_column_name = prewhere_actions->getOutputs().at(0)->result_name; + table_expression_query_info.prewhere_info->remove_prewhere_column = true; + table_expression_query_info.prewhere_info->need_filter = true; + } + + updatePrewhereOutputsIfNeeded(table_expression_query_info, table_expression_data.getColumnNames(), storage_snapshot); + + const auto & columns_names = table_expression_data.getColumnNames(); from_stage = storage->getQueryProcessingStage(query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); storage->read(query_plan, columns_names, storage_snapshot, table_expression_query_info, query_context, from_stage, max_block_size, max_streams); @@ -493,7 +579,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres else { /// Create step which reads from empty source if storage has no data. - auto source_header = storage_snapshot->getSampleBlockForColumns(columns_names); + auto source_header = storage_snapshot->getSampleBlockForColumns(table_expression_data.getColumnNames()); Pipe pipe(std::make_shared(source_header)); auto read_from_pipe = std::make_unique(std::move(pipe)); read_from_pipe->setStepDescription("Read from NullSource"); diff --git a/src/Planner/TableExpressionData.h b/src/Planner/TableExpressionData.h index 0f74e671ac7..e945fe26bb0 100644 --- a/src/Planner/TableExpressionData.h +++ b/src/Planner/TableExpressionData.h @@ -3,6 +3,8 @@ #include #include +#include + namespace DB { @@ -238,6 +240,26 @@ public: is_remote = is_remote_value; } + const ActionsDAGPtr & getPrewhereFilterActions() const + { + return prewhere_filter_actions; + } + + void setPrewhereFilterActions(ActionsDAGPtr prewhere_filter_actions_value) + { + prewhere_filter_actions = std::move(prewhere_filter_actions_value); + } + + const ActionsDAGPtr & getFilterActions() const + { + return filter_actions; + } + + void setFilterActions(ActionsDAGPtr filter_actions_value) + { + filter_actions = std::move(filter_actions_value); + } + private: void addColumnImpl(const NameAndTypePair & column, const ColumnIdentifier & column_identifier) { @@ -262,6 +284,12 @@ private: /// Valid for table, table function, array join, query, union nodes ColumnIdentifierToColumnName column_identifier_to_column_name; + /// Valid for table, table function + ActionsDAGPtr filter_actions; + + /// Valid for table, table function + ActionsDAGPtr prewhere_filter_actions; + /// Is storage remote bool is_remote = false; }; diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 2018ddafcdd..4b669ca0b80 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -192,7 +192,9 @@ StorageLimits buildStorageLimits(const Context & context, const SelectQueryOptio return {limits, leaf_limits}; } -ActionsDAGPtr buildActionsDAGFromExpressionNode(const QueryTreeNodePtr & expression_node, const ColumnsWithTypeAndName & input_columns, const PlannerContextPtr & planner_context) +ActionsDAGPtr buildActionsDAGFromExpressionNode(const QueryTreeNodePtr & expression_node, + const ColumnsWithTypeAndName & input_columns, + const PlannerContextPtr & planner_context) { ActionsDAGPtr action_dag = std::make_shared(input_columns); PlannerActionsVisitor actions_visitor(planner_context); diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 774626bffa4..fbffcc29a9c 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -105,6 +105,7 @@ using Stack = std::vector; /// Second pass optimizations void optimizePrimaryKeyCondition(const Stack & stack); +void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes); void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp new file mode 100644 index 00000000000..87132c72097 --- /dev/null +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -0,0 +1,387 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace +{ + +void matchDAGOutputNodesWithHeader(ActionsDAGPtr & actions_dag, const Block & expected_header) +{ + std::unordered_map output_name_to_node; + for (const auto * output_node : actions_dag->getOutputs()) + output_name_to_node.emplace(output_node->result_name, output_node); + + std::unordered_set used_output_nodes; + + ActionsDAG::NodeRawConstPtrs updated_outputs; + updated_outputs.reserve(expected_header.columns()); + + for (const auto & column : expected_header) + { + auto output_node_it = output_name_to_node.find(column.name); + if (output_node_it == output_name_to_node.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Invalid move to PREWHERE optimization. Cannot find column {} in output", + column.name); + + updated_outputs.push_back(output_node_it->second); + used_output_nodes.insert(output_node_it->second); + } + + ActionsDAG::NodeRawConstPtrs unused_outputs; + for (const auto * output_node : actions_dag->getOutputs()) + { + if (used_output_nodes.contains(output_node)) + continue; + + unused_outputs.push_back(output_node); + } + + auto & actions_dag_outputs = actions_dag->getOutputs(); + actions_dag_outputs = std::move(updated_outputs); + actions_dag_outputs.insert(actions_dag_outputs.end(), unused_outputs.begin(), unused_outputs.end()); +} + +} + + +namespace QueryPlanOptimizations +{ + +void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) +{ + if (stack.size() < 3) + return; + + const auto & frame = stack.back(); + + /** Assume that on stack there are at least 3 nodes: + * + * 1. SomeNode + * 2. FilterNode + * 3. ReadFromMergeTreeNode + */ + auto * read_from_merge_tree = typeid_cast(frame.node->step.get()); + if (!read_from_merge_tree) + return; + + const auto & storage_prewhere_info = read_from_merge_tree->getPrewhereInfo(); + if (storage_prewhere_info && storage_prewhere_info->prewhere_actions) + return; + + const QueryPlan::Node * filter_node = (stack.rbegin() + 1)->node; + const auto * filter_step = typeid_cast(filter_node->step.get()); + if (!filter_step) + return; + + /** Collect required filter output columns. + * Collect output nodes that are mapped to input nodes. + * Collect input node to output nodes mapping. + */ + ColumnsWithTypeAndName required_columns_after_filter; + std::unordered_set output_nodes_mapped_to_input; + std::unordered_map> input_node_to_output_names; + + for (const auto * output_node : filter_step->getExpression()->getOutputs()) + { + if (output_node->type == ActionsDAG::ActionType::INPUT) + { + output_nodes_mapped_to_input.insert(output_node->result_name); + + auto output_names_it = input_node_to_output_names.find(output_node->result_name); + if (output_names_it == input_node_to_output_names.end()) + { + auto [insert_it, _] = input_node_to_output_names.emplace(output_node->result_name, std::vector()); + output_names_it = insert_it; + } + + output_names_it->second.push_back(output_node->result_name); + } + else if (output_node->type == ActionsDAG::ActionType::ALIAS) + { + const auto * node_without_alias = output_node; + while (node_without_alias->type == ActionsDAG::ActionType::ALIAS) + node_without_alias = node_without_alias->children[0]; + + if (node_without_alias->type == ActionsDAG::ActionType::INPUT) + { + output_nodes_mapped_to_input.emplace(output_node->result_name); + + auto output_names_it = input_node_to_output_names.find(node_without_alias->result_name); + if (output_names_it == input_node_to_output_names.end()) + { + auto [insert_it, _] = input_node_to_output_names.emplace(node_without_alias->result_name, std::vector()); + output_names_it = insert_it; + } + + output_names_it->second.push_back(output_node->result_name); + } + } + + if (output_node->result_name == filter_step->getFilterColumnName() && filter_step->removesFilterColumn()) + continue; + + required_columns_after_filter.push_back(ColumnWithTypeAndName(output_node->result_type, output_node->result_name)); + } + + const auto & context = read_from_merge_tree->getContext(); + const auto & settings = context->getSettingsRef(); + + if (!settings.allow_experimental_analyzer) + return; + + const auto & table_expression_modifiers = read_from_merge_tree->getQueryInfo().table_expression_modifiers; + bool is_final = table_expression_modifiers && table_expression_modifiers->hasFinal(); + bool optimize_move_to_prewhere = settings.optimize_move_to_prewhere && (!is_final || settings.optimize_move_to_prewhere_if_final); + if (!optimize_move_to_prewhere) + return; + + const auto & storage = read_from_merge_tree->getStorageSnapshot()->storage; + const auto & storage_metadata = read_from_merge_tree->getStorageSnapshot()->metadata; + auto column_sizes = storage.getColumnSizes(); + if (column_sizes.empty()) + return; + + /// Extract column compressed sizes + std::unordered_map column_compressed_sizes; + for (const auto & [name, sizes] : column_sizes) + column_compressed_sizes[name] = sizes.data_compressed; + + Names queried_columns = read_from_merge_tree->getRealColumnNames(); + + MergeTreeWhereOptimizer where_optimizer{ + std::move(column_compressed_sizes), + storage_metadata, + queried_columns, + storage.supportedPrewhereColumns(), + &Poco::Logger::get("QueryPlanOptimizePrewhere")}; + + auto optimize_result = where_optimizer.optimize(filter_step->getExpression(), + filter_step->getFilterColumnName(), + read_from_merge_tree->getContext(), + is_final); + if (!optimize_result) + return; + + PrewhereInfoPtr prewhere_info; + if (storage_prewhere_info) + prewhere_info = storage_prewhere_info->clone(); + else + prewhere_info = std::make_shared(); + + prewhere_info->need_filter = true; + + auto & prewhere_filter_actions = optimize_result->prewhere_filter_actions; + + ActionsChain actions_chain; + + std::string prewere_filter_node_name = prewhere_filter_actions->getOutputs().at(0)->result_name; + actions_chain.addStep(std::make_unique(prewhere_filter_actions)); + + auto & filter_actions = optimize_result->filter_actions; + + /** Merge tree where optimizer splits conjunctions in filter expression into 2 parts: + * 1. Filter expressions. + * 2. Prewhere filter expressions. + * + * There can be cases when all expressions are moved to PREWHERE, but it is not + * enought to produce required filter output columns. + * + * Example: SELECT (a AND b) AS cond FROM test_table WHERE cond AND c; + * In this example conditions a, b, c can move to PREWHERE, but PREWHERE will not contain expression and(a, b). + * It will contain only a, b, c, and(a, b, c) expressions. + * + * In such scenario we need to create additional filter expressions after PREWHERE. + */ + bool need_additional_filter_after_prewhere = false; + + if (!filter_actions) + { + /// Any node from PREWHERE filter actions can be used as possible output node + std::unordered_set possible_prewhere_output_nodes; + for (const auto & node : prewhere_filter_actions->getNodes()) + possible_prewhere_output_nodes.insert(node.result_name); + + for (auto & required_column : required_columns_after_filter) + { + if (!possible_prewhere_output_nodes.contains(required_column.name) && + !output_nodes_mapped_to_input.contains(required_column.name)) + { + need_additional_filter_after_prewhere = true; + break; + } + } + } + + /** If there are additional filter actions after PREWHERE filter actions, we create filter actions dag using PREWHERE filter + * actions output columns as filter actions dag input columns. + * Then we merge this filter actions dag nodes with old filter step actions dag nodes, to reuse some expressions from + * PREWHERE filter actions. + */ + if (need_additional_filter_after_prewhere || filter_actions) + { + auto merged_filter_actions = std::make_shared(actions_chain.getLastStepAvailableOutputColumns()); + merged_filter_actions->getOutputs().clear(); + merged_filter_actions->mergeNodes(std::move(*filter_step->getExpression()->clone())); + + /// Add old filter step filter column to outputs + for (const auto & node : merged_filter_actions->getNodes()) + { + if (node.result_name == filter_step->getFilterColumnName()) + { + merged_filter_actions->getOutputs().push_back(&node); + break; + } + } + + filter_actions = std::move(merged_filter_actions); + + /// If there is filter after PREWHERE, we can ignore filtering during PREWHERE stage + prewhere_info->need_filter = false; + + actions_chain.addStep(std::make_unique(filter_actions)); + } + + auto required_output_actions = std::make_shared(required_columns_after_filter); + actions_chain.addStep(std::make_unique(required_output_actions)); + + actions_chain.finalize(); + + prewhere_filter_actions->projectInput(false); + + auto & prewhere_actions_chain_node = actions_chain[0]; + prewhere_info->prewhere_actions = std::move(prewhere_filter_actions); + prewhere_info->prewhere_column_name = prewere_filter_node_name; + prewhere_info->remove_prewhere_column = !prewhere_actions_chain_node->getChildRequiredOutputColumnsNames().contains(prewere_filter_node_name); + + read_from_merge_tree->updatePrewhereInfo(prewhere_info); + + QueryPlan::Node * replace_old_filter_node = frame.node; + bool remove_filter_node = false; + + if (filter_actions) + { + filter_actions->projectInput(false); + + /// Match dag output nodes with old filter step header + matchDAGOutputNodesWithHeader(filter_actions, filter_step->getOutputStream().header); + + auto & filter_actions_chain_node = actions_chain[1]; + bool remove_filter_column = !filter_actions_chain_node->getChildRequiredOutputColumnsNames().contains(filter_step->getFilterColumnName()); + auto after_prewhere_filter_step = std::make_unique(read_from_merge_tree->getOutputStream(), + filter_actions, + filter_step->getFilterColumnName(), + remove_filter_column); + + auto & node = nodes.emplace_back(); + node.children.emplace_back(frame.node); + node.step = std::move(after_prewhere_filter_step); + + replace_old_filter_node = &node; + } + else + { + auto rename_actions_dag = std::make_shared(read_from_merge_tree->getOutputStream().header.getColumnsWithTypeAndName()); + bool apply_rename_step = false; + + ActionsDAG::NodeRawConstPtrs updated_outputs; + + /** If in output after read from merge tree there are column names without aliases, + * apply old filter step aliases to them. + */ + for (const auto * output_node : rename_actions_dag->getOutputs()) + { + const auto alias_it = input_node_to_output_names.find(output_node->result_name); + if (alias_it == input_node_to_output_names.end()) + { + updated_outputs.push_back(output_node); + continue; + } + + for (auto & output_name : alias_it->second) + { + if (output_name == output_node->result_name) + { + updated_outputs.push_back(output_node); + continue; + } + + updated_outputs.push_back(&rename_actions_dag->addAlias(*output_node, output_name)); + apply_rename_step = true; + } + } + + rename_actions_dag->getOutputs() = std::move(updated_outputs); + + bool apply_match_step = false; + + /// If column order does not match old filter step column order, match dag output nodes with header + if (!blocksHaveEqualStructure(read_from_merge_tree->getOutputStream().header, filter_step->getOutputStream().header)) + { + apply_match_step = true; + matchDAGOutputNodesWithHeader(rename_actions_dag, filter_step->getOutputStream().header); + } + + if (apply_rename_step || apply_match_step) + { + auto rename_step = std::make_unique(read_from_merge_tree->getOutputStream(), rename_actions_dag); + if (apply_rename_step) + rename_step->setStepDescription("Change column names to column identifiers"); + + auto & node = nodes.emplace_back(); + node.children.emplace_back(frame.node); + node.step = std::move(rename_step); + + replace_old_filter_node = &node; + } + else + { + replace_old_filter_node = frame.node; + remove_filter_node = true; + } + } + + QueryPlan::Node * filter_parent_node = (stack.rbegin() + 2)->node; + + for (auto & filter_parent_child : filter_parent_node->children) + { + if (filter_parent_child == filter_node) + { + filter_parent_child = replace_old_filter_node; + + size_t stack_size = stack.size(); + + /** If filter step is completely replaced with PREWHERE filter actions, remove it from stack. + * Otherwise replace old filter step with new filter step after PREWHERE. + */ + if (remove_filter_node) + { + std::swap(stack[stack_size - 1], stack[stack_size - 2]); + stack.pop_back(); + } + else + { + stack[stack_size - 2] = Frame{.node = replace_old_filter_node, .next_child = 1}; + } + + break; + } + } +} + +} + +} diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 9407504579b..5d0288698e2 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -183,7 +183,7 @@ void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns & IQueryPlanStep * step = node.step.get(); if (auto * reading = typeid_cast(step)) { - if (const auto * prewhere_info = reading->getPrewhereInfo()) + if (const auto prewhere_info = reading->getPrewhereInfo()) { /// Should ignore limit if there is filtering. limit = 0; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 0378c5ef416..c48119ece10 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -131,6 +131,7 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s continue; } + optimizePrewhere(stack, nodes); optimizePrimaryKeyCondition(stack); enableMemoryBoundMerging(*frame.node, nodes); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 51afe96545d..c47f86f1c33 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1399,6 +1399,17 @@ bool ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction, return true; } +void ReadFromMergeTree::updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info_value) +{ + query_info.prewhere_info = prewhere_info_value; + prewhere_info = prewhere_info_value; + output_stream = DataStream{.header = IMergeTreeSelectAlgorithm::transformHeader( + storage_snapshot->getSampleBlockForColumns(real_column_names), + prewhere_info_value, + data.getPartitionValueType(), + virt_column_names)}; +} + bool ReadFromMergeTree::requestOutputEachPartitionThroughSeparatePort() { if (isQueryWithFinal()) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 9b28e903d21..c17c3b23a39 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -121,7 +121,11 @@ public: void describeActions(JSONBuilder::JSONMap & map) const override; void describeIndexes(JSONBuilder::JSONMap & map) const override; + const Names & getRealColumnNames() const { return real_column_names; } + const Names & getVirtualColumnNames() const { return virt_column_names; } + StorageID getStorageID() const { return data.getStorageID(); } + const StorageSnapshotPtr & getStorageSnapshot() const { return storage_snapshot; } UInt64 getSelectedParts() const { return selected_parts; } UInt64 getSelectedRows() const { return selected_rows; } UInt64 getSelectedMarks() const { return selected_marks; } @@ -144,11 +148,13 @@ public: ContextPtr getContext() const { return context; } const SelectQueryInfo & getQueryInfo() const { return query_info; } StorageMetadataPtr getStorageMetadata() const { return metadata_for_reading; } - const PrewhereInfo * getPrewhereInfo() const { return prewhere_info.get(); } + const PrewhereInfoPtr & getPrewhereInfo() const { return prewhere_info; } /// Returns `false` if requested reading cannot be performed. bool requestReadingInOrder(size_t prefix_size, int direction, size_t limit); + void updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info_value); + static bool isFinal(const SelectQueryInfo & query_info); /// Returns true if the optimisation is applicable (and applies it then). diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 0ad91d84d29..5d317d8ee90 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -635,28 +635,33 @@ Block IMergeTreeSelectAlgorithm::applyPrewhereActions(Block block, const Prewher } if (prewhere_info->prewhere_actions) + { block = prewhere_info->prewhere_actions->updateHeader(std::move(block)); - auto & prewhere_column = block.getByName(prewhere_info->prewhere_column_name); - if (!prewhere_column.type->canBeUsedInBooleanContext()) - { - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, "Invalid type for filter in PREWHERE: {}", - prewhere_column.type->getName()); - } + auto & prewhere_column = block.getByName(prewhere_info->prewhere_column_name); + if (!prewhere_column.type->canBeUsedInBooleanContext()) + { + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, "Invalid type for filter in PREWHERE: {}", + prewhere_column.type->getName()); + } - if (prewhere_info->remove_prewhere_column) - block.erase(prewhere_info->prewhere_column_name); - else - { - WhichDataType which(removeNullable(recursiveRemoveLowCardinality(prewhere_column.type))); - if (which.isNativeInt() || which.isNativeUInt()) - prewhere_column.column = prewhere_column.type->createColumnConst(block.rows(), 1u)->convertToFullColumnIfConst(); - else if (which.isFloat()) - prewhere_column.column = prewhere_column.type->createColumnConst(block.rows(), 1.0f)->convertToFullColumnIfConst(); - else - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, - "Illegal type {} of column for filter", prewhere_column.type->getName()); + if (prewhere_info->remove_prewhere_column) + { + block.erase(prewhere_info->prewhere_column_name); + } + else if (prewhere_info->need_filter) + { + WhichDataType which(removeNullable(recursiveRemoveLowCardinality(prewhere_column.type))); + + if (which.isNativeInt() || which.isNativeUInt()) + prewhere_column.column = prewhere_column.type->createColumnConst(block.rows(), 1u); + else if (which.isFloat()) + prewhere_column.column = prewhere_column.type->createColumnConst(block.rows(), 1.0f); + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, + "Illegal type {} of column for filter", + prewhere_column.type->getName()); + } } } diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp index 533875d80cd..bfc674a7eef 100644 --- a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -93,7 +93,13 @@ const ActionsDAG::Node & addClonedDAGToDAG(const ActionsDAG::Node * original_dag return new_node; } - /// TODO: Do we need to handle ALIAS nodes in cloning? + if (original_dag_node->type == ActionsDAG::ActionType::ALIAS) + { + const auto & alias_child = addClonedDAGToDAG(original_dag_node->children[0], new_dag, node_remap); + const auto & new_node = new_dag->addAlias(alias_child, original_dag_node->result_name); + node_remap[node_name] = {new_dag, &new_node}; + return new_node; + } /// If the node is a function, add it as a function and add its children if (original_dag_node->type == ActionsDAG::ActionType::FUNCTION) diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index fdddc29048b..f9a49c35759 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -12,23 +12,18 @@ #include #include #include +#include #include namespace DB { -namespace ErrorCodes -{ -} - /// Conditions like "x = N" are considered good if abs(N) > threshold. /// This is used to assume that condition is likely to have good selectivity. static constexpr auto threshold = 2; MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( - SelectQueryInfo & query_info, - ContextPtr context, std::unordered_map column_sizes_, const StorageMetadataPtr & metadata_snapshot, const Names & queried_columns_, @@ -40,10 +35,8 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( , supported_columns{supported_columns_} , sorting_key_names{NameSet( metadata_snapshot->getSortingKey().column_names.begin(), metadata_snapshot->getSortingKey().column_names.end())} - , block_with_constants{KeyCondition::getBlockWithConstants(query_info.query->clone(), query_info.syntax_analyzer_result, context)} , log{log_} , column_sizes{std::move(column_sizes_)} - , move_all_conditions_to_prewhere(context->getSettingsRef().move_all_conditions_to_prewhere) { for (const auto & name : queried_columns) { @@ -51,180 +44,213 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( if (it != column_sizes.end()) total_size_of_queried_columns += it->second; } - - determineArrayJoinedNames(query_info.query->as()); - optimize(query_info.query->as()); } - -static void collectIdentifiersNoSubqueries(const ASTPtr & ast, NameSet & set) +void MergeTreeWhereOptimizer::optimize(SelectQueryInfo & select_query_info, const ContextPtr & context) const { - if (auto opt_name = tryGetIdentifierName(ast)) - return (void)set.insert(*opt_name); - - if (ast->as()) + auto & select = select_query_info.query->as(); + if (!select.where() || select.prewhere()) return; - for (const auto & child : ast->children) - collectIdentifiersNoSubqueries(child, set); + auto block_with_constants = KeyCondition::getBlockWithConstants(select_query_info.query->clone(), + select_query_info.syntax_analyzer_result, + context); + + WhereOptimizerContext where_optimizer_context; + where_optimizer_context.context = context; + where_optimizer_context.array_joined_names = determineArrayJoinedNames(select); + where_optimizer_context.move_all_conditions_to_prewhere = context->getSettingsRef().move_all_conditions_to_prewhere; + where_optimizer_context.is_final = select.final(); + + RPNBuilderTreeContext tree_context(context, std::move(block_with_constants), {} /*prepared_sets*/); + RPNBuilderTreeNode node(select.where().get(), tree_context); + auto optimize_result = optimizeImpl(node, where_optimizer_context); + if (!optimize_result) + return; + + /// Rewrite the SELECT query. + + auto where_filter_ast = reconstructAST(optimize_result->where_conditions); + auto prewhere_filter_ast = reconstructAST(optimize_result->prewhere_conditions); + + select.setExpression(ASTSelectQuery::Expression::WHERE, std::move(where_filter_ast)); + select.setExpression(ASTSelectQuery::Expression::PREWHERE, std::move(prewhere_filter_ast)); + + LOG_DEBUG(log, "MergeTreeWhereOptimizer: condition \"{}\" moved to PREWHERE", select.prewhere()); } -static bool isConditionGood(const ASTPtr & condition) +std::optional MergeTreeWhereOptimizer::optimize(const ActionsDAGPtr & filter_dag, + const std::string & filter_column_name, + const ContextPtr & context, + bool is_final) { - const auto * function = condition->as(); - if (!function) - return false; + WhereOptimizerContext where_optimizer_context; + where_optimizer_context.context = context; + where_optimizer_context.array_joined_names = {}; + where_optimizer_context.move_all_conditions_to_prewhere = context->getSettingsRef().move_all_conditions_to_prewhere; + where_optimizer_context.is_final = is_final; - /** we are only considering conditions of form `equals(one, another)` or `one = another`, - * especially if either `one` or `another` is ASTIdentifier */ - if (function->name != "equals") - return false; + RPNBuilderTreeContext tree_context(context); + RPNBuilderTreeNode node(&filter_dag->findInOutputs(filter_column_name), tree_context); - auto * left_arg = function->arguments->children.front().get(); - auto * right_arg = function->arguments->children.back().get(); + auto optimize_result = optimizeImpl(node, where_optimizer_context); + if (!optimize_result) + return {}; - /// try to ensure left_arg points to ASTIdentifier - if (!left_arg->as() && right_arg->as()) - std::swap(left_arg, right_arg); + auto filter_actions = reconstructDAG(optimize_result->where_conditions, context); + auto prewhere_filter_actions = reconstructDAG(optimize_result->prewhere_conditions, context); - if (left_arg->as()) + FilterActionsOptimizeResult result = { std::move(filter_actions), std::move(prewhere_filter_actions) }; + return result; +} + +static void collectColumns(const RPNBuilderTreeNode & node, const NameSet & columns_names, NameSet & result_set, bool & has_invalid_column) +{ + if (node.isConstant()) + return; + + if (!node.isFunction()) { - /// condition may be "good" if only right_arg is a constant and its value is outside the threshold - if (const auto * literal = right_arg->as()) + auto column_name = node.getColumnName(); + if (!columns_names.contains(column_name)) { - const auto & field = literal->value; - const auto type = field.getType(); - - /// check the value with respect to threshold - if (type == Field::Types::UInt64) - { - const auto value = field.get(); - return value > threshold; - } - else if (type == Field::Types::Int64) - { - const auto value = field.get(); - return value < -threshold || threshold < value; - } - else if (type == Field::Types::Float64) - { - const auto value = field.get(); - return value < threshold || threshold < value; - } + has_invalid_column = true; + return; } + + result_set.insert(column_name); + return; + } + + auto function_node = node.toFunctionNode(); + size_t arguments_size = function_node.getArgumentsSize(); + for (size_t i = 0; i < arguments_size; ++i) + { + auto function_argument = function_node.getArgumentAt(i); + collectColumns(function_argument, columns_names, result_set, has_invalid_column); + } +} + +static bool isConditionGood(const RPNBuilderTreeNode & condition, const NameSet & columns_names) +{ + if (!condition.isFunction()) + return false; + + auto function_node = condition.toFunctionNode(); + + /** We are only considering conditions of form `equals(one, another)` or `one = another`, + * especially if either `one` or `another` is ASTIdentifier + */ + if (function_node.getFunctionName() != "equals") + return false; + + auto lhs_argument = function_node.getArgumentAt(0); + auto rhs_argument = function_node.getArgumentAt(1); + + auto lhs_argument_column_name = lhs_argument.getColumnName(); + auto rhs_argument_column_name = rhs_argument.getColumnName(); + + bool lhs_argument_is_column = columns_names.contains(lhs_argument_column_name); + bool rhs_argument_is_column = columns_names.contains(rhs_argument_column_name); + + bool lhs_argument_is_constant = lhs_argument.isConstant(); + bool rhs_argument_is_constant = rhs_argument.isConstant(); + + RPNBuilderTreeNode * constant_node = nullptr; + + if (lhs_argument_is_column && rhs_argument_is_constant) + constant_node = &rhs_argument; + else if (lhs_argument_is_constant && rhs_argument_is_column) + constant_node = &lhs_argument; + else + return false; + + Field output_value; + DataTypePtr output_type; + if (!constant_node->tryGetConstant(output_value, output_type)) + return false; + + const auto type = output_value.getType(); + + /// check the value with respect to threshold + if (type == Field::Types::UInt64) + { + const auto value = output_value.get(); + return value > threshold; + } + else if (type == Field::Types::Int64) + { + const auto value = output_value.get(); + return value < -threshold || threshold < value; + } + else if (type == Field::Types::Float64) + { + const auto value = output_value.get(); + return value < threshold || threshold < value; } return false; } -static const ASTFunction * getAsTuple(const ASTPtr & node) +void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTreeNode & node, const WhereOptimizerContext & where_optimizer_context) const { - if (const auto * func = node->as(); func && func->name == "tuple") - return func; - return {}; -} + auto function_node_optional = node.toFunctionNodeOrNull(); -static bool getAsTupleLiteral(const ASTPtr & node, Tuple & tuple) -{ - if (const auto * value_tuple = node->as()) - return value_tuple && value_tuple->value.tryGet(tuple); - return false; -} - -bool MergeTreeWhereOptimizer::tryAnalyzeTuple(Conditions & res, const ASTFunction * func, bool is_final) const -{ - if (!func || func->name != "equals" || func->arguments->children.size() != 2) - return false; - - Tuple tuple_lit; - const ASTFunction * tuple_other = nullptr; - if (getAsTupleLiteral(func->arguments->children[0], tuple_lit)) - tuple_other = getAsTuple(func->arguments->children[1]); - else if (getAsTupleLiteral(func->arguments->children[1], tuple_lit)) - tuple_other = getAsTuple(func->arguments->children[0]); - - if (!tuple_other || tuple_lit.size() != tuple_other->arguments->children.size()) - return false; - - for (size_t i = 0; i < tuple_lit.size(); ++i) + if (function_node_optional.has_value() && function_node_optional->getFunctionName() == "and") { - const auto & child = tuple_other->arguments->children[i]; - std::shared_ptr fetch_sign_column = nullptr; - /// tuple in tuple like (a, (b, c)) = (1, (2, 3)) - if (const auto * child_func = getAsTuple(child)) - fetch_sign_column = std::make_shared(*child_func); - else if (const auto * child_ident = child->as()) - fetch_sign_column = std::make_shared(child_ident->name()); - else - return false; + size_t arguments_size = function_node_optional->getArgumentsSize(); - ASTPtr fetch_sign_value = std::make_shared(tuple_lit.at(i)); - ASTPtr func_node = makeASTFunction("equals", fetch_sign_column, fetch_sign_value); - analyzeImpl(res, func_node, is_final); - } - - return true; -} - -void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const ASTPtr & node, bool is_final) const -{ - const auto * func = node->as(); - - if (func && func->name == "and") - { - for (const auto & elem : func->arguments->children) - analyzeImpl(res, elem, is_final); - } - else if (tryAnalyzeTuple(res, func, is_final)) - { - /// analyzed + for (size_t i = 0; i < arguments_size; ++i) + { + auto argument = function_node_optional->getArgumentAt(i); + analyzeImpl(res, argument, where_optimizer_context); + } } else { - Condition cond; - cond.node = node; + Condition cond(node); + bool has_invalid_column = false; + collectColumns(node, table_columns, cond.table_columns, has_invalid_column); - collectIdentifiersNoSubqueries(node, cond.identifiers); - - cond.columns_size = getIdentifiersColumnSize(cond.identifiers); + cond.columns_size = getIdentifiersColumnSize(cond.table_columns); cond.viable = + !has_invalid_column && /// Condition depend on some column. Constant expressions are not moved. - !cond.identifiers.empty() - && !cannotBeMoved(node, is_final) + !cond.table_columns.empty() + && !cannotBeMoved(node, where_optimizer_context) /// When use final, do not take into consideration the conditions with non-sorting keys. Because final select /// need to use all sorting keys, it will cause correctness issues if we filter other columns before final merge. - && (!is_final || isExpressionOverSortingKey(node)) - /// Only table columns are considered. Not array joined columns. NOTE We're assuming that aliases was expanded. - && isSubsetOfTableColumns(cond.identifiers) + && (!where_optimizer_context.is_final || isExpressionOverSortingKey(node)) /// Some identifiers can unable to support PREWHERE (usually because of different types in Merge engine) - && identifiersSupportsPrewhere(cond.identifiers) + && identifiersSupportsPrewhere(cond.table_columns) /// Do not move conditions involving all queried columns. - && cond.identifiers.size() < queried_columns.size(); + && cond.table_columns.size() < queried_columns.size(); if (cond.viable) - cond.good = isConditionGood(node); + cond.good = isConditionGood(node, table_columns); res.emplace_back(std::move(cond)); } } /// Transform conjunctions chain in WHERE expression to Conditions list. -MergeTreeWhereOptimizer::Conditions MergeTreeWhereOptimizer::analyze(const ASTPtr & expression, bool is_final) const +MergeTreeWhereOptimizer::Conditions MergeTreeWhereOptimizer::analyze(const RPNBuilderTreeNode & node, + const WhereOptimizerContext & where_optimizer_context) const { Conditions res; - analyzeImpl(res, expression, is_final); + analyzeImpl(res, node, where_optimizer_context); return res; } /// Transform Conditions list to WHERE or PREWHERE expression. -ASTPtr MergeTreeWhereOptimizer::reconstruct(const Conditions & conditions) +ASTPtr MergeTreeWhereOptimizer::reconstructAST(const Conditions & conditions) { if (conditions.empty()) return {}; if (conditions.size() == 1) - return conditions.front().node; + return conditions.front().node.getASTNode()->clone(); const auto function = std::make_shared(); @@ -233,18 +259,29 @@ ASTPtr MergeTreeWhereOptimizer::reconstruct(const Conditions & conditions) function->children.push_back(function->arguments); for (const auto & elem : conditions) - function->arguments->children.push_back(elem.node); + function->arguments->children.push_back(elem.node.getASTNode()->clone()); return function; } - -void MergeTreeWhereOptimizer::optimize(ASTSelectQuery & select) const +ActionsDAGPtr MergeTreeWhereOptimizer::reconstructDAG(const Conditions & conditions, const ContextPtr & context) { - if (!select.where() || select.prewhere()) - return; + if (conditions.empty()) + return {}; - Conditions where_conditions = analyze(select.where(), select.final()); + ActionsDAG::NodeRawConstPtrs filter_nodes; + filter_nodes.reserve(conditions.size()); + + for (const auto & condition : conditions) + filter_nodes.push_back(condition.node.getDAGNode()); + + return ActionsDAG::buildFilterActionsDAG(filter_nodes, {} /*node_name_to_input_node_column*/, context); +} + +std::optional MergeTreeWhereOptimizer::optimizeImpl(const RPNBuilderTreeNode & node, + const WhereOptimizerContext & where_optimizer_context) const +{ + Conditions where_conditions = analyze(node, where_optimizer_context); Conditions prewhere_conditions; UInt64 total_size_of_moved_conditions = 0; @@ -255,12 +292,12 @@ void MergeTreeWhereOptimizer::optimize(ASTSelectQuery & select) const { prewhere_conditions.splice(prewhere_conditions.end(), where_conditions, cond_it); total_size_of_moved_conditions += cond_it->columns_size; - total_number_of_moved_columns += cond_it->identifiers.size(); + total_number_of_moved_columns += cond_it->table_columns.size(); /// Move all other viable conditions that depend on the same set of columns. for (auto jt = where_conditions.begin(); jt != where_conditions.end();) { - if (jt->viable && jt->columns_size == cond_it->columns_size && jt->identifiers == cond_it->identifiers) + if (jt->viable && jt->columns_size == cond_it->columns_size && jt->table_columns == cond_it->table_columns) prewhere_conditions.splice(prewhere_conditions.end(), where_conditions, jt++); else ++jt; @@ -277,7 +314,7 @@ void MergeTreeWhereOptimizer::optimize(ASTSelectQuery & select) const if (!it->viable) break; - if (!move_all_conditions_to_prewhere) + if (!where_optimizer_context.move_all_conditions_to_prewhere) { bool moved_enough = false; if (total_size_of_queried_columns > 0) @@ -291,7 +328,7 @@ void MergeTreeWhereOptimizer::optimize(ASTSelectQuery & select) const /// Otherwise, use number of moved columns as a fallback. /// It can happen, if table has only compact parts. 25% ratio is just a guess. moved_enough = total_number_of_moved_columns > 0 - && (total_number_of_moved_columns + it->identifiers.size()) * 4 > queried_columns.size(); + && (total_number_of_moved_columns + it->table_columns.size()) * 4 > queried_columns.size(); } if (moved_enough) @@ -303,14 +340,10 @@ void MergeTreeWhereOptimizer::optimize(ASTSelectQuery & select) const /// Nothing was moved. if (prewhere_conditions.empty()) - return; + return {}; - /// Rewrite the SELECT query. - - select.setExpression(ASTSelectQuery::Expression::WHERE, reconstruct(where_conditions)); - select.setExpression(ASTSelectQuery::Expression::PREWHERE, reconstruct(prewhere_conditions)); - - LOG_DEBUG(log, "MergeTreeWhereOptimizer: condition \"{}\" moved to PREWHERE", select.prewhere()); + OptimizeResult result = {std::move(where_conditions), std::move(prewhere_conditions)}; + return result; } @@ -337,40 +370,36 @@ bool MergeTreeWhereOptimizer::identifiersSupportsPrewhere(const NameSet & identi return true; } -bool MergeTreeWhereOptimizer::isExpressionOverSortingKey(const ASTPtr & ast) const +bool MergeTreeWhereOptimizer::isExpressionOverSortingKey(const RPNBuilderTreeNode & node) const { - if (const auto * func = ast->as()) + if (node.isFunction()) { - const auto & args = func->arguments->children; - for (const auto & arg : args) + auto function_node = node.toFunctionNode(); + size_t arguments_size = function_node.getArgumentsSize(); + + for (size_t i = 0; i < arguments_size; ++i) { - if (isConstant(ast) || sorting_key_names.contains(arg->getColumnName())) + auto argument = function_node.getArgumentAt(i); + auto argument_column_name = argument.getColumnName(); + + if (argument.isConstant() || sorting_key_names.contains(argument_column_name)) continue; - if (!isExpressionOverSortingKey(arg)) + + if (!isExpressionOverSortingKey(argument)) return false; } + return true; } - return isConstant(ast) || sorting_key_names.contains(ast->getColumnName()); + return node.isConstant() || sorting_key_names.contains(node.getColumnName()); } - bool MergeTreeWhereOptimizer::isSortingKey(const String & column_name) const { return sorting_key_names.contains(column_name); } - -bool MergeTreeWhereOptimizer::isConstant(const ASTPtr & expr) const -{ - const auto column_name = expr->getColumnName(); - - return expr->as() - || (block_with_constants.has(column_name) && isColumnConst(*block_with_constants.getByName(column_name).column)); -} - - bool MergeTreeWhereOptimizer::isSubsetOfTableColumns(const NameSet & identifiers) const { for (const auto & identifier : identifiers) @@ -380,52 +409,61 @@ bool MergeTreeWhereOptimizer::isSubsetOfTableColumns(const NameSet & identifiers return true; } - -bool MergeTreeWhereOptimizer::cannotBeMoved(const ASTPtr & ptr, bool is_final) const +bool MergeTreeWhereOptimizer::cannotBeMoved(const RPNBuilderTreeNode & node, const WhereOptimizerContext & where_optimizer_context) const { - if (const auto * function_ptr = ptr->as()) + if (node.isFunction()) { + auto function_node = node.toFunctionNode(); + auto function_name = function_node.getFunctionName(); + /// disallow arrayJoin expressions to be moved to PREWHERE for now - if ("arrayJoin" == function_ptr->name) + if (function_name == "arrayJoin") return true; /// disallow GLOBAL IN, GLOBAL NOT IN /// TODO why? - if ("globalIn" == function_ptr->name - || "globalNotIn" == function_ptr->name) + if (function_name == "globalIn" || function_name == "globalNotIn") return true; /// indexHint is a special function that it does not make sense to transfer to PREWHERE - if ("indexHint" == function_ptr->name) + if (function_name == "indexHint") return true; - } - else if (auto opt_name = IdentifierSemantic::getColumnName(ptr)) - { - /// disallow moving result of ARRAY JOIN to PREWHERE - if (array_joined_names.contains(*opt_name) || - array_joined_names.contains(Nested::extractTableName(*opt_name)) || - (is_final && !isSortingKey(*opt_name))) - return true; - } - for (const auto & child : ptr->children) - if (cannotBeMoved(child, is_final)) + size_t arguments_size = function_node.getArgumentsSize(); + for (size_t i = 0; i < arguments_size; ++i) + { + auto argument = function_node.getArgumentAt(i); + if (cannotBeMoved(argument, where_optimizer_context)) + return true; + } + } + else + { + auto column_name = node.getColumnName(); + + /// disallow moving result of ARRAY JOIN to PREWHERE + if (where_optimizer_context.array_joined_names.contains(column_name) || + where_optimizer_context.array_joined_names.contains(Nested::extractTableName(column_name)) || + (where_optimizer_context.is_final && !isSortingKey(column_name))) return true; + } return false; } - -void MergeTreeWhereOptimizer::determineArrayJoinedNames(ASTSelectQuery & select) +NameSet MergeTreeWhereOptimizer::determineArrayJoinedNames(const ASTSelectQuery & select) { auto [array_join_expression_list, _] = select.arrayJoinExpressionList(); /// much simplified code from ExpressionAnalyzer::getArrayJoinedColumns() if (!array_join_expression_list) - return; + return {}; + NameSet array_joined_names; for (const auto & ast : array_join_expression_list->children) array_joined_names.emplace(ast->getAliasOrColumnName()); + + return array_joined_names; } } diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index 8953923542e..7db2d3a9073 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -34,22 +35,36 @@ class MergeTreeWhereOptimizer : private boost::noncopyable { public: MergeTreeWhereOptimizer( - SelectQueryInfo & query_info, - ContextPtr context, std::unordered_map column_sizes_, const StorageMetadataPtr & metadata_snapshot, const Names & queried_columns_, const std::optional & supported_columns_, Poco::Logger * log_); -private: - void optimize(ASTSelectQuery & select) const; + void optimize(SelectQueryInfo & select_query_info, const ContextPtr & context) const; + struct FilterActionsOptimizeResult + { + ActionsDAGPtr filter_actions; + ActionsDAGPtr prewhere_filter_actions; + }; + + std::optional optimize(const ActionsDAGPtr & filter_dag, + const std::string & filter_column_name, + const ContextPtr & context, + bool is_final); + +private: struct Condition { - ASTPtr node; + explicit Condition(RPNBuilderTreeNode node_) + : node(std::move(node_)) + {} + + RPNBuilderTreeNode node; + UInt64 columns_size = 0; - NameSet identifiers; + NameSet table_columns; /// Can condition be moved to prewhere? bool viable = false; @@ -59,7 +74,7 @@ private: auto tuple() const { - return std::make_tuple(!viable, !good, columns_size, identifiers.size()); + return std::make_tuple(!viable, !good, columns_size, table_columns.size()); } /// Is condition a better candidate for moving to PREWHERE? @@ -71,21 +86,40 @@ private: using Conditions = std::list; - bool tryAnalyzeTuple(Conditions & res, const ASTFunction * func, bool is_final) const; - void analyzeImpl(Conditions & res, const ASTPtr & node, bool is_final) const; + struct WhereOptimizerContext + { + ContextPtr context; + NameSet array_joined_names; + bool move_all_conditions_to_prewhere = false; + bool is_final = false; + }; + + struct OptimizeResult + { + Conditions where_conditions; + Conditions prewhere_conditions; + }; + + std::optional optimizeImpl(const RPNBuilderTreeNode & node, const WhereOptimizerContext & where_optimizer_context) const; + + void analyzeImpl(Conditions & res, const RPNBuilderTreeNode & node, const WhereOptimizerContext & where_optimizer_context) const; /// Transform conjunctions chain in WHERE expression to Conditions list. - Conditions analyze(const ASTPtr & expression, bool is_final) const; + Conditions analyze(const RPNBuilderTreeNode & node, const WhereOptimizerContext & where_optimizer_context) const; - /// Transform Conditions list to WHERE or PREWHERE expression. - static ASTPtr reconstruct(const Conditions & conditions); + /// Reconstruct AST from conditions + static ASTPtr reconstructAST(const Conditions & conditions); + + /// Reconstruct DAG from conditions + static ActionsDAGPtr reconstructDAG(const Conditions & conditions, const ContextPtr & context); void optimizeArbitrary(ASTSelectQuery & select) const; UInt64 getIdentifiersColumnSize(const NameSet & identifiers) const; + bool identifiersSupportsPrewhere(const NameSet & identifiers) const; - bool isExpressionOverSortingKey(const ASTPtr & ast) const; + bool isExpressionOverSortingKey(const RPNBuilderTreeNode & node) const; bool isSortingKey(const String & column_name) const; @@ -99,22 +133,17 @@ private: * * Also, disallow moving expressions with GLOBAL [NOT] IN. */ - bool cannotBeMoved(const ASTPtr & ptr, bool is_final) const; + bool cannotBeMoved(const RPNBuilderTreeNode & node, const WhereOptimizerContext & where_optimizer_context) const; - void determineArrayJoinedNames(ASTSelectQuery & select); + static NameSet determineArrayJoinedNames(const ASTSelectQuery & select); - using StringSet = std::unordered_set; - - const StringSet table_columns; + const NameSet table_columns; const Names queried_columns; const std::optional supported_columns; const NameSet sorting_key_names; - const Block block_with_constants; Poco::Logger * log; std::unordered_map column_sizes; UInt64 total_size_of_queried_columns = 0; - NameSet array_joined_names; - const bool move_all_conditions_to_prewhere = false; }; diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index fb3592a1541..dd6dbf7e02e 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -86,6 +86,16 @@ String getColumnNameWithoutAlias(const ActionsDAG::Node & node, bool allow_exper return std::move(out.str()); } +const ActionsDAG::Node * getNodeWithoutAlias(const ActionsDAG::Node * node) +{ + const ActionsDAG::Node * result = node; + + while (result->type == ActionsDAG::ActionType::ALIAS) + result = result->children[0]; + + return result; +} + } RPNBuilderTreeContext::RPNBuilderTreeContext(ContextPtr query_context_) @@ -137,9 +147,14 @@ std::string RPNBuilderTreeNode::getColumnNameWithModuloLegacy() const bool RPNBuilderTreeNode::isFunction() const { if (ast_node) + { return typeid_cast(ast_node); + } else - return dag_node->type == ActionsDAG::ActionType::FUNCTION; + { + const auto * node_without_alias = getNodeWithoutAlias(dag_node); + return node_without_alias->type == ActionsDAG::ActionType::FUNCTION; + } } bool RPNBuilderTreeNode::isConstant() const @@ -160,7 +175,8 @@ bool RPNBuilderTreeNode::isConstant() const } else { - return dag_node->column && isColumnConst(*dag_node->column); + const auto * node_without_alias = getNodeWithoutAlias(dag_node); + return node_without_alias->column && isColumnConst(*node_without_alias->column); } } @@ -189,8 +205,9 @@ ColumnWithTypeAndName RPNBuilderTreeNode::getConstantColumn() const } else { - result.type = dag_node->result_type; - result.column = dag_node->column; + const auto * node_without_alias = getNodeWithoutAlias(dag_node); + result.type = node_without_alias->result_type; + result.column = node_without_alias->column; } return result; @@ -238,10 +255,12 @@ bool RPNBuilderTreeNode::tryGetConstant(Field & output_value, DataTypePtr & outp } else { - if (dag_node->column && isColumnConst(*dag_node->column)) + const auto * node_without_alias = getNodeWithoutAlias(dag_node); + + if (node_without_alias->column && isColumnConst(*node_without_alias->column)) { - output_value = (*dag_node->column)[0]; - output_type = dag_node->result_type; + output_value = (*node_without_alias->column)[0]; + output_type = node_without_alias->result_type; if (!output_value.isNull()) output_type = removeNullable(output_type); @@ -291,7 +310,8 @@ ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet() const } else if (dag_node) { - return tryGetSetFromDAGNode(dag_node); + const auto * node_without_alias = getNodeWithoutAlias(dag_node); + return tryGetSetFromDAGNode(node_without_alias); } return {}; @@ -310,7 +330,8 @@ ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet(const DataTypes & data_types) } else if (dag_node) { - return tryGetSetFromDAGNode(dag_node); + const auto * node_without_alias = getNodeWithoutAlias(dag_node); + return tryGetSetFromDAGNode(node_without_alias); } return nullptr; @@ -351,9 +372,11 @@ ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet( return set; } } - else if (dag_node->column) + else { - return tryGetSetFromDAGNode(dag_node); + const auto * node_without_alias = getNodeWithoutAlias(dag_node); + if (node_without_alias->column) + return tryGetSetFromDAGNode(node_without_alias); } return nullptr; @@ -364,10 +387,10 @@ RPNBuilderFunctionTreeNode RPNBuilderTreeNode::toFunctionNode() const if (!isFunction()) throw Exception(ErrorCodes::LOGICAL_ERROR, "RPNBuilderTree node is not a function"); - if (this->ast_node) - return RPNBuilderFunctionTreeNode(this->ast_node, tree_context); + if (ast_node) + return RPNBuilderFunctionTreeNode(ast_node, tree_context); else - return RPNBuilderFunctionTreeNode(this->dag_node, tree_context); + return RPNBuilderFunctionTreeNode(getNodeWithoutAlias(dag_node), tree_context); } std::optional RPNBuilderTreeNode::toFunctionNodeOrNull() const @@ -375,10 +398,10 @@ std::optional RPNBuilderTreeNode::toFunctionNodeOrNu if (!isFunction()) return {}; - if (this->ast_node) + if (ast_node) return RPNBuilderFunctionTreeNode(this->ast_node, tree_context); else - return RPNBuilderFunctionTreeNode(this->dag_node, tree_context); + return RPNBuilderFunctionTreeNode(getNodeWithoutAlias(dag_node), tree_context); } std::string RPNBuilderFunctionTreeNode::getFunctionName() const diff --git a/src/Storages/MergeTree/RPNBuilder.h b/src/Storages/MergeTree/RPNBuilder.h index 741821f75fb..626eb288493 100644 --- a/src/Storages/MergeTree/RPNBuilder.h +++ b/src/Storages/MergeTree/RPNBuilder.h @@ -78,6 +78,12 @@ public: /// Construct RPNBuilderTreeNode with non null ast node and tree context explicit RPNBuilderTreeNode(const IAST * ast_node_, RPNBuilderTreeContext & tree_context_); + /// Get AST node + const IAST * getASTNode() const { return ast_node; } + + /// Get DAG node + const ActionsDAG::Node * getDAGNode() const { return dag_node; } + /// Get column name std::string getColumnName() const; From 6c2084052c28f1e63f12e59396b68fe17ad6cdb6 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 14 Mar 2023 21:50:40 +0100 Subject: [PATCH 179/559] Fixed tests --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 1 - src/Planner/Planner.cpp | 1 - src/Planner/PlannerJoinTree.cpp | 1 - src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp | 2 +- src/Storages/LiveView/StorageLiveView.cpp | 5 ----- 5 files changed, 1 insertion(+), 9 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 5353dde2078..e3cb350bf94 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -111,7 +111,6 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int ALIAS_REQUIRED; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ILLEGAL_PREWHERE; extern const int UNKNOWN_TABLE; } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index d7f95a1598f..1e5347b81fe 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -79,7 +79,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int TOO_DEEP_SUBQUERIES; extern const int NOT_IMPLEMENTED; - extern const int ILLEGAL_PREWHERE; } /** ClickHouse query planner. diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index b88fa4f751a..ab35ed5d583 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -67,7 +67,6 @@ namespace ErrorCodes extern const int PARAMETER_OUT_OF_BOUND; extern const int TOO_MANY_COLUMNS; extern const int UNSUPPORTED_METHOD; - extern const int BAD_ARGUMENTS; } namespace diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index 87132c72097..9db2f1210fb 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -198,7 +198,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) * 2. Prewhere filter expressions. * * There can be cases when all expressions are moved to PREWHERE, but it is not - * enought to produce required filter output columns. + * enough to produce required filter output columns. * * Example: SELECT (a AND b) AS cond FROM test_table WHERE cond AND c; * In this example conditions a, b, c can move to PREWHERE, but PREWHERE will not contain expression and(a, b). diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 2c3e452de92..e0c5677f430 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -78,7 +78,6 @@ SelectQueryDescription buildSelectQueryDescription(const ASTPtr & select_query, { ASTPtr inner_query = select_query; std::optional dependent_table_storage_id; - bool allow_experimental_analyzer = context->getSettingsRef().allow_experimental_analyzer; while (true) { @@ -100,10 +99,6 @@ SelectQueryDescription buildSelectQueryDescription(const ASTPtr & select_query, if (auto db_and_table = getDatabaseAndTable(*inner_select_query, 0)) { - const auto * table_expression = getTableExpression(*inner_select_query, 0); - if (allow_experimental_analyzer && table_expression->database_and_table_name->tryGetAlias().empty()) - table_expression->database_and_table_name->setAlias("__dependent_table"); - String select_database_name = db_and_table->database; String select_table_name = db_and_table->table; From 87f8138c55dc5260405b11ef0c67e7509ae38cc7 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 15 Mar 2023 13:06:29 +0100 Subject: [PATCH 180/559] Analyzer support row policy --- src/Planner/CollectTableExpressionData.cpp | 19 ++-- src/Planner/PlannerJoinTree.cpp | 99 ++++++++++++++++++- .../Optimizations/optimizePrewhere.cpp | 2 +- .../00973_live_view_select_prewhere.sql | 15 ++- .../0_stateless/01548_with_totals_having.sql | 2 +- .../01674_where_prewhere_array_crash.sql | 4 +- .../02006_test_positional_arguments.reference | 4 +- .../02006_test_positional_arguments.sql | 4 +- .../02233_with_total_empty_chunk.sql | 4 +- ...75_predicate_push_down_filled_join_fix.sql | 1 + 10 files changed, 123 insertions(+), 31 deletions(-) diff --git a/src/Planner/CollectTableExpressionData.cpp b/src/Planner/CollectTableExpressionData.cpp index b00709ba651..95499380d2b 100644 --- a/src/Planner/CollectTableExpressionData.cpp +++ b/src/Planner/CollectTableExpressionData.cpp @@ -27,8 +27,9 @@ namespace class CollectSourceColumnsVisitor : public InDepthQueryTreeVisitor { public: - explicit CollectSourceColumnsVisitor(PlannerContext & planner_context_) + explicit CollectSourceColumnsVisitor(PlannerContext & planner_context_, QueryTreeNodePtr child_node_to_ignore_ = {}) : planner_context(planner_context_) + , child_node_to_ignore(std::move(child_node_to_ignore_)) {} void visitImpl(QueryTreeNodePtr & node) @@ -78,14 +79,18 @@ public: table_expression_data.addColumn(column_node->getColumn(), column_identifier); } - static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) + bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) { + if (child_node == child_node_to_ignore) + return false; + auto child_node_type = child_node->getNodeType(); return !(child_node_type == QueryTreeNodeType::QUERY || child_node_type == QueryTreeNodeType::UNION); } private: PlannerContext & planner_context; + QueryTreeNodePtr child_node_to_ignore; }; class CollectPrewhereTableExpressionVisitor : public ConstInDepthQueryTreeVisitor @@ -215,14 +220,8 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr } } - CollectSourceColumnsVisitor collect_source_columns_visitor(*planner_context); - for (auto & node : query_node_typed.getChildren()) - { - if (!node || node == query_node_typed.getPrewhere()) - continue; - - collect_source_columns_visitor.visit(node); - } + CollectSourceColumnsVisitor collect_source_columns_visitor(*planner_context, query_node_typed.getPrewhere()); + collect_source_columns_visitor.visit(query_node); if (query_node_typed.hasPrewhere()) { diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index ab35ed5d583..79005612b0a 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1,11 +1,17 @@ #include +#include + +#include + #include #include #include #include +#include + #include #include @@ -50,9 +56,6 @@ #include #include -#include -#include -#include namespace DB { @@ -67,6 +70,7 @@ namespace ErrorCodes extern const int PARAMETER_OUT_OF_BOUND; extern const int TOO_MANY_COLUMNS; extern const int UNSUPPORTED_METHOD; + extern const int BAD_ARGUMENTS; } namespace @@ -377,6 +381,51 @@ 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, + SelectQueryInfo & table_expression_query_info, + PlannerContextPtr & planner_context) +{ + auto storage_id = storage->getStorageID(); + const auto & query_context = planner_context->getQueryContext(); + + auto row_policy_filter = query_context->getRowPolicyFilter(storage_id.getDatabaseName(), storage_id.getTableName(), RowPolicyFilterType::SELECT_FILTER); + if (!row_policy_filter) + return {}; + + auto row_policy_filter_query_tree = buildQueryTree(row_policy_filter->expression, query_context); + + QueryAnalysisPass query_analysis_pass(table_expression_query_info.table_expression); + query_analysis_pass.run(row_policy_filter_query_tree, query_context); + + auto & table_expression_data = planner_context->getTableExpressionDataOrThrow(table_expression_query_info.table_expression); + const auto table_expression_names = table_expression_data.getColumnNames(); + NameSet table_expression_required_names_without_row_policy(table_expression_names.begin(), table_expression_names.end()); + + collectSourceColumns(row_policy_filter_query_tree, planner_context); + collectSets(row_policy_filter_query_tree, *planner_context); + + auto row_policy_actions_dag = std::make_shared(); + + PlannerActionsVisitor actions_visitor(planner_context, false /*use_column_identifier_as_action_node_name*/); + auto expression_nodes = actions_visitor.visit(row_policy_actions_dag, row_policy_filter_query_tree); + if (expression_nodes.size() != 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Row policy filter actions must return single output node. Actual {}", + expression_nodes.size()); + + auto & row_policy_actions_outputs = row_policy_actions_dag->getOutputs(); + row_policy_actions_outputs = std::move(expression_nodes); + + std::string filter_node_name = row_policy_actions_outputs[0]->result_name; + bool remove_filter_column = true; + + for (const auto & row_policy_input_node : row_policy_actions_dag->getInputs()) + if (table_expression_required_names_without_row_policy.contains(row_policy_input_node->result_name)) + row_policy_actions_outputs.push_back(row_policy_input_node); + + return {std::move(row_policy_actions_dag), std::move(filter_node_name), remove_filter_column}; +} + JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, const SelectQueryInfo & select_query_info, const SelectQueryOptions & select_query_options, @@ -547,10 +596,54 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres updatePrewhereOutputsIfNeeded(table_expression_query_info, table_expression_data.getColumnNames(), storage_snapshot); + auto row_policy_filter_info = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context); + bool moved_row_policy_to_prewhere = false; + + if (row_policy_filter_info.actions) + { + bool is_final = table_expression_query_info.table_expression_modifiers && + table_expression_query_info.table_expression_modifiers->hasFinal(); + bool optimize_move_to_prewhere = settings.optimize_move_to_prewhere && (!is_final || settings.optimize_move_to_prewhere_if_final); + + if (storage->supportsPrewhere() && optimize_move_to_prewhere) + { + if (!table_expression_query_info.prewhere_info) + table_expression_query_info.prewhere_info = std::make_shared(); + + if (!table_expression_query_info.prewhere_info->prewhere_actions) + { + table_expression_query_info.prewhere_info->prewhere_actions = row_policy_filter_info.actions; + table_expression_query_info.prewhere_info->prewhere_column_name = row_policy_filter_info.column_name; + table_expression_query_info.prewhere_info->remove_prewhere_column = row_policy_filter_info.do_remove_column; + } + else + { + table_expression_query_info.prewhere_info->row_level_filter = row_policy_filter_info.actions; + table_expression_query_info.prewhere_info->row_level_column_name = row_policy_filter_info.column_name; + } + + table_expression_query_info.prewhere_info->need_filter = true; + moved_row_policy_to_prewhere = true; + } + } + const auto & columns_names = table_expression_data.getColumnNames(); from_stage = storage->getQueryProcessingStage(query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); storage->read(query_plan, columns_names, storage_snapshot, table_expression_query_info, query_context, from_stage, max_block_size, max_streams); + if (query_plan.isInitialized() && + from_stage == QueryProcessingStage::FetchColumns && + row_policy_filter_info.actions && + !moved_row_policy_to_prewhere) + { + auto row_level_filter_step = std::make_unique(query_plan.getCurrentDataStream(), + row_policy_filter_info.actions, + row_policy_filter_info.column_name, + row_policy_filter_info.do_remove_column); + row_level_filter_step->setStepDescription("Row-level security filter"); + query_plan.addStep(std::move(row_level_filter_step)); + } + if (query_context->hasQueryContext() && !select_query_options.is_internal) { auto local_storage_id = storage->getStorageID(); diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index 9db2f1210fb..477aa6a7c75 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -269,7 +269,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) read_from_merge_tree->updatePrewhereInfo(prewhere_info); - QueryPlan::Node * replace_old_filter_node = frame.node; + QueryPlan::Node * replace_old_filter_node = nullptr; bool remove_filter_node = false; if (filter_actions) diff --git a/tests/queries/0_stateless/00973_live_view_select_prewhere.sql b/tests/queries/0_stateless/00973_live_view_select_prewhere.sql index d0d27464618..c40ca5ab410 100644 --- a/tests/queries/0_stateless/00973_live_view_select_prewhere.sql +++ b/tests/queries/0_stateless/00973_live_view_select_prewhere.sql @@ -1,5 +1,6 @@ -- Tags: no-replicated-database, no-parallel, no-fasttest +SET allow_experimental_analyzer = 1; SET allow_experimental_live_view = 1; DROP TABLE IF EXISTS lv; @@ -8,21 +9,17 @@ DROP TABLE IF EXISTS mt; CREATE TABLE mt (a Int32) Engine=MergeTree order by tuple(); CREATE LIVE VIEW lv AS SELECT sum(a) AS sum_a FROM mt PREWHERE a > 1; -CREATE LIVE VIEW lv2 AS SELECT sum(number) AS sum_number FROM system.numbers PREWHERE number > 1; +CREATE LIVE VIEW lv2 AS SELECT sum(number) AS sum_number FROM system.numbers PREWHERE number > 1; -- { serverError 182 } INSERT INTO mt VALUES (1),(2),(3); -SELECT *,_version FROM lv; -SELECT *,_version FROM lv PREWHERE sum_a > 5; -- { serverError 182 } +SELECT *, _version FROM lv; +SELECT *, _version FROM lv PREWHERE sum_a > 5; -- { serverError 182 } INSERT INTO mt VALUES (1),(2),(3); -SELECT *,_version FROM lv; -SELECT *,_version FROM lv PREWHERE sum_a > 10; -- { serverError 182 } - -SELECT *,_version FROM lv2; -- { serverError 182 } -SELECT *,_version FROM lv2 PREWHERE sum_number > 10; -- { serverError 182 } +SELECT *, _version FROM lv; +SELECT *, _version FROM lv PREWHERE sum_a > 10; -- { serverError 182 } DROP TABLE lv; -DROP TABLE lv2; DROP TABLE mt; diff --git a/tests/queries/0_stateless/01548_with_totals_having.sql b/tests/queries/0_stateless/01548_with_totals_having.sql index 669d989aa5f..2562ea3f3e5 100644 --- a/tests/queries/0_stateless/01548_with_totals_having.sql +++ b/tests/queries/0_stateless/01548_with_totals_having.sql @@ -1,2 +1,2 @@ -SELECT * FROM numbers(4) GROUP BY number WITH TOTALS HAVING sum(number) <= arrayJoin([]); -- { serverError 44 } +SELECT * FROM numbers(4) GROUP BY number WITH TOTALS HAVING sum(number) <= arrayJoin([]); -- { serverError 44, 59 } SELECT * FROM numbers(4) GROUP BY number WITH TOTALS HAVING sum(number) <= arrayJoin([3, 2, 1, 0]) ORDER BY number; -- { serverError 44 } diff --git a/tests/queries/0_stateless/01674_where_prewhere_array_crash.sql b/tests/queries/0_stateless/01674_where_prewhere_array_crash.sql index 98094f06509..478e0039177 100644 --- a/tests/queries/0_stateless/01674_where_prewhere_array_crash.sql +++ b/tests/queries/0_stateless/01674_where_prewhere_array_crash.sql @@ -1,5 +1,5 @@ drop table if exists tab; create table tab (x UInt64, `arr.a` Array(UInt64), `arr.b` Array(UInt64)) engine = MergeTree order by x; -select x from tab array join arr prewhere x != 0 where arr; -- { serverError 43, 47 } -select x from tab array join arr prewhere arr where x != 0; -- { serverError 43, 47 } +select x from tab array join arr prewhere x != 0 where arr; -- { serverError 47, 59 } +select x from tab array join arr prewhere arr where x != 0; -- { serverError 47, 59 } drop table if exists tab; diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.reference b/tests/queries/0_stateless/02006_test_positional_arguments.reference index e2bbea2149d..40100e8d5be 100644 --- a/tests/queries/0_stateless/02006_test_positional_arguments.reference +++ b/tests/queries/0_stateless/02006_test_positional_arguments.reference @@ -86,8 +86,8 @@ FROM test GROUP BY 1 + greatest(x1, 1), x2 -select max(x1), x2 from test group by 1, 2; -- { serverError 43 } -select 1 + max(x1), x2 from test group by 1, 2; -- { serverError 43 } +select max(x1), x2 from test group by 1, 2; -- { serverError 43, 184 } +select 1 + max(x1), x2 from test group by 1, 2; -- { serverError 43, 184 } explain syntax select x1 + x3, x3 from test group by 1, 2; SELECT x1 + x3, diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.sql b/tests/queries/0_stateless/02006_test_positional_arguments.sql index 67f4fe24c55..159ad6bd427 100644 --- a/tests/queries/0_stateless/02006_test_positional_arguments.sql +++ b/tests/queries/0_stateless/02006_test_positional_arguments.sql @@ -30,8 +30,8 @@ explain syntax select x3, if(x3 > 10, x3, plus(x1, x2)), x1 + x2 from test order explain syntax select max(x1), x2 from test group by 2 order by 1, 2; explain syntax select 1 + greatest(x1, 1), x2 from test group by 1, 2; -select max(x1), x2 from test group by 1, 2; -- { serverError 43 } -select 1 + max(x1), x2 from test group by 1, 2; -- { serverError 43 } +select max(x1), x2 from test group by 1, 2; -- { serverError 43, 184 } +select 1 + max(x1), x2 from test group by 1, 2; -- { serverError 43, 184 } explain syntax select x1 + x3, x3 from test group by 1, 2; diff --git a/tests/queries/0_stateless/02233_with_total_empty_chunk.sql b/tests/queries/0_stateless/02233_with_total_empty_chunk.sql index bf9ce85b6ed..e1e8186ed76 100644 --- a/tests/queries/0_stateless/02233_with_total_empty_chunk.sql +++ b/tests/queries/0_stateless/02233_with_total_empty_chunk.sql @@ -1 +1,3 @@ -SELECT (NULL, NULL, NULL, NULL, NULL, NULL, NULL) FROM numbers(0) GROUP BY number WITH TOTALS HAVING sum(number) <= arrayJoin([]); +SET allow_experimental_analyzer = 1; + +SELECT (NULL, NULL, NULL, NULL, NULL, NULL, NULL) FROM numbers(0) GROUP BY number WITH TOTALS HAVING sum(number) <= arrayJoin([]) -- { serverError 59 }; diff --git a/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.sql b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.sql index 73baad11634..930127497ae 100644 --- a/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.sql +++ b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.sql @@ -1,5 +1,6 @@ SET allow_experimental_analyzer = 1; SET single_join_prefer_left_table = 0; +SET optimize_move_to_prewhere = 0; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table From 7b108c9788fbc1d33fe4324207e1e96bc12684da Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 15 Mar 2023 15:11:48 +0100 Subject: [PATCH 181/559] Fixed tests --- src/Planner/CollectTableExpressionData.cpp | 23 +++++++++++++--------- 1 file changed, 14 insertions(+), 9 deletions(-) diff --git a/src/Planner/CollectTableExpressionData.cpp b/src/Planner/CollectTableExpressionData.cpp index 95499380d2b..4b44374e8eb 100644 --- a/src/Planner/CollectTableExpressionData.cpp +++ b/src/Planner/CollectTableExpressionData.cpp @@ -27,9 +27,8 @@ namespace class CollectSourceColumnsVisitor : public InDepthQueryTreeVisitor { public: - explicit CollectSourceColumnsVisitor(PlannerContext & planner_context_, QueryTreeNodePtr child_node_to_ignore_ = {}) + explicit CollectSourceColumnsVisitor(PlannerContext & planner_context_) : planner_context(planner_context_) - , child_node_to_ignore(std::move(child_node_to_ignore_)) {} void visitImpl(QueryTreeNodePtr & node) @@ -79,18 +78,14 @@ public: table_expression_data.addColumn(column_node->getColumn(), column_identifier); } - bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) { - if (child_node == child_node_to_ignore) - return false; - auto child_node_type = child_node->getNodeType(); return !(child_node_type == QueryTreeNodeType::QUERY || child_node_type == QueryTreeNodeType::UNION); } private: PlannerContext & planner_context; - QueryTreeNodePtr child_node_to_ignore; }; class CollectPrewhereTableExpressionVisitor : public ConstInDepthQueryTreeVisitor @@ -220,8 +215,18 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr } } - CollectSourceColumnsVisitor collect_source_columns_visitor(*planner_context, query_node_typed.getPrewhere()); - collect_source_columns_visitor.visit(query_node); + CollectSourceColumnsVisitor collect_source_columns_visitor(*planner_context); + for (auto & node : query_node_typed.getChildren()) + { + if (!node || node == query_node_typed.getPrewhere()) + continue; + + auto node_type = node->getNodeType(); + if (node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION) + continue; + + collect_source_columns_visitor.visit(node); + } if (query_node_typed.hasPrewhere()) { From d78a9e03ad0381049b8d0322528b6d2261bce048 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 15 Mar 2023 15:38:11 +0100 Subject: [PATCH 182/559] refine --- src/Common/OptimizedRegularExpression.cpp | 18 +- src/Core/Settings.h | 2 +- src/Dictionaries/RegExpTreeDictionary.cpp | 192 +++++++++++++++------- src/Dictionaries/RegExpTreeDictionary.h | 4 + 4 files changed, 149 insertions(+), 67 deletions(-) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index c2354f14bcb..a8aadd2fecd 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -1,6 +1,3 @@ -#include -#include - #include #include #include @@ -88,18 +85,17 @@ const char * analyzeImpl( if (global_alters.empty()) { - global_alters = std::move(cur_alters); + global_alters = cur_alters; + cur_alters.clear(); return; } - if (shortest_alter_length(global_alters) > shortest_alter_length(cur_alters)) - { - cur_alters.clear(); - } - else + /// that means current alternatives have better quality. + if (shortest_alter_length(global_alters) < shortest_alter_length(cur_alters)) { global_alters.clear(); - global_alters = std::move(cur_alters); + global_alters = cur_alters; } + cur_alters.clear(); }; auto finish_non_trivial_char = [&](bool create_new_substr = true) @@ -140,7 +136,7 @@ const char * analyzeImpl( else { finish_non_trivial_char(); - last_substring->first = std::move(group_required_string).literal; + last_substring->first = group_required_string.literal; } /// if we can still append, no need to finish it. e.g. abc(de)fg should capture abcdefg if (!last_substring->first.empty() && !group_required_string.suffix) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3908254b6f1..0f5e83389c2 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -917,7 +917,7 @@ class IColumn; M(Bool, input_format_bson_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip fields with unsupported types while schema inference for format BSON.", 0) \ \ M(Bool, regexp_dict_allow_other_sources, false, "Allow regexp_tree dictionary to use sources other than yaml source.", 0) \ - M(Bool, regexp_dict_allow_hyperscan, false, "Allow regexp_tree dictionary using Hyperscan library.", 0) \ + M(Bool, regexp_dict_allow_hyperscan, true, "Allow regexp_tree dictionary using Hyperscan library.", 0) \ // End of FORMAT_FACTORY_SETTINGS // Please add settings non-related to formats into the COMMON_SETTINGS above. diff --git a/src/Dictionaries/RegExpTreeDictionary.cpp b/src/Dictionaries/RegExpTreeDictionary.cpp index caba2a52a51..4e8ae4260d0 100644 --- a/src/Dictionaries/RegExpTreeDictionary.cpp +++ b/src/Dictionaries/RegExpTreeDictionary.cpp @@ -4,14 +4,16 @@ #include #include +#include #include #include #include -#include "Common/Exception.h" #include +#include #include +#include #include #include #include @@ -172,10 +174,6 @@ void RegExpTreeDictionary::initRegexNodes(Block & block) auto keys_column = block.getByName(kKeys).column; auto values_column = block.getByName(kValues).column; -#ifdef USE_VECTORSCAN - SlowWithHyperscanChecker checker; -#endif - size_t size = block.rows(); for (size_t i = 0; i < size; i++) { @@ -219,12 +217,36 @@ void RegExpTreeDictionary::initRegexNodes(Block & block) } } regex_nodes.emplace(id, node); -#if USE_VECTORSCAN - if (use_vectorscan && !checker.isSlow(regex)) + +#ifdef USE_VECTORSCAN + String required_substring; + bool is_trivial, required_substring_is_prefix; + std::vector alternatives; + + if (use_vectorscan) + OptimizedRegularExpression::analyze(regex, required_substring, is_trivial, required_substring_is_prefix, alternatives); + + for (auto & alter : alternatives) { - simple_regexps.push_back(regex); + if (alter.size() < 3) + { + alternatives.clear(); + break; + } + } + if (!required_substring.empty()) + { + simple_regexps.push_back(required_substring); regexp_ids.push_back(id); } + else if (!alternatives.empty()) + { + for (auto & alter : alternatives) + { + simple_regexps.push_back(alter); + regexp_ids.push_back(id); + } + } else #endif complex_regexp_nodes.push_back(node); @@ -284,20 +306,52 @@ void RegExpTreeDictionary::loadData() use_vectorscan = false; if (!use_vectorscan) return; - #if USE_VECTORSCAN - try + +#ifdef USE_VECTORSCAN + std::vector patterns; + std::vector flags; + std::vector lens; + + for (const std::string & ref : simple_regexps) { - std::vector regexps_views(simple_regexps.begin(), simple_regexps.end()); - hyperscan_regex = MultiRegexps::getOrSet(regexps_views, std::nullopt); - hyperscan_regex->get(); + patterns.push_back(ref.data()); + lens.push_back(ref.size()); + flags.push_back(HS_FLAG_SINGLEMATCH); } - catch (Exception & e) + + hs_database_t * db = nullptr; + hs_compile_error_t * compile_error; + + std::unique_ptr ids; + ids.reset(new unsigned int[patterns.size()]); + for (size_t i = 0; i < patterns.size(); i++) + ids[i] = static_cast(i+1); + + hs_error_t err = hs_compile_lit_multi(patterns.data(), flags.data(), ids.get(), lens.data(), static_cast(patterns.size()), HS_MODE_BLOCK, nullptr, &db, &compile_error); + origin_db = (db); + if (err != HS_SUCCESS) { - /// Some compile errors will be thrown as LOGICAL ERROR and cause crash, e.g. empty expression or expressions are too large. - /// We catch the error here and rethrow again. - throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Error occurs when compiling regular expressions, reason: {}", e.message()); + /// CompilerError is a unique_ptr, so correct memory free after the exception is thrown. + MultiRegexps::CompilerErrorPtr error(compile_error); + + if (error->expression < 0) + throw Exception::createRuntime(ErrorCodes::LOGICAL_ERROR, String(error->message)); + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Pattern '{}' failed with error '{}'", patterns[error->expression], String(error->message)); } - #endif + + ProfileEvents::increment(ProfileEvents::RegexpCreated); + + /// We allocate the scratch space only once, then copy it across multiple threads with hs_clone_scratch + /// function which is faster than allocating scratch space each time in each thread. + hs_scratch_t * scratch = nullptr; + err = hs_alloc_scratch(db, &scratch); + origin_scratch.reset(scratch); + /// If not HS_SUCCESS, it is guaranteed that the memory would not be allocated for scratch. + if (err != HS_SUCCESS) + throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not allocate scratch space for vectorscan"); +#endif + } else { @@ -396,47 +450,70 @@ bool RegExpTreeDictionary::setAttributes( return attributes_to_set.size() == attributes.size(); } -namespace +/// a temp struct to store all the matched result. +struct MatchContext { - struct MatchContext + std::set matched_idx_set; + std::vector> matched_idx_sorted_list; + + const std::vector & regexp_ids ; + const std::unordered_map & topology_order; + const char * data; + size_t length; + const std::map & regex_nodes; + + size_t pre_match_counter = 0; + size_t match_counter = 0; + + MatchContext( + const std::vector & regexp_ids_, + const std::unordered_map & topology_order_, + const char * data_, size_t length_, + const std::map & regex_nodes_ + ) + : regexp_ids(regexp_ids_), + topology_order(topology_order_), + data(data_), + length(length_), + regex_nodes(regex_nodes_) + {} + + [[maybe_unused]] + void insertIdx(unsigned int idx) { - std::set matched_idx_set; - std::vector> matched_idx_sorted_list; - - const std::vector & regexp_ids ; - const std::unordered_map & topology_order; - - MatchContext(const std::vector & regexp_ids_, const std::unordered_map & topology_order_) - : regexp_ids(regexp_ids_), topology_order(topology_order_) {} - - [[maybe_unused]] - void insertIdx(unsigned int idx) + UInt64 node_id = regexp_ids[idx-1]; + pre_match_counter++; + if (!regex_nodes.at(node_id)->match(data, length)) { - UInt64 node_id = regexp_ids[idx-1]; - UInt64 topological_order = topology_order.at(node_id); - matched_idx_set.emplace(node_id); - matched_idx_sorted_list.push_back(std::make_pair(topological_order, node_id)); + return; } + match_counter++; + matched_idx_set.emplace(node_id); - void insertNodeID(UInt64 id) - { - UInt64 topological_order = topology_order.at(id); - matched_idx_set.emplace(id); - matched_idx_sorted_list.push_back(std::make_pair(topological_order, id)); - } + UInt64 topological_order = topology_order.at(node_id); + matched_idx_sorted_list.push_back(std::make_pair(topological_order, node_id)); + } - /// Sort by topological order, which indicates the matching priorities. - void sort() - { - std::sort(matched_idx_sorted_list.begin(), matched_idx_sorted_list.end()); - } + [[maybe_unused]] + void insertNodeID(UInt64 id) + { + matched_idx_set.emplace(id); - bool contains(UInt64 idx) const - { - return matched_idx_set.contains(idx); - } - }; -} + UInt64 topological_order = topology_order.at(id); + matched_idx_sorted_list.push_back(std::make_pair(topological_order, id)); + } + + /// Sort by topological order, which indicates the matching priorities. + void sort() + { + std::sort(matched_idx_sorted_list.begin(), matched_idx_sorted_list.end()); + } + + bool contains(UInt64 idx) const + { + return matched_idx_set.contains(idx); + } +}; std::unordered_map RegExpTreeDictionary::match( const ColumnString::Chars & keys_data, @@ -449,7 +526,7 @@ std::unordered_map RegExpTreeDictionary::match( hs_scratch_t * scratch = nullptr; if (use_vectorscan) { - hs_error_t err = hs_clone_scratch(hyperscan_regex->get()->getScratch(), &scratch); + hs_error_t err = hs_clone_scratch(origin_scratch.get(), &scratch); if (err != HS_SUCCESS) { @@ -476,11 +553,14 @@ std::unordered_map RegExpTreeDictionary::match( auto key_offset = keys_offsets[key_idx]; UInt64 length = key_offset - offset - 1; - MatchContext match_result(regexp_ids, topology_order); + const char * begin = reinterpret_cast(keys_data.data()) + offset; + + MatchContext match_result(regexp_ids, topology_order, begin, length, regex_nodes); #if USE_VECTORSCAN if (use_vectorscan) { + /// pre-select all the possible matches auto on_match = [](unsigned int id, unsigned long long /* from */, // NOLINT unsigned long long /* to */, // NOLINT @@ -490,8 +570,9 @@ std::unordered_map RegExpTreeDictionary::match( static_cast(context)->insertIdx(id); return 0; }; + hs_error_t err = hs_scan( - hyperscan_regex->get()->getDB(), + origin_db, reinterpret_cast(keys_data.data()) + offset, static_cast(length), 0, @@ -501,6 +582,7 @@ std::unordered_map RegExpTreeDictionary::match( if (err != HS_SUCCESS) throw Exception(ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT, "Failed to scan data with vectorscan"); + } #endif diff --git a/src/Dictionaries/RegExpTreeDictionary.h b/src/Dictionaries/RegExpTreeDictionary.h index 32206f25429..87cdfb808d1 100644 --- a/src/Dictionaries/RegExpTreeDictionary.h +++ b/src/Dictionaries/RegExpTreeDictionary.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -33,6 +34,7 @@ namespace ErrorCodes class RegExpTreeDictionary : public IDictionary { + friend struct MatchContext; public: struct Configuration { @@ -162,6 +164,8 @@ private: std::unordered_map topology_order; #if USE_VECTORSCAN MultiRegexps::DeferredConstructedRegexpsPtr hyperscan_regex; + MultiRegexps::ScratchPtr origin_scratch; + hs_database_t* origin_db; #endif Poco::Logger * logger; From dc1a6884ca69a2c9a9b7e316356c32097c43cec0 Mon Sep 17 00:00:00 2001 From: alexX512 Date: Wed, 15 Mar 2023 14:49:24 +0000 Subject: [PATCH 183/559] Add documentation for a setting stop_reading_on_first_cancel --- docs/en/operations/settings/settings.md | 29 +++++++++++++++++++++++++ docs/ru/operations/settings/settings.md | 29 +++++++++++++++++++++++++ 2 files changed, 58 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 37c6841225b..f705dd292d1 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4049,3 +4049,32 @@ Possible values: - 1 - enabled Default value: `0`. + +## stop_reading_on_first_cancel {#stop_reading_on_first_cancel} +When set to true and the user wants to interrupt a query (for example using Ctrl+C on the client), then the query continues execution only on data that was already read from the table. Afterward, it will return a partial result of the query for the part of the table that was read. To fully stop the execution of a query without a partial result, the user should send 2 cancel requests. + +**Example without setting on Ctrl+C** +```sql +SELECT sum(number) FROM numbers(10000000000) + +Cancelling query. +Ok. +Query was cancelled. + +0 rows in set. Elapsed: 1.334 sec. Processed 52.65 million rows, 421.23 MB (39.48 million rows/s., 315.85 MB/s.) +``` + +**Example with setting on Ctrl+C** +```sql +SELECT sum(number) FROM numbers(10000000000) SETTINGS stop_reading_on_first_cancel=true + +┌──────sum(number)─┐ +│ 1355411451286266 │ +└──────────────────┘ + +1 row in set. Elapsed: 1.331 sec. Processed 52.13 million rows, 417.05 MB (39.17 million rows/s., 313.33 MB/s.) +``` + +Possible values: `true`, `false` + +Default value: `false` diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 4025966ac21..d77f7ba6a46 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4084,3 +4084,32 @@ ALTER TABLE test FREEZE SETTINGS alter_partition_verbose_result = 1; Задает символ, который интерпретируется как суффикс после результирующего набора данных формата [CustomSeparated](../../interfaces/formats.md#format-customseparated). Значение по умолчанию: `''`. + +## stop_reading_on_first_cancel {#stop_reading_on_first_cancel} +Если установлено значение `true` и пользователь хочет прервать запрос (например, с помощью `Ctrl+C` на клиенте), то запрос продолжает выполнение только для данных, которые уже были считаны из таблицы. После этого он вернет частичный результат запроса для той части таблицы, которая была прочитана. Чтобы полностью остановить выполнение запроса без частичного результата, пользователь должен отправить 2 запроса отмены. + +**Пример с выключенной настройкой при нажатии Ctrl+C** +```sql +SELECT sum(number) FROM numbers(10000000000) + +Cancelling query. +Ok. +Query was cancelled. + +0 rows in set. Elapsed: 1.334 sec. Processed 52.65 million rows, 421.23 MB (39.48 million rows/s., 315.85 MB/s.) +``` + +**Пример с включенной настройкой при нажатии Ctrl+C** +```sql +SELECT sum(number) FROM numbers(10000000000) SETTINGS stop_reading_on_first_cancel=true + +┌──────sum(number)─┐ +│ 1355411451286266 │ +└──────────────────┘ + +1 row in set. Elapsed: 1.331 sec. Processed 52.13 million rows, 417.05 MB (39.17 million rows/s., 313.33 MB/s.) +``` + +Возможные значения:: `true`, `false` + +Значение по умолчанию: `false` From 424e8df9ad8493f904f54ca2d9d63f0a32b6b8d7 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 15 Mar 2023 16:01:12 +0100 Subject: [PATCH 184/559] fix style --- src/Dictionaries/RegExpTreeDictionary.cpp | 5 ++--- src/Dictionaries/RegExpTreeDictionary.h | 1 - 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Dictionaries/RegExpTreeDictionary.cpp b/src/Dictionaries/RegExpTreeDictionary.cpp index 4e8ae4260d0..b0f3dcb0249 100644 --- a/src/Dictionaries/RegExpTreeDictionary.cpp +++ b/src/Dictionaries/RegExpTreeDictionary.cpp @@ -4,7 +4,6 @@ #include #include -#include #include #include @@ -36,6 +35,7 @@ #if USE_VECTORSCAN # include +# include #endif namespace DB @@ -48,6 +48,7 @@ namespace ErrorCodes extern const int HYPERSCAN_CANNOT_SCAN_TEXT; extern const int UNSUPPORTED_METHOD; extern const int INCORRECT_DICTIONARY_DEFINITION; + extern const int LOGICAL_ERROR; } const std::string kRegExp = "regexp"; @@ -340,8 +341,6 @@ void RegExpTreeDictionary::loadData() throw Exception(ErrorCodes::BAD_ARGUMENTS, "Pattern '{}' failed with error '{}'", patterns[error->expression], String(error->message)); } - ProfileEvents::increment(ProfileEvents::RegexpCreated); - /// We allocate the scratch space only once, then copy it across multiple threads with hs_clone_scratch /// function which is faster than allocating scratch space each time in each thread. hs_scratch_t * scratch = nullptr; diff --git a/src/Dictionaries/RegExpTreeDictionary.h b/src/Dictionaries/RegExpTreeDictionary.h index 87cdfb808d1..17a0c6bbef3 100644 --- a/src/Dictionaries/RegExpTreeDictionary.h +++ b/src/Dictionaries/RegExpTreeDictionary.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include From 1f9c84e74757ca588ebb60b5f3fffd2eea9a6795 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 15 Mar 2023 16:53:01 +0000 Subject: [PATCH 185/559] Revert "Un-friend system view (no longer necessary)" This reverts commit b2aa9324ac5f6ed0a46bba0ec66d7033d98d0d82. --- src/Interpreters/Cache/QueryCache.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index 4c91d7f1ae7..763e797ac07 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -157,6 +157,8 @@ private: size_t max_entry_size_in_rows TSA_GUARDED_BY(mutex) = 0; size_t cache_size_in_bytes TSA_GUARDED_BY(mutex) = 0; /// Updated in each cache insert/delete + + friend class StorageSystemQueryCache; }; using QueryCachePtr = std::shared_ptr; From 1f21ac8fcf448cfa988d565c7e87902d9d0fc61c Mon Sep 17 00:00:00 2001 From: alexX512 Date: Wed, 15 Mar 2023 19:53:58 +0000 Subject: [PATCH 186/559] Fix fast test --- src/Server/TCPHandler.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index b018d7de580..7a931464020 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1337,7 +1337,6 @@ bool TCPHandler::receivePacket() return false; case Protocol::Client::Cancel: - { decreaseCancellationStatus("Received 'Cancel' packet from the client, canceling the query."); return false; @@ -1356,11 +1355,13 @@ bool TCPHandler::receivePacket() } } + void TCPHandler::receiveIgnoredPartUUIDs() { readVectorBinary(state.part_uuids_to_ignore.emplace(), *in); } + void TCPHandler::receiveUnexpectedIgnoredPartUUIDs() { std::vector skip_part_uuids; @@ -1368,6 +1369,7 @@ void TCPHandler::receiveUnexpectedIgnoredPartUUIDs() throw NetException(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, "Unexpected packet IgnoredPartUUIDs received from client"); } + String TCPHandler::receiveReadTaskResponseAssumeLocked() { UInt64 packet_type = 0; @@ -1416,6 +1418,7 @@ std::optional TCPHandler::receivePartitionMergeTreeReadTas return response; } + void TCPHandler::receiveClusterNameAndSalt() { readStringBinary(cluster, *in); @@ -1671,6 +1674,7 @@ bool TCPHandler::receiveData(bool scalar) return true; } + bool TCPHandler::receiveUnexpectedData(bool throw_exception) { String skip_external_table_name; @@ -1694,6 +1698,7 @@ bool TCPHandler::receiveUnexpectedData(bool throw_exception) return read_ok; } + void TCPHandler::initBlockInput() { if (!state.block_in) @@ -1764,6 +1769,7 @@ void TCPHandler::initLogsBlockOutput(const Block & block) } } + void TCPHandler::initProfileEventsBlockOutput(const Block & block) { if (!state.profile_events_block_out) @@ -1852,6 +1858,7 @@ QueryState::CancellationStatus TCPHandler::getQueryCancellationStatus() return state.cancellation_status; } + void TCPHandler::sendData(const Block & block) { initBlockOutput(block); @@ -1908,6 +1915,7 @@ void TCPHandler::sendData(const Block & block) } } + void TCPHandler::sendLogData(const Block & block) { initLogsBlockOutput(block); From 67ac858a5273b13992676c70236e7dd893f75041 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 14 Mar 2023 12:36:26 +0100 Subject: [PATCH 187/559] remove counters for threads, fix negative counters --- src/Client/ClientBase.cpp | 47 ++- src/Common/CurrentThread.cpp | 13 - src/Common/CurrentThread.h | 22 +- src/Common/ProgressIndication.cpp | 48 +-- src/Common/ProgressIndication.h | 11 +- src/Common/ThreadStatus.cpp | 78 ++--- src/Common/ThreadStatus.h | 119 ++++---- src/Interpreters/ProcessList.cpp | 5 +- src/Interpreters/ProfileEventsExt.cpp | 31 +- src/Interpreters/ThreadStatusExt.cpp | 289 ++++++++---------- .../tests/gtest_lru_file_cache.cpp | 5 +- .../Transforms/buildPushingToViewsChain.cpp | 7 +- src/Server/TCPHandler.cpp | 1 + ...events_from_query_log_and_client.reference | 23 ++ ...rofile_events_from_query_log_and_client.sh | 69 +++++ 15 files changed, 353 insertions(+), 415 deletions(-) create mode 100644 tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference create mode 100755 tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 876b1d1906d..ce922f32cfb 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -117,6 +117,11 @@ namespace ProfileEvents extern const Event SystemTimeMicroseconds; } +namespace +{ +constexpr UInt64 THREAD_GROUP_ID = 0; +} + namespace DB { @@ -195,8 +200,14 @@ static void incrementProfileEventsBlock(Block & dst, const Block & src) } }; std::map rows_by_name; + for (size_t src_row = 0; src_row < src.rows(); ++src_row) { + /// Filter out threads stats, use thead group stats + auto thread_id = src_array_thread_id[src_row]; + if (thread_id != THREAD_GROUP_ID) + continue; + Id id{ src_column_name.getDataAt(src_row), src_column_host_name.getDataAt(src_row), @@ -204,16 +215,6 @@ static void incrementProfileEventsBlock(Block & dst, const Block & src) rows_by_name[id] = src_row; } - /// Filter out snapshots - std::set thread_id_filter_mask; - for (size_t i = 0; i < src_array_thread_id.size(); ++i) - { - if (src_array_thread_id[i] != 0) - { - thread_id_filter_mask.emplace(i); - } - } - /// Merge src into dst. for (size_t dst_row = 0; dst_row < dst_rows; ++dst_row) { @@ -225,10 +226,6 @@ static void incrementProfileEventsBlock(Block & dst, const Block & src) if (auto it = rows_by_name.find(id); it != rows_by_name.end()) { size_t src_row = it->second; - if (thread_id_filter_mask.contains(src_row)) - { - continue; - } dst_array_current_time[dst_row] = src_array_current_time[src_row]; @@ -249,11 +246,6 @@ static void incrementProfileEventsBlock(Block & dst, const Block & src) /// Copy rows from src that dst does not contains. for (const auto & [id, pos] : rows_by_name) { - if (thread_id_filter_mask.contains(pos)) - { - continue; - } - for (size_t col = 0; col < src.columns(); ++col) { mutable_columns[col]->insert((*src.getByPosition(col).column)[pos]); @@ -1080,13 +1072,18 @@ void ClientBase::onProfileEvents(Block & block) const auto * user_time_name = ProfileEvents::getName(ProfileEvents::UserTimeMicroseconds); const auto * system_time_name = ProfileEvents::getName(ProfileEvents::SystemTimeMicroseconds); - HostToThreadTimesMap thread_times; + HostToTimesMap thread_times; for (size_t i = 0; i < rows; ++i) { auto thread_id = array_thread_id[i]; auto host_name = host_names.getDataAt(i).toString(); - if (thread_id != 0) - progress_indication.addThreadIdToList(host_name, thread_id); + + /// In ProfileEvents packets thread id 0 specifies common profiling information + /// for all threads executing current query on specific host. So instead of summing per thread + /// consumption it's enough to look for data with thread id 0. + if (thread_id != THREAD_GROUP_ID) + continue; + auto event_name = names.getDataAt(i); auto value = array_values[i]; @@ -1095,11 +1092,11 @@ void ClientBase::onProfileEvents(Block & block) continue; if (event_name == user_time_name) - thread_times[host_name][thread_id].user_ms = value; + thread_times[host_name].user_ms = value; else if (event_name == system_time_name) - thread_times[host_name][thread_id].system_ms = value; + thread_times[host_name].system_ms = value; else if (event_name == MemoryTracker::USAGE_EVENT_NAME) - thread_times[host_name][thread_id].memory_usage = value; + thread_times[host_name].memory_usage = value; } progress_indication.updateThreadEventData(thread_times); diff --git a/src/Common/CurrentThread.cpp b/src/Common/CurrentThread.cpp index 188e78fe69b..ee1486fec7b 100644 --- a/src/Common/CurrentThread.cpp +++ b/src/Common/CurrentThread.cpp @@ -65,22 +65,12 @@ void CurrentThread::attachInternalTextLogsQueue(const std::shared_ptrattachInternalTextLogsQueue(logs_queue, client_logs_level); } -void CurrentThread::setFatalErrorCallback(std::function callback) -{ - /// It does not make sense to set a callback for sending logs to a client if there's no thread status - chassert(current_thread); - current_thread->setFatalErrorCallback(callback); -} - std::shared_ptr CurrentThread::getInternalTextLogsQueue() { /// NOTE: this method could be called at early server startup stage if (unlikely(!current_thread)) return nullptr; - if (current_thread->getCurrentState() == ThreadStatus::ThreadState::Died) - return nullptr; - return current_thread->getInternalTextLogsQueue(); } @@ -96,9 +86,6 @@ InternalProfileEventsQueuePtr CurrentThread::getInternalProfileEventsQueue() if (unlikely(!current_thread)) return nullptr; - if (current_thread->getCurrentState() == ThreadStatus::ThreadState::Died) - return nullptr; - return current_thread->getInternalProfileEventsQueue(); } diff --git a/src/Common/CurrentThread.h b/src/Common/CurrentThread.h index f4975e800ca..3010085c514 100644 --- a/src/Common/CurrentThread.h +++ b/src/Common/CurrentThread.h @@ -48,8 +48,6 @@ public: static void attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & queue); static InternalProfileEventsQueuePtr getInternalProfileEventsQueue(); - static void setFatalErrorCallback(std::function callback); - /// Makes system calls to update ProfileEvents that contain info from rusage and taskstats static void updatePerformanceCounters(); @@ -65,17 +63,16 @@ public: static void updateProgressIn(const Progress & value); static void updateProgressOut(const Progress & value); - /// Query management: - - /// Call from master thread as soon as possible (e.g. when thread accepted connection) - static void initializeQuery(); - /// You must call one of these methods when create a query child thread: /// Add current thread to a group associated with the thread group static void attachTo(const ThreadGroupStatusPtr & thread_group); /// Is useful for a ThreadPool tasks static void attachToIfDetached(const ThreadGroupStatusPtr & thread_group); + /// Non-master threads call this method in destructor automatically + static void detachGroupIfNotDetached(); + static void detachQueryIfNotDetached(); + /// Update ProfileEvents and dumps info to system.query_thread_log static void finalizePerformanceCounters(); @@ -87,10 +84,6 @@ public: return current_thread->getQueryId(); } - /// Non-master threads call this method in destructor automatically - static void detachQuery(); - static void detachQueryIfNotDetached(); - /// Initializes query with current thread as master thread in constructor, and detaches it in destructor struct QueryScope : private boost::noncopyable { @@ -101,13 +94,6 @@ public: void logPeakMemoryUsage(); bool log_peak_memory_usage_in_destructor = true; }; - -private: - static void defaultThreadDeleter(); - - /// Sets query_context for current thread group - /// Can by used only through QueryScope - static void attachQueryContext(ContextPtr query_context); }; } diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index b049edcdcf7..df8778eb0d1 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -15,24 +15,6 @@ /// http://en.wikipedia.org/wiki/ANSI_escape_code #define CLEAR_TO_END_OF_LINE "\033[K" - -namespace -{ - constexpr UInt64 ALL_THREADS = 0; - - UInt64 aggregateCPUUsageNs(DB::ThreadIdToTimeMap times) - { - constexpr UInt64 us_to_ns = 1000; - return us_to_ns * std::accumulate(times.begin(), times.end(), 0ull, - [](UInt64 acc, const auto & elem) - { - if (elem.first == ALL_THREADS) - return acc; - return acc + elem.second.time(); - }); - } -} - namespace DB { @@ -58,7 +40,7 @@ void ProgressIndication::resetProgress() { std::lock_guard lock(profile_events_mutex); cpu_usage_meter.reset(getElapsedNanoseconds()); - thread_data.clear(); + hosts_data.clear(); } } @@ -71,25 +53,17 @@ void ProgressIndication::setFileProgressCallback(ContextMutablePtr context, Writ }); } -void ProgressIndication::addThreadIdToList(String const & host, UInt64 thread_id) +void ProgressIndication::updateThreadEventData(HostToTimesMap & new_hosts_data) { std::lock_guard lock(profile_events_mutex); - auto & thread_to_times = thread_data[host]; - if (thread_to_times.contains(thread_id)) - return; - thread_to_times[thread_id] = {}; -} - -void ProgressIndication::updateThreadEventData(HostToThreadTimesMap & new_thread_data) -{ - std::lock_guard lock(profile_events_mutex); + constexpr UInt64 us_to_ns = 1000; UInt64 total_cpu_ns = 0; - for (auto & new_host_map : new_thread_data) + for (auto & new_host : new_hosts_data) { - total_cpu_ns += aggregateCPUUsageNs(new_host_map.second); - thread_data[new_host_map.first] = std::move(new_host_map.second); + total_cpu_ns += us_to_ns * new_host.second.time(); + hosts_data[new_host.first] = new_host.second; } cpu_usage_meter.add(getElapsedNanoseconds(), total_cpu_ns); } @@ -104,16 +78,10 @@ ProgressIndication::MemoryUsage ProgressIndication::getMemoryUsage() const { std::lock_guard lock(profile_events_mutex); - return std::accumulate(thread_data.cbegin(), thread_data.cend(), MemoryUsage{}, + return std::accumulate(hosts_data.cbegin(), hosts_data.cend(), MemoryUsage{}, [](MemoryUsage const & acc, auto const & host_data) { - UInt64 host_usage = 0; - // In ProfileEvents packets thread id 0 specifies common profiling information - // for all threads executing current query on specific host. So instead of summing per thread - // memory consumption it's enough to look for data with thread id 0. - if (auto it = host_data.second.find(ALL_THREADS); it != host_data.second.end()) - host_usage = it->second.memory_usage; - + UInt64 host_usage = host_data.second.memory_usage; return MemoryUsage{.total = acc.total + host_usage, .max = std::max(acc.max, host_usage)}; }); } diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index 717de5debb9..af5d69c0255 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -24,8 +24,7 @@ struct ThreadEventData UInt64 memory_usage = 0; }; -using ThreadIdToTimeMap = std::unordered_map; -using HostToThreadTimesMap = std::unordered_map; +using HostToTimesMap = std::unordered_map; class ProgressIndication { @@ -56,9 +55,7 @@ public: /// How much seconds passed since query execution start. double elapsedSeconds() const { return getElapsedNanoseconds() / 1e9; } - void addThreadIdToList(String const & host, UInt64 thread_id); - - void updateThreadEventData(HostToThreadTimesMap & new_thread_data); + void updateThreadEventData(HostToTimesMap & new_hosts_data); private: double getCPUUsage(); @@ -91,7 +88,7 @@ private: bool write_progress_on_update = false; EventRateMeter cpu_usage_meter{static_cast(clock_gettime_ns()), 2'000'000'000 /*ns*/}; // average cpu utilization last 2 second - HostToThreadTimesMap thread_data; + HostToTimesMap hosts_data; /// In case of all of the above: /// - clickhouse-local /// - input_format_parallel_parsing=true @@ -99,7 +96,7 @@ private: /// /// It is possible concurrent access to the following: /// - writeProgress() (class properties) (guarded with progress_mutex) - /// - thread_data/cpu_usage_meter (guarded with profile_events_mutex) + /// - hosts_data/cpu_usage_meter (guarded with profile_events_mutex) mutable std::mutex profile_events_mutex; mutable std::mutex progress_mutex; }; diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index b5a48c48ffe..ef383aa5e5e 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -71,23 +71,34 @@ static thread_local ThreadStack alt_stack; static thread_local bool has_alt_stack = false; #endif +ContextWeakPtr ThreadGroupStatus::getQueryContextWeak() const +{ + return query_context; +} -std::vector ThreadGroupStatus::getProfileEventsCountersAndMemoryForThreads() +ContextWeakPtr ThreadGroupStatus::getGlobalContextWeak() const +{ + return global_context; +} + +ThreadGroupStatus::FatalErrorCallback ThreadGroupStatus::getFatalErrorCallback() const +{ + return fatal_error_callback; +} + +void ThreadGroupStatus::link(ThreadStatusPtr thread) +{ + std::lock_guard lock(mutex); + + /// NOTE: thread may be attached multiple times if it is reused from a thread pool. + thread_ids.insert(thread->thread_id); + threads.insert(thread); +} + +void ThreadGroupStatus::unlink(ThreadStatusPtr thread) { std::lock_guard guard(mutex); - - /// It is OK to move it, since it is enough to report statistics for the thread at least once. - auto stats = std::move(finished_threads_counters_memory); - for (auto * thread : threads) - { - stats.emplace_back(ProfileEventsCountersAndMemory{ - thread->performance_counters.getPartiallyAtomicSnapshot(), - thread->memory_tracker.get(), - thread->thread_id, - }); - } - - return stats; + threads.erase(thread); } ThreadStatus::ThreadStatus() @@ -157,24 +168,11 @@ ThreadStatus::~ThreadStatus() { flushUntrackedMemory(); - if (thread_group) - { - ThreadGroupStatus::ProfileEventsCountersAndMemory counters - { - performance_counters.getPartiallyAtomicSnapshot(), - memory_tracker.get(), - thread_id - }; - - std::lock_guard guard(thread_group->mutex); - thread_group->finished_threads_counters_memory.emplace_back(std::move(counters)); - thread_group->threads.erase(this); - } - /// It may cause segfault if query_context was destroyed, but was not detached auto query_context_ptr = query_context.lock(); assert((!query_context_ptr && query_id.empty()) || (query_context_ptr && query_id == query_context_ptr->getCurrentQueryId())); + /// detachGroup if it was attached if (deleter) deleter(); @@ -200,22 +198,22 @@ void ThreadStatus::updatePerformanceCounters() void ThreadStatus::assertState(ThreadState permitted_state, const char * description) const { - if (getCurrentState() == permitted_state) + auto curr_state = thread_state.load(); + + if (curr_state == permitted_state) return; if (description) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected thread state {}: {}", getCurrentState(), description); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected thread state {}: {}", curr_state, description); else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected thread state {}", getCurrentState()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected thread state {}", curr_state); } void ThreadStatus::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue, LogsLevel client_logs_level) { logs_queue_ptr = logs_queue; - - if (!thread_group) - return; + chassert(thread_group); std::lock_guard lock(thread_group->mutex); thread_group->logs_queue_ptr = logs_queue; @@ -226,22 +224,12 @@ void ThreadStatus::attachInternalProfileEventsQueue(const InternalProfileEventsQ { profile_queue_ptr = profile_queue; - if (!thread_group) - return; + chassert(thread_group); std::lock_guard lock(thread_group->mutex); thread_group->profile_queue_ptr = profile_queue; } -void ThreadStatus::setFatalErrorCallback(std::function callback) -{ - /// It does not make sense to set a callback for sending logs to a client if there's no thread group - chassert(thread_group); - std::lock_guard lock(thread_group->mutex); - fatal_error_callback = std::move(callback); - thread_group->fatal_error_callback = fatal_error_callback; -} - void ThreadStatus::onFatalError() { /// No thread group - no callback diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index e620413c8eb..162defd8bd5 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -57,46 +57,58 @@ using ThreadStatusPtr = ThreadStatus *; * Create via CurrentThread::initializeQuery (for queries) or directly (for various background tasks). * Use via CurrentThread::getGroup. */ +class ThreadGroupStatus; +using ThreadGroupStatusPtr = std::shared_ptr; + class ThreadGroupStatus { public: - struct ProfileEventsCountersAndMemory - { - ProfileEvents::Counters::Snapshot counters; - Int64 memory_usage; - UInt64 thread_id; - }; - - mutable std::mutex mutex; + /// The first thread created this thread group + UInt64 master_thread_id = 0; ProfileEvents::Counters performance_counters{VariableContext::Process}; MemoryTracker memory_tracker{VariableContext::Process}; - ContextWeakPtr query_context; - ContextWeakPtr global_context; + /// Access to the members below has to be in critical section with mutex + mutable std::mutex mutex; InternalTextLogsQueueWeakPtr logs_queue_ptr; InternalProfileEventsQueueWeakPtr profile_queue_ptr; - std::function fatal_error_callback; - - std::unordered_set thread_ids; - std::unordered_set threads; - - /// The first thread created this thread group - UInt64 master_thread_id = 0; LogsLevel client_logs_level = LogsLevel::none; String query; UInt64 normalized_query_hash = 0; - std::vector finished_threads_counters_memory; + /// When new query starts, new thread group is created for it, current thread becomes master thread of the query + static ThreadGroupStatusPtr createForQuery(ContextPtr query_context_, std::function fatal_error_callback_ = {}); - std::vector getProfileEventsCountersAndMemoryForThreads(); + const std::vector getInvolvedThreadIds() const; + + void link(ThreadStatusPtr thread); + void unlink(ThreadStatusPtr thread); + + ContextWeakPtr getQueryContextWeak() const; + ContextWeakPtr getGlobalContextWeak() const; + + using FatalErrorCallback = std::function; + FatalErrorCallback getFatalErrorCallback() const; + +private: + /// Set up at creation, no race when reading + ContextWeakPtr query_context; + ContextWeakPtr global_context; + + /// Set up at creation, no race when reading + FatalErrorCallback fatal_error_callback; + + /// Set of all thread ids which has been attached to the group + std::unordered_set thread_ids; + + /// Set of active threads + std::unordered_set threads; }; -using ThreadGroupStatusPtr = std::shared_ptr; - /** * We use **constinit** here to tell the compiler the current_thread variable is initialized. * If we didn't help the compiler, then it would most likely add a check before every use of the variable to initialize it if needed. @@ -124,12 +136,11 @@ public: /// TODO: merge them into common entity ProfileEvents::Counters performance_counters{VariableContext::Thread}; - /// Points to performance_counters by default. /// Could be changed to point to another object to calculate performance counters for some narrow scope. ProfileEvents::Counters * current_performance_counters{&performance_counters}; - MemoryTracker memory_tracker{VariableContext::Thread}; + MemoryTracker memory_tracker{VariableContext::Thread}; /// Small amount of untracked memory (per thread atomic-less counter) Int64 untracked_memory = 0; /// Each thread could new/delete memory in range of (-untracked_memory_limit, untracked_memory_limit) without access to common counters. @@ -139,9 +150,6 @@ public: Progress progress_in; Progress progress_out; - using Deleter = std::function; - Deleter deleter; - protected: /// Group of threads, to which this thread attached ThreadGroupStatusPtr thread_group; @@ -153,18 +161,25 @@ protected: /// Use it only from current thread ContextWeakPtr query_context; - String query_id; + String query_id_from_query_context; /// A logs queue used by TCPHandler to pass logs to a client InternalTextLogsQueueWeakPtr logs_queue_ptr; InternalProfileEventsQueueWeakPtr profile_queue_ptr; + struct TimePoint + { + void setUp(); + void SetUp(std::chrono::time_point now); + + UInt64 nanoseconds = 0; + UInt64 microseconds = 0; + time_t seconds = 0; + }; + bool performance_counters_finalized = false; - UInt64 query_start_time_nanoseconds = 0; - UInt64 query_start_time_microseconds = 0; - time_t query_start_time = 0; - size_t queries_started = 0; + TimePoint query_start_time{}; // CPU and Real time query profilers std::unique_ptr query_profiler_real; @@ -172,8 +187,6 @@ protected: Poco::Logger * log = nullptr; - friend class CurrentThread; - /// Use ptr not to add extra dependencies in the header std::unique_ptr last_rusage; std::unique_ptr taskstats; @@ -188,9 +201,13 @@ protected: friend class MemoryTrackerThreadSwitcher; void setQueryId(const String & query_id_) { - query_id = query_id_; + query_id_from_query_context = query_id_; } + /// This is helpful for cut linking dependencies for clickhouse_common_io + using Deleter = std::function; + Deleter deleter; + public: ThreadStatus(); ~ThreadStatus(); @@ -204,17 +221,11 @@ public: { DetachedFromQuery = 0, /// We just created thread or it is a background thread AttachedToQuery, /// Thread executes enqueued query - Died, /// Thread does not exist }; - int getCurrentState() const - { - return thread_state.load(std::memory_order_relaxed); - } - std::string_view getQueryId() const { - return query_id; + return query_id_from_query_context; } auto getQueryContext() const @@ -243,11 +254,11 @@ public: /// query. void setInternalThread(); - /// Starts new query and create new thread group for it, current thread becomes master thread of the query - void initializeQuery(); - /// Attaches slave thread to existing thread group - void attachQuery(const ThreadGroupStatusPtr & thread_group_, bool check_detached = true); + void attachTo(const ThreadGroupStatusPtr & thread_group_, bool check_detached = true); + + /// Detaches thread from the thread group and the query, dumps performance counters if they have not been dumped + void detachGroup(); /// Returns pointer to the current profile counters to restore them back. /// Note: consequent call with new scope will detach previous scope. @@ -255,7 +266,7 @@ public: InternalTextLogsQueuePtr getInternalTextLogsQueue() const { - return thread_state == Died ? nullptr : logs_queue_ptr.lock(); + return logs_queue_ptr.lock(); } void attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue, @@ -263,19 +274,14 @@ public: InternalProfileEventsQueuePtr getInternalProfileEventsQueue() const { - return thread_state == Died ? nullptr : profile_queue_ptr.lock(); + return profile_queue_ptr.lock(); } void attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue); - /// Callback that is used to trigger sending fatal error messages to client. - void setFatalErrorCallback(std::function callback); + /// Proper cal for fatal_error_callback void onFatalError(); - /// Sets query context for current master thread and its thread group - /// NOTE: query_context have to be alive until detachQuery() is called - void attachQueryContext(ContextPtr query_context); - /// Update several ProfileEvents counters void updatePerformanceCounters(); @@ -285,9 +291,6 @@ public: /// Set the counters last usage to now void resetPerformanceCountersLastUsage(); - /// Detaches thread from the thread group and the query, dumps performance counters if they have not been dumped - void detachQuery(bool exit_if_already_detached = false, bool thread_exits = false); - void logToQueryViewsLog(const ViewRuntimeData & vinfo); void flushUntrackedMemory(); @@ -303,12 +306,10 @@ protected: void logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database, std::chrono::time_point now); - void assertState(ThreadState permitted_state, const char * description = nullptr) const; - private: - void setupState(const ThreadGroupStatusPtr & thread_group_); + void attachGroupImp(const ThreadGroupStatusPtr & thread_group_); }; /** diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index b792ea538ae..9f557bf3ff5 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -576,10 +576,7 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even res.peak_memory_usage = thread_group->memory_tracker.getPeak(); if (get_thread_list) - { - std::lock_guard lock(thread_group->mutex); - res.thread_ids.assign(thread_group->thread_ids.begin(), thread_group->thread_ids.end()); - } + res.thread_ids = thread_group->getInvolvedThreadIds(); if (get_profile_events) res.profile_counters = std::make_shared(thread_group->performance_counters.getPartiallyAtomicSnapshot()); diff --git a/src/Interpreters/ProfileEventsExt.cpp b/src/Interpreters/ProfileEventsExt.cpp index 0f6b52b2611..7fbbe3c662b 100644 --- a/src/Interpreters/ProfileEventsExt.cpp +++ b/src/Interpreters/ProfileEventsExt.cpp @@ -113,34 +113,10 @@ void getProfileEvents( block = std::move(temp_columns); MutableColumns columns = block.mutateColumns(); auto thread_group = CurrentThread::getGroup(); - auto const current_thread_id = CurrentThread::get().thread_id; - std::vector snapshots; ThreadIdToCountersSnapshot new_snapshots; + ProfileEventsSnapshot group_snapshot; { - auto stats = thread_group->getProfileEventsCountersAndMemoryForThreads(); - snapshots.reserve(stats.size()); - - for (auto & stat : stats) - { - auto const thread_id = stat.thread_id; - if (thread_id == current_thread_id) - continue; - auto current_time = time(nullptr); - auto previous_snapshot = last_sent_snapshots.find(thread_id); - auto increment = - previous_snapshot != last_sent_snapshots.end() - ? CountersIncrement(stat.counters, previous_snapshot->second) - : CountersIncrement(stat.counters); - snapshots.push_back(ProfileEventsSnapshot{ - thread_id, - std::move(increment), - stat.memory_usage, - current_time - }); - new_snapshots[thread_id] = std::move(stat.counters); - } - group_snapshot.thread_id = 0; group_snapshot.current_time = time(nullptr); group_snapshot.memory_usage = thread_group->memory_tracker.get(); @@ -154,11 +130,6 @@ void getProfileEvents( } last_sent_snapshots = std::move(new_snapshots); - for (auto & snapshot : snapshots) - { - dumpProfileEvents(snapshot, columns, server_display_name); - dumpMemoryTracker(snapshot, columns, server_display_name); - } dumpProfileEvents(group_snapshot, columns, server_display_name); dumpMemoryTracker(group_snapshot, columns, server_display_name); diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index da4f97c1483..70c4b15dd42 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -40,13 +40,39 @@ namespace ErrorCodes extern const int CANNOT_SET_THREAD_PRIORITY; } +const std::vector ThreadGroupStatus::getInvolvedThreadIds() const +{ + std::vector res; + + { + std::lock_guard lock(mutex); + res.assign(thread_ids.begin(), thread_ids.end()); + } + + return res; +} + +ThreadGroupStatusPtr ThreadGroupStatus::createForQuery(ContextPtr query_context_, std::function fatal_error_callback_) +{ + auto group = std::make_shared(); + group->memory_tracker.setDescription("(for query)"); + group->master_thread_id = CurrentThread::get().thread_id; + + group->query_context = query_context_; + group->global_context = query_context_->getGlobalContext(); + + group->fatal_error_callback = std::move(fatal_error_callback_); + + return group; +} + void ThreadStatus::applyQuerySettings() { auto query_context_ptr = query_context.lock(); assert(query_context_ptr); const Settings & settings = query_context_ptr->getSettingsRef(); - query_id = query_context_ptr->getCurrentQueryId(); + query_id_from_query_context = query_context_ptr->getCurrentQueryId(); initQueryProfiler(); untracked_memory_limit = settings.max_untracked_memory; @@ -68,34 +94,7 @@ void ThreadStatus::applyQuerySettings() #endif } - -void ThreadStatus::attachQueryContext(ContextPtr query_context_) -{ - query_context = query_context_; - - if (global_context.expired()) - global_context = query_context_->getGlobalContext(); - - if (thread_group) - { - std::lock_guard lock(thread_group->mutex); - - thread_group->query_context = query_context; - if (thread_group->global_context.expired()) - thread_group->global_context = global_context; - } - - applyQuerySettings(); -} - -void CurrentThread::defaultThreadDeleter() -{ - if (unlikely(!current_thread)) - return; - current_thread->detachQuery(true, true); -} - -void ThreadStatus::setupState(const ThreadGroupStatusPtr & thread_group_) +void ThreadStatus::attachGroupImp(const ThreadGroupStatusPtr & thread_group_) { assertState(ThreadState::DetachedFromQuery, __PRETTY_FUNCTION__); @@ -105,20 +104,19 @@ void ThreadStatus::setupState(const ThreadGroupStatusPtr & thread_group_) performance_counters.setParent(&thread_group->performance_counters); memory_tracker.setParent(&thread_group->memory_tracker); + thread_group->link(this); + + query_context = thread_group->getQueryContextWeak(); + if (global_context.expired()) + global_context = thread_group->getGlobalContextWeak(); + + fatal_error_callback = thread_group->getFatalErrorCallback(); + { std::lock_guard lock(thread_group->mutex); - /// NOTE: thread may be attached multiple times if it is reused from a thread pool. - thread_group->thread_ids.insert(thread_id); - thread_group->threads.insert(this); - logs_queue_ptr = thread_group->logs_queue_ptr; - fatal_error_callback = thread_group->fatal_error_callback; - query_context = thread_group->query_context; profile_queue_ptr = thread_group->profile_queue_ptr; - - if (global_context.expired()) - global_context = thread_group->global_context; } if (auto query_context_ptr = query_context.lock()) @@ -131,34 +129,66 @@ void ThreadStatus::setupState(const ThreadGroupStatusPtr & thread_group_) thread_state = ThreadState::AttachedToQuery; } +void ThreadStatus::detachGroup() +{ + LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global); + + if (thread_state == ThreadState::DetachedFromQuery) + return; + + finalizeQueryProfiler(); + finalizePerformanceCounters(); + + thread_group->unlink(this); + + performance_counters.setParent(&ProfileEvents::global_counters); + + flushUntrackedMemory(); + + memory_tracker.reset(); + memory_tracker.setParent(thread_group->memory_tracker.getParent()); + + query_id_from_query_context.clear(); + query_context.reset(); + + fatal_error_callback = {}; + + thread_group.reset(); + + thread_state = ThreadState::DetachedFromQuery; + +#if defined(OS_LINUX) + if (os_thread_priority) + { + LOG_TRACE(log, "Resetting nice"); + + if (0 != setpriority(PRIO_PROCESS, static_cast(thread_id), 0)) + LOG_ERROR(log, "Cannot 'setpriority' back to zero: {}", errnoToString()); + + os_thread_priority = 0; + } +#endif +} + void ThreadStatus::setInternalThread() { chassert(!query_profiler_real && !query_profiler_cpu); internal_thread = true; } -void ThreadStatus::initializeQuery() +void ThreadStatus::attachTo(const ThreadGroupStatusPtr & thread_group_, bool check_detached) { - setupState(std::make_shared()); - - /// No need to lock on mutex here - thread_group->memory_tracker.setDescription("(for query)"); - thread_group->master_thread_id = thread_id; -} - -void ThreadStatus::attachQuery(const ThreadGroupStatusPtr & thread_group_, bool check_detached) -{ - if (thread_state == ThreadState::AttachedToQuery) - { - if (check_detached) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't attach query to the thread, it is already attached"); - return; - } + if (check_detached) + assertState(ThreadState::DetachedFromQuery, "Can't attach query to the thread, it is already attached"); if (!thread_group_) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to attach to nullptr thread group"); - setupState(thread_group_); + if (thread_state == ThreadState::AttachedToQuery) + return; + + deleter = [this] () { detachGroup(); }; + attachGroupImp(thread_group_); } ProfileEvents::Counters * ThreadStatus::attachProfileCountersScope(ProfileEvents::Counters * performance_counters_scope) @@ -178,6 +208,20 @@ ProfileEvents::Counters * ThreadStatus::attachProfileCountersScope(ProfileEvents return prev_counters; } +void ThreadStatus::TimePoint::SetUp(std::chrono::time_point now) +{ + // query_start_time_{microseconds, nanoseconds} are all constructed from the same time point + // to ensure that they are all equal up to the precision of a second. + nanoseconds = timeInNanoseconds(now); + microseconds = timeInMicroseconds(now); + seconds = timeInSeconds(now); +} + +void ThreadStatus::TimePoint::setUp() +{ + SetUp(std::chrono::system_clock::now()); +} + void ThreadStatus::initPerformanceCounters() { performance_counters_finalized = false; @@ -188,16 +232,9 @@ void ThreadStatus::initPerformanceCounters() memory_tracker.resetCounters(); memory_tracker.setDescription("(for thread)"); - // query_start_time_{microseconds, nanoseconds} are all constructed from the same time point - // to ensure that they are all equal up to the precision of a second. - const auto now = std::chrono::system_clock::now(); + query_start_time.setUp(); - query_start_time_nanoseconds = timeInNanoseconds(now); - query_start_time = timeInSeconds(now); - query_start_time_microseconds = timeInMicroseconds(now); - ++queries_started; - - // query_start_time_nanoseconds cannot be used here since RUsageCounters expect CLOCK_MONOTONIC + // query_start_time.nanoseconds cannot be used here since RUsageCounters expect CLOCK_MONOTONIC *last_rusage = RUsageCounters::current(); if (!internal_thread) @@ -272,7 +309,7 @@ void ThreadStatus::finalizePerformanceCounters() if (settings.log_queries && settings.log_query_threads) { const auto now = std::chrono::system_clock::now(); - Int64 query_duration_ms = (timeInMicroseconds(now) - query_start_time_microseconds) / 1000; + Int64 query_duration_ms = (timeInMicroseconds(now) - query_start_time.microseconds) / 1000; if (query_duration_ms >= settings.log_queries_min_query_duration_ms.totalMilliseconds()) { if (auto thread_log = global_context_ptr->getQueryThreadLog()) @@ -331,77 +368,20 @@ void ThreadStatus::finalizeQueryProfiler() query_profiler_cpu.reset(); } -void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) -{ - LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global); - - if (exit_if_already_detached && thread_state == ThreadState::DetachedFromQuery) - { - thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery; - return; - } - - assertState(ThreadState::AttachedToQuery, __PRETTY_FUNCTION__); - - finalizeQueryProfiler(); - finalizePerformanceCounters(); - - /// Detach from thread group - { - std::lock_guard guard(thread_group->mutex); - thread_group->threads.erase(this); - } - performance_counters.setParent(&ProfileEvents::global_counters); - - flushUntrackedMemory(); - - memory_tracker.reset(); - memory_tracker.setParent(thread_group->memory_tracker.getParent()); - - query_id.clear(); - query_context.reset(); - - /// The memory of thread_group->finished_threads_counters_memory is temporarily moved to this vector, which is deallocated out of critical section. - std::vector move_to_temp; - - /// Avoid leaking of ThreadGroupStatus::finished_threads_counters_memory - /// (this is in case someone uses system thread but did not call getProfileEventsCountersAndMemoryForThreads()) - { - std::lock_guard guard(thread_group->mutex); - move_to_temp = std::move(thread_group->finished_threads_counters_memory); - } - - thread_group.reset(); - - thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery; - -#if defined(OS_LINUX) - if (os_thread_priority) - { - LOG_TRACE(log, "Resetting nice"); - - if (0 != setpriority(PRIO_PROCESS, static_cast(thread_id), 0)) - LOG_ERROR(log, "Cannot 'setpriority' back to zero: {}", errnoToString()); - - os_thread_priority = 0; - } -#endif -} - void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database, std::chrono::time_point now) { QueryThreadLogElement elem; // construct current_time and current_time_microseconds using the same time point // so that the two times will always be equal up to a precision of a second. - auto current_time = timeInSeconds(now); - auto current_time_microseconds = timeInMicroseconds(now); + TimePoint current_time; + current_time.SetUp(now); - elem.event_time = current_time; - elem.event_time_microseconds = current_time_microseconds; - elem.query_start_time = query_start_time; - elem.query_start_time_microseconds = query_start_time_microseconds; - elem.query_duration_ms = (timeInNanoseconds(now) - query_start_time_nanoseconds) / 1000000U; + elem.event_time = current_time.seconds; + elem.event_time_microseconds = current_time.microseconds; + elem.query_start_time = query_start_time.seconds; + elem.query_start_time_microseconds = query_start_time.microseconds; + elem.query_duration_ms = (current_time.nanoseconds - query_start_time.nanoseconds) / 1000000U; elem.read_rows = progress_in.read_rows.load(std::memory_order_relaxed); elem.read_bytes = progress_in.read_bytes.load(std::memory_order_relaxed); @@ -457,6 +437,7 @@ void ThreadStatus::logToQueryViewsLog(const ViewRuntimeData & vinfo) auto query_context_ptr = query_context.lock(); if (!query_context_ptr) return; + auto views_log = query_context_ptr->getQueryViewsLog(); if (!views_log) return; @@ -467,7 +448,7 @@ void ThreadStatus::logToQueryViewsLog(const ViewRuntimeData & vinfo) element.event_time_microseconds = timeInMicroseconds(vinfo.runtime_stats->event_time); element.view_duration_ms = vinfo.runtime_stats->elapsed_ms; - element.initial_query_id = query_id; + element.initial_query_id = query_id_from_query_context; element.view_name = vinfo.table_id.getFullTableName(); element.view_uuid = vinfo.table_id.uuid; element.view_type = vinfo.runtime_stats->type; @@ -475,16 +456,14 @@ void ThreadStatus::logToQueryViewsLog(const ViewRuntimeData & vinfo) element.view_query = getCleanQueryAst(vinfo.query, query_context_ptr); element.view_target = vinfo.runtime_stats->target_name; - auto events = std::make_shared(performance_counters.getPartiallyAtomicSnapshot()); element.read_rows = progress_in.read_rows.load(std::memory_order_relaxed); element.read_bytes = progress_in.read_bytes.load(std::memory_order_relaxed); element.written_rows = progress_out.written_rows.load(std::memory_order_relaxed); element.written_bytes = progress_out.written_bytes.load(std::memory_order_relaxed); element.peak_memory_usage = memory_tracker.getPeak() > 0 ? memory_tracker.getPeak() : 0; if (query_context_ptr->getSettingsRef().log_profile_events != 0) - { - element.profile_counters = events; - } + element.profile_counters = std::make_shared( + performance_counters.getPartiallyAtomicSnapshot()); element.status = vinfo.runtime_stats->event_status; element.exception_code = 0; @@ -499,35 +478,18 @@ void ThreadStatus::logToQueryViewsLog(const ViewRuntimeData & vinfo) views_log->add(element); } -void CurrentThread::initializeQuery() -{ - if (unlikely(!current_thread)) - return; - current_thread->initializeQuery(); - current_thread->deleter = CurrentThread::defaultThreadDeleter; -} - void CurrentThread::attachTo(const ThreadGroupStatusPtr & thread_group) { if (unlikely(!current_thread)) return; - current_thread->attachQuery(thread_group, true); - current_thread->deleter = CurrentThread::defaultThreadDeleter; + current_thread->attachTo(thread_group, true); } void CurrentThread::attachToIfDetached(const ThreadGroupStatusPtr & thread_group) { if (unlikely(!current_thread)) return; - current_thread->attachQuery(thread_group, false); - current_thread->deleter = CurrentThread::defaultThreadDeleter; -} - -void CurrentThread::attachQueryContext(ContextPtr query_context) -{ - if (unlikely(!current_thread)) - return; - current_thread->attachQueryContext(query_context); + current_thread->attachTo(thread_group, false); } void CurrentThread::finalizePerformanceCounters() @@ -537,28 +499,25 @@ void CurrentThread::finalizePerformanceCounters() current_thread->finalizePerformanceCounters(); } -void CurrentThread::detachQuery() +void CurrentThread::detachGroupIfNotDetached() { if (unlikely(!current_thread)) return; - current_thread->detachQuery(false); + current_thread->detachGroup(); } void CurrentThread::detachQueryIfNotDetached() { - if (unlikely(!current_thread)) - return; - current_thread->detachQuery(true); + return detachGroupIfNotDetached(); } - CurrentThread::QueryScope::QueryScope(ContextMutablePtr query_context, std::function fatal_error_callback) { - CurrentThread::initializeQuery(); - CurrentThread::attachQueryContext(query_context); if (!query_context->hasQueryContext()) query_context->makeQueryContext(); - setFatalErrorCallback(fatal_error_callback); + + auto group = ThreadGroupStatus::createForQuery(query_context, std::move(fatal_error_callback)); + CurrentThread::attachTo(std::move(group)); } CurrentThread::QueryScope::QueryScope(ContextPtr query_context, std::function fatal_error_callback) @@ -567,9 +526,8 @@ CurrentThread::QueryScope::QueryScope(ContextPtr query_context, std::functionmakeQueryContext(); query_context->setCurrentQueryId(query_id); + chassert(&DB::CurrentThread::get() == &thread_status); DB::CurrentThread::QueryScope query_scope_holder(query_context); DB::FileCacheSettings settings; @@ -398,8 +399,8 @@ TEST_F(FileCacheTest, get) auto query_context_1 = DB::Context::createCopy(getContext().context); query_context_1->makeQueryContext(); query_context_1->setCurrentQueryId("query_id_1"); + chassert(&DB::CurrentThread::get() == &thread_status_1); DB::CurrentThread::QueryScope query_scope_holder_1(query_context_1); - thread_status_1.attachQueryContext(query_context_1); auto holder_2 = cache.getOrSet(key, 25, 5, {}); /// Get [25, 29] once again. auto segments_2 = fromHolder(holder_2); @@ -467,8 +468,8 @@ TEST_F(FileCacheTest, get) auto query_context_1 = DB::Context::createCopy(getContext().context); query_context_1->makeQueryContext(); query_context_1->setCurrentQueryId("query_id_1"); + chassert(&DB::CurrentThread::get() == &thread_status_1); DB::CurrentThread::QueryScope query_scope_holder_1(query_context_1); - thread_status_1.attachQueryContext(query_context_1); auto holder_2 = cache.getOrSet(key, 3, 23, {}); /// Get [3, 25] once again auto segments_2 = fromHolder(*holder); diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index c27e73804ad..8cfa7160078 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -286,12 +286,7 @@ Chain buildPushingToViewsChain( std::unique_ptr view_thread_status_ptr = std::make_unique(); /// Copy of a ThreadStatus should be internal. view_thread_status_ptr->setInternalThread(); - /// view_thread_status_ptr will be moved later (on and on), so need to capture raw pointer. - view_thread_status_ptr->deleter = [thread_status = view_thread_status_ptr.get(), running_group] - { - thread_status->detachQuery(); - }; - view_thread_status_ptr->attachQuery(running_group); + view_thread_status_ptr->attachTo(running_group); auto * view_thread_status = view_thread_status_ptr.get(); views_data->thread_status_holder->thread_statuses.push_front(std::move(view_thread_status_ptr)); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index cc050730853..830d3b2e0f5 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -501,6 +501,7 @@ void TCPHandler::runImpl() /// (i.e. deallocations from the Aggregator with two-level aggregation) state.reset(); query_scope.reset(); + last_sent_snapshots.clear(); thread_trace_context.reset(); } catch (const Exception & e) diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference new file mode 100644 index 00000000000..7d7688db881 --- /dev/null +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference @@ -0,0 +1,23 @@ +INSERT TO S3 + [ 0 ] S3CompleteMultipartUpload: 1 + [ 0 ] S3CreateMultipartUpload: 1 + [ 0 ] S3HeadObject: 1 + [ 0 ] S3ReadRequestsCount: 1 + [ 0 ] S3UploadPart: 1 + [ 0 ] S3WriteRequestsCount: 3 +CHECK WITH query_log +QueryFinish S3CreateMultipartUpload 1 S3UploadPart 1 S3CompleteMultipartUpload 1 S3PutObject 0 +CREATE +INSERT + [ 0 ] FileOpen: 7 +READ +INSERT and READ INSERT + [ 0 ] FileOpen: 7 + [ 0 ] FileOpen: 7 +DROP +CHECK with query_log +QueryFinish INSERT INTO times SELECT now() + INTERVAL 1 day; FileOpen 7 +QueryFinish SELECT \'1\', min(t) FROM times; FileOpen 0 +QueryFinish INSERT INTO times SELECT now() + INTERVAL 2 day; FileOpen 7 +QueryFinish SELECT \'2\', min(t) FROM times; FileOpen 0 +QueryFinish INSERT INTO times SELECT now() + INTERVAL 3 day; FileOpen 7 diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh new file mode 100755 index 00000000000..ce6a7e114af --- /dev/null +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh @@ -0,0 +1,69 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# Tag no-fasttest: needs s3 + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +echo "INSERT TO S3" +$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " +INSERT INTO TABLE FUNCTION s3('http://localhost:11111/test/profile_events.csv', 'test', 'testtest', 'CSV', 'number UInt64') SELECT number FROM numbers(1000000) SETTINGS s3_max_single_part_upload_size = 10, s3_truncate_on_insert = 1; +" 2>&1 | grep -o -e '\ \[\ .*\ \]\ S3.*:\ .*\ ' | grep -v 'Microseconds' | sort + +echo "CHECK WITH query_log" +$CLICKHOUSE_CLIENT -nq " +SYSTEM FLUSH LOGS; +SELECT type, + 'S3CreateMultipartUpload', ProfileEvents['S3CreateMultipartUpload'], + 'S3UploadPart', ProfileEvents['S3UploadPart'], + 'S3CompleteMultipartUpload', ProfileEvents['S3CompleteMultipartUpload'], + 'S3PutObject', ProfileEvents['S3PutObject'] +FROM system.query_log +WHERE query LIKE '%profile_events.csv%' +AND type = 'QueryFinish' +AND current_database = currentDatabase() +ORDER BY query_start_time DESC; +" + +echo "CREATE" +$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " +DROP TABLE IF EXISTS times; +CREATE TABLE times (t DateTime) ENGINE MergeTree ORDER BY t; +" 2>&1 | grep -o -e '\ \[\ .*\ \]\ FileOpen:\ .*\ ' + +echo "INSERT" +$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " +INSERT INTO times SELECT now() + INTERVAL 1 day; +" 2>&1 | grep -o -e '\ \[\ .*\ \]\ FileOpen:\ .*\ ' + +echo "READ" +$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " +SELECT '1', min(t) FROM times; +" 2>&1 | grep -o -e '\ \[\ .*\ \]\ FileOpen:\ .*\ ' + +echo "INSERT and READ INSERT" +$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " +INSERT INTO times SELECT now() + INTERVAL 2 day; +SELECT '2', min(t) FROM times; +INSERT INTO times SELECT now() + INTERVAL 3 day; +" 2>&1 | grep -o -e '\ \[\ .*\ \]\ FileOpen:\ .*\ ' + +echo "DROP" +$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " +DROP TABLE times; +" + +echo "CHECK with query_log" +$CLICKHOUSE_CLIENT -nq " +SYSTEM FLUSH LOGS; +SELECT type, + query, + 'FileOpen', ProfileEvents['FileOpen'] +FROM system.query_log +WHERE current_database = currentDatabase() +AND ( query LIKE '%SELECT % FROM times%' OR query LIKE '%INSERT INTO times%') +AND type = 'QueryFinish' +ORDER BY query_start_time_microseconds ASC, query DESC; +" + From c5b30336b9011c42170c3056855318554db5a0c7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 15 Mar 2023 23:58:55 +0100 Subject: [PATCH 188/559] Update test --- .../queries/0_stateless/02685_bson2.reference | 0 tests/queries/0_stateless/02685_bson2.sql | Bin 21295 -> 21329 bytes 2 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/queries/0_stateless/02685_bson2.reference diff --git a/tests/queries/0_stateless/02685_bson2.reference b/tests/queries/0_stateless/02685_bson2.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02685_bson2.sql b/tests/queries/0_stateless/02685_bson2.sql index 55334f212b18fc66992f2e62e89115cad3af2eff..fc65d2952df89771e3496f9e4d8da9ff479c31d1 100644 GIT binary patch delta 45 zcmZ3#jPc?!#tnMG$_l!=3e^h5sYPX}MXp6f`9%tze$M_uL9Wgr@h*-bjtaG00B~Cl A;s5{u delta 11 Scmcb(jB))k#tnMGj9dU7zyu2b From e0954ce7beb9b38c16ebe222fc6abc354d1cd919 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 16 Mar 2023 00:22:05 +0100 Subject: [PATCH 189/559] fix compile --- src/Dictionaries/RegExpTreeDictionary.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/RegExpTreeDictionary.cpp b/src/Dictionaries/RegExpTreeDictionary.cpp index b0f3dcb0249..5618da3505b 100644 --- a/src/Dictionaries/RegExpTreeDictionary.cpp +++ b/src/Dictionaries/RegExpTreeDictionary.cpp @@ -219,7 +219,7 @@ void RegExpTreeDictionary::initRegexNodes(Block & block) } regex_nodes.emplace(id, node); -#ifdef USE_VECTORSCAN +#if USE_VECTORSCAN String required_substring; bool is_trivial, required_substring_is_prefix; std::vector alternatives; @@ -308,7 +308,7 @@ void RegExpTreeDictionary::loadData() if (!use_vectorscan) return; -#ifdef USE_VECTORSCAN +#if USE_VECTORSCAN std::vector patterns; std::vector flags; std::vector lens; From cf75dd25951d6a598fe8a087db656d321b2e2737 Mon Sep 17 00:00:00 2001 From: alexX512 Date: Thu, 16 Mar 2023 04:39:11 +0000 Subject: [PATCH 190/559] Remove sleep_in_receive_cancel --- src/Server/TCPHandler.cpp | 9 --------- 1 file changed, 9 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index f6a6441b922..fa1de538280 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1828,15 +1828,6 @@ void TCPHandler::decreaseCancellationStatus(const std::string& log_message) auto current_status = QueryState::cancellationStatusToName(state.cancellation_status); LOG_INFO(log, "Change cancellation status from {} to {}. Log message: {}", prev_status, current_status, log_message); - - /// For testing connection collector. - { - if (unlikely(sleep_in_receive_cancel.totalMilliseconds())) - { - std::chrono::milliseconds ms(sleep_in_receive_cancel.totalMilliseconds()); - std::this_thread::sleep_for(ms); - } - } } QueryState::CancellationStatus TCPHandler::getQueryCancellationStatus() From 29dd2e56f9f2d09bdbc57d183ddb0e512e48dd72 Mon Sep 17 00:00:00 2001 From: alexX512 Date: Thu, 16 Mar 2023 12:41:17 +0000 Subject: [PATCH 191/559] Fix pulling pipeline --- docs/en/operations/settings/settings.md | 2 +- .../Executors/PullingAsyncPipelineExecutor.cpp | 9 ++++++--- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 881faf3faec..298174ce1da 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4051,7 +4051,7 @@ Possible values: Default value: `0`. ## stop_reading_on_first_cancel {#stop_reading_on_first_cancel} -When set to true and the user wants to interrupt a query (for example using Ctrl+C on the client), then the query continues execution only on data that was already read from the table. Afterward, it will return a partial result of the query for the part of the table that was read. To fully stop the execution of a query without a partial result, the user should send 2 cancel requests. +When set to `true` and the user wants to interrupt a query (for example using `Ctrl+C` on the client), then the query continues execution only on data that was already read from the table. Afterward, it will return a partial result of the query for the part of the table that was read. To fully stop the execution of a query without a partial result, the user should send 2 cancel requests. **Example without setting on Ctrl+C** ```sql diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index af68802a640..21c6a30bb5a 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -175,6 +175,9 @@ bool PullingAsyncPipelineExecutor::pull(Block & block, uint64_t milliseconds) void PullingAsyncPipelineExecutor::cancel() { + if (!data) + return; + /// Cancel execution if it wasn't finished. cancelWithExceptionHandling([&]() { @@ -194,6 +197,9 @@ void PullingAsyncPipelineExecutor::cancel() void PullingAsyncPipelineExecutor::cancelReading() { + if (!data) + return; + /// Stop reading from source if pipeline wasn't finished. cancelWithExceptionHandling([&]() { @@ -203,9 +209,6 @@ void PullingAsyncPipelineExecutor::cancelReading() void PullingAsyncPipelineExecutor::cancelWithExceptionHandling(CancelFunc && cancel_func) { - if (!data) - return; - try { if (!data->is_finished && data->executor) From 716a27ccea5436fc9cbbbc3ce23dab43bee2b98e Mon Sep 17 00:00:00 2001 From: alexX512 Date: Thu, 16 Mar 2023 13:41:58 +0000 Subject: [PATCH 192/559] Fix test stop_reading_on_first_cancel --- .../0_stateless/25341_stop_reading_on_first_cancel.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/25341_stop_reading_on_first_cancel.sh b/tests/queries/0_stateless/25341_stop_reading_on_first_cancel.sh index 52111cdd8f3..9694907d679 100755 --- a/tests/queries/0_stateless/25341_stop_reading_on_first_cancel.sh +++ b/tests/queries/0_stateless/25341_stop_reading_on_first_cancel.sh @@ -4,8 +4,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --query="SELECT sum(number * 0) FROM numbers(1000000000) SETTINGS stop_reading_on_first_cancel=true;" & +$CLICKHOUSE_CLIENT -n --query="SELECT sum(number * 0) FROM numbers(10000000000) SETTINGS stop_reading_on_first_cancel=true;" & pid=$! -sleep 1 -kill -INT $pid +sleep 2 +kill -SIGINT $pid wait $pid From 6ea522adee6ef6c52a1364e4d651d6bf74597f38 Mon Sep 17 00:00:00 2001 From: alexX512 Date: Thu, 16 Mar 2023 14:23:17 +0000 Subject: [PATCH 193/559] Style fix --- src/Server/TCPHandler.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index fa1de538280..caf1a3dc075 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1409,6 +1409,7 @@ String TCPHandler::receiveReadTaskResponseAssumeLocked() return response; } + std::optional TCPHandler::receivePartitionMergeTreeReadTaskResponseAssumeLocked() { UInt64 packet_type = 0; @@ -1721,7 +1722,6 @@ bool TCPHandler::receiveUnexpectedData(bool throw_exception) return read_ok; } - void TCPHandler::initBlockInput() { if (!state.block_in) @@ -1747,6 +1747,7 @@ void TCPHandler::initBlockInput() } } + void TCPHandler::initBlockOutput(const Block & block) { if (!state.block_out) From 2cc47b5bb6801dc5f534d857e78233e84f9549fc Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 16 Mar 2023 14:43:37 +0000 Subject: [PATCH 194/559] Allow reading/writing nested arrays in Protobuf with only root field name as column name --- src/Formats/ProtobufSerializer.cpp | 34 +++++++++++++++++++ .../02591_protobuf_nested_arrays.reference | 1 + .../02591_protobuf_nested_arrays.sh | 10 ++++++ 3 files changed, 45 insertions(+) create mode 100644 tests/queries/0_stateless/02591_protobuf_nested_arrays.reference create mode 100755 tests/queries/0_stateless/02591_protobuf_nested_arrays.sh diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index 36cd2534fe8..4f3b19ac803 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -3401,7 +3401,41 @@ namespace const auto & array_data_type = assert_cast(*data_type); if (!allow_repeat) + { + /// Case of nested Arrays. Nested Array can be a message with one repeated field. + /// For example we have an column `arr Array(Array(UInt32))` and the next proto schema: + /// message Message { + /// message NestedArray { + /// repeated uint32 nested = 2; + /// } + /// repeated NestedArray arr = 1; + /// } + if (field_descriptor.message_type() && field_descriptor.message_type()->field_count() == 1) + { + Names column_names = {field_descriptor.message_type()->field(0)->name()}; + DataTypes data_types = {data_type}; + /// Try to serialize as a nested message. + std::vector used_column_indices; + auto message_serializer = buildMessageSerializerImpl( + 1, + column_names.data(), + data_types.data(), + *field_descriptor.message_type(), + /* with_length_delimiter = */ false, + google_wrappers_special_treatment, + &field_descriptor, + used_column_indices, + /* columns_are_reordered_outside = */ false, + /* check_nested_while_filling_missing_columns = */ false); + + if (!message_serializer) + return nullptr; + + return message_serializer; + } + throwFieldNotRepeated(field_descriptor, column_name); + } auto nested_serializer = buildFieldSerializer(column_name, array_data_type.getNestedType(), field_descriptor, /* allow_repeat = */ false, // We do our repeating now, so for nested type we forget about the repeating. diff --git a/tests/queries/0_stateless/02591_protobuf_nested_arrays.reference b/tests/queries/0_stateless/02591_protobuf_nested_arrays.reference new file mode 100644 index 00000000000..ff21f7fc2c9 --- /dev/null +++ b/tests/queries/0_stateless/02591_protobuf_nested_arrays.reference @@ -0,0 +1 @@ +[[[42,42],[],[42]],[[],[42],[42,42,42,42]]] diff --git a/tests/queries/0_stateless/02591_protobuf_nested_arrays.sh b/tests/queries/0_stateless/02591_protobuf_nested_arrays.sh new file mode 100755 index 00000000000..b6714932f3a --- /dev/null +++ b/tests/queries/0_stateless/02591_protobuf_nested_arrays.sh @@ -0,0 +1,10 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +SCHEMADIR=$CURDIR/format_schemas +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "select [[[42, 42], [], [42]], [[], [42], [42, 42, 42, 42]]] as a format Protobuf settings format_schema = '$SCHEMADIR/00825_protobuf_format_array_3dim:ABC'" | $CLICKHOUSE_LOCAL --input-format Protobuf --format_schema="$SCHEMADIR/00825_protobuf_format_array_3dim:ABC" --structure="a Array(Array(Array(Int32)))" -q "select * from table" + From 8e36f883d3c6ea1b28f137527a6fc8bf5f587eb6 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 16 Mar 2023 16:04:42 +0100 Subject: [PATCH 195/559] add 'strict' mode for sync replica, fix 00993 --- src/Interpreters/InterpreterSystemQuery.cpp | 7 ++++--- src/Interpreters/InterpreterSystemQuery.h | 2 +- src/Parsers/ASTSystemQuery.cpp | 3 +++ src/Parsers/ASTSystemQuery.h | 2 ++ src/Parsers/ParserSystemQuery.cpp | 2 ++ src/Storages/StorageReplicatedMergeTree.cpp | 13 +++++++++++-- src/Storages/StorageReplicatedMergeTree.h | 2 +- tests/queries/0_stateless/replication.lib | 6 ++++-- 8 files changed, 28 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 66ab8e43f7e..fb6b1635f28 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -509,7 +509,7 @@ BlockIO InterpreterSystemQuery::execute() dropDatabaseReplica(query); break; case Type::SYNC_REPLICA: - syncReplica(); + syncReplica(query); break; case Type::SYNC_DATABASE_REPLICA: syncReplicatedDatabase(query); @@ -879,7 +879,7 @@ void InterpreterSystemQuery::dropDatabaseReplica(ASTSystemQuery & query) throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid query"); } -void InterpreterSystemQuery::syncReplica() +void InterpreterSystemQuery::syncReplica(ASTSystemQuery & query) { getContext()->checkAccess(AccessType::SYSTEM_SYNC_REPLICA, table_id); StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext()); @@ -887,7 +887,8 @@ void InterpreterSystemQuery::syncReplica() if (auto * storage_replicated = dynamic_cast(table.get())) { LOG_TRACE(log, "Synchronizing entries in replica's queue with table's log and waiting for current last entry to be processed"); - if (!storage_replicated->waitForProcessingQueue(getContext()->getSettingsRef().receive_timeout.totalMilliseconds())) + auto sync_timeout = getContext()->getSettingsRef().receive_timeout.totalMilliseconds(); + if (!storage_replicated->waitForProcessingQueue(sync_timeout, query.strict_sync)) { LOG_ERROR(log, "SYNC REPLICA {}: Timed out!", table_id.getNameForLogs()); throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "SYNC REPLICA {}: command timed out. " \ diff --git a/src/Interpreters/InterpreterSystemQuery.h b/src/Interpreters/InterpreterSystemQuery.h index df06a2fa4ef..8a1cdaf8edd 100644 --- a/src/Interpreters/InterpreterSystemQuery.h +++ b/src/Interpreters/InterpreterSystemQuery.h @@ -56,7 +56,7 @@ private: void restartReplica(const StorageID & replica, ContextMutablePtr system_context); void restartReplicas(ContextMutablePtr system_context); - void syncReplica(); + void syncReplica(ASTSystemQuery & query); void waitLoadingParts(); void syncReplicatedDatabase(ASTSystemQuery & query); diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index ffb018f23fe..c38604690ca 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -181,6 +181,9 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, print_identifier(target_function); else if (!disk.empty()) print_identifier(disk); + + if (strict_sync) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " STRICT" << (settings.hilite ? hilite_none : ""); } else if (type == Type::SYNC_DATABASE_REPLICA) { diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 80a891712a6..2c7d42edebe 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -108,6 +108,8 @@ public: String schema_cache_storage; + bool strict_sync = false; + String getID(char) const override { return "SYSTEM query"; } ASTPtr clone() const override diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index 0eb263869b1..7c8d42b6bce 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -259,6 +259,8 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & return false; if (!parseDatabaseAndTableAsAST(pos, expected, res->database, res->table)) return false; + if (res->type == Type::SYNC_REPLICA && ParserKeyword{"STRICT"}.ignore(pos, expected)) + res->strict_sync = true; break; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index fe4a144deaa..3ac1c52a99f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7581,7 +7581,7 @@ void StorageReplicatedMergeTree::onActionLockRemove(StorageActionBlockType actio background_moves_assignee.trigger(); } -bool StorageReplicatedMergeTree::waitForProcessingQueue(UInt64 max_wait_milliseconds) +bool StorageReplicatedMergeTree::waitForProcessingQueue(UInt64 max_wait_milliseconds, bool strict) { Stopwatch watch; @@ -7595,8 +7595,17 @@ bool StorageReplicatedMergeTree::waitForProcessingQueue(UInt64 max_wait_millisec bool set_ids_to_wait = true; Poco::Event target_entry_event; - auto callback = [&target_entry_event, &wait_for_ids, &set_ids_to_wait](size_t new_queue_size, std::unordered_set log_entry_ids, std::optional removed_log_entry_id) + auto callback = [&target_entry_event, &wait_for_ids, &set_ids_to_wait, strict] + (size_t new_queue_size, std::unordered_set log_entry_ids, std::optional removed_log_entry_id) { + if (strict) + { + /// In strict mode we wait for queue to become empty + if (new_queue_size == 0) + target_entry_event.set(); + return; + } + if (set_ids_to_wait) { wait_for_ids = log_entry_ids; diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 46c78e9064a..c8552b87bad 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -181,7 +181,7 @@ public: /// Wait till replication queue's current last entry is processed or till size becomes 0 /// If timeout is exceeded returns false - bool waitForProcessingQueue(UInt64 max_wait_milliseconds = 0); + bool waitForProcessingQueue(UInt64 max_wait_milliseconds, bool strict); /// Get the status of the table. If with_zk_fields = false - do not fill in the fields that require queries to ZK. void getStatus(ReplicatedTableStatus & res, bool with_zk_fields = true); diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index 7c7db48e078..6784fee6395 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -29,8 +29,10 @@ function try_sync_replicas() i=0 for t in "${tables_arr[@]}" do - # The size of log may be big, so increase timeout. - $CLICKHOUSE_CLIENT --receive_timeout $time_left -q "SYSTEM SYNC REPLICA $t" || ($CLICKHOUSE_CLIENT -q \ + # Do not start new merges (it can make SYNC a bit faster) + $CLICKHOUSE_CLIENT -q "ALTER TABLE $t MODIFY SETTING max_replicated_merges_in_queue=0" + + $CLICKHOUSE_CLIENT --receive_timeout $time_left -q "SYSTEM SYNC REPLICA $t STRICT" || ($CLICKHOUSE_CLIENT -q \ "select 'sync failed, queue:', * from system.replication_queue where database=currentDatabase() and table='$t' order by database, table, node_name" && exit 1) & pids[${i}]=$! i=$((i + 1)) From 8b8215f91a00be590dfc9a07e1172b7b70428d8e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 16 Mar 2023 16:40:08 +0100 Subject: [PATCH 196/559] fix some other tests, add logs for further debugging --- src/Storages/MergeTree/MergeTreeData.cpp | 9 +++++++++ .../test_replicated_merge_tree_s3_zero_copy/test.py | 2 +- .../0_stateless/01164_detach_attach_partition_race.sh | 2 +- 3 files changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ee5cf0ea450..f9424f334db 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1293,6 +1293,7 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPart( { std::lock_guard lock(part_loading_mutex); + LOG_TEST(log, "loadDataPart: inserting {} into data_parts_indexes", res.part->getNameWithState()); std::tie(it, inserted) = data_parts_indexes.insert(res.part); } @@ -1483,6 +1484,7 @@ void MergeTreeData::loadDataPartsFromWAL(MutableDataPartsVector & parts_from_wal continue; part->setState(DataPartState::Active); + LOG_TEST(log, "loadDataPartsFromWAL: inserting {} into data_parts_indexes", part->getNameWithState()); auto [it, inserted] = data_parts_indexes.insert(part); if (!inserted) @@ -1618,6 +1620,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) } auto part_lock = lockParts(); + LOG_TEST(log, "loadDataParts: clearing data_parts_indexes (had {} parts)", data_parts_indexes.size()); data_parts_indexes.clear(); MutableDataPartsVector broken_parts_to_detach; @@ -2156,6 +2159,7 @@ void MergeTreeData::removePartsFinally(const MergeTreeData::DataPartsVector & pa (*it)->assertState({DataPartState::Deleting}); + LOG_TEST(log, "removePartsFinally: removing {} from data_parts_indexes", (*it)->getNameWithState()); data_parts_indexes.erase(it); } } @@ -3492,6 +3496,7 @@ void MergeTreeData::preparePartForCommit(MutableDataPartPtr & part, Transaction if (need_rename) part->renameTo(part->name, true); + LOG_TEST(log, "preparePartForCommit: inserting {} into data_parts_indexes", part->getNameWithState()); data_parts_indexes.insert(part); out_transaction.addPart(part); } @@ -3672,6 +3677,7 @@ void MergeTreeData::removePartsFromWorkingSetImmediatelyAndSetTemporaryState(con modifyPartState(part, MergeTreeDataPartState::Temporary); /// Erase immediately + LOG_TEST(log, "removePartsFromWorkingSetImmediatelyAndSetTemporaryState: removing {} from data_parts_indexes", part->getNameWithState()); data_parts_indexes.erase(it_part); } } @@ -3862,6 +3868,7 @@ void MergeTreeData::forcefullyMovePartToDetachedAndRemoveFromMemory(const MergeT modifyPartState(it_part, DataPartState::Deleting); asMutableDeletingPart(part)->renameToDetached(prefix); + LOG_TEST(log, "forcefullyMovePartToDetachedAndRemoveFromMemory: removing {} from data_parts_indexes", part->getNameWithState()); data_parts_indexes.erase(it_part); if (restore_covered && part->info.level == 0) @@ -4274,8 +4281,10 @@ void MergeTreeData::swapActivePart(MergeTreeData::DataPartPtr part_copy) } modifyPartState(original_active_part, DataPartState::DeleteOnDestroy); + LOG_TEST(log, "swapActivePart: removing {} from data_parts_indexes", (*active_part_it)->getNameWithState()); data_parts_indexes.erase(active_part_it); + LOG_TEST(log, "swapActivePart: inserting {} into data_parts_indexes", part_copy->getNameWithState()); auto part_it = data_parts_indexes.insert(part_copy).first; modifyPartState(part_it, DataPartState::Active); diff --git a/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py b/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py index 5bbd5293453..1941ec734c2 100644 --- a/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py +++ b/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py @@ -196,7 +196,7 @@ def test_drop_table(cluster): "system sync replica test_drop_table", settings={"receive_timeout": 5}, sleep_time=5, - retry_count=10, + retry_count=20, ) node2.query("drop table test_drop_table sync") assert "1000\t499500\n" == node.query( diff --git a/tests/queries/0_stateless/01164_detach_attach_partition_race.sh b/tests/queries/0_stateless/01164_detach_attach_partition_race.sh index 086f409c9a3..7640b9dddf2 100755 --- a/tests/queries/0_stateless/01164_detach_attach_partition_race.sh +++ b/tests/queries/0_stateless/01164_detach_attach_partition_race.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "create table mt (n int) engine=MergeTree order by n" +$CLICKHOUSE_CLIENT -q "create table mt (n int) engine=MergeTree order by n settings parts_to_throw_insert=1000" $CLICKHOUSE_CLIENT -q "insert into mt values (1)" $CLICKHOUSE_CLIENT -q "insert into mt values (2)" $CLICKHOUSE_CLIENT -q "insert into mt values (3)" From 8495deb7e3618efd6dabd1679cf52bf7a70cbdb7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 15 Mar 2023 16:35:20 +0100 Subject: [PATCH 197/559] Remove unused removePartFromZooKeeper() Signed-off-by: Azat Khuzhin (cherry picked from commit dbdb46e9068e1fd859c25f195394b2fff74ac7ad) --- src/Storages/StorageReplicatedMergeTree.cpp | 16 ---------------- src/Storages/StorageReplicatedMergeTree.h | 3 --- 2 files changed, 19 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 3ac1c52a99f..0a281adb200 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3488,22 +3488,6 @@ void StorageReplicatedMergeTree::getRemovePartFromZooKeeperOps(const String & pa ops.emplace_back(zkutil::makeRemoveRequest(part_path, -1)); } -void StorageReplicatedMergeTree::removePartFromZooKeeper(const String & part_name) -{ - auto zookeeper = getZooKeeper(); - String part_path = fs::path(replica_path) / "parts" / part_name; - Coordination::Stat stat; - - /// Part doesn't exist, nothing to remove - if (!zookeeper->exists(part_path, &stat)) - return; - - Coordination::Requests ops; - - getRemovePartFromZooKeeperOps(part_name, ops, stat.numChildren > 0); - zookeeper->multi(ops); -} - void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_name, bool storage_init) { auto zookeeper = getZooKeeper(); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index c8552b87bad..000d6d5da73 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -543,9 +543,6 @@ private: /// Set has_children to true for "old-style" parts (those with /columns and /checksums child znodes). void getRemovePartFromZooKeeperOps(const String & part_name, Coordination::Requests & ops, bool has_children); - /// Just removes part from ZooKeeper using previous method - void removePartFromZooKeeper(const String & part_name); - /// Quickly removes big set of parts from ZooKeeper (using async multi queries) void removePartsFromZooKeeper(zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names, NameSet * parts_should_be_retried = nullptr); From eb206bbe254e8bcf474d6a1063ee4d3034b44757 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 16 Mar 2023 16:08:24 +0000 Subject: [PATCH 198/559] Fix clang-tidy build --- src/Common/tests/gtest_lru_cache.cpp | 12 ++++++------ src/Common/tests/gtest_slru_cache.cpp | 22 +++++++++++----------- 2 files changed, 17 insertions(+), 17 deletions(-) diff --git a/src/Common/tests/gtest_lru_cache.cpp b/src/Common/tests/gtest_lru_cache.cpp index 9a2cb354bd5..1185dd58e5e 100644 --- a/src/Common/tests/gtest_lru_cache.cpp +++ b/src/Common/tests/gtest_lru_cache.cpp @@ -6,7 +6,7 @@ TEST(LRUCache, set) { using SimpleCacheBase = DB::CacheBase; - auto lru_cache = SimpleCacheBase("LRU", /*max_size*/ 10, /*max_elements_size*/ 10); + auto lru_cache = SimpleCacheBase("LRU", /*max_size_in_bytes*/ 10, /*max_count*/ 10); lru_cache.set(1, std::make_shared(2)); lru_cache.set(2, std::make_shared(3)); @@ -19,7 +19,7 @@ TEST(LRUCache, set) TEST(LRUCache, update) { using SimpleCacheBase = DB::CacheBase; - auto lru_cache = SimpleCacheBase("LRU", /*max_size*/ 10, /*max_elements_size*/ 10); + auto lru_cache = SimpleCacheBase("LRU", /*max_size_in_bytes*/ 10, /*max_count*/ 10); lru_cache.set(1, std::make_shared(2)); lru_cache.set(1, std::make_shared(3)); auto val = lru_cache.get(1); @@ -30,7 +30,7 @@ TEST(LRUCache, update) TEST(LRUCache, get) { using SimpleCacheBase = DB::CacheBase; - auto lru_cache = SimpleCacheBase("LRU", /*max_size*/ 10, /*max_elements_size*/ 10); + auto lru_cache = SimpleCacheBase("LRU", /*max_size_in_bytes*/ 10, /*max_count*/ 10); lru_cache.set(1, std::make_shared(2)); lru_cache.set(2, std::make_shared(3)); SimpleCacheBase::MappedPtr value = lru_cache.get(1); @@ -50,7 +50,7 @@ struct ValueWeight TEST(LRUCache, evictOnSize) { using SimpleCacheBase = DB::CacheBase; - auto lru_cache = SimpleCacheBase("LRU", /*max_size*/ 20, /*max_elements_size*/ 3); + auto lru_cache = SimpleCacheBase("LRU", /*max_size_in_bytes*/ 20, /*max_count*/ 3); lru_cache.set(1, std::make_shared(2)); lru_cache.set(2, std::make_shared(3)); lru_cache.set(3, std::make_shared(4)); @@ -66,7 +66,7 @@ TEST(LRUCache, evictOnSize) TEST(LRUCache, evictOnWeight) { using SimpleCacheBase = DB::CacheBase, ValueWeight>; - auto lru_cache = SimpleCacheBase("LRU", /*max_size*/ 10, /*max_elements_size*/ 10); + auto lru_cache = SimpleCacheBase("LRU", /*max_size_in_bytes*/ 10, /*max_count*/ 10); lru_cache.set(1, std::make_shared(2)); lru_cache.set(2, std::make_shared(3)); lru_cache.set(3, std::make_shared(4)); @@ -87,7 +87,7 @@ TEST(LRUCache, evictOnWeight) TEST(LRUCache, getOrSet) { using SimpleCacheBase = DB::CacheBase, ValueWeight>; - auto lru_cache = SimpleCacheBase("LRU", /*max_size*/ 10, /*max_elements_size*/ 10); + auto lru_cache = SimpleCacheBase("LRU", /*max_size_in_bytes*/ 10, /*max_count*/ 10); size_t x = 10; auto load_func = [&] { return std::make_shared(x); }; auto [value, loaded] = lru_cache.getOrSet(1, load_func); diff --git a/src/Common/tests/gtest_slru_cache.cpp b/src/Common/tests/gtest_slru_cache.cpp index f7ae9f9b16e..52549592f0e 100644 --- a/src/Common/tests/gtest_slru_cache.cpp +++ b/src/Common/tests/gtest_slru_cache.cpp @@ -6,7 +6,7 @@ TEST(SLRUCache, set) { using SimpleCacheBase = DB::CacheBase; - auto slru_cache = SimpleCacheBase("SLRU", /*max_size=*/10, /*max_elements_size=*/0, /*size_ratio*/0.5); + auto slru_cache = SimpleCacheBase("SLRU", /*max_size_in_bytes=*/10, /*max_count=*/0, /*size_ratio*/0.5); slru_cache.set(1, std::make_shared(2)); slru_cache.set(2, std::make_shared(3)); @@ -19,7 +19,7 @@ TEST(SLRUCache, set) TEST(SLRUCache, update) { using SimpleCacheBase = DB::CacheBase; - auto slru_cache = SimpleCacheBase("SLRU", /*max_size=*/10, /*max_elements_size=*/0, /*size_ratio*/0.5); + auto slru_cache = SimpleCacheBase("SLRU", /*max_size_in_bytes=*/10, /*max_count=*/0, /*size_ratio*/0.5); slru_cache.set(1, std::make_shared(2)); slru_cache.set(1, std::make_shared(3)); @@ -31,7 +31,7 @@ TEST(SLRUCache, update) TEST(SLRUCache, get) { using SimpleCacheBase = DB::CacheBase; - auto slru_cache = SimpleCacheBase("SLRU", /*max_size=*/10, /*max_elements_size=*/0, /*size_ratio*/0.5); + auto slru_cache = SimpleCacheBase("SLRU", /*max_size_in_bytes=*/10, /*max_count=*/0, /*size_ratio*/0.5); slru_cache.set(1, std::make_shared(2)); slru_cache.set(2, std::make_shared(3)); @@ -47,7 +47,7 @@ TEST(SLRUCache, get) TEST(SLRUCache, remove) { using SimpleCacheBase = DB::CacheBase; - auto slru_cache = SimpleCacheBase("SLRU", /*max_size=*/10, /*max_elements_size=*/0, /*size_ratio*/0.5); + auto slru_cache = SimpleCacheBase("SLRU", /*max_size_in_bytes=*/10, /*max_count=*/0, /*size_ratio*/0.5); slru_cache.set(1, std::make_shared(2)); slru_cache.set(2, std::make_shared(3)); @@ -63,7 +63,7 @@ TEST(SLRUCache, remove) TEST(SLRUCache, removeFromProtected) { using SimpleCacheBase = DB::CacheBase; - auto slru_cache = SimpleCacheBase("SLRU", /*max_size=*/2, /*max_elements_size=*/0, /*size_ratio*/0.5); + auto slru_cache = SimpleCacheBase("SLRU", /*max_size_in_bytes=*/2, /*max_count=*/0, /*size_ratio*/0.5); slru_cache.set(1, std::make_shared(2)); slru_cache.set(1, std::make_shared(3)); @@ -96,7 +96,7 @@ TEST(SLRUCache, removeFromProtected) TEST(SLRUCache, reset) { using SimpleCacheBase = DB::CacheBase; - auto slru_cache = SimpleCacheBase("SLRU", /*max_size=*/10, /*max_elements_size=*/0, /*size_ratio*/0.5); + auto slru_cache = SimpleCacheBase("SLRU", /*max_size_in_bytes=*/10, /*max_count=*/0, /*size_ratio*/0.5); slru_cache.set(1, std::make_shared(2)); slru_cache.set(2, std::make_shared(3)); @@ -119,7 +119,7 @@ struct ValueWeight TEST(SLRUCache, evictOnElements) { using SimpleCacheBase = DB::CacheBase, ValueWeight>; - auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/1, /*size_ratio*/0.5); + auto slru_cache = SimpleCacheBase(/*max_size_in_bytes=*/10, /*max_count=*/1, /*size_ratio*/0.5); slru_cache.set(1, std::make_shared(2)); slru_cache.set(2, std::make_shared(3)); @@ -140,7 +140,7 @@ TEST(SLRUCache, evictOnElements) TEST(SLRUCache, evictOnWeight) { using SimpleCacheBase = DB::CacheBase, ValueWeight>; - auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/0, /*size_ratio*/0.5); + auto slru_cache = SimpleCacheBase(/*max_size_in_bytes=*/10, /*max_count=*/0, /*size_ratio*/0.5); slru_cache.set(1, std::make_shared(2)); slru_cache.set(2, std::make_shared(3)); slru_cache.set(3, std::make_shared(4)); @@ -161,7 +161,7 @@ TEST(SLRUCache, evictOnWeight) TEST(SLRUCache, evictFromProtectedPart) { using SimpleCacheBase = DB::CacheBase, ValueWeight>; - auto slru_cache = SimpleCacheBase("SLRU", /*max_size=*/10, /*max_elements_size=*/0, /*size_ratio*/0.5); + auto slru_cache = SimpleCacheBase("SLRU", /*max_size_in_bytes=*/10, /*max_count=*/0, /*size_ratio*/0.5); slru_cache.set(1, std::make_shared(2)); slru_cache.set(1, std::make_shared(2)); @@ -177,7 +177,7 @@ TEST(SLRUCache, evictFromProtectedPart) TEST(SLRUCache, evictStreamProtected) { using SimpleCacheBase = DB::CacheBase, ValueWeight>; - auto slru_cache = SimpleCacheBase("SLRU", /*max_size=*/10, /*max_elements_size=*/0, /*size_ratio*/0.5); + auto slru_cache = SimpleCacheBase("SLRU", /*max_size_in_bytes=*/10, /*max_count=*/0, /*size_ratio*/0.5); slru_cache.set(1, std::make_shared(2)); slru_cache.set(1, std::make_shared(2)); @@ -201,7 +201,7 @@ TEST(SLRUCache, evictStreamProtected) TEST(SLRUCache, getOrSet) { using SimpleCacheBase = DB::CacheBase, ValueWeight>; - auto slru_cache = SimpleCacheBase("SLRU", /*max_size=*/10, /*max_elements_size=*/0, /*size_ratio*/0.5); + auto slru_cache = SimpleCacheBase("SLRU", /*max_size_in_bytes=*/10, /*max_count=*/0, /*size_ratio*/0.5); size_t x = 5; auto load_func = [&] { return std::make_shared(x); }; auto [value, loaded] = slru_cache.getOrSet(1, load_func); From 7d8b643f25cb0945cff8996b276e89a585d77208 Mon Sep 17 00:00:00 2001 From: Johannes Visintini Date: Thu, 16 Mar 2023 17:17:56 +0100 Subject: [PATCH 199/559] Documentation: Update debian setup This change is necessary. `apt-key` is deprecated and will be removed, see https://manpages.debian.org/bullseye/apt/apt-key.8.en.html#DESCRIPTION --- docs/en/getting-started/install.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index 0867f3a0795..f5ca44415f9 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -118,9 +118,9 @@ It is recommended to use official pre-compiled `deb` packages for Debian or Ubun #### Setup the Debian repository ``` bash sudo apt-get install -y apt-transport-https ca-certificates dirmngr -sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754 +sudo gpg --no-default-keyring --keyring /usr/share/keyrings/clickhouse-keyring.gpg --keyserver hkp://keyserver.ubuntu.com:80 --recv-keys 8919F6BD2B48D754 -echo "deb https://packages.clickhouse.com/deb stable main" | sudo tee \ +echo "deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb stable main" | sudo tee \ /etc/apt/sources.list.d/clickhouse.list sudo apt-get update ``` From 00aaa29a85acec8882bf3433cb0f4a61ad31503d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 16 Mar 2023 18:00:39 +0000 Subject: [PATCH 200/559] better hint for compound identifiers --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 40 +++++-------------- .../02476_analyzer_identifier_hints.sh | 9 +++++ 2 files changed, 19 insertions(+), 30 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index b12fd19a961..3c12d3135d1 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1,3 +1,4 @@ +#include "Analyzer/Identifier.h" #include #include @@ -1577,41 +1578,20 @@ void QueryAnalyzer::collectCompoundExpressionValidIdentifiersForTypoCorrection( const Identifier & valid_identifier_prefix, std::unordered_set & valid_identifiers_result) { - std::vector> identifiers_with_types_to_process; - identifiers_with_types_to_process.emplace_back(valid_identifier_prefix, compound_expression_type.get()); - - while (!identifiers_with_types_to_process.empty()) + IDataType::forEachSubcolumn([&](const auto &, const auto & name, const auto &) { - auto [identifier, type] = identifiers_with_types_to_process.back(); - identifiers_with_types_to_process.pop_back(); + Identifier subcolumn_indentifier(name); + size_t new_identifier_size = valid_identifier_prefix.getPartsSize() + subcolumn_indentifier.getPartsSize(); - if (identifier.getPartsSize() + 1 > unresolved_identifier.getPartsSize()) - continue; - - while (const DataTypeArray * array = checkAndGetDataType(type)) - type = array->getNestedType().get(); - - const DataTypeTuple * tuple = checkAndGetDataType(type); - - if (!tuple) - continue; - - const auto & tuple_element_names = tuple->getElementNames(); - size_t tuple_element_names_size = tuple_element_names.size(); - - for (size_t i = 0; i < tuple_element_names_size; ++i) + if (new_identifier_size == unresolved_identifier.getPartsSize()) { - const auto & element_name = tuple_element_names[i]; - const auto & element_type = tuple->getElements()[i]; + auto new_identifier = valid_identifier_prefix; + for (auto && part : subcolumn_indentifier) + new_identifier.emplace_back(std::move(part)); - identifier.push_back(element_name); - - valid_identifiers_result.insert(identifier); - identifiers_with_types_to_process.emplace_back(identifier, element_type.get()); - - identifier.pop_back(); + valid_identifiers_result.insert(std::move(new_identifier)); } - } + }, ISerialization::SubstreamData(compound_expression_type->getDefaultSerialization())); } /// Get valid identifiers for typo correction from table expression diff --git a/tests/queries/0_stateless/02476_analyzer_identifier_hints.sh b/tests/queries/0_stateless/02476_analyzer_identifier_hints.sh index cd99d1fbd02..0702c146426 100755 --- a/tests/queries/0_stateless/02476_analyzer_identifier_hints.sh +++ b/tests/queries/0_stateless/02476_analyzer_identifier_hints.sh @@ -174,6 +174,15 @@ $CLICKHOUSE_CLIENT -q "SELECT t1.value_ FROM test_table_1 AS t1 INNER JOIN test_ $CLICKHOUSE_CLIENT -q "SELECT t2.value_ FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 ON t1.id = t2.id SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['t2.value'\]" &>/dev/null; +$CLICKHOUSE_CLIENT -q "SELECT [1] AS a, a.size1 SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['a.size0'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT ((1))::Tuple(a Tuple(b UInt32)) AS t, t.c SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['t.a'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT ((1))::Tuple(a Tuple(b UInt32)) AS t, t.a.c SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['t.a.b'\]" &>/dev/null; + $CLICKHOUSE_CLIENT -q "SELECT 1"; $CLICKHOUSE_CLIENT -n -q " From 3c6deddd1d793625b47b04777af714baf88dc577 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 15 Mar 2023 22:12:29 +0100 Subject: [PATCH 201/559] work with comments on PR --- src/AggregateFunctions/UniqExactSet.h | 4 +- src/Backups/BackupUtils.cpp | 8 +- src/Client/ClientBase.cpp | 7 +- src/Common/CurrentThread.cpp | 32 +-- src/Common/CurrentThread.h | 17 +- src/Common/ThreadStatus.cpp | 137 ++++++----- src/Common/ThreadStatus.h | 183 +++++++-------- src/Common/logger_useful.h | 2 +- src/Daemon/BaseDaemon.cpp | 8 +- src/Dictionaries/HashedDictionary.cpp | 4 +- src/Interpreters/Aggregator.cpp | 8 +- src/Interpreters/ExternalLoader.cpp | 4 +- src/Interpreters/ProcessList.cpp | 4 +- src/Interpreters/ThreadStatusExt.cpp | 218 +++++++++++------- src/Interpreters/threadPoolCallbackRunner.h | 4 +- .../Executors/CompletedPipelineExecutor.cpp | 4 +- src/Processors/Executors/PipelineExecutor.cpp | 4 +- .../PullingAsyncPipelineExecutor.cpp | 4 +- .../PushingAsyncPipelineExecutor.cpp | 4 +- .../Impl/ParallelFormattingOutputFormat.cpp | 8 +- .../Impl/ParallelParsingInputFormat.cpp | 8 +- .../Transforms/AggregatingTransform.h | 4 +- .../Transforms/buildPushingToViewsChain.cpp | 2 +- src/Storages/Distributed/DistributedSink.cpp | 4 +- src/Storages/MergeTree/MergeTreeData.cpp | 12 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +- .../System/StorageSystemStackTrace.cpp | 2 +- ...events_from_query_log_and_client.reference | 6 +- ...rofile_events_from_query_log_and_client.sh | 20 +- 29 files changed, 371 insertions(+), 355 deletions(-) diff --git a/src/AggregateFunctions/UniqExactSet.h b/src/AggregateFunctions/UniqExactSet.h index 4a3ef576e4d..916dfe4a424 100644 --- a/src/AggregateFunctions/UniqExactSet.h +++ b/src/AggregateFunctions/UniqExactSet.h @@ -54,10 +54,10 @@ public: { SCOPE_EXIT_SAFE( if (thread_group) - CurrentThread::detachQueryIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); ); if (thread_group) - CurrentThread::attachToIfDetached(thread_group); + CurrentThread::attachToGroupIfDetached(thread_group); setThreadName("UniqExactMerger"); while (true) diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index 9ff91050177..c6a0840964b 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -89,13 +89,13 @@ void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries if (!--num_active_jobs) event.notify_all(); if (async) - CurrentThread::detachQueryIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); ); try { if (async && thread_group) - CurrentThread::attachTo(thread_group); + CurrentThread::attachToGroup(thread_group); if (async) setThreadName("BackupWorker"); @@ -154,13 +154,13 @@ void restoreTablesData(DataRestoreTasks && tasks, ThreadPool & thread_pool) if (!--num_active_jobs) event.notify_all(); if (async) - CurrentThread::detachQueryIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); ); try { if (async && thread_group) - CurrentThread::attachTo(thread_group); + CurrentThread::attachToGroup(thread_group); if (async) setThreadName("RestoreWorker"); diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index ce922f32cfb..aa6ea9a46db 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -203,7 +203,12 @@ static void incrementProfileEventsBlock(Block & dst, const Block & src) for (size_t src_row = 0; src_row < src.rows(); ++src_row) { - /// Filter out threads stats, use thead group stats + /// Filter out threads stats, use stats from thread group + /// Exactly stats from thread group is stored to the table system.query_log + /// The stats from threads are less useful. + /// They take more records, they need to be combined, + /// there even could be several records from one thread. + /// Server doesn't send it any more to the clients, so this code left for compatible auto thread_id = src_array_thread_id[src_row]; if (thread_id != THREAD_GROUP_ID) continue; diff --git a/src/Common/CurrentThread.cpp b/src/Common/CurrentThread.cpp index ee1486fec7b..7fd82426522 100644 --- a/src/Common/CurrentThread.cpp +++ b/src/Common/CurrentThread.cpp @@ -57,14 +57,6 @@ void CurrentThread::updateProgressOut(const Progress & value) current_thread->progress_out.incrementPiecewiseAtomically(value); } -void CurrentThread::attachInternalTextLogsQueue(const std::shared_ptr & logs_queue, - LogsLevel client_logs_level) -{ - if (unlikely(!current_thread)) - return; - current_thread->attachInternalTextLogsQueue(logs_queue, client_logs_level); -} - std::shared_ptr CurrentThread::getInternalTextLogsQueue() { /// NOTE: this method could be called at early server startup stage @@ -74,13 +66,6 @@ std::shared_ptr CurrentThread::getInternalTextLogsQueue() return current_thread->getInternalTextLogsQueue(); } -void CurrentThread::attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & queue) -{ - if (unlikely(!current_thread)) - return; - current_thread->attachInternalProfileEventsQueue(queue); -} - InternalProfileEventsQueuePtr CurrentThread::getInternalProfileEventsQueue() { if (unlikely(!current_thread)) @@ -89,6 +74,15 @@ InternalProfileEventsQueuePtr CurrentThread::getInternalProfileEventsQueue() return current_thread->getInternalProfileEventsQueue(); } +void CurrentThread::attachInternalTextLogsQueue(const std::shared_ptr & logs_queue, + LogsLevel client_logs_level) +{ + if (unlikely(!current_thread)) + return; + current_thread->attachInternalTextLogsQueue(logs_queue, client_logs_level); +} + + ThreadGroupStatusPtr CurrentThread::getGroup() { if (unlikely(!current_thread)) @@ -97,4 +91,12 @@ ThreadGroupStatusPtr CurrentThread::getGroup() return current_thread->getThreadGroup(); } +std::string_view CurrentThread::getQueryId() +{ + if (unlikely(!current_thread)) + return {}; + + return current_thread->getQueryId(); +} + } diff --git a/src/Common/CurrentThread.h b/src/Common/CurrentThread.h index 3010085c514..3b16163b1ba 100644 --- a/src/Common/CurrentThread.h +++ b/src/Common/CurrentThread.h @@ -5,6 +5,7 @@ #include #include +#include namespace ProfileEvents @@ -48,6 +49,8 @@ public: static void attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & queue); static InternalProfileEventsQueuePtr getInternalProfileEventsQueue(); + static void attachQueryForLog(const String & query_); + /// Makes system calls to update ProfileEvents that contain info from rusage and taskstats static void updatePerformanceCounters(); @@ -65,24 +68,18 @@ public: /// You must call one of these methods when create a query child thread: /// Add current thread to a group associated with the thread group - static void attachTo(const ThreadGroupStatusPtr & thread_group); + static void attachToGroup(const ThreadGroupStatusPtr & thread_group); /// Is useful for a ThreadPool tasks - static void attachToIfDetached(const ThreadGroupStatusPtr & thread_group); + static void attachToGroupIfDetached(const ThreadGroupStatusPtr & thread_group); /// Non-master threads call this method in destructor automatically - static void detachGroupIfNotDetached(); - static void detachQueryIfNotDetached(); + static void detachFromGroupIfNotDetached(); /// Update ProfileEvents and dumps info to system.query_thread_log static void finalizePerformanceCounters(); /// Returns a non-empty string if the thread is attached to a query - static std::string_view getQueryId() - { - if (unlikely(!current_thread)) - return {}; - return current_thread->getQueryId(); - } + static std::string_view getQueryId(); /// Initializes query with current thread as master thread in constructor, and detaches it in destructor struct QueryScope : private boost::noncopyable diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index ef383aa5e5e..5b858ef5532 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include #include @@ -11,14 +10,12 @@ #include #include -#include #include namespace DB { - namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -71,35 +68,9 @@ static thread_local ThreadStack alt_stack; static thread_local bool has_alt_stack = false; #endif -ContextWeakPtr ThreadGroupStatus::getQueryContextWeak() const -{ - return query_context; -} - -ContextWeakPtr ThreadGroupStatus::getGlobalContextWeak() const -{ - return global_context; -} - -ThreadGroupStatus::FatalErrorCallback ThreadGroupStatus::getFatalErrorCallback() const -{ - return fatal_error_callback; -} - -void ThreadGroupStatus::link(ThreadStatusPtr thread) -{ - std::lock_guard lock(mutex); - - /// NOTE: thread may be attached multiple times if it is reused from a thread pool. - thread_ids.insert(thread->thread_id); - threads.insert(thread); -} - -void ThreadGroupStatus::unlink(ThreadStatusPtr thread) -{ - std::lock_guard guard(mutex); - threads.erase(thread); -} +ThreadGroupStatus::ThreadGroupStatus() + : master_thread_id(CurrentThread::get().thread_id) +{} ThreadStatus::ThreadStatus() : thread_id{getThreadId()} @@ -155,6 +126,64 @@ ThreadStatus::ThreadStatus() #endif } +ThreadGroupStatusPtr ThreadStatus::getThreadGroup() const +{ + return thread_group; +} + +const String & ThreadStatus::getQueryId() const +{ + return query_id_from_query_context; +} + +ContextPtr ThreadStatus::getQueryContext() const +{ + return query_context.lock(); +} + +ContextPtr ThreadStatus::getGlobalContext() const +{ + return global_context.lock(); +} + +void ThreadGroupStatus::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue, LogsLevel logs_level) +{ + std::lock_guard lock(mutex); + shared_data.logs_queue_ptr = logs_queue; + shared_data.client_logs_level = logs_level; +} + +void ThreadStatus::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue, + LogsLevel logs_level) +{ + if (!thread_group) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No thread group attached to the thread {}", thread_id); + + shared_data.logs_queue_ptr = logs_queue; + shared_data.client_logs_level = logs_level; + thread_group->attachInternalTextLogsQueue(logs_queue, logs_level); +} + +InternalTextLogsQueuePtr ThreadStatus::getInternalTextLogsQueue() const +{ + return shared_data.logs_queue_ptr.lock(); +} + +InternalProfileEventsQueuePtr ThreadStatus::getInternalProfileEventsQueue() const +{ + return shared_data.profile_queue_ptr.lock(); +} + +const String & ThreadStatus::getQueryForLog() const +{ + return shared_data.query_for_logs; +} + +LogsLevel ThreadStatus::getClientLogsLevel() const +{ + return shared_data.client_logs_level; +} + void ThreadStatus::flushUntrackedMemory() { if (untracked_memory == 0) @@ -170,7 +199,7 @@ ThreadStatus::~ThreadStatus() /// It may cause segfault if query_context was destroyed, but was not detached auto query_context_ptr = query_context.lock(); - assert((!query_context_ptr && query_id.empty()) || (query_context_ptr && query_id == query_context_ptr->getCurrentQueryId())); + assert((!query_context_ptr && getQueryId().empty()) || (query_context_ptr && getQueryId() == query_context_ptr->getCurrentQueryId())); /// detachGroup if it was attached if (deleter) @@ -196,61 +225,25 @@ void ThreadStatus::updatePerformanceCounters() } } -void ThreadStatus::assertState(ThreadState permitted_state, const char * description) const -{ - auto curr_state = thread_state.load(); - - if (curr_state == permitted_state) - return; - - if (description) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected thread state {}: {}", curr_state, description); - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected thread state {}", curr_state); -} - -void ThreadStatus::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue, - LogsLevel client_logs_level) -{ - logs_queue_ptr = logs_queue; - chassert(thread_group); - - std::lock_guard lock(thread_group->mutex); - thread_group->logs_queue_ptr = logs_queue; - thread_group->client_logs_level = client_logs_level; -} - -void ThreadStatus::attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue) -{ - profile_queue_ptr = profile_queue; - - chassert(thread_group); - - std::lock_guard lock(thread_group->mutex); - thread_group->profile_queue_ptr = profile_queue; -} - void ThreadStatus::onFatalError() { - /// No thread group - no callback - if (!thread_group) - return; - - std::lock_guard lock(thread_group->mutex); if (fatal_error_callback) fatal_error_callback(); } ThreadStatus * MainThreadStatus::main_thread = nullptr; + MainThreadStatus & MainThreadStatus::getInstance() { static MainThreadStatus thread_status; return thread_status; } + MainThreadStatus::MainThreadStatus() { main_thread = current_thread; } + MainThreadStatus::~MainThreadStatus() { main_thread = nullptr; diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 162defd8bd5..4815a6e0971 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -63,50 +63,57 @@ using ThreadGroupStatusPtr = std::shared_ptr; class ThreadGroupStatus { public: + ThreadGroupStatus(); + using FatalErrorCallback = std::function; + ThreadGroupStatus(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_ = {}); + /// The first thread created this thread group - UInt64 master_thread_id = 0; + const UInt64 master_thread_id; + + /// Set up at creation, no race when reading + const ContextWeakPtr query_context; + const ContextWeakPtr global_context; + + const FatalErrorCallback fatal_error_callback; ProfileEvents::Counters performance_counters{VariableContext::Process}; MemoryTracker memory_tracker{VariableContext::Process}; - /// Access to the members below has to be in critical section with mutex - mutable std::mutex mutex; + struct SharedData + { + InternalProfileEventsQueueWeakPtr profile_queue_ptr; - InternalTextLogsQueueWeakPtr logs_queue_ptr; - InternalProfileEventsQueueWeakPtr profile_queue_ptr; + InternalTextLogsQueueWeakPtr logs_queue_ptr; + LogsLevel client_logs_level = LogsLevel::none; - LogsLevel client_logs_level = LogsLevel::none; + String query_for_logs; + UInt64 normalized_query_hash = 0; + }; - String query; - UInt64 normalized_query_hash = 0; + SharedData getSharedData() + { + std::lock_guard lock(mutex); + return shared_data; + } + + /// Mutation shared data + void attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue, LogsLevel logs_level); + void attachQueryForLog(const String & query_, UInt64 normalized_hash = 0); + void attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue); /// When new query starts, new thread group is created for it, current thread becomes master thread of the query - static ThreadGroupStatusPtr createForQuery(ContextPtr query_context_, std::function fatal_error_callback_ = {}); + static ThreadGroupStatusPtr createForQuery(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_ = {}); - const std::vector getInvolvedThreadIds() const; - - void link(ThreadStatusPtr thread); - void unlink(ThreadStatusPtr thread); - - ContextWeakPtr getQueryContextWeak() const; - ContextWeakPtr getGlobalContextWeak() const; - - using FatalErrorCallback = std::function; - FatalErrorCallback getFatalErrorCallback() const; + std::vector getInvolvedThreadIds() const; + void linkThread(UInt64 thread_it); private: - /// Set up at creation, no race when reading - ContextWeakPtr query_context; - ContextWeakPtr global_context; + mutable std::mutex mutex; /// Set up at creation, no race when reading - FatalErrorCallback fatal_error_callback; - + SharedData shared_data; /// Set of all thread ids which has been attached to the group std::unordered_set thread_ids; - - /// Set of active threads - std::unordered_set threads; }; /** @@ -150,53 +157,24 @@ public: Progress progress_in; Progress progress_out; -protected: +private: /// Group of threads, to which this thread attached ThreadGroupStatusPtr thread_group; - std::atomic thread_state{ThreadState::DetachedFromQuery}; - /// Is set once ContextWeakPtr global_context; /// Use it only from current thread ContextWeakPtr query_context; - String query_id_from_query_context; + /// Is used to send logs from logs_queue to client in case of fatal errors. + using FatalErrorCallback = std::function; + FatalErrorCallback fatal_error_callback; - /// A logs queue used by TCPHandler to pass logs to a client - InternalTextLogsQueueWeakPtr logs_queue_ptr; - - InternalProfileEventsQueueWeakPtr profile_queue_ptr; - - struct TimePoint - { - void setUp(); - void SetUp(std::chrono::time_point now); - - UInt64 nanoseconds = 0; - UInt64 microseconds = 0; - time_t seconds = 0; - }; + ThreadGroupStatus::SharedData shared_data; bool performance_counters_finalized = false; - TimePoint query_start_time{}; - - // CPU and Real time query profilers - std::unique_ptr query_profiler_real; - std::unique_ptr query_profiler_cpu; - - Poco::Logger * log = nullptr; - - /// Use ptr not to add extra dependencies in the header - std::unique_ptr last_rusage; - std::unique_ptr taskstats; - - /// Is used to send logs from logs_queue to client in case of fatal errors. - std::function fatal_error_callback; - - /// See setInternalThread() - bool internal_thread = false; + String query_id_from_query_context; /// Requires access to query_id. friend class MemoryTrackerThreadSwitcher; void setQueryId(const String & query_id_) @@ -204,39 +182,45 @@ protected: query_id_from_query_context = query_id_; } + struct TimePoint + { + void setUp(); + UInt64 nanoseconds() const; + UInt64 microseconds() const; + UInt64 seconds() const; + + std::chrono::time_point point; + }; + + TimePoint query_start_time{}; + + // CPU and Real time query profilers + std::unique_ptr query_profiler_real; + std::unique_ptr query_profiler_cpu; + + /// Use ptr not to add extra dependencies in the header + std::unique_ptr last_rusage; + std::unique_ptr taskstats; + + /// See setInternalThread() + bool internal_thread = false; + /// This is helpful for cut linking dependencies for clickhouse_common_io using Deleter = std::function; Deleter deleter; + Poco::Logger * log = nullptr; + public: ThreadStatus(); ~ThreadStatus(); - ThreadGroupStatusPtr getThreadGroup() const - { - return thread_group; - } + ThreadGroupStatusPtr getThreadGroup() const; - enum ThreadState - { - DetachedFromQuery = 0, /// We just created thread or it is a background thread - AttachedToQuery, /// Thread executes enqueued query - }; + const String & getQueryId() const; - std::string_view getQueryId() const - { - return query_id_from_query_context; - } - - auto getQueryContext() const - { - return query_context.lock(); - } - - auto getGlobalContext() const - { - return global_context.lock(); - } + ContextPtr getQueryContext() const; + ContextPtr getGlobalContext() const; /// "Internal" ThreadStatus is used for materialized views for separate /// tracking into system.query_views_log @@ -255,29 +239,25 @@ public: void setInternalThread(); /// Attaches slave thread to existing thread group - void attachTo(const ThreadGroupStatusPtr & thread_group_, bool check_detached = true); + void attachToGroup(const ThreadGroupStatusPtr & thread_group_, bool check_detached = true); /// Detaches thread from the thread group and the query, dumps performance counters if they have not been dumped - void detachGroup(); + void detachFromGroup(); /// Returns pointer to the current profile counters to restore them back. /// Note: consequent call with new scope will detach previous scope. ProfileEvents::Counters * attachProfileCountersScope(ProfileEvents::Counters * performance_counters_scope); - InternalTextLogsQueuePtr getInternalTextLogsQueue() const - { - return logs_queue_ptr.lock(); - } - void attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue, LogsLevel client_logs_level); - - InternalProfileEventsQueuePtr getInternalProfileEventsQueue() const - { - return profile_queue_ptr.lock(); - } + InternalTextLogsQueuePtr getInternalTextLogsQueue() const; + LogsLevel getClientLogsLevel() const; void attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue); + InternalProfileEventsQueuePtr getInternalProfileEventsQueue() const; + + void attachQueryForLog(const String & query_); + const String & getQueryForLog() const; /// Proper cal for fatal_error_callback void onFatalError(); @@ -295,7 +275,7 @@ public: void flushUntrackedMemory(); -protected: +private: void applyQuerySettings(); void initPerformanceCounters(); @@ -304,12 +284,9 @@ protected: void finalizeQueryProfiler(); - void logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database, std::chrono::time_point now); + void logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database); - void assertState(ThreadState permitted_state, const char * description = nullptr) const; - -private: - void attachGroupImp(const ThreadGroupStatusPtr & thread_group_); + void attachToGroupImpl(const ThreadGroupStatusPtr & thread_group_); }; /** diff --git a/src/Common/logger_useful.h b/src/Common/logger_useful.h index 2a0c582331d..ba1e2e7789b 100644 --- a/src/Common/logger_useful.h +++ b/src/Common/logger_useful.h @@ -34,7 +34,7 @@ namespace { \ auto _logger = ::getLogger(logger); \ const bool _is_clients_log = (DB::CurrentThread::getGroup() != nullptr) && \ - (DB::CurrentThread::getGroup()->client_logs_level >= (priority)); \ + (DB::CurrentThread::get().getClientLogsLevel() >= (priority)); \ if (_is_clients_log || _logger->is((PRIORITY))) \ { \ std::string formatted_message = numArgs(__VA_ARGS__) > 1 ? fmt::format(__VA_ARGS__) : firstArg(__VA_ARGS__); \ diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 18c4c0d97a0..5564ac7c9cd 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -312,12 +312,8 @@ private: /// It will allow client to see failure messages directly. if (thread_ptr) { - query_id = std::string(thread_ptr->getQueryId()); - - if (auto thread_group = thread_ptr->getThreadGroup()) - { - query = DB::toOneLineQuery(thread_group->query); - } + query_id = thread_ptr->getQueryId(); + query = thread_ptr->getQueryForLog(); if (auto logs_queue = thread_ptr->getInternalTextLogsQueue()) { diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 309a3dda2e4..d6c9ac50dbe 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -75,7 +75,7 @@ public: pool.scheduleOrThrowOnError([this, shard, thread_group = CurrentThread::getGroup()] { if (thread_group) - CurrentThread::attachToIfDetached(thread_group); + CurrentThread::attachToGroupIfDetached(thread_group); setThreadName("HashedDictLoad"); threadWorker(shard); @@ -224,7 +224,7 @@ HashedDictionary::~HashedDictionary() pool.trySchedule([&container, thread_group = CurrentThread::getGroup()] { if (thread_group) - CurrentThread::attachToIfDetached(thread_group); + CurrentThread::attachToGroupIfDetached(thread_group); setThreadName("HashedDictDtor"); if constexpr (sparse) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 09c2eebfdd6..0f7cb961e34 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -2309,10 +2309,10 @@ BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl( { SCOPE_EXIT_SAFE( if (thread_group) - CurrentThread::detachQueryIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); ); if (thread_group) - CurrentThread::attachToIfDetached(thread_group); + CurrentThread::attachToGroupIfDetached(thread_group); BlocksList blocks; while (true) @@ -3030,10 +3030,10 @@ void Aggregator::mergeBlocks(BucketToBlocks bucket_to_blocks, AggregatedDataVari { SCOPE_EXIT_SAFE( if (thread_group) - CurrentThread::detachQueryIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); ); if (thread_group) - CurrentThread::attachToIfDetached(thread_group); + CurrentThread::attachToGroupIfDetached(thread_group); for (Block & block : bucket_to_blocks[bucket]) { diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index 9858b27d57a..04a116ec0c7 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -971,11 +971,11 @@ private: { SCOPE_EXIT_SAFE( if (thread_group) - CurrentThread::detachQueryIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); ); if (thread_group) - CurrentThread::attachTo(thread_group); + CurrentThread::attachToGroup(thread_group); LOG_TRACE(log, "Start loading object '{}'", name); try diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 9f557bf3ff5..5db39ece2e5 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -203,10 +203,10 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q ProcessListForUser & user_process_list = user_process_list_it->second; /// Actualize thread group info + CurrentThread::attachQueryForLog(query_); auto thread_group = CurrentThread::getGroup(); if (thread_group) { - std::lock_guard lock_thread_group(thread_group->mutex); thread_group->performance_counters.setParent(&user_process_list.user_performance_counters); thread_group->memory_tracker.setParent(&user_process_list.user_memory_tracker); if (user_process_list.user_temp_data_on_disk) @@ -214,8 +214,6 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q query_context->setTempDataOnDisk(std::make_shared( user_process_list.user_temp_data_on_disk, settings.max_temporary_data_on_disk_size_for_query)); } - thread_group->query = query_; - thread_group->normalized_query_hash = normalizedQueryHash(query_); /// Set query-level memory trackers thread_group->memory_tracker.setOrRaiseHardLimit(settings.max_memory_usage); diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 70c4b15dd42..11ca3c7a707 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -40,7 +41,14 @@ namespace ErrorCodes extern const int CANNOT_SET_THREAD_PRIORITY; } -const std::vector ThreadGroupStatus::getInvolvedThreadIds() const +ThreadGroupStatus::ThreadGroupStatus(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_) + : master_thread_id(CurrentThread::get().thread_id) + , query_context(query_context_) + , global_context(query_context_->getGlobalContext()) + , fatal_error_callback(fatal_error_callback_) +{} + +std::vector ThreadGroupStatus::getInvolvedThreadIds() const { std::vector res; @@ -52,24 +60,75 @@ const std::vector ThreadGroupStatus::getInvolvedThreadIds() const return res; } +void ThreadGroupStatus::linkThread(UInt64 thread_it) +{ + std::lock_guard lock(mutex); + thread_ids.insert(thread_it); +} + ThreadGroupStatusPtr ThreadGroupStatus::createForQuery(ContextPtr query_context_, std::function fatal_error_callback_) { - auto group = std::make_shared(); + auto group = std::make_shared(query_context_, std::move(fatal_error_callback_)); group->memory_tracker.setDescription("(for query)"); - group->master_thread_id = CurrentThread::get().thread_id; - - group->query_context = query_context_; - group->global_context = query_context_->getGlobalContext(); - - group->fatal_error_callback = std::move(fatal_error_callback_); - return group; } +void ThreadGroupStatus::attachQueryForLog(const String & query_, UInt64 normalized_hash) +{ + auto hash = normalized_hash ? normalized_hash : normalizedQueryHash(query_); + + std::lock_guard lock(mutex); + shared_data.query_for_logs = query_; + shared_data.normalized_query_hash = hash; +} + +void ThreadStatus::attachQueryForLog(const String & query_) +{ + shared_data.query_for_logs = query_; + shared_data.normalized_query_hash = normalizedQueryHash(query_); + + if (!thread_group) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No thread group attached to the thread {}", thread_id); + + thread_group->attachQueryForLog(shared_data.query_for_logs, shared_data.normalized_query_hash); +} + +void ThreadGroupStatus::attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue) +{ + std::lock_guard lock(mutex); + shared_data.profile_queue_ptr = profile_queue; +} + +void ThreadStatus::attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue) +{ + if (!thread_group) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No thread group attached to the thread {}", thread_id); + + shared_data.profile_queue_ptr = profile_queue; + thread_group->attachInternalProfileEventsQueue(profile_queue); +} + +void CurrentThread::attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & queue) +{ + if (unlikely(!current_thread)) + return; + current_thread->attachInternalProfileEventsQueue(queue); +} + +void CurrentThread::attachQueryForLog(const String & query_) +{ + if (unlikely(!current_thread)) + return; + current_thread->attachQueryForLog(query_); +} + + void ThreadStatus::applyQuerySettings() { auto query_context_ptr = query_context.lock(); - assert(query_context_ptr); + if (!query_context_ptr) + return; + const Settings & settings = query_context_ptr->getSettingsRef(); query_id_from_query_context = query_context_ptr->getCurrentQueryId(); @@ -94,69 +153,53 @@ void ThreadStatus::applyQuerySettings() #endif } -void ThreadStatus::attachGroupImp(const ThreadGroupStatusPtr & thread_group_) +void ThreadStatus::attachToGroupImpl(const ThreadGroupStatusPtr & thread_group_) { - assertState(ThreadState::DetachedFromQuery, __PRETTY_FUNCTION__); - /// Attach or init current thread to thread group and copy useful information from it thread_group = thread_group_; + thread_group->linkThread(thread_id); performance_counters.setParent(&thread_group->performance_counters); memory_tracker.setParent(&thread_group->memory_tracker); - thread_group->link(this); + query_context = thread_group->query_context; + global_context = thread_group->global_context; - query_context = thread_group->getQueryContextWeak(); - if (global_context.expired()) - global_context = thread_group->getGlobalContextWeak(); + fatal_error_callback = thread_group->fatal_error_callback; - fatal_error_callback = thread_group->getFatalErrorCallback(); - - { - std::lock_guard lock(thread_group->mutex); - - logs_queue_ptr = thread_group->logs_queue_ptr; - profile_queue_ptr = thread_group->profile_queue_ptr; - } - - if (auto query_context_ptr = query_context.lock()) - { - applyQuerySettings(); - } + shared_data = thread_group->getSharedData(); + applyQuerySettings(); initPerformanceCounters(); - - thread_state = ThreadState::AttachedToQuery; } -void ThreadStatus::detachGroup() +void ThreadStatus::detachFromGroup() { + if (!thread_group) + return; + LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global); - if (thread_state == ThreadState::DetachedFromQuery) - return; + /// flash untracked memory before resetting memory_tracker parent + flushUntrackedMemory(); finalizeQueryProfiler(); finalizePerformanceCounters(); - thread_group->unlink(this); - performance_counters.setParent(&ProfileEvents::global_counters); - flushUntrackedMemory(); - memory_tracker.reset(); memory_tracker.setParent(thread_group->memory_tracker.getParent()); + thread_group.reset(); + query_id_from_query_context.clear(); query_context.reset(); + shared_data = {}; + fatal_error_callback = {}; - thread_group.reset(); - - thread_state = ThreadState::DetachedFromQuery; - #if defined(OS_LINUX) if (os_thread_priority) { @@ -176,19 +219,19 @@ void ThreadStatus::setInternalThread() internal_thread = true; } -void ThreadStatus::attachTo(const ThreadGroupStatusPtr & thread_group_, bool check_detached) +void ThreadStatus::attachToGroup(const ThreadGroupStatusPtr & thread_group_, bool check_detached) { - if (check_detached) - assertState(ThreadState::DetachedFromQuery, "Can't attach query to the thread, it is already attached"); + if (thread_group && check_detached) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't attach query to the thread, it is already attached"); if (!thread_group_) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to attach to nullptr thread group"); - if (thread_state == ThreadState::AttachedToQuery) + if (thread_group) return; - deleter = [this] () { detachGroup(); }; - attachGroupImp(thread_group_); + deleter = [this] () { detachFromGroup(); }; + attachToGroupImpl(thread_group_); } ProfileEvents::Counters * ThreadStatus::attachProfileCountersScope(ProfileEvents::Counters * performance_counters_scope) @@ -208,18 +251,24 @@ ProfileEvents::Counters * ThreadStatus::attachProfileCountersScope(ProfileEvents return prev_counters; } -void ThreadStatus::TimePoint::SetUp(std::chrono::time_point now) -{ - // query_start_time_{microseconds, nanoseconds} are all constructed from the same time point - // to ensure that they are all equal up to the precision of a second. - nanoseconds = timeInNanoseconds(now); - microseconds = timeInMicroseconds(now); - seconds = timeInSeconds(now); -} - void ThreadStatus::TimePoint::setUp() { - SetUp(std::chrono::system_clock::now()); + point = std::chrono::system_clock::now(); +} + +UInt64 ThreadStatus::TimePoint::nanoseconds() const +{ + return timeInNanoseconds(point); +} + +UInt64 ThreadStatus::TimePoint::microseconds() const +{ + return timeInMicroseconds(point); +} + +UInt64 ThreadStatus::TimePoint::seconds() const +{ + return timeInSeconds(point); } void ThreadStatus::initPerformanceCounters() @@ -309,11 +358,11 @@ void ThreadStatus::finalizePerformanceCounters() if (settings.log_queries && settings.log_query_threads) { const auto now = std::chrono::system_clock::now(); - Int64 query_duration_ms = (timeInMicroseconds(now) - query_start_time.microseconds) / 1000; + Int64 query_duration_ms = std::chrono::duration_cast(now - query_start_time.point).count(); if (query_duration_ms >= settings.log_queries_min_query_duration_ms.totalMilliseconds()) { if (auto thread_log = global_context_ptr->getQueryThreadLog()) - logToQueryThreadLog(*thread_log, query_context_ptr->getCurrentDatabase(), now); + logToQueryThreadLog(*thread_log, query_context_ptr->getCurrentDatabase()); } } } @@ -368,20 +417,20 @@ void ThreadStatus::finalizeQueryProfiler() query_profiler_cpu.reset(); } -void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database, std::chrono::time_point now) +void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database) { QueryThreadLogElement elem; // construct current_time and current_time_microseconds using the same time point // so that the two times will always be equal up to a precision of a second. TimePoint current_time; - current_time.SetUp(now); + current_time.setUp(); - elem.event_time = current_time.seconds; - elem.event_time_microseconds = current_time.microseconds; - elem.query_start_time = query_start_time.seconds; - elem.query_start_time_microseconds = query_start_time.microseconds; - elem.query_duration_ms = (current_time.nanoseconds - query_start_time.nanoseconds) / 1000000U; + elem.event_time = current_time.seconds(); + elem.event_time_microseconds = current_time.microseconds(); + elem.query_start_time = query_start_time.seconds(); + elem.query_start_time_microseconds = query_start_time.microseconds(); + elem.query_duration_ms = std::chrono::duration_cast(current_time.point - query_start_time.point).count(); elem.read_rows = progress_in.read_rows.load(std::memory_order_relaxed); elem.read_bytes = progress_in.read_bytes.load(std::memory_order_relaxed); @@ -397,13 +446,9 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log, const String elem.current_database = current_database; if (thread_group) { - { - std::lock_guard lock(thread_group->mutex); - - elem.master_thread_id = thread_group->master_thread_id; - elem.query = thread_group->query; - elem.normalized_query_hash = thread_group->normalized_query_hash; - } + elem.master_thread_id = thread_group->master_thread_id; + elem.query = shared_data.query_for_logs; + elem.normalized_query_hash = shared_data.normalized_query_hash; } auto query_context_ptr = query_context.lock(); @@ -478,18 +523,18 @@ void ThreadStatus::logToQueryViewsLog(const ViewRuntimeData & vinfo) views_log->add(element); } -void CurrentThread::attachTo(const ThreadGroupStatusPtr & thread_group) +void CurrentThread::attachToGroup(const ThreadGroupStatusPtr & thread_group) { if (unlikely(!current_thread)) return; - current_thread->attachTo(thread_group, true); + current_thread->attachToGroup(thread_group, true); } -void CurrentThread::attachToIfDetached(const ThreadGroupStatusPtr & thread_group) +void CurrentThread::attachToGroupIfDetached(const ThreadGroupStatusPtr & thread_group) { if (unlikely(!current_thread)) return; - current_thread->attachTo(thread_group, false); + current_thread->attachToGroup(thread_group, false); } void CurrentThread::finalizePerformanceCounters() @@ -499,16 +544,11 @@ void CurrentThread::finalizePerformanceCounters() current_thread->finalizePerformanceCounters(); } -void CurrentThread::detachGroupIfNotDetached() +void CurrentThread::detachFromGroupIfNotDetached() { if (unlikely(!current_thread)) return; - current_thread->detachGroup(); -} - -void CurrentThread::detachQueryIfNotDetached() -{ - return detachGroupIfNotDetached(); + current_thread->detachFromGroup(); } CurrentThread::QueryScope::QueryScope(ContextMutablePtr query_context, std::function fatal_error_callback) @@ -517,7 +557,7 @@ CurrentThread::QueryScope::QueryScope(ContextMutablePtr query_context, std::func query_context->makeQueryContext(); auto group = ThreadGroupStatus::createForQuery(query_context, std::move(fatal_error_callback)); - CurrentThread::attachTo(std::move(group)); + CurrentThread::attachToGroup(std::move(group)); } CurrentThread::QueryScope::QueryScope(ContextPtr query_context, std::function fatal_error_callback) @@ -527,7 +567,7 @@ CurrentThread::QueryScope::QueryScope(ContextPtr query_context, std::function threadPoolCallbackRunner(ThreadPool & auto task = std::make_shared>([thread_group, thread_name, callback = std::move(callback)]() mutable -> Result { if (thread_group) - CurrentThread::attachTo(thread_group); + CurrentThread::attachToGroup(thread_group); SCOPE_EXIT_SAFE({ { @@ -33,7 +33,7 @@ ThreadPoolCallbackRunner threadPoolCallbackRunner(ThreadPool & } if (thread_group) - CurrentThread::detachQueryIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); }); diff --git a/src/Processors/Executors/CompletedPipelineExecutor.cpp b/src/Processors/Executors/CompletedPipelineExecutor.cpp index 22b924337c5..e624ecd52de 100644 --- a/src/Processors/Executors/CompletedPipelineExecutor.cpp +++ b/src/Processors/Executors/CompletedPipelineExecutor.cpp @@ -36,14 +36,14 @@ static void threadFunction(CompletedPipelineExecutor::Data & data, ThreadGroupSt { SCOPE_EXIT_SAFE( if (thread_group) - CurrentThread::detachQueryIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); ); setThreadName("QueryCompPipeEx"); try { if (thread_group) - CurrentThread::attachTo(thread_group); + CurrentThread::attachToGroup(thread_group); data.executor->execute(num_threads); } diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index f1e044e470b..313a5139581 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -308,12 +308,12 @@ void PipelineExecutor::spawnThreads() SCOPE_EXIT_SAFE( if (thread_group) - CurrentThread::detachQueryIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); ); setThreadName("QueryPipelineEx"); if (thread_group) - CurrentThread::attachTo(thread_group); + CurrentThread::attachToGroup(thread_group); try { diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index 0a7a9025b30..95ed4eb813d 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -71,14 +71,14 @@ static void threadFunction(PullingAsyncPipelineExecutor::Data & data, ThreadGrou { SCOPE_EXIT_SAFE( if (thread_group) - CurrentThread::detachQueryIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); ); setThreadName("QueryPullPipeEx"); try { if (thread_group) - CurrentThread::attachTo(thread_group); + CurrentThread::attachToGroup(thread_group); data.executor->execute(num_threads); } diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp index 4478f1548a4..3aec7608e6d 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp @@ -101,14 +101,14 @@ static void threadFunction(PushingAsyncPipelineExecutor::Data & data, ThreadGrou { SCOPE_EXIT_SAFE( if (thread_group) - CurrentThread::detachQueryIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); ); setThreadName("QueryPushPipeEx"); try { if (thread_group) - CurrentThread::attachTo(thread_group); + CurrentThread::attachToGroup(thread_group); data.executor->execute(num_threads); } diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp index 35d4dd9ddcd..3fc57ca1c1e 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp @@ -100,11 +100,11 @@ namespace DB { SCOPE_EXIT_SAFE( if (thread_group) - CurrentThread::detachQueryIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); ); setThreadName("Collector"); if (thread_group) - CurrentThread::attachToIfDetached(thread_group); + CurrentThread::attachToGroupIfDetached(thread_group); try { @@ -161,11 +161,11 @@ namespace DB { SCOPE_EXIT_SAFE( if (thread_group) - CurrentThread::detachQueryIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); ); setThreadName("Formatter"); if (thread_group) - CurrentThread::attachToIfDetached(thread_group); + CurrentThread::attachToGroupIfDetached(thread_group); try { diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp index 293bf4f73f3..5ba32251a71 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp @@ -12,10 +12,10 @@ void ParallelParsingInputFormat::segmentatorThreadFunction(ThreadGroupStatusPtr { SCOPE_EXIT_SAFE( if (thread_group) - CurrentThread::detachQueryIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); ); if (thread_group) - CurrentThread::attachTo(thread_group); + CurrentThread::attachToGroup(thread_group); setThreadName("Segmentator"); try @@ -62,10 +62,10 @@ void ParallelParsingInputFormat::parserThreadFunction(ThreadGroupStatusPtr threa { SCOPE_EXIT_SAFE( if (thread_group) - CurrentThread::detachQueryIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); ); if (thread_group) - CurrentThread::attachToIfDetached(thread_group); + CurrentThread::attachToGroupIfDetached(thread_group); const auto parser_unit_number = current_ticket_number % processing_units.size(); auto & unit = processing_units[parser_unit_number]; diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 83dfc01e6b0..3abd2ac3346 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -100,10 +100,10 @@ struct ManyAggregatedData { SCOPE_EXIT_SAFE( if (thread_group) - CurrentThread::detachQueryIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); ); if (thread_group) - CurrentThread::attachToIfDetached(thread_group); + CurrentThread::attachToGroupIfDetached(thread_group); setThreadName("AggregDestruct"); }); diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 8cfa7160078..5ab1e811efb 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -286,7 +286,7 @@ Chain buildPushingToViewsChain( std::unique_ptr view_thread_status_ptr = std::make_unique(); /// Copy of a ThreadStatus should be internal. view_thread_status_ptr->setInternalThread(); - view_thread_status_ptr->attachTo(running_group); + view_thread_status_ptr->attachToGroup(running_group); auto * view_thread_status = view_thread_status_ptr.get(); views_data->thread_status_holder->thread_statuses.push_front(std::move(view_thread_status_ptr)); diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index d388a403031..19eab0b5837 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -293,12 +293,12 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si { SCOPE_EXIT_SAFE( if (thread_group) - CurrentThread::detachQueryIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); ); OpenTelemetry::SpanHolder span(__PRETTY_FUNCTION__); if (thread_group) - CurrentThread::attachToIfDetached(thread_group); + CurrentThread::attachToGroupIfDetached(thread_group); setThreadName("DistrOutStrProc"); ++job.blocks_started; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ee5cf0ea450..415466926f4 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1395,10 +1395,10 @@ std::vector MergeTreeData::loadDataPartsFromDisk( { SCOPE_EXIT_SAFE( if (thread_group) - CurrentThread::detachQueryIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); ); if (thread_group) - CurrentThread::attachToIfDetached(thread_group); + CurrentThread::attachToGroupIfDetached(thread_group); while (true) { @@ -2314,10 +2314,10 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t { SCOPE_EXIT_SAFE( if (thread_group) - CurrentThread::detachQueryIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); ); if (thread_group) - CurrentThread::attachToIfDetached(thread_group); + CurrentThread::attachToGroupIfDetached(thread_group); asMutableDeletingPart(part)->remove(); if (part_names_succeed) @@ -2375,10 +2375,10 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t { SCOPE_EXIT_SAFE( if (thread_group) - CurrentThread::detachQueryIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); ); if (thread_group) - CurrentThread::attachToIfDetached(thread_group); + CurrentThread::attachToGroupIfDetached(thread_group); LOG_TRACE(log, "Removing {} parts in blocks range {}", batch.size(), range.getPartNameForLogs()); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 07da66e4378..936b9561725 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1119,10 +1119,10 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd { SCOPE_EXIT_SAFE( if (thread_group) - CurrentThread::detachQueryIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); ); if (thread_group) - CurrentThread::attachToIfDetached(thread_group); + CurrentThread::attachToGroupIfDetached(thread_group); process_part(part_index); }); diff --git a/src/Storages/System/StorageSystemStackTrace.cpp b/src/Storages/System/StorageSystemStackTrace.cpp index da3d6b98dc5..26411bf3bcb 100644 --- a/src/Storages/System/StorageSystemStackTrace.cpp +++ b/src/Storages/System/StorageSystemStackTrace.cpp @@ -90,7 +90,7 @@ namespace const ucontext_t signal_context = *reinterpret_cast(context); stack_trace = StackTrace(signal_context); - std::string_view query_id = CurrentThread::getQueryId(); + auto query_id = CurrentThread::getQueryId(); query_id_size = std::min(query_id.size(), max_query_id_size); if (!query_id.empty()) memcpy(query_id_data, query_id.data(), query_id_size); diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference index 7d7688db881..2d41f5dae89 100644 --- a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference @@ -16,8 +16,8 @@ INSERT and READ INSERT [ 0 ] FileOpen: 7 DROP CHECK with query_log -QueryFinish INSERT INTO times SELECT now() + INTERVAL 1 day; FileOpen 7 +QueryFinish INSERT INTO times SELECT now() + INTERVAL 1 day SETTINGS optimize_on_insert = 0; FileOpen 7 QueryFinish SELECT \'1\', min(t) FROM times; FileOpen 0 -QueryFinish INSERT INTO times SELECT now() + INTERVAL 2 day; FileOpen 7 +QueryFinish INSERT INTO times SELECT now() + INTERVAL 2 day SETTINGS optimize_on_insert = 0; FileOpen 7 QueryFinish SELECT \'2\', min(t) FROM times; FileOpen 0 -QueryFinish INSERT INTO times SELECT now() + INTERVAL 3 day; FileOpen 7 +QueryFinish INSERT INTO times SELECT now() + INTERVAL 3 day SETTINGS optimize_on_insert = 0; FileOpen 7 diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh index ce6a7e114af..0101386e100 100755 --- a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh @@ -29,12 +29,20 @@ ORDER BY query_start_time DESC; echo "CREATE" $CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " DROP TABLE IF EXISTS times; -CREATE TABLE times (t DateTime) ENGINE MergeTree ORDER BY t; +CREATE TABLE times (t DateTime) ENGINE MergeTree ORDER BY t + SETTINGS + storage_policy='default', + min_rows_for_compact_part = 0, + min_bytes_for_compact_part = 0, + min_rows_for_wide_part = 1000000, + min_bytes_for_wide_part = 1000000, + in_memory_parts_enable_wal = 0, + ratio_of_defaults_for_sparse_serialization=1.0; " 2>&1 | grep -o -e '\ \[\ .*\ \]\ FileOpen:\ .*\ ' echo "INSERT" $CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " -INSERT INTO times SELECT now() + INTERVAL 1 day; +INSERT INTO times SELECT now() + INTERVAL 1 day SETTINGS optimize_on_insert = 0; " 2>&1 | grep -o -e '\ \[\ .*\ \]\ FileOpen:\ .*\ ' echo "READ" @@ -44,13 +52,13 @@ SELECT '1', min(t) FROM times; echo "INSERT and READ INSERT" $CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " -INSERT INTO times SELECT now() + INTERVAL 2 day; +INSERT INTO times SELECT now() + INTERVAL 2 day SETTINGS optimize_on_insert = 0; SELECT '2', min(t) FROM times; -INSERT INTO times SELECT now() + INTERVAL 3 day; +INSERT INTO times SELECT now() + INTERVAL 3 day SETTINGS optimize_on_insert = 0; " 2>&1 | grep -o -e '\ \[\ .*\ \]\ FileOpen:\ .*\ ' echo "DROP" -$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " +$CLICKHOUSE_CLIENT -nq " DROP TABLE times; " @@ -62,7 +70,7 @@ SELECT type, 'FileOpen', ProfileEvents['FileOpen'] FROM system.query_log WHERE current_database = currentDatabase() -AND ( query LIKE '%SELECT % FROM times%' OR query LIKE '%INSERT INTO times%') +AND ( query LIKE '%SELECT % FROM times%' OR query LIKE '%INSERT INTO times%' ) AND type = 'QueryFinish' ORDER BY query_start_time_microseconds ASC, query DESC; " From 4c9e2aecbd3ca3d797aa602c7b0e1959cd84293d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 16 Mar 2023 19:49:30 +0000 Subject: [PATCH 202/559] allow to use qualified matcher for Map type --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 20 ++++++++++++------- ...77_analyzer_compound_expressions.reference | 1 + .../02677_analyzer_compound_expressions.sql | 2 +- 3 files changed, 15 insertions(+), 8 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 3c12d3135d1..cee004df536 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1,4 +1,3 @@ -#include "Analyzer/Identifier.h" #include #include @@ -3666,8 +3665,15 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveQualifiedMatcher(Qu { auto result_type = expression_query_tree_node->getResultType(); - while (const auto * array_type = typeid_cast(result_type.get())) - result_type = array_type->getNestedType(); + while (true) + { + if (const auto * array_type = typeid_cast(result_type.get())) + result_type = array_type->getNestedType(); + else if (const auto * map_type = typeid_cast(result_type.get())) + result_type = map_type->getNestedType(); + else + break; + } const auto * tuple_data_type = typeid_cast(result_type.get()); if (!tuple_data_type) @@ -3687,11 +3693,11 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveQualifiedMatcher(Qu if (!matcher_node_typed.isMatchingColumn(element_name)) continue; - auto tuple_element_function = std::make_shared("tupleElement"); - tuple_element_function->getArguments().getNodes().push_back(expression_query_tree_node); - tuple_element_function->getArguments().getNodes().push_back(std::make_shared(element_name)); + auto get_subcolumn_function = std::make_shared("getSubcolumn"); + get_subcolumn_function->getArguments().getNodes().push_back(expression_query_tree_node); + get_subcolumn_function->getArguments().getNodes().push_back(std::make_shared(element_name)); - QueryTreeNodePtr function_query_node = tuple_element_function; + QueryTreeNodePtr function_query_node = get_subcolumn_function; resolveFunction(function_query_node, scope); qualified_matcher_element_identifier.push_back(element_name); diff --git a/tests/queries/0_stateless/02677_analyzer_compound_expressions.reference b/tests/queries/0_stateless/02677_analyzer_compound_expressions.reference index b630a020fb9..96c10398c90 100644 --- a/tests/queries/0_stateless/02677_analyzer_compound_expressions.reference +++ b/tests/queries/0_stateless/02677_analyzer_compound_expressions.reference @@ -10,6 +10,7 @@ a b 3 [2,0,1] ['foo','bar'] [1,2] ['foo','bar'] [1,2] +['foo','bar'] [1,2] ['foo','bar'] [(1,2),(3,4)] [1,3] [2,4] ['foo','bar'] [(1,2),(3,4)] [1,3] [2,4] ['foo','bar'] [(1,2),(3,4)] [1,3] [2,4] diff --git a/tests/queries/0_stateless/02677_analyzer_compound_expressions.sql b/tests/queries/0_stateless/02677_analyzer_compound_expressions.sql index 50df99bef57..6b7fdab8993 100644 --- a/tests/queries/0_stateless/02677_analyzer_compound_expressions.sql +++ b/tests/queries/0_stateless/02677_analyzer_compound_expressions.sql @@ -23,6 +23,7 @@ WITH materialize([[1, 2], [], [3]]) AS arr SELECT arr.size0, arr.size1; WITH map('foo', 1, 'bar', 2) AS m SELECT m.keys, m.values; WITH materialize(map('foo', 1, 'bar', 2)) AS m SELECT m.keys, m.values; +WITH map('foo', 1, 'bar', 2) AS m SELECT m.*; WITH map('foo', (1, 2), 'bar', (3, 4))::Map(String, Tuple(a UInt64, b UInt64)) AS m SELECT m.keys, m.values, m.values.a, m.values.b; @@ -37,7 +38,6 @@ WITH materialize(map('foo', (1, 2), 'bar', (3, 4))::Map(String, Tuple(a UInt64, SELECT m.keys, m.values, m.values.*; WITH [1, 2, 3] AS arr SELECT arr.*; -- { serverError UNSUPPORTED_METHOD } -WITH map('foo', 1, 'bar', 2) AS m SELECT m.*; -- { serverError UNSUPPORTED_METHOD } SELECT getSubcolumn([1, 2, 3], 'size0'); SELECT getSubcolumn([1, 2, 3], materialize('size0')); -- { serverError ILLEGAL_COLUMN } From d621b2c4ad07cc0b348cc7e1a48a14275cc6823d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 16 Mar 2023 21:28:07 +0100 Subject: [PATCH 203/559] fix intersecting parts, better fault injections --- docker/test/stress/run.sh | 3 ++ src/Common/CurrentMemoryTracker.cpp | 6 +++ src/Common/CurrentMemoryTracker.h | 3 ++ src/Common/MemoryTracker.cpp | 21 ++++++++ src/Common/MemoryTracker.h | 2 + src/Common/ThreadFuzzer.cpp | 20 +++++++- src/Common/ThreadFuzzer.h | 4 ++ src/Storages/MergeTree/IMergeTreeDataPart.h | 4 ++ src/Storages/MergeTree/MergeTreeData.cpp | 3 ++ .../ReplicatedMergeTreePartCheckThread.cpp | 7 +++ src/Storages/StorageReplicatedMergeTree.cpp | 13 +++-- ...70_lost_part_intersecting_merges.reference | 8 +++ .../02370_lost_part_intersecting_merges.sh | 49 +++++++++++++++++++ 13 files changed, 138 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02370_lost_part_intersecting_merges.reference create mode 100755 tests/queries/0_stateless/02370_lost_part_intersecting_merges.sh diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 15f58d6c3a3..a4871324c48 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -41,6 +41,9 @@ if [ "$is_tsan_build" -eq "0" ]; then export THREAD_FUZZER_pthread_mutex_lock_AFTER_SLEEP_TIME_US=10000 export THREAD_FUZZER_pthread_mutex_unlock_BEFORE_SLEEP_TIME_US=10000 export THREAD_FUZZER_pthread_mutex_unlock_AFTER_SLEEP_TIME_US=10000 + + export THREAD_FUZZER_EXPLICIT_SLEEP_PROBABILITY=0.01 + export THREAD_FUZZER_EXPLICIT_MEMORY_EXCEPTION_PROBABILITY=0.01 fi export ZOOKEEPER_FAULT_INJECTION=1 diff --git a/src/Common/CurrentMemoryTracker.cpp b/src/Common/CurrentMemoryTracker.cpp index 720df07efb9..78a6c65f43c 100644 --- a/src/Common/CurrentMemoryTracker.cpp +++ b/src/Common/CurrentMemoryTracker.cpp @@ -118,3 +118,9 @@ void CurrentMemoryTracker::free(Int64 size) } } +void CurrentMemoryTracker::injectFault() +{ + if (auto * memory_tracker = getMemoryTracker()) + memory_tracker->injectFault(); +} + diff --git a/src/Common/CurrentMemoryTracker.h b/src/Common/CurrentMemoryTracker.h index e125e4cbe4a..2721d89d564 100644 --- a/src/Common/CurrentMemoryTracker.h +++ b/src/Common/CurrentMemoryTracker.h @@ -14,6 +14,9 @@ struct CurrentMemoryTracker static void free(Int64 size); static void check(); + /// Throws MEMORY_LIMIT_EXCEEDED (if it's allowed to throw exceptions) + static void injectFault(); + private: static void allocImpl(Int64 size, bool throw_if_memory_exceeded); }; diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 6c6aa7c6faf..e2129e1013e 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -134,6 +134,27 @@ void MemoryTracker::logMemoryUsage(Int64 current) const "Current memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(current)); } +void MemoryTracker::injectFault() const +{ + if (!memoryTrackerCanThrow(level, true)) + { + LOG_WARNING(&Poco::Logger::get("MemoryTracker"), + "Cannot inject fault at specific point. Uncaught exceptions: {}, stack trace:\n{}", + std::uncaught_exceptions(), StackTrace().toString()); + return; + } + + /// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc + MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); + + ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded); + const auto * description = description_ptr.load(std::memory_order_relaxed); + throw DB::Exception( + DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, + "Memory tracker{}{}: fault injected (at specific point)", + description ? " " : "", + description ? description : ""); +} void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker) { diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index f6113d31423..66b56730b75 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -141,6 +141,8 @@ public: fault_probability = value; } + void injectFault() const; + void setSampleProbability(double value) { sample_probability = value; diff --git a/src/Common/ThreadFuzzer.cpp b/src/Common/ThreadFuzzer.cpp index df6f860e588..fbdc8ad46ac 100644 --- a/src/Common/ThreadFuzzer.cpp +++ b/src/Common/ThreadFuzzer.cpp @@ -109,6 +109,8 @@ void ThreadFuzzer::initConfiguration() initFromEnv(migrate_probability, "THREAD_FUZZER_MIGRATE_PROBABILITY"); initFromEnv(sleep_probability, "THREAD_FUZZER_SLEEP_PROBABILITY"); initFromEnv(sleep_time_us, "THREAD_FUZZER_SLEEP_TIME_US"); + initFromEnv(explicit_sleep_probability, "THREAD_FUZZER_EXPLICIT_SLEEP_PROBABILITY"); + initFromEnv(explicit_memory_exception_probability, "THREAD_FUZZER_EXPLICIT_MEMORY_EXCEPTION_PROBABILITY"); #if THREAD_FUZZER_WRAP_PTHREAD # define INIT_WRAPPER_PARAMS(RET, NAME, ...) \ @@ -225,14 +227,28 @@ static void injection( void ThreadFuzzer::maybeInjectSleep() { auto & fuzzer = ThreadFuzzer::instance(); - injection(fuzzer.yield_probability, fuzzer.migrate_probability, fuzzer.sleep_probability, fuzzer.sleep_time_us); + injection(fuzzer.yield_probability, fuzzer.migrate_probability, fuzzer.explicit_sleep_probability, fuzzer.sleep_time_us); +} + +/// Sometimes maybeInjectSleep() is not enough and we need to inject an exception. +/// The most suitable exception for this purpose is MEMORY_LIMIT_EXCEEDED: it can be thrown almost from everywhere. +/// NOTE We also have a query setting fault_probability, but it does not work for background operations (maybe we should fix it). +void ThreadFuzzer::maybeInjectMemoryLimitException() +{ + auto & fuzzer = ThreadFuzzer::instance(); + if (fuzzer.explicit_memory_exception_probability <= 0.0) + return; + std::bernoulli_distribution fault(fuzzer.explicit_memory_exception_probability); + if (fault(thread_local_rng)) + CurrentMemoryTracker::injectFault(); } void ThreadFuzzer::signalHandler(int) { DENY_ALLOCATIONS_IN_SCOPE; auto saved_errno = errno; - maybeInjectSleep(); + auto & fuzzer = ThreadFuzzer::instance(); + injection(fuzzer.yield_probability, fuzzer.migrate_probability, fuzzer.sleep_probability, fuzzer.sleep_time_us); errno = saved_errno; } diff --git a/src/Common/ThreadFuzzer.h b/src/Common/ThreadFuzzer.h index ff391dfcd8f..9dd55fe7995 100644 --- a/src/Common/ThreadFuzzer.h +++ b/src/Common/ThreadFuzzer.h @@ -59,12 +59,16 @@ public: static bool isStarted(); static void maybeInjectSleep(); + static void maybeInjectMemoryLimitException(); + private: uint64_t cpu_time_period_us = 0; double yield_probability = 0; double migrate_probability = 0; double sleep_probability = 0; double sleep_time_us = 0; + double explicit_sleep_probability = 0; + double explicit_memory_exception_probability = 0; inline static std::atomic started{true}; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index ea1fd209a20..4403f79dfaa 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -220,6 +220,10 @@ public: /// Frozen by ALTER TABLE ... FREEZE ... It is used for information purposes in system.parts table. mutable std::atomic is_frozen {false}; + /// Indicated that the part was marked Outdated because it's broken, not because it's actually outdated + /// See outdateBrokenPartAndCloneToDetached(...) + mutable bool outdated_because_broken = false; + /// Flag for keep S3 data when zero-copy replication over S3 turned on. mutable bool force_keep_shared_data = false; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f9424f334db..30ab9037436 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3836,7 +3836,10 @@ void MergeTreeData::outdateBrokenPartAndCloneToDetached(const DataPartPtr & part DataPartsLock lock = lockParts(); if (part_to_detach->getState() == DataPartState::Active) + { + part_to_detach->outdated_because_broken = true; removePartsFromWorkingSet(NO_TRANSACTION_RAW, {part_to_detach}, true, &lock); + } } void MergeTreeData::forcefullyMovePartToDetachedAndRemoveFromMemory(const MergeTreeData::DataPartPtr & part_to_detach, const String & prefix, bool restore_covered) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index 85933dbafad..3120205b4e3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include @@ -263,6 +264,8 @@ void ReplicatedMergeTreePartCheckThread::searchForMissingPartAndFetchIfPossible( } } + ThreadFuzzer::maybeInjectSleep(); + if (storage.createEmptyPartInsteadOfLost(zookeeper, part_name)) { /** This situation is possible if on all the replicas where the part was, it deteriorated. @@ -383,6 +386,9 @@ CheckResult ReplicatedMergeTreePartCheckThread::checkPart(const String & part_na /// Delete part locally. storage.outdateBrokenPartAndCloneToDetached(part, "broken"); + ThreadFuzzer::maybeInjectMemoryLimitException(); + ThreadFuzzer::maybeInjectSleep(); + /// Part is broken, let's try to find it and fetch. searchForMissingPartAndFetchIfPossible(part_name, exists_in_zookeeper); @@ -399,6 +405,7 @@ CheckResult ReplicatedMergeTreePartCheckThread::checkPart(const String & part_na String message = fmt::format(fmt_string, part_name); LOG_ERROR(log, fmt_string, part_name); storage.outdateBrokenPartAndCloneToDetached(part, "unexpected"); + ThreadFuzzer::maybeInjectSleep(); return {part_name, false, message}; } else diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0a281adb200..229495a9a06 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include @@ -3508,6 +3509,8 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n part->makeCloneInDetached("covered-by-broken", getInMemoryMetadataPtr()); } + ThreadFuzzer::maybeInjectSleep(); + /// It's possible that queue contains entries covered by part_name. /// For example, we had GET_PART all_1_42_5 and MUTATE_PART all_1_42_5_63, /// then all_1_42_5_63 was executed by fetching, but part was written to disk incorrectly. @@ -3520,6 +3523,8 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n /// because we may have some covered parts (more precisely, parts with the same min and max blocks) queue.removePartProducingOpsInRange(zookeeper, broken_part_info, /* covering_entry= */ {}); + ThreadFuzzer::maybeInjectSleep(); + String part_path = fs::path(replica_path) / "parts" / part_name; while (true) @@ -6513,10 +6518,12 @@ void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() for (const auto & part : parts) { - if (!part->is_duplicate) - parts_to_delete_completely.emplace_back(part); - else + /// Broken part can be removed from zk by removePartAndEnqueueFetch(...) only. + /// Removal without enqueueing a fetch leads to intersecting parts. + if (part->is_duplicate || part->outdated_because_broken) parts_to_delete_only_from_filesystem.emplace_back(part); + else + parts_to_delete_completely.emplace_back(part); } parts.clear(); diff --git a/tests/queries/0_stateless/02370_lost_part_intersecting_merges.reference b/tests/queries/0_stateless/02370_lost_part_intersecting_merges.reference new file mode 100644 index 00000000000..bc44a664c04 --- /dev/null +++ b/tests/queries/0_stateless/02370_lost_part_intersecting_merges.reference @@ -0,0 +1,8 @@ +1 0 all_0_0_0 +1 1 all_1_2_1 +1 2 all_1_2_1 +0 +3 0 all_0_3_2 +3 1 all_0_3_2 +3 2 all_0_3_2 +3 3 all_0_3_2 diff --git a/tests/queries/0_stateless/02370_lost_part_intersecting_merges.sh b/tests/queries/0_stateless/02370_lost_part_intersecting_merges.sh new file mode 100755 index 00000000000..f55eb4d74a9 --- /dev/null +++ b/tests/queries/0_stateless/02370_lost_part_intersecting_merges.sh @@ -0,0 +1,49 @@ +#!/usr/bin/env bash +# Tags: long, zookeeper + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "drop table if exists rmt1 sync;" +$CLICKHOUSE_CLIENT -q "drop table if exists rmt2 sync;" + +$CLICKHOUSE_CLIENT -q "create table rmt1 (n int) engine=ReplicatedMergeTree('/test/02369/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{database}', '1') order by n + settings cleanup_delay_period=0, cleanup_delay_period_random_add=0, old_parts_lifetime=0" +$CLICKHOUSE_CLIENT -q "create table rmt2 (n int) engine=ReplicatedMergeTree('/test/02369/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{database}', '2') order by n" + +$CLICKHOUSE_CLIENT -q "system stop replicated sends rmt2" +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into rmt2 values (0);" + +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into rmt1 values (1);" +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into rmt1 values (2);" + +$CLICKHOUSE_CLIENT --receive_timeout=3 -q "system sync replica rmt1;" 2>/dev/null 1>/dev/null +$CLICKHOUSE_CLIENT --optimize_throw_if_noop=1 -q "optimize table rmt1;" +$CLICKHOUSE_CLIENT -q "system start replicated sends rmt2" +$CLICKHOUSE_CLIENT -q "system sync replica rmt1;" + +$CLICKHOUSE_CLIENT -q "select 1, *, _part from rmt1 order by n;" + +path=$($CLICKHOUSE_CLIENT -q "select path from system.parts where database='$CLICKHOUSE_DATABASE' and table='rmt1' and name='all_1_2_1'") +# ensure that path is absolute before removing +$CLICKHOUSE_CLIENT -q "select throwIf(substring('$path', 1, 1) != '/', 'Path is relative: $path')" || exit +rm -rf $path + +$CLICKHOUSE_CLIENT -q "select * from rmt1;" 2>&1 | grep LOGICAL_ERROR +$CLICKHOUSE_CLIENT --min_bytes_to_use_direct_io=1 --local_filesystem_read_method=pread_threadpool -q "select * from rmt1;" 2>&1 | grep LOGICAL_ERROR + +$CLICKHOUSE_CLIENT -q "select sleep(0.1) from numbers($(($RANDOM % 30))) settings max_block_size=1 format Null" + +$CLICKHOUSE_CLIENT -q "detach table rmt1;" +$CLICKHOUSE_CLIENT -q "attach table rmt1;" + +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into rmt1 values (3);" +$CLICKHOUSE_CLIENT -q "system sync replica rmt1;" +$CLICKHOUSE_CLIENT -q "optimize table rmt1 final;" + +$CLICKHOUSE_CLIENT -q "system sync replica rmt1;" +$CLICKHOUSE_CLIENT -q "select 3, *, _part from rmt1 order by n;" + +$CLICKHOUSE_CLIENT -q "drop table rmt1 sync;" +$CLICKHOUSE_CLIENT -q "drop table rmt2 sync;" From e536c80274f343685690b455f3e27dea3602fe89 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 16 Mar 2023 20:22:25 +0000 Subject: [PATCH 204/559] add test with array of tuples --- src/Functions/getSubcolumn.cpp | 10 +++++++++- .../02677_get_subcolumn_array_of_tuples.reference | 3 +++ .../02677_get_subcolumn_array_of_tuples.sql | 13 +++++++++++++ 3 files changed, 25 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02677_get_subcolumn_array_of_tuples.reference create mode 100644 tests/queries/0_stateless/02677_get_subcolumn_array_of_tuples.sql diff --git a/src/Functions/getSubcolumn.cpp b/src/Functions/getSubcolumn.cpp index 268e5af79be..a8998056d33 100644 --- a/src/Functions/getSubcolumn.cpp +++ b/src/Functions/getSubcolumn.cpp @@ -53,7 +53,15 @@ private: REGISTER_FUNCTION(GetSubcolumn) { - factory.registerFunction(); + factory.registerFunction({ + R"( +Receives the expression or identifier and constant string with the name of subcolumn. + +Returns requested subcolumn extracted from the expression. +)", + Documentation::Examples{{"getSubcolumn", "SELECT getSubcolumn(array_col, 'size0'), getSubcolumn(tuple_col, 'elem_name')"}}, + Documentation::Categories{"OtherFunctions"} + }); } } diff --git a/tests/queries/0_stateless/02677_get_subcolumn_array_of_tuples.reference b/tests/queries/0_stateless/02677_get_subcolumn_array_of_tuples.reference new file mode 100644 index 00000000000..f2700561f4e --- /dev/null +++ b/tests/queries/0_stateless/02677_get_subcolumn_array_of_tuples.reference @@ -0,0 +1,3 @@ +[42] +['foo'] +1 diff --git a/tests/queries/0_stateless/02677_get_subcolumn_array_of_tuples.sql b/tests/queries/0_stateless/02677_get_subcolumn_array_of_tuples.sql new file mode 100644 index 00000000000..5779821afaa --- /dev/null +++ b/tests/queries/0_stateless/02677_get_subcolumn_array_of_tuples.sql @@ -0,0 +1,13 @@ +SET allow_experimental_analyzer = 1; + +DROP TABLE IF EXISTS t_get_subcolumn; + +CREATE TABLE t_get_subcolumn (id UInt64, n Nested(u UInt64, s String)) ENGINE = MergeTree ORDER BY id; + +INSERT INTO t_get_subcolumn VALUES (1, [42], ['foo']); + +SELECT getSubcolumn(n, 'u') FROM t_get_subcolumn; +SELECT getSubcolumn(n, 's') FROM t_get_subcolumn; +SELECT getSubcolumn(n, 'size0') FROM t_get_subcolumn; + +DROP TABLE t_get_subcolumn; From 83799c52aed3eb79c42f227fbe4adb9dbdf651e5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 16 Mar 2023 23:14:12 +0100 Subject: [PATCH 205/559] Add another bug --- tests/queries/0_stateless/02686_bson3.sql | Bin 0 -> 21293 bytes 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/queries/0_stateless/02686_bson3.sql diff --git a/tests/queries/0_stateless/02686_bson3.sql b/tests/queries/0_stateless/02686_bson3.sql new file mode 100644 index 0000000000000000000000000000000000000000..07ed206e94b2179aa6e9c4da12dfe964d8e534f1 GIT binary patch literal 21293 zcmeI4ZEPGz8OOJjBf>PWv1q~vDs5@%G{MfsXXAiMV0|~a+xl+z zvb$%;w1rYsiINmhBicxm3IwY7L{*@PcqvrnCF+NQ0YMO@h$g&>wn60sQc)?-GcP-{ z=Zlh9j_knXgR}FWmz$k=W_~*}&u&|rl?|=D=ldidmKUwCbCa83Dw^~z92!UMaJ8rSn+Ut3R%sg>J5lsVAPK8*5c1ZA|)uI{( zsn%+xy_y>mpZ!Xz6NIt@MPw(i5DZU}BHV?HpAJN7tP{_kF55vMes(-3t}yLDQK`}N zWO-Wn#rkxUB=E{L?3)>< z5Gp?>C;joI~P9fo2lqeWx16^=vTe%{tO}SZJc;xsg37(x_J;<*3pVwr^L0D43mqS}%D1 zoHUCm-#+N%b6(BQ6WLQaV*B}0#w|MTGIgTl+kv#+fx>)!x@<7uhYl2Lmqi=TIBvmP zU^W`-JvAcUe=IRt{|A*@Won}ll5@)Q#4n=>yr1|1m`R8D_2p5W=Z{KZ6poj)gD zQ4(m?=c2962m5voLYO48~_X_zxzHK>pmMH|vEik+aVvsdyog^A|IW zF`Jcv{V`r8F7Z`m*ai;t zLOaY6e;x{^eIyDJacR3!wVfg&sXl6;&_ImlB$0Z0YLsbFeo#h9!KqbLMf^@o1B*wT zDMd{YUuGNfy~>nbjN)M+>`jlz`{U#i;{zKFMx+`PsZ*WXTAQ|RJ+Q$rn)!2pk>9Z^ zX^qk(4Z{>PbHF^x$jd;~DAdk+Npu+I(}n=n@J z-}w4ZK6=s=>@@aX4TT|vAl+%}fzS7KpMG9srS!+54Rbul{*QP?g9*4vdSN7Z3$TzG zF`wmHM*Nr@)df`(ij=fx=|GC7k?5wZ5T(UAU$0WiGf^^L4I_bRI3bDONd?1%;X}F2 z(qkw}ZANV(QzWED)4dV%o%_L)yyxaR zb{m^tegSPAD!Clptd!*utiu}QNS!Z77ma|Xi;ztP96%AUSzd-~aV=mi!V#*e(9A(i z1NbMQpqhy$fSO)GV@ga5g^p$sp#6%7WiDFe{V zvICge1;etq=1^wA2+2IifovQY%>KxfIHrARN*J?n48>GR)PdN5nTQS&5y44J?Gy*k zHHjFDX)O${F%mP$j5tigVB&(6EF3YI1!DweQUm_RBO>}TSz#{|RJcaurBS@T3{+@y zR5hAu4^!$g0sgYl%(=@{CJE}Y78@IWr&)(xnm8fAsz8^QB4PvzxgCOya!w;~>(UC` zayf#n3^UZquqa(o7-pq_kmVtcSQat`U}>;f!K<eB4a zBPJr2WslMIORo<7_Ls*x53P)@?$$qQy22DJk*m()QoTax%BMTAW~Sz1DlnyjrYma~ zP1le;O99>3;+L7n1?YO}xTY&Cd6^BZ6S^M1->zu7vOY0Zldk3K|N7vaUwhy1%INyT z?V7GIPBdM`)2?2-Hd^iKwX59jy6X{5*Y!qMJVnxUy$p0seB;#T`j?K*tcJ!>59j-bj-j3fO0m=B)v0yW$z3h*`S=eZu;9RpUSk1EUt{M z`(D1ZbJy=2$$mthyKVz}66dam9(rltmO@XW@ww|8Af-HGG~fmddhXhW7Fi0zYF{#0 z0#A&BQ+A!XoqjFp*Mh*YlkK~1ua!I7J5ntbbo%LUU-d&;29@YkTsd$Q{todYN8#-e zKXi<%+IUM`p#j5F}!&~=tUl0W~taoaJf&mTfIHqR##`2)tO3JKmWwi6H6zTzQ6S3yXWFgBmt;b6!g}-*>}J+RYhFXF1LP58a%<_m$OT^Ok!zOlvo1{)!RU z7+M*9H;OHjwVN|-6I?LExdFaSM8m7Lg5KX&`sQnI+HWarKCZI)>F>7v-n=&b;gzwu zC$+Y0?!(67E9eWbd7!;L)w8=bwYzus_M0ugw!LL`L~|L}$N18+p}8C?PfPmsBQ^`I ztJja39@SjdYv%uvHS_#a^H)9l)P1KEE`L_#@{7Ot_*;*fTW(t!mrq-l#^o;-H&4qo z^Pgbwi#78ngNGIu|MJg+jn~YrkWyJQM?2dcIDK#~hwu@IAeotC-|cV~?HfCXS^~eH zM+art#fG=RokeoFaEz6SEtBaMHVG!x?|xt+b?v;iTi9=g%3S0!nby1RZQ#rEzS4<@ zuhYw9v=Y5ct`Oq`53DAR@4mW!PA`-7GI@>O1}+`{)~0KFzc8raIQBO1bIHs-zuWiR z9V^4}uRHiqIqyO9+i_aa-`c=0Y9Y@Ib3K1+ Date: Fri, 17 Mar 2023 00:13:40 +0100 Subject: [PATCH 206/559] Add a test --- .../queries/0_stateless/02686_bson3.reference | 0 tests/queries/0_stateless/02686_bson3.sql | Bin 21293 -> 21327 bytes 2 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/queries/0_stateless/02686_bson3.reference diff --git a/tests/queries/0_stateless/02686_bson3.reference b/tests/queries/0_stateless/02686_bson3.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02686_bson3.sql b/tests/queries/0_stateless/02686_bson3.sql index 07ed206e94b2179aa6e9c4da12dfe964d8e534f1..05a73e814dd103f30981e7232085569f1192d3a9 100644 GIT binary patch delta 45 zcmZ3xjPd+3#tk~b$_l!=3e^h5sYPX}MXp6f`9%tze$M_uL9Wgr@h*-bjtaG00B>Oq A+yDRo delta 11 ScmX@VjB)KU#tk~bj9dU7k^}_+ From 31b46df85c65170ed452fd8148bdc84419f119bc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Mar 2023 00:13:52 +0100 Subject: [PATCH 207/559] Fix an error --- src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp index 64b50fcd771..faf829aa38c 100644 --- a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp @@ -282,6 +282,8 @@ static void readAndInsertString(ReadBuffer & in, IColumn & column, BSONType bson if (bson_type == BSONType::STRING || bson_type == BSONType::SYMBOL || bson_type == BSONType::JAVA_SCRIPT_CODE) { auto size = readBSONSize(in); + if (size == 0) + throw Exception(ErrorCodes::INCORRECT_DATA, "Incorrect size of a string (zero) in BSON"); readAndInsertStringImpl(in, column, size - 1); assertChar(0, in); } From 651aba0d1be813696e42cd573e2b315d86ff938b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Mar 2023 00:19:49 +0100 Subject: [PATCH 208/559] Slightly better --- src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp index faf829aa38c..2702d7fe61c 100644 --- a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp @@ -1009,7 +1009,7 @@ fileSegmentationEngineBSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t size_t old_size = memory.size(); memory.resize(old_size + document_size); - memcpy(memory.data() + old_size, reinterpret_cast(&document_size), sizeof(document_size)); + unalignedStore(memory.data() + old_size, document_size); in.readStrict(memory.data() + old_size + sizeof(document_size), document_size - sizeof(document_size)); ++number_of_rows; } From fc1e7c6e06f6c986c7afefeb3dd3db7c93ce372d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Mar 2023 03:14:38 +0100 Subject: [PATCH 209/559] Add a bug --- .../0_stateless/02687_native_fuzz.reference | 0 tests/queries/0_stateless/02687_native_fuzz.sql | Bin 0 -> 630 bytes 2 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/queries/0_stateless/02687_native_fuzz.reference create mode 100644 tests/queries/0_stateless/02687_native_fuzz.sql diff --git a/tests/queries/0_stateless/02687_native_fuzz.reference b/tests/queries/0_stateless/02687_native_fuzz.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02687_native_fuzz.sql b/tests/queries/0_stateless/02687_native_fuzz.sql new file mode 100644 index 0000000000000000000000000000000000000000..0cd11390918a283d863e3c28bf9825513f69dbc0 GIT binary patch literal 630 zcmb_Y%Syvg5RKpi0$nHwE(a{sMjTl1vzdF&Ju`D=bRE(Rwvdu+_BMbWDb}h$>o%>hD}2DB zSY>eicL|h`u?DM5SOQT71W&B#9tGt6sA7izJx|!kLXN9y&d3-E3;JNv%+5er$O>}L zEV3Jr2C;c$+es%O?epk}2Gbaz;bYOGkQp4+V9uIJpyhIn$+2|(Werk7l#GCUZfNSj zaw`N$WC|Y|)VUDWISmqG)n}@FG?|p5V&U=!p2BJ>k$H0x&NZn_l4Ol`8s_HA%vboS zj%jK%szj8Ne+os}J&0Z3J?Hzs=eBnTE%j?%ZVf+O6$kcz%ZDARxP5q}mz~H>_w7x(=IAB^i)3q!bdY XY_^bzn<=9ZH_iA$A(vh>U|W3!ub=3Z literal 0 HcmV?d00001 From 6275c472a79a70b568b08b83af2b740d2c351f7a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Mar 2023 03:14:49 +0100 Subject: [PATCH 210/559] Better exceptions --- src/Formats/NativeReader.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Formats/NativeReader.cpp b/src/Formats/NativeReader.cpp index eca88a41c13..aff479b5320 100644 --- a/src/Formats/NativeReader.cpp +++ b/src/Formats/NativeReader.cpp @@ -136,6 +136,11 @@ Block NativeReader::read() { readVarUInt(columns, istr); readVarUInt(rows, istr); + + if (columns > 1'000'000uz) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Suspiciously many columns in Native format: {}", columns); + if (rows > 1'000'000'000'000uz) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Suspiciously many rows in Native format: {}", rows); } else { From 1abe5ea58ece77b4ebabde9013b81658c3142cea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Mar 2023 04:44:14 +0100 Subject: [PATCH 211/559] Add data type fuzzer --- src/DataTypes/CMakeLists.txt | 4 + src/DataTypes/fuzzers/CMakeLists.txt | 2 + .../data_type_deserialization_fuzzer.cpp | 74 +++++++++++++++++++ src/Formats/FormatSettings.h | 5 +- src/Formats/fuzzers/format_fuzzer.cpp | 11 +++ 5 files changed, 94 insertions(+), 2 deletions(-) create mode 100644 src/DataTypes/fuzzers/CMakeLists.txt create mode 100644 src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp diff --git a/src/DataTypes/CMakeLists.txt b/src/DataTypes/CMakeLists.txt index 4a60d6c54cf..2902ef0a955 100644 --- a/src/DataTypes/CMakeLists.txt +++ b/src/DataTypes/CMakeLists.txt @@ -3,3 +3,7 @@ add_subdirectory (Serializations) if (ENABLE_EXAMPLES) add_subdirectory (examples) endif () + +if (ENABLE_FUZZING) + add_subdirectory(fuzzers) +endif() diff --git a/src/DataTypes/fuzzers/CMakeLists.txt b/src/DataTypes/fuzzers/CMakeLists.txt new file mode 100644 index 00000000000..d9c19cb7d01 --- /dev/null +++ b/src/DataTypes/fuzzers/CMakeLists.txt @@ -0,0 +1,2 @@ +clickhouse_add_executable(data_type_deserialization_fuzzer data_type_deserialization_fuzzer.cpp ${SRCS}) +target_link_libraries(data_type_deserialization_fuzzer PRIVATE dbms clickhouse_aggregate_functions ${LIB_FUZZING_ENGINE}) diff --git a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp new file mode 100644 index 00000000000..76b8c9c0171 --- /dev/null +++ b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp @@ -0,0 +1,74 @@ +#include + +#include +#include + +#include + +#include + +#include + +#include + + +extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) +try +{ + using namespace DB; + + static SharedContextHolder shared_context; + static ContextMutablePtr context; + + auto initialize = [&]() mutable + { + shared_context = Context::createShared(); + context = Context::createGlobal(shared_context.get()); + context->makeGlobalContext(); + context->setApplicationType(Context::ApplicationType::LOCAL); + + registerAggregateFunctions(); + return true; + }; + + static bool initialized = initialize(); + (void) initialized; + + /// The input format is as follows: + /// - data type name on the first line, + /// - the data for the rest of the input. + + /// Compile the code as follows: + /// mkdir build_asan_fuzz + /// cd build_asan_fuzz + /// CC=clang CXX=clang++ cmake -D SANITIZE=address -D ENABLE_FUZZING=1 -D WITH_COVERAGE=1 .. + /// + /// The corpus is located here: + /// https://github.com/ClickHouse/fuzz-corpus/tree/main/data_type_deserialization + /// + /// The fuzzer can be run as follows: + /// ../../../build_asan_fuzz/src/DataTypes/fuzzers/data_type_deserialization_fuzzer corpus -jobs=64 -rss_limit_mb=8192 + + /// clickhouse-local --query "SELECT toJSONString(*) FROM (SELECT name FROM system.functions UNION ALL SELECT name FROM system.data_type_families)" > dictionary + + DB::ReadBufferFromMemory in(data, size); + + String data_type; + readStringUntilNewlineInto(data_type, in); + assertChar('\n', in); + + DataTypePtr type = DataTypeFactory::instance().get(data_type); + + FormatSettings settings; + settings.max_binary_string_size = 100; + settings.max_binary_array_size = 100; + + Field field; + type->getDefaultSerialization()->deserializeBinary(field, in, settings); + + return 0; +} +catch (...) +{ + return 1; +} diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index ea5b34e4255..053b461244b 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -81,8 +82,8 @@ struct FormatSettings UInt64 input_allow_errors_num = 0; Float32 input_allow_errors_ratio = 0; - UInt64 max_binary_string_size = 0; - UInt64 max_binary_array_size = 0; + UInt64 max_binary_string_size = 1_GiB; + UInt64 max_binary_array_size = 1_GiB; UInt64 client_protocol_version = 0; UInt64 max_parser_depth = DBMS_DEFAULT_MAX_PARSER_DEPTH; diff --git a/src/Formats/fuzzers/format_fuzzer.cpp b/src/Formats/fuzzers/format_fuzzer.cpp index 81aaac88691..9983d493e45 100644 --- a/src/Formats/fuzzers/format_fuzzer.cpp +++ b/src/Formats/fuzzers/format_fuzzer.cpp @@ -62,6 +62,17 @@ try done */ + /** And: + + for format in $(clickhouse-client --query "SELECT name FROM system.formats WHERE is_output"); do + echo $format; + echo $format >> $format; + echo "WatchID Int64, JavaEnable Int16, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID Int32, ClientIP Int32, RegionID Int32, UserID Int64, CounterClass Int16, OS Int16, UserAgent Int16, URL String, Referer String, IsRefresh Int16, RefererCategoryID Int16, RefererRegionID Int32, URLCategoryID Int16, URLRegionID Int32, ResolutionWidth Int16, ResolutionHeight Int16, ResolutionDepth Int16, FlashMajor Int16, FlashMinor Int16, FlashMinor2 String, NetMajor Int16, NetMinor Int16, UserAgentMajor Int16, UserAgentMinor String, CookieEnable Int16, JavascriptEnable Int16, IsMobile Int16, MobilePhone Int16, MobilePhoneModel String, Params String, IPNetworkID Int32, TraficSourceID Int16, SearchEngineID Int16, SearchPhrase String, AdvEngineID Int16, IsArtifical Int16, WindowClientWidth Int16, WindowClientHeight Int16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 Int16, SilverlightVersion2 Int16, SilverlightVersion3 Int32, SilverlightVersion4 Int16, PageCharset String, CodeVersion Int32, IsLink Int16, IsDownload Int16, IsNotBounce Int16, FUniqID Int64, OriginalURL String, HID Int32, IsOldCounter Int16, IsEvent Int16, IsParameter Int16, DontCountHits Int16, WithHash Int16, HitColor String, LocalEventTime DateTime, Age Int16, Sex Int16, Income Int16, Interests Int16, Robotness Int16, RemoteIP Int32, WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage String, BrowserCountry String, SocialNetwork String, SocialAction String, HTTPError Int16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, SocialSourceNetworkID Int16, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency String, ParamCurrencyID Int16, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID Int16, RefererHash Int64, URLHash Int64, CLID Int32" >> $format; + clickhouse-client --query "SELECT * FROM hits LIMIT 10 FORMAT $format" >> $format || rm $format; + done + + */ + /// Compile the code as follows: /// mkdir build_asan_fuzz /// cd build_asan_fuzz From 53a82d445be565d45fc82107d85a4e74c425f13b Mon Sep 17 00:00:00 2001 From: clickhouse-adrianfraguela <119855513+clickhouse-adrianfraguela@users.noreply.github.com> Date: Fri, 17 Mar 2023 09:22:53 +0000 Subject: [PATCH 212/559] Updated events and recordings --- README.md | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index 17b4df154a9..509fd395f58 100644 --- a/README.md +++ b/README.md @@ -21,11 +21,10 @@ curl https://clickhouse.com/ | sh * [Contacts](https://clickhouse.com/company/contact) can help to get your questions answered if there are any. ## Upcoming Events -* [**v23.2 Release Webinar**](https://clickhouse.com/company/events/v23-2-release-webinar?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-02) - Feb 23 - 23.2 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. -* [**ClickHouse Meetup in Amsterdam**](https://www.meetup.com/clickhouse-netherlands-user-group/events/291485868/) - Mar 9 - The first ClickHouse Amsterdam Meetup of 2023 is here! 🎉 Join us for short lightning talks and long discussions. Food, drinks & good times on us. -* [**ClickHouse Meetup in SF Bay Area**](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/291490121/) - Mar 14 - A night to meet with ClickHouse team in the San Francisco area! Food and drink are a given...but networking is the primary focus. * [**ClickHouse Meetup in Austin**](https://www.meetup.com/clickhouse-austin-user-group/events/291486654/) - Mar 16 - The first ClickHouse Meetup in Austin is happening soon! Interested in speaking, let us know! +* [**v23.3 Release Webinar**](https://clickhouse.com/company/events/v23-3-release-webinar?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-02) - Mar 30 - 23.3 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. ## Recent Recordings * **FOSDEM 2023**: In the "Fast and Streaming Data" room Alexey gave a talk entitled "Building Analytical Apps With ClickHouse" that looks at the landscape of data tools, an interesting data set, and how you can interact with data quickly. Check out the recording on **[YouTube](https://www.youtube.com/watch?v=JlcI2Vfz_uk)**. -* **Recording available**: [**v23.1 Release Webinar**](https://www.youtube.com/watch?v=zYSZXBnTMSE) 23.1 is the ClickHouse New Year release. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. Inverted indices, query cache, and so -- very -- much more. +* **Recording available**: [**v23.2 Release Webinar**](https://www.youtube.com/watch?v=2o0vRMMIrkY) NTILE Window Function support, Partition Key for GROUP By, io_uring, Apache Iceberg support, Dynamic Disks, integrations updates! Watch it now! +* **All release webinar recordings**: [YouTube playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3jAlSy1JxyP8zluvXaN3nxU) From df9e247d67d89d0720d98c3ae40be24ca9c3cef8 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 17 Mar 2023 10:49:41 +0100 Subject: [PATCH 213/559] fix tidy build, fix test when Databasereplicated --- src/Interpreters/ThreadStatusExt.cpp | 4 ++-- .../02675_profile_events_from_query_log_and_client.sh | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 11ca3c7a707..bee7f55887f 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -557,7 +557,7 @@ CurrentThread::QueryScope::QueryScope(ContextMutablePtr query_context, std::func query_context->makeQueryContext(); auto group = ThreadGroupStatus::createForQuery(query_context, std::move(fatal_error_callback)); - CurrentThread::attachToGroup(std::move(group)); + CurrentThread::attachToGroup(group); } CurrentThread::QueryScope::QueryScope(ContextPtr query_context, std::function fatal_error_callback) @@ -567,7 +567,7 @@ CurrentThread::QueryScope::QueryScope(ContextPtr query_context, std::function&1 | grep -o -e '\ \[\ .*\ \]\ FileOpen:\ .*\ ' +" echo "INSERT" $CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " From 3e8fea7342a08251283b80c59015b7935e2f9498 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 17 Mar 2023 10:42:48 +0000 Subject: [PATCH 214/559] Enable parallel execution for two tests --- .../01054_cache_dictionary_bunch_update.sh | 20 +++++++------ ...72_window_view_multiple_columns_groupby.sh | 30 ++++++++++--------- 2 files changed, 27 insertions(+), 23 deletions(-) diff --git a/tests/queries/0_stateless/01054_cache_dictionary_bunch_update.sh b/tests/queries/0_stateless/01054_cache_dictionary_bunch_update.sh index 02ea1fa699c..6a67fdf7f22 100755 --- a/tests/queries/0_stateless/01054_cache_dictionary_bunch_update.sh +++ b/tests/queries/0_stateless/01054_cache_dictionary_bunch_update.sh @@ -1,20 +1,22 @@ #!/usr/bin/env bash -# Tags: no-parallel, no-fasttest +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --query="create database if not exists test_01054;" -$CLICKHOUSE_CLIENT --query="drop table if exists test_01054.ints;" +RAND_ID=$(shuf --input-range 10000000-99999999 --head-count=1) -$CLICKHOUSE_CLIENT --query="create table test_01054.ints +$CLICKHOUSE_CLIENT --query="create database if not exists test_$RAND_ID;" +$CLICKHOUSE_CLIENT --query="drop table if exists test_$RAND_ID.ints;" + +$CLICKHOUSE_CLIENT --query="create table test_$RAND_ID.ints (key UInt64, i8 Int8, i16 Int16, i32 Int32, i64 Int64, u8 UInt8, u16 UInt16, u32 UInt32, u64 UInt64) Engine = Memory;" -$CLICKHOUSE_CLIENT --query="insert into test_01054.ints values (1, 1, 1, 1, 1, 1, 1, 1, 1);" -$CLICKHOUSE_CLIENT --query="insert into test_01054.ints values (2, 2, 2, 2, 2, 2, 2, 2, 2);" -$CLICKHOUSE_CLIENT --query="insert into test_01054.ints values (3, 3, 3, 3, 3, 3, 3, 3, 3);" +$CLICKHOUSE_CLIENT --query="insert into test_$RAND_ID.ints values (1, 1, 1, 1, 1, 1, 1, 1, 1);" +$CLICKHOUSE_CLIENT --query="insert into test_$RAND_ID.ints values (2, 2, 2, 2, 2, 2, 2, 2, 2);" +$CLICKHOUSE_CLIENT --query="insert into test_$RAND_ID.ints values (3, 3, 3, 3, 3, 3, 3, 3, 3);" function thread1() { @@ -73,5 +75,5 @@ wait echo OK -$CLICKHOUSE_CLIENT --query "DROP TABLE if exists test_01054.ints" -$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01054" +$CLICKHOUSE_CLIENT --query "DROP TABLE if exists test_$RAND_ID.ints" +$CLICKHOUSE_CLIENT -q "DROP DATABASE test_$RAND_ID" diff --git a/tests/queries/0_stateless/01072_window_view_multiple_columns_groupby.sh b/tests/queries/0_stateless/01072_window_view_multiple_columns_groupby.sh index 3deb16fa439..ad9bd0d8ad7 100755 --- a/tests/queries/0_stateless/01072_window_view_multiple_columns_groupby.sh +++ b/tests/queries/0_stateless/01072_window_view_multiple_columns_groupby.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-random-settings, no-parallel, no-fasttest +# Tags: no-random-settings, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -9,25 +9,27 @@ opts=( "--allow_experimental_analyzer=0" ) +RAND_ID=$(shuf --input-range 10000000-99999999 --head-count=1) + $CLICKHOUSE_CLIENT "${opts[@]}" --multiquery < Date: Mon, 6 Feb 2023 18:49:42 +0300 Subject: [PATCH 215/559] Add classes for replication UDF to ZooKeeper --- programs/server/config.xml | 4 + .../ZooKeeper/ZooKeeperCachingGetter.cpp | 49 ++ src/Common/ZooKeeper/ZooKeeperCachingGetter.h | 30 ++ .../UserDefined/UserDefinedSQLObjectType.cpp | 17 + .../UserDefined/UserDefinedSQLObjectType.h | 2 + ...erDefinedSQLObjectsLoaderFromZooKeeper.cpp | 417 ++++++++++++++++++ ...UserDefinedSQLObjectsLoaderFromZooKeeper.h | 79 ++++ .../createUserDefinedSQLObjectsLoader.cpp | 25 +- 8 files changed, 622 insertions(+), 1 deletion(-) create mode 100644 src/Common/ZooKeeper/ZooKeeperCachingGetter.cpp create mode 100644 src/Common/ZooKeeper/ZooKeeperCachingGetter.h create mode 100644 src/Functions/UserDefined/UserDefinedSQLObjectType.cpp create mode 100644 src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp create mode 100644 src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.h diff --git a/programs/server/config.xml b/programs/server/config.xml index 0ea2de18e22..bd1df3ca035 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1288,6 +1288,10 @@ *_function.xml + + + diff --git a/src/Common/ZooKeeper/ZooKeeperCachingGetter.cpp b/src/Common/ZooKeeper/ZooKeeperCachingGetter.cpp new file mode 100644 index 00000000000..4b30e35a0ee --- /dev/null +++ b/src/Common/ZooKeeper/ZooKeeperCachingGetter.cpp @@ -0,0 +1,49 @@ +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NO_ZOOKEEPER; +} + +} + +namespace zkutil +{ + +ZooKeeperCachingGetter::ZooKeeperCachingGetter(zkutil::GetZooKeeper get_zookeeper_) : get_zookeeper{get_zookeeper_} +{ +} + + +void ZooKeeperCachingGetter::resetCache() +{ + std::lock_guard lock{cached_zookeeper_ptr_mutex}; + cached_zookeeper_ptr = nullptr; +} + + +std::pair ZooKeeperCachingGetter::getZooKeeper() +{ + std::lock_guard lock{cached_zookeeper_ptr_mutex}; + return getZooKeeperNoLock(); +} + + +std::pair ZooKeeperCachingGetter::getZooKeeperNoLock() +{ + if (!cached_zookeeper_ptr || cached_zookeeper_ptr->expired()) + { + auto zookeeper = get_zookeeper(); + if (!zookeeper) + throw DB::Exception(DB::ErrorCodes::NO_ZOOKEEPER, "Can't get ZooKeeper session"); + + cached_zookeeper_ptr = zookeeper; + return {zookeeper, true}; + } + return {cached_zookeeper_ptr, false}; +} + +} diff --git a/src/Common/ZooKeeper/ZooKeeperCachingGetter.h b/src/Common/ZooKeeper/ZooKeeperCachingGetter.h new file mode 100644 index 00000000000..ef845db8e2b --- /dev/null +++ b/src/Common/ZooKeeper/ZooKeeperCachingGetter.h @@ -0,0 +1,30 @@ +#pragma once +#include + + +namespace zkutil +{ + +class ZooKeeperCachingGetter : boost::noncopyable +{ +public: + explicit ZooKeeperCachingGetter(zkutil::GetZooKeeper get_zookeeper_); + + ZooKeeperCachingGetter(const ZooKeeperCachingGetter &) = delete; + ZooKeeperCachingGetter & operator=(const ZooKeeperCachingGetter &) = delete; + + /// Returns the ZooKeeper session and the flag whether it was taken from the cache(false) or opened new(true), + /// because the session has expired or the cache was empty + std::pair getZooKeeper(); + void resetCache(); + +private: + std::pair getZooKeeperNoLock() TSA_REQUIRES(cached_zookeeper_ptr_mutex); + + std::mutex cached_zookeeper_ptr_mutex; + zkutil::ZooKeeperPtr cached_zookeeper_ptr TSA_GUARDED_BY(cached_zookeeper_ptr_mutex); + + zkutil::GetZooKeeper get_zookeeper; +}; + +} diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectType.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectType.cpp new file mode 100644 index 00000000000..47b75fb3291 --- /dev/null +++ b/src/Functions/UserDefined/UserDefinedSQLObjectType.cpp @@ -0,0 +1,17 @@ +#include "UserDefinedSQLObjectType.h" +#include + +namespace DB +{ + +const char * toString(UserDefinedSQLObjectType type) +{ + switch (type) + { + case UserDefinedSQLObjectType::Function: + return "FUNCTION"; + } + UNREACHABLE(); +} + +} diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectType.h b/src/Functions/UserDefined/UserDefinedSQLObjectType.h index f7e6fff5cad..2efab5e7737 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectType.h +++ b/src/Functions/UserDefined/UserDefinedSQLObjectType.h @@ -9,4 +9,6 @@ enum class UserDefinedSQLObjectType Function }; +const char * toString(UserDefinedSQLObjectType type); + } diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp new file mode 100644 index 00000000000..b41fd146753 --- /dev/null +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp @@ -0,0 +1,417 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int FUNCTION_ALREADY_EXISTS; + extern const int UNKNOWN_FUNCTION; + extern const int BAD_ARGUMENTS; +} + +static String getRootNodeName(UserDefinedSQLObjectType object_type) +{ + switch (object_type) + { + case UserDefinedSQLObjectType::Function: + return "functions"; + } + UNREACHABLE(); +} + +static String getRootNodePath(const String & root_path, UserDefinedSQLObjectType object_type) +{ + return root_path + "/" + getRootNodeName(object_type); +} + +static String getNodePath(const String & root_path, UserDefinedSQLObjectType object_type, const String & object_name) +{ + return getRootNodePath(root_path, object_type) + "/" + escapeForFileName(object_name); +} + + +UserDefinedSQLObjectsLoaderFromZooKeeper::UserDefinedSQLObjectsLoaderFromZooKeeper( + const ContextPtr & global_context_, const String & zookeeper_path_) + : global_context{global_context_} + , zookeeper_getter{[global_context_]() { return global_context_->getZooKeeper(); }} + , zookeeper_path{zookeeper_path_} + , watch_queue{std::make_shared>>(std::numeric_limits::max())} + , log{&Poco::Logger::get("UserDefinedSQLObjectsLoaderFromZooKeeper")} +{ + if (zookeeper_path.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "ZooKeeper path must be non-empty"); + + if (zookeeper_path.back() == '/') + zookeeper_path.resize(zookeeper_path.size() - 1); + + /// If zookeeper chroot prefix is used, path should start with '/', because chroot concatenates without it. + if (zookeeper_path.front() != '/') + zookeeper_path = "/" + zookeeper_path; +} + +UserDefinedSQLObjectsLoaderFromZooKeeper::~UserDefinedSQLObjectsLoaderFromZooKeeper() +{ + SCOPE_EXIT_SAFE(stopWatchingThread()); +} + +void UserDefinedSQLObjectsLoaderFromZooKeeper::startWatchingThread() +{ + if (!watching_flag.exchange(true)) + { + watching_thread = ThreadFromGlobalPool(&UserDefinedSQLObjectsLoaderFromZooKeeper::processWatchQueue, this); + } +} + +void UserDefinedSQLObjectsLoaderFromZooKeeper::stopWatchingThread() +{ + if (watching_flag.exchange(false)) + { + watch_queue->finish(); + if (watching_thread.joinable()) + watching_thread.join(); + } +} + +zkutil::ZooKeeperPtr UserDefinedSQLObjectsLoaderFromZooKeeper::getZooKeeper() +{ + auto [zookeeper, is_new_session] = zookeeper_getter.getZooKeeper(); + + if (is_new_session) + { + /// It's possible that we connected to different [Zoo]Keeper instance + /// so we may read a bit stale state. + zookeeper->sync(zookeeper_path); + + createRootNodes(zookeeper); + refreshAllObjects(zookeeper); + } + + return zookeeper; +} + +void UserDefinedSQLObjectsLoaderFromZooKeeper::initZooKeeperIfNeeded() +{ + getZooKeeper(); +} + +void UserDefinedSQLObjectsLoaderFromZooKeeper::resetAfterError() +{ + zookeeper_getter.resetCache(); +} + + +void UserDefinedSQLObjectsLoaderFromZooKeeper::loadObjects() +{ + if (!objects_loaded) + { + reloadObjects(); + } +} + + +void UserDefinedSQLObjectsLoaderFromZooKeeper::processWatchQueue() +{ + LOG_DEBUG(log, "Started watching thread"); + setThreadName("UserDefObjWatch"); + + while (watching_flag) + { + try + { + std::pair watched_object; + + /// Re-initialize ZooKeeper session if expired and refresh objects + initZooKeeperIfNeeded(); + + if (!watch_queue->tryPop(watched_object, /* timeout_ms: */ 10000)) + continue; + + auto zookeeper = getZooKeeper(); + + const auto & [object_type, object_name] = watched_object; + + if (object_name.empty()) + syncObjects(zookeeper, object_type); + else + refreshObject(zookeeper, object_type, object_name); + } + catch (...) + { + tryLogCurrentException(log, "Will try to restart watching thread after error"); + resetAfterError(); + sleepForSeconds(5); + } + } + + LOG_DEBUG(log, "Stopped watching thread"); +} + + +void UserDefinedSQLObjectsLoaderFromZooKeeper::stopWatching() +{ + stopWatchingThread(); +} + + +void UserDefinedSQLObjectsLoaderFromZooKeeper::reloadObjects() +{ + auto zookeeper = getZooKeeper(); + refreshAllObjects(zookeeper); + startWatchingThread(); +} + + +void UserDefinedSQLObjectsLoaderFromZooKeeper::reloadObject(UserDefinedSQLObjectType object_type, const String & object_name) +{ + auto zookeeper = getZooKeeper(); + refreshObject(zookeeper, object_type, object_name); +} + + +void UserDefinedSQLObjectsLoaderFromZooKeeper::createRootNodes(const zkutil::ZooKeeperPtr & zookeeper) +{ + zookeeper->createAncestors(zookeeper_path); + zookeeper->createIfNotExists(zookeeper_path, ""); + zookeeper->createIfNotExists(zookeeper_path + "/functions", ""); +} + +bool UserDefinedSQLObjectsLoaderFromZooKeeper::storeObject( + UserDefinedSQLObjectType object_type, + const String & object_name, + const IAST & create_object_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings &) +{ + String path = getNodePath(zookeeper_path, object_type, object_name); + LOG_DEBUG(log, "Storing user-defined object {} at zk path {}", backQuote(object_name), path); + + WriteBufferFromOwnString create_statement_buf; + formatAST(create_object_query, create_statement_buf, false); + writeChar('\n', create_statement_buf); + String create_statement = create_statement_buf.str(); + + auto zookeeper = getZooKeeper(); + + size_t num_attempts = 10; + while (true) + { + auto code = zookeeper->tryCreate(path, create_statement, zkutil::CreateMode::Persistent); + if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) + throw zkutil::KeeperException(code, path); + + if (code == Coordination::Error::ZNODEEXISTS) + { + if (throw_if_exists) + throw Exception(ErrorCodes::FUNCTION_ALREADY_EXISTS, "User-defined function '{}' already exists", object_name); + else if (!replace_if_exists) + return false; + + code = zookeeper->trySet(path, create_statement); + if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNONODE)) + throw zkutil::KeeperException(code, path); + } + + if (code == Coordination::Error::ZOK) + break; + + if (!--num_attempts) + throw zkutil::KeeperException(code, path); + } + LOG_DEBUG(log, "Object {} stored", backQuote(object_name)); + + /// Refresh object and set watch for it. Because it can be replaced by another node after creation. + refreshObject(zookeeper, object_type, object_name); + + return true; +} + + +bool UserDefinedSQLObjectsLoaderFromZooKeeper::removeObject( + UserDefinedSQLObjectType object_type, const String & object_name, bool throw_if_not_exists) +{ + String path = getNodePath(zookeeper_path, object_type, object_name); + LOG_DEBUG(log, "Removing user-defined object {} at zk path {}", backQuote(object_name), path); + + auto zookeeper = getZooKeeper(); + + auto code = zookeeper->tryRemove(path); + if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNONODE)) + throw zkutil::KeeperException(code, path); + + if (code == Coordination::Error::ZNONODE) + { + if (throw_if_not_exists) + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "User-defined object '{}' doesn't exist", object_name); + else + return false; + } + + LOG_DEBUG(log, "Object {} removed", backQuote(object_name)); + return true; +} + +bool UserDefinedSQLObjectsLoaderFromZooKeeper::getObjectDataAndSetWatch( + const zkutil::ZooKeeperPtr & zookeeper, + String & data, + const String & path, + UserDefinedSQLObjectType object_type, + const String & object_name) +{ + const auto object_watcher = [watch_queue = watch_queue, object_type, object_name](const Coordination::WatchResponse & response) + { + if (response.type == Coordination::Event::CHANGED) [[maybe_unused]] + bool push_result = watch_queue->emplace(object_type, object_name); + /// NOTE: Event::DELETED is processed as child event by getChildren watch + }; + + Coordination::Stat entity_stat; + String object_create_query; + return zookeeper->tryGetWatch(path, data, &entity_stat, object_watcher); +} + +ASTPtr UserDefinedSQLObjectsLoaderFromZooKeeper::parseObjectData(const String & object_data, UserDefinedSQLObjectType object_type) +{ + switch (object_type) + { + case UserDefinedSQLObjectType::Function: { + ParserCreateFunctionQuery parser; + ASTPtr ast = parseQuery( + parser, + object_data.data(), + object_data.data() + object_data.size(), + "", + 0, + global_context->getSettingsRef().max_parser_depth); + return ast; + } + } + UNREACHABLE(); +} + +ASTPtr UserDefinedSQLObjectsLoaderFromZooKeeper::tryLoadObject( + const zkutil::ZooKeeperPtr & zookeeper, UserDefinedSQLObjectType object_type, const String & object_name) +{ + String path = getNodePath(zookeeper_path, object_type, object_name); + LOG_DEBUG(log, "Loading user defined object {} from zk path {}", backQuote(object_name), path); + + try + { + String object_data; + bool exists = getObjectDataAndSetWatch(zookeeper, object_data, path, object_type, object_name); + + if (!exists) + { + LOG_INFO(log, "User-defined object '{}' can't be loaded from path {}, because it doesn't exist", backQuote(object_name), path); + return nullptr; + } + + return parseObjectData(object_data, object_type); + } + catch (...) + { + tryLogCurrentException(log, fmt::format("while loading user defined SQL object {}", backQuote(object_name))); + return nullptr; /// Failed to load this sql object, will ignore it + } +} + +Strings UserDefinedSQLObjectsLoaderFromZooKeeper::getObjectNamesAndSetWatch( + const zkutil::ZooKeeperPtr & zookeeper, UserDefinedSQLObjectType object_type) +{ + auto object_list_watcher = [watch_queue = watch_queue, object_type](const Coordination::WatchResponse &) + { [[maybe_unused]] bool push_result = watch_queue->emplace(object_type, ""); }; + + Coordination::Stat stat; + const auto path = getRootNodePath(zookeeper_path, object_type); + const auto node_names = zookeeper->getChildrenWatch(path, &stat, object_list_watcher); + + Strings object_names; + object_names.reserve(node_names.size()); + for (const auto & node_name : node_names) + { + String object_name = unescapeForFileName(node_name); + + if (!object_name.empty()) + object_names.push_back(std::move(object_name)); + } + + return object_names; +} + +void UserDefinedSQLObjectsLoaderFromZooKeeper::refreshAllObjects(const zkutil::ZooKeeperPtr & zookeeper) +{ + /// It doesn't make sense to keep the old watch events because we will reread everything in this function. + watch_queue->clear(); + + refreshObjects(zookeeper, UserDefinedSQLObjectType::Function); + objects_loaded = true; +} + +void UserDefinedSQLObjectsLoaderFromZooKeeper::refreshObjects(const zkutil::ZooKeeperPtr & zookeeper, UserDefinedSQLObjectType object_type) +{ + LOG_DEBUG(log, "Refreshing all user-defined {} objects", toString(object_type)); + Strings object_names = getObjectNamesAndSetWatch(zookeeper, object_type); + + /// Read & parse all SQL objects from ZooKeeper + std::vector> function_names_and_asts; + for (const auto & function_name : object_names) + { + if (auto ast = tryLoadObject(zookeeper, UserDefinedSQLObjectType::Function, function_name)) + function_names_and_asts.emplace_back(function_name, ast); + } + + UserDefinedSQLFunctionFactory::instance().setAllFunctions(function_names_and_asts); + + LOG_DEBUG(log, "All user-defined {} objects refreshed", toString(object_type)); +} + +void UserDefinedSQLObjectsLoaderFromZooKeeper::syncObjects(const zkutil::ZooKeeperPtr & zookeeper, UserDefinedSQLObjectType object_type) +{ + LOG_DEBUG(log, "Syncing user-defined {} objects", toString(object_type)); + Strings object_names = getObjectNamesAndSetWatch(zookeeper, object_type); + + auto & factory = UserDefinedSQLFunctionFactory::instance(); + auto lock = factory.getLock(); + + /// Remove stale objects + factory.removeAllFunctionsExcept(object_names); + /// Read & parse only new SQL objects from ZooKeeper + for (const auto & function_name : object_names) + { + if (!UserDefinedSQLFunctionFactory::instance().has(function_name)) + refreshObject(zookeeper, UserDefinedSQLObjectType::Function, function_name); + } + + LOG_DEBUG(log, "User-defined {} objects synced", toString(object_type)); +} + +void UserDefinedSQLObjectsLoaderFromZooKeeper::refreshObject( + const zkutil::ZooKeeperPtr & zookeeper, UserDefinedSQLObjectType object_type, const String & object_name) +{ + auto ast = tryLoadObject(zookeeper, object_type, object_name); + auto & factory = UserDefinedSQLFunctionFactory::instance(); + + if (ast) + factory.setFunction(object_name, *ast); + else + factory.removeFunction(object_name); +} + +} diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.h b/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.h new file mode 100644 index 00000000000..16a78c24dac --- /dev/null +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.h @@ -0,0 +1,79 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +/// Loads user-defined sql objects from ZooKeeper. +class UserDefinedSQLObjectsLoaderFromZooKeeper : public IUserDefinedSQLObjectsLoader +{ +public: + UserDefinedSQLObjectsLoaderFromZooKeeper(const ContextPtr & global_context_, const String & zookeeper_path_); + ~UserDefinedSQLObjectsLoaderFromZooKeeper() override; + + bool isReplicated() const override { return true; } + String getReplicationID() const override { return zookeeper_path; } + + void loadObjects() override; + void stopWatching() override; + void reloadObjects() override; + void reloadObject(UserDefinedSQLObjectType object_type, const String & object_name) override; + + bool storeObject( + UserDefinedSQLObjectType object_type, + const String & object_name, + const IAST & create_object_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) override; + bool removeObject(UserDefinedSQLObjectType object_type, const String & object_name, bool throw_if_not_exists) override; + +private: + void processWatchQueue(); + + zkutil::ZooKeeperPtr getZooKeeper(); + void initZooKeeperIfNeeded(); + void resetAfterError(); + + void startWatchingThread(); + void stopWatchingThread(); + + void createRootNodes(const zkutil::ZooKeeperPtr & zookeeper); + + ASTPtr tryLoadObject(const zkutil::ZooKeeperPtr & zookeeper, UserDefinedSQLObjectType object_type, const String & object_name); + void refreshObject(const zkutil::ZooKeeperPtr & zookeeper, UserDefinedSQLObjectType object_type, const String & object_name); + + bool getObjectDataAndSetWatch( + const zkutil::ZooKeeperPtr & zookeeper, + String & data, + const String & path, + UserDefinedSQLObjectType object_type, + const String & object_name); + Strings getObjectNamesAndSetWatch(const zkutil::ZooKeeperPtr & zookeeper, UserDefinedSQLObjectType object_type); + ASTPtr parseObjectData(const String & object_data, UserDefinedSQLObjectType object_type); + + void refreshAllObjects(const zkutil::ZooKeeperPtr & zookeeper); + void refreshObjects(const zkutil::ZooKeeperPtr & zookeeper, UserDefinedSQLObjectType object_type); + void syncObjects(const zkutil::ZooKeeperPtr & zookeeper, UserDefinedSQLObjectType object_type); + + ContextPtr global_context; + + zkutil::ZooKeeperCachingGetter zookeeper_getter; + String zookeeper_path; + std::atomic objects_loaded = false; + + ThreadFromGlobalPool watching_thread; + std::atomic watching_flag = false; + std::shared_ptr>> watch_queue; + + Poco::Logger * log; +}; + +} diff --git a/src/Functions/UserDefined/createUserDefinedSQLObjectsLoader.cpp b/src/Functions/UserDefined/createUserDefinedSQLObjectsLoader.cpp index 9d0137328d1..b7ebc7abf14 100644 --- a/src/Functions/UserDefined/createUserDefinedSQLObjectsLoader.cpp +++ b/src/Functions/UserDefined/createUserDefinedSQLObjectsLoader.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -10,11 +11,33 @@ namespace fs = std::filesystem; namespace DB { + +namespace ErrorCodes +{ + extern const int INVALID_CONFIG_PARAMETER; +} + std::unique_ptr createUserDefinedSQLObjectsLoader(const ContextMutablePtr & global_context) { + const String zookeeper_path_key = "user_defined_zookeeper_path"; + const String disk_path_key = "user_defined_path"; + const auto & config = global_context->getConfigRef(); + if (config.has(zookeeper_path_key)) + { + if (config.has(disk_path_key)) + { + throw Exception( + ErrorCodes::INVALID_CONFIG_PARAMETER, + "'{}' and '{}' must not be both specified in the config", + zookeeper_path_key, + disk_path_key); + } + return std::make_unique(global_context, config.getString(zookeeper_path_key)); + } + String default_path = fs::path{global_context->getPath()} / "user_defined/"; - String path = config.getString("user_defined_path", default_path); + String path = config.getString(disk_path_key, default_path); return std::make_unique(global_context, path); } From 1321f87eee06907840c71acaa51c4a0bf1557525 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Mon, 6 Feb 2023 18:51:22 +0300 Subject: [PATCH 216/559] Tests for replication UDF to ZooKeeper --- .../__init__.py | 0 .../configs/config.xml | 17 ++ .../configs/zookeeper.xml | 17 ++ .../test.py | 255 ++++++++++++++++++ 4 files changed, 289 insertions(+) create mode 100644 tests/integration/test_replicated_user_defined_functions/__init__.py create mode 100644 tests/integration/test_replicated_user_defined_functions/configs/config.xml create mode 100644 tests/integration/test_replicated_user_defined_functions/configs/zookeeper.xml create mode 100644 tests/integration/test_replicated_user_defined_functions/test.py diff --git a/tests/integration/test_replicated_user_defined_functions/__init__.py b/tests/integration/test_replicated_user_defined_functions/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_replicated_user_defined_functions/configs/config.xml b/tests/integration/test_replicated_user_defined_functions/configs/config.xml new file mode 100644 index 00000000000..01553dec179 --- /dev/null +++ b/tests/integration/test_replicated_user_defined_functions/configs/config.xml @@ -0,0 +1,17 @@ + + /clickhouse/udf + + + + + node1 + 9000 + + + node2 + 9000 + + + + + diff --git a/tests/integration/test_replicated_user_defined_functions/configs/zookeeper.xml b/tests/integration/test_replicated_user_defined_functions/configs/zookeeper.xml new file mode 100644 index 00000000000..f8302e667ba --- /dev/null +++ b/tests/integration/test_replicated_user_defined_functions/configs/zookeeper.xml @@ -0,0 +1,17 @@ + + + + zoo1 + 2181 + + + zoo2 + 2181 + + + zoo3 + 2181 + + 20000 + + diff --git a/tests/integration/test_replicated_user_defined_functions/test.py b/tests/integration/test_replicated_user_defined_functions/test.py new file mode 100644 index 00000000000..199136a75fa --- /dev/null +++ b/tests/integration/test_replicated_user_defined_functions/test.py @@ -0,0 +1,255 @@ +import inspect + +import pytest +import time +import os.path + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry, TSV + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + +cluster = ClickHouseCluster(__file__, zookeeper_config_path="configs/zookeeper.xml") + +node1 = cluster.add_instance( + "node1", + main_configs=["configs/config.xml"], + with_zookeeper=True, + stay_alive=True, +) + +node2 = cluster.add_instance( + "node2", + main_configs=["configs/config.xml"], + with_zookeeper=True, + stay_alive=True, +) + +all_nodes = [node1, node2] + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def wait_zookeeper_node_to_start(zk_nodes, timeout=60): + start = time.time() + while time.time() - start < timeout: + try: + for instance in zk_nodes: + conn = cluster.get_kazoo_client(instance) + conn.get_children("/") + print("All instances of ZooKeeper started") + return + except Exception as ex: + print(("Can't connect to ZooKeeper " + str(ex))) + time.sleep(0.5) + + +def replace_zookeeper_config(new_config): + node1.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config) + node2.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config) + node1.query("SYSTEM RELOAD CONFIG") + node2.query("SYSTEM RELOAD CONFIG") + + +def revert_zookeeper_config(): + with open(os.path.join(SCRIPT_DIR, "configs/zookeeper.xml"), "r") as f: + replace_zookeeper_config(f.read()) + + +def get_active_zk_connections(): + return str( + node1.exec_in_container( + [ + "bash", + "-c", + "lsof -a -i4 -i6 -itcp -w | grep 2181 | grep ESTABLISHED | wc -l", + ], + privileged=True, + user="root", + ) + ).strip() + + +def test_create_and_drop(): + node1.query("CREATE FUNCTION f1 AS (x, y) -> x + y") + assert node1.query("SELECT f1(12, 3)") == "15\n" + node1.query("DROP FUNCTION f1") + + +def test_create_and_replace(): + node1.query("CREATE FUNCTION f1 AS (x, y) -> x + y") + + expected_error = "User-defined function 'f1' already exists" + assert expected_error in node1.query_and_get_error( + "CREATE FUNCTION f1 AS (x, y) -> x + 2 * y" + ) + + node1.query("CREATE FUNCTION IF NOT EXISTS f1 AS (x, y) -> x + 3 * y") + assert node1.query("SELECT f1(12, 3)") == "15\n" + + node1.query("CREATE OR REPLACE FUNCTION f1 AS (x, y) -> x + 4 * y") + assert node1.query("SELECT f1(12, 3)") == "24\n" + + node1.query("DROP FUNCTION f1") + + +def test_drop_if_exists(): + node1.query("CREATE FUNCTION f1 AS (x, y) -> x + y") + node1.query("DROP FUNCTION IF EXISTS f1") + node1.query("DROP FUNCTION IF EXISTS f1") + + expected_error = "User-defined function 'f1' doesn't exist" + assert expected_error in node1.query_and_get_error("DROP FUNCTION f1") + + +def test_replication(): + node1.query("CREATE FUNCTION f2 AS (x, y) -> x - y") + + assert ( + node1.query("SELECT create_query FROM system.functions WHERE name='f2'") + == "CREATE FUNCTION f2 AS (x, y) -> (x - y)\n" + ) + assert_eq_with_retry( + node2, + "SELECT create_query FROM system.functions WHERE name='f2'", + "CREATE FUNCTION f2 AS (x, y) -> (x - y)\n", + ) + assert node1.query("SELECT f2(12,3)") == "9\n" + assert node2.query("SELECT f2(12,3)") == "9\n" + + node1.query("DROP FUNCTION f2") + assert ( + node1.query("SELECT create_query FROM system.functions WHERE name='f2'") == "" + ) + assert_eq_with_retry( + node2, "SELECT create_query FROM system.functions WHERE name='f2'", "" + ) + + +def test_replication_replace_by_another_node_after_creation(): + node1.query("CREATE FUNCTION f2 AS (x, y) -> x - y") + + assert_eq_with_retry( + node2, + "SELECT create_query FROM system.functions WHERE name='f2'", + "CREATE FUNCTION f2 AS (x, y) -> (x - y)\n", + ) + + node2.query("CREATE OR REPLACE FUNCTION f2 AS (x, y) -> x + y") + + assert_eq_with_retry( + node1, + "SELECT create_query FROM system.functions WHERE name='f2'", + "CREATE FUNCTION f2 AS (x, y) -> (x + y)\n", + ) + + node1.query("DROP FUNCTION f2") + assert_eq_with_retry( + node1, "SELECT create_query FROM system.functions WHERE name='f2'", "" + ) + assert_eq_with_retry( + node2, "SELECT create_query FROM system.functions WHERE name='f2'", "" + ) + + +# UserDefinedSQLObjectsLoaderFromZooKeeper must be able to continue working after reloading ZooKeeper. +def test_reload_zookeeper(): + node1.query("CREATE FUNCTION f1 AS (x, y) -> x + y") + assert_eq_with_retry( + node2, "SELECT name FROM system.functions WHERE name ='f1'", "f1\n" + ) + + # remove zoo2, zoo3 from configs + replace_zookeeper_config( + inspect.cleandoc( + """ + + + + zoo1 + 2181 + + 2000 + + + """ + ) + ) + + # config reloads, but can still work + node1.query("CREATE FUNCTION f2 AS (x, y) -> x - y") + assert_eq_with_retry( + node2, + "SELECT name FROM system.functions WHERE name IN ['f1', 'f2'] ORDER BY name", + TSV(["f1", "f2"]), + ) + + # stop all zookeepers, user-defined functions will be readonly + cluster.stop_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) + assert node2.query( + "SELECT name FROM system.functions WHERE name IN ['f1', 'f2'] ORDER BY name" + ) == TSV(["f1", "f2"]) + assert "ZooKeeper" in node1.query_and_get_error( + "CREATE FUNCTION f3 AS (x, y) -> x * y" + ) + + # start zoo2, zoo3, user-defined functions will be readonly too, because it only connect to zoo1 + cluster.start_zookeeper_nodes(["zoo2", "zoo3"]) + wait_zookeeper_node_to_start(["zoo2", "zoo3"]) + assert node2.query( + "SELECT name FROM system.functions WHERE name IN ['f1', 'f2', 'f3'] ORDER BY name" + ) == TSV(["f1", "f2"]) + assert "ZooKeeper" in node1.query_and_get_error( + "CREATE FUNCTION f3 AS (x, y) -> x * y" + ) + + # set config to zoo2, server will be normal + replace_zookeeper_config( + inspect.cleandoc( + """ + + + + zoo2 + 2181 + + 2000 + + + """ + ) + ) + + active_zk_connections = get_active_zk_connections() + assert ( + active_zk_connections == "1" + ), "Total connections to ZooKeeper not equal to 1, {}".format(active_zk_connections) + + node1.query("CREATE FUNCTION f3 AS (x, y) -> x / y") + assert_eq_with_retry( + node2, + "SELECT name FROM system.functions WHERE name IN ['f1', 'f2', 'f3'] ORDER BY name", + TSV(["f1", "f2", "f3"]), + ) + + assert node2.query("SELECT f1(12, 3), f2(12, 3), f3(12, 3)") == TSV([[15, 9, 4]]) + + active_zk_connections = get_active_zk_connections() + assert ( + active_zk_connections == "1" + ), "Total connections to ZooKeeper not equal to 1, {}".format(active_zk_connections) + + node1.query("DROP FUNCTION f1") + node1.query("DROP FUNCTION f2") + node1.query("DROP FUNCTION f3") + + # switch to the original version of zookeeper config + cluster.start_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) + revert_zookeeper_config() From 690d8355ca8e5734a2113a09348721bed6eecb7b Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Mon, 6 Feb 2023 18:52:39 +0300 Subject: [PATCH 217/559] Add classes for backup/restore UDF --- src/Backups/BackupCoordinationLocal.cpp | 13 ++++ src/Backups/BackupCoordinationLocal.h | 6 ++ src/Backups/BackupCoordinationRemote.cpp | 61 +++++++++++++++++++ src/Backups/BackupCoordinationRemote.h | 10 +++ ...BackupCoordinationReplicatedSQLObjects.cpp | 33 ++++++++++ .../BackupCoordinationReplicatedSQLObjects.h | 49 +++++++++++++++ src/Backups/IBackupCoordination.h | 5 ++ src/Backups/IRestoreCoordination.h | 6 ++ src/Backups/RestoreCoordinationLocal.cpp | 5 ++ src/Backups/RestoreCoordinationLocal.h | 4 ++ src/Backups/RestoreCoordinationRemote.cpp | 24 ++++++++ src/Backups/RestoreCoordinationRemote.h | 4 ++ .../UserDefinedSQLObjectsBackup.cpp | 61 ++++++++++++++++--- 13 files changed, 271 insertions(+), 10 deletions(-) create mode 100644 src/Backups/BackupCoordinationReplicatedSQLObjects.cpp create mode 100644 src/Backups/BackupCoordinationReplicatedSQLObjects.h diff --git a/src/Backups/BackupCoordinationLocal.cpp b/src/Backups/BackupCoordinationLocal.cpp index 91da16097cc..2d0bf978b83 100644 --- a/src/Backups/BackupCoordinationLocal.cpp +++ b/src/Backups/BackupCoordinationLocal.cpp @@ -83,6 +83,19 @@ Strings BackupCoordinationLocal::getReplicatedAccessFilePaths(const String & acc } +void BackupCoordinationLocal::addReplicatedSQLObjectsDir(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & host_id, const String & dir_path) +{ + std::lock_guard lock{mutex}; + replicated_sql_objects.addDirectory(loader_zk_path, object_type, host_id, dir_path); +} + +Strings BackupCoordinationLocal::getReplicatedSQLObjectsDirs(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & host_id) const +{ + std::lock_guard lock{mutex}; + return replicated_sql_objects.getDirectories(loader_zk_path, object_type, host_id); +} + + void BackupCoordinationLocal::addFileInfo(const FileInfo & file_info, bool & is_data_file_required) { std::lock_guard lock{mutex}; diff --git a/src/Backups/BackupCoordinationLocal.h b/src/Backups/BackupCoordinationLocal.h index 8e54eb6fb27..edfa6c9973b 100644 --- a/src/Backups/BackupCoordinationLocal.h +++ b/src/Backups/BackupCoordinationLocal.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -39,6 +40,9 @@ public: void addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id, const String & file_path) override; Strings getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const override; + void addReplicatedSQLObjectsDir(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & host_id, const String & dir_path) override; + Strings getReplicatedSQLObjectsDirs(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & host_id) const override; + void addFileInfo(const FileInfo & file_info, bool & is_data_file_required) override; void updateFileInfo(const FileInfo & file_info) override; @@ -58,6 +62,8 @@ private: mutable std::mutex mutex; BackupCoordinationReplicatedTables replicated_tables TSA_GUARDED_BY(mutex); BackupCoordinationReplicatedAccess replicated_access TSA_GUARDED_BY(mutex); + BackupCoordinationReplicatedSQLObjects replicated_sql_objects TSA_GUARDED_BY(mutex); + std::map file_names TSA_GUARDED_BY(mutex); /// Should be ordered alphabetically, see listFiles(). For empty files we assume checksum = 0. std::map file_infos TSA_GUARDED_BY(mutex); /// Information about files. Without empty files. Strings archive_suffixes TSA_GUARDED_BY(mutex); diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 8e43676f59c..647e48a6beb 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -231,6 +232,7 @@ void BackupCoordinationRemote::createRootNodes() zk->createIfNotExists(zookeeper_path + "/repl_mutations", ""); zk->createIfNotExists(zookeeper_path + "/repl_data_paths", ""); zk->createIfNotExists(zookeeper_path + "/repl_access", ""); + zk->createIfNotExists(zookeeper_path + "/repl_sql_objects", ""); zk->createIfNotExists(zookeeper_path + "/file_names", ""); zk->createIfNotExists(zookeeper_path + "/file_infos", ""); zk->createIfNotExists(zookeeper_path + "/archive_suffixes", ""); @@ -451,6 +453,65 @@ void BackupCoordinationRemote::prepareReplicatedAccess() const } } +void BackupCoordinationRemote::addReplicatedSQLObjectsDir(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & host_id, const String & dir_path) +{ + { + std::lock_guard lock{mutex}; + if (replicated_sql_objects) + throw Exception(ErrorCodes::LOGICAL_ERROR, "addReplicatedSQLObjectsDir() must not be called after preparing"); + } + + auto zk = getZooKeeper(); + String path = zookeeper_path + "/repl_sql_objects/" + escapeForFileName(loader_zk_path); + zk->createIfNotExists(path, ""); + + path += "/"; + switch (object_type) + { + case UserDefinedSQLObjectType::Function: + path += "functions"; + break; + } + + zk->createIfNotExists(path, ""); + path += "/" + host_id; + zk->createIfNotExists(path, dir_path); +} + +Strings BackupCoordinationRemote::getReplicatedSQLObjectsDirs(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & host_id) const +{ + std::lock_guard lock{mutex}; + prepareReplicatedSQLObjects(); + return replicated_sql_objects->getDirectories(loader_zk_path, object_type, host_id); +} + +void BackupCoordinationRemote::prepareReplicatedSQLObjects() const +{ + if (replicated_sql_objects) + return; + + replicated_sql_objects.emplace(); + auto zk = getZooKeeperNoLock(); + + String path = zookeeper_path + "/repl_sql_objects"; + for (const String & escaped_loader_zk_path : zk->getChildren(path)) + { + String loader_zk_path = unescapeForFileName(escaped_loader_zk_path); + String path2 = path + "/" + escaped_loader_zk_path; + + if (zk->exists(path2 + "/functions")) + { + String path3 = path2 + "/functions"; + UserDefinedSQLObjectType object_type = UserDefinedSQLObjectType::Function; + for (const String & host_id : zk->getChildren(path3)) + { + String dir = zk->get(path3 + "/" + host_id); + replicated_sql_objects->addDirectory(loader_zk_path, object_type, host_id, dir); + } + } + } +} + void BackupCoordinationRemote::addFileInfo(const FileInfo & file_info, bool & is_data_file_required) { diff --git a/src/Backups/BackupCoordinationRemote.h b/src/Backups/BackupCoordinationRemote.h index 23c76f5be47..74c6b58341a 100644 --- a/src/Backups/BackupCoordinationRemote.h +++ b/src/Backups/BackupCoordinationRemote.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -60,6 +61,9 @@ public: void addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id, const String & file_path) override; Strings getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const override; + void addReplicatedSQLObjectsDir(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & host_id, const String & dir_path) override; + Strings getReplicatedSQLObjectsDirs(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & host_id) const override; + void addFileInfo(const FileInfo & file_info, bool & is_data_file_required) override; void updateFileInfo(const FileInfo & file_info) override; @@ -79,8 +83,13 @@ private: zkutil::ZooKeeperPtr getZooKeeperNoLock() const; void createRootNodes(); void removeAllNodes(); + + /// Reads data of all objects from ZooKeeper that replicas have added to backup and add it to the corresponding + /// BackupCoordinationReplicated* objects. + /// After that, calling addReplicated* functions is not allowed and throws an exception. void prepareReplicatedTables() const; void prepareReplicatedAccess() const; + void prepareReplicatedSQLObjects() const; const BackupKeeperSettings keeper_settings; const String root_zookeeper_path; @@ -96,6 +105,7 @@ private: mutable zkutil::ZooKeeperPtr zookeeper; mutable std::optional replicated_tables; mutable std::optional replicated_access; + mutable std::optional replicated_sql_objects; }; } diff --git a/src/Backups/BackupCoordinationReplicatedSQLObjects.cpp b/src/Backups/BackupCoordinationReplicatedSQLObjects.cpp new file mode 100644 index 00000000000..87b9390e48d --- /dev/null +++ b/src/Backups/BackupCoordinationReplicatedSQLObjects.cpp @@ -0,0 +1,33 @@ +#include + + +namespace DB +{ + +BackupCoordinationReplicatedSQLObjects::BackupCoordinationReplicatedSQLObjects() = default; +BackupCoordinationReplicatedSQLObjects::~BackupCoordinationReplicatedSQLObjects() = default; + +void BackupCoordinationReplicatedSQLObjects::addDirectory(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & host_id, const String & dir_path) +{ + auto & ref = dir_paths_by_zk_path[std::make_pair(loader_zk_path, object_type)]; + ref.dir_paths.emplace(dir_path); + + /// std::max() because the calculation must give the same result being repeated on a different replica. + ref.host_to_store = std::max(ref.host_to_store, host_id); +} + +Strings BackupCoordinationReplicatedSQLObjects::getDirectories(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & host_id) const +{ + auto it = dir_paths_by_zk_path.find(std::make_pair(loader_zk_path, object_type)); + if (it == dir_paths_by_zk_path.end()) + return {}; + + const auto & dir_paths = it->second; + if (dir_paths.host_to_store != host_id) + return {}; + + Strings res{dir_paths.dir_paths.begin(), dir_paths.dir_paths.end()}; + return res; +} + +} diff --git a/src/Backups/BackupCoordinationReplicatedSQLObjects.h b/src/Backups/BackupCoordinationReplicatedSQLObjects.h new file mode 100644 index 00000000000..b70c48edc54 --- /dev/null +++ b/src/Backups/BackupCoordinationReplicatedSQLObjects.h @@ -0,0 +1,49 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ +enum class UserDefinedSQLObjectType; + +/// This class is used by hosts to coordinate the user-defined SQL objects they're going to write to a backup. +/// It's designed to make all hosts save the same objects to the backup even in case some objects change while +/// the backup is being produced. This is important to make RESTORE more predicitible. +/// +/// For example, let's consider three replicas having access to the user-defined function `f1`. +/// This class ensures that the following files in the backup will be the same: +/// /shards/1/replicas/1/data/system/functions/f1.sql +/// /shards/1/replicas/2/data/system/functions/f1.sql +/// /shards/1/replicas/3/data/system/functions/f1.sql +/// +/// To implement that this class chooses one host to write user-defined SQL objects for all the hosts so in fact all those files +/// in the example above are written by single host. + +class BackupCoordinationReplicatedSQLObjects +{ +public: + BackupCoordinationReplicatedSQLObjects(); + ~BackupCoordinationReplicatedSQLObjects(); + + /// Adds a path to directory keeping user defined SQL objects. + void addDirectory(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & host_id, const String & dir_path); + + /// Returns all added paths to directories if `host_id` is a host chosen to store user-defined SQL objects. + Strings getDirectories(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & host_id) const; + +private: + using ZkPathAndObjectType = std::pair; + + struct DirPathsAndHost + { + std::unordered_set dir_paths; + String host_to_store; + }; + + std::map dir_paths_by_zk_path; +}; + +} diff --git a/src/Backups/IBackupCoordination.h b/src/Backups/IBackupCoordination.h index 588a20d9eeb..f87aa8b8f41 100644 --- a/src/Backups/IBackupCoordination.h +++ b/src/Backups/IBackupCoordination.h @@ -10,6 +10,7 @@ namespace DB { class Exception; enum class AccessEntityType; +enum class UserDefinedSQLObjectType; /// Replicas use this class to coordinate what they're writing to a backup while executing BACKUP ON CLUSTER. /// There are two implementation of this interface: BackupCoordinationLocal and BackupCoordinationRemote. @@ -68,6 +69,10 @@ public: virtual void addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id, const String & file_path) = 0; virtual Strings getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const = 0; + /// Adds a path to a directory with user-defined SQL objects inside the backup. + virtual void addReplicatedSQLObjectsDir(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & host_id, const String & dir_path) = 0; + virtual Strings getReplicatedSQLObjectsDirs(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & host_id) const = 0; + struct FileInfo { String file_name; diff --git a/src/Backups/IRestoreCoordination.h b/src/Backups/IRestoreCoordination.h index 098d048f6a3..61aee533dd2 100644 --- a/src/Backups/IRestoreCoordination.h +++ b/src/Backups/IRestoreCoordination.h @@ -6,6 +6,7 @@ namespace DB { class Exception; +enum class UserDefinedSQLObjectType; /// Replicas use this class to coordinate what they're reading from a backup while executing RESTORE ON CLUSTER. /// There are two implementation of this interface: RestoreCoordinationLocal and RestoreCoordinationRemote. @@ -35,9 +36,14 @@ public: /// The function returns false if this access storage is being already restored by another replica. virtual bool acquireReplicatedAccessStorage(const String & access_storage_zk_path) = 0; + /// Sets that this replica is going to restore replicated user-defined functions. + /// The function returns false if user-defined function at a specified zk path are being already restored by another replica. + virtual bool acquireReplicatedSQLObjects(const String & loader_zk_path, UserDefinedSQLObjectType object_type) = 0; + /// This function is used to check if concurrent restores are running /// other than the restore passed to the function virtual bool hasConcurrentRestores(const std::atomic & num_active_restores) const = 0; + }; } diff --git a/src/Backups/RestoreCoordinationLocal.cpp b/src/Backups/RestoreCoordinationLocal.cpp index 4e908d5d67d..d5334e2c6aa 100644 --- a/src/Backups/RestoreCoordinationLocal.cpp +++ b/src/Backups/RestoreCoordinationLocal.cpp @@ -42,6 +42,11 @@ bool RestoreCoordinationLocal::acquireReplicatedAccessStorage(const String &) return true; } +bool RestoreCoordinationLocal::acquireReplicatedSQLObjects(const String &, UserDefinedSQLObjectType) +{ + return true; +} + bool RestoreCoordinationLocal::hasConcurrentRestores(const std::atomic & num_active_restores) const { return (num_active_restores > 1); diff --git a/src/Backups/RestoreCoordinationLocal.h b/src/Backups/RestoreCoordinationLocal.h index ab9d1ce0a59..cb3a8c55d22 100644 --- a/src/Backups/RestoreCoordinationLocal.h +++ b/src/Backups/RestoreCoordinationLocal.h @@ -35,6 +35,10 @@ public: /// The function returns false if this access storage is being already restored by another replica. bool acquireReplicatedAccessStorage(const String & access_storage_zk_path) override; + /// Sets that this replica is going to restore replicated user-defined functions. + /// The function returns false if user-defined function at a specified zk path are being already restored by another replica. + bool acquireReplicatedSQLObjects(const String & loader_zk_path, UserDefinedSQLObjectType object_type) override; + bool hasConcurrentRestores(const std::atomic & num_active_restores) const override; private: diff --git a/src/Backups/RestoreCoordinationRemote.cpp b/src/Backups/RestoreCoordinationRemote.cpp index f829cd20c0d..646c2c68d3b 100644 --- a/src/Backups/RestoreCoordinationRemote.cpp +++ b/src/Backups/RestoreCoordinationRemote.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -58,6 +59,7 @@ void RestoreCoordinationRemote::createRootNodes() zk->createIfNotExists(zookeeper_path + "/repl_databases_tables_acquired", ""); zk->createIfNotExists(zookeeper_path + "/repl_tables_data_acquired", ""); zk->createIfNotExists(zookeeper_path + "/repl_access_storages_acquired", ""); + zk->createIfNotExists(zookeeper_path + "/repl_sql_objects_acquired", ""); } @@ -121,6 +123,28 @@ bool RestoreCoordinationRemote::acquireReplicatedAccessStorage(const String & ac return (code == Coordination::Error::ZOK); } +bool RestoreCoordinationRemote::acquireReplicatedSQLObjects(const String & loader_zk_path, UserDefinedSQLObjectType object_type) +{ + auto zk = getZooKeeper(); + + String path = zookeeper_path + "/repl_sql_objects_acquired/" + escapeForFileName(loader_zk_path); + zk->createIfNotExists(path, ""); + + path += "/"; + switch (object_type) + { + case UserDefinedSQLObjectType::Function: + path += "functions"; + break; + } + + auto code = zk->tryCreate(path, "", zkutil::CreateMode::Persistent); + if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) + throw zkutil::KeeperException(code, path); + + return (code == Coordination::Error::ZOK); +} + void RestoreCoordinationRemote::removeAllNodes() { /// Usually this function is called by the initiator when a restore operation is complete so we don't need the coordination anymore. diff --git a/src/Backups/RestoreCoordinationRemote.h b/src/Backups/RestoreCoordinationRemote.h index d72781bac2f..4ee87796e8e 100644 --- a/src/Backups/RestoreCoordinationRemote.h +++ b/src/Backups/RestoreCoordinationRemote.h @@ -31,6 +31,10 @@ public: /// The function returns false if this access storage is being already restored by another replica. bool acquireReplicatedAccessStorage(const String & access_storage_zk_path) override; + /// Sets that this replica is going to restore replicated user-defined functions. + /// The function returns false if user-defined function at a specified zk path are being already restored by another replica. + bool acquireReplicatedSQLObjects(const String & loader_zk_path, UserDefinedSQLObjectType object_type) override; + bool hasConcurrentRestores(const std::atomic & num_active_restores) const override; private: diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsBackup.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsBackup.cpp index 25c309aeb65..c5541e61af8 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsBackup.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsBackup.cpp @@ -23,21 +23,57 @@ namespace ErrorCodes extern const int CANNOT_RESTORE_TABLE; } + void backupUserDefinedSQLObjects( BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, - UserDefinedSQLObjectType /* object_type */, + UserDefinedSQLObjectType object_type, const std::vector> & objects) { std::vector> backup_entries; - backup_entries.reserve(objects.size()); - for (const auto & [function_name, create_function_query] : objects) + for (const auto & [object_name, create_object_query] : objects) backup_entries.emplace_back( - escapeForFileName(function_name) + ".sql", std::make_shared(queryToString(create_function_query))); + escapeForFileName(object_name) + ".sql", std::make_shared(queryToString(create_object_query))); - fs::path data_path_in_backup_fs{data_path_in_backup}; - for (const auto & entry : backup_entries) - backup_entries_collector.addBackupEntry(data_path_in_backup_fs / entry.first, entry.second); + auto context = backup_entries_collector.getContext(); + const auto & loader = context->getUserDefinedSQLObjectsLoader(); + + if (!loader.isReplicated()) + { + fs::path data_path_in_backup_fs{data_path_in_backup}; + for (const auto & [file_name, entry] : backup_entries) + backup_entries_collector.addBackupEntry(data_path_in_backup_fs / file_name, entry); + return; + } + + String replication_id = loader.getReplicationID(); + String current_host_id = backup_entries_collector.getBackupSettings().host_id; + + auto backup_coordination = backup_entries_collector.getBackupCoordination(); + backup_coordination->addReplicatedSQLObjectsDir(replication_id, object_type, current_host_id, data_path_in_backup); + + // On the stage of running post tasks, all directories will already be added to the backup coordination object. + // They will only be returned for one of the hosts below, for the rest an empty list. + // See also BackupCoordinationReplicatedSQLObjects class. + backup_entries_collector.addPostTask( + [backup_entries = std::move(backup_entries), + replication_id = std::move(replication_id), + object_type, + current_host_id = std::move(current_host_id), + &backup_entries_collector, + backup_coordination] + { + auto dirs = backup_coordination->getReplicatedSQLObjectsDirs(replication_id, object_type, current_host_id); + + for (const auto & dir : dirs) + { + fs::path dir_fs{dir}; + for (const auto & [file_name, entry] : backup_entries) + { + backup_entries_collector.addBackupEntry(dir_fs / file_name, entry); + } + } + }); } @@ -45,6 +81,11 @@ std::vector> restoreUserDefinedSQLObjects(RestorerFromBackup & restorer, const String & data_path_in_backup, UserDefinedSQLObjectType object_type) { auto context = restorer.getContext(); + const auto & loader = context->getUserDefinedSQLObjectsLoader(); + + if (loader.isReplicated() && !restorer.getRestoreCoordination()->acquireReplicatedSQLObjects(loader.getReplicationID(), object_type)) + return {}; /// Other replica is already restoring user-defined SQL objects. + auto backup = restorer.getBackup(); fs::path data_path_in_backup_fs{data_path_in_backup}; @@ -67,8 +108,8 @@ restoreUserDefinedSQLObjects(RestorerFromBackup & restorer, const String & data_ for (const auto & filename : filenames) { - String escaped_function_name = filename.substr(0, filename.length() - strlen(".sql")); - String function_name = unescapeForFileName(escaped_function_name); + String escaped_object_name = filename.substr(0, filename.length() - strlen(".sql")); + String object_name = unescapeForFileName(escaped_object_name); String filepath = data_path_in_backup_fs / filename; auto in = backup->readFile(filepath); @@ -93,7 +134,7 @@ restoreUserDefinedSQLObjects(RestorerFromBackup & restorer, const String & data_ } } - res.emplace_back(std::move(function_name), ast); + res.emplace_back(std::move(object_name), ast); } return res; From cf6ca10c8352d5eb85aed568f84813c6f42bb2d6 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Mon, 6 Feb 2023 18:53:20 +0300 Subject: [PATCH 218/559] Tests for replicated `system.functions` --- .../UserDefined/UserDefinedSQLObjectType.cpp | 2 - .../UserDefinedSQLObjectsBackup.cpp | 1 + ...erDefinedSQLObjectsLoaderFromZooKeeper.cpp | 1 - .../replicated_user_defined_sql_objects.xml | 3 ++ .../test_backup_restore_on_cluster/test.py | 45 +++++++++++++++++++ 5 files changed, 49 insertions(+), 3 deletions(-) create mode 100644 tests/integration/test_backup_restore_on_cluster/configs/replicated_user_defined_sql_objects.xml diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectType.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectType.cpp index 47b75fb3291..cba1316c35c 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectType.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectType.cpp @@ -1,5 +1,4 @@ #include "UserDefinedSQLObjectType.h" -#include namespace DB { @@ -11,7 +10,6 @@ const char * toString(UserDefinedSQLObjectType type) case UserDefinedSQLObjectType::Function: return "FUNCTION"; } - UNREACHABLE(); } } diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsBackup.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsBackup.cpp index c5541e61af8..0fc86bf39ba 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsBackup.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsBackup.cpp @@ -31,6 +31,7 @@ void backupUserDefinedSQLObjects( const std::vector> & objects) { std::vector> backup_entries; + backup_entries.reserve(objects.size()); for (const auto & [object_name, create_object_query] : objects) backup_entries.emplace_back( escapeForFileName(object_name) + ".sql", std::make_shared(queryToString(create_object_query))); diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp index b41fd146753..ce2efdd6d03 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp @@ -144,7 +144,6 @@ void UserDefinedSQLObjectsLoaderFromZooKeeper::processWatchQueue() continue; auto zookeeper = getZooKeeper(); - const auto & [object_type, object_name] = watched_object; if (object_name.empty()) diff --git a/tests/integration/test_backup_restore_on_cluster/configs/replicated_user_defined_sql_objects.xml b/tests/integration/test_backup_restore_on_cluster/configs/replicated_user_defined_sql_objects.xml new file mode 100644 index 00000000000..8fd34feea46 --- /dev/null +++ b/tests/integration/test_backup_restore_on_cluster/configs/replicated_user_defined_sql_objects.xml @@ -0,0 +1,3 @@ + + /clickhouse/udf + diff --git a/tests/integration/test_backup_restore_on_cluster/test.py b/tests/integration/test_backup_restore_on_cluster/test.py index 27448b95b51..2a3d34c559c 100644 --- a/tests/integration/test_backup_restore_on_cluster/test.py +++ b/tests/integration/test_backup_restore_on_cluster/test.py @@ -11,6 +11,7 @@ cluster = ClickHouseCluster(__file__) main_configs = [ "configs/remote_servers.xml", "configs/replicated_access_storage.xml", + "configs/replicated_user_defined_sql_objects.xml", "configs/backups_disk.xml", "configs/lesser_timeouts.xml", # Default timeouts are quite big (a few minutes), the tests don't need them to be that big. ] @@ -604,6 +605,50 @@ def test_system_users(): assert node1.query("SHOW GRANTS FOR u1") == "GRANT SELECT ON default.tbl TO u1\n" +def test_system_functions(): + node1.query("CREATE FUNCTION linear_equation AS (x, k, b) -> k*x + b;") + + node1.query("CREATE FUNCTION parity_str AS (n) -> if(n % 2, 'odd', 'even');") + + backup_name = new_backup_name() + node1.query(f"BACKUP TABLE system.functions ON CLUSTER 'cluster' TO {backup_name}") + + node1.query("DROP FUNCTION linear_equation") + node1.query("DROP FUNCTION parity_str") + assert_eq_with_retry( + node2, "SELECT name FROM system.functions WHERE name='parity_str'", "" + ) + + node1.query( + f"RESTORE TABLE system.functions ON CLUSTER 'cluster' FROM {backup_name}" + ) + + assert node1.query( + "SELECT number, linear_equation(number, 2, 1) FROM numbers(3)" + ) == TSV([[0, 1], [1, 3], [2, 5]]) + + assert node1.query("SELECT number, parity_str(number) FROM numbers(3)") == TSV( + [[0, "even"], [1, "odd"], [2, "even"]] + ) + + assert node2.query( + "SELECT number, linear_equation(number, 2, 1) FROM numbers(3)" + ) == TSV([[0, 1], [1, 3], [2, 5]]) + + assert node2.query("SELECT number, parity_str(number) FROM numbers(3)") == TSV( + [[0, "even"], [1, "odd"], [2, "even"]] + ) + + assert_eq_with_retry( + node2, + "SELECT name FROM system.functions WHERE name='parity_str'", + "parity_str\n", + ) + assert node2.query("SELECT number, parity_str(number) FROM numbers(3)") == TSV( + [[0, "even"], [1, "odd"], [2, "even"]] + ) + + def test_projection(): node1.query( "CREATE TABLE tbl ON CLUSTER 'cluster' (x UInt32, y String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/tbl/', '{replica}') " From 886b5309632eb2b2d9b2d5122b1a492bbdf82e09 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Fri, 17 Mar 2023 13:51:45 +0300 Subject: [PATCH 219/559] [rev: 1] Fix review remarks --- src/Backups/BackupCoordinationRemote.cpp | 9 ++-- src/Common/ConcurrentBoundedQueue.h | 4 +- .../ZooKeeper/ZooKeeperCachingGetter.cpp | 8 ++-- src/Common/ZooKeeper/ZooKeeperCachingGetter.h | 12 +++-- .../UserDefined/UserDefinedSQLObjectType.cpp | 15 ------ .../UserDefined/UserDefinedSQLObjectType.h | 2 - ...erDefinedSQLObjectsLoaderFromZooKeeper.cpp | 26 ++++++---- ...UserDefinedSQLObjectsLoaderFromZooKeeper.h | 4 +- .../_gen/cluster_for_concurrency_test.xml | 48 ------------------- 9 files changed, 38 insertions(+), 90 deletions(-) delete mode 100644 src/Functions/UserDefined/UserDefinedSQLObjectType.cpp delete mode 100644 tests/integration/test_backup_restore_on_cluster/_gen/cluster_for_concurrency_test.xml diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 647e48a6beb..e7ea9becd24 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -497,15 +497,14 @@ void BackupCoordinationRemote::prepareReplicatedSQLObjects() const for (const String & escaped_loader_zk_path : zk->getChildren(path)) { String loader_zk_path = unescapeForFileName(escaped_loader_zk_path); - String path2 = path + "/" + escaped_loader_zk_path; + String objects_path = path + "/" + escaped_loader_zk_path; - if (zk->exists(path2 + "/functions")) + if (String functions_path = objects_path + "/functions"; zk->exists(functions_path)) { - String path3 = path2 + "/functions"; UserDefinedSQLObjectType object_type = UserDefinedSQLObjectType::Function; - for (const String & host_id : zk->getChildren(path3)) + for (const String & host_id : zk->getChildren(functions_path)) { - String dir = zk->get(path3 + "/" + host_id); + String dir = zk->get(functions_path + "/" + host_id); replicated_sql_objects->addDirectory(loader_zk_path, object_type, host_id, dir); } } diff --git a/src/Common/ConcurrentBoundedQueue.h b/src/Common/ConcurrentBoundedQueue.h index 8b813911e23..01910c4caff 100644 --- a/src/Common/ConcurrentBoundedQueue.h +++ b/src/Common/ConcurrentBoundedQueue.h @@ -128,7 +128,7 @@ public: template [[nodiscard]] bool emplace(Args &&... args) { - return emplaceImpl(std::nullopt /* timeout in milliseconds */, std::forward(args...)); + return emplaceImpl(std::nullopt /* timeout in milliseconds */, std::forward(args)...); } /// Returns false if queue is finished or object was not pushed during timeout @@ -146,7 +146,7 @@ public: template [[nodiscard]] bool tryEmplace(UInt64 milliseconds, Args &&... args) { - return emplaceImpl(milliseconds, std::forward(args...)); + return emplaceImpl(milliseconds, std::forward(args)...); } /// Returns false if queue is finished and empty diff --git a/src/Common/ZooKeeper/ZooKeeperCachingGetter.cpp b/src/Common/ZooKeeper/ZooKeeperCachingGetter.cpp index 4b30e35a0ee..6577cc4ac95 100644 --- a/src/Common/ZooKeeper/ZooKeeperCachingGetter.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCachingGetter.cpp @@ -25,14 +25,14 @@ void ZooKeeperCachingGetter::resetCache() } -std::pair ZooKeeperCachingGetter::getZooKeeper() +std::pair ZooKeeperCachingGetter::getZooKeeper() { std::lock_guard lock{cached_zookeeper_ptr_mutex}; return getZooKeeperNoLock(); } -std::pair ZooKeeperCachingGetter::getZooKeeperNoLock() +std::pair ZooKeeperCachingGetter::getZooKeeperNoLock() { if (!cached_zookeeper_ptr || cached_zookeeper_ptr->expired()) { @@ -41,9 +41,9 @@ std::pair ZooKeeperCachingGetter::getZooKeeperNoLock throw DB::Exception(DB::ErrorCodes::NO_ZOOKEEPER, "Can't get ZooKeeper session"); cached_zookeeper_ptr = zookeeper; - return {zookeeper, true}; + return {zookeeper, ZooKeeperCachingGetter::SessionStatus::New}; } - return {cached_zookeeper_ptr, false}; + return {cached_zookeeper_ptr, ZooKeeperCachingGetter::SessionStatus::Cached}; } } diff --git a/src/Common/ZooKeeper/ZooKeeperCachingGetter.h b/src/Common/ZooKeeper/ZooKeeperCachingGetter.h index ef845db8e2b..936ee660d8d 100644 --- a/src/Common/ZooKeeper/ZooKeeperCachingGetter.h +++ b/src/Common/ZooKeeper/ZooKeeperCachingGetter.h @@ -8,18 +8,24 @@ namespace zkutil class ZooKeeperCachingGetter : boost::noncopyable { public: + enum class SessionStatus + { + New, + Cached + }; + explicit ZooKeeperCachingGetter(zkutil::GetZooKeeper get_zookeeper_); ZooKeeperCachingGetter(const ZooKeeperCachingGetter &) = delete; ZooKeeperCachingGetter & operator=(const ZooKeeperCachingGetter &) = delete; - /// Returns the ZooKeeper session and the flag whether it was taken from the cache(false) or opened new(true), + /// Returns the ZooKeeper session and the status whether it was taken from the cache or opened new, /// because the session has expired or the cache was empty - std::pair getZooKeeper(); + std::pair getZooKeeper(); void resetCache(); private: - std::pair getZooKeeperNoLock() TSA_REQUIRES(cached_zookeeper_ptr_mutex); + std::pair getZooKeeperNoLock() TSA_REQUIRES(cached_zookeeper_ptr_mutex); std::mutex cached_zookeeper_ptr_mutex; zkutil::ZooKeeperPtr cached_zookeeper_ptr TSA_GUARDED_BY(cached_zookeeper_ptr_mutex); diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectType.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectType.cpp deleted file mode 100644 index cba1316c35c..00000000000 --- a/src/Functions/UserDefined/UserDefinedSQLObjectType.cpp +++ /dev/null @@ -1,15 +0,0 @@ -#include "UserDefinedSQLObjectType.h" - -namespace DB -{ - -const char * toString(UserDefinedSQLObjectType type) -{ - switch (type) - { - case UserDefinedSQLObjectType::Function: - return "FUNCTION"; - } -} - -} diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectType.h b/src/Functions/UserDefined/UserDefinedSQLObjectType.h index 2efab5e7737..f7e6fff5cad 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectType.h +++ b/src/Functions/UserDefined/UserDefinedSQLObjectType.h @@ -9,6 +9,4 @@ enum class UserDefinedSQLObjectType Function }; -const char * toString(UserDefinedSQLObjectType type); - } diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp index ce2efdd6d03..6655155aff9 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp @@ -91,9 +91,9 @@ void UserDefinedSQLObjectsLoaderFromZooKeeper::stopWatchingThread() zkutil::ZooKeeperPtr UserDefinedSQLObjectsLoaderFromZooKeeper::getZooKeeper() { - auto [zookeeper, is_new_session] = zookeeper_getter.getZooKeeper(); + auto [zookeeper, session_status] = zookeeper_getter.getZooKeeper(); - if (is_new_session) + if (session_status == zkutil::ZooKeeperCachingGetter::SessionStatus::New) { /// It's possible that we connected to different [Zoo]Keeper instance /// so we may read a bit stale state. @@ -135,7 +135,7 @@ void UserDefinedSQLObjectsLoaderFromZooKeeper::processWatchQueue() { try { - std::pair watched_object; + UserDefinedSQLObjectTypeAndName watched_object; /// Re-initialize ZooKeeper session if expired and refresh objects initZooKeeperIfNeeded(); @@ -276,8 +276,11 @@ bool UserDefinedSQLObjectsLoaderFromZooKeeper::getObjectDataAndSetWatch( { const auto object_watcher = [watch_queue = watch_queue, object_type, object_name](const Coordination::WatchResponse & response) { - if (response.type == Coordination::Event::CHANGED) [[maybe_unused]] - bool push_result = watch_queue->emplace(object_type, object_name); + if (response.type == Coordination::Event::CHANGED) + { + [[maybe_unused]] bool inserted = watch_queue->emplace(object_type, object_name); + chassert(inserted); + } /// NOTE: Event::DELETED is processed as child event by getChildren watch }; @@ -335,7 +338,10 @@ Strings UserDefinedSQLObjectsLoaderFromZooKeeper::getObjectNamesAndSetWatch( const zkutil::ZooKeeperPtr & zookeeper, UserDefinedSQLObjectType object_type) { auto object_list_watcher = [watch_queue = watch_queue, object_type](const Coordination::WatchResponse &) - { [[maybe_unused]] bool push_result = watch_queue->emplace(object_type, ""); }; + { + [[maybe_unused]] bool inserted = watch_queue->emplace(object_type, ""); + chassert(inserted); + }; Coordination::Stat stat; const auto path = getRootNodePath(zookeeper_path, object_type); @@ -365,7 +371,7 @@ void UserDefinedSQLObjectsLoaderFromZooKeeper::refreshAllObjects(const zkutil::Z void UserDefinedSQLObjectsLoaderFromZooKeeper::refreshObjects(const zkutil::ZooKeeperPtr & zookeeper, UserDefinedSQLObjectType object_type) { - LOG_DEBUG(log, "Refreshing all user-defined {} objects", toString(object_type)); + LOG_DEBUG(log, "Refreshing all user-defined {} objects", object_type); Strings object_names = getObjectNamesAndSetWatch(zookeeper, object_type); /// Read & parse all SQL objects from ZooKeeper @@ -378,12 +384,12 @@ void UserDefinedSQLObjectsLoaderFromZooKeeper::refreshObjects(const zkutil::ZooK UserDefinedSQLFunctionFactory::instance().setAllFunctions(function_names_and_asts); - LOG_DEBUG(log, "All user-defined {} objects refreshed", toString(object_type)); + LOG_DEBUG(log, "All user-defined {} objects refreshed", object_type); } void UserDefinedSQLObjectsLoaderFromZooKeeper::syncObjects(const zkutil::ZooKeeperPtr & zookeeper, UserDefinedSQLObjectType object_type) { - LOG_DEBUG(log, "Syncing user-defined {} objects", toString(object_type)); + LOG_DEBUG(log, "Syncing user-defined {} objects", object_type); Strings object_names = getObjectNamesAndSetWatch(zookeeper, object_type); auto & factory = UserDefinedSQLFunctionFactory::instance(); @@ -398,7 +404,7 @@ void UserDefinedSQLObjectsLoaderFromZooKeeper::syncObjects(const zkutil::ZooKeep refreshObject(zookeeper, UserDefinedSQLObjectType::Function, function_name); } - LOG_DEBUG(log, "User-defined {} objects synced", toString(object_type)); + LOG_DEBUG(log, "User-defined {} objects synced", object_type); } void UserDefinedSQLObjectsLoaderFromZooKeeper::refreshObject( diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.h b/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.h index 16a78c24dac..38e061fd4d9 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.h +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.h @@ -71,7 +71,9 @@ private: ThreadFromGlobalPool watching_thread; std::atomic watching_flag = false; - std::shared_ptr>> watch_queue; + + using UserDefinedSQLObjectTypeAndName = std::pair; + std::shared_ptr> watch_queue; Poco::Logger * log; }; diff --git a/tests/integration/test_backup_restore_on_cluster/_gen/cluster_for_concurrency_test.xml b/tests/integration/test_backup_restore_on_cluster/_gen/cluster_for_concurrency_test.xml deleted file mode 100644 index 08684e34e45..00000000000 --- a/tests/integration/test_backup_restore_on_cluster/_gen/cluster_for_concurrency_test.xml +++ /dev/null @@ -1,48 +0,0 @@ - - - - - - node0 - 9000 - - - node1 - 9000 - - - node2 - 9000 - - - node3 - 9000 - - - node4 - 9000 - - - node5 - 9000 - - - node6 - 9000 - - - node7 - 9000 - - - node8 - 9000 - - - node9 - 9000 - - - - - \ No newline at end of file From e2af8f6ed6665e184399f864fdb4299711986891 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 17 Mar 2023 11:59:44 +0100 Subject: [PATCH 220/559] add a comment, rename a field --- src/Common/ThreadStatus.cpp | 12 ++++++------ src/Common/ThreadStatus.h | 3 ++- src/Interpreters/ThreadStatusExt.cpp | 16 ++++++++-------- 3 files changed, 16 insertions(+), 15 deletions(-) diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 5b858ef5532..a42211c612b 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -159,29 +159,29 @@ void ThreadStatus::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & if (!thread_group) throw Exception(ErrorCodes::LOGICAL_ERROR, "No thread group attached to the thread {}", thread_id); - shared_data.logs_queue_ptr = logs_queue; - shared_data.client_logs_level = logs_level; + local_data.logs_queue_ptr = logs_queue; + local_data.client_logs_level = logs_level; thread_group->attachInternalTextLogsQueue(logs_queue, logs_level); } InternalTextLogsQueuePtr ThreadStatus::getInternalTextLogsQueue() const { - return shared_data.logs_queue_ptr.lock(); + return local_data.logs_queue_ptr.lock(); } InternalProfileEventsQueuePtr ThreadStatus::getInternalProfileEventsQueue() const { - return shared_data.profile_queue_ptr.lock(); + return local_data.profile_queue_ptr.lock(); } const String & ThreadStatus::getQueryForLog() const { - return shared_data.query_for_logs; + return local_data.query_for_logs; } LogsLevel ThreadStatus::getClientLogsLevel() const { - return shared_data.client_logs_level; + return local_data.client_logs_level; } void ThreadStatus::flushUntrackedMemory() diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 4815a6e0971..77c924f9650 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -92,6 +92,7 @@ public: SharedData getSharedData() { + /// Critical section for making the copy of shared_data std::lock_guard lock(mutex); return shared_data; } @@ -170,7 +171,7 @@ private: using FatalErrorCallback = std::function; FatalErrorCallback fatal_error_callback; - ThreadGroupStatus::SharedData shared_data; + ThreadGroupStatus::SharedData local_data; bool performance_counters_finalized = false; diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index bee7f55887f..088a35c7f1c 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -84,13 +84,13 @@ void ThreadGroupStatus::attachQueryForLog(const String & query_, UInt64 normaliz void ThreadStatus::attachQueryForLog(const String & query_) { - shared_data.query_for_logs = query_; - shared_data.normalized_query_hash = normalizedQueryHash(query_); + local_data.query_for_logs = query_; + local_data.normalized_query_hash = normalizedQueryHash(query_); if (!thread_group) throw Exception(ErrorCodes::LOGICAL_ERROR, "No thread group attached to the thread {}", thread_id); - thread_group->attachQueryForLog(shared_data.query_for_logs, shared_data.normalized_query_hash); + thread_group->attachQueryForLog(local_data.query_for_logs, local_data.normalized_query_hash); } void ThreadGroupStatus::attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue) @@ -104,7 +104,7 @@ void ThreadStatus::attachInternalProfileEventsQueue(const InternalProfileEventsQ if (!thread_group) throw Exception(ErrorCodes::LOGICAL_ERROR, "No thread group attached to the thread {}", thread_id); - shared_data.profile_queue_ptr = profile_queue; + local_data.profile_queue_ptr = profile_queue; thread_group->attachInternalProfileEventsQueue(profile_queue); } @@ -167,7 +167,7 @@ void ThreadStatus::attachToGroupImpl(const ThreadGroupStatusPtr & thread_group_) fatal_error_callback = thread_group->fatal_error_callback; - shared_data = thread_group->getSharedData(); + local_data = thread_group->getSharedData(); applyQuerySettings(); initPerformanceCounters(); @@ -196,7 +196,7 @@ void ThreadStatus::detachFromGroup() query_id_from_query_context.clear(); query_context.reset(); - shared_data = {}; + local_data = {}; fatal_error_callback = {}; @@ -447,8 +447,8 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log, const String if (thread_group) { elem.master_thread_id = thread_group->master_thread_id; - elem.query = shared_data.query_for_logs; - elem.normalized_query_hash = shared_data.normalized_query_hash; + elem.query = local_data.query_for_logs; + elem.normalized_query_hash = local_data.normalized_query_hash; } auto query_context_ptr = query_context.lock(); From d7b2ea60ebc88ac1a370164661479de0b08e655b Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 17 Mar 2023 13:56:02 +0100 Subject: [PATCH 221/559] Review fixes --- src/Storages/NamedCollectionsHelpers.cpp | 4 ++-- src/Storages/StorageExternalDistributed.cpp | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/NamedCollectionsHelpers.cpp b/src/Storages/NamedCollectionsHelpers.cpp index c6e928d9412..83128ab025a 100644 --- a/src/Storages/NamedCollectionsHelpers.cpp +++ b/src/Storages/NamedCollectionsHelpers.cpp @@ -61,7 +61,7 @@ namespace } auto value = literal_value->as()->value; - return std::pair{key, value}; + return std::pair{key, Field(value)}; } } @@ -85,7 +85,7 @@ MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides( for (auto * it = std::next(asts.begin()); it != asts.end(); ++it) { - auto value_override = getKeyValueFromAST(*it, complex_args != nullptr, context); + auto value_override = getKeyValueFromAST(*it, /* fallback_to_ast_value */complex_args != nullptr, context); if (!value_override && !(*it)->as()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value argument or function"); diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp index 45ca659a8fe..db1f33193ac 100644 --- a/src/Storages/StorageExternalDistributed.cpp +++ b/src/Storages/StorageExternalDistributed.cpp @@ -102,8 +102,8 @@ void registerStorageExternalDistributed(StorageFactory & factory) "engine_name, named_collection and/or description"); auto context = args.getLocalContext(); - [[maybe_unused]] const auto & settings = context->getSettingsRef(); - size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements; + const auto & settings = context->getSettingsRef(); + size_t max_addresses = settings.glob_expansion_max_elements; auto get_addresses = [&](const std::string addresses_expr) { return parseRemoteDescription(addresses_expr, 0, addresses_expr.size(), ',', max_addresses); From 3ee3666db71173e5a381193ec03b3b81e73c20f4 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 17 Mar 2023 13:02:15 +0000 Subject: [PATCH 222/559] fix clang-tidy --- src/Functions/getSubcolumn.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/getSubcolumn.cpp b/src/Functions/getSubcolumn.cpp index a8998056d33..b2f63ea4c75 100644 --- a/src/Functions/getSubcolumn.cpp +++ b/src/Functions/getSubcolumn.cpp @@ -38,7 +38,7 @@ public: } private: - std::string_view getSubcolumnName(const ColumnsWithTypeAndName & arguments) const + static std::string_view getSubcolumnName(const ColumnsWithTypeAndName & arguments) { const auto * column = arguments[1].column.get(); if (!isString(arguments[1].type) || !column || !checkAndGetColumnConstStringOrFixedString(column)) From f8dab850a988498497bf562b25f202dbb2255acd Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 15 Mar 2023 11:37:25 +0000 Subject: [PATCH 223/559] Check for duplicate column in BSONEachRow format --- src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp | 3 +++ .../queries/0_stateless/02590_bson_duplicate_column.reference | 0 tests/queries/0_stateless/02590_bson_duplicate_column.sql | 1 + 3 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/02590_bson_duplicate_column.reference create mode 100644 tests/queries/0_stateless/02590_bson_duplicate_column.sql diff --git a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp index 02fe58094ae..f768d3b568f 100644 --- a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp @@ -786,6 +786,9 @@ bool BSONEachRowRowInputFormat::readRow(MutableColumns & columns, RowReadExtensi } else { + if (seen_columns[index]) + throw Exception(ErrorCodes::INCORRECT_DATA, "Duplicate field found while parsing BSONNEachRow format: {}", name); + seen_columns[index] = true; read_columns[index] = readField(*columns[index], types[index], BSONType(type)); } diff --git a/tests/queries/0_stateless/02590_bson_duplicate_column.reference b/tests/queries/0_stateless/02590_bson_duplicate_column.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02590_bson_duplicate_column.sql b/tests/queries/0_stateless/02590_bson_duplicate_column.sql new file mode 100644 index 00000000000..decc6b27e8b --- /dev/null +++ b/tests/queries/0_stateless/02590_bson_duplicate_column.sql @@ -0,0 +1 @@ +select * from format(BSONEachRow, x'130000001078002a0000001078002a00000000'); -- {serverError INCORRECT_DATA} From 4522b74fc68e9bf26dc3f776ef74fd775e67e0cd Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 15 Mar 2023 12:44:58 +0100 Subject: [PATCH 224/559] Fix typo --- src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp index f768d3b568f..2a5b517cd1c 100644 --- a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp @@ -787,7 +787,7 @@ bool BSONEachRowRowInputFormat::readRow(MutableColumns & columns, RowReadExtensi else { if (seen_columns[index]) - throw Exception(ErrorCodes::INCORRECT_DATA, "Duplicate field found while parsing BSONNEachRow format: {}", name); + throw Exception(ErrorCodes::INCORRECT_DATA, "Duplicate field found while parsing BSONEachRow format: {}", name); seen_columns[index] = true; read_columns[index] = readField(*columns[index], types[index], BSONType(type)); From ee8bbb468463c7121fc9b61ef963d306a21f3a4c Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 16 Mar 2023 12:19:01 +0100 Subject: [PATCH 225/559] Update test --- tests/queries/0_stateless/02590_bson_duplicate_column.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02590_bson_duplicate_column.sql b/tests/queries/0_stateless/02590_bson_duplicate_column.sql index decc6b27e8b..ea70fb9ba68 100644 --- a/tests/queries/0_stateless/02590_bson_duplicate_column.sql +++ b/tests/queries/0_stateless/02590_bson_duplicate_column.sql @@ -1 +1 @@ -select * from format(BSONEachRow, x'130000001078002a0000001078002a00000000'); -- {serverError INCORRECT_DATA} +select * from format(BSONEachRow, 'x UInt32, y UInt32', x'1a0000001078002a0000001078002a0000001079002a00000000'); -- {serverError INCORRECT_DATA} From dfd8b2670a14f02e1c07e50c5f9035d1a7d785e6 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 17 Mar 2023 14:15:03 +0100 Subject: [PATCH 226/559] fix tests, update docs --- docs/en/sql-reference/statements/system.md | 6 +++--- docs/ru/sql-reference/statements/system.md | 6 +++--- .../test_replicated_merge_tree_s3_zero_copy/test.py | 7 ++++++- .../00002_log_and_exception_messages_formatting.sql | 3 ++- 4 files changed, 14 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index f9f55acfcec..14fa7bfd8e2 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -280,13 +280,13 @@ SYSTEM START REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] ### SYNC REPLICA -Wait until a `ReplicatedMergeTree` table will be synced with other replicas in a cluster. Will run until `receive_timeout` if fetches currently disabled for the table. +Wait until a `ReplicatedMergeTree` table will be synced with other replicas in a cluster, but no more than `receive_timeout` seconds. ``` sql -SYSTEM SYNC REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name +SYSTEM SYNC REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name [STRICT] ``` -After running this statement the `[db.]replicated_merge_tree_family_table_name` fetches commands from the common replicated log into its own replication queue, and then the query waits till the replica processes all of the fetched commands. +After running this statement the `[db.]replicated_merge_tree_family_table_name` fetches commands from the common replicated log into its own replication queue, and then the query waits till the replica processes all of the fetched commands. If a `STRICT` modifier was specified then the query waits for the replication queue to become empty. The `STRICT` version may never succeed if new entries constantly appear in the replication queue. ### RESTART REPLICA diff --git a/docs/ru/sql-reference/statements/system.md b/docs/ru/sql-reference/statements/system.md index a7dec7abe27..847f60ca35c 100644 --- a/docs/ru/sql-reference/statements/system.md +++ b/docs/ru/sql-reference/statements/system.md @@ -269,13 +269,13 @@ SYSTEM START REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] ### SYNC REPLICA {#query_language-system-sync-replica} -Ждет когда таблица семейства `ReplicatedMergeTree` будет синхронизирована с другими репликами в кластере, будет работать до достижения `receive_timeout`, если синхронизация для таблицы отключена в настоящий момент времени: +Ждет когда таблица семейства `ReplicatedMergeTree` будет синхронизирована с другими репликами в кластере, но не более `receive_timeout` секунд: ``` sql -SYSTEM SYNC REPLICA [db.]replicated_merge_tree_family_table_name +SYSTEM SYNC REPLICA [db.]replicated_merge_tree_family_table_name [STRICT] ``` -После выполнения этого запроса таблица `[db.]replicated_merge_tree_family_table_name` синхронизирует команды из общего реплицированного лога в свою собственную очередь репликации. Затем запрос ждет, пока реплика не обработает все синхронизированные команды. +После выполнения этого запроса таблица `[db.]replicated_merge_tree_family_table_name` загружает команды из общего реплицированного лога в свою собственную очередь репликации. Затем запрос ждет, пока реплика не обработает все загруженные команды. Если указан модификатор `STRICT`, то запрос ждёт когда очередь репликации станет пустой. Строгий вариант запроса может никогда не завершиться успешно, если в очереди репликации постоянно появляются новые записи. ### RESTART REPLICA {#query_language-system-restart-replica} diff --git a/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py b/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py index 1941ec734c2..1a5f2e12706 100644 --- a/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py +++ b/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py @@ -192,11 +192,16 @@ def test_drop_table(cluster): "create table if not exists test_drop_table (n int) engine=ReplicatedMergeTree('/test/drop_table', '1') " "order by n partition by n % 99 settings storage_policy='s3'" ) + + # A table may get stuck in readonly mode if zk connection was lost during CREATE + node.query("detach table test_drop_table sync") + node.query("attach table test_drop_table") + node.query_with_retry( "system sync replica test_drop_table", settings={"receive_timeout": 5}, sleep_time=5, - retry_count=20, + retry_count=10, ) node2.query("drop table test_drop_table sync") assert "1000\t499500\n" == node.query( diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index 12870361cfe..2bcceec8cbe 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -57,7 +57,8 @@ select 'number of noisy messages', max2(count(), 10) from (select count() / (sel -- FIXME maybe we should make it stricter ('Code:%Exception: '||s||'%'), but it's not easy because of addMessage select 'incorrect patterns', max2(countDistinct(message_format_string), 15) from ( select message_format_string, any(message) as any_message from logs - where message not like (replaceRegexpAll(message_format_string, '{[:.0-9dfx]*}', '%') as s) + where ((rand() % 8) = 0) + and message not like (replaceRegexpAll(message_format_string, '{[:.0-9dfx]*}', '%') as s) and message not like (s || ' (skipped % similar messages)') and message not like ('%Exception: '||s||'%') group by message_format_string ) where any_message not like '%Poco::Exception%'; From 2d2696bc208d820bfe98db1ded31135794c7190e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 17 Mar 2023 14:34:23 +0100 Subject: [PATCH 227/559] Fixed code review issues --- src/Planner/Planner.cpp | 2 + .../Optimizations/optimizePrewhere.cpp | 48 +++++++------------ .../MergeTreeBaseSelectProcessor.cpp | 4 +- .../MergeTree/MergeTreeWhereOptimizer.cpp | 26 +++++----- .../MergeTree/MergeTreeWhereOptimizer.h | 6 +-- 5 files changed, 36 insertions(+), 50 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 1e5347b81fe..b79fe9bcd46 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -84,6 +84,8 @@ namespace ErrorCodes /** ClickHouse query planner. * * TODO: Support projections. + * TODO: Support trivial count using partition predicates. + * TODO: Support trivial count for table functions. * TODO: Support indexes for IN function. */ diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index 477aa6a7c75..557b76293ea 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -18,7 +18,7 @@ namespace ErrorCodes namespace { -void matchDAGOutputNodesWithHeader(ActionsDAGPtr & actions_dag, const Block & expected_header) +void matchDAGOutputNodesOrderWithHeader(ActionsDAGPtr & actions_dag, const Block & expected_header) { std::unordered_map output_name_to_node; for (const auto * output_node : actions_dag->getOutputs()) @@ -97,39 +97,23 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) for (const auto * output_node : filter_step->getExpression()->getOutputs()) { - if (output_node->type == ActionsDAG::ActionType::INPUT) - { - output_nodes_mapped_to_input.insert(output_node->result_name); + const auto * node_without_alias = output_node; + while (node_without_alias->type == ActionsDAG::ActionType::ALIAS) + node_without_alias = node_without_alias->children[0]; - auto output_names_it = input_node_to_output_names.find(output_node->result_name); + if (node_without_alias->type == ActionsDAG::ActionType::INPUT) + { + output_nodes_mapped_to_input.emplace(output_node->result_name); + + auto output_names_it = input_node_to_output_names.find(node_without_alias->result_name); if (output_names_it == input_node_to_output_names.end()) { - auto [insert_it, _] = input_node_to_output_names.emplace(output_node->result_name, std::vector()); + auto [insert_it, _] = input_node_to_output_names.emplace(node_without_alias->result_name, std::vector()); output_names_it = insert_it; } output_names_it->second.push_back(output_node->result_name); } - else if (output_node->type == ActionsDAG::ActionType::ALIAS) - { - const auto * node_without_alias = output_node; - while (node_without_alias->type == ActionsDAG::ActionType::ALIAS) - node_without_alias = node_without_alias->children[0]; - - if (node_without_alias->type == ActionsDAG::ActionType::INPUT) - { - output_nodes_mapped_to_input.emplace(output_node->result_name); - - auto output_names_it = input_node_to_output_names.find(node_without_alias->result_name); - if (output_names_it == input_node_to_output_names.end()) - { - auto [insert_it, _] = input_node_to_output_names.emplace(node_without_alias->result_name, std::vector()); - output_names_it = insert_it; - } - - output_names_it->second.push_back(output_node->result_name); - } - } if (output_node->result_name == filter_step->getFilterColumnName() && filter_step->removesFilterColumn()) continue; @@ -173,7 +157,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) filter_step->getFilterColumnName(), read_from_merge_tree->getContext(), is_final); - if (!optimize_result) + if (!optimize_result.has_value()) return; PrewhereInfoPtr prewhere_info; @@ -201,10 +185,10 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) * enough to produce required filter output columns. * * Example: SELECT (a AND b) AS cond FROM test_table WHERE cond AND c; - * In this example conditions a, b, c can move to PREWHERE, but PREWHERE will not contain expression and(a, b). - * It will contain only a, b, c, and(a, b, c) expressions. + * In this example condition expressions `a`, `b`, `c` can move to PREWHERE, but PREWHERE will not contain expression `and(a, b)`. + * It will contain only `a`, `b`, `c`, `and(a, b, c)` expressions. * - * In such scenario we need to create additional filter expressions after PREWHERE. + * In such scenario we need to create additional step to calculate `and(a, b)` expression after PREWHERE. */ bool need_additional_filter_after_prewhere = false; @@ -277,7 +261,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) filter_actions->projectInput(false); /// Match dag output nodes with old filter step header - matchDAGOutputNodesWithHeader(filter_actions, filter_step->getOutputStream().header); + matchDAGOutputNodesOrderWithHeader(filter_actions, filter_step->getOutputStream().header); auto & filter_actions_chain_node = actions_chain[1]; bool remove_filter_column = !filter_actions_chain_node->getChildRequiredOutputColumnsNames().contains(filter_step->getFilterColumnName()); @@ -332,7 +316,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) if (!blocksHaveEqualStructure(read_from_merge_tree->getOutputStream().header, filter_step->getOutputStream().header)) { apply_match_step = true; - matchDAGOutputNodesWithHeader(rename_actions_dag, filter_step->getOutputStream().header); + matchDAGOutputNodesOrderWithHeader(rename_actions_dag, filter_step->getOutputStream().header); } if (apply_rename_step || apply_match_step) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 5d317d8ee90..a41b697f9e6 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -654,9 +654,9 @@ Block IMergeTreeSelectAlgorithm::applyPrewhereActions(Block block, const Prewher WhichDataType which(removeNullable(recursiveRemoveLowCardinality(prewhere_column.type))); if (which.isNativeInt() || which.isNativeUInt()) - prewhere_column.column = prewhere_column.type->createColumnConst(block.rows(), 1u); + prewhere_column.column = prewhere_column.type->createColumnConst(block.rows(), 1u)->convertToFullColumnIfConst(); else if (which.isFloat()) - prewhere_column.column = prewhere_column.type->createColumnConst(block.rows(), 1.0f); + prewhere_column.column = prewhere_column.type->createColumnConst(block.rows(), 1.0f)->convertToFullColumnIfConst(); else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, "Illegal type {} of column for filter", diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index f9a49c35759..9c4c70a36b2 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -212,7 +212,7 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTree bool has_invalid_column = false; collectColumns(node, table_columns, cond.table_columns, has_invalid_column); - cond.columns_size = getIdentifiersColumnSize(cond.table_columns); + cond.columns_size = getColumnsSize(cond.table_columns); cond.viable = !has_invalid_column && @@ -223,7 +223,7 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTree /// need to use all sorting keys, it will cause correctness issues if we filter other columns before final merge. && (!where_optimizer_context.is_final || isExpressionOverSortingKey(node)) /// Some identifiers can unable to support PREWHERE (usually because of different types in Merge engine) - && identifiersSupportsPrewhere(cond.table_columns) + && columnsSupportPrewhere(cond.table_columns) /// Do not move conditions involving all queried columns. && cond.table_columns.size() < queried_columns.size(); @@ -347,24 +347,24 @@ std::optional MergeTreeWhereOptimizer:: } -UInt64 MergeTreeWhereOptimizer::getIdentifiersColumnSize(const NameSet & identifiers) const +UInt64 MergeTreeWhereOptimizer::getColumnsSize(const NameSet & columns) const { UInt64 size = 0; - for (const auto & identifier : identifiers) - if (column_sizes.contains(identifier)) - size += column_sizes.at(identifier); + for (const auto & column : columns) + if (column_sizes.contains(column)) + size += column_sizes.at(column); return size; } -bool MergeTreeWhereOptimizer::identifiersSupportsPrewhere(const NameSet & identifiers) const +bool MergeTreeWhereOptimizer::columnsSupportPrewhere(const NameSet & columns) const { if (!supported_columns.has_value()) return true; - for (const auto & identifier : identifiers) - if (!supported_columns->contains(identifier)) + for (const auto & column : columns) + if (!supported_columns->contains(column)) return false; return true; @@ -400,10 +400,10 @@ bool MergeTreeWhereOptimizer::isSortingKey(const String & column_name) const return sorting_key_names.contains(column_name); } -bool MergeTreeWhereOptimizer::isSubsetOfTableColumns(const NameSet & identifiers) const +bool MergeTreeWhereOptimizer::isSubsetOfTableColumns(const NameSet & columns) const { - for (const auto & identifier : identifiers) - if (!table_columns.contains(identifier)) + for (const auto & column : columns) + if (!table_columns.contains(column)) return false; return true; @@ -444,7 +444,7 @@ bool MergeTreeWhereOptimizer::cannotBeMoved(const RPNBuilderTreeNode & node, con /// disallow moving result of ARRAY JOIN to PREWHERE if (where_optimizer_context.array_joined_names.contains(column_name) || where_optimizer_context.array_joined_names.contains(Nested::extractTableName(column_name)) || - (where_optimizer_context.is_final && !isSortingKey(column_name))) + (table_columns.contains(column_name) && where_optimizer_context.is_final && !isSortingKey(column_name))) return true; } diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index 7db2d3a9073..18555a72db1 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -115,9 +115,9 @@ private: void optimizeArbitrary(ASTSelectQuery & select) const; - UInt64 getIdentifiersColumnSize(const NameSet & identifiers) const; + UInt64 getColumnsSize(const NameSet & columns) const; - bool identifiersSupportsPrewhere(const NameSet & identifiers) const; + bool columnsSupportPrewhere(const NameSet & columns) const; bool isExpressionOverSortingKey(const RPNBuilderTreeNode & node) const; @@ -125,7 +125,7 @@ private: bool isConstant(const ASTPtr & expr) const; - bool isSubsetOfTableColumns(const NameSet & identifiers) const; + bool isSubsetOfTableColumns(const NameSet & columns) const; /** ARRAY JOIN'ed columns as well as arrayJoin() result cannot be used in PREWHERE, therefore expressions * containing said columns should not be moved to PREWHERE at all. From f72a337074267f6292578e9b48de93f5f0e9e7e0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 17 Mar 2023 13:27:48 +0000 Subject: [PATCH 228/559] Remove cruft from build No need to check compiler flags, clang >= 15 supports all of them. --- CMakeLists.txt | 26 ++++++------------- base/base/phdr_cache.cpp | 2 -- base/base/unit.h | 10 +++---- base/readpassphrase/readpassphrase.c | 2 -- cmake/check_flags.cmake | 6 ++--- src/Client/LineReader.cpp | 2 -- src/Common/ThreadFuzzer.cpp | 2 -- src/Common/UTF8Helpers.h | 4 +-- src/Common/examples/int_hashes_perf.cpp | 2 -- src/Common/memcmpSmall.h | 4 +-- src/Common/memcpySmall.h | 4 +-- .../CompressionCodecDoubleDelta.cpp | 2 -- src/Compression/CompressionCodecGorilla.cpp | 2 -- src/Daemon/BaseDaemon.cpp | 2 -- src/Functions/FunctionsCodingIP.cpp | 2 -- src/Functions/GatherUtils/CMakeLists.txt | 8 ------ src/Functions/GatherUtils/Sources.h | 8 ++---- src/Functions/toValidUTF8.cpp | 4 +-- src/IO/ReadBufferFromFileDescriptor.cpp | 2 -- src/IO/ReadHelpers.cpp | 4 +-- src/IO/WriteBufferValidUTF8.cpp | 4 +-- src/IO/tests/gtest_bit_io.cpp | 2 -- .../Formats/Impl/ArrowBufferedStreams.cpp | 2 -- .../MergeTree/MergeTreeRangeReader.cpp | 4 +-- utils/memcpy-bench/memcpy-bench.cpp | 2 -- 25 files changed, 22 insertions(+), 90 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 6accb2e09b7..23df42b20ea 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -121,6 +121,7 @@ if (ENABLE_COLORED_BUILD AND CMAKE_GENERATOR STREQUAL "Ninja") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fdiagnostics-color=always") # ... such manually setting of flags can be removed once CMake supports a variable to # activate colors in *all* build systems: https://gitlab.kitware.com/cmake/cmake/-/issues/15502 + # --> available since CMake 3.24: https://stackoverflow.com/a/73349744 endif () include (cmake/check_flags.cmake) @@ -134,24 +135,15 @@ if (COMPILER_CLANG) set(COMPILER_FLAGS "${COMPILER_FLAGS} -gdwarf-aranges") endif () - if (HAS_USE_CTOR_HOMING) - # For more info see https://blog.llvm.org/posts/2021-04-05-constructor-homing-for-debug-info/ - if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO") - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Xclang -fuse-ctor-homing") - set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Xclang -fuse-ctor-homing") - endif() + # See https://blog.llvm.org/posts/2021-04-05-constructor-homing-for-debug-info/ + if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO") + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Xclang -fuse-ctor-homing") + set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Xclang -fuse-ctor-homing") endif() no_warning(enum-constexpr-conversion) # breaks Protobuf in clang-16 endif () -# If compiler has support for -Wreserved-identifier. It is difficult to detect by clang version, -# because there are two different branches of clang: clang and AppleClang. -# (AppleClang is not supported by ClickHouse, but some developers have misfortune to use it). -if (HAS_RESERVED_IDENTIFIER) - add_compile_definitions (HAS_RESERVED_IDENTIFIER) -endif () - option(ENABLE_TESTS "Provide unit_test_dbms target with Google.Test unit tests" ON) option(ENABLE_EXAMPLES "Build all example programs in 'examples' subdirectories" OFF) option(ENABLE_BENCHMARKS "Build all benchmark programs in 'benchmarks' subdirectories" OFF) @@ -311,6 +303,7 @@ set (CMAKE_C_STANDARD_REQUIRED ON) if (COMPILER_GCC OR COMPILER_CLANG) # Enable C++14 sized global deallocation functions. It should be enabled by setting -std=c++14 but I'm not sure. + # See https://reviews.llvm.org/D112921 set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fsized-deallocation") endif () @@ -329,11 +322,7 @@ if (ARCH_AMD64) set(BRANCHES_WITHIN_32B_BOUNDARIES "-Wa,${BRANCHES_WITHIN_32B_BOUNDARIES}") endif() - include(CheckCXXCompilerFlag) - check_cxx_compiler_flag("${BRANCHES_WITHIN_32B_BOUNDARIES}" HAS_BRANCHES_WITHIN_32B_BOUNDARIES) - if (HAS_BRANCHES_WITHIN_32B_BOUNDARIES) - set(COMPILER_FLAGS "${COMPILER_FLAGS} ${BRANCHES_WITHIN_32B_BOUNDARIES}") - endif() + set(COMPILER_FLAGS "${COMPILER_FLAGS} ${BRANCHES_WITHIN_32B_BOUNDARIES}") endif() if (COMPILER_GCC) @@ -445,6 +434,7 @@ option(WERROR "Enable -Werror compiler option" ON) if (WERROR) # Don't pollute CMAKE_CXX_FLAGS with -Werror as it will break some CMake checks. # Instead, adopt modern cmake usage requirement. + # TODO: Set CMAKE_COMPILE_WARNING_AS_ERROR (cmake 3.24) target_compile_options(global-group INTERFACE "-Werror") endif () diff --git a/base/base/phdr_cache.cpp b/base/base/phdr_cache.cpp index c3d7fed2d3f..7d37f01b560 100644 --- a/base/base/phdr_cache.cpp +++ b/base/base/phdr_cache.cpp @@ -1,6 +1,4 @@ -#ifdef HAS_RESERVED_IDENTIFIER #pragma clang diagnostic ignored "-Wreserved-identifier" -#endif /// This code was based on the code by Fedor Korotkiy https://www.linkedin.com/in/fedor-korotkiy-659a1838/ diff --git a/base/base/unit.h b/base/base/unit.h index 1fb530be1f0..0fc314af479 100644 --- a/base/base/unit.h +++ b/base/base/unit.h @@ -5,10 +5,8 @@ constexpr size_t KiB = 1024; constexpr size_t MiB = 1024 * KiB; constexpr size_t GiB = 1024 * MiB; -#ifdef HAS_RESERVED_IDENTIFIER -# pragma clang diagnostic push -# pragma clang diagnostic ignored "-Wreserved-identifier" -#endif +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wreserved-identifier" // NOLINTBEGIN(google-runtime-int) constexpr size_t operator"" _KiB(unsigned long long val) { return val * KiB; } @@ -16,6 +14,4 @@ constexpr size_t operator"" _MiB(unsigned long long val) { return val * MiB; } constexpr size_t operator"" _GiB(unsigned long long val) { return val * GiB; } // NOLINTEND(google-runtime-int) -#ifdef HAS_RESERVED_IDENTIFIER -# pragma clang diagnostic pop -#endif +#pragma clang diagnostic pop diff --git a/base/readpassphrase/readpassphrase.c b/base/readpassphrase/readpassphrase.c index a84ec43767c..fbd582ffe79 100644 --- a/base/readpassphrase/readpassphrase.c +++ b/base/readpassphrase/readpassphrase.c @@ -27,9 +27,7 @@ #define _PATH_TTY "/dev/tty" #endif -#ifdef HAS_RESERVED_IDENTIFIER #pragma clang diagnostic ignored "-Wreserved-identifier" -#endif #include #include diff --git a/cmake/check_flags.cmake b/cmake/check_flags.cmake index 518f9ecf8de..294f135e8ee 100644 --- a/cmake/check_flags.cmake +++ b/cmake/check_flags.cmake @@ -1,7 +1,5 @@ include (CheckCXXCompilerFlag) include (CheckCCompilerFlag) -check_cxx_compiler_flag("-Wreserved-identifier" HAS_RESERVED_IDENTIFIER) -check_cxx_compiler_flag("-Wsuggest-destructor-override" HAS_SUGGEST_DESTRUCTOR_OVERRIDE) -check_cxx_compiler_flag("-Wsuggest-override" HAS_SUGGEST_OVERRIDE) -check_cxx_compiler_flag("-Xclang -fuse-ctor-homing" HAS_USE_CTOR_HOMING) +# Set/unset variable based on existence of compiler flags. Example: +# check_cxx_compiler_flag("-Wreserved-identifier" HAS_RESERVED_IDENTIFIER) diff --git a/src/Client/LineReader.cpp b/src/Client/LineReader.cpp index f49e48be617..04b387c9f7d 100644 --- a/src/Client/LineReader.cpp +++ b/src/Client/LineReader.cpp @@ -12,9 +12,7 @@ #include -#ifdef HAS_RESERVED_IDENTIFIER #pragma clang diagnostic ignored "-Wreserved-identifier" -#endif namespace { diff --git a/src/Common/ThreadFuzzer.cpp b/src/Common/ThreadFuzzer.cpp index df6f860e588..fd6ef12693d 100644 --- a/src/Common/ThreadFuzzer.cpp +++ b/src/Common/ThreadFuzzer.cpp @@ -34,9 +34,7 @@ M(int, pthread_mutex_unlock, pthread_mutex_t * arg) #endif -#ifdef HAS_RESERVED_IDENTIFIER #pragma clang diagnostic ignored "-Wreserved-identifier" -#endif namespace DB { diff --git a/src/Common/UTF8Helpers.h b/src/Common/UTF8Helpers.h index 623a62a6f79..1dac8f60c5e 100644 --- a/src/Common/UTF8Helpers.h +++ b/src/Common/UTF8Helpers.h @@ -11,9 +11,7 @@ #if defined(__aarch64__) && defined(__ARM_NEON) # include -# ifdef HAS_RESERVED_IDENTIFIER -# pragma clang diagnostic ignored "-Wreserved-identifier" -# endif +# pragma clang diagnostic ignored "-Wreserved-identifier" #endif diff --git a/src/Common/examples/int_hashes_perf.cpp b/src/Common/examples/int_hashes_perf.cpp index fecfa0adba8..2b260dca809 100644 --- a/src/Common/examples/int_hashes_perf.cpp +++ b/src/Common/examples/int_hashes_perf.cpp @@ -1,6 +1,4 @@ -#ifdef HAS_RESERVED_IDENTIFIER #pragma clang diagnostic ignored "-Wreserved-identifier" -#endif #if defined (OS_LINUX) # include diff --git a/src/Common/memcmpSmall.h b/src/Common/memcmpSmall.h index e95a21b836d..e0b232a3485 100644 --- a/src/Common/memcmpSmall.h +++ b/src/Common/memcmpSmall.h @@ -502,9 +502,7 @@ inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size) #elif defined(__aarch64__) && defined(__ARM_NEON) # include -# ifdef HAS_RESERVED_IDENTIFIER -# pragma clang diagnostic ignored "-Wreserved-identifier" -# endif +# pragma clang diagnostic ignored "-Wreserved-identifier" inline uint64_t getNibbleMask(uint8x16_t res) { diff --git a/src/Common/memcpySmall.h b/src/Common/memcpySmall.h index 4f38095c7f1..5eaa1203f05 100644 --- a/src/Common/memcpySmall.h +++ b/src/Common/memcpySmall.h @@ -8,9 +8,7 @@ #if defined(__aarch64__) && defined(__ARM_NEON) # include -# ifdef HAS_RESERVED_IDENTIFIER -# pragma clang diagnostic ignored "-Wreserved-identifier" -# endif +# pragma clang diagnostic ignored "-Wreserved-identifier" #endif /** memcpy function could work suboptimal if all the following conditions are met: diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index dea15f99a5a..40271726697 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -1,6 +1,4 @@ -#ifdef HAS_RESERVED_IDENTIFIER #pragma clang diagnostic ignored "-Wreserved-identifier" -#endif #include #include diff --git a/src/Compression/CompressionCodecGorilla.cpp b/src/Compression/CompressionCodecGorilla.cpp index 568640153ac..2c6f862d38b 100644 --- a/src/Compression/CompressionCodecGorilla.cpp +++ b/src/Compression/CompressionCodecGorilla.cpp @@ -1,6 +1,4 @@ -#ifdef HAS_RESERVED_IDENTIFIER #pragma clang diagnostic ignored "-Wreserved-identifier" -#endif #include #include diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 18c4c0d97a0..aa456a65611 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -1,6 +1,4 @@ -#ifdef HAS_RESERVED_IDENTIFIER #pragma clang diagnostic ignored "-Wreserved-identifier" -#endif #include #include diff --git a/src/Functions/FunctionsCodingIP.cpp b/src/Functions/FunctionsCodingIP.cpp index 4784368db9b..fb54fb951d1 100644 --- a/src/Functions/FunctionsCodingIP.cpp +++ b/src/Functions/FunctionsCodingIP.cpp @@ -1,7 +1,5 @@ #include -#ifdef HAS_RESERVED_IDENTIFIER #pragma clang diagnostic ignored "-Wreserved-identifier" -#endif #include diff --git a/src/Functions/GatherUtils/CMakeLists.txt b/src/Functions/GatherUtils/CMakeLists.txt index fe600f86d07..376ca6613d8 100644 --- a/src/Functions/GatherUtils/CMakeLists.txt +++ b/src/Functions/GatherUtils/CMakeLists.txt @@ -4,14 +4,6 @@ add_headers_and_sources(clickhouse_functions_gatherutils .) add_library(clickhouse_functions_gatherutils ${clickhouse_functions_gatherutils_sources} ${clickhouse_functions_gatherutils_headers}) target_link_libraries(clickhouse_functions_gatherutils PRIVATE dbms) -if (HAS_SUGGEST_OVERRIDE) - target_compile_definitions(clickhouse_functions_gatherutils PUBLIC HAS_SUGGEST_OVERRIDE) -endif() - -if (HAS_SUGGEST_DESTRUCTOR_OVERRIDE) - target_compile_definitions(clickhouse_functions_gatherutils PUBLIC HAS_SUGGEST_DESTRUCTOR_OVERRIDE) -endif() - if (OMIT_HEAVY_DEBUG_SYMBOLS) target_compile_options(clickhouse_functions_gatherutils PRIVATE "-g0") endif() diff --git a/src/Functions/GatherUtils/Sources.h b/src/Functions/GatherUtils/Sources.h index af6fc84beba..b75239d8678 100644 --- a/src/Functions/GatherUtils/Sources.h +++ b/src/Functions/GatherUtils/Sources.h @@ -141,12 +141,8 @@ struct NumericArraySource : public ArraySourceImpl> /// The methods can be virtual or not depending on the template parameter. See IStringSource. #pragma GCC diagnostic push -#ifdef HAS_SUGGEST_OVERRIDE -# pragma GCC diagnostic ignored "-Wsuggest-override" -#endif -#ifdef HAS_SUGGEST_DESTRUCTOR_OVERRIDE -# pragma GCC diagnostic ignored "-Wsuggest-destructor-override" -#endif +#pragma GCC diagnostic ignored "-Wsuggest-override" +#pragma GCC diagnostic ignored "-Wsuggest-destructor-override" template struct ConstSource : public Base diff --git a/src/Functions/toValidUTF8.cpp b/src/Functions/toValidUTF8.cpp index e509b59a23e..528cef93dd3 100644 --- a/src/Functions/toValidUTF8.cpp +++ b/src/Functions/toValidUTF8.cpp @@ -13,9 +13,7 @@ #if defined(__aarch64__) && defined(__ARM_NEON) # include -# ifdef HAS_RESERVED_IDENTIFIER -# pragma clang diagnostic ignored "-Wreserved-identifier" -# endif +# pragma clang diagnostic ignored "-Wreserved-identifier" #endif namespace DB diff --git a/src/IO/ReadBufferFromFileDescriptor.cpp b/src/IO/ReadBufferFromFileDescriptor.cpp index 417e985159d..65df93413dd 100644 --- a/src/IO/ReadBufferFromFileDescriptor.cpp +++ b/src/IO/ReadBufferFromFileDescriptor.cpp @@ -12,9 +12,7 @@ #include -#ifdef HAS_RESERVED_IDENTIFIER #pragma clang diagnostic ignored "-Wreserved-identifier" -#endif namespace ProfileEvents { diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index e14b3ae9129..809a2404746 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -18,9 +18,7 @@ #if defined(__aarch64__) && defined(__ARM_NEON) # include -# ifdef HAS_RESERVED_IDENTIFIER -# pragma clang diagnostic ignored "-Wreserved-identifier" -# endif +# pragma clang diagnostic ignored "-Wreserved-identifier" #endif namespace DB diff --git a/src/IO/WriteBufferValidUTF8.cpp b/src/IO/WriteBufferValidUTF8.cpp index 4c8e172f43c..b72bc627220 100644 --- a/src/IO/WriteBufferValidUTF8.cpp +++ b/src/IO/WriteBufferValidUTF8.cpp @@ -8,9 +8,7 @@ #if defined(__aarch64__) && defined(__ARM_NEON) # include -# ifdef HAS_RESERVED_IDENTIFIER -# pragma clang diagnostic ignored "-Wreserved-identifier" -# endif +# pragma clang diagnostic ignored "-Wreserved-identifier" #endif namespace DB diff --git a/src/IO/tests/gtest_bit_io.cpp b/src/IO/tests/gtest_bit_io.cpp index 6a88f7baa64..6ad6bec894e 100644 --- a/src/IO/tests/gtest_bit_io.cpp +++ b/src/IO/tests/gtest_bit_io.cpp @@ -1,6 +1,4 @@ -#ifdef HAS_RESERVED_IDENTIFIER #pragma clang diagnostic ignored "-Wreserved-identifier" -#endif #include #include diff --git a/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp b/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp index 96ed2a7021e..27fd7ac1575 100644 --- a/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp +++ b/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp @@ -1,6 +1,4 @@ -#ifdef HAS_RESERVED_IDENTIFIER #pragma clang diagnostic ignored "-Wreserved-identifier" -#endif #include "ArrowBufferedStreams.h" diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index e3b87d48ce6..5cba3e34169 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -18,9 +18,7 @@ #if defined(__aarch64__) && defined(__ARM_NEON) # include -# ifdef HAS_RESERVED_IDENTIFIER -# pragma clang diagnostic ignored "-Wreserved-identifier" -# endif +# pragma clang diagnostic ignored "-Wreserved-identifier" #endif namespace DB diff --git a/utils/memcpy-bench/memcpy-bench.cpp b/utils/memcpy-bench/memcpy-bench.cpp index 8b75164eb60..bb571200d07 100644 --- a/utils/memcpy-bench/memcpy-bench.cpp +++ b/utils/memcpy-bench/memcpy-bench.cpp @@ -1,6 +1,4 @@ -#ifdef HAS_RESERVED_IDENTIFIER #pragma clang diagnostic ignored "-Wreserved-identifier" -#endif #include #include From 39724d3a15b667f2780c5f3fa1a18b542e885319 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 17 Mar 2023 13:57:00 +0000 Subject: [PATCH 229/559] Fix --- .../0_stateless/02589_bson_invalid_document_size.reference | 0 .../queries/0_stateless/02589_bson_invalid_document_size.sql | 4 ++++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/02589_bson_invalid_document_size.reference create mode 100644 tests/queries/0_stateless/02589_bson_invalid_document_size.sql diff --git a/tests/queries/0_stateless/02589_bson_invalid_document_size.reference b/tests/queries/0_stateless/02589_bson_invalid_document_size.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02589_bson_invalid_document_size.sql b/tests/queries/0_stateless/02589_bson_invalid_document_size.sql new file mode 100644 index 00000000000..b536b8d5c92 --- /dev/null +++ b/tests/queries/0_stateless/02589_bson_invalid_document_size.sql @@ -0,0 +1,4 @@ +set input_format_parallel_parsing=1; +set max_threads=0; +select * from format(BSONEachRow, 'x UInt32', x'00000000'); -- {serverError INCORRECT_DATA} + From 000b11d7bd987f0ee3d369897e935699c90ef6d5 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 17 Mar 2023 15:07:32 +0100 Subject: [PATCH 230/559] Fix tests --- .../02149_schema_inference_formats_with_schema.reference | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference index 47391a77ee8..9a7a1611a7b 100644 --- a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference +++ b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference @@ -422,9 +422,9 @@ float32 Float32 float64 Float64 0 0 1.2 0.7692307692307692 -date Int32 -0 -1 +date Date32 +1970-01-01 +1970-01-02 str String fixed_string FixedString(3) Str: 0 100 From 292d12f86e41a1437ebea6b1f86d645dff4d274a Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 17 Mar 2023 11:09:55 -0300 Subject: [PATCH 231/559] fix test / remove hardcoded database --- .../0_stateless/01656_join_defaul_enum.sql | 22 +++++-------------- 1 file changed, 6 insertions(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/01656_join_defaul_enum.sql b/tests/queries/0_stateless/01656_join_defaul_enum.sql index 8a0fc089b16..878936da3b5 100644 --- a/tests/queries/0_stateless/01656_join_defaul_enum.sql +++ b/tests/queries/0_stateless/01656_join_defaul_enum.sql @@ -1,18 +1,9 @@ -DROP DATABASE IF EXISTS test_01656; - -CREATE DATABASE test_01656; -USE test_01656; - -DROP TABLE IF EXISTS table_key; -DROP TABLE IF EXISTS table_with_enum; - -CREATE TABLE table_key (keycol UInt16) ENGINE = MergeTree() ORDER BY (keycol) PARTITION BY tuple(); +CREATE TABLE table_key (keycol UInt16) ENGINE = MergeTree() ORDER BY (keycol) PARTITION BY tuple() +as SELECT * FROM VALUES ( (1), (2), (3) ); CREATE TABLE table_with_enum (keycol UInt16, enum_col Enum8('First' = 1,'Second' = 2)) - ENGINE = MergeTree() ORDER BY (keycol) PARTITION BY tuple(); - -INSERT INTO table_key VALUES (1), (2), (3); -INSERT INTO table_with_enum VALUES (2, 'Second'), (4, 'Second'); + ENGINE = MergeTree() ORDER BY (keycol) PARTITION BY tuple() +as SELECT * FROM VALUES ( (2, 'Second'), (4, 'Second') ); SET join_algorithm = 'hash'; @@ -34,7 +25,6 @@ SELECT keycol, enum_col FROM table_with_enum LEFT JOIN table_key USING (keycol) SELECT keycol, enum_col FROM table_with_enum RIGHT JOIN table_key USING (keycol) ORDER BY keycol; SELECT keycol, enum_col FROM table_with_enum FULL JOIN table_key USING (keycol) ORDER BY keycol; -DROP TABLE IF EXISTS table_key; -DROP TABLE IF EXISTS table_with_enum; +DROP TABLE table_key; +DROP TABLE table_with_enum; -DROP DATABASE IF EXISTS test_01656; From 1b7401b58a3d4a0bc3d3bfb6fb6f85f389ac0f4d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 17 Mar 2023 15:46:15 +0100 Subject: [PATCH 232/559] Update src/Storages/StorageS3.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/Storages/StorageS3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index baf18844b55..df9705b9c9a 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1267,7 +1267,7 @@ void StorageS3::updateConfiguration(ContextPtr ctx, StorageS3::Configuration & u std::move(headers), upd.auth_settings.use_environment_credentials.value_or(ctx->getConfigRef().getBool("s3.use_environment_credentials", false)), upd.auth_settings.use_insecure_imds_request.value_or(ctx->getConfigRef().getBool("s3.use_insecure_imds_request", false)), - upd.auth_settings.expiration_window_seconds.value_or(ctx->getConfigRef().getUInt64("s3.expiration_window_seconds", 120))); + upd.auth_settings.expiration_window_seconds.value_or(ctx->getConfigRef().getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS))); } void StorageS3::processNamedCollectionResult(StorageS3::Configuration & configuration, const NamedCollection & collection) From e8295be9be877b90bf0cf1664f8ce3e1ccb774b4 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 17 Mar 2023 14:52:41 +0100 Subject: [PATCH 233/559] if no group then do attachInternalTextLogsQueue to the thread only --- src/Common/ThreadStatus.cpp | 12 +++--------- src/Interpreters/ThreadStatusExt.cpp | 1 - 2 files changed, 3 insertions(+), 10 deletions(-) diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index a42211c612b..11f35bc7a6b 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -16,11 +16,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - thread_local ThreadStatus constinit * current_thread = nullptr; #if !defined(SANITIZER) @@ -156,12 +151,11 @@ void ThreadGroupStatus::attachInternalTextLogsQueue(const InternalTextLogsQueueP void ThreadStatus::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue, LogsLevel logs_level) { - if (!thread_group) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No thread group attached to the thread {}", thread_id); - local_data.logs_queue_ptr = logs_queue; local_data.client_logs_level = logs_level; - thread_group->attachInternalTextLogsQueue(logs_queue, logs_level); + + if (thread_group) + thread_group->attachInternalTextLogsQueue(logs_queue, logs_level); } InternalTextLogsQueuePtr ThreadStatus::getInternalTextLogsQueue() const diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 088a35c7f1c..b4e1da2c697 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -122,7 +122,6 @@ void CurrentThread::attachQueryForLog(const String & query_) current_thread->attachQueryForLog(query_); } - void ThreadStatus::applyQuerySettings() { auto query_context_ptr = query_context.lock(); From 2958c5f0f130c6308b00af7f3c41c91357dedb72 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 17 Mar 2023 15:08:16 +0000 Subject: [PATCH 234/559] Fix logical error in evaluate constant expression --- .../evaluateConstantExpression.cpp | 3 +++ src/Parsers/ASTAsterisk.h | 1 + src/Parsers/ASTColumnsMatcher.h | 6 ++++++ src/Parsers/ASTColumnsTransformers.h | 7 +++++++ src/Parsers/ASTFunction.cpp | 21 +++++++++++++++++++ src/Parsers/ASTFunction.h | 2 ++ src/Parsers/ASTIdentifier.h | 1 + src/Parsers/ASTLiteral.h | 1 + src/Parsers/ASTQualifiedAsterisk.h | 1 + src/Parsers/ASTWithAlias.h | 2 ++ src/Parsers/IAST.h | 3 +++ .../02680_mysql_ast_logical_err.reference | 0 .../02680_mysql_ast_logical_err.sql | 2 ++ 13 files changed, 50 insertions(+) create mode 100644 tests/queries/0_stateless/02680_mysql_ast_logical_err.reference create mode 100644 tests/queries/0_stateless/02680_mysql_ast_logical_err.sql diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index ebefa0d9ce7..3e5684946c2 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -44,6 +44,9 @@ static std::pair> getFieldAndDataTypeFro std::pair> evaluateConstantExpression(const ASTPtr & node, const ContextPtr & context) { + if (!node->hasColumnName()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression '{}' is not a constant expression", node->formatForErrorMessage()); + if (ASTLiteral * literal = node->as()) return getFieldAndDataTypeFromLiteral(literal); diff --git a/src/Parsers/ASTAsterisk.h b/src/Parsers/ASTAsterisk.h index 840b7996536..72fd82fc9a7 100644 --- a/src/Parsers/ASTAsterisk.h +++ b/src/Parsers/ASTAsterisk.h @@ -15,6 +15,7 @@ public: String getID(char) const override { return "Asterisk"; } ASTPtr clone() const override; void appendColumnName(WriteBuffer & ostr) const override; + bool hasColumnName() const override { return true; } ASTPtr expression; ASTPtr transformers; diff --git a/src/Parsers/ASTColumnsMatcher.h b/src/Parsers/ASTColumnsMatcher.h index f31a8bd9a22..5e0149756f1 100644 --- a/src/Parsers/ASTColumnsMatcher.h +++ b/src/Parsers/ASTColumnsMatcher.h @@ -23,6 +23,8 @@ public: ASTPtr clone() const override; void appendColumnName(WriteBuffer & ostr) const override; + bool hasColumnName() const override { return true; } + void setPattern(String pattern); const String & getPattern() const; const std::shared_ptr & getMatcher() const; @@ -46,6 +48,7 @@ public: String getID(char) const override { return "ColumnsListMatcher"; } ASTPtr clone() const override; void appendColumnName(WriteBuffer & ostr) const override; + bool hasColumnName() const override { return true; } ASTPtr expression; ASTPtr column_list; @@ -62,6 +65,8 @@ public: ASTPtr clone() const override; void appendColumnName(WriteBuffer & ostr) const override; + bool hasColumnName() const override { return true; } + const std::shared_ptr & getMatcher() const; void setPattern(String pattern, bool set_matcher = true); void setMatcher(std::shared_ptr matcher); @@ -84,6 +89,7 @@ public: String getID(char) const override { return "QualifiedColumnsListMatcher"; } ASTPtr clone() const override; void appendColumnName(WriteBuffer & ostr) const override; + bool hasColumnName() const override { return true; } ASTPtr qualifier; ASTPtr column_list; diff --git a/src/Parsers/ASTColumnsTransformers.h b/src/Parsers/ASTColumnsTransformers.h index e42949ebfd8..ae84f424a8b 100644 --- a/src/Parsers/ASTColumnsTransformers.h +++ b/src/Parsers/ASTColumnsTransformers.h @@ -48,6 +48,8 @@ public: } void transform(ASTs & nodes) const override; void appendColumnName(WriteBuffer & ostr) const override; + bool hasColumnName() const override { return true; } + void updateTreeHashImpl(SipHash & hash_state) const override; // Case 1 APPLY (quantile(0.9)) @@ -80,6 +82,7 @@ public: const std::shared_ptr & getMatcher() const; bool isColumnMatching(const String & column_name) const; void appendColumnName(WriteBuffer & ostr) const override; + bool hasColumnName() const override { return true; } void updateTreeHashImpl(SipHash & hash_state) const override; protected: @@ -103,6 +106,8 @@ public: } void appendColumnName(WriteBuffer & ostr) const override; + bool hasColumnName() const override { return true; } + void updateTreeHashImpl(SipHash & hash_state) const override; String name; @@ -121,6 +126,8 @@ public: } void transform(ASTs & nodes) const override; void appendColumnName(WriteBuffer & ostr) const override; + bool hasColumnName() const override { return true; } + void updateTreeHashImpl(SipHash & hash_state) const override; protected: diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index 129d3d60744..3dd301b1b64 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -468,6 +468,27 @@ namespace }; } +bool ASTFunction::hasColumnName() const +{ + if (parameters) + { + for (const auto & child : parameters->children) + { + if (!child->hasColumnName()) + return false; + } + } + + if (arguments) + { + for (const auto & child : arguments->children) + { + if (!child->hasColumnName()) + return false; + } + } + return true; +} void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const { diff --git a/src/Parsers/ASTFunction.h b/src/Parsers/ASTFunction.h index 4a036c5e94a..39867e4fee0 100644 --- a/src/Parsers/ASTFunction.h +++ b/src/Parsers/ASTFunction.h @@ -79,6 +79,8 @@ public: protected: void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void appendColumnNameImpl(WriteBuffer & ostr) const override; + bool hasColumnName() const override; + private: void finishFormatWithWindow(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const; }; diff --git a/src/Parsers/ASTIdentifier.h b/src/Parsers/ASTIdentifier.h index 0e030c797ce..301542ce6e5 100644 --- a/src/Parsers/ASTIdentifier.h +++ b/src/Parsers/ASTIdentifier.h @@ -58,6 +58,7 @@ protected: void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void appendColumnNameImpl(WriteBuffer & ostr) const override; + bool hasColumnName() const override { return true; } private: using ASTWithAlias::children; /// ASTIdentifier is child free diff --git a/src/Parsers/ASTLiteral.h b/src/Parsers/ASTLiteral.h index e57bcfcd9d5..84140ade189 100644 --- a/src/Parsers/ASTLiteral.h +++ b/src/Parsers/ASTLiteral.h @@ -47,6 +47,7 @@ protected: void formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; void appendColumnNameImpl(WriteBuffer & ostr) const override; + bool hasColumnName() const override { return true; } private: /// Legacy version of 'appendColumnNameImpl'. It differs only with tuple literals. diff --git a/src/Parsers/ASTQualifiedAsterisk.h b/src/Parsers/ASTQualifiedAsterisk.h index 079b83ae171..0e6c69f2269 100644 --- a/src/Parsers/ASTQualifiedAsterisk.h +++ b/src/Parsers/ASTQualifiedAsterisk.h @@ -31,6 +31,7 @@ public: return clone; } void appendColumnName(WriteBuffer & ostr) const override; + bool hasColumnName() const override { return true; } ASTPtr qualifier; ASTPtr transformers; diff --git a/src/Parsers/ASTWithAlias.h b/src/Parsers/ASTWithAlias.h index ea4419402b0..8b05385cff6 100644 --- a/src/Parsers/ASTWithAlias.h +++ b/src/Parsers/ASTWithAlias.h @@ -21,6 +21,8 @@ public: using IAST::IAST; void appendColumnName(WriteBuffer & ostr) const final; + bool hasColumnName() const override { return true; } + void appendColumnNameWithoutAlias(WriteBuffer & ostr) const final; String getAliasOrColumnName() const override { return alias.empty() ? getColumnName() : alias; } String tryGetAlias() const override { return alias; } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 5928506aa5b..554484ea737 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -56,6 +56,9 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to get name of not a column: {}", getID()); } + /* This method should be overridden with appendColumnName */ + virtual bool hasColumnName() const { return false; } + /** Get the alias, if any, or the canonical name of the column, if it is not. */ virtual String getAliasOrColumnName() const { return getColumnName(); } diff --git a/tests/queries/0_stateless/02680_mysql_ast_logical_err.reference b/tests/queries/0_stateless/02680_mysql_ast_logical_err.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02680_mysql_ast_logical_err.sql b/tests/queries/0_stateless/02680_mysql_ast_logical_err.sql new file mode 100644 index 00000000000..39f743ee332 --- /dev/null +++ b/tests/queries/0_stateless/02680_mysql_ast_logical_err.sql @@ -0,0 +1,2 @@ +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 BAD_ARGUMENTS } + From 5d0d7695aa640d079a3ade17f0a10b83be8aed2a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 17 Mar 2023 16:55:13 +0100 Subject: [PATCH 235/559] Fix --- src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp index 2a5b517cd1c..115878f3a02 100644 --- a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp @@ -995,6 +995,10 @@ fileSegmentationEngineBSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t { BSONSizeT document_size; readBinary(document_size, in); + + if (document_size < sizeof(document_size)) + throw ParsingException(ErrorCodes::INCORRECT_DATA, "Size of BSON document is invalid"); + if (min_bytes != 0 && document_size > 10 * min_bytes) throw ParsingException( ErrorCodes::INCORRECT_DATA, From c376712007a6c093047fab5b179ad73583450dd9 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 17 Mar 2023 16:56:24 +0100 Subject: [PATCH 236/559] Fix --- src/Processors/Formats/Impl/BSONEachRowRowOutputFormat.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/BSONEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/BSONEachRowRowOutputFormat.cpp index 95dd3079687..451d533b963 100644 --- a/src/Processors/Formats/Impl/BSONEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/BSONEachRowRowOutputFormat.cpp @@ -116,7 +116,7 @@ size_t BSONEachRowRowOutputFormat::countBSONFieldSize(const IColumn & column, co { size_t size = 1; // Field type size += name.size() + 1; // Field name and \0 - switch (column.getDataType()) + switch (data_type->getTypeId()) { case TypeIndex::Int8: [[fallthrough]]; case TypeIndex::Int16: [[fallthrough]]; @@ -263,7 +263,7 @@ size_t BSONEachRowRowOutputFormat::countBSONFieldSize(const IColumn & column, co void BSONEachRowRowOutputFormat::serializeField(const IColumn & column, const DataTypePtr & data_type, size_t row_num, const String & name) { - switch (column.getDataType()) + switch (data_type->getTypeId()) { case TypeIndex::Float32: { From 86a409e7a9f6dd9af3cc12abcf55d1abd6903859 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 17 Mar 2023 16:07:12 +0000 Subject: [PATCH 237/559] Assert znode count --- .../test_keeper_mntr_data_size/test.py | 27 ++++++++++--------- 1 file changed, 15 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_keeper_mntr_data_size/test.py b/tests/integration/test_keeper_mntr_data_size/test.py index 8789ca0354c..ad6e5f02af7 100644 --- a/tests/integration/test_keeper_mntr_data_size/test.py +++ b/tests/integration/test_keeper_mntr_data_size/test.py @@ -5,8 +5,6 @@ from helpers.cluster import ClickHouseCluster import helpers.keeper_utils as keeper_utils import random import string -import os -import time from kazoo.client import KazooClient, KazooState @@ -17,7 +15,6 @@ node = cluster.add_instance( "node", main_configs=["configs/enable_keeper.xml"], stay_alive=True, - with_zookeeper=True, ) @@ -60,10 +57,14 @@ def test_mntr_data_size_after_restart(started_cluster): "/test_mntr_data_size/node" + str(i), random_string(123).encode() ) - def get_line_with_size(): + node_zk.stop() + node_zk.close() + node_zk = None + + def get_line_from_mntr(key): return next( filter( - lambda line: "zk_approximate_data_size" in line, + lambda line: key in line, keeper_utils.send_4lw_cmd(started_cluster, node, "mntr").split( "\n" ), @@ -71,19 +72,21 @@ def test_mntr_data_size_after_restart(started_cluster): None, ) - line_size_before = get_line_with_size() + line_size_before = get_line_from_mntr("zk_approximate_data_size") + node_count_before = get_line_from_mntr("zk_znode_count") + assert get_line_from_mntr("zk_ephemerals_count") == "zk_ephemerals_count\t0" assert line_size_before != None - node_zk.stop() - node_zk.close() - node_zk = None - restart_clickhouse() - assert get_line_with_size() == line_size_before + def assert_mntr_stats(): + assert get_line_from_mntr("zk_ephemerals_count") == "zk_ephemerals_count\t0" + assert get_line_from_mntr("zk_znode_count") == node_count_before + assert get_line_from_mntr("zk_approximate_data_size") == line_size_before + assert_mntr_stats() keeper_utils.send_4lw_cmd(started_cluster, node, "rclc") - assert get_line_with_size() == line_size_before + assert_mntr_stats() finally: try: if node_zk is not None: From 868e86e9edefd29ee148abbc88a229716d486814 Mon Sep 17 00:00:00 2001 From: flynn Date: Fri, 17 Mar 2023 16:27:24 +0000 Subject: [PATCH 238/559] Fix limit offset --- .../InterpreterSelectWithUnionQuery.cpp | 28 +++++++++++-------- .../01596_setting_limit_offset.reference | 24 ++++++++++++++++ .../01596_setting_limit_offset.sql | 3 ++ 3 files changed, 43 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index bfa3d16bf29..025bb916720 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -2,22 +2,23 @@ #include #include +#include #include #include -#include #include +#include +#include #include #include -#include #include #include #include #include -#include -#include #include #include #include +#include +#include #include #include @@ -106,32 +107,35 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( const ASTPtr limit_offset_ast = select_query->limitOffset(); if (limit_offset_ast) { - limit_offset = limit_offset_ast->as().value.safeGet(); + limit_offset = evaluateConstantExpressionAsLiteral(limit_offset_ast, context)->as().value.safeGet(); UInt64 new_limit_offset = settings.offset + limit_offset; - limit_offset_ast->as().value = Field(new_limit_offset); + ASTPtr new_limit_offset_ast = std::make_shared(new_limit_offset); + select_query->setExpression(ASTSelectQuery::Expression::LIMIT_OFFSET, std::move(new_limit_offset_ast)); } else if (settings.offset) { - ASTPtr new_limit_offset_ast = std::make_shared(Field(static_cast(settings.offset))); + ASTPtr new_limit_offset_ast = std::make_shared(settings.offset.value); select_query->setExpression(ASTSelectQuery::Expression::LIMIT_OFFSET, std::move(new_limit_offset_ast)); } const ASTPtr limit_length_ast = select_query->limitLength(); if (limit_length_ast) { - limit_length = limit_length_ast->as().value.safeGet(); + limit_length = evaluateConstantExpressionAsLiteral(limit_length_ast, context)->as().value.safeGet(); UInt64 new_limit_length = 0; if (settings.offset == 0) - new_limit_length = std::min(limit_length, static_cast(settings.limit)); + new_limit_length = std::min(limit_length, settings.limit.value); else if (settings.offset < limit_length) - new_limit_length = settings.limit ? std::min(static_cast(settings.limit), limit_length - settings.offset) : (limit_length - settings.offset); + new_limit_length = settings.limit ? std::min(settings.limit.value, limit_length - settings.offset.value) + : (limit_length - settings.offset.value); - limit_length_ast->as().value = Field(new_limit_length); + ASTPtr new_limit_length_ast = std::make_shared(new_limit_length); + select_query->setExpression(ASTSelectQuery::Expression::LIMIT_LENGTH, std::move(new_limit_length_ast)); } else if (settings.limit) { - ASTPtr new_limit_length_ast = std::make_shared(Field(static_cast(settings.limit))); + ASTPtr new_limit_length_ast = std::make_shared(settings.limit.value); select_query->setExpression(ASTSelectQuery::Expression::LIMIT_LENGTH, std::move(new_limit_length_ast)); } diff --git a/tests/queries/0_stateless/01596_setting_limit_offset.reference b/tests/queries/0_stateless/01596_setting_limit_offset.reference index 96483268d43..fe6390b172c 100644 --- a/tests/queries/0_stateless/01596_setting_limit_offset.reference +++ b/tests/queries/0_stateless/01596_setting_limit_offset.reference @@ -22,6 +22,10 @@ 107 108 109 +102 +103 +104 +105 105 106 107 @@ -38,6 +42,26 @@ 64 64 60 +60 +60 +61 +61 +62 +62 +63 +63 +64 +64 +60 +60 +61 +61 +62 +62 +63 +63 +64 +64 35 35 36 diff --git a/tests/queries/0_stateless/01596_setting_limit_offset.sql b/tests/queries/0_stateless/01596_setting_limit_offset.sql index 3c91e3542bb..0c2ab5fb4dc 100644 --- a/tests/queries/0_stateless/01596_setting_limit_offset.sql +++ b/tests/queries/0_stateless/01596_setting_limit_offset.sql @@ -11,6 +11,7 @@ SELECT * FROM test OFFSET 20; -- 5 rows SELECT * FROM (SELECT i FROM test LIMIT 10 OFFSET 50) TMP; -- 5 rows SELECT * FROM test LIMIT 4 OFFSET 192; -- 4 rows SELECT * FROM test LIMIT 10 OFFSET 195; -- 5 rows +SELECT * FROM test LIMIT 2*2 OFFSET 192; -- Only set offset SET limit = 0; @@ -21,6 +22,8 @@ SELECT * FROM test LIMIT 100; -- no result SET offset = 10; SELECT * FROM test LIMIT 20 OFFSET 100; -- 10 rows SELECT * FROM test LIMIT 11 OFFSET 100; -- 1 rows +SELECT * FROM test LIMIT 20 OFFSET 10*10; +SELECT * FROM test LIMIT 4*5 OFFSET 10*10; -- offset and limit together SET limit = 10; From 4930683aa87467f21a9c25fd067857b818d549fb Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 17 Mar 2023 17:01:40 +0000 Subject: [PATCH 239/559] Revert "Fix logical error in evaluate constant expression" This reverts commit 2958c5f0f130c6308b00af7f3c41c91357dedb72. --- .../evaluateConstantExpression.cpp | 3 --- src/Parsers/ASTAsterisk.h | 1 - src/Parsers/ASTColumnsMatcher.h | 6 ------ src/Parsers/ASTColumnsTransformers.h | 7 ------- src/Parsers/ASTFunction.cpp | 21 ------------------- src/Parsers/ASTFunction.h | 2 -- src/Parsers/ASTIdentifier.h | 1 - src/Parsers/ASTLiteral.h | 1 - src/Parsers/ASTQualifiedAsterisk.h | 1 - src/Parsers/ASTWithAlias.h | 2 -- src/Parsers/IAST.h | 3 --- .../02680_mysql_ast_logical_err.reference | 0 .../02680_mysql_ast_logical_err.sql | 2 -- 13 files changed, 50 deletions(-) delete mode 100644 tests/queries/0_stateless/02680_mysql_ast_logical_err.reference delete mode 100644 tests/queries/0_stateless/02680_mysql_ast_logical_err.sql diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index 3e5684946c2..ebefa0d9ce7 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -44,9 +44,6 @@ static std::pair> getFieldAndDataTypeFro std::pair> evaluateConstantExpression(const ASTPtr & node, const ContextPtr & context) { - if (!node->hasColumnName()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression '{}' is not a constant expression", node->formatForErrorMessage()); - if (ASTLiteral * literal = node->as()) return getFieldAndDataTypeFromLiteral(literal); diff --git a/src/Parsers/ASTAsterisk.h b/src/Parsers/ASTAsterisk.h index 72fd82fc9a7..840b7996536 100644 --- a/src/Parsers/ASTAsterisk.h +++ b/src/Parsers/ASTAsterisk.h @@ -15,7 +15,6 @@ public: String getID(char) const override { return "Asterisk"; } ASTPtr clone() const override; void appendColumnName(WriteBuffer & ostr) const override; - bool hasColumnName() const override { return true; } ASTPtr expression; ASTPtr transformers; diff --git a/src/Parsers/ASTColumnsMatcher.h b/src/Parsers/ASTColumnsMatcher.h index 5e0149756f1..f31a8bd9a22 100644 --- a/src/Parsers/ASTColumnsMatcher.h +++ b/src/Parsers/ASTColumnsMatcher.h @@ -23,8 +23,6 @@ public: ASTPtr clone() const override; void appendColumnName(WriteBuffer & ostr) const override; - bool hasColumnName() const override { return true; } - void setPattern(String pattern); const String & getPattern() const; const std::shared_ptr & getMatcher() const; @@ -48,7 +46,6 @@ public: String getID(char) const override { return "ColumnsListMatcher"; } ASTPtr clone() const override; void appendColumnName(WriteBuffer & ostr) const override; - bool hasColumnName() const override { return true; } ASTPtr expression; ASTPtr column_list; @@ -65,8 +62,6 @@ public: ASTPtr clone() const override; void appendColumnName(WriteBuffer & ostr) const override; - bool hasColumnName() const override { return true; } - const std::shared_ptr & getMatcher() const; void setPattern(String pattern, bool set_matcher = true); void setMatcher(std::shared_ptr matcher); @@ -89,7 +84,6 @@ public: String getID(char) const override { return "QualifiedColumnsListMatcher"; } ASTPtr clone() const override; void appendColumnName(WriteBuffer & ostr) const override; - bool hasColumnName() const override { return true; } ASTPtr qualifier; ASTPtr column_list; diff --git a/src/Parsers/ASTColumnsTransformers.h b/src/Parsers/ASTColumnsTransformers.h index ae84f424a8b..e42949ebfd8 100644 --- a/src/Parsers/ASTColumnsTransformers.h +++ b/src/Parsers/ASTColumnsTransformers.h @@ -48,8 +48,6 @@ public: } void transform(ASTs & nodes) const override; void appendColumnName(WriteBuffer & ostr) const override; - bool hasColumnName() const override { return true; } - void updateTreeHashImpl(SipHash & hash_state) const override; // Case 1 APPLY (quantile(0.9)) @@ -82,7 +80,6 @@ public: const std::shared_ptr & getMatcher() const; bool isColumnMatching(const String & column_name) const; void appendColumnName(WriteBuffer & ostr) const override; - bool hasColumnName() const override { return true; } void updateTreeHashImpl(SipHash & hash_state) const override; protected: @@ -106,8 +103,6 @@ public: } void appendColumnName(WriteBuffer & ostr) const override; - bool hasColumnName() const override { return true; } - void updateTreeHashImpl(SipHash & hash_state) const override; String name; @@ -126,8 +121,6 @@ public: } void transform(ASTs & nodes) const override; void appendColumnName(WriteBuffer & ostr) const override; - bool hasColumnName() const override { return true; } - void updateTreeHashImpl(SipHash & hash_state) const override; protected: diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index 3dd301b1b64..129d3d60744 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -468,27 +468,6 @@ namespace }; } -bool ASTFunction::hasColumnName() const -{ - if (parameters) - { - for (const auto & child : parameters->children) - { - if (!child->hasColumnName()) - return false; - } - } - - if (arguments) - { - for (const auto & child : arguments->children) - { - if (!child->hasColumnName()) - return false; - } - } - return true; -} void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const { diff --git a/src/Parsers/ASTFunction.h b/src/Parsers/ASTFunction.h index 39867e4fee0..4a036c5e94a 100644 --- a/src/Parsers/ASTFunction.h +++ b/src/Parsers/ASTFunction.h @@ -79,8 +79,6 @@ public: protected: void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void appendColumnNameImpl(WriteBuffer & ostr) const override; - bool hasColumnName() const override; - private: void finishFormatWithWindow(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const; }; diff --git a/src/Parsers/ASTIdentifier.h b/src/Parsers/ASTIdentifier.h index 301542ce6e5..0e030c797ce 100644 --- a/src/Parsers/ASTIdentifier.h +++ b/src/Parsers/ASTIdentifier.h @@ -58,7 +58,6 @@ protected: void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void appendColumnNameImpl(WriteBuffer & ostr) const override; - bool hasColumnName() const override { return true; } private: using ASTWithAlias::children; /// ASTIdentifier is child free diff --git a/src/Parsers/ASTLiteral.h b/src/Parsers/ASTLiteral.h index 84140ade189..e57bcfcd9d5 100644 --- a/src/Parsers/ASTLiteral.h +++ b/src/Parsers/ASTLiteral.h @@ -47,7 +47,6 @@ protected: void formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; void appendColumnNameImpl(WriteBuffer & ostr) const override; - bool hasColumnName() const override { return true; } private: /// Legacy version of 'appendColumnNameImpl'. It differs only with tuple literals. diff --git a/src/Parsers/ASTQualifiedAsterisk.h b/src/Parsers/ASTQualifiedAsterisk.h index 0e6c69f2269..079b83ae171 100644 --- a/src/Parsers/ASTQualifiedAsterisk.h +++ b/src/Parsers/ASTQualifiedAsterisk.h @@ -31,7 +31,6 @@ public: return clone; } void appendColumnName(WriteBuffer & ostr) const override; - bool hasColumnName() const override { return true; } ASTPtr qualifier; ASTPtr transformers; diff --git a/src/Parsers/ASTWithAlias.h b/src/Parsers/ASTWithAlias.h index 8b05385cff6..ea4419402b0 100644 --- a/src/Parsers/ASTWithAlias.h +++ b/src/Parsers/ASTWithAlias.h @@ -21,8 +21,6 @@ public: using IAST::IAST; void appendColumnName(WriteBuffer & ostr) const final; - bool hasColumnName() const override { return true; } - void appendColumnNameWithoutAlias(WriteBuffer & ostr) const final; String getAliasOrColumnName() const override { return alias.empty() ? getColumnName() : alias; } String tryGetAlias() const override { return alias; } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 554484ea737..5928506aa5b 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -56,9 +56,6 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to get name of not a column: {}", getID()); } - /* This method should be overridden with appendColumnName */ - virtual bool hasColumnName() const { return false; } - /** Get the alias, if any, or the canonical name of the column, if it is not. */ virtual String getAliasOrColumnName() const { return getColumnName(); } diff --git a/tests/queries/0_stateless/02680_mysql_ast_logical_err.reference b/tests/queries/0_stateless/02680_mysql_ast_logical_err.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02680_mysql_ast_logical_err.sql b/tests/queries/0_stateless/02680_mysql_ast_logical_err.sql deleted file mode 100644 index 39f743ee332..00000000000 --- a/tests/queries/0_stateless/02680_mysql_ast_logical_err.sql +++ /dev/null @@ -1,2 +0,0 @@ -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 BAD_ARGUMENTS } - From dcf8314122ea5804512893693a8f1986e0751217 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 17 Mar 2023 17:04:28 +0000 Subject: [PATCH 240/559] Better fix logical error in evaluate constant expression --- src/Interpreters/evaluateConstantExpression.cpp | 2 +- tests/queries/0_stateless/02680_mysql_ast_logical_err.reference | 0 tests/queries/0_stateless/02680_mysql_ast_logical_err.sql | 2 ++ 3 files changed, 3 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02680_mysql_ast_logical_err.reference create mode 100644 tests/queries/0_stateless/02680_mysql_ast_logical_err.sql diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index ebefa0d9ce7..5a333172b14 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -70,7 +70,6 @@ std::pair> evaluateConstantExpression(co if (context->getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY && context->getSettingsRef().normalize_function_names) FunctionNameNormalizer().visit(ast.get()); - String result_name = ast->getColumnName(); auto syntax_result = TreeRewriter(context).analyze(ast, source_columns); /// AST potentially could be transformed to literal during TreeRewriter analyze. @@ -82,6 +81,7 @@ std::pair> evaluateConstantExpression(co ColumnPtr result_column; DataTypePtr result_type; + String result_name = ast->getColumnName(); for (const auto & action_node : actions->getOutputs()) { if ((action_node->result_name == result_name) && action_node->column) diff --git a/tests/queries/0_stateless/02680_mysql_ast_logical_err.reference b/tests/queries/0_stateless/02680_mysql_ast_logical_err.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02680_mysql_ast_logical_err.sql b/tests/queries/0_stateless/02680_mysql_ast_logical_err.sql new file mode 100644 index 00000000000..39f743ee332 --- /dev/null +++ b/tests/queries/0_stateless/02680_mysql_ast_logical_err.sql @@ -0,0 +1,2 @@ +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 BAD_ARGUMENTS } + From bb7cc975016d68c875e1988a1a1037e9b5813214 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 17 Mar 2023 18:34:24 +0100 Subject: [PATCH 241/559] fix new test with zero copy --- .../0_stateless/02370_lost_part_intersecting_merges.sh | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02370_lost_part_intersecting_merges.sh b/tests/queries/0_stateless/02370_lost_part_intersecting_merges.sh index f55eb4d74a9..db9bd2dd430 100755 --- a/tests/queries/0_stateless/02370_lost_part_intersecting_merges.sh +++ b/tests/queries/0_stateless/02370_lost_part_intersecting_merges.sh @@ -19,8 +19,15 @@ $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into rmt1 values (2);" $CLICKHOUSE_CLIENT --receive_timeout=3 -q "system sync replica rmt1;" 2>/dev/null 1>/dev/null + +# There's a stupid effect from "zero copy replication": +# MERGE_PARTS all_1_2_1 can be executed by rmt2 even if it was assigned by rmt1 +# After that, rmt2 will not be able to execute that merge and will only try to fetch the part from rmt2 +# But sends are stopped on rmt2... + +(sleep 5 && $CLICKHOUSE_CLIENT -q "system start replicated sends rmt2") & + $CLICKHOUSE_CLIENT --optimize_throw_if_noop=1 -q "optimize table rmt1;" -$CLICKHOUSE_CLIENT -q "system start replicated sends rmt2" $CLICKHOUSE_CLIENT -q "system sync replica rmt1;" $CLICKHOUSE_CLIENT -q "select 1, *, _part from rmt1 order by n;" From 2bffed06de5e673476ad30c65d9120e9a6d741f4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Mar 2023 18:35:19 +0100 Subject: [PATCH 242/559] Fix style --- src/Formats/NativeReader.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Formats/NativeReader.cpp b/src/Formats/NativeReader.cpp index aff479b5320..ff75ffb3c02 100644 --- a/src/Formats/NativeReader.cpp +++ b/src/Formats/NativeReader.cpp @@ -27,6 +27,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int CANNOT_READ_ALL_DATA; extern const int INCORRECT_DATA; + extern const int TOO_LARGE_ARRAY_SIZE; } From 6d9d88efc1d73019d1285bb4ab263dc051ab4628 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Mar 2023 18:37:13 +0100 Subject: [PATCH 243/559] Fix style --- src/DataTypes/Serializations/SerializationString.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index 76e71322b95..46fd9d5272d 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -162,11 +162,11 @@ static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnSt static constexpr size_t max_string_size = 16_GiB; /// Arbitrary value to prevent logical errors and overflows, but large enough. if (size > max_string_size) - throw Exception( - ErrorCodes::TOO_LARGE_STRING_SIZE, - "Too large string size: {}. The maximum is: {}.", - size, - max_string_size); + throw Exception( + ErrorCodes::TOO_LARGE_STRING_SIZE, + "Too large string size: {}. The maximum is: {}.", + size, + max_string_size); offset += size + 1; offsets.push_back(offset); From facb97883031afeac9880a9ed63fa12cb59d5e00 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 17 Mar 2023 17:57:56 +0000 Subject: [PATCH 244/559] Fix bug in tuple as array serialization in BSONEachRow format --- .../Formats/Impl/BSONEachRowRowOutputFormat.cpp | 2 +- .../0_stateless/02591_bson_long_tuple.reference | Bin 0 -> 91 bytes .../queries/0_stateless/02591_bson_long_tuple.sql | 2 ++ 3 files changed, 3 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02591_bson_long_tuple.reference create mode 100644 tests/queries/0_stateless/02591_bson_long_tuple.sql diff --git a/src/Processors/Formats/Impl/BSONEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/BSONEachRowRowOutputFormat.cpp index 95dd3079687..3d705f9d749 100644 --- a/src/Processors/Formats/Impl/BSONEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/BSONEachRowRowOutputFormat.cpp @@ -463,7 +463,7 @@ void BSONEachRowRowOutputFormat::serializeField(const IColumn & column, const Da writeBSONSize(document_size, out); for (size_t i = 0; i < nested_columns.size(); ++i) - serializeField(*nested_columns[i], nested_types[i], row_num, toValidUTF8String(nested_names[i])); + serializeField(*nested_columns[i], nested_types[i], row_num, have_explicit_names ? toValidUTF8String(nested_names[i]) : std::to_string(i)); writeChar(BSON_DOCUMENT_END, out); break; diff --git a/tests/queries/0_stateless/02591_bson_long_tuple.reference b/tests/queries/0_stateless/02591_bson_long_tuple.reference new file mode 100644 index 0000000000000000000000000000000000000000..008a3c51138ef4f8cc81d2056a66b865cd25fab9 GIT binary patch literal 91 zcmWl~w*i14006;PF*l9^nCGx`5yzLm=3XFtJf$U4S)#T~V=KWnTHERDptqC3E=IeV IEJfxr{O+^_ivR!s literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/02591_bson_long_tuple.sql b/tests/queries/0_stateless/02591_bson_long_tuple.sql new file mode 100644 index 00000000000..e24150c8e6d --- /dev/null +++ b/tests/queries/0_stateless/02591_bson_long_tuple.sql @@ -0,0 +1,2 @@ +select tuple(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11) as x format BSONEachRow; + From d72bf0af7119967663d2cadb92839a0be415beaf Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 17 Mar 2023 19:35:20 +0100 Subject: [PATCH 245/559] Fix style --- src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp index 115878f3a02..ebc612a4ce3 100644 --- a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp @@ -995,7 +995,7 @@ fileSegmentationEngineBSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t { BSONSizeT document_size; readBinary(document_size, in); - + if (document_size < sizeof(document_size)) throw ParsingException(ErrorCodes::INCORRECT_DATA, "Size of BSON document is invalid"); From 6509d9e525c89987f14ae290962c865072499406 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 17 Mar 2023 18:36:49 +0000 Subject: [PATCH 246/559] Refactir a bit more. --- .../QueryPlan/Optimizations/Optimizations.h | 2 +- .../QueryPlan/Optimizations/optimizeTree.cpp | 22 +- ...cpp => optimizeUseAggregateProjection.cpp} | 635 ++---------------- .../optimizeUseNormalProjection.cpp | 311 +++++++++ .../Optimizations/projectionsCommon.cpp | 210 ++++++ .../Optimizations/projectionsCommon.h | 81 +++ 6 files changed, 673 insertions(+), 588 deletions(-) rename src/Processors/QueryPlan/Optimizations/{optimizeUseProjections.cpp => optimizeUseAggregateProjection.cpp} (56%) create mode 100644 src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp create mode 100644 src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp create mode 100644 src/Processors/QueryPlan/Optimizations/projectionsCommon.h diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 3c29e4792fb..3ac3f63f783 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -107,7 +107,7 @@ using Stack = std::vector; void optimizePrimaryKeyCondition(const Stack & stack); void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); -bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes); +bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes); bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes); /// Enable memory bound merging of aggregation states for remote queries diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 121cca610f1..5b46b604d1e 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -124,7 +124,7 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s optimizeReadInOrder(*frame.node, nodes); if (optimization_settings.optimize_projection) - num_applied_projection += optimizeUseAggProjections(*frame.node, nodes); + num_applied_projection += optimizeUseAggregateProjections(*frame.node, nodes); if (optimization_settings.aggregation_in_order) optimizeAggregationInOrder(*frame.node, nodes); @@ -144,18 +144,20 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s if (optimization_settings.optimize_projection) { - bool applied = optimizeUseNormalProjections(stack, nodes); - /// This is actually some internal knowledge - bool stack_was_updated = !stack.back().node->children.empty(); - num_applied_projection += applied; + if (optimizeUseNormalProjections(stack, nodes)) + { + ++num_applied_projection; - if (max_optimizations_to_apply && max_optimizations_to_apply < num_applied_projection) - throw Exception(ErrorCodes::TOO_MANY_QUERY_PLAN_OPTIMIZATIONS, - "Too many projection optimizations applied to query plan. Current limit {}", - max_optimizations_to_apply); + if (max_optimizations_to_apply && max_optimizations_to_apply < num_applied_projection) + throw Exception(ErrorCodes::TOO_MANY_QUERY_PLAN_OPTIMIZATIONS, + "Too many projection optimizations applied to query plan. Current limit {}", + max_optimizations_to_apply); - if (applied && stack_was_updated) + /// Stack is updated after this optimization and frame is not valid anymore. + /// Try to apply optimizations again to newly added plan steps. + --stack.back().next_child; continue; + } } optimizePrimaryKeyCondition(stack); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp similarity index 56% rename from src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp rename to src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 2694285c2c9..0d18e8367cf 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseProjections.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -1,218 +1,25 @@ -#include +#include #include -#include #include +#include #include #include -#include -#include +#include + #include +#include + +#include +#include #include #include +#include #include -#include -#include -#include -#include -#include -#include -#include -#include +#include namespace DB::QueryPlanOptimizations { -/// This is a common DAG which is a merge of DAGs from Filter and Expression steps chain. -/// Additionally, for all the Filter steps, we collect filter conditions into filter_nodes. -/// Flag remove_last_filter_node is set in case if the last step is a Filter step and it should remove filter column. -struct QueryDAG -{ - ActionsDAGPtr dag; - ActionsDAG::NodeRawConstPtrs filter_nodes; - bool remove_last_filter_node = false; - - bool build(QueryPlan::Node & node); - -private: - void appendExpression(const ActionsDAGPtr & expression) - { - if (dag) - dag->mergeInplace(std::move(*expression->clone())); - else - dag = expression->clone(); - } -}; - -bool QueryDAG::build(QueryPlan::Node & node) -{ - IQueryPlanStep * step = node.step.get(); - if (auto * reading = typeid_cast(step)) - { - if (const auto * prewhere_info = reading->getPrewhereInfo()) - { - if (prewhere_info->row_level_filter) - { - remove_last_filter_node = false; - appendExpression(prewhere_info->row_level_filter); - if (const auto * filter_node = dag->tryFindInOutputs(prewhere_info->row_level_column_name)) - filter_nodes.push_back(filter_node); - else - return false; - } - - if (prewhere_info->prewhere_actions) - { - remove_last_filter_node = prewhere_info->remove_prewhere_column; - appendExpression(prewhere_info->prewhere_actions); - if (const auto * filter_node = dag->tryFindInOutputs(prewhere_info->prewhere_column_name)) - filter_nodes.push_back(filter_node); - else - return false; - } - } - return true; - } - - if (node.children.size() != 1) - return false; - - if (!build(*node.children.front())) - return false; - - if (auto * expression = typeid_cast(step)) - { - const auto & actions = expression->getExpression(); - if (actions->hasArrayJoin()) - return false; - - appendExpression(actions); - remove_last_filter_node = false; - return true; - } - - if (auto * filter = typeid_cast(step)) - { - const auto & actions = filter->getExpression(); - if (actions->hasArrayJoin()) - return false; - - appendExpression(actions); - remove_last_filter_node = filter->removesFilterColumn(); - const auto * filter_expression = dag->tryFindInOutputs(filter->getFilterColumnName()); - if (!filter_expression) - return false; - - filter_nodes.push_back(filter_expression); - return true; - } - - return false; -} - -struct AggregateQueryDAG -{ - ActionsDAGPtr dag; - const ActionsDAG::Node * filter_node = nullptr; - - bool build(QueryPlan::Node & node) - { - QueryDAG query; - if (!query.build(node)) - return false; - - dag = std::move(query.dag); - auto filter_nodes = std::move(query.filter_nodes); - - if (!filter_nodes.empty()) - { - filter_node = filter_nodes.front(); - if (filter_nodes.size() > 1) - { - FunctionOverloadResolverPtr func_builder_and = - std::make_unique( - std::make_shared()); - - filter_node = &dag->addFunction(func_builder_and, std::move(filter_nodes), {}); - } - - dag->getOutputs().push_back(filter_node); - } - - return true; - } -}; - -struct NormalQueryDAG -{ - ActionsDAGPtr dag; - bool need_remove_column = false; - const ActionsDAG::Node * filter_node = nullptr; - - bool build(QueryPlan::Node & node) - { - QueryDAG query; - if (!query.build(node)) - return false; - - dag = std::move(query.dag); - auto filter_nodes = std::move(query.filter_nodes); - need_remove_column = query.remove_last_filter_node; - - if (!filter_nodes.empty()) - { - auto & outputs = dag->getOutputs(); - filter_node = filter_nodes.back(); - - if (filter_nodes.size() > 1) - { - /// Add a conjunction of all the filters. - if (need_remove_column) - { - /// Last filter column is not needed; remove it right here - size_t pos = 0; - while (pos < outputs.size() && outputs[pos] != filter_node) - ++pos; - - if (pos < outputs.size()) - outputs.erase(outputs.begin() + pos); - } - else - { - /// Last filter is needed; we must replace it to constant 1, - /// As well as FilterStep does to make a compatible header. - for (auto & output : outputs) - { - if (output == filter_node) - { - ColumnWithTypeAndName col; - col.name = filter_node->result_name; - col.type = filter_node->result_type; - col.column = col.type->createColumnConst(1, 1); - output = &dag->addColumn(std::move(col)); - } - } - } - - FunctionOverloadResolverPtr func_builder_and = - std::make_unique( - std::make_shared()); - - filter_node = &dag->addFunction(func_builder_and, std::move(filter_nodes), {}); - outputs.insert(outputs.begin(), filter_node); - need_remove_column = true; - } - } - - if (dag) - { - dag->removeUnusedActions(); - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Header {}, Query DAG: {}", header.dumpStructure(), dag->dumpDAG()); - } - - return true; - } -}; - /// Required analysis info from aggregate projection. struct AggregateProjectionInfo { @@ -225,38 +32,6 @@ struct AggregateProjectionInfo ContextPtr context; }; -struct ProjectionCandidate -{ - const ProjectionDescription * projection; - - /// The number of marks we are going to read - size_t sum_marks = 0; - - /// Analysis result, separate for parts with and without projection. - /// Analysis is done in order to estimate the number of marks we are going to read. - /// For chosen projection, it is reused for reading step. - MergeTreeDataSelectAnalysisResultPtr merge_tree_projection_select_result_ptr; - MergeTreeDataSelectAnalysisResultPtr merge_tree_normal_select_result_ptr; -}; - -/// Aggregate projection analysis result in case it can be applied. -struct AggregateProjectionCandidate : public ProjectionCandidate -{ - AggregateProjectionInfo info; - - /// Actions which need to be applied to columns from projection - /// in order to get all the columns required for aggregation. - ActionsDAGPtr dag; -}; - -/// Normal projection analysis result in case it can be applied. -/// For now, it is empty. -/// Normal projection can be used only if it contains all required source columns. -/// It would not be hard to support pre-computed expressions and filtration. -struct NormalProjectionCandidate : public ProjectionCandidate -{ -}; - /// Get required info from aggregate projection. /// Ideally, this should be pre-calculated and stored inside ProjectionDescription. static AggregateProjectionInfo getAggregatingProjectionInfo( @@ -297,6 +72,41 @@ static AggregateProjectionInfo getAggregatingProjectionInfo( return info; } +struct AggregateQueryDAG +{ + ActionsDAGPtr dag; + const ActionsDAG::Node * filter_node = nullptr; + + bool build(QueryPlan::Node & node); +}; + +bool AggregateQueryDAG::build(QueryPlan::Node & node) +{ + QueryDAG query; + if (!query.build(node)) + return false; + + dag = std::move(query.dag); + auto filter_nodes = std::move(query.filter_nodes); + + if (!filter_nodes.empty()) + { + filter_node = filter_nodes.front(); + if (filter_nodes.size() > 1) + { + FunctionOverloadResolverPtr func_builder_and = + std::make_unique( + std::make_shared()); + + filter_node = &dag->addFunction(func_builder_and, std::move(filter_nodes), {}); + } + + dag->getOutputs().push_back(filter_node); + } + + return true; +} + using DAGIndex = std::unordered_map; static DAGIndex buildDAGIndex(const ActionsDAG & dag) { @@ -319,6 +129,7 @@ static bool hasNullableOrMissingColumn(const DAGIndex & index, const Names & nam return false; } + /// Here we try to match aggregate functions from the query to /// aggregate functions from projection. bool areAggregatesMatch( @@ -574,6 +385,17 @@ ActionsDAGPtr analyzeAggregateProjection( return proj_dag; } + +/// Aggregate projection analysis result in case it can be applied. +struct AggregateProjectionCandidate : public ProjectionCandidate +{ + AggregateProjectionInfo info; + + /// Actions which need to be applied to columns from projection + /// in order to get all the columns required for aggregation. + ActionsDAGPtr dag; +}; + struct MinMaxProjectionCandidate { AggregateProjectionCandidate candidate; @@ -684,81 +506,6 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( return candidates; } -static std::shared_ptr getMaxAddedBlocks(ReadFromMergeTree * reading) -{ - ContextPtr context = reading->getContext(); - - if (context->getSettingsRef().select_sequential_consistency) - { - if (const auto * replicated = dynamic_cast(&reading->getMergeTreeData())) - return std::make_shared(replicated->getMaxAddedBlocks()); - } - - return {}; -} - -static bool analyzeProjectionCandidate( - ProjectionCandidate & candidate, - const ReadFromMergeTree & reading, - const MergeTreeDataSelectExecutor & reader, - const Names & required_column_names, - const MergeTreeData::DataPartsVector & parts, - const StorageMetadataPtr & metadata, - const SelectQueryInfo & query_info, - const ContextPtr & context, - const std::shared_ptr & max_added_blocks, - const ActionDAGNodes & added_filter_nodes) -{ - MergeTreeData::DataPartsVector projection_parts; - MergeTreeData::DataPartsVector normal_parts; - for (const auto & part : parts) - { - const auto & created_projections = part->getProjectionParts(); - auto it = created_projections.find(candidate.projection->name); - if (it != created_projections.end()) - projection_parts.push_back(it->second); - else - normal_parts.push_back(part); - } - - if (projection_parts.empty()) - return false; - - auto projection_result_ptr = reader.estimateNumMarksToRead( - std::move(projection_parts), - nullptr, - required_column_names, - metadata, - candidate.projection->metadata, - query_info, /// How it is actually used? I hope that for index we need only added_filter_nodes - added_filter_nodes, - context, - context->getSettingsRef().max_threads, - max_added_blocks); - - if (projection_result_ptr->error()) - return false; - - candidate.merge_tree_projection_select_result_ptr = std::move(projection_result_ptr); - candidate.sum_marks += candidate.merge_tree_projection_select_result_ptr->marks(); - - if (!normal_parts.empty()) - { - auto normal_result_ptr = reading.selectRangesToRead(std::move(normal_parts)); - - if (normal_result_ptr->error()) - return false; - - if (normal_result_ptr->marks() != 0) - { - candidate.sum_marks += normal_result_ptr->marks(); - candidate.merge_tree_normal_select_result_ptr = std::move(normal_result_ptr); - } - } - - return true; -} - static QueryPlan::Node * findReadingStep(QueryPlan::Node & node) { IQueryPlanStep * step = node.step.get(); @@ -774,29 +521,7 @@ static QueryPlan::Node * findReadingStep(QueryPlan::Node & node) return nullptr; } -static bool canUseProjectionForReadingStep(ReadFromMergeTree * reading) -{ - /// Probably some projection already was applied. - if (reading->hasAnalyzedResult()) - return false; - - if (reading->isQueryWithFinal()) - return false; - - if (reading->isQueryWithSampling()) - return false; - - if (reading->isParallelReadingEnabled()) - return false; - - // Currently projection don't support deduplication when moving parts between shards. - if (reading->getContext()->getSettingsRef().allow_experimental_query_deduplication) - return false; - - return true; -} - -bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) +bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { if (node.children.size() != 1) return false; @@ -945,248 +670,4 @@ bool optimizeUseAggProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) return true; } - -static ActionsDAGPtr 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. - - 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; - - std::vector const_positions; - for (size_t i = 0; i < num_columns; ++i) - { - auto col_proj = proj_header.getByPosition(i).column; - auto col_main = main_header.getByPosition(i).column; - bool is_proj_const = col_proj && isColumnConst(*col_proj); - bool is_main_proj = col_main && isColumnConst(*col_main); - if (is_proj_const && !is_main_proj) - const_positions.push_back(i); - } - - if (const_positions.empty()) - return nullptr; - - ActionsDAGPtr dag = std::make_unique(); - auto & outputs = dag->getOutputs(); - for (const auto & col : proj_header.getColumnsWithTypeAndName()) - outputs.push_back(&dag->addInput(col)); - - for (auto pos : const_positions) - { - auto & output = outputs[pos]; - output = &dag->materializeNode(*output); - } - - return dag; -} - -static bool hasAllRequiredColumns(const ProjectionDescription * projection, const Names & required_columns) -{ - for (const auto & col : required_columns) - { - if (!projection->sample_block.has(col)) - return false; - } - - return true; -} - -bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) -{ - const auto & frame = stack.back(); - - auto * reading = typeid_cast(frame.node->step.get()); - if (!reading) - return false; - - if (!canUseProjectionForReadingStep(reading)) - return false; - - auto iter = stack.rbegin(); - while (iter != stack.rend()) - { - auto next = std::next(iter); - if (next == stack.rend()) - break; - - if (!typeid_cast(next->node->step.get()) && - !typeid_cast(next->node->step.get())) - break; - - iter = next; - } - - if (iter == stack.rbegin()) - return false; - - const auto metadata = reading->getStorageMetadata(); - const auto & projections = metadata->projections; - - std::vector normal_projections; - for (const auto & projection : projections) - if (projection.type == ProjectionDescription::Type::Normal) - normal_projections.push_back(&projection); - - if (normal_projections.empty()) - return false; - - NormalQueryDAG query; - { - if (!query.build(*iter->node->children.front())) - return false; - } - - std::list candidates; - NormalProjectionCandidate * best_candidate = nullptr; - - const Names & required_columns = reading->getRealColumnNames(); - const auto & parts = reading->getParts(); - const auto & query_info = reading->getQueryInfo(); - ContextPtr context = reading->getContext(); - MergeTreeDataSelectExecutor reader(reading->getMergeTreeData()); - - auto ordinary_reading_select_result = reading->selectRangesToRead(parts); - size_t ordinary_reading_marks = ordinary_reading_select_result->marks(); - - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), - // "Marks for ordinary reading {}", ordinary_reading_marks); - - std::shared_ptr max_added_blocks = getMaxAddedBlocks(reading); - - for (const auto * projection : normal_projections) - { - if (!hasAllRequiredColumns(projection, required_columns)) - continue; - - auto & candidate = candidates.emplace_back(); - candidate.projection = projection; - - ActionDAGNodes added_filter_nodes; - if (query.filter_node) - added_filter_nodes.nodes.push_back(query.filter_node); - - bool analyzed = analyzeProjectionCandidate( - candidate, *reading, reader, required_columns, parts, - metadata, query_info, context, max_added_blocks, added_filter_nodes); - - if (!analyzed) - continue; - - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), - // "Marks for projection {} {}", projection->name ,candidate.sum_marks); - - if (candidate.sum_marks >= ordinary_reading_marks) - continue; - - if (best_candidate == nullptr || candidate.sum_marks < best_candidate->sum_marks) - best_candidate = &candidate; - } - - if (!best_candidate) - { - reading->setAnalyzedResult(std::move(ordinary_reading_select_result)); - return false; - } - - auto storage_snapshot = reading->getStorageSnapshot(); - auto proj_snapshot = std::make_shared( - storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); //, storage_snapshot->data); - proj_snapshot->addProjection(best_candidate->projection); - - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}", - // proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString()); - - auto query_info_copy = query_info; - query_info_copy.prewhere_info = nullptr; - - auto projection_reading = reader.readFromParts( - {}, - required_columns, - proj_snapshot, - query_info_copy, - context, - reading->getMaxBlockSize(), - reading->getNumStreams(), - max_added_blocks, - best_candidate->merge_tree_projection_select_result_ptr, - reading->isParallelReadingEnabled()); - - if (!projection_reading) - { - Pipe pipe(std::make_shared(proj_snapshot->getSampleBlockForColumns(required_columns))); - projection_reading = std::make_unique(std::move(pipe)); - } - - bool has_nornal_parts = best_candidate->merge_tree_normal_select_result_ptr != nullptr; - if (has_nornal_parts) - reading->setAnalyzedResult(std::move(best_candidate->merge_tree_normal_select_result_ptr)); - - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}", - // projection_reading->getOutputStream().header.dumpStructure()); - - projection_reading->setStepDescription(best_candidate->projection->name); - - auto & projection_reading_node = nodes.emplace_back(QueryPlan::Node{.step = std::move(projection_reading)}); - auto * next_node = &projection_reading_node; - - if (query.dag) - { - auto & expr_or_filter_node = nodes.emplace_back(); - - if (query.filter_node) - { - expr_or_filter_node.step = std::make_unique( - projection_reading_node.step->getOutputStream(), - query.dag, - query.filter_node->result_name, - query.need_remove_column); - } - else - expr_or_filter_node.step = std::make_unique( - projection_reading_node.step->getOutputStream(), - query.dag); - - expr_or_filter_node.children.push_back(&projection_reading_node); - next_node = &expr_or_filter_node; - } - - if (!has_nornal_parts) - { - /// All parts are taken from projection - iter->node->children.front() = next_node; - } - else - { - const auto & main_stream = iter->node->children.front()->step->getOutputStream(); - const auto * proj_stream = &next_node->step->getOutputStream(); - - if (auto materializing = makeMaterializingDAG(proj_stream->header, main_stream.header)) - { - auto converting = std::make_unique(*proj_stream, materializing); - proj_stream = &converting->getOutputStream(); - auto & expr_node = nodes.emplace_back(); - expr_node.step = std::move(converting); - expr_node.children.push_back(next_node); - next_node = &expr_node; - } - - auto & union_node = nodes.emplace_back(); - DataStreams input_streams = {main_stream, *proj_stream}; - union_node.step = std::make_unique(std::move(input_streams)); - union_node.children = {iter->node->children.front(), next_node}; - iter->node->children.front() = &union_node; - - /// Here we remove last steps from stack to be able to optimize again. - /// In theory, read-in-order can be applied to projection. - iter->next_child = 0; - stack.resize(iter.base() - stack.begin() + 1); - } - - return true; -} - } diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp new file mode 100644 index 00000000000..847173788dd --- /dev/null +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -0,0 +1,311 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB::QueryPlanOptimizations +{ + +/// Normal projection analysis result in case it can be applied. +/// For now, it is empty. +/// Normal projection can be used only if it contains all required source columns. +/// It would not be hard to support pre-computed expressions and filtration. +struct NormalProjectionCandidate : public ProjectionCandidate +{ +}; + +static ActionsDAGPtr 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. + + 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; + + std::vector const_positions; + for (size_t i = 0; i < num_columns; ++i) + { + auto col_proj = proj_header.getByPosition(i).column; + auto col_main = main_header.getByPosition(i).column; + bool is_proj_const = col_proj && isColumnConst(*col_proj); + bool is_main_proj = col_main && isColumnConst(*col_main); + if (is_proj_const && !is_main_proj) + const_positions.push_back(i); + } + + if (const_positions.empty()) + return nullptr; + + ActionsDAGPtr dag = std::make_unique(); + auto & outputs = dag->getOutputs(); + for (const auto & col : proj_header.getColumnsWithTypeAndName()) + outputs.push_back(&dag->addInput(col)); + + for (auto pos : const_positions) + { + auto & output = outputs[pos]; + output = &dag->materializeNode(*output); + } + + return dag; +} + +static bool hasAllRequiredColumns(const ProjectionDescription * projection, const Names & required_columns) +{ + for (const auto & col : required_columns) + { + if (!projection->sample_block.has(col)) + return false; + } + + return true; +} + +struct NormalQueryDAG +{ + ActionsDAGPtr dag; + const ActionsDAG::Node * filter_node = nullptr; + + bool build(QueryPlan::Node & node); +}; + +bool NormalQueryDAG::build(QueryPlan::Node & node) +{ + QueryDAG query; + if (!query.build(node)) + return false; + + dag = std::move(query.dag); + auto & outputs = dag->getOutputs(); + auto filter_nodes = std::move(query.filter_nodes); + + if (!filter_nodes.empty()) + { + filter_node = filter_nodes.back(); + + if (filter_nodes.size() > 1) + { + /// Add a conjunction of all the filters. + + FunctionOverloadResolverPtr func_builder_and = + std::make_unique( + std::make_shared()); + + filter_node = &dag->addFunction(func_builder_and, std::move(filter_nodes), {}); + } + else + filter_node = &dag->addAlias(*filter_node, "_projection_filter"); + + outputs.insert(outputs.begin(), filter_node); + } + + if (dag) + { + dag->removeUnusedActions(); + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", dag->dumpDAG()); + } + + return true; +} + +bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) +{ + const auto & frame = stack.back(); + + auto * reading = typeid_cast(frame.node->step.get()); + if (!reading) + return false; + + if (!canUseProjectionForReadingStep(reading)) + return false; + + auto iter = stack.rbegin(); + while (std::next(iter) != stack.rend()) + { + iter = std::next(iter); + + std::cerr << "... " << iter->node->step->getName() << std::endl; + + if (!typeid_cast(iter->node->step.get()) && + !typeid_cast(iter->node->step.get())) + break; + } + + const auto metadata = reading->getStorageMetadata(); + const auto & projections = metadata->projections; + + std::vector normal_projections; + for (const auto & projection : projections) + if (projection.type == ProjectionDescription::Type::Normal) + normal_projections.push_back(&projection); + + if (normal_projections.empty()) + return false; + + NormalQueryDAG query; + { + auto & clild = iter->node->children[iter->next_child - 1]; + if (!query.build(*clild)) + return false; + } + + std::list candidates; + NormalProjectionCandidate * best_candidate = nullptr; + + const Names & required_columns = reading->getRealColumnNames(); + const auto & parts = reading->getParts(); + const auto & query_info = reading->getQueryInfo(); + ContextPtr context = reading->getContext(); + MergeTreeDataSelectExecutor reader(reading->getMergeTreeData()); + + auto ordinary_reading_select_result = reading->selectRangesToRead(parts); + size_t ordinary_reading_marks = ordinary_reading_select_result->marks(); + + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), + // "Marks for ordinary reading {}", ordinary_reading_marks); + + std::shared_ptr max_added_blocks = getMaxAddedBlocks(reading); + + for (const auto * projection : normal_projections) + { + if (!hasAllRequiredColumns(projection, required_columns)) + continue; + + auto & candidate = candidates.emplace_back(); + candidate.projection = projection; + + ActionDAGNodes added_filter_nodes; + if (query.filter_node) + added_filter_nodes.nodes.push_back(query.filter_node); + + bool analyzed = analyzeProjectionCandidate( + candidate, *reading, reader, required_columns, parts, + metadata, query_info, context, max_added_blocks, added_filter_nodes); + + if (!analyzed) + continue; + + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), + // "Marks for projection {} {}", projection->name ,candidate.sum_marks); + + if (candidate.sum_marks >= ordinary_reading_marks) + continue; + + if (best_candidate == nullptr || candidate.sum_marks < best_candidate->sum_marks) + best_candidate = &candidate; + } + + if (!best_candidate) + { + reading->setAnalyzedResult(std::move(ordinary_reading_select_result)); + return false; + } + + auto storage_snapshot = reading->getStorageSnapshot(); + auto proj_snapshot = std::make_shared( + storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); //, storage_snapshot->data); + proj_snapshot->addProjection(best_candidate->projection); + + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}", + // proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString()); + + auto query_info_copy = query_info; + query_info_copy.prewhere_info = nullptr; + + auto projection_reading = reader.readFromParts( + {}, + required_columns, + proj_snapshot, + query_info_copy, + context, + reading->getMaxBlockSize(), + reading->getNumStreams(), + max_added_blocks, + best_candidate->merge_tree_projection_select_result_ptr, + reading->isParallelReadingEnabled()); + + if (!projection_reading) + { + Pipe pipe(std::make_shared(proj_snapshot->getSampleBlockForColumns(required_columns))); + projection_reading = std::make_unique(std::move(pipe)); + } + + bool has_nornal_parts = best_candidate->merge_tree_normal_select_result_ptr != nullptr; + if (has_nornal_parts) + reading->setAnalyzedResult(std::move(best_candidate->merge_tree_normal_select_result_ptr)); + + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}", + // projection_reading->getOutputStream().header.dumpStructure()); + + projection_reading->setStepDescription(best_candidate->projection->name); + + auto & projection_reading_node = nodes.emplace_back(QueryPlan::Node{.step = std::move(projection_reading)}); + auto * next_node = &projection_reading_node; + + if (query.dag) + { + auto & expr_or_filter_node = nodes.emplace_back(); + + if (query.filter_node) + { + expr_or_filter_node.step = std::make_unique( + projection_reading_node.step->getOutputStream(), + query.dag, + query.filter_node->result_name, + true); + } + else + expr_or_filter_node.step = std::make_unique( + projection_reading_node.step->getOutputStream(), + query.dag); + + expr_or_filter_node.children.push_back(&projection_reading_node); + next_node = &expr_or_filter_node; + } + + if (!has_nornal_parts) + { + /// All parts are taken from projection + iter->node->children[iter->next_child - 1] = next_node; + } + else + { + const auto & main_stream = iter->node->children.front()->step->getOutputStream(); + const auto * proj_stream = &next_node->step->getOutputStream(); + + if (auto materializing = makeMaterializingDAG(proj_stream->header, main_stream.header)) + { + auto converting = std::make_unique(*proj_stream, materializing); + proj_stream = &converting->getOutputStream(); + auto & expr_node = nodes.emplace_back(); + expr_node.step = std::move(converting); + expr_node.children.push_back(next_node); + next_node = &expr_node; + } + + auto & union_node = nodes.emplace_back(); + DataStreams input_streams = {main_stream, *proj_stream}; + union_node.step = std::make_unique(std::move(input_streams)); + union_node.children = {iter->node->children.front(), next_node}; + iter->node->children[iter->next_child - 1] = &union_node; + } + + /// Here we remove last steps from stack to be able to optimize again. + /// In theory, read-in-order can be applied to projection. + stack.resize(iter.base() - stack.begin()); + + return true; +} + +} diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp new file mode 100644 index 00000000000..b4d1e5f02ff --- /dev/null +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -0,0 +1,210 @@ +#include + +#include +#include +#include + +#include +#include +#include + + +namespace DB::QueryPlanOptimizations +{ + +bool canUseProjectionForReadingStep(ReadFromMergeTree * reading) +{ + /// Probably some projection already was applied. + if (reading->hasAnalyzedResult()) + return false; + + if (reading->isQueryWithFinal()) + return false; + + if (reading->isQueryWithSampling()) + return false; + + if (reading->isParallelReadingEnabled()) + return false; + + // Currently projection don't support deduplication when moving parts between shards. + if (reading->getContext()->getSettingsRef().allow_experimental_query_deduplication) + return false; + + return true; +} + +std::shared_ptr getMaxAddedBlocks(ReadFromMergeTree * reading) +{ + ContextPtr context = reading->getContext(); + + if (context->getSettingsRef().select_sequential_consistency) + { + if (const auto * replicated = dynamic_cast(&reading->getMergeTreeData())) + return std::make_shared(replicated->getMaxAddedBlocks()); + } + + return {}; +} + +void QueryDAG::appendExpression(const ActionsDAGPtr & expression) +{ + if (dag) + dag->mergeInplace(std::move(*expression->clone())); + else + dag = expression->clone(); +} + +const ActionsDAG::Node * findInOutputs(ActionsDAG & dag, const std::string & name, bool remove) +{ + auto & outputs = dag.getOutputs(); + for (auto it = outputs.begin(); it != outputs.end(); ++it) + { + if ((*it)->result_name == name) + { + const auto * node = *it; + if (remove) + { + outputs.erase(it); + } + else + { + ColumnWithTypeAndName col; + col.name = node->result_name; + col.type = node->result_type; + col.column = col.type->createColumnConst(1, 1); + *it = &dag.addColumn(std::move(col)); + } + + return node; + } + } + + return nullptr; +} + +bool QueryDAG::build(QueryPlan::Node & node) +{ + IQueryPlanStep * step = node.step.get(); + if (auto * reading = typeid_cast(step)) + { + if (const auto * prewhere_info = reading->getPrewhereInfo()) + { + if (prewhere_info->row_level_filter) + { + appendExpression(prewhere_info->row_level_filter); + if (const auto * filter_node = findInOutputs(*dag, prewhere_info->row_level_column_name, false)) + filter_nodes.push_back(filter_node); + else + return false; + } + + if (prewhere_info->prewhere_actions) + { + appendExpression(prewhere_info->prewhere_actions); + if (const auto * filter_node = findInOutputs(*dag, prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column)) + filter_nodes.push_back(filter_node); + else + return false; + } + } + return true; + } + + if (node.children.size() != 1) + return false; + + if (!build(*node.children.front())) + return false; + + if (auto * expression = typeid_cast(step)) + { + const auto & actions = expression->getExpression(); + if (actions->hasArrayJoin()) + return false; + + appendExpression(actions); + return true; + } + + if (auto * filter = typeid_cast(step)) + { + const auto & actions = filter->getExpression(); + if (actions->hasArrayJoin()) + return false; + + appendExpression(actions); + const auto * filter_expression = findInOutputs(*dag, filter->getFilterColumnName(), filter->removesFilterColumn()); + if (!filter_expression) + return false; + + filter_nodes.push_back(filter_expression); + return true; + } + + return false; +} + +bool analyzeProjectionCandidate( + ProjectionCandidate & candidate, + const ReadFromMergeTree & reading, + const MergeTreeDataSelectExecutor & reader, + const Names & required_column_names, + const MergeTreeData::DataPartsVector & parts, + const StorageMetadataPtr & metadata, + const SelectQueryInfo & query_info, + const ContextPtr & context, + const std::shared_ptr & max_added_blocks, + const ActionDAGNodes & added_filter_nodes) +{ + MergeTreeData::DataPartsVector projection_parts; + MergeTreeData::DataPartsVector normal_parts; + for (const auto & part : parts) + { + const auto & created_projections = part->getProjectionParts(); + auto it = created_projections.find(candidate.projection->name); + if (it != created_projections.end()) + projection_parts.push_back(it->second); + else + normal_parts.push_back(part); + } + + if (projection_parts.empty()) + return false; + + auto projection_result_ptr = reader.estimateNumMarksToRead( + std::move(projection_parts), + nullptr, + required_column_names, + metadata, + candidate.projection->metadata, + query_info, /// How it is actually used? I hope that for index we need only added_filter_nodes + added_filter_nodes, + context, + context->getSettingsRef().max_threads, + max_added_blocks); + + if (projection_result_ptr->error()) + return false; + + candidate.merge_tree_projection_select_result_ptr = std::move(projection_result_ptr); + candidate.sum_marks += candidate.merge_tree_projection_select_result_ptr->marks(); + + if (!normal_parts.empty()) + { + auto normal_result_ptr = reading.selectRangesToRead(std::move(normal_parts)); + + if (normal_result_ptr->error()) + return false; + + if (normal_result_ptr->marks() != 0) + { + candidate.sum_marks += normal_result_ptr->marks(); + candidate.merge_tree_normal_select_result_ptr = std::move(normal_result_ptr); + } + } + + return true; +} + +} diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.h b/src/Processors/QueryPlan/Optimizations/projectionsCommon.h new file mode 100644 index 00000000000..04a268edab3 --- /dev/null +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.h @@ -0,0 +1,81 @@ +#pragma once +#include +#include + +namespace DB +{ + +class ReadFromMergeTree; + +using PartitionIdToMaxBlock = std::unordered_map; + +struct ProjectionDescription; + +class MergeTreeDataSelectExecutor; + +struct MergeTreeDataSelectAnalysisResult; +using MergeTreeDataSelectAnalysisResultPtr = std::shared_ptr; + +class IMergeTreeDataPart; +using DataPartPtr = std::shared_ptr; +using DataPartsVector = std::vector; + +struct StorageInMemoryMetadata; +using StorageMetadataPtr = std::shared_ptr; + +struct SelectQueryInfo; + +} + +namespace DB::QueryPlanOptimizations +{ + +/// Common checks that projection can be used for this step. +bool canUseProjectionForReadingStep(ReadFromMergeTree * reading); + +/// Max blocks for sequential consistency reading from replicated table. +std::shared_ptr getMaxAddedBlocks(ReadFromMergeTree * reading); + +/// This is a common DAG which is a merge of DAGs from Filter and Expression steps chain. +/// Additionally, for all the Filter steps, we collect filter conditions into filter_nodes. +/// Flag remove_last_filter_node is set in case if the last step is a Filter step and it should remove filter column. +struct QueryDAG +{ + ActionsDAGPtr dag; + ActionsDAG::NodeRawConstPtrs filter_nodes; + + bool build(QueryPlan::Node & node); + +private: + void appendExpression(const ActionsDAGPtr & expression); +}; + +struct ProjectionCandidate +{ + const ProjectionDescription * projection; + + /// The number of marks we are going to read + size_t sum_marks = 0; + + /// Analysis result, separate for parts with and without projection. + /// Analysis is done in order to estimate the number of marks we are going to read. + /// For chosen projection, it is reused for reading step. + MergeTreeDataSelectAnalysisResultPtr merge_tree_projection_select_result_ptr; + MergeTreeDataSelectAnalysisResultPtr merge_tree_normal_select_result_ptr; +}; + +/// This function fills ProjectionCandidate structure for specified projection. +/// It returns false if for some reason we cannot read from projection. +bool analyzeProjectionCandidate( + ProjectionCandidate & candidate, + const ReadFromMergeTree & reading, + const MergeTreeDataSelectExecutor & reader, + const Names & required_column_names, + const DataPartsVector & parts, + const StorageMetadataPtr & metadata, + const SelectQueryInfo & query_info, + const ContextPtr & context, + const std::shared_ptr & max_added_blocks, + const ActionDAGNodes & added_filter_nodes); + +} From 68da4f713e99b18d46ae2193e54aec6cae444938 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 17 Mar 2023 18:28:55 +0000 Subject: [PATCH 247/559] Real better fix logical error in evaluate constant expression --- src/Interpreters/ActionsVisitor.cpp | 12 +++++++++--- .../0_stateless/02680_mysql_ast_logical_err.sql | 4 ++-- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 43db25e775f..96765683abc 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -33,6 +33,8 @@ #include +#include + #include #include #include @@ -67,6 +69,7 @@ namespace ErrorCodes extern const int UNKNOWN_IDENTIFIER; extern const int NOT_AN_AGGREGATE; extern const int UNEXPECTED_EXPRESSION; + extern const int UNKNOWN_FUNCTION; extern const int TYPE_MISMATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int INCORRECT_ELEMENT_OF_SET; @@ -880,13 +883,16 @@ void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr &, Dat void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & data) { - auto column_name = ast->getColumnName(); - if (data.hasColumn(column_name)) - return; + if (TableFunctionFactory::instance().isTableFunctionName(node.name)) + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unexpected table function '{}'", node.name); if (node.name == "lambda") throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION, "Unexpected lambda expression"); + auto column_name = ast->getColumnName(); + if (data.hasColumn(column_name)) + return; + /// Function arrayJoin. if (node.name == "arrayJoin") { 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 39f743ee332..5b0530e05ae 100644 --- a/tests/queries/0_stateless/02680_mysql_ast_logical_err.sql +++ b/tests/queries/0_stateless/02680_mysql_ast_logical_err.sql @@ -1,2 +1,2 @@ -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 BAD_ARGUMENTS } - +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 } From b48c679328605f204dd1b47e567326bb2bf4f5a1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Mar 2023 20:51:58 +0100 Subject: [PATCH 248/559] Add style check for #47647 --- utils/check-style/check-style | 3 +++ 1 file changed, 3 insertions(+) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 53165d14f96..701e801ee5a 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -392,3 +392,6 @@ find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | # Check for existence of __init__.py files for i in "${ROOT_PATH}"/tests/integration/test_*; do FILE="${i}/__init__.py"; [ ! -f "${FILE}" ] && echo "${FILE} should exist for every integration test"; done + +# A small typo can lead to debug code in release builds, see https://github.com/ClickHouse/ClickHouse/pull/47647 +find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -l -F '#ifdef NDEBUG' | xargs -I@FILE awk '/#ifdef NDEBUG/ { inside = 1; dirty = 1 } /#endif/ { inside = 0; if (dirty) { print "File @FILE has suspicious #ifdef NDEBUG, possibly confused with #ifndef NDEBUG" } } /#else/ { dirty = 0 }' @FILE From c7762bd6671998d23887a53e13cd6324dc99ea8e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Mar 2023 22:55:01 +0300 Subject: [PATCH 249/559] Update check-style --- utils/check-style/check-style | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 701e801ee5a..f9b7a9bbbb7 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -394,4 +394,4 @@ find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | for i in "${ROOT_PATH}"/tests/integration/test_*; do FILE="${i}/__init__.py"; [ ! -f "${FILE}" ] && echo "${FILE} should exist for every integration test"; done # A small typo can lead to debug code in release builds, see https://github.com/ClickHouse/ClickHouse/pull/47647 -find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -l -F '#ifdef NDEBUG' | xargs -I@FILE awk '/#ifdef NDEBUG/ { inside = 1; dirty = 1 } /#endif/ { inside = 0; if (dirty) { print "File @FILE has suspicious #ifdef NDEBUG, possibly confused with #ifndef NDEBUG" } } /#else/ { dirty = 0 }' @FILE +find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -l -F '#ifdef NDEBUG' | xargs -I@FILE awk '/#ifdef NDEBUG/ { inside = 1; dirty = 1 } /#endif/ { if (inside && dirty) { print "File @FILE has suspicious #ifdef NDEBUG, possibly confused with #ifndef NDEBUG" }; inside = 0 } /#else/ { dirty = 0 }' @FILE From 023c0ba46eadaed590f961bc5d168f368cc5b285 Mon Sep 17 00:00:00 2001 From: alexX512 Date: Fri, 17 Mar 2023 20:05:10 +0000 Subject: [PATCH 250/559] Fix comments --- src/Core/Settings.h | 2 +- src/Processors/Executors/ExecutingGraph.cpp | 2 +- .../PullingAsyncPipelineExecutor.cpp | 9 ++++---- src/Server/TCPHandler.cpp | 21 +++---------------- src/Server/TCPHandler.h | 4 ++-- 5 files changed, 12 insertions(+), 26 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d671c92780a..eaed800ae88 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -280,7 +280,7 @@ class IColumn; \ M(Bool, final, false, "Query with the FINAL modifier by default. If the engine does not support final, it does not have any effect. On queries with multiple tables final is applied only on those that support it. It also works on distributed tables", 0) \ \ - M(Bool, stop_reading_on_first_cancel, false, "If setting is enabled, pipeline needs to return a result on a partially read table before the query is completely cancelled.", 0) \ + M(Bool, stop_reading_on_first_cancel, false, "Allows query to return a partial result after cancel.", 0) \ /** Settings for testing hedged requests */ \ M(Milliseconds, sleep_in_send_tables_status_ms, 0, "Time to sleep in sending tables status response in TCPHandler", 0) \ M(Milliseconds, sleep_in_send_data_ms, 0, "Time to sleep in sending data in TCPHandler", 0) \ diff --git a/src/Processors/Executors/ExecutingGraph.cpp b/src/Processors/Executors/ExecutingGraph.cpp index 37d58d01bee..9ec9cd2e0e4 100644 --- a/src/Processors/Executors/ExecutingGraph.cpp +++ b/src/Processors/Executors/ExecutingGraph.cpp @@ -124,7 +124,7 @@ bool ExecutingGraph::expandPipeline(std::stack & stack, uint64_t pid) source_processors.reserve(source_processors.size() + new_processors.size()); - for (auto& proc: new_processors) + for (auto & proc: new_processors) { bool is_source = proc->getInputs().empty(); source_processors.emplace_back(is_source); diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index 21c6a30bb5a..abe0cd6320b 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -181,7 +181,8 @@ void PullingAsyncPipelineExecutor::cancel() /// Cancel execution if it wasn't finished. cancelWithExceptionHandling([&]() { - data->executor->cancel(); + if (!data->is_finished && data->executor) + data->executor->cancel(); }); /// The following code is needed to rethrow exception from PipelineExecutor. @@ -203,7 +204,8 @@ void PullingAsyncPipelineExecutor::cancelReading() /// Stop reading from source if pipeline wasn't finished. cancelWithExceptionHandling([&]() { - data->executor->cancelReading(); + if (!data->is_finished && data->executor) + data->executor->cancelReading(); }); } @@ -211,8 +213,7 @@ void PullingAsyncPipelineExecutor::cancelWithExceptionHandling(CancelFunc && can { try { - if (!data->is_finished && data->executor) - cancel_func(); + cancel_func(); } catch (...) { diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index caf1a3dc075..f1b05d63250 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -113,21 +113,6 @@ namespace ErrorCodes extern const int QUERY_WAS_CANCELLED; } -std::string QueryState::cancellationStatusToName(CancellationStatus status) -{ - switch (status) - { - case CancellationStatus::FULLY_CANCELLED: - return "FULLY_CANCELLED"; - case CancellationStatus::READ_CANCELLED: - return "READ_CANCELLED"; - case CancellationStatus::NOT_CANCELLED: - return "NOT_CANCELLED"; - } - - UNREACHABLE(); -} - TCPHandler::TCPHandler(IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_, bool parse_proxy_protocol_, std::string server_display_name_) : Poco::Net::TCPServerConnection(socket_) , server(server_) @@ -1807,9 +1792,9 @@ void TCPHandler::initProfileEventsBlockOutput(const Block & block) } } -void TCPHandler::decreaseCancellationStatus(const std::string& log_message) +void TCPHandler::decreaseCancellationStatus(const std::string & log_message) { - auto prev_status = QueryState::cancellationStatusToName(state.cancellation_status); + auto prev_status = magic_enum::enum_name(state.cancellation_status); bool stop_reading_on_first_cancel = false; if (query_context) @@ -1827,7 +1812,7 @@ void TCPHandler::decreaseCancellationStatus(const std::string& log_message) state.cancellation_status = CancellationStatus::FULLY_CANCELLED; } - auto current_status = QueryState::cancellationStatusToName(state.cancellation_status); + auto current_status = magic_enum::enum_name(state.cancellation_status); LOG_INFO(log, "Change cancellation status from {} to {}. Log message: {}", prev_status, current_status, log_message); } diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 0bca2572d4f..cb50b149629 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -281,9 +281,9 @@ private: void initLogsBlockOutput(const Block & block); void initProfileEventsBlockOutput(const Block & block); - using CancellationStatus = typename QueryState::CancellationStatus; + using CancellationStatus = QueryState::CancellationStatus; - void decreaseCancellationStatus(const std::string& log_message); + void decreaseCancellationStatus(const std::string & log_message); CancellationStatus getQueryCancellationStatus(); /// This function is called from different threads. From 5034449d564eaa8b12d52a4b134707f8eb156792 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 17 Mar 2023 22:12:34 +0100 Subject: [PATCH 251/559] more random query id in tests --- .../queries/0_stateless/02434_cancel_insert_when_client_dies.sh | 2 +- tests/queries/0_stateless/02435_rollback_cancelled_queries.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh index 85aa992d98c..2b27d88f23c 100755 --- a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh +++ b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh @@ -47,7 +47,7 @@ function thread_insert { # supress "Killed" messages from bash while true; do - export ID="$TEST_MARK$RANDOM" + export ID="$TEST_MARK$RANDOM-$RANDOM-$RANDOM" bash -c insert_data 2>&1| grep -Fav "Killed" done } diff --git a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh index e4aec6503a4..640975f1ef7 100755 --- a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh +++ b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh @@ -61,7 +61,7 @@ function thread_insert { # supress "Killed" messages from bash while true; do - export ID="$TEST_MARK$RANDOM" + export ID="$TEST_MARK$RANDOM-$RANDOM-$RANDOM" bash -c insert_data 2>&1| grep -Fav "Killed" | grep -Fav "SESSION_IS_LOCKED" | grep -Fav "SESSION_NOT_FOUND" done } From c14c083cdff487abfff299fe2b6c08e6a5b28d07 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 17 Mar 2023 19:30:12 -0400 Subject: [PATCH 252/559] Fix crash in polygonsSymDifferenceCartesian --- src/Functions/geometryConverters.h | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/src/Functions/geometryConverters.h b/src/Functions/geometryConverters.h index 3dbf3763fdc..21cc19665dc 100644 --- a/src/Functions/geometryConverters.h +++ b/src/Functions/geometryConverters.h @@ -86,7 +86,9 @@ struct ColumnToPointsConverter } }; - +/** + * Class which converts Column with type Array(Tuple(Float64, Float64)) to a vector of boost ring type. +*/ template struct ColumnToRingsConverter { @@ -106,7 +108,9 @@ struct ColumnToRingsConverter } }; - +/** + * Class which converts Column with type Array(Array(Tuple(Float64, Float64))) to a vector of boost polygon type. +*/ template struct ColumnToPolygonsConverter { @@ -116,6 +120,9 @@ struct ColumnToPolygonsConverter std::vector> answer(offsets.size()); auto all_rings = ColumnToRingsConverter::convert(typeid_cast(*col).getDataPtr()); + if (all_rings.empty()) + return answer; + size_t prev_offset = 0; for (size_t iter = 0; iter < offsets.size(); ++iter) { @@ -131,7 +138,9 @@ struct ColumnToPolygonsConverter } }; - +/** + * Class which converts Column with type Array(Array(Array(Tuple(Float64, Float64)))) to a vector of boost multi_polygon type. +*/ template struct ColumnToMultiPolygonsConverter { @@ -143,6 +152,9 @@ struct ColumnToMultiPolygonsConverter auto all_polygons = ColumnToPolygonsConverter::convert(typeid_cast(*col).getDataPtr()); + if (all_polygons.empty()) + return answer; + for (size_t iter = 0; iter < offsets.size(); ++iter) { for (size_t polygon_iter = prev_offset; polygon_iter < offsets[iter]; ++polygon_iter) From aea421cc9b08dec2cedd6e20e1c5e57bf3f472b5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 18 Mar 2023 03:29:14 +0100 Subject: [PATCH 253/559] Add a style check for unsafe code --- src/Functions/FunctionsBitmap.h | 36 +++++++++++++------------- src/Functions/FunctionsHashing.h | 2 +- src/Functions/array/arrayIntersect.cpp | 6 ++--- src/Interpreters/ExpressionActions.cpp | 4 +-- utils/check-style/check-style | 3 +++ 5 files changed, 27 insertions(+), 24 deletions(-) diff --git a/src/Functions/FunctionsBitmap.h b/src/Functions/FunctionsBitmap.h index bb83c58699a..cea04f1c49e 100644 --- a/src/Functions/FunctionsBitmap.h +++ b/src/Functions/FunctionsBitmap.h @@ -404,19 +404,19 @@ private: } if (is_column_const[0]) - col_agg_func = typeid_cast(typeid_cast(column_ptrs[0])->getDataColumnPtr().get()); + col_agg_func = &typeid_cast(*typeid_cast(*column_ptrs[0]).getDataColumnPtr()); else - col_agg_func = typeid_cast(column_ptrs[0]); + col_agg_func = &typeid_cast(*column_ptrs[0]); container0 = &col_agg_func->getData(); if (is_column_const[1]) - container1 = &typeid_cast(typeid_cast(column_ptrs[1])->getDataColumnPtr().get())->getData(); + container1 = &typeid_cast(*typeid_cast(column_ptrs[1]).getDataColumnPtr()).getData(); else - container1 = &typeid_cast(column_ptrs[1])->getData(); + container1 = &typeid_cast(*column_ptrs[1]).getData(); if (is_column_const[2]) - container2 = &typeid_cast(typeid_cast(column_ptrs[2])->getDataColumnPtr().get())->getData(); + container2 = &typeid_cast(*typeid_cast(column_ptrs[2]).getDataColumnPtr()).getData(); else - container2 = &typeid_cast(column_ptrs[2])->getData(); + container2 = &typeid_cast(*column_ptrs[2]).getData(); auto col_to = ColumnAggregateFunction::create(col_agg_func->getAggregateFunction()); col_to->reserve(input_rows_count); @@ -587,29 +587,29 @@ private: if (is_column_const[0]) { - col_agg_func = typeid_cast(typeid_cast(column_ptrs[0])->getDataColumnPtr().get()); + col_agg_func = &typeid_cast(*typeid_cast(*column_ptrs[0]).getDataColumnPtr()); } else { - col_agg_func = typeid_cast(column_ptrs[0]); + col_agg_func = &typeid_cast(*column_ptrs[0]); } container0 = &col_agg_func->getData(); if (is_column_const[1]) - array1 = typeid_cast(typeid_cast(column_ptrs[1])->getDataColumnPtr().get()); + array1 = &typeid_cast(*typeid_cast(*column_ptrs[1]).getDataColumnPtr()); else - array1 = typeid_cast(column_ptrs[1]); + array1 = &typeid_cast(*column_ptrs[1]); const ColumnArray::Offsets & from_offsets = array1->getOffsets(); - const ColumnVector::Container & from_container = typeid_cast *>(&array1->getData())->getData(); + const ColumnVector::Container & from_container = typeid_cast &>(array1->getData()).getData(); if (is_column_const[2]) - array2 = typeid_cast(typeid_cast(column_ptrs[2])->getDataColumnPtr().get()); + array2 = &typeid_cast(*typeid_cast(*column_ptrs[2]).getDataColumnPtr()); else - array2 = typeid_cast(column_ptrs[2]); + array2 = &typeid_cast(*column_ptrs[2]); const ColumnArray::Offsets & to_offsets = array2->getOffsets(); - const ColumnVector::Container & to_container = typeid_cast *>(&array2->getData())->getData(); + const ColumnVector::Container & to_container = typeid_cast &>(array2->getData()).getData(); auto col_to = ColumnAggregateFunction::create(col_agg_func->getAggregateFunction()); col_to->reserve(input_rows_count); @@ -911,9 +911,9 @@ private: is_column_const[0] = isColumnConst(*column_ptrs[0]); if (is_column_const[0]) - container0 = &typeid_cast(typeid_cast(column_ptrs[0])->getDataColumnPtr().get())->getData(); + container0 = &typeid_cast(*typeid_cast(*column_ptrs[0]).getDataColumnPtr()).getData(); else - container0 = &typeid_cast(column_ptrs[0])->getData(); + container0 = &typeid_cast(*column_ptrs[0]).getData(); // we can always cast the second column to ColumnUInt64 auto uint64_column = castColumn(arguments[1], std::make_shared()); @@ -921,9 +921,9 @@ private: is_column_const[1] = isColumnConst(*column_ptrs[1]); if (is_column_const[1]) - container1 = &typeid_cast(typeid_cast(column_ptrs[1])->getDataColumnPtr().get())->getData(); + container1 = &typeid_cast(*typeid_cast(*column_ptrs[1]).getDataColumnPtr()).getData(); else - container1 = &typeid_cast(column_ptrs[1])->getData(); + container1 = &typeid_cast(*column_ptrs[1]).getData(); for (size_t i = 0; i < input_rows_count; ++i) { diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 7b6f4213cd3..4a3d698135e 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1216,7 +1216,7 @@ private: template void executeArray(const KeyType & key, const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to) const { - const IDataType * nested_type = typeid_cast(type)->getNestedType().get(); + const IDataType * nested_type = typeid_cast(*type).getNestedType().get(); if (const ColumnArray * col_from = checkAndGetColumn(column)) { diff --git a/src/Functions/array/arrayIntersect.cpp b/src/Functions/array/arrayIntersect.cpp index c6f0a5afa62..5da9a2025bb 100644 --- a/src/Functions/array/arrayIntersect.cpp +++ b/src/Functions/array/arrayIntersect.cpp @@ -313,7 +313,7 @@ FunctionArrayIntersect::UnpackedArrays FunctionArrayIntersect::prepareArrays( { arg.is_const = true; argument_column = argument_column_const->getDataColumnPtr().get(); - initial_column = typeid_cast(initial_column)->getDataColumnPtr().get(); + initial_column = &typeid_cast(*initial_column).getDataColumn(); } if (const auto * argument_column_array = typeid_cast(argument_column)) @@ -324,13 +324,13 @@ FunctionArrayIntersect::UnpackedArrays FunctionArrayIntersect::prepareArrays( arg.offsets = &argument_column_array->getOffsets(); arg.nested_column = &argument_column_array->getData(); - initial_column = &typeid_cast(initial_column)->getData(); + initial_column = &typeid_cast(*initial_column).getData(); if (const auto * column_nullable = typeid_cast(arg.nested_column)) { arg.null_map = &column_nullable->getNullMapData(); arg.nested_column = &column_nullable->getNestedColumn(); - initial_column = &typeid_cast(initial_column)->getNestedColumn(); + initial_column = &typeid_cast(*initial_column).getNestedColumn(); } /// In case column was casted need to create overflow mask for integer types. diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 4c9f47e5915..d47d5880fde 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -1111,12 +1111,12 @@ void ExpressionActionsChain::JoinStep::finalize(const NameSet & required_output_ ActionsDAGPtr & ExpressionActionsChain::Step::actions() { - return typeid_cast(this)->actions_dag; + return typeid_cast(*this).actions_dag; } const ActionsDAGPtr & ExpressionActionsChain::Step::actions() const { - return typeid_cast(this)->actions_dag; + return typeid_cast(*this).actions_dag; } } diff --git a/utils/check-style/check-style b/utils/check-style/check-style index f9b7a9bbbb7..946e29856d1 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -395,3 +395,6 @@ for i in "${ROOT_PATH}"/tests/integration/test_*; do FILE="${i}/__init__.py"; [ # A small typo can lead to debug code in release builds, see https://github.com/ClickHouse/ClickHouse/pull/47647 find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -l -F '#ifdef NDEBUG' | xargs -I@FILE awk '/#ifdef NDEBUG/ { inside = 1; dirty = 1 } /#endif/ { if (inside && dirty) { print "File @FILE has suspicious #ifdef NDEBUG, possibly confused with #ifndef NDEBUG" }; inside = 0 } /#else/ { dirty = 0 }' @FILE + +# If a user is doing dynamic or typeid cast with a pointer, and immediately dereferencing it, it is unsafe. +find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep --line-number -P '(dynamic|typeid)_cast<[^>]+\*>\([^\(\)]+\)->' | grep -P '.' && echo "It's suspicious when you are doing a dynamic_cast or typeid_cast with a pointer and immediately dereferencing it. Use references instead of pointers or check a pointer to nullptr." From 292793a231e7244cda43428e0e0caf88c0af08fb Mon Sep 17 00:00:00 2001 From: rfraposa Date: Fri, 17 Mar 2023 21:30:21 -0500 Subject: [PATCH 254/559] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index f1bba4dc2fc..753d8167a86 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1,4 +1,4 @@ -personal_ws-1.1 en 543 +personal_ws-1.1 en 543 AArch ACLs AMQP @@ -22,6 +22,7 @@ CSVWithNamesAndTypes CamelCase CapnProto CentOS +ClickableSquare ClickHouse ClickHouse's CodeBlock @@ -53,6 +54,7 @@ HDDs Heredoc Homebrew Homebrew's +HorizontalDivide Hostname IPv IntN @@ -87,6 +89,7 @@ LOCALTIME LOCALTIMESTAMP LibFuzzer LineAsString +LinksDeployment LowCardinality MEMTABLE MSan @@ -169,6 +172,7 @@ Testflows Tgz Toolset Tradeoff +TwoColumnList UBSan UInt UIntN @@ -178,6 +182,8 @@ Updatable Util Valgrind Vectorized +VideoContainer +ViewAllLink VirtualBox Werror WithNamesAndTypes From ac5ed141d8651ef87c72b53d78bb1afd5e79df56 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Fri, 17 Mar 2023 21:45:43 -0500 Subject: [PATCH 255/559] New nav - reverting the revert --- docs/en/development/tests.md | 2 +- docs/en/engines/_category_.yml | 1 - .../table-engines/integrations/mysql.md | 2 +- .../table-engines/integrations/odbc.md | 2 +- .../table-engines/integrations/postgresql.md | 2 +- .../mergetree-family/mergetree.md | 2 +- .../mergetree-family/replication.md | 4 +- .../table-engines/special/dictionary.md | 2 +- .../table-engines/special/distributed.md | 2 +- .../example-datasets/cell-towers.md | 7 +- .../example-datasets/criteo.md | 93 +- .../example-datasets/github.md | 17 +- .../example-datasets/metrica.md | 6 +- .../example-datasets/nypd_complaint_data.md | 110 +- .../example-datasets/recipes.md | 4 +- .../example-datasets/uk-price-paid.md | 14 +- docs/en/getting-started/install.md | 71 +- docs/en/getting-started/playground.md | 4 +- docs/en/interfaces/formats.md | 4 +- docs/en/interfaces/postgresql.md | 2 +- docs/en/operations/_category_.yml | 4 - docs/en/operations/access-rights.md | 152 -- docs/en/operations/backup.md | 3 +- docs/en/operations/caches.md | 1 + docs/en/operations/clickhouse-keeper.md | 378 --- .../external-authenticators/kerberos.md | 2 +- .../external-authenticators/ldap.md | 6 +- docs/en/operations/monitoring.md | 1 + .../settings.md | 7 +- .../operations/settings/settings-profiles.md | 2 +- docs/en/operations/settings/settings-users.md | 4 +- docs/en/operations/settings/settings.md | 34 +- .../operations/system-tables/dictionaries.md | 20 +- .../system-tables/marked_dropped_tables.md | 37 - docs/en/operations/system-tables/quotas.md | 2 +- docs/en/operations/system-tables/roles.md | 2 +- docs/en/operations/system-tables/users.md | 2 +- docs/en/operations/tips.md | 4 +- .../operations/utilities/clickhouse-local.md | 4 +- docs/en/operations/utilities/index.md | 4 +- docs/en/sql-reference/_category_.yml | 6 +- .../reference/contingency.md | 2 +- .../reference/cramersvbiascorrected.md | 2 +- .../reference/exponentialmovingaverage.md | 2 +- docs/en/sql-reference/data-types/json.md | 2 +- .../_snippet_dictionary_in_cloud.md | 0 .../external-dictionaries/_category_.yml | 8 - .../external-dicts-dict-hierarchical.md | 67 - .../external-dicts-dict-layout.md | 751 ------ .../external-dicts-dict-lifetime.md | 142 - .../external-dicts-dict-polygon.md | 140 - .../external-dicts-dict-sources.md | 847 ------ .../external-dicts-dict-structure.md | 181 -- .../external-dicts-dict.md | 57 - .../external-dictionaries/external-dicts.md | 84 - .../external-dictionaries/regexp-tree.md | 76 - docs/en/sql-reference/dictionaries/index.md | 2355 ++++++++++++++++- .../dictionaries/internal-dicts.md | 55 - .../functions/date-time-functions.md | 6 +- .../functions/ext-dict-functions.md | 12 +- .../functions/other-functions.md | 28 +- docs/en/sql-reference/index.md | 22 + .../en/sql-reference/sql-reference-links.json | 12 + .../statements/alter/projection.md | 6 +- .../statements/create/dictionary.md | 10 +- .../sql-reference/statements/create/quota.md | 2 +- .../sql-reference/statements/create/role.md | 4 +- .../statements/create/row-policy.md | 6 +- .../statements/create/settings-profile.md | 6 +- .../sql-reference/statements/create/user.md | 2 +- docs/en/sql-reference/statements/delete.md | 8 +- docs/en/sql-reference/statements/detach.md | 2 +- .../sql-reference/statements/insert-into.md | 8 +- .../statements/select/array-join.md | 8 +- .../statements/select/group-by.md | 18 +- .../sql-reference/statements/select/index.md | 4 +- .../sql-reference/statements/select/join.md | 4 +- docs/en/sql-reference/statements/show.md | 14 +- docs/en/sql-reference/statements/system.md | 4 +- .../table-functions/dictionary.md | 2 +- .../table-functions/executable.md | 2 +- .../sql-reference/table-functions/mongodb.md | 4 +- .../en/sql-reference/table-functions/mysql.md | 4 +- docs/en/sql-reference/table-functions/odbc.md | 2 +- .../table-functions/postgresql.md | 6 +- .../database-engines/materialized-mysql.md | 2 +- .../mergetree-family/mergetree.md | 24 +- .../engines/table-engines/special/buffer.md | 2 +- .../operations/multi-region-replication.md | 2 +- docs/ru/getting-started/tutorial.md | 2 +- docs/ru/interfaces/formats.md | 8 +- docs/ru/interfaces/http.md | 2 +- .../sampling-query-profiler.md | 2 +- .../settings.md | 6 +- .../system-tables/information_schema.md | 2 +- .../system-tables/replicated_fetches.md | 2 +- .../utilities/clickhouse-benchmark.md | 2 +- docs/ru/sql-reference/data-types/datetime.md | 8 +- .../functions/date-time-functions.md | 2 +- .../functions/other-functions.md | 4 +- .../statements/select/array-join.md | 4 +- docs/zh/development/continuous-integration.md | 2 +- docs/zh/engines/database-engines/index.md | 2 +- .../database-engines/materialize-mysql.md | 20 +- .../database-engines/materialized-mysql.md | 18 +- .../mergetree-family/mergetree.md | 12 +- .../faq/general/why-clickhouse-is-so-fast.md | 2 +- docs/zh/faq/integration/index.md | 4 +- docs/zh/faq/integration/json-import.md | 12 +- docs/zh/faq/operations/delete-old-data.md | 4 +- docs/zh/faq/operations/index.md | 6 +- docs/zh/faq/operations/production.md | 4 +- docs/zh/faq/use-cases/index.md | 2 +- .../example-datasets/recipes.mdx | 26 +- .../skipping-indexes.md | 6 +- .../sparse-primary-indexes.md | 196 +- docs/zh/interfaces/http.md | 2 +- .../sampling-query-profiler.md | 2 +- docs/zh/operations/settings/settings-users.md | 4 +- .../system-tables/data_type_families.md | 2 +- .../system-tables/replicated_fetches.md | 2 +- .../reference/grouparrayinsertat.md | 2 +- ...gon.mdx => external-dicts-dict-polygon.md} | 4 +- .../statements/create/database.md | 2 +- .../sql-reference/statements/create/view.md | 2 +- docs/zh/sql-reference/statements/index.md | 2 +- .../statements/select/array-join.md | 4 +- .../statements/select/group-by.md | 4 +- .../sql-reference/statements/select/index.md | 2 +- 129 files changed, 2936 insertions(+), 3491 deletions(-) delete mode 100644 docs/en/operations/access-rights.md delete mode 100644 docs/en/operations/clickhouse-keeper.md delete mode 100644 docs/en/operations/system-tables/marked_dropped_tables.md rename docs/en/sql-reference/dictionaries/{external-dictionaries => }/_snippet_dictionary_in_cloud.md (100%) delete mode 100644 docs/en/sql-reference/dictionaries/external-dictionaries/_category_.yml delete mode 100644 docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md delete mode 100644 docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md delete mode 100644 docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md delete mode 100644 docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md delete mode 100644 docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md delete mode 100644 docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md delete mode 100644 docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md delete mode 100644 docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md delete mode 100644 docs/en/sql-reference/dictionaries/external-dictionaries/regexp-tree.md delete mode 100644 docs/en/sql-reference/dictionaries/internal-dicts.md create mode 100644 docs/en/sql-reference/index.md create mode 100644 docs/en/sql-reference/sql-reference-links.json rename docs/zh/sql-reference/dictionaries/external-dictionaries/{external-dicts-dict-polygon.mdx => external-dicts-dict-polygon.md} (59%) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index eb424ee7cbe..98dbe5f8d57 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -71,7 +71,7 @@ SELECT 1 | `global` | Same as `shard`. Prefer `shard` || | `zookeeper` | Test requires Zookeeper or ClickHouse Keeper to run | Test uses `ReplicatedMergeTree` | | `replica` | Same as `zookeeper`. Prefer `zookeeper` || -| `no-fasttest`| Test is not run under [Fast test](continuous-integration#fast-test) | Test uses `MySQL` table engine which is disabled in Fast test| +| `no-fasttest`| Test is not run under [Fast test](continuous-integration.md#fast-test) | Test uses `MySQL` table engine which is disabled in Fast test| | `no-[asan, tsan, msan, ubsan]` | Disables tests in build with [sanitizers](#sanitizers) | Test is run under QEMU which doesn't work with sanitizers | | `no-replicated-database` ||| | `no-ordinary-database` ||| diff --git a/docs/en/engines/_category_.yml b/docs/en/engines/_category_.yml index 8c6ba12c6f1..2aa5df72955 100644 --- a/docs/en/engines/_category_.yml +++ b/docs/en/engines/_category_.yml @@ -4,5 +4,4 @@ collapsible: true collapsed: true link: type: generated-index - title: Database & Table Engines slug: /en/engines diff --git a/docs/en/engines/table-engines/integrations/mysql.md b/docs/en/engines/table-engines/integrations/mysql.md index e00347c3163..4b285ee80a5 100644 --- a/docs/en/engines/table-engines/integrations/mysql.md +++ b/docs/en/engines/table-engines/integrations/mysql.md @@ -180,4 +180,4 @@ Default value: `300`. ## See Also {#see-also} - [The mysql table function](../../../sql-reference/table-functions/mysql.md) -- [Using MySQL as a dictionary source](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-mysql) +- [Using MySQL as a dictionary source](../../../sql-reference/dictionaries/index.md#dictionary-sources#dicts-external_dicts_dict_sources-mysql) diff --git a/docs/en/engines/table-engines/integrations/odbc.md b/docs/en/engines/table-engines/integrations/odbc.md index aabc37442f9..8aac9dc3af0 100644 --- a/docs/en/engines/table-engines/integrations/odbc.md +++ b/docs/en/engines/table-engines/integrations/odbc.md @@ -126,5 +126,5 @@ SELECT * FROM odbc_t ## See Also {#see-also} -- [ODBC dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-odbc) +- [ODBC dictionaries](../../../sql-reference/dictionaries/index.md#dictionary-sources#dicts-external_dicts_dict_sources-odbc) - [ODBC table function](../../../sql-reference/table-functions/odbc.md) diff --git a/docs/en/engines/table-engines/integrations/postgresql.md b/docs/en/engines/table-engines/integrations/postgresql.md index b73d28c8508..18e884f3bcc 100644 --- a/docs/en/engines/table-engines/integrations/postgresql.md +++ b/docs/en/engines/table-engines/integrations/postgresql.md @@ -174,7 +174,7 @@ CREATE TABLE pg_table_schema_with_dots (a UInt32) **See Also** - [The `postgresql` table function](../../../sql-reference/table-functions/postgresql.md) -- [Using PostgreSQL as a dictionary source](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-postgresql) +- [Using PostgreSQL as a dictionary source](../../../sql-reference/dictionaries/index.md#dictionary-sources#dicts-external_dicts_dict_sources-postgresql) ## Related content - Blog: [ClickHouse and PostgreSQL - a match made in data heaven - part 1](https://clickhouse.com/blog/migrating-data-between-clickhouse-postgres) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index fc8060077b0..f60afe2b829 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -901,7 +901,7 @@ User can assign new big parts to different disks of a [JBOD](https://en.wikipedi ## Using S3 for Data Storage {#table_engine-mergetree-s3} :::note -Google Cloud Storage (GCS) is also supported using the type `s3`. See [GCS backed MergeTree](/docs/en/integrations/data-ingestion/s3/gcs-merge-tree.md). +Google Cloud Storage (GCS) is also supported using the type `s3`. See [GCS backed MergeTree](/docs/en/integrations/gcs). ::: `MergeTree` family table engines can store data to [S3](https://aws.amazon.com/s3/) using a disk with type `s3`. diff --git a/docs/en/engines/table-engines/mergetree-family/replication.md b/docs/en/engines/table-engines/mergetree-family/replication.md index 37ab8ac9fd3..c50433f2aeb 100644 --- a/docs/en/engines/table-engines/mergetree-family/replication.md +++ b/docs/en/engines/table-engines/mergetree-family/replication.md @@ -39,7 +39,7 @@ Compressed data for `INSERT` and `ALTER` queries is replicated (for more informa - The `DROP TABLE` query deletes the replica located on the server where the query is run. - The `RENAME` query renames the table on one of the replicas. In other words, replicated tables can have different names on different replicas. -ClickHouse uses [ClickHouse Keeper](/docs/en/guides/sre/keeper/clickhouse-keeper.md) for storing replicas meta information. It is possible to use ZooKeeper version 3.4.5 or newer, but ClickHouse Keeper is recommended. +ClickHouse uses [ClickHouse Keeper](/docs/en/guides/sre/keeper/index.md) for storing replicas meta information. It is possible to use ZooKeeper version 3.4.5 or newer, but ClickHouse Keeper is recommended. To use replication, set parameters in the [zookeeper](/docs/en/operations/server-configuration-parameters/settings.md/#server-settings_zookeeper) server configuration section. @@ -144,7 +144,7 @@ ENGINE = ReplicatedReplacingMergeTree The `Replicated` prefix is added to the table engine name. For example:`ReplicatedMergeTree`. :::tip -Adding `Replicated` is optional in ClickHouse Cloud, as all of the tables are replicated. +Adding `Replicated` is optional in ClickHouse Cloud, as all of the tables are replicated. ::: ### Replicated\*MergeTree parameters diff --git a/docs/en/engines/table-engines/special/dictionary.md b/docs/en/engines/table-engines/special/dictionary.md index e487ca2002f..05d07d94e56 100644 --- a/docs/en/engines/table-engines/special/dictionary.md +++ b/docs/en/engines/table-engines/special/dictionary.md @@ -6,7 +6,7 @@ sidebar_label: Dictionary # Dictionary Table Engine -The `Dictionary` engine displays the [dictionary](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) data as a ClickHouse table. +The `Dictionary` engine displays the [dictionary](../../../sql-reference/dictionaries/index.md) data as a ClickHouse table. ## Example {#example} diff --git a/docs/en/engines/table-engines/special/distributed.md b/docs/en/engines/table-engines/special/distributed.md index f4f541843d3..52d82483a46 100644 --- a/docs/en/engines/table-engines/special/distributed.md +++ b/docs/en/engines/table-engines/special/distributed.md @@ -184,7 +184,7 @@ The parameters `host`, `port`, and optionally `user`, `password`, `secure`, `com - `host` – The address of the remote server. You can use either the domain or the IPv4 or IPv6 address. If you specify the domain, the server makes a DNS request when it starts, and the result is stored as long as the server is running. If the DNS request fails, the server does not start. If you change the DNS record, restart the server. - `port` – The TCP port for messenger activity (`tcp_port` in the config, usually set to 9000). Not to be confused with `http_port`. -- `user` – Name of the user for connecting to a remote server. Default value is the `default` user. This user must have access to connect to the specified server. Access is configured in the `users.xml` file. For more information, see the section [Access rights](../../../operations/access-rights.md). +- `user` – Name of the user for connecting to a remote server. Default value is the `default` user. This user must have access to connect to the specified server. Access is configured in the `users.xml` file. For more information, see the section [Access rights](../../../guides/sre/user-management/index.md). - `password` – The password for connecting to a remote server (not masked). Default value: empty string. - `secure` - Whether to use a secure SSL/TLS connection. Usually also requires specifying the port (the default secure port is `9440`). The server should listen on `9440` and be configured with correct certificates. - `compression` - Use data compression. Default value: `true`. diff --git a/docs/en/getting-started/example-datasets/cell-towers.md b/docs/en/getting-started/example-datasets/cell-towers.md index b19d09c777a..d88ce5159d4 100644 --- a/docs/en/getting-started/example-datasets/cell-towers.md +++ b/docs/en/getting-started/example-datasets/cell-towers.md @@ -1,9 +1,10 @@ --- slug: /en/getting-started/example-datasets/cell-towers -sidebar_label: Cell Towers +sidebar_label: Geo Data sidebar_position: 3 -title: "Cell Towers" +title: "Geo Data using the Cell Tower Dataset" --- + import ConnectionDetails from '@site/docs/en/_snippets/_gather_your_details_http.mdx'; import Tabs from '@theme/Tabs'; @@ -163,7 +164,7 @@ SELECT mcc, count() FROM cell_towers GROUP BY mcc ORDER BY count() DESC LIMIT 10 Based on the above query and the [MCC list](https://en.wikipedia.org/wiki/Mobile_country_code), the countries with the most cell towers are: the USA, Germany, and Russia. -You may want to create a [Dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) in ClickHouse to decode these values. +You may want to create a [Dictionary](../../sql-reference/dictionaries/index.md) in ClickHouse to decode these values. ## Use case: Incorporate geo data {#use-case} diff --git a/docs/en/getting-started/example-datasets/criteo.md b/docs/en/getting-started/example-datasets/criteo.md index 3bd0230d4cc..a2e0fda0cb0 100644 --- a/docs/en/getting-started/example-datasets/criteo.md +++ b/docs/en/getting-started/example-datasets/criteo.md @@ -3,14 +3,56 @@ slug: /en/getting-started/example-datasets/criteo sidebar_label: Terabyte Click Logs from Criteo --- -# Terabyte of Click Logs from Criteo +# Terabyte of Click Logs from Criteo Download the data from http://labs.criteo.com/downloads/download-terabyte-click-logs/ Create a table to import the log to: ``` sql -CREATE TABLE criteo_log (date Date, clicked UInt8, int1 Int32, int2 Int32, int3 Int32, int4 Int32, int5 Int32, int6 Int32, int7 Int32, int8 Int32, int9 Int32, int10 Int32, int11 Int32, int12 Int32, int13 Int32, cat1 String, cat2 String, cat3 String, cat4 String, cat5 String, cat6 String, cat7 String, cat8 String, cat9 String, cat10 String, cat11 String, cat12 String, cat13 String, cat14 String, cat15 String, cat16 String, cat17 String, cat18 String, cat19 String, cat20 String, cat21 String, cat22 String, cat23 String, cat24 String, cat25 String, cat26 String) ENGINE = Log +CREATE TABLE criteo_log ( + date Date, + clicked UInt8, + int1 Int32, + int2 Int32, + int3 Int32, + int4 Int32, + int5 Int32, + int6 Int32, + int7 Int32, + int8 Int32, + int9 Int32, + int10 Int32, + int11 Int32, + int12 Int32, + int13 Int32, + cat1 String, + cat2 String, + cat3 String, + cat4 String, + cat5 String, + cat6 String, + cat7 String, + cat8 String, + cat9 String, + cat10 String, + cat11 String, + cat12 String, + cat13 String, + cat14 String, + cat15 String, + cat16 String, + cat17 String, + cat18 String, + cat19 String, + cat20 String, + cat21 String, + cat22 String, + cat23 String, + cat24 String, + cat25 String, + cat26 String +) ENGINE = Log; ``` Download the data: @@ -73,7 +115,52 @@ ORDER BY (date, icat1) Transform data from the raw log and put it in the second table: ``` sql -INSERT INTO criteo SELECT date, clicked, int1, int2, int3, int4, int5, int6, int7, int8, int9, int10, int11, int12, int13, reinterpretAsUInt32(unhex(cat1)) AS icat1, reinterpretAsUInt32(unhex(cat2)) AS icat2, reinterpretAsUInt32(unhex(cat3)) AS icat3, reinterpretAsUInt32(unhex(cat4)) AS icat4, reinterpretAsUInt32(unhex(cat5)) AS icat5, reinterpretAsUInt32(unhex(cat6)) AS icat6, reinterpretAsUInt32(unhex(cat7)) AS icat7, reinterpretAsUInt32(unhex(cat8)) AS icat8, reinterpretAsUInt32(unhex(cat9)) AS icat9, reinterpretAsUInt32(unhex(cat10)) AS icat10, reinterpretAsUInt32(unhex(cat11)) AS icat11, reinterpretAsUInt32(unhex(cat12)) AS icat12, reinterpretAsUInt32(unhex(cat13)) AS icat13, reinterpretAsUInt32(unhex(cat14)) AS icat14, reinterpretAsUInt32(unhex(cat15)) AS icat15, reinterpretAsUInt32(unhex(cat16)) AS icat16, reinterpretAsUInt32(unhex(cat17)) AS icat17, reinterpretAsUInt32(unhex(cat18)) AS icat18, reinterpretAsUInt32(unhex(cat19)) AS icat19, reinterpretAsUInt32(unhex(cat20)) AS icat20, reinterpretAsUInt32(unhex(cat21)) AS icat21, reinterpretAsUInt32(unhex(cat22)) AS icat22, reinterpretAsUInt32(unhex(cat23)) AS icat23, reinterpretAsUInt32(unhex(cat24)) AS icat24, reinterpretAsUInt32(unhex(cat25)) AS icat25, reinterpretAsUInt32(unhex(cat26)) AS icat26 FROM criteo_log; +INSERT INTO + criteo +SELECT + date, + clicked, + int1, + int2, + int3, + int4, + int5, + int6, + int7, + int8, + int9, + int10, + int11, + int12, + int13, + reinterpretAsUInt32(unhex(cat1)) AS icat1, + reinterpretAsUInt32(unhex(cat2)) AS icat2, + reinterpretAsUInt32(unhex(cat3)) AS icat3, + reinterpretAsUInt32(unhex(cat4)) AS icat4, + reinterpretAsUInt32(unhex(cat5)) AS icat5, + reinterpretAsUInt32(unhex(cat6)) AS icat6, + reinterpretAsUInt32(unhex(cat7)) AS icat7, + reinterpretAsUInt32(unhex(cat8)) AS icat8, + reinterpretAsUInt32(unhex(cat9)) AS icat9, + reinterpretAsUInt32(unhex(cat10)) AS icat10, + reinterpretAsUInt32(unhex(cat11)) AS icat11, + reinterpretAsUInt32(unhex(cat12)) AS icat12, + reinterpretAsUInt32(unhex(cat13)) AS icat13, + reinterpretAsUInt32(unhex(cat14)) AS icat14, + reinterpretAsUInt32(unhex(cat15)) AS icat15, + reinterpretAsUInt32(unhex(cat16)) AS icat16, + reinterpretAsUInt32(unhex(cat17)) AS icat17, + reinterpretAsUInt32(unhex(cat18)) AS icat18, + reinterpretAsUInt32(unhex(cat19)) AS icat19, + reinterpretAsUInt32(unhex(cat20)) AS icat20, + reinterpretAsUInt32(unhex(cat21)) AS icat21, + reinterpretAsUInt32(unhex(cat22)) AS icat22, + reinterpretAsUInt32(unhex(cat23)) AS icat23, + reinterpretAsUInt32(unhex(cat24)) AS icat24, + reinterpretAsUInt32(unhex(cat25)) AS icat25, + reinterpretAsUInt32(unhex(cat26)) AS icat26 +FROM + criteo_log; DROP TABLE criteo_log; ``` diff --git a/docs/en/getting-started/example-datasets/github.md b/docs/en/getting-started/example-datasets/github.md index 239637a34e9..e18c7dec1a6 100644 --- a/docs/en/getting-started/example-datasets/github.md +++ b/docs/en/getting-started/example-datasets/github.md @@ -1,12 +1,13 @@ --- slug: /en/getting-started/example-datasets/github -sidebar_label: GitHub Repo Analysis +sidebar_label: Github Repo +sidebar_position: 1 description: Analyze the ClickHouse GitHub repo or any repository of your choosing --- -# ClickHouse GitHub data +# Writing Queries in ClickHouse using GitHub Data -This dataset contains all of the commits and changes for the ClickHouse repository. It can be generated using the native `git-import` tool distributed with ClickHouse. +This dataset contains all of the commits and changes for the ClickHouse repository. It can be generated using the native `git-import` tool distributed with ClickHouse. The generated data provides a `tsv` file for each of the following tables: @@ -323,7 +324,7 @@ Note a more complex variant of this query exists where we find the [line-by-line ## Find the current active files -This is important for later analysis when we only want to consider the current files in the repository. We estimate this set as the files which haven't been renamed or deleted (and then re-added/re-named). +This is important for later analysis when we only want to consider the current files in the repository. We estimate this set as the files which haven't been renamed or deleted (and then re-added/re-named). **Note there appears to have been a broken commit history in relation to files under the `dbms`, `libs`, `tests/testflows/` directories during their renames. We also thus exclude these.** @@ -417,7 +418,7 @@ git ls-files | grep -v -E 'generated\.cpp|^(contrib|docs?|website|libs/(libcityh The difference here is caused by a few factors: -- A rename can occur alongside other modifications to the file. These are listed as separate events in file_changes but with the same time. The `argMax` function has no way of distinguishing these - it picks the first value. The natural ordering of the inserts (the only means of knowing the correct order) is not maintained across the union so modified events can be selected. For example, below the `src/Functions/geometryFromColumn.h` file has several modifications before being renamed to `src/Functions/geometryConverters.h`. Our current solution may pick a Modify event as the latest change causing `src/Functions/geometryFromColumn.h` to be retained. +- A rename can occur alongside other modifications to the file. These are listed as separate events in file_changes but with the same time. The `argMax` function has no way of distinguishing these - it picks the first value. The natural ordering of the inserts (the only means of knowing the correct order) is not maintained across the union so modified events can be selected. For example, below the `src/Functions/geometryFromColumn.h` file has several modifications before being renamed to `src/Functions/geometryConverters.h`. Our current solution may pick a Modify event as the latest change causing `src/Functions/geometryFromColumn.h` to be retained. [play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICAgIGNoYW5nZV90eXBlLAogICAgICBwYXRoLAogICAgICBvbGRfcGF0aCwKICAgICAgdGltZSwKICAgICAgY29tbWl0X2hhc2gKICBGUk9NIGdpdF9jbGlja2hvdXNlLmZpbGVfY2hhbmdlcwogIFdIRVJFIChwYXRoID0gJ3NyYy9GdW5jdGlvbnMvZ2VvbWV0cnlGcm9tQ29sdW1uLmgnKSBPUiAob2xkX3BhdGggPSAnc3JjL0Z1bmN0aW9ucy9nZW9tZXRyeUZyb21Db2x1bW4uaCcpCg==) @@ -1386,7 +1387,7 @@ LIMIT 1 BY day_of_week 7 rows in set. Elapsed: 0.004 sec. Processed 21.82 thousand rows, 140.02 KB (4.88 million rows/s., 31.29 MB/s.) ``` -This is still a little simple and doesn't reflect people's work. +This is still a little simple and doesn't reflect people's work. A better metric might be who is the top contributor each day as a fraction of the total work performed in the last year. Note that we treat the deletion and adding code equally. @@ -1952,7 +1953,7 @@ SELECT Most contributors write more code than tests, as you'd expect. -What about who adds the most comments when contributing code? +What about who adds the most comments when contributing code? [play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBhdXRob3IsCiAgICBhdmcocmF0aW9fY29tbWVudHMpIEFTIGF2Z19yYXRpb19jb21tZW50cywKICAgIHN1bShjb2RlKSBBUyBjb2RlCkZST00KKAogICAgU0VMRUNUCiAgICAgICAgYXV0aG9yLAogICAgICAgIGNvbW1pdF9oYXNoLAogICAgICAgIGNvdW50SWYobGluZV90eXBlID0gJ0NvbW1lbnQnKSBBUyBjb21tZW50cywKICAgICAgICBjb3VudElmKGxpbmVfdHlwZSA9ICdDb2RlJykgQVMgY29kZSwKICAgICAgICBpZihjb21tZW50cyA+IDAsIGNvbW1lbnRzIC8gKGNvbW1lbnRzICsgY29kZSksIDApIEFTIHJhdGlvX2NvbW1lbnRzCiAgICBGUk9NIGdpdF9jbGlja2hvdXNlLmxpbmVfY2hhbmdlcwogICAgR1JPVVAgQlkKICAgICAgICBhdXRob3IsCiAgICAgICAgY29tbWl0X2hhc2gKKQpHUk9VUCBCWSBhdXRob3IKT1JERVIgQlkgY29kZSBERVNDCkxJTUlUIDEwCg==) @@ -2393,7 +2394,7 @@ WHERE (path = 'src/Storages/StorageReplicatedMergeTree.cpp') AND (change_type = This makes viewing the full history of a file challenging since we don't have a single value connecting all line or file changes. -To address this, we can use User Defined Functions (UDFs). These cannot, currently, be recursive, so to identify the history of a file we must define a series of UDFs which call each other explicitly. +To address this, we can use User Defined Functions (UDFs). These cannot, currently, be recursive, so to identify the history of a file we must define a series of UDFs which call each other explicitly. This means we can only track renames to a maximum depth - the below example is 5 deep. It is unlikely a file will be renamed more times than this, so for now, this is sufficient. diff --git a/docs/en/getting-started/example-datasets/metrica.md b/docs/en/getting-started/example-datasets/metrica.md index e966f6c20d6..e21237f39bb 100644 --- a/docs/en/getting-started/example-datasets/metrica.md +++ b/docs/en/getting-started/example-datasets/metrica.md @@ -84,7 +84,7 @@ clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1" 1680609 ``` -## An example JOIN +## An example JOIN The hits and visits dataset is used in the ClickHouse test routines, this is one of the queries from the test suite. The rest @@ -131,10 +131,10 @@ FORMAT PrettyCompact" ## Next Steps -[A Practical Introduction to Sparse Primary Indexes in ClickHouse](../../guides/improving-query-performance/sparse-primary-indexes/sparse-primary-indexes-intro.md) uses the hits dataset to discuss the differences in ClickHouse indexing compared to traditional relational databases, how ClickHouse builds and uses a sparse primary index, and indexing best practices. +[A Practical Introduction to Sparse Primary Indexes in ClickHouse](/docs/en/guides/best-practices/sparse-primary-indexes.md) uses the hits dataset to discuss the differences in ClickHouse indexing compared to traditional relational databases, how ClickHouse builds and uses a sparse primary index, and indexing best practices. Additional examples of queries to these tables can be found among the ClickHouse [stateful tests](https://github.com/ClickHouse/ClickHouse/blob/d7129855757f38ceec3e4ecc6dafacdabe9b178f/tests/queries/1_stateful/00172_parallel_join.sql). :::note -The test suite uses a database name `test`, and the tables are named `hits` and `visits`. You can rename your database and tables, or edit the SQL from the test file. +The test suite uses a database name `test`, and the tables are named `hits` and `visits`. You can rename your database and tables, or edit the SQL from the test file. ::: diff --git a/docs/en/getting-started/example-datasets/nypd_complaint_data.md b/docs/en/getting-started/example-datasets/nypd_complaint_data.md index 8b02ac23cf9..154cfa78e53 100644 --- a/docs/en/getting-started/example-datasets/nypd_complaint_data.md +++ b/docs/en/getting-started/example-datasets/nypd_complaint_data.md @@ -16,7 +16,7 @@ While working through this guide you will: The dataset used in this guide comes from the NYC Open Data team, and contains data about "all valid felony, misdemeanor, and violation crimes reported to the New York City Police Department (NYPD)". At the time of writing, the data file is 166MB, but it is updated regularly. -**Source**: [data.cityofnewyork.us](https://data.cityofnewyork.us/Public-Safety/NYPD-Complaint-Data-Current-Year-To-Date-/5uac-w243) +**Source**: [data.cityofnewyork.us](https://data.cityofnewyork.us/Public-Safety/NYPD-Complaint-Data-Current-Year-To-Date-/5uac-w243) **Terms of use**: https://www1.nyc.gov/home/terms-of-use.page ## Prerequisites @@ -35,7 +35,7 @@ The examples in this guide assume that you have saved the TSV file to `${HOME}/N ## Familiarize yourself with the TSV file -Before starting to work with the ClickHouse database familiarize yourself with the data. +Before starting to work with the ClickHouse database familiarize yourself with the data. ### Look at the fields in the source TSV file @@ -47,15 +47,15 @@ clickhouse-local --query \ Sample response ```response -CMPLNT_NUM Nullable(Float64) -ADDR_PCT_CD Nullable(Float64) -BORO_NM Nullable(String) -CMPLNT_FR_DT Nullable(String) -CMPLNT_FR_TM Nullable(String) +CMPLNT_NUM Nullable(Float64) +ADDR_PCT_CD Nullable(Float64) +BORO_NM Nullable(String) +CMPLNT_FR_DT Nullable(String) +CMPLNT_FR_TM Nullable(String) ``` :::tip -Most of the time the above command will let you know which fields in the input data are numeric, and which are strings, and which are tuples. This is not always the case. Because ClickHouse is routineley used with datasets containing billions of records there is a default number (100) of rows examined to [infer the schema](../../guides/developer/working-with-json/json-semi-structured.md/#relying-on-schema-inference) in order to avoid parsing billions of rows to infer the schema. The response below may not match what you see, as the dataset is updated several times each year. Looking at the Data Dictionary you can see that CMPLNT_NUM is specified as text, and not numeric. By overriding the default of 100 rows for inference with the setting `SETTINGS input_format_max_rows_to_read_for_schema_inference=2000` +Most of the time the above command will let you know which fields in the input data are numeric, and which are strings, and which are tuples. This is not always the case. Because ClickHouse is routineley used with datasets containing billions of records there is a default number (100) of rows examined to [infer the schema](/docs/en/integrations/data-ingestion/data-formats/json.md#relying-on-schema-inference) in order to avoid parsing billions of rows to infer the schema. The response below may not match what you see, as the dataset is updated several times each year. Looking at the Data Dictionary you can see that CMPLNT_NUM is specified as text, and not numeric. By overriding the default of 100 rows for inference with the setting `SETTINGS input_format_max_rows_to_read_for_schema_inference=2000` you can get a better idea of the content. Note: as of version 22.5 the default is now 25,000 rows for inferring the schema, so only change the setting if you are on an older version or if you need more than 25,000 rows to be sampled. @@ -65,46 +65,46 @@ Run this command at your command prompt. You will be using `clickhouse-local` t ```sh clickhouse-local --input_format_max_rows_to_read_for_schema_inference=2000 \ --query \ -"describe file('${HOME}/NYPD_Complaint_Data_Current__Year_To_Date_.tsv', 'TSVWithNames')" +"describe file('${HOME}/NYPD_Complaint_Data_Current__Year_To_Date_.tsv', 'TSVWithNames')" ``` Result: ```response -CMPLNT_NUM Nullable(String) -ADDR_PCT_CD Nullable(Float64) -BORO_NM Nullable(String) -CMPLNT_FR_DT Nullable(String) -CMPLNT_FR_TM Nullable(String) -CMPLNT_TO_DT Nullable(String) -CMPLNT_TO_TM Nullable(String) -CRM_ATPT_CPTD_CD Nullable(String) -HADEVELOPT Nullable(String) -HOUSING_PSA Nullable(Float64) -JURISDICTION_CODE Nullable(Float64) -JURIS_DESC Nullable(String) -KY_CD Nullable(Float64) -LAW_CAT_CD Nullable(String) -LOC_OF_OCCUR_DESC Nullable(String) -OFNS_DESC Nullable(String) -PARKS_NM Nullable(String) -PATROL_BORO Nullable(String) -PD_CD Nullable(Float64) -PD_DESC Nullable(String) -PREM_TYP_DESC Nullable(String) -RPT_DT Nullable(String) -STATION_NAME Nullable(String) -SUSP_AGE_GROUP Nullable(String) -SUSP_RACE Nullable(String) -SUSP_SEX Nullable(String) -TRANSIT_DISTRICT Nullable(Float64) -VIC_AGE_GROUP Nullable(String) -VIC_RACE Nullable(String) -VIC_SEX Nullable(String) -X_COORD_CD Nullable(Float64) -Y_COORD_CD Nullable(Float64) -Latitude Nullable(Float64) -Longitude Nullable(Float64) -Lat_Lon Tuple(Nullable(Float64), Nullable(Float64)) +CMPLNT_NUM Nullable(String) +ADDR_PCT_CD Nullable(Float64) +BORO_NM Nullable(String) +CMPLNT_FR_DT Nullable(String) +CMPLNT_FR_TM Nullable(String) +CMPLNT_TO_DT Nullable(String) +CMPLNT_TO_TM Nullable(String) +CRM_ATPT_CPTD_CD Nullable(String) +HADEVELOPT Nullable(String) +HOUSING_PSA Nullable(Float64) +JURISDICTION_CODE Nullable(Float64) +JURIS_DESC Nullable(String) +KY_CD Nullable(Float64) +LAW_CAT_CD Nullable(String) +LOC_OF_OCCUR_DESC Nullable(String) +OFNS_DESC Nullable(String) +PARKS_NM Nullable(String) +PATROL_BORO Nullable(String) +PD_CD Nullable(Float64) +PD_DESC Nullable(String) +PREM_TYP_DESC Nullable(String) +RPT_DT Nullable(String) +STATION_NAME Nullable(String) +SUSP_AGE_GROUP Nullable(String) +SUSP_RACE Nullable(String) +SUSP_SEX Nullable(String) +TRANSIT_DISTRICT Nullable(Float64) +VIC_AGE_GROUP Nullable(String) +VIC_RACE Nullable(String) +VIC_SEX Nullable(String) +X_COORD_CD Nullable(Float64) +Y_COORD_CD Nullable(Float64) +Latitude Nullable(Float64) +Longitude Nullable(Float64) +Lat_Lon Tuple(Nullable(Float64), Nullable(Float64)) New Georeferenced Column Nullable(String) ``` @@ -362,7 +362,7 @@ The dates shown as `1925` above are from errors in the data. There are several The decisions made above on the data types used for the columns are reflected in the table schema below. We also need to decide on the `ORDER BY` and `PRIMARY KEY` used for the table. At least one -of `ORDER BY` or `PRIMARY KEY` must be specified. Here are some guidelines on deciding on the +of `ORDER BY` or `PRIMARY KEY` must be specified. Here are some guidelines on deciding on the columns to includes in `ORDER BY`, and more information is in the *Next Steps* section at the end of this document. @@ -420,7 +420,7 @@ ORDER BY ( borough, offense_description, date_reported ) Putting together the changes to data types and the `ORDER BY` tuple gives this table structure: ```sql -CREATE TABLE NYPD_Complaint ( +CREATE TABLE NYPD_Complaint ( complaint_number String, precinct UInt8, borough LowCardinality(String), @@ -429,7 +429,7 @@ CREATE TABLE NYPD_Complaint ( was_crime_completed String, housing_authority String, housing_level_code UInt32, - jurisdiction_code UInt8, + jurisdiction_code UInt8, jurisdiction LowCardinality(String), offense_code UInt8, offense_level LowCardinality(String), @@ -478,7 +478,7 @@ Query id: 6a5b10bf-9333-4090-b36e-c7f08b1d9e01 Row 1: ────── -partition_key: +partition_key: sorting_key: borough, offense_description, date_reported primary_key: borough, offense_description, date_reported table: NYPD_Complaint @@ -495,7 +495,7 @@ We will use `clickhouse-local` tool for data preprocessing and `clickhouse-clien :::tip `table='input'` appears in the arguments to clickhouse-local below. clickhouse-local takes the provided input (`cat ${HOME}/NYPD_Complaint_Data_Current__Year_To_Date_.tsv`) and inserts the input into a table. By default the table is named `table`. In this guide the name of the table is set to `input` to make the data flow clearer. The final argument to clickhouse-local is a query that selects from the table (`FROM input`) which is then piped to `clickhouse-client` to populate the table `NYPD_Complaint`. ::: - + ```sql cat ${HOME}/NYPD_Complaint_Data_Current__Year_To_Date_.tsv \ | clickhouse-local --table='input' --input-format='TSVWithNames' \ @@ -512,12 +512,12 @@ cat ${HOME}/NYPD_Complaint_Data_Current__Year_To_Date_.tsv \ CRM_ATPT_CPTD_CD AS was_crime_completed, HADEVELOPT AS housing_authority_development, HOUSING_PSA AS housing_level_code, - JURISDICTION_CODE AS jurisdiction_code, + JURISDICTION_CODE AS jurisdiction_code, JURIS_DESC AS jurisdiction, KY_CD AS offense_code, LAW_CAT_CD AS offense_level, LOC_OF_OCCUR_DESC AS location_descriptor, - OFNS_DESC AS offense_description, + OFNS_DESC AS offense_description, PARKS_NM AS park_name, PATROL_BORO AS patrol_borough, PD_CD, @@ -529,7 +529,7 @@ cat ${HOME}/NYPD_Complaint_Data_Current__Year_To_Date_.tsv \ SUSP_RACE AS suspect_race, SUSP_SEX AS suspect_sex, TRANSIT_DISTRICT AS transit_district, - VIC_AGE_GROUP AS victim_age_group, + VIC_AGE_GROUP AS victim_age_group, VIC_RACE AS victim_race, VIC_SEX AS victim_sex, X_COORD_CD AS NY_x_coordinate, @@ -538,7 +538,7 @@ cat ${HOME}/NYPD_Complaint_Data_Current__Year_To_Date_.tsv \ Longitude FROM input" \ | clickhouse-client --query='INSERT INTO NYPD_Complaint FORMAT TSV' -``` +``` ## Validate the Data {#validate-data} @@ -560,7 +560,7 @@ Result: │ 208993 │ └─────────┘ -1 row in set. Elapsed: 0.001 sec. +1 row in set. Elapsed: 0.001 sec. ``` The size of the dataset in ClickHouse is just 12% of the original TSV file, compare the size of the original TSV file with the size of the table: @@ -651,4 +651,4 @@ Query id: 8cdcdfd4-908f-4be0-99e3-265722a2ab8d ## Next Steps -[A Practical Introduction to Sparse Primary Indexes in ClickHouse](../../guides/improving-query-performance/sparse-primary-indexes/sparse-primary-indexes-intro.md) discusses the differences in ClickHouse indexing compared to traditional relational databases, how ClickHouse builds and uses a sparse primary index, and indexing best practices. +[A Practical Introduction to Sparse Primary Indexes in ClickHouse](/docs/en/guides/best-practices/sparse-primary-indexes.md) discusses the differences in ClickHouse indexing compared to traditional relational databases, how ClickHouse builds and uses a sparse primary index, and indexing best practices. diff --git a/docs/en/getting-started/example-datasets/recipes.md b/docs/en/getting-started/example-datasets/recipes.md index 4cc94c3ce5b..729d3d17015 100644 --- a/docs/en/getting-started/example-datasets/recipes.md +++ b/docs/en/getting-started/example-datasets/recipes.md @@ -80,7 +80,7 @@ Result: ### Top Components by the Number of Recipes: -In this example we learn how to use [arrayJoin](../../sql-reference/functions/array-join/) function to expand an array into a set of rows. +In this example we learn how to use [arrayJoin](../../sql-reference/functions/array-join.md) function to expand an array into a set of rows. Query: @@ -185,7 +185,7 @@ Result: 10 rows in set. Elapsed: 0.215 sec. Processed 2.23 million rows, 1.48 GB (10.35 million rows/s., 6.86 GB/s.) ``` -In this example, we involve [has](../../sql-reference/functions/array-functions/#hasarr-elem) function to filter by array elements and sort by the number of directions. +In this example, we involve [has](../../sql-reference/functions/array-functions.md#hasarr-elem) function to filter by array elements and sort by the number of directions. There is a wedding cake that requires the whole 126 steps to produce! Show that directions: diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index 2a89bfda2e7..8ed79c3986f 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -1,17 +1,17 @@ --- slug: /en/getting-started/example-datasets/uk-price-paid -sidebar_label: UK Property Price Paid +sidebar_label: UK Property Prices sidebar_position: 1 -title: "UK Property Price Paid" --- -The dataset contains data about prices paid for real-estate property in England and Wales. The data is available since year 1995. -The size of the dataset in uncompressed form is about 4 GiB and it will take about 278 MiB in ClickHouse. +# The UK property prices dataset -Source: https://www.gov.uk/government/statistical-data-sets/price-paid-data-downloads -Description of the fields: https://www.gov.uk/guidance/about-the-price-paid-data +Projections are a great way to improve the performance of queries that you run frequently. We will demonstrate the power of projections +using the UK property dataset, which contains data about prices paid for real-estate property in England and Wales. The data is available since 1995, and the size of the dataset in uncompressed form is about 4 GiB (which will only take about 278 MiB in ClickHouse). -Contains HM Land Registry data © Crown copyright and database right 2021. This data is licensed under the Open Government Licence v3.0. +- Source: https://www.gov.uk/government/statistical-data-sets/price-paid-data-downloads +- Description of the fields: https://www.gov.uk/guidance/about-the-price-paid-data +- Contains HM Land Registry data © Crown copyright and database right 2021. This data is licensed under the Open Government Licence v3.0. ## Create the Table {#create-table} diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index 0867f3a0795..ef4b79dcf63 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -14,75 +14,35 @@ import CodeBlock from '@theme/CodeBlock'; You have three options for getting up and running with ClickHouse: - **[ClickHouse Cloud](https://clickhouse.com/cloud/):** The official ClickHouse as a service, - built by, maintained and supported by the creators of ClickHouse -- **[Self-managed ClickHouse](#self-managed-install):** ClickHouse can run on any Linux, FreeBSD, or macOS with x86-64, ARM, or PowerPC64LE CPU architecture -- **[Docker Image](https://hub.docker.com/r/clickhouse/clickhouse-server/):** Read the guide with the official image in Docker Hub +- **[Quick Install](#quick-install):** an easy-to-download binary for testing and developing with ClickHouse +- **[Production Deployments](#available-installation-options):** ClickHouse can run on any Linux, FreeBSD, or macOS with x86-64, ARM, or PowerPC64LE CPU architecture +- **[Docker Image](https://hub.docker.com/r/clickhouse/clickhouse-server/):** use the official Docker image in Docker Hub ## ClickHouse Cloud The quickest and easiest way to get up and running with ClickHouse is to create a new service in [ClickHouse Cloud](https://clickhouse.cloud/). -## Self-Managed Install +## Quick Install :::tip For production installs of a specific release version see the [installation options](#available-installation-options) down below. ::: - - +On Linux and macOS: -1. The simplest way to download ClickHouse locally is to run the following command. If your operating system is supported, an appropriate ClickHouse binary will be downloaded and made runnable: +1. If you are just getting started and want to see what ClickHouse can do, the simplest way to download ClickHouse locally is to run the following command. It downloads a single binary for your operating system that can be used to run the ClickHouse server, clickhouse-client, clickhouse-local, +ClickHouse Keeper, and other tools: ```bash curl https://clickhouse.com/ | sh ``` -1. Run the `install` command, which defines a collection of useful symlinks along with the files and folders used by ClickHouse - all of which you can see in the output of the install script: - - ```bash - sudo ./clickhouse install - ``` - -1. At the end of the install script, you are prompted for a password for the `default` user. Feel free to enter a password, or you can optionally leave it blank: - - ```response - Creating log directory /var/log/clickhouse-server. - Creating data directory /var/lib/clickhouse. - Creating pid directory /var/run/clickhouse-server. - chown -R clickhouse:clickhouse '/var/log/clickhouse-server' - chown -R clickhouse:clickhouse '/var/run/clickhouse-server' - chown clickhouse:clickhouse '/var/lib/clickhouse' - Enter password for default user: - ``` - You should see the following output: - - ```response - ClickHouse has been successfully installed. - - Start clickhouse-server with: - sudo clickhouse start - - Start clickhouse-client with: - clickhouse-client - ``` - 1. Run the following command to start the ClickHouse server: ```bash - sudo clickhouse start + ./clickhouse server ``` - - - -1. The simplest way to download ClickHouse locally is to run the following command. If your operating system is supported, an appropriate ClickHouse binary will be downloaded and made runnable: - ```bash - curl https://clickhouse.com/ | sh - ``` - -1. Run the ClickHouse server: - - ```bash - ./clickhouse server - ``` + The first time you run this script, the necessary files and folders are created in the current directory, then the server starts. 1. Open a new terminal and use the **clickhouse-client** to connect to your service: @@ -101,15 +61,14 @@ For production installs of a specific release version see the [installation opti You are ready to start sending DDL and SQL commands to ClickHouse! - - - :::tip -The [Quick Start](/docs/en/quick-start.mdx/#step-1-get-clickhouse) walks through the steps to download and run ClickHouse, connect to it, and insert data. +The [Quick Start](/docs/en/quick-start.mdx) walks through the steps for creating tables and inserting data. ::: -## Available Installation Options {#available-installation-options} +## Production Deployments {#available-installation-options} + +For production deployments of ClickHouse, choose from one of the following install options. ### From DEB Packages {#install-from-deb-packages} @@ -174,7 +133,7 @@ clickhouse-client # or "clickhouse-client --password" if you set up a password. -You can replace `stable` with `lts` to use different [release kinds](/docs/en/faq/operations/production.md) based on your needs. +You can replace `stable` with `lts` to use different [release kinds](/knowledgebase/production) based on your needs. You can also download and install packages manually from [here](https://packages.clickhouse.com/deb/pool/main/c/). @@ -272,7 +231,7 @@ clickhouse-client # or "clickhouse-client --password" if you set up a password. -You can replace `stable` with `lts` to use different [release kinds](/docs/en/faq/operations/production.md) based on your needs. +You can replace `stable` with `lts` to use different [release kinds](/knowledgebase/production) based on your needs. Then run these commands to install packages: diff --git a/docs/en/getting-started/playground.md b/docs/en/getting-started/playground.md index e995ea6ef8b..dbb8d46a2fc 100644 --- a/docs/en/getting-started/playground.md +++ b/docs/en/getting-started/playground.md @@ -1,5 +1,5 @@ --- -sidebar_label: Playground +sidebar_label: ClickHouse Playground sidebar_position: 2 keywords: [clickhouse, playground, getting, started, docs] description: The ClickHouse Playground allows people to experiment with ClickHouse by running queries instantly, without setting up their server or cluster. @@ -11,7 +11,7 @@ slug: /en/getting-started/playground [ClickHouse Playground](https://play.clickhouse.com/play?user=play) allows people to experiment with ClickHouse by running queries instantly, without setting up their server or cluster. Several example datasets are available in Playground. -You can make queries to Playground using any HTTP client, for example [curl](https://curl.haxx.se) or [wget](https://www.gnu.org/software/wget/), or set up a connection using [JDBC](../interfaces/jdbc.md) or [ODBC](../interfaces/odbc.md) drivers. More information about software products that support ClickHouse is available [here](../interfaces). +You can make queries to Playground using any HTTP client, for example [curl](https://curl.haxx.se) or [wget](https://www.gnu.org/software/wget/), or set up a connection using [JDBC](../interfaces/jdbc.md) or [ODBC](../interfaces/odbc.md) drivers. More information about software products that support ClickHouse is available [here](../integrations/index.mdx). ## Credentials {#credentials} diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index db2e773a685..788b82dfa30 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1,7 +1,7 @@ --- slug: /en/interfaces/formats sidebar_position: 21 -sidebar_label: Input and Output Formats +sidebar_label: View all formats... title: Formats for Input and Output Data --- @@ -684,7 +684,7 @@ Example: ## JSONColumns {#jsoncolumns} :::tip -The output of the JSONColumns* formats provides the ClickHouse field name and then the content of each row of the table for that field; +The output of the JSONColumns* formats provides the ClickHouse field name and then the content of each row of the table for that field; visually, the data is rotated 90 degrees to the left. ::: diff --git a/docs/en/interfaces/postgresql.md b/docs/en/interfaces/postgresql.md index 9ff83559787..f7a619ca620 100644 --- a/docs/en/interfaces/postgresql.md +++ b/docs/en/interfaces/postgresql.md @@ -8,7 +8,7 @@ sidebar_label: PostgreSQL Interface ClickHouse supports the PostgreSQL wire protocol, which allows you to use Postgres clients to connect to ClickHouse. In a sense, ClickHouse can pretend to be a PostgreSQL instance - allowing you to connect a PostgreSQL client application to ClickHouse that is not already directly supported by ClickHouse (for example, Amazon Redshift). -To enable the PostgreSQL wire protocol, add the [postgresql_port](../operations/server-configuration-parameters/settings#server_configuration_parameters-postgresql_port) setting to your server's configuration file. For example, you could define the port in a new XML file in your `config.d` folder: +To enable the PostgreSQL wire protocol, add the [postgresql_port](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-postgresql_port) setting to your server's configuration file. For example, you could define the port in a new XML file in your `config.d` folder: ```xml diff --git a/docs/en/operations/_category_.yml b/docs/en/operations/_category_.yml index 08849e7489d..352809f663b 100644 --- a/docs/en/operations/_category_.yml +++ b/docs/en/operations/_category_.yml @@ -2,7 +2,3 @@ position: 70 label: 'Operations' collapsible: true collapsed: true -link: - type: generated-index - title: Operations - slug: /en/operations diff --git a/docs/en/operations/access-rights.md b/docs/en/operations/access-rights.md deleted file mode 100644 index 4c4a06dbe1e..00000000000 --- a/docs/en/operations/access-rights.md +++ /dev/null @@ -1,152 +0,0 @@ ---- -slug: /en/operations/access-rights -sidebar_position: 48 -sidebar_label: Access Control and Account Management -title: Access Control and Account Management ---- - -ClickHouse supports access control management based on [RBAC](https://en.wikipedia.org/wiki/Role-based_access_control) approach. - -ClickHouse access entities: -- [User account](#user-account-management) -- [Role](#role-management) -- [Row Policy](#row-policy-management) -- [Settings Profile](#settings-profiles-management) -- [Quota](#quotas-management) - -You can configure access entities using: - -- SQL-driven workflow. - - You need to [enable](#enabling-access-control) this functionality. - -- Server [configuration files](../operations/configuration-files.md) `users.xml` and `config.xml`. - -We recommend using SQL-driven workflow. Both of the configuration methods work simultaneously, so if you use the server configuration files for managing accounts and access rights, you can smoothly switch to SQL-driven workflow. - -:::warning -You can’t manage the same access entity by both configuration methods simultaneously. -::: - -To see all users, roles, profiles, etc. and all their grants use [SHOW ACCESS](../sql-reference/statements/show.md#show-access-statement) statement. - -## Usage {#access-control-usage} - -By default, the ClickHouse server provides the `default` user account which is not allowed using SQL-driven access control and account management but has all the rights and permissions. The `default` user account is used in any cases when the username is not defined, for example, at login from client or in distributed queries. In distributed query processing a default user account is used, if the configuration of the server or cluster does not specify the [user and password](../engines/table-engines/special/distributed.md) properties. - -If you just started using ClickHouse, consider the following scenario: - -1. [Enable](#enabling-access-control) SQL-driven access control and account management for the `default` user. -2. Log in to the `default` user account and create all the required users. Don’t forget to create an administrator account (`GRANT ALL ON *.* TO admin_user_account WITH GRANT OPTION`). -3. [Restrict permissions](../operations/settings/permissions-for-queries.md#permissions_for_queries) for the `default` user and disable SQL-driven access control and account management for it. - -### Properties of Current Solution {#access-control-properties} - -- You can grant permissions for databases and tables even if they do not exist. -- If a table was deleted, all the privileges that correspond to this table are not revoked. This means that even if you create a new table with the same name later, all the privileges remain valid. To revoke privileges corresponding to the deleted table, you need to execute, for example, the `REVOKE ALL PRIVILEGES ON db.table FROM ALL` query. -- There are no lifetime settings for privileges. - -## User Account {#user-account-management} - -A user account is an access entity that allows to authorize someone in ClickHouse. A user account contains: - -- Identification information. -- [Privileges](../sql-reference/statements/grant.md#grant-privileges) that define a scope of queries the user can execute. -- Hosts allowed to connect to the ClickHouse server. -- Assigned and default roles. -- Settings with their constraints applied by default at user login. -- Assigned settings profiles. - -Privileges can be granted to a user account by the [GRANT](../sql-reference/statements/grant.md) query or by assigning [roles](#role-management). To revoke privileges from a user, ClickHouse provides the [REVOKE](../sql-reference/statements/revoke.md) query. To list privileges for a user, use the [SHOW GRANTS](../sql-reference/statements/show.md#show-grants-statement) statement. - -Management queries: - -- [CREATE USER](../sql-reference/statements/create/user.md) -- [ALTER USER](../sql-reference/statements/alter/user.md#alter-user-statement) -- [DROP USER](../sql-reference/statements/drop.md) -- [SHOW CREATE USER](../sql-reference/statements/show.md#show-create-user-statement) -- [SHOW USERS](../sql-reference/statements/show.md#show-users-statement) - -### Settings Applying {#access-control-settings-applying} - -Settings can be configured differently: for a user account, in its granted roles and in settings profiles. At user login, if a setting is configured for different access entities, the value and constraints of this setting are applied as follows (from higher to lower priority): - -1. User account settings. -2. The settings of default roles of the user account. If a setting is configured in some roles, then order of the setting application is undefined. -3. The settings from settings profiles assigned to a user or to its default roles. If a setting is configured in some profiles, then order of setting application is undefined. -4. Settings applied to all the server by default or from the [default profile](../operations/server-configuration-parameters/settings.md#default-profile). - -## Role {#role-management} - -Role is a container for access entities that can be granted to a user account. - -Role contains: - -- [Privileges](../sql-reference/statements/grant.md#grant-privileges) -- Settings and constraints -- List of assigned roles - -Management queries: - -- [CREATE ROLE](../sql-reference/statements/create/role.md) -- [ALTER ROLE](../sql-reference/statements/alter/role.md#alter-role-statement) -- [DROP ROLE](../sql-reference/statements/drop.md) -- [SET ROLE](../sql-reference/statements/set-role.md) -- [SET DEFAULT ROLE](../sql-reference/statements/set-role.md#set-default-role-statement) -- [SHOW CREATE ROLE](../sql-reference/statements/show.md#show-create-role-statement) -- [SHOW ROLES](../sql-reference/statements/show.md#show-roles-statement) - -Privileges can be granted to a role by the [GRANT](../sql-reference/statements/grant.md) query. To revoke privileges from a role ClickHouse provides the [REVOKE](../sql-reference/statements/revoke.md) query. - -## Row Policy {#row-policy-management} - -Row policy is a filter that defines which of the rows are available to a user or a role. Row policy contains filters for one particular table, as well as a list of roles and/or users which should use this row policy. - -:::warning -Row policies makes sense only for users with readonly access. If user can modify table or copy partitions between tables, it defeats the restrictions of row policies. -::: - -Management queries: - -- [CREATE ROW POLICY](../sql-reference/statements/create/row-policy.md) -- [ALTER ROW POLICY](../sql-reference/statements/alter/row-policy.md#alter-row-policy-statement) -- [DROP ROW POLICY](../sql-reference/statements/drop.md#drop-row-policy-statement) -- [SHOW CREATE ROW POLICY](../sql-reference/statements/show.md#show-create-row-policy-statement) -- [SHOW POLICIES](../sql-reference/statements/show.md#show-policies-statement) - -## Settings Profile {#settings-profiles-management} - -Settings profile is a collection of [settings](../operations/settings/index.md). Settings profile contains settings and constraints, as well as a list of roles and/or users to which this profile is applied. - -Management queries: - -- [CREATE SETTINGS PROFILE](../sql-reference/statements/create/settings-profile.md#create-settings-profile-statement) -- [ALTER SETTINGS PROFILE](../sql-reference/statements/alter/settings-profile.md#alter-settings-profile-statement) -- [DROP SETTINGS PROFILE](../sql-reference/statements/drop.md#drop-settings-profile-statement) -- [SHOW CREATE SETTINGS PROFILE](../sql-reference/statements/show.md#show-create-settings-profile-statement) -- [SHOW PROFILES](../sql-reference/statements/show.md#show-profiles-statement) - -## Quota {#quotas-management} - -Quota limits resource usage. See [Quotas](../operations/quotas.md). - -Quota contains a set of limits for some durations, as well as a list of roles and/or users which should use this quota. - -Management queries: - -- [CREATE QUOTA](../sql-reference/statements/create/quota.md) -- [ALTER QUOTA](../sql-reference/statements/alter/quota.md#alter-quota-statement) -- [DROP QUOTA](../sql-reference/statements/drop.md#drop-quota-statement) -- [SHOW CREATE QUOTA](../sql-reference/statements/show.md#show-create-quota-statement) -- [SHOW QUOTA](../sql-reference/statements/show.md#show-quota-statement) -- [SHOW QUOTAS](../sql-reference/statements/show.md#show-quotas-statement) - -## Enabling SQL-driven Access Control and Account Management {#enabling-access-control} - -- Setup a directory for configurations storage. - - ClickHouse stores access entity configurations in the folder set in the [access_control_path](../operations/server-configuration-parameters/settings.md#access_control_path) server configuration parameter. - -- Enable SQL-driven access control and account management for at least one user account. - - By default, SQL-driven access control and account management is disabled for all users. You need to configure at least one user in the `users.xml` configuration file and set the value of the [access_management](../operations/settings/settings-users.md#access_management-user-setting) setting to 1. diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index f1a5649cd4c..69eb782868a 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -1,5 +1,6 @@ --- slug: /en/operations/backup +description: In order to effectively mitigate possible human errors, you should carefully prepare a strategy for backing up and restoring your data. --- # Backup and Restore @@ -213,7 +214,7 @@ To write backups to an S3 bucket you need three pieces of information: for example `Abc+123` :::note -Creating an S3 bucket is covered in [Use S3 Object Storage as a ClickHouse disk](/docs/en/integrations/data-ingestion/s3/configuring-s3-for-clickhouse-use.md), just come back to this doc after saving the policy, there is no need to configure ClickHouse to use the S3 bucket. +Creating an S3 bucket is covered in [Use S3 Object Storage as a ClickHouse disk](/docs/en/integrations/data-ingestion/s3/index.md#configuring-s3-for-clickhouse-use), just come back to this doc after saving the policy, there is no need to configure ClickHouse to use the S3 bucket. ::: The destination for a backup will be specified like this: diff --git a/docs/en/operations/caches.md b/docs/en/operations/caches.md index 0f9156048c4..86bf8065d94 100644 --- a/docs/en/operations/caches.md +++ b/docs/en/operations/caches.md @@ -3,6 +3,7 @@ slug: /en/operations/caches sidebar_position: 65 sidebar_label: Caches title: "Cache Types" +description: When performing queries, ClickHouse uses different caches. --- When performing queries, ClickHouse uses different caches. diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md deleted file mode 100644 index 10bad586a54..00000000000 --- a/docs/en/operations/clickhouse-keeper.md +++ /dev/null @@ -1,378 +0,0 @@ ---- -slug: /en/operations/clickhouse-keeper -sidebar_position: 66 -sidebar_label: ClickHouse Keeper ---- - -# ClickHouse Keeper -import SelfManaged from '@site/docs/en/_snippets/_self_managed_only_automated.md'; - - - -ClickHouse Keeper provides the coordination system for data [replication](../engines/table-engines/mergetree-family/replication.md) and [distributed DDL](../sql-reference/distributed-ddl.md) queries execution. ClickHouse Keeper is compatible with ZooKeeper. - -## Implementation details {#implementation-details} - -ZooKeeper is one of the first well-known open-source coordination systems. It's implemented in Java, and has quite a simple and powerful data model. ZooKeeper's coordination algorithm, ZooKeeper Atomic Broadcast (ZAB), doesn't provide linearizability guarantees for reads, because each ZooKeeper node serves reads locally. Unlike ZooKeeper ClickHouse Keeper is written in C++ and uses the [RAFT algorithm](https://raft.github.io/) [implementation](https://github.com/eBay/NuRaft). This algorithm allows linearizability for reads and writes, and has several open-source implementations in different languages. - -By default, ClickHouse Keeper provides the same guarantees as ZooKeeper (linearizable writes, non-linearizable reads). It has a compatible client-server protocol, so any standard ZooKeeper client can be used to interact with ClickHouse Keeper. Snapshots and logs have an incompatible format with ZooKeeper, but the `clickhouse-keeper-converter` tool enables the conversion of ZooKeeper data to ClickHouse Keeper snapshots. The interserver protocol in ClickHouse Keeper is also incompatible with ZooKeeper so a mixed ZooKeeper / ClickHouse Keeper cluster is impossible. - -ClickHouse Keeper supports Access Control Lists (ACLs) the same way as [ZooKeeper](https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html#sc_ZooKeeperAccessControl) does. ClickHouse Keeper supports the same set of permissions and has the identical built-in schemes: `world`, `auth` and `digest`. The digest authentication scheme uses the pair `username:password`, the password is encoded in Base64. - -:::note -External integrations are not supported. -::: - -## Configuration {#configuration} - -ClickHouse Keeper can be used as a standalone replacement for ZooKeeper or as an internal part of the ClickHouse server. In both cases the configuration is almost the same `.xml` file. The main ClickHouse Keeper configuration tag is ``. Keeper configuration has the following parameters: - -- `tcp_port` — Port for a client to connect (default for ZooKeeper is `2181`). -- `tcp_port_secure` — Secure port for an SSL connection between client and keeper-server. -- `server_id` — Unique server id, each participant of the ClickHouse Keeper cluster must have a unique number (1, 2, 3, and so on). -- `log_storage_path` — Path to coordination logs, just like ZooKeeper it is best to store logs on non-busy nodes. -- `snapshot_storage_path` — Path to coordination snapshots. - -Other common parameters are inherited from the ClickHouse server config (`listen_host`, `logger`, and so on). - -Internal coordination settings are located in the `.` section: - -- `operation_timeout_ms` — Timeout for a single client operation (ms) (default: 10000). -- `min_session_timeout_ms` — Min timeout for client session (ms) (default: 10000). -- `session_timeout_ms` — Max timeout for client session (ms) (default: 100000). -- `dead_session_check_period_ms` — How often ClickHouse Keeper checks for dead sessions and removes them (ms) (default: 500). -- `heart_beat_interval_ms` — How often a ClickHouse Keeper leader will send heartbeats to followers (ms) (default: 500). -- `election_timeout_lower_bound_ms` — If the follower does not receive a heartbeat from the leader in this interval, then it can initiate leader election (default: 1000). Must be less than or equal to `election_timeout_upper_bound_ms`. Ideally they shouldn't be equal. -- `election_timeout_upper_bound_ms` — If the follower does not receive a heartbeat from the leader in this interval, then it must initiate leader election (default: 2000). -- `rotate_log_storage_interval` — How many log records to store in a single file (default: 100000). -- `reserved_log_items` — How many coordination log records to store before compaction (default: 100000). -- `snapshot_distance` — How often ClickHouse Keeper will create new snapshots (in the number of records in logs) (default: 100000). -- `snapshots_to_keep` — How many snapshots to keep (default: 3). -- `stale_log_gap` — Threshold when leader considers follower as stale and sends the snapshot to it instead of logs (default: 10000). -- `fresh_log_gap` — When node became fresh (default: 200). -- `max_requests_batch_size` - Max size of batch in requests count before it will be sent to RAFT (default: 100). -- `force_sync` — Call `fsync` on each write to coordination log (default: true). -- `quorum_reads` — Execute read requests as writes through whole RAFT consensus with similar speed (default: false). -- `raft_logs_level` — Text logging level about coordination (trace, debug, and so on) (default: system default). -- `auto_forwarding` — Allow to forward write requests from followers to the leader (default: true). -- `shutdown_timeout` — Wait to finish internal connections and shutdown (ms) (default: 5000). -- `startup_timeout` — If the server doesn't connect to other quorum participants in the specified timeout it will terminate (ms) (default: 30000). -- `four_letter_word_white_list` — White list of 4lw commands (default: `conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv,csnp,lgif,rqld`). - -Quorum configuration is located in the `.` section and contain servers description. - -The only parameter for the whole quorum is `secure`, which enables encrypted connection for communication between quorum participants. The parameter can be set `true` if SSL connection is required for internal communication between nodes, or left unspecified otherwise. - -The main parameters for each `` are: - -- `id` — Server identifier in a quorum. -- `hostname` — Hostname where this server is placed. -- `port` — Port where this server listens for connections. - -:::note -In the case of a change in the topology of your ClickHouse Keeper cluster (e.g., replacing a server), please make sure to keep the mapping of `server_id` to `hostname` consistent and avoid shuffling or reusing an existing `server_id` for different servers (e.g., it can happen if your rely on automation scripts to deploy ClickHouse Keeper) -::: - -Examples of configuration for quorum with three nodes can be found in [integration tests](https://github.com/ClickHouse/ClickHouse/tree/master/tests/integration) with `test_keeper_` prefix. Example configuration for server #1: - -```xml - - 2181 - 1 - /var/lib/clickhouse/coordination/log - /var/lib/clickhouse/coordination/snapshots - - - 10000 - 30000 - trace - - - - - 1 - zoo1 - 9444 - - - 2 - zoo2 - 9444 - - - 3 - zoo3 - 9444 - - - -``` - -## How to run {#how-to-run} - -ClickHouse Keeper is bundled into the ClickHouse server package, just add configuration of `` and start ClickHouse server as always. If you want to run standalone ClickHouse Keeper you can start it in a similar way with: - -```bash -clickhouse-keeper --config /etc/your_path_to_config/config.xml -``` - -If you don't have the symlink (`clickhouse-keeper`) you can create it or specify `keeper` as an argument to `clickhouse`: - -```bash -clickhouse keeper --config /etc/your_path_to_config/config.xml -``` - -## Four Letter Word Commands {#four-letter-word-commands} - -ClickHouse Keeper also provides 4lw commands which are almost the same with Zookeeper. Each command is composed of four letters such as `mntr`, `stat` etc. There are some more interesting commands: `stat` gives some general information about the server and connected clients, while `srvr` and `cons` give extended details on server and connections respectively. - -The 4lw commands has a white list configuration `four_letter_word_white_list` which has default value `conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv,csnp,lgif,rqld`. - -You can issue the commands to ClickHouse Keeper via telnet or nc, at the client port. - -``` -echo mntr | nc localhost 9181 -``` - -Bellow is the detailed 4lw commands: - -- `ruok`: Tests if server is running in a non-error state. The server will respond with `imok` if it is running. Otherwise it will not respond at all. A response of `imok` does not necessarily indicate that the server has joined the quorum, just that the server process is active and bound to the specified client port. Use "stat" for details on state wrt quorum and client connection information. - -``` -imok -``` - -- `mntr`: Outputs a list of variables that could be used for monitoring the health of the cluster. - -``` -zk_version v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 -zk_avg_latency 0 -zk_max_latency 0 -zk_min_latency 0 -zk_packets_received 68 -zk_packets_sent 68 -zk_num_alive_connections 1 -zk_outstanding_requests 0 -zk_server_state leader -zk_znode_count 4 -zk_watch_count 1 -zk_ephemerals_count 0 -zk_approximate_data_size 723 -zk_open_file_descriptor_count 310 -zk_max_file_descriptor_count 10240 -zk_followers 0 -zk_synced_followers 0 -``` - -- `srvr`: Lists full details for the server. - -``` -ClickHouse Keeper version: v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 -Latency min/avg/max: 0/0/0 -Received: 2 -Sent : 2 -Connections: 1 -Outstanding: 0 -Zxid: 34 -Mode: leader -Node count: 4 -``` - -- `stat`: Lists brief details for the server and connected clients. - -``` -ClickHouse Keeper version: v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 -Clients: - 192.168.1.1:52852(recved=0,sent=0) - 192.168.1.1:52042(recved=24,sent=48) -Latency min/avg/max: 0/0/0 -Received: 4 -Sent : 4 -Connections: 1 -Outstanding: 0 -Zxid: 36 -Mode: leader -Node count: 4 -``` - -- `srst`: Reset server statistics. The command will affect the result of `srvr`, `mntr` and `stat`. - -``` -Server stats reset. -``` - -- `conf`: Print details about serving configuration. - -``` -server_id=1 -tcp_port=2181 -four_letter_word_white_list=* -log_storage_path=./coordination/logs -snapshot_storage_path=./coordination/snapshots -max_requests_batch_size=100 -session_timeout_ms=30000 -operation_timeout_ms=10000 -dead_session_check_period_ms=500 -heart_beat_interval_ms=500 -election_timeout_lower_bound_ms=1000 -election_timeout_upper_bound_ms=2000 -reserved_log_items=1000000000000000 -snapshot_distance=10000 -auto_forwarding=true -shutdown_timeout=5000 -startup_timeout=240000 -raft_logs_level=information -snapshots_to_keep=3 -rotate_log_storage_interval=100000 -stale_log_gap=10000 -fresh_log_gap=200 -max_requests_batch_size=100 -quorum_reads=false -force_sync=false -compress_logs=true -compress_snapshots_with_zstd_format=true -configuration_change_tries_count=20 -``` - -- `cons`: List full connection/session details for all clients connected to this server. Includes information on numbers of packets received/sent, session id, operation latencies, last operation performed, etc... - -``` - 192.168.1.1:52163(recved=0,sent=0,sid=0xffffffffffffffff,lop=NA,est=1636454787393,to=30000,lzxid=0xffffffffffffffff,lresp=0,llat=0,minlat=0,avglat=0,maxlat=0) - 192.168.1.1:52042(recved=9,sent=18,sid=0x0000000000000001,lop=List,est=1636454739887,to=30000,lcxid=0x0000000000000005,lzxid=0x0000000000000005,lresp=1636454739892,llat=0,minlat=0,avglat=0,maxlat=0) -``` - -- `crst`: Reset connection/session statistics for all connections. - -``` -Connection stats reset. -``` - -- `envi`: Print details about serving environment - -``` -Environment: -clickhouse.keeper.version=v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 -host.name=ZBMAC-C02D4054M.local -os.name=Darwin -os.arch=x86_64 -os.version=19.6.0 -cpu.count=12 -user.name=root -user.home=/Users/JackyWoo/ -user.dir=/Users/JackyWoo/project/jd/clickhouse/cmake-build-debug/programs/ -user.tmp=/var/folders/b4/smbq5mfj7578f2jzwn602tt40000gn/T/ -``` - - -- `dirs`: Shows the total size of snapshot and log files in bytes - -``` -snapshot_dir_size: 0 -log_dir_size: 3875 -``` - -- `isro`: Tests if server is running in read-only mode. The server will respond with "ro" if in read-only mode or "rw" if not in read-only mode. - -``` -rw -``` - -- `wchs`: Lists brief information on watches for the server. - -``` -1 connections watching 1 paths -Total watches:1 -``` - -- `wchc`: Lists detailed information on watches for the server, by session. This outputs a list of sessions (connections) with associated watches (paths). Note, depending on the number of watches this operation may be expensive (ie impact server performance), use it carefully. - -``` -0x0000000000000001 - /clickhouse/task_queue/ddl -``` - -- `wchp`: Lists detailed information on watches for the server, by path. This outputs a list of paths (znodes) with associated sessions. Note, depending on the number of watches this operation may be expensive (i. e. impact server performance), use it carefully. - -``` -/clickhouse/task_queue/ddl - 0x0000000000000001 -``` - -- `dump`: Lists the outstanding sessions and ephemeral nodes. This only works on the leader. - -``` -Sessions dump (2): -0x0000000000000001 -0x0000000000000002 -Sessions with Ephemerals (1): -0x0000000000000001 - /clickhouse/task_queue/ddl -``` - -- `csnp`: Schedule a snapshot creation task. Return the last committed log index of the scheduled snapshot if success or `Failed to schedule snapshot creation task.` if failed. Note that `lgif` command can help you determine whether the snapshot is done. - -``` -100 -``` - -- `lgif`: Keeper log information. `first_log_idx` : my first log index in log store; `first_log_term` : my first log term; `last_log_idx` : my last log index in log store; `last_log_term` : my last log term; `last_committed_log_idx` : my last committed log index in state machine; `leader_committed_log_idx` : leader's committed log index from my perspective; `target_committed_log_idx` : target log index should be committed to; `last_snapshot_idx` : the largest committed log index in last snapshot. - -``` -first_log_idx 1 -first_log_term 1 -last_log_idx 101 -last_log_term 1 -last_committed_log_idx 100 -leader_committed_log_idx 101 -target_committed_log_idx 101 -last_snapshot_idx 50 -``` - -- `rqld`: Request to become new leader. Return `Sent leadership request to leader.` if request sent or `Failed to send leadership request to leader.` if request not sent. Note that if node is already leader the outcome is same as the request is sent. - -``` -Sent leadership request to leader. -``` - -## Migration from ZooKeeper {#migration-from-zookeeper} - -Seamlessly migration from ZooKeeper to ClickHouse Keeper is impossible you have to stop your ZooKeeper cluster, convert data and start ClickHouse Keeper. `clickhouse-keeper-converter` tool allows converting ZooKeeper logs and snapshots to ClickHouse Keeper snapshot. It works only with ZooKeeper > 3.4. Steps for migration: - -1. Stop all ZooKeeper nodes. - -2. Optional, but recommended: find ZooKeeper leader node, start and stop it again. It will force ZooKeeper to create a consistent snapshot. - -3. Run `clickhouse-keeper-converter` on a leader, for example: - -```bash -clickhouse-keeper-converter --zookeeper-logs-dir /var/lib/zookeeper/version-2 --zookeeper-snapshots-dir /var/lib/zookeeper/version-2 --output-dir /path/to/clickhouse/keeper/snapshots -``` - -4. Copy snapshot to ClickHouse server nodes with a configured `keeper` or start ClickHouse Keeper instead of ZooKeeper. The snapshot must persist on all nodes, otherwise, empty nodes can be faster and one of them can become a leader. - - - -## Recovering after losing quorum - -Because ClickHouse Keeper uses Raft it can tolerate certain amount of node crashes depending on the cluster size. \ -E.g. for a 3-node cluster, it will continue working correctly if only 1 node crashes. - -Cluster configuration can be dynamically configured but there are some limitations. Reconfiguration relies on Raft also -so to add/remove a node from the cluster you need to have a quorum. If you lose too many nodes in your cluster at the same time without any chance -of starting them again, Raft will stop working and not allow you to reconfigure your cluster using the conventional way. - -Nevertheless, ClickHouse Keeper has a recovery mode which allows you to forcefully reconfigure your cluster with only 1 node. -This should be done only as your last resort if you cannot start your nodes again, or start a new instance on the same endpoint. - -Important things to note before continuing: -- Make sure that the failed nodes cannot connect to the cluster again. -- Do not start any of the new nodes until it's specified in the steps. - -After making sure that the above things are true, you need to do following: -1. Pick a single Keeper node to be your new leader. Be aware that the data of that node will be used for the entire cluster so we recommend to use a node with the most up to date state. -2. Before doing anything else, make a backup of the `log_storage_path` and `snapshot_storage_path` folders of the picked node. -3. Reconfigure the cluster on all of the nodes you want to use. -4. Send the four letter command `rcvr` to the node you picked which will move the node to the recovery mode OR stop Keeper instance on the picked node and start it again with the `--force-recovery` argument. -5. One by one, start Keeper instances on the new nodes making sure that `mntr` returns `follower` for the `zk_server_state` before starting the next one. -6. While in the recovery mode, the leader node will return error message for `mntr` command until it achieves quorum with the new nodes and refuse any requests from the client and the followers. -7. After quorum is achieved, the leader node will return to the normal mode of operation, accepting all the requests using Raft - verify with `mntr` which should return `leader` for the `zk_server_state`. diff --git a/docs/en/operations/external-authenticators/kerberos.md b/docs/en/operations/external-authenticators/kerberos.md index 95944e96194..b7a11d7445b 100644 --- a/docs/en/operations/external-authenticators/kerberos.md +++ b/docs/en/operations/external-authenticators/kerberos.md @@ -113,7 +113,7 @@ Note, that now, once user `my_user` uses `kerberos`, Kerberos must be enabled in ### Enabling Kerberos using SQL {#enabling-kerberos-using-sql} -When [SQL-driven Access Control and Account Management](../access-rights.md#access-control) is enabled in ClickHouse, users identified by Kerberos can also be created using SQL statements. +When [SQL-driven Access Control and Account Management](/docs/en/guides/sre/user-management/index.md#access-control) is enabled in ClickHouse, users identified by Kerberos can also be created using SQL statements. ```sql CREATE USER my_user IDENTIFIED WITH kerberos REALM 'EXAMPLE.COM' diff --git a/docs/en/operations/external-authenticators/ldap.md b/docs/en/operations/external-authenticators/ldap.md index eba560f6ea5..fa44e6e2978 100644 --- a/docs/en/operations/external-authenticators/ldap.md +++ b/docs/en/operations/external-authenticators/ldap.md @@ -112,7 +112,7 @@ At each login attempt, ClickHouse tries to "bind" to the specified DN defined by Note, that user `my_user` refers to `my_ldap_server`. This LDAP server must be configured in the main `config.xml` file as described previously. -When SQL-driven [Access Control and Account Management](../access-rights.md#access-control) is enabled, users that are authenticated by LDAP servers can also be created using the [CREATE USER](../../sql-reference/statements/create/user.md#create-user-statement) statement. +When SQL-driven [Access Control and Account Management](/docs/en/guides/sre/user-management/index.md#access-control) is enabled, users that are authenticated by LDAP servers can also be created using the [CREATE USER](/docs/en/sql-reference/statements/create/user.md#create-user-statement) statement. Query: @@ -124,7 +124,7 @@ CREATE USER my_user IDENTIFIED WITH ldap SERVER 'my_ldap_server'; In addition to the locally defined users, a remote LDAP server can be used as a source of user definitions. To achieve this, specify previously defined LDAP server name (see [LDAP Server Definition](#ldap-server-definition)) in the `ldap` section inside the `users_directories` section of the `config.xml` file. -At each login attempt, ClickHouse tries to find the user definition locally and authenticate it as usual. If the user is not defined, ClickHouse will assume the definition exists in the external LDAP directory and will try to "bind" to the specified DN at the LDAP server using the provided credentials. If successful, the user will be considered existing and authenticated. The user will be assigned roles from the list specified in the `roles` section. Additionally, LDAP "search" can be performed and results can be transformed and treated as role names and then be assigned to the user if the `role_mapping` section is also configured. All this implies that the SQL-driven [Access Control and Account Management](../access-rights.md#access-control) is enabled and roles are created using the [CREATE ROLE](../../sql-reference/statements/create/role.md#create-role-statement) statement. +At each login attempt, ClickHouse tries to find the user definition locally and authenticate it as usual. If the user is not defined, ClickHouse will assume the definition exists in the external LDAP directory and will try to "bind" to the specified DN at the LDAP server using the provided credentials. If successful, the user will be considered existing and authenticated. The user will be assigned roles from the list specified in the `roles` section. Additionally, LDAP "search" can be performed and results can be transformed and treated as role names and then be assigned to the user if the `role_mapping` section is also configured. All this implies that the SQL-driven [Access Control and Account Management](/docs/en/guides/sre/user-management/index.md#access-control) is enabled and roles are created using the [CREATE ROLE](/docs/en/sql-reference/statements/create/role.md#create-role-statement) statement. **Example** @@ -173,7 +173,7 @@ Note that `my_ldap_server` referred in the `ldap` section inside the `user_direc - `roles` — Section with a list of locally defined roles that will be assigned to each user retrieved from the LDAP server. - If no roles are specified here or assigned during role mapping (below), user will not be able to perform any actions after authentication. - `role_mapping` — Section with LDAP search parameters and mapping rules. - - When a user authenticates, while still bound to LDAP, an LDAP search is performed using `search_filter` and the name of the logged-in user. For each entry found during that search, the value of the specified attribute is extracted. For each attribute value that has the specified prefix, the prefix is removed, and the rest of the value becomes the name of a local role defined in ClickHouse, which is expected to be created beforehand by the [CREATE ROLE](../../sql-reference/statements/create/role.md#create-role-statement) statement. + - When a user authenticates, while still bound to LDAP, an LDAP search is performed using `search_filter` and the name of the logged-in user. For each entry found during that search, the value of the specified attribute is extracted. For each attribute value that has the specified prefix, the prefix is removed, and the rest of the value becomes the name of a local role defined in ClickHouse, which is expected to be created beforehand by the [CREATE ROLE](/docs/en/sql-reference/statements/create/role.md#create-role-statement) statement. - There can be multiple `role_mapping` sections defined inside the same `ldap` section. All of them will be applied. - `base_dn` — Template used to construct the base DN for the LDAP search. - The resulting DN will be constructed by replacing all `{user_name}`, `{bind_dn}`, and `{user_dn}` substrings of the template with the actual user name, bind DN, and user DN during each LDAP search. diff --git a/docs/en/operations/monitoring.md b/docs/en/operations/monitoring.md index 2b3c4bdbbdf..04c5840d514 100644 --- a/docs/en/operations/monitoring.md +++ b/docs/en/operations/monitoring.md @@ -2,6 +2,7 @@ slug: /en/operations/monitoring sidebar_position: 45 sidebar_label: Monitoring +description: You can monitor the utilization of hardware resources and also ClickHouse server metrics. --- # Monitoring diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 0424c3520e0..08be318f334 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -2,6 +2,7 @@ slug: /en/operations/server-configuration-parameters/settings sidebar_position: 57 sidebar_label: Server Settings +description: This section contains descriptions of server settings that cannot be changed at the session or query level. --- # Server Settings @@ -275,7 +276,7 @@ Path: - Specify the absolute path or the path relative to the server config file. - The path can contain wildcards \* and ?. -See also “[Dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md)”. +See also “[Dictionaries](../../sql-reference/dictionaries/index.md)”. **Example** @@ -1025,7 +1026,7 @@ If the number of **idle** threads in the Backups IO Thread pool exceeds `max_bac Possible values: - Positive integer. -- Zero. +- Zero. Default value: `0`. @@ -1917,7 +1918,7 @@ Default value: `/var/lib/clickhouse/access/`. **See also** -- [Access Control and Account Management](../../operations/access-rights.md#access-control) +- [Access Control and Account Management](../../guides/sre/user-management/index.md#access-control) ## user_directories {#user_directories} diff --git a/docs/en/operations/settings/settings-profiles.md b/docs/en/operations/settings/settings-profiles.md index 4527152583f..2f39a75453c 100644 --- a/docs/en/operations/settings/settings-profiles.md +++ b/docs/en/operations/settings/settings-profiles.md @@ -9,7 +9,7 @@ sidebar_label: Settings Profiles A settings profile is a collection of settings grouped under the same name. :::note -ClickHouse also supports [SQL-driven workflow](../../operations/access-rights.md#access-control) for managing settings profiles. We recommend using it. +ClickHouse also supports [SQL-driven workflow](../../guides/sre/user-management/index.md#access-control) for managing settings profiles. We recommend using it. ::: The profile can have any name. You can specify the same profile for different users. The most important thing you can write in the settings profile is `readonly=1`, which ensures read-only access. diff --git a/docs/en/operations/settings/settings-users.md b/docs/en/operations/settings/settings-users.md index b55d64fc4f7..9b27af61851 100644 --- a/docs/en/operations/settings/settings-users.md +++ b/docs/en/operations/settings/settings-users.md @@ -9,7 +9,7 @@ sidebar_label: User Settings The `users` section of the `user.xml` configuration file contains user settings. :::note -ClickHouse also supports [SQL-driven workflow](../../operations/access-rights.md#access-control) for managing users. We recommend using it. +ClickHouse also supports [SQL-driven workflow](../../guides/sre/user-management/index.md#access-control) for managing users. We recommend using it. ::: Structure of the `users` section: @@ -77,7 +77,7 @@ Password can be specified in plaintext or in SHA256 (hex format). ### access_management {#access_management-user-setting} -This setting enables or disables using of SQL-driven [access control and account management](../../operations/access-rights.md#access-control) for the user. +This setting enables or disables using of SQL-driven [access control and account management](../../guides/sre/user-management/index.md#access-control) for the user. Possible values: diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 3c53f4fd0cf..daaa79e90db 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -2999,7 +2999,7 @@ It can be useful when merges are CPU bounded not IO bounded (performing heavy da ## max_final_threads {#max-final-threads} -Sets the maximum number of parallel threads for the `SELECT` query data read phase with the [FINAL](../../sql-reference/statements/select/from.md/#select-from-final) modifier. +Sets the maximum number of parallel threads for the `SELECT` query data read phase with the [FINAL](../../sql-reference/statements/select/from.md#select-from-final) modifier. Possible values: @@ -3094,9 +3094,9 @@ Possible values: Default value: `0`. -## s3_truncate_on_insert +## s3_truncate_on_insert -Enables or disables truncate before inserts in s3 engine tables. If disabled, an exception will be thrown on insert attempts if an S3 object already exists. +Enables or disables truncate before inserts in s3 engine tables. If disabled, an exception will be thrown on insert attempts if an S3 object already exists. Possible values: - 0 — `INSERT` query appends new data to the end of the file. @@ -3104,9 +3104,9 @@ Possible values: Default value: `0`. -## hdfs_truncate_on_insert +## hdfs_truncate_on_insert -Enables or disables truncation before an insert in hdfs engine tables. If disabled, an exception will be thrown on an attempt to insert if a file in HDFS already exists. +Enables or disables truncation before an insert in hdfs engine tables. If disabled, an exception will be thrown on an attempt to insert if a file in HDFS already exists. Possible values: - 0 — `INSERT` query appends new data to the end of the file. @@ -3114,11 +3114,11 @@ Possible values: Default value: `0`. -## engine_file_allow_create_multiple_files +## engine_file_allow_create_multiple_files Enables or disables creating a new file on each insert in file engine tables if the format has the suffix (`JSON`, `ORC`, `Parquet`, etc.). If enabled, on each insert a new file will be created with a name following this pattern: -`data.Parquet` -> `data.1.Parquet` -> `data.2.Parquet`, etc. +`data.Parquet` -> `data.1.Parquet` -> `data.2.Parquet`, etc. Possible values: - 0 — `INSERT` query appends new data to the end of the file. @@ -3126,11 +3126,11 @@ Possible values: Default value: `0`. -## s3_create_new_file_on_insert +## s3_create_new_file_on_insert Enables or disables creating a new file on each insert in s3 engine tables. If enabled, on each insert a new S3 object will be created with the key, similar to this pattern: -initial: `data.Parquet.gz` -> `data.1.Parquet.gz` -> `data.2.Parquet.gz`, etc. +initial: `data.Parquet.gz` -> `data.1.Parquet.gz` -> `data.2.Parquet.gz`, etc. Possible values: - 0 — `INSERT` query appends new data to the end of the file. @@ -3142,7 +3142,7 @@ Default value: `0`. Enables or disables creating a new file on each insert in HDFS engine tables. If enabled, on each insert a new HDFS file will be created with the name, similar to this pattern: -initial: `data.Parquet.gz` -> `data.1.Parquet.gz` -> `data.2.Parquet.gz`, etc. +initial: `data.Parquet.gz` -> `data.1.Parquet.gz` -> `data.2.Parquet.gz`, etc. Possible values: - 0 — `INSERT` query appends new data to the end of the file. @@ -3753,7 +3753,7 @@ Default value: `1`. ## optimize_move_to_prewhere_if_final {#optimize_move_to_prewhere_if_final} -Enables or disables automatic [PREWHERE](../../sql-reference/statements/select/prewhere.md) optimization in [SELECT](../../sql-reference/statements/select/index.md) queries with [FINAL](../../sql-reference/statements/select/from.md/#select-from-final) modifier. +Enables or disables automatic [PREWHERE](../../sql-reference/statements/select/prewhere.md) optimization in [SELECT](../../sql-reference/statements/select/index.md) queries with [FINAL](../../sql-reference/statements/select/from.md#select-from-final) modifier. Works only for [*MergeTree](../../engines/table-engines/mergetree-family/index.md) tables. @@ -3770,7 +3770,7 @@ Default value: `0`. ## optimize_using_constraints -Use [constraints](../../sql-reference/statements/create/table#constraints) for query optimization. The default is `false`. +Use [constraints](../../sql-reference/statements/create/table.md#constraints) for query optimization. The default is `false`. Possible values: @@ -3778,7 +3778,7 @@ Possible values: ## optimize_append_index -Use [constraints](../../sql-reference/statements/create/table#constraints) in order to append index condition. The default is `false`. +Use [constraints](../../sql-reference/statements/create/table.md#constraints) in order to append index condition. The default is `false`. Possible values: @@ -3786,7 +3786,7 @@ Possible values: ## optimize_substitute_columns -Use [constraints](../../sql-reference/statements/create/table#constraints) for column substitution. The default is `false`. +Use [constraints](../../sql-reference/statements/create/table.md#constraints) for column substitution. The default is `false`. Possible values: @@ -3984,7 +3984,7 @@ Use this setting only for backward compatibility if your use cases depend on old ## final {#final} -Automatically applies [FINAL](../../sql-reference/statements/select/from/#final-modifier) modifier to all tables in a query, to tables where [FINAL](../../sql-reference/statements/select/from/#final-modifier) is applicable, including joined tables and tables in sub-queries, and +Automatically applies [FINAL](../../sql-reference/statements/select/from.md#final-modifier) modifier to all tables in a query, to tables where [FINAL](../../sql-reference/statements/select/from.md#final-modifier) is applicable, including joined tables and tables in sub-queries, and distributed tables. Possible values: @@ -4030,7 +4030,7 @@ SELECT * FROM test; ## asterisk_include_materialized_columns {#asterisk_include_materialized_columns} -Include [MATERIALIZED](../../sql-reference/statements/create/table/#materialized) columns for wildcard query (`SELECT *`). +Include [MATERIALIZED](../../sql-reference/statements/create/table.md#materialized) columns for wildcard query (`SELECT *`). Possible values: @@ -4041,7 +4041,7 @@ Default value: `0`. ## asterisk_include_alias_columns {#asterisk_include_alias_columns} -Include [ALIAS](../../sql-reference/statements/create/table/#alias) columns for wildcard query (`SELECT *`). +Include [ALIAS](../../sql-reference/statements/create/table.md#alias) columns for wildcard query (`SELECT *`). Possible values: diff --git a/docs/en/operations/system-tables/dictionaries.md b/docs/en/operations/system-tables/dictionaries.md index 4b256f0de97..ca6b7faaa78 100644 --- a/docs/en/operations/system-tables/dictionaries.md +++ b/docs/en/operations/system-tables/dictionaries.md @@ -3,12 +3,12 @@ slug: /en/operations/system-tables/dictionaries --- # dictionaries -Contains information about [dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md). +Contains information about [dictionaries](../../sql-reference/dictionaries/index.md). Columns: - `database` ([String](../../sql-reference/data-types/string.md)) — Name of the database containing the dictionary created by DDL query. Empty string for other dictionaries. -- `name` ([String](../../sql-reference/data-types/string.md)) — [Dictionary name](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md). +- `name` ([String](../../sql-reference/data-types/string.md)) — [Dictionary name](../../sql-reference/dictionaries/index.md). - `uuid` ([UUID](../../sql-reference/data-types/uuid.md)) — Dictionary UUID. - `status` ([Enum8](../../sql-reference/data-types/enum.md)) — Dictionary status. Possible values: - `NOT_LOADED` — Dictionary was not loaded because it was not used. @@ -18,20 +18,20 @@ Columns: - `LOADED_AND_RELOADING` — Dictionary is loaded successfully, and is being reloaded right now (frequent reasons: [SYSTEM RELOAD DICTIONARY](../../sql-reference/statements/system.md#query_language-system-reload-dictionary) query, timeout, dictionary config has changed). - `FAILED_AND_RELOADING` — Could not load the dictionary as a result of an error and is loading now. - `origin` ([String](../../sql-reference/data-types/string.md)) — Path to the configuration file that describes the dictionary. -- `type` ([String](../../sql-reference/data-types/string.md)) — Type of a dictionary allocation. [Storing Dictionaries in Memory](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md). -- `key.names` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Array of [key names](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-key) provided by the dictionary. -- `key.types` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Corresponding array of [key types](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-key) provided by the dictionary. -- `attribute.names` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Array of [attribute names](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-attributes) provided by the dictionary. -- `attribute.types` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Corresponding array of [attribute types](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-attributes) provided by the dictionary. +- `type` ([String](../../sql-reference/data-types/string.md)) — Type of a dictionary allocation. [Storing Dictionaries in Memory](../../sql-reference/dictionaries/index.md#storig-dictionaries-in-memory). +- `key.names` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Array of [key names](../../sql-reference/dictionaries/index.md#dictionary-key-and-fields#ext_dict_structure-key) provided by the dictionary. +- `key.types` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Corresponding array of [key types](../../sql-reference/dictionaries/index.md#dictionary-key-and-fields#ext_dict_structure-key) provided by the dictionary. +- `attribute.names` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Array of [attribute names](../../sql-reference/dictionaries/index.md#dictionary-key-and-fields#ext_dict_structure-attributes) provided by the dictionary. +- `attribute.types` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Corresponding array of [attribute types](../../sql-reference/dictionaries/index.md#dictionary-key-and-fields#ext_dict_structure-attributes) provided by the dictionary. - `bytes_allocated` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Amount of RAM allocated for the dictionary. - `query_count` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Number of queries since the dictionary was loaded or since the last successful reboot. - `hit_rate` ([Float64](../../sql-reference/data-types/float.md)) — For cache dictionaries, the percentage of uses for which the value was in the cache. - `found_rate` ([Float64](../../sql-reference/data-types/float.md)) — The percentage of uses for which the value was found. - `element_count` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Number of items stored in the dictionary. - `load_factor` ([Float64](../../sql-reference/data-types/float.md)) — Percentage filled in the dictionary (for a hashed dictionary, the percentage filled in the hash table). -- `source` ([String](../../sql-reference/data-types/string.md)) — Text describing the [data source](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md) for the dictionary. -- `lifetime_min` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Minimum [lifetime](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) of the dictionary in memory, after which ClickHouse tries to reload the dictionary (if `invalidate_query` is set, then only if it has changed). Set in seconds. -- `lifetime_max` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Maximum [lifetime](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) of the dictionary in memory, after which ClickHouse tries to reload the dictionary (if `invalidate_query` is set, then only if it has changed). Set in seconds. +- `source` ([String](../../sql-reference/data-types/string.md)) — Text describing the [data source](../../sql-reference/dictionaries/index.md#dictionary-sources) for the dictionary. +- `lifetime_min` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Minimum [lifetime](../../sql-reference/dictionaries/index.md#dictionary-updates) of the dictionary in memory, after which ClickHouse tries to reload the dictionary (if `invalidate_query` is set, then only if it has changed). Set in seconds. +- `lifetime_max` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Maximum [lifetime](../../sql-reference/dictionaries/index.md#dictionary-updates) of the dictionary in memory, after which ClickHouse tries to reload the dictionary (if `invalidate_query` is set, then only if it has changed). Set in seconds. - `loading_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Start time for loading the dictionary. - `last_successful_update_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — End time for loading or updating the dictionary. Helps to monitor some troubles with dictionary sources and investigate the causes. - `loading_duration` ([Float32](../../sql-reference/data-types/float.md)) — Duration of a dictionary loading. diff --git a/docs/en/operations/system-tables/marked_dropped_tables.md b/docs/en/operations/system-tables/marked_dropped_tables.md deleted file mode 100644 index 23e969f7624..00000000000 --- a/docs/en/operations/system-tables/marked_dropped_tables.md +++ /dev/null @@ -1,37 +0,0 @@ ---- -slug: /en/operations/system-tables/marked_dropped_tables ---- -# marked_dropped_tables - -Contains information about tables that drop table has been executed but data cleanup has not been actually performed. - -Columns: - -- `index` ([UInt32](../../sql-reference/data-types/int-uint.md)) — Index in marked_dropped_tables queue. -- `database` ([String](../../sql-reference/data-types/string.md)) — Database. -- `table` ([String](../../sql-reference/data-types/string.md)) — Table name. -- `uuid` ([UUID](../../sql-reference/data-types/uuid.md)) — Table uuid. -- `engine` ([String](../../sql-reference/data-types/string.md)) — Table engine name. -- `metadata_dropped_path` ([String](../../sql-reference/data-types/string.md)) — Path of table's metadata file in metadate_dropped directory. -- `table_dropped_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — The time when the next attempt to remove table's data is scheduled on. Usually it's the table when the table was dropped plus `database_atomic_delay_before_drop_table_sec` - -**Example** - -The following example shows how to get information about marked_dropped_tables. - -``` sql -SELECT * -FROM system.marked_dropped_tables\G -``` - -``` text -Row 1: -────── -index: 0 -database: default -table: test -uuid: 03141bb2-e97a-4d7c-a172-95cc066bb3bd -engine: MergeTree -metadata_dropped_path: /data/ClickHouse/build/programs/data/metadata_dropped/default.test.03141bb2-e97a-4d7c-a172-95cc066bb3bd.sql -table_dropped_time: 2023-03-16 23:43:31 -``` diff --git a/docs/en/operations/system-tables/quotas.md b/docs/en/operations/system-tables/quotas.md index ca8fc4d166f..ffe7a95df5b 100644 --- a/docs/en/operations/system-tables/quotas.md +++ b/docs/en/operations/system-tables/quotas.md @@ -20,7 +20,7 @@ Columns: - `apply_to_all` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Logical value. It shows which users the quota is applied to. Values: - `0` — The quota applies to users specify in the `apply_to_list`. - `1` — The quota applies to all users except those listed in `apply_to_except`. -- `apply_to_list` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — List of user names/[roles](../../operations/access-rights.md#role-management) that the quota should be applied to. +- `apply_to_list` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — List of user names/[roles](../../guides/sre/user-management/index.md#role-management) that the quota should be applied to. - `apply_to_except` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — List of user names/roles that the quota should not apply to. ## See Also {#see-also} diff --git a/docs/en/operations/system-tables/roles.md b/docs/en/operations/system-tables/roles.md index 729c98c89f3..5ef5e765c0f 100644 --- a/docs/en/operations/system-tables/roles.md +++ b/docs/en/operations/system-tables/roles.md @@ -3,7 +3,7 @@ slug: /en/operations/system-tables/roles --- # roles -Contains information about configured [roles](../../operations/access-rights.md#role-management). +Contains information about configured [roles](../../guides/sre/user-management/index.md#role-management). Columns: diff --git a/docs/en/operations/system-tables/users.md b/docs/en/operations/system-tables/users.md index b8c0403b8d6..385e3151eb7 100644 --- a/docs/en/operations/system-tables/users.md +++ b/docs/en/operations/system-tables/users.md @@ -3,7 +3,7 @@ slug: /en/operations/system-tables/users --- # users -Contains a list of [user accounts](../../operations/access-rights.md#user-account-management) configured at the server. +Contains a list of [user accounts](../../guides/sre/user-management/index.md#user-account-management) configured at the server. Columns: - `name` ([String](../../sql-reference/data-types/string.md)) — User name. diff --git a/docs/en/operations/tips.md b/docs/en/operations/tips.md index da34a6b7e9c..13353cd8e6a 100644 --- a/docs/en/operations/tips.md +++ b/docs/en/operations/tips.md @@ -126,7 +126,7 @@ Otherwise you may get `Illegal instruction` crashes when hypervisor is run on ol ## ClickHouse Keeper and ZooKeeper {#zookeeper} -ClickHouse Keeper is recommended to replace ZooKeeper for ClickHouse clusters. See the documentation for [ClickHouse Keeper](clickhouse-keeper.md) +ClickHouse Keeper is recommended to replace ZooKeeper for ClickHouse clusters. See the documentation for [ClickHouse Keeper](../guides/sre/keeper/index.md) If you would like to continue using ZooKeeper then it is best to use a fresh version of ZooKeeper – 3.4.9 or later. The version in stable Linux distributions may be outdated. @@ -134,7 +134,7 @@ You should never use manually written scripts to transfer data between different If you want to divide an existing ZooKeeper cluster into two, the correct way is to increase the number of its replicas and then reconfigure it as two independent clusters. -You can run ClickHouse Keeper on the same server as ClickHouse in test environments, or in environments with low ingestion rate. +You can run ClickHouse Keeper on the same server as ClickHouse in test environments, or in environments with low ingestion rate. For production environments we suggest to use separate servers for ClickHouse and ZooKeeper/Keeper, or place ClickHouse files and Keeper files on to separate disks. Because ZooKeeper/Keeper are very sensitive for disk latency and ClickHouse may utilize all available system resources. You can have ZooKeeper observers in an ensemble but ClickHouse servers should not interact with observers. diff --git a/docs/en/operations/utilities/clickhouse-local.md b/docs/en/operations/utilities/clickhouse-local.md index 08640b5c16b..6bf1269c1d9 100644 --- a/docs/en/operations/utilities/clickhouse-local.md +++ b/docs/en/operations/utilities/clickhouse-local.md @@ -4,9 +4,9 @@ sidebar_position: 60 sidebar_label: clickhouse-local --- -# clickhouse-local +# clickhouse-local -The `clickhouse-local` program enables you to perform fast processing on local files, without having to deploy and configure the ClickHouse server. It accepts data that represent tables and queries them using [ClickHouse SQL dialect](../../sql-reference/). `clickhouse-local` uses the same core as ClickHouse server, so it supports most of the features and the same set of formats and table engines. +The `clickhouse-local` program enables you to perform fast processing on local files, without having to deploy and configure the ClickHouse server. It accepts data that represent tables and queries them using [ClickHouse SQL dialect](../../sql-reference/index.md). `clickhouse-local` uses the same core as ClickHouse server, so it supports most of the features and the same set of formats and table engines. By default `clickhouse-local` has access to data on the same host, and it does not depend on the server's configuration. It also supports loading server configuration using `--config-file` argument. For temporary data, a unique temporary data directory is created by default. diff --git a/docs/en/operations/utilities/index.md b/docs/en/operations/utilities/index.md index a8c0239c102..b2f66af1084 100644 --- a/docs/en/operations/utilities/index.md +++ b/docs/en/operations/utilities/index.md @@ -1,11 +1,11 @@ --- slug: /en/operations/utilities/ sidebar_position: 56 -sidebar_label: Overview +sidebar_label: Utilities pagination_next: 'en/operations/utilities/clickhouse-copier' --- -# ClickHouse Utilities +# List of tools and utilities - [clickhouse-local](../../operations/utilities/clickhouse-local.md) — Allows running SQL queries on data without starting the ClickHouse server, similar to how `awk` does this. - [clickhouse-copier](../../operations/utilities/clickhouse-copier.md) — Copies (and reshards) data from one cluster to another cluster. diff --git a/docs/en/sql-reference/_category_.yml b/docs/en/sql-reference/_category_.yml index d799ecef539..45eaa6e7c16 100644 --- a/docs/en/sql-reference/_category_.yml +++ b/docs/en/sql-reference/_category_.yml @@ -1,7 +1,7 @@ -position: 15 +position: 1 label: 'SQL Reference' collapsible: true collapsed: true link: - type: doc - id: en/sql-reference/index + type: generated-index + slug: /en/sql-reference diff --git a/docs/en/sql-reference/aggregate-functions/reference/contingency.md b/docs/en/sql-reference/aggregate-functions/reference/contingency.md index e75537778fe..9e89e99e66d 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/contingency.md +++ b/docs/en/sql-reference/aggregate-functions/reference/contingency.md @@ -5,7 +5,7 @@ sidebar_position: 350 # contingency -The `contingency` function calculates the [contingency coefficient](https://en.wikipedia.org/wiki/Contingency_table#Cram%C3%A9r's_V_and_the_contingency_coefficient_C), a value that measures the association between two columns in a table. The computation is similar to [the `cramersV` function](./cramersv) but with a different denominator in the square root. +The `contingency` function calculates the [contingency coefficient](https://en.wikipedia.org/wiki/Contingency_table#Cram%C3%A9r's_V_and_the_contingency_coefficient_C), a value that measures the association between two columns in a table. The computation is similar to [the `cramersV` function](./cramersv.md) but with a different denominator in the square root. **Syntax** diff --git a/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md b/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md index 51524033147..651b5e7b5a2 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md +++ b/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md @@ -6,7 +6,7 @@ sidebar_position: 352 # cramersVBiasCorrected -Cramér's V is a measure of association between two columns in a table. The result of the [`cramersV` function](./cramersv) ranges from 0 (corresponding to no association between the variables) to 1 and can reach 1 only when each value is completely determined by the other. The function can be heavily biased, so this version of Cramér's V uses the [bias correction](https://en.wikipedia.org/wiki/Cram%C3%A9r%27s_V#Bias_correction). +Cramér's V is a measure of association between two columns in a table. The result of the [`cramersV` function](./cramersv.md) ranges from 0 (corresponding to no association between the variables) to 1 and can reach 1 only when each value is completely determined by the other. The function can be heavily biased, so this version of Cramér's V uses the [bias correction](https://en.wikipedia.org/wiki/Cram%C3%A9r%27s_V#Bias_correction). diff --git a/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md b/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md index 5546ade1758..5d82d3575fc 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md +++ b/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md @@ -19,7 +19,7 @@ Each `value` corresponds to the determinate `timeunit`. The half-life `x` is the **Arguments** - `value` — Value. [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md) or [Decimal](../../../sql-reference/data-types/decimal.md). -- `timeunit` — Timeunit. [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md) or [Decimal](../../../sql-reference/data-types/decimal.md). Timeunit is not timestamp (seconds), it's -- an index of the time interval. Can be calculated using [intDiv](../../functions/arithmetic-functions/#intdiva-b). +- `timeunit` — Timeunit. [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md) or [Decimal](../../../sql-reference/data-types/decimal.md). Timeunit is not timestamp (seconds), it's -- an index of the time interval. Can be calculated using [intDiv](../../functions/arithmetic-functions.md#intdiva-b). **Parameters** diff --git a/docs/en/sql-reference/data-types/json.md b/docs/en/sql-reference/data-types/json.md index d9099ba5ad3..a21898de9a2 100644 --- a/docs/en/sql-reference/data-types/json.md +++ b/docs/en/sql-reference/data-types/json.md @@ -7,7 +7,7 @@ sidebar_label: JSON # JSON :::warning -This feature is experimental and is not production ready. If you need to work with JSON documents, consider using [this guide](/docs/en/guides/developer/working-with-json/json-load-data.md) instead. +This feature is experimental and is not production ready. If you need to work with JSON documents, consider using [this guide](/docs/en/integrations/data-ingestion/data-formats/json.md) instead. ::: Stores JavaScript Object Notation (JSON) documents in a single column. diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/_snippet_dictionary_in_cloud.md b/docs/en/sql-reference/dictionaries/_snippet_dictionary_in_cloud.md similarity index 100% rename from docs/en/sql-reference/dictionaries/external-dictionaries/_snippet_dictionary_in_cloud.md rename to docs/en/sql-reference/dictionaries/_snippet_dictionary_in_cloud.md diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/_category_.yml b/docs/en/sql-reference/dictionaries/external-dictionaries/_category_.yml deleted file mode 100644 index af79ff9af23..00000000000 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/_category_.yml +++ /dev/null @@ -1,8 +0,0 @@ -position: 37 -label: 'Dictionaries' -collapsible: true -collapsed: true -link: - type: generated-index - title: Dictionaries - slug: /en/sql-reference/dictionaries/external-dictionaries diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md deleted file mode 100644 index ee9cd2c1f2e..00000000000 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md +++ /dev/null @@ -1,67 +0,0 @@ ---- -slug: /en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical -sidebar_position: 45 -sidebar_label: Hierarchical dictionaries ---- - -# Hierarchical Dictionaries - -ClickHouse supports hierarchical dictionaries with a [numeric key](../../dictionaries/external-dictionaries/external-dicts-dict-structure.md#numeric-key). - -Look at the following hierarchical structure: - -``` text -0 (Common parent) -│ -├── 1 (Russia) -│ │ -│ └── 2 (Moscow) -│ │ -│ └── 3 (Center) -│ -└── 4 (Great Britain) - │ - └── 5 (London) -``` - -This hierarchy can be expressed as the following dictionary table. - -| region_id | parent_region | region_name | -|------------|----------------|---------------| -| 1 | 0 | Russia | -| 2 | 1 | Moscow | -| 3 | 2 | Center | -| 4 | 0 | Great Britain | -| 5 | 4 | London | - -This table contains a column `parent_region` that contains the key of the nearest parent for the element. - -ClickHouse supports the [hierarchical](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#hierarchical-dict-attr) property for [external dictionary](../../../sql-reference/dictionaries/external-dictionaries/) attributes. This property allows you to configure the hierarchical dictionary similar to described above. - -The [dictGetHierarchy](../../../sql-reference/functions/ext-dict-functions.md#dictgethierarchy) function allows you to get the parent chain of an element. - -For our example, the structure of dictionary can be the following: - -``` xml - - - - region_id - - - - parent_region - UInt64 - 0 - true - - - - region_name - String - - - - - -``` diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md deleted file mode 100644 index 4dc6fd33849..00000000000 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ /dev/null @@ -1,751 +0,0 @@ ---- -slug: /en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout -sidebar_position: 41 -sidebar_label: Storing Dictionaries in Memory ---- -import CloudDetails from '@site/docs/en/sql-reference/dictionaries/external-dictionaries/_snippet_dictionary_in_cloud.md'; - -# Storing Dictionaries in Memory - -There are a variety of ways to store dictionaries in memory. - -We recommend [flat](#flat), [hashed](#dicts-external_dicts_dict_layout-hashed) and [complex_key_hashed](#complex-key-hashed), which provide optimal processing speed. - -Caching is not recommended because of potentially poor performance and difficulties in selecting optimal parameters. Read more in the section [cache](#cache). - -There are several ways to improve dictionary performance: - -- Call the function for working with the dictionary after `GROUP BY`. -- Mark attributes to extract as injective. An attribute is called injective if different attribute values correspond to different keys. So when `GROUP BY` uses a function that fetches an attribute value by the key, this function is automatically taken out of `GROUP BY`. - -ClickHouse generates an exception for errors with dictionaries. Examples of errors: - -- The dictionary being accessed could not be loaded. -- Error querying a `cached` dictionary. - -You can view the list of dictionaries and their statuses in the [system.dictionaries](../../../operations/system-tables/dictionaries.md) table. - - - -The configuration looks like this: - -``` xml - - - ... - - - - - - ... - - -``` - -Corresponding [DDL-query](../../../sql-reference/statements/create/dictionary.md): - -``` sql -CREATE DICTIONARY (...) -... -LAYOUT(LAYOUT_TYPE(param value)) -- layout settings -... -``` - -Dictionaries without word `complex-key*` in a layout have a key with [UInt64](../../../sql-reference/data-types/int-uint.md) type, `complex-key*` dictionaries have a composite key (complex, with arbitrary types). - -[UInt64](../../../sql-reference/data-types/int-uint.md) keys in XML dictionaries are defined with `` tag. - -Configuration example (column key_column has UInt64 type): -```xml -... - - - key_column - -... -``` - -Composite `complex` keys XML dictionaries are defined `` tag. - -Configuration example of a composite key (key has one element with [String](../../../sql-reference/data-types/string.md) type): -```xml -... - - - - country_code - String - - -... -``` - -## Ways to Store Dictionaries in Memory - -- [flat](#flat) -- [hashed](#dicts-external_dicts_dict_layout-hashed) -- [sparse_hashed](#dicts-external_dicts_dict_layout-sparse_hashed) -- [complex_key_hashed](#complex-key-hashed) -- [complex_key_sparse_hashed](#complex-key-sparse-hashed) -- [hashed_array](#dicts-external_dicts_dict_layout-hashed-array) -- [complex_key_hashed_array](#complex-key-hashed-array) -- [range_hashed](#range-hashed) -- [complex_key_range_hashed](#complex-key-range-hashed) -- [cache](#cache) -- [complex_key_cache](#complex-key-cache) -- [ssd_cache](#ssd-cache) -- [complex_key_ssd_cache](#complex-key-ssd-cache) -- [direct](#direct) -- [complex_key_direct](#complex-key-direct) -- [ip_trie](#ip-trie) - -### flat - -The dictionary is completely stored in memory in the form of flat arrays. How much memory does the dictionary use? The amount is proportional to the size of the largest key (in space used). - -The dictionary key has the [UInt64](../../../sql-reference/data-types/int-uint.md) type and the value is limited to `max_array_size` (by default — 500,000). If a larger key is discovered when creating the dictionary, ClickHouse throws an exception and does not create the dictionary. Dictionary flat arrays initial size is controlled by `initial_array_size` setting (by default — 1024). - -All types of sources are supported. When updating, data (from a file or from a table) is read in it entirety. - -This method provides the best performance among all available methods of storing the dictionary. - -Configuration example: - -``` xml - - - 50000 - 5000000 - - -``` - -or - -``` sql -LAYOUT(FLAT(INITIAL_ARRAY_SIZE 50000 MAX_ARRAY_SIZE 5000000)) -``` - -### hashed - -The dictionary is completely stored in memory in the form of a hash table. The dictionary can contain any number of elements with any identifiers In practice, the number of keys can reach tens of millions of items. - -The dictionary key has the [UInt64](../../../sql-reference/data-types/int-uint.md) type. - -All types of sources are supported. When updating, data (from a file or from a table) is read in its entirety. - -Configuration example: - -``` xml - - - -``` - -or - -``` sql -LAYOUT(HASHED()) -``` - -If `shards` greater then 1 (default is `1`) the dictionary will load data in parallel, useful if you have huge amount of elements in one dictionary. - -Configuration example: - -``` xml - - - 10 - - 10000 - - -``` - -or - -``` sql -LAYOUT(HASHED(SHARDS 10 [SHARD_LOAD_QUEUE_BACKLOG 10000])) -``` - -### sparse_hashed - -Similar to `hashed`, but uses less memory in favor more CPU usage. - -The dictionary key has the [UInt64](../../../sql-reference/data-types/int-uint.md) type. - -Configuration example: - -``` xml - - - -``` - -or - -``` sql -LAYOUT(SPARSE_HASHED()) -``` - -It is also possible to use `shards` for this type of dictionary, and again it is more important for `sparse_hashed` then for `hashed`, since `sparse_hashed` is slower. - -### complex_key_hashed - -This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to `hashed`. - -Configuration example: - -``` xml - - - 1 - - - -``` - -or - -``` sql -LAYOUT(COMPLEX_KEY_HASHED([SHARDS 1] [SHARD_LOAD_QUEUE_BACKLOG 10000])) -``` - -### complex_key_sparse_hashed - -This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to [sparse_hashed](#dicts-external_dicts_dict_layout-sparse_hashed). - -Configuration example: - -``` xml - - - 1 - - -``` - -or - -``` sql -LAYOUT(COMPLEX_KEY_SPARSE_HASHED([SHARDS 1] [SHARD_LOAD_QUEUE_BACKLOG 10000])) -``` - -### hashed_array - -The dictionary is completely stored in memory. Each attribute is stored in an array. The key attribute is stored in the form of a hashed table where value is an index in the attributes array. The dictionary can contain any number of elements with any identifiers. In practice, the number of keys can reach tens of millions of items. - -The dictionary key has the [UInt64](../../../sql-reference/data-types/int-uint.md) type. - -All types of sources are supported. When updating, data (from a file or from a table) is read in its entirety. - -Configuration example: - -``` xml - - - - -``` - -or - -``` sql -LAYOUT(HASHED_ARRAY()) -``` - -### complex_key_hashed_array - -This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to [hashed_array](#dicts-external_dicts_dict_layout-hashed-array). - -Configuration example: - -``` xml - - - -``` - -or - -``` sql -LAYOUT(COMPLEX_KEY_HASHED_ARRAY()) -``` - -### range_hashed - -The dictionary is stored in memory in the form of a hash table with an ordered array of ranges and their corresponding values. - -The dictionary key has the [UInt64](../../../sql-reference/data-types/int-uint.md) type. -This storage method works the same way as hashed and allows using date/time (arbitrary numeric type) ranges in addition to the key. - -Example: The table contains discounts for each advertiser in the format: - -``` text -┌─advertiser_id─┬─discount_start_date─┬─discount_end_date─┬─amount─┐ -│ 123 │ 2015-01-16 │ 2015-01-31 │ 0.25 │ -│ 123 │ 2015-01-01 │ 2015-01-15 │ 0.15 │ -│ 456 │ 2015-01-01 │ 2015-01-15 │ 0.05 │ -└───────────────┴─────────────────────┴───────────────────┴────────┘ -``` - -To use a sample for date ranges, define the `range_min` and `range_max` elements in the [structure](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). These elements must contain elements `name` and `type` (if `type` is not specified, the default type will be used - Date). `type` can be any numeric type (Date / DateTime / UInt64 / Int32 / others). - -:::warning -Values of `range_min` and `range_max` should fit in `Int64` type. -::: - -Example: - -``` xml - - - - min - - - - - advertiser_id - - - discount_start_date - Date - - - discount_end_date - Date - - ... -``` - -or - -``` sql -CREATE DICTIONARY discounts_dict ( - advertiser_id UInt64, - discount_start_date Date, - discount_end_date Date, - amount Float64 -) -PRIMARY KEY id -SOURCE(CLICKHOUSE(TABLE 'discounts')) -LIFETIME(MIN 1 MAX 1000) -LAYOUT(RANGE_HASHED(range_lookup_strategy 'max')) -RANGE(MIN discount_start_date MAX discount_end_date) -``` - -To work with these dictionaries, you need to pass an additional argument to the `dictGet` function, for which a range is selected: - -``` sql -dictGet('dict_name', 'attr_name', id, date) -``` -Query example: - -``` sql -SELECT dictGet('discounts_dict', 'amount', 1, '2022-10-20'::Date); -``` - -This function returns the value for the specified `id`s and the date range that includes the passed date. - -Details of the algorithm: - -- If the `id` is not found or a range is not found for the `id`, it returns the default value of the attribute's type. -- If there are overlapping ranges and `range_lookup_strategy=min`, it returns a matching range with minimal `range_min`, if several ranges found, it returns a range with minimal `range_max`, if again several ranges found (several ranges had the same `range_min` and `range_max` it returns a random range of them. -- If there are overlapping ranges and `range_lookup_strategy=max`, it returns a matching range with maximal `range_min`, if several ranges found, it returns a range with maximal `range_max`, if again several ranges found (several ranges had the same `range_min` and `range_max` it returns a random range of them. -- If the `range_max` is `NULL`, the range is open. `NULL` is treated as maximal possible value. For the `range_min` `1970-01-01` or `0` (-MAX_INT) can be used as the open value. - -Configuration example: - -``` xml - - - ... - - - - - - - - Abcdef - - - StartTimeStamp - UInt64 - - - EndTimeStamp - UInt64 - - - XXXType - String - - - - - - -``` - -or - -``` sql -CREATE DICTIONARY somedict( - Abcdef UInt64, - StartTimeStamp UInt64, - EndTimeStamp UInt64, - XXXType String DEFAULT '' -) -PRIMARY KEY Abcdef -RANGE(MIN StartTimeStamp MAX EndTimeStamp) -``` - -Configuration example with overlapping ranges and open ranges: - -```sql -CREATE TABLE discounts -( - advertiser_id UInt64, - discount_start_date Date, - discount_end_date Nullable(Date), - amount Float64 -) -ENGINE = Memory; - -INSERT INTO discounts VALUES (1, '2015-01-01', Null, 0.1); -INSERT INTO discounts VALUES (1, '2015-01-15', Null, 0.2); -INSERT INTO discounts VALUES (2, '2015-01-01', '2015-01-15', 0.3); -INSERT INTO discounts VALUES (2, '2015-01-04', '2015-01-10', 0.4); -INSERT INTO discounts VALUES (3, '1970-01-01', '2015-01-15', 0.5); -INSERT INTO discounts VALUES (3, '1970-01-01', '2015-01-10', 0.6); - -SELECT * FROM discounts ORDER BY advertiser_id, discount_start_date; -┌─advertiser_id─┬─discount_start_date─┬─discount_end_date─┬─amount─┐ -│ 1 │ 2015-01-01 │ ᴺᵁᴸᴸ │ 0.1 │ -│ 1 │ 2015-01-15 │ ᴺᵁᴸᴸ │ 0.2 │ -│ 2 │ 2015-01-01 │ 2015-01-15 │ 0.3 │ -│ 2 │ 2015-01-04 │ 2015-01-10 │ 0.4 │ -│ 3 │ 1970-01-01 │ 2015-01-15 │ 0.5 │ -│ 3 │ 1970-01-01 │ 2015-01-10 │ 0.6 │ -└───────────────┴─────────────────────┴───────────────────┴────────┘ - --- RANGE_LOOKUP_STRATEGY 'max' - -CREATE DICTIONARY discounts_dict -( - advertiser_id UInt64, - discount_start_date Date, - discount_end_date Nullable(Date), - amount Float64 -) -PRIMARY KEY advertiser_id -SOURCE(CLICKHOUSE(TABLE discounts)) -LIFETIME(MIN 600 MAX 900) -LAYOUT(RANGE_HASHED(RANGE_LOOKUP_STRATEGY 'max')) -RANGE(MIN discount_start_date MAX discount_end_date); - -select dictGet('discounts_dict', 'amount', 1, toDate('2015-01-14')) res; -┌─res─┐ -│ 0.1 │ -- the only one range is matching: 2015-01-01 - Null -└─────┘ - -select dictGet('discounts_dict', 'amount', 1, toDate('2015-01-16')) res; -┌─res─┐ -│ 0.2 │ -- two ranges are matching, range_min 2015-01-15 (0.2) is bigger than 2015-01-01 (0.1) -└─────┘ - -select dictGet('discounts_dict', 'amount', 2, toDate('2015-01-06')) res; -┌─res─┐ -│ 0.4 │ -- two ranges are matching, range_min 2015-01-04 (0.4) is bigger than 2015-01-01 (0.3) -└─────┘ - -select dictGet('discounts_dict', 'amount', 3, toDate('2015-01-01')) res; -┌─res─┐ -│ 0.5 │ -- two ranges are matching, range_min are equal, 2015-01-15 (0.5) is bigger than 2015-01-10 (0.6) -└─────┘ - -DROP DICTIONARY discounts_dict; - --- RANGE_LOOKUP_STRATEGY 'min' - -CREATE DICTIONARY discounts_dict -( - advertiser_id UInt64, - discount_start_date Date, - discount_end_date Nullable(Date), - amount Float64 -) -PRIMARY KEY advertiser_id -SOURCE(CLICKHOUSE(TABLE discounts)) -LIFETIME(MIN 600 MAX 900) -LAYOUT(RANGE_HASHED(RANGE_LOOKUP_STRATEGY 'min')) -RANGE(MIN discount_start_date MAX discount_end_date); - -select dictGet('discounts_dict', 'amount', 1, toDate('2015-01-14')) res; -┌─res─┐ -│ 0.1 │ -- the only one range is matching: 2015-01-01 - Null -└─────┘ - -select dictGet('discounts_dict', 'amount', 1, toDate('2015-01-16')) res; -┌─res─┐ -│ 0.1 │ -- two ranges are matching, range_min 2015-01-01 (0.1) is less than 2015-01-15 (0.2) -└─────┘ - -select dictGet('discounts_dict', 'amount', 2, toDate('2015-01-06')) res; -┌─res─┐ -│ 0.3 │ -- two ranges are matching, range_min 2015-01-01 (0.3) is less than 2015-01-04 (0.4) -└─────┘ - -select dictGet('discounts_dict', 'amount', 3, toDate('2015-01-01')) res; -┌─res─┐ -│ 0.6 │ -- two ranges are matching, range_min are equal, 2015-01-10 (0.6) is less than 2015-01-15 (0.5) -└─────┘ -``` - -### complex_key_range_hashed - -The dictionary is stored in memory in the form of a hash table with an ordered array of ranges and their corresponding values (see [range_hashed](#range-hashed)). This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). - -Configuration example: - -``` sql -CREATE DICTIONARY range_dictionary -( - CountryID UInt64, - CountryKey String, - StartDate Date, - EndDate Date, - Tax Float64 DEFAULT 0.2 -) -PRIMARY KEY CountryID, CountryKey -SOURCE(CLICKHOUSE(TABLE 'date_table')) -LIFETIME(MIN 1 MAX 1000) -LAYOUT(COMPLEX_KEY_RANGE_HASHED()) -RANGE(MIN StartDate MAX EndDate); -``` - -### cache - -The dictionary is stored in a cache that has a fixed number of cells. These cells contain frequently used elements. - -The dictionary key has the [UInt64](../../../sql-reference/data-types/int-uint.md) type. - -When searching for a dictionary, the cache is searched first. For each block of data, all keys that are not found in the cache or are outdated are requested from the source using `SELECT attrs... FROM db.table WHERE id IN (k1, k2, ...)`. The received data is then written to the cache. - -If keys are not found in dictionary, then update cache task is created and added into update queue. Update queue properties can be controlled with settings `max_update_queue_size`, `update_queue_push_timeout_milliseconds`, `query_wait_timeout_milliseconds`, `max_threads_for_updates`. - -For cache dictionaries, the expiration [lifetime](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) of data in the cache can be set. If more time than `lifetime` has passed since loading the data in a cell, the cell’s value is not used and key becomes expired. The key is re-requested the next time it needs to be used. This behaviour can be configured with setting `allow_read_expired_keys`. - -This is the least effective of all the ways to store dictionaries. The speed of the cache depends strongly on correct settings and the usage scenario. A cache type dictionary performs well only when the hit rates are high enough (recommended 99% and higher). You can view the average hit rate in the [system.dictionaries](../../../operations/system-tables/dictionaries.md) table. - -If setting `allow_read_expired_keys` is set to 1, by default 0. Then dictionary can support asynchronous updates. If a client requests keys and all of them are in cache, but some of them are expired, then dictionary will return expired keys for a client and request them asynchronously from the source. - -To improve cache performance, use a subquery with `LIMIT`, and call the function with the dictionary externally. - -All types of sources are supported. - -Example of settings: - -``` xml - - - - 1000000000 - - 0 - - 100000 - - 10 - - 60000 - - 4 - - -``` - -or - -``` sql -LAYOUT(CACHE(SIZE_IN_CELLS 1000000000)) -``` - -Set a large enough cache size. You need to experiment to select the number of cells: - -1. Set some value. -2. Run queries until the cache is completely full. -3. Assess memory consumption using the `system.dictionaries` table. -4. Increase or decrease the number of cells until the required memory consumption is reached. - -:::warning -Do not use ClickHouse as a source, because it is slow to process queries with random reads. -::: - -### complex_key_cache - -This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to `cache`. - -### ssd_cache - -Similar to `cache`, but stores data on SSD and index in RAM. All cache dictionary settings related to update queue can also be applied to SSD cache dictionaries. - -The dictionary key has the [UInt64](../../../sql-reference/data-types/int-uint.md) type. - -``` xml - - - - 4096 - - 16777216 - - 131072 - - 1048576 - - /var/lib/clickhouse/user_files/test_dict - - -``` - -or - -``` sql -LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 16777216 READ_BUFFER_SIZE 1048576 - PATH '/var/lib/clickhouse/user_files/test_dict')) -``` - -### complex_key_ssd_cache - -This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to `ssd_cache`. - -### direct - -The dictionary is not stored in memory and directly goes to the source during the processing of a request. - -The dictionary key has the [UInt64](../../../sql-reference/data-types/int-uint.md) type. - -All types of [sources](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md), except local files, are supported. - -Configuration example: - -``` xml - - - -``` - -or - -``` sql -LAYOUT(DIRECT()) -``` - -### complex_key_direct - -This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to `direct`. - -### ip_trie - -This type of storage is for mapping network prefixes (IP addresses) to metadata such as ASN. - -**Example** - -Suppose we have a table in ClickHouse that contains our IP prefixes and mappings: - -```sql -CREATE TABLE my_ip_addresses ( - prefix String, - asn UInt32, - cca2 String -) -ENGINE = MergeTree -PRIMARY KEY prefix; -``` - -```sql -INSERT INTO my_ip_addresses VALUES - ('202.79.32.0/20', 17501, 'NP'), - ('2620:0:870::/48', 3856, 'US'), - ('2a02:6b8:1::/48', 13238, 'RU'), - ('2001:db8::/32', 65536, 'ZZ') -; -``` - -Let's define an `ip_trie` dictionary for this table. The `ip_trie` layout requires a composite key: - -``` xml - - - - prefix - String - - - - asn - UInt32 - - - - cca2 - String - ?? - - ... - - - - - - true - - -``` - -or - -``` sql -CREATE DICTIONARY my_ip_trie_dictionary ( - prefix String, - asn UInt32, - cca2 String DEFAULT '??' -) -PRIMARY KEY prefix -SOURCE(CLICKHOUSE(TABLE 'my_ip_addresses')) -LAYOUT(IP_TRIE) -LIFETIME(3600); -``` - -The key must have only one `String` type attribute that contains an allowed IP prefix. Other types are not supported yet. - -For queries, you must use the same functions (`dictGetT` with a tuple) as for dictionaries with composite keys. The syntax is: - -``` sql -dictGetT('dict_name', 'attr_name', tuple(ip)) -``` - -The function takes either `UInt32` for IPv4, or `FixedString(16)` for IPv6. For example: - -``` sql -select dictGet('my_ip_trie_dictionary', 'asn', tuple(IPv6StringToNum('2001:db8::1'))) -``` - -Other types are not supported yet. The function returns the attribute for the prefix that corresponds to this IP address. If there are overlapping prefixes, the most specific one is returned. - -Data must completely fit into RAM. - -## Related Content - -- [Using dictionaries to accelerate queries](https://clickhouse.com/blog/faster-queries-dictionaries-clickhouse) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md deleted file mode 100644 index 8e9dbd392aa..00000000000 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -slug: /en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime -sidebar_position: 42 -sidebar_label: Dictionary Updates ---- -import CloudDetails from '@site/docs/en/sql-reference/dictionaries/external-dictionaries/_snippet_dictionary_in_cloud.md'; - -# Dictionary Updates - -ClickHouse periodically updates the dictionaries. The update interval for fully downloaded dictionaries and the invalidation interval for cached dictionaries are defined in the `lifetime` tag in seconds. - -Dictionary updates (other than loading for first use) do not block queries. During updates, the old version of a dictionary is used. If an error occurs during an update, the error is written to the server log, and queries continue using the old version of dictionaries. - -Example of settings: - - - -``` xml - - ... - 300 - ... - -``` - -or - -``` sql -CREATE DICTIONARY (...) -... -LIFETIME(300) -... -``` - -Setting `0` (`LIFETIME(0)`) prevents dictionaries from updating. - -You can set a time interval for updates, and ClickHouse will choose a uniformly random time within this range. This is necessary in order to distribute the load on the dictionary source when updating on a large number of servers. - -Example of settings: - -``` xml - - ... - - 300 - 360 - - ... - -``` - -or - -``` sql -LIFETIME(MIN 300 MAX 360) -``` - -If `0` and `0`, ClickHouse does not reload the dictionary by timeout. -In this case, ClickHouse can reload the dictionary earlier if the dictionary configuration file was changed or the `SYSTEM RELOAD DICTIONARY` command was executed. - -When updating the dictionaries, the ClickHouse server applies different logic depending on the type of [source](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md): - -- For a text file, it checks the time of modification. If the time differs from the previously recorded time, the dictionary is updated. -- For MySQL source, the time of modification is checked using a `SHOW TABLE STATUS` query (in case of MySQL 8 you need to disable meta-information caching in MySQL by `set global information_schema_stats_expiry=0`). -- Dictionaries from other sources are updated every time by default. - -For other sources (ODBC, PostgreSQL, ClickHouse, etc), you can set up a query that will update the dictionaries only if they really changed, rather than each time. To do this, follow these steps: - -- The dictionary table must have a field that always changes when the source data is updated. -- The settings of the source must specify a query that retrieves the changing field. The ClickHouse server interprets the query result as a row, and if this row has changed relative to its previous state, the dictionary is updated. Specify the query in the `` field in the settings for the [source](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md). - -Example of settings: - -``` xml - - ... - - ... - SELECT update_time FROM dictionary_source where id = 1 - - ... - -``` - -or - -``` sql -... -SOURCE(ODBC(... invalidate_query 'SELECT update_time FROM dictionary_source where id = 1')) -... -``` - -For `Cache`, `ComplexKeyCache`, `SSDCache`, and `SSDComplexKeyCache` dictionaries both synchronious and asynchronious updates are supported. - -It is also possible for `Flat`, `Hashed`, `ComplexKeyHashed` dictionaries to only request data that was changed after the previous update. If `update_field` is specified as part of the dictionary source configuration, value of the previous update time in seconds will be added to the data request. Depends on source type (Executable, HTTP, MySQL, PostgreSQL, ClickHouse, or ODBC) different logic will be applied to `update_field` before request data from an external source. - -- If the source is HTTP then `update_field` will be added as a query parameter with the last update time as the parameter value. -- If the source is Executable then `update_field` will be added as an executable script argument with the last update time as the argument value. -- If the source is ClickHouse, MySQL, PostgreSQL, ODBC there will be an additional part of `WHERE`, where `update_field` is compared as greater or equal with the last update time. - - Per default, this `WHERE`-condition is checked at the highest level of the SQL-Query. Alternatively, the condition can be checked in any other `WHERE`-clause within the query using the `{condition}`-keyword. Example: - ```sql - ... - SOURCE(CLICKHOUSE(... - update_field 'added_time' - QUERY ' - SELECT my_arr.1 AS x, my_arr.2 AS y, creation_time - FROM ( - SELECT arrayZip(x_arr, y_arr) AS my_arr, creation_time - FROM dictionary_source - WHERE {condition} - )' - )) - ... - ``` - -If `update_field` option is set, additional option `update_lag` can be set. Value of `update_lag` option is subtracted from previous update time before request updated data. - -Example of settings: - -``` xml - - ... - - ... - added_time - 15 - - ... - -``` - -or - -``` sql -... -SOURCE(CLICKHOUSE(... update_field 'added_time' update_lag 15)) -... -``` - -## Related Content - -- [Using dictionaries to accelerate queries](https://clickhouse.com/blog/faster-queries-dictionaries-clickhouse) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md deleted file mode 100644 index 8ef19a181e7..00000000000 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md +++ /dev/null @@ -1,140 +0,0 @@ ---- -slug: /en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon -sidebar_position: 46 -sidebar_label: Polygon Dictionaries With Grids -title: "Polygon dictionaries" ---- -import CloudDetails from '@site/docs/en/sql-reference/dictionaries/external-dictionaries/_snippet_dictionary_in_cloud.md'; - -Polygon dictionaries allow you to efficiently search for the polygon containing specified points. -For example: defining a city area by geographical coordinates. - -Example of a polygon dictionary configuration: - - - -``` xml - - - - - key - Array(Array(Array(Array(Float64)))) - - - - - name - String - - - - - value - UInt64 - 0 - - - - - - 1 - - - - ... - -``` - -The corresponding [DDL-query](../../../sql-reference/statements/create/dictionary.md#create-dictionary-query): -``` sql -CREATE DICTIONARY polygon_dict_name ( - key Array(Array(Array(Array(Float64)))), - name String, - value UInt64 -) -PRIMARY KEY key -LAYOUT(POLYGON(STORE_POLYGON_KEY_COLUMN 1)) -... -``` - -When configuring the polygon dictionary, the key must have one of two types: - -- A simple polygon. It is an array of points. -- MultiPolygon. It is an array of polygons. Each polygon is a two-dimensional array of points. The first element of this array is the outer boundary of the polygon, and subsequent elements specify areas to be excluded from it. - -Points can be specified as an array or a tuple of their coordinates. In the current implementation, only two-dimensional points are supported. - -The user can [upload their own data](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md) in all formats supported by ClickHouse. - -There are 3 types of [in-memory storage](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md) available: - -- `POLYGON_SIMPLE`. This is a naive implementation, where a linear pass through all polygons is made for each query, and membership is checked for each one without using additional indexes. - -- `POLYGON_INDEX_EACH`. A separate index is built for each polygon, which allows you to quickly check whether it belongs in most cases (optimized for geographical regions). -Also, a grid is superimposed on the area under consideration, which significantly narrows the number of polygons under consideration. -The grid is created by recursively dividing the cell into 16 equal parts and is configured with two parameters. -The division stops when the recursion depth reaches `MAX_DEPTH` or when the cell crosses no more than `MIN_INTERSECTIONS` polygons. -To respond to the query, there is a corresponding cell, and the index for the polygons stored in it is accessed alternately. - -- `POLYGON_INDEX_CELL`. This placement also creates the grid described above. The same options are available. For each sheet cell, an index is built on all pieces of polygons that fall into it, which allows you to quickly respond to a request. - -- `POLYGON`. Synonym to `POLYGON_INDEX_CELL`. - -Dictionary queries are carried out using standard [functions](../../../sql-reference/functions/ext-dict-functions.md) for working with dictionaries. -An important difference is that here the keys will be the points for which you want to find the polygon containing them. - -**Example** - -Example of working with the dictionary defined above: - -``` sql -CREATE TABLE points ( - x Float64, - y Float64 -) -... -SELECT tuple(x, y) AS key, dictGet(dict_name, 'name', key), dictGet(dict_name, 'value', key) FROM points ORDER BY x, y; -``` - -As a result of executing the last command for each point in the 'points' table, a minimum area polygon containing this point will be found, and the requested attributes will be output. - -**Example** - -You can read columns from polygon dictionaries via SELECT query, just turn on the `store_polygon_key_column = 1` in the dictionary configuration or corresponding DDL-query. - -Query: - -``` sql -CREATE TABLE polygons_test_table -( - key Array(Array(Array(Tuple(Float64, Float64)))), - name String -) ENGINE = TinyLog; - -INSERT INTO polygons_test_table VALUES ([[[(3, 1), (0, 1), (0, -1), (3, -1)]]], 'Value'); - -CREATE DICTIONARY polygons_test_dictionary -( - key Array(Array(Array(Tuple(Float64, Float64)))), - name String -) -PRIMARY KEY key -SOURCE(CLICKHOUSE(TABLE 'polygons_test_table')) -LAYOUT(POLYGON(STORE_POLYGON_KEY_COLUMN 1)) -LIFETIME(0); - -SELECT * FROM polygons_test_dictionary; -``` - -Result: - -``` text -┌─key─────────────────────────────┬─name──┐ -│ [[[(3,1),(0,1),(0,-1),(3,-1)]]] │ Value │ -└─────────────────────────────────┴───────┘ -``` - -## Related Content - -- [Exploring massive, real-world data sets: 100+ Years of Weather Records in ClickHouse](https://clickhouse.com/blog/real-world-data-noaa-climate-data) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md deleted file mode 100644 index 897945a6d9d..00000000000 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ /dev/null @@ -1,847 +0,0 @@ ---- -slug: /en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources -sidebar_position: 43 -sidebar_label: Dictionary Sources ---- -import CloudDetails from '@site/docs/en/sql-reference/dictionaries/external-dictionaries/_snippet_dictionary_in_cloud.md'; - -# Dictionary Sources - - - -A dictionary can be connected to ClickHouse from many different sources. - -If the dictionary is configured using an xml-file, the configuration looks like this: - -``` xml - - - ... - - - - - - ... - - ... - -``` - -In case of [DDL-query](../../../sql-reference/statements/create/dictionary.md), the configuration described above will look like: - -``` sql -CREATE DICTIONARY dict_name (...) -... -SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration -... -``` - -The source is configured in the `source` section. - -For source types [Local file](#dicts-external_dicts_dict_sources-local_file), [Executable file](#dicts-external_dicts_dict_sources-executable), [HTTP(s)](#dicts-external_dicts_dict_sources-http), [ClickHouse](#dicts-external_dicts_dict_sources-clickhouse) -optional settings are available: - -``` xml - - - /opt/dictionaries/os.tsv - TabSeparated - - - 0 - - -``` - -or - -``` sql -SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated')) -SETTINGS(format_csv_allow_single_quotes = 0) -``` - -Types of sources (`source_type`): - -- [Local file](#dicts-external_dicts_dict_sources-local_file) -- [Executable File](#dicts-external_dicts_dict_sources-executable) -- [Executable Pool](#dicts-external_dicts_dict_sources-executable_pool) -- [HTTP(s)](#dicts-external_dicts_dict_sources-http) -- DBMS - - [ODBC](#odbc) - - [MySQL](#mysql) - - [ClickHouse](#clickhouse) - - [MongoDB](#mongodb) - - [Redis](#redis) - - [Cassandra](#cassandra) - - [PostgreSQL](#postgresql) - -## Local File - -Example of settings: - -``` xml - - - /opt/dictionaries/os.tsv - TabSeparated - - -``` - -or - -``` sql -SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated')) -``` - -Setting fields: - -- `path` – The absolute path to the file. -- `format` – The file format. All the formats described in [Formats](../../../interfaces/formats.md#formats) are supported. - -When a dictionary with source `FILE` is created via DDL command (`CREATE DICTIONARY ...`), the source file needs to be located in the `user_files` directory to prevent DB users from accessing arbitrary files on the ClickHouse node. - -**See Also** - -- [Dictionary function](../../../sql-reference/table-functions/dictionary.md#dictionary-function) - -## Executable File - -Working with executable files depends on [how the dictionary is stored in memory](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request to the executable file’s STDIN. Otherwise, ClickHouse starts the executable file and treats its output as dictionary data. - -Example of settings: - -``` xml - - - cat /opt/dictionaries/os.tsv - TabSeparated - false - - -``` - -Setting fields: - -- `command` — The absolute path to the executable file, or the file name (if the command's directory is in the `PATH`). -- `format` — The file format. All the formats described in [Formats](../../../interfaces/formats.md#formats) are supported. -- `command_termination_timeout` — The executable script should contain a main read-write loop. After the dictionary is destroyed, the pipe is closed, and the executable file will have `command_termination_timeout` seconds to shutdown before ClickHouse will send a SIGTERM signal to the child process. `command_termination_timeout` is specified in seconds. Default value is 10. Optional parameter. -- `command_read_timeout` - Timeout for reading data from command stdout in milliseconds. Default value 10000. Optional parameter. -- `command_write_timeout` - Timeout for writing data to command stdin in milliseconds. Default value 10000. Optional parameter. -- `implicit_key` — The executable source file can return only values, and the correspondence to the requested keys is determined implicitly — by the order of rows in the result. Default value is false. -- `execute_direct` - If `execute_direct` = `1`, then `command` will be searched inside user_scripts folder specified by [user_scripts_path](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_scripts_path). Additional script arguments can be specified using a whitespace separator. Example: `script_name arg1 arg2`. If `execute_direct` = `0`, `command` is passed as argument for `bin/sh -c`. Default value is `0`. Optional parameter. -- `send_chunk_header` - controls whether to send row count before sending a chunk of data to process. Optional. Default value is `false`. - -That dictionary source can be configured only via XML configuration. Creating dictionaries with executable source via DDL is disabled; otherwise, the DB user would be able to execute arbitrary binaries on the ClickHouse node. - -## Executable Pool - -Executable pool allows loading data from pool of processes. This source does not work with dictionary layouts that need to load all data from source. Executable pool works if the dictionary [is stored](external-dicts-dict-layout.md#ways-to-store-dictionaries-in-memory) using `cache`, `complex_key_cache`, `ssd_cache`, `complex_key_ssd_cache`, `direct`, or `complex_key_direct` layouts. - -Executable pool will spawn a pool of processes with the specified command and keep them running until they exit. The program should read data from STDIN while it is available and output the result to STDOUT. It can wait for the next block of data on STDIN. ClickHouse will not close STDIN after processing a block of data, but will pipe another chunk of data when needed. The executable script should be ready for this way of data processing — it should poll STDIN and flush data to STDOUT early. - -Example of settings: - -``` xml - - - while read key; do printf "$key\tData for key $key\n"; done - TabSeparated - 10 - 10 - false - - -``` - -Setting fields: - -- `command` — The absolute path to the executable file, or the file name (if the program directory is written to `PATH`). -- `format` — The file format. All the formats described in “[Formats](../../../interfaces/formats.md#formats)” are supported. -- `pool_size` — Size of pool. If 0 is specified as `pool_size` then there is no pool size restrictions. Default value is `16`. -- `command_termination_timeout` — executable script should contain main read-write loop. After dictionary is destroyed, pipe is closed, and executable file will have `command_termination_timeout` seconds to shutdown, before ClickHouse will send SIGTERM signal to child process. Specified in seconds. Default value is 10. Optional parameter. -- `max_command_execution_time` — Maximum executable script command execution time for processing block of data. Specified in seconds. Default value is 10. Optional parameter. -- `command_read_timeout` - timeout for reading data from command stdout in milliseconds. Default value 10000. Optional parameter. -- `command_write_timeout` - timeout for writing data to command stdin in milliseconds. Default value 10000. Optional parameter. -- `implicit_key` — The executable source file can return only values, and the correspondence to the requested keys is determined implicitly — by the order of rows in the result. Default value is false. Optional parameter. -- `execute_direct` - If `execute_direct` = `1`, then `command` will be searched inside user_scripts folder specified by [user_scripts_path](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_scripts_path). Additional script arguments can be specified using whitespace separator. Example: `script_name arg1 arg2`. If `execute_direct` = `0`, `command` is passed as argument for `bin/sh -c`. Default value is `1`. Optional parameter. -- `send_chunk_header` - controls whether to send row count before sending a chunk of data to process. Optional. Default value is `false`. - -That dictionary source can be configured only via XML configuration. Creating dictionaries with executable source via DDL is disabled, otherwise, the DB user would be able to execute arbitrary binary on ClickHouse node. - -## Http(s) - -Working with an HTTP(s) server depends on [how the dictionary is stored in memory](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request via the `POST` method. - -Example of settings: - -``` xml - - - http://[::1]/os.tsv - TabSeparated - - user - password - - -
- API-KEY - key -
-
-
- -``` - -or - -``` sql -SOURCE(HTTP( - url 'http://[::1]/os.tsv' - format 'TabSeparated' - credentials(user 'user' password 'password') - headers(header(name 'API-KEY' value 'key')) -)) -``` - -In order for ClickHouse to access an HTTPS resource, you must [configure openSSL](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-openssl) in the server configuration. - -Setting fields: - -- `url` – The source URL. -- `format` – The file format. All the formats described in “[Formats](../../../interfaces/formats.md#formats)” are supported. -- `credentials` – Basic HTTP authentication. Optional parameter. -- `user` – Username required for the authentication. -- `password` – Password required for the authentication. -- `headers` – All custom HTTP headers entries used for the HTTP request. Optional parameter. -- `header` – Single HTTP header entry. -- `name` – Identifiant name used for the header send on the request. -- `value` – Value set for a specific identifiant name. - -When creating a dictionary using the DDL command (`CREATE DICTIONARY ...`) remote hosts for HTTP dictionaries are checked against the contents of `remote_url_allow_hosts` section from config to prevent database users to access arbitrary HTTP server. - -### Known Vulnerability of the ODBC Dictionary Functionality - -:::note -When connecting to the database through the ODBC driver connection parameter `Servername` can be substituted. In this case values of `USERNAME` and `PASSWORD` from `odbc.ini` are sent to the remote server and can be compromised. -::: - -**Example of insecure use** - -Let’s configure unixODBC for PostgreSQL. Content of `/etc/odbc.ini`: - -``` text -[gregtest] -Driver = /usr/lib/psqlodbca.so -Servername = localhost -PORT = 5432 -DATABASE = test_db -#OPTION = 3 -USERNAME = test -PASSWORD = test -``` - -If you then make a query such as - -``` sql -SELECT * FROM odbc('DSN=gregtest;Servername=some-server.com', 'test_db'); -``` - -ODBC driver will send values of `USERNAME` and `PASSWORD` from `odbc.ini` to `some-server.com`. - -### Example of Connecting Postgresql - -Ubuntu OS. - -Installing unixODBC and the ODBC driver for PostgreSQL: - -``` bash -$ sudo apt-get install -y unixodbc odbcinst odbc-postgresql -``` - -Configuring `/etc/odbc.ini` (or `~/.odbc.ini` if you signed in under a user that runs ClickHouse): - -``` text - [DEFAULT] - Driver = myconnection - - [myconnection] - Description = PostgreSQL connection to my_db - Driver = PostgreSQL Unicode - Database = my_db - Servername = 127.0.0.1 - UserName = username - Password = password - Port = 5432 - Protocol = 9.3 - ReadOnly = No - RowVersioning = No - ShowSystemTables = No - ConnSettings = -``` - -The dictionary configuration in ClickHouse: - -``` xml - - - table_name - - - - - DSN=myconnection - postgresql_table
-
- - - 300 - 360 - - - - - - - id - - - some_column - UInt64 - 0 - - -
-
-``` - -or - -``` sql -CREATE DICTIONARY table_name ( - id UInt64, - some_column UInt64 DEFAULT 0 -) -PRIMARY KEY id -SOURCE(ODBC(connection_string 'DSN=myconnection' table 'postgresql_table')) -LAYOUT(HASHED()) -LIFETIME(MIN 300 MAX 360) -``` - -You may need to edit `odbc.ini` to specify the full path to the library with the driver `DRIVER=/usr/local/lib/psqlodbcw.so`. - -### Example of Connecting MS SQL Server - -Ubuntu OS. - -Installing the ODBC driver for connecting to MS SQL: - -``` bash -$ sudo apt-get install tdsodbc freetds-bin sqsh -``` - -Configuring the driver: - -```bash - $ cat /etc/freetds/freetds.conf - ... - - [MSSQL] - host = 192.168.56.101 - port = 1433 - tds version = 7.0 - client charset = UTF-8 - - # test TDS connection - $ sqsh -S MSSQL -D database -U user -P password - - - $ cat /etc/odbcinst.ini - - [FreeTDS] - Description = FreeTDS - Driver = /usr/lib/x86_64-linux-gnu/odbc/libtdsodbc.so - Setup = /usr/lib/x86_64-linux-gnu/odbc/libtdsS.so - FileUsage = 1 - UsageCount = 5 - - $ cat /etc/odbc.ini - # $ cat ~/.odbc.ini # if you signed in under a user that runs ClickHouse - - [MSSQL] - Description = FreeTDS - Driver = FreeTDS - Servername = MSSQL - Database = test - UID = test - PWD = test - Port = 1433 - - - # (optional) test ODBC connection (to use isql-tool install the [unixodbc](https://packages.debian.org/sid/unixodbc)-package) - $ isql -v MSSQL "user" "password" -``` - -Remarks: -- to determine the earliest TDS version that is supported by a particular SQL Server version, refer to the product documentation or look at [MS-TDS Product Behavior](https://docs.microsoft.com/en-us/openspecs/windows_protocols/ms-tds/135d0ebe-5c4c-4a94-99bf-1811eccb9f4a) - -Configuring the dictionary in ClickHouse: - -``` xml - - - test - - - dict
- DSN=MSSQL;UID=test;PWD=test -
- - - - 300 - 360 - - - - - - - - - k - - - s - String - - - -
-
-``` - -or - -``` sql -CREATE DICTIONARY test ( - k UInt64, - s String DEFAULT '' -) -PRIMARY KEY k -SOURCE(ODBC(table 'dict' connection_string 'DSN=MSSQL;UID=test;PWD=test')) -LAYOUT(FLAT()) -LIFETIME(MIN 300 MAX 360) -``` - -## DBMS - -### ODBC - -You can use this method to connect any database that has an ODBC driver. - -Example of settings: - -``` xml - - - DatabaseName - ShemaName.TableName
- DSN=some_parameters - SQL_QUERY - SELECT id, value_1, value_2 FROM ShemaName.TableName -
- -``` - -or - -``` sql -SOURCE(ODBC( - db 'DatabaseName' - table 'SchemaName.TableName' - connection_string 'DSN=some_parameters' - invalidate_query 'SQL_QUERY' - query 'SELECT id, value_1, value_2 FROM db_name.table_name' -)) -``` - -Setting fields: - -- `db` – Name of the database. Omit it if the database name is set in the `` parameters. -- `table` – Name of the table and schema if exists. -- `connection_string` – Connection string. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md). -- `query` – The custom query. Optional parameter. - -:::note -The `table` and `query` fields cannot be used together. And either one of the `table` or `query` fields must be declared. -::: - -ClickHouse receives quoting symbols from ODBC-driver and quote all settings in queries to driver, so it’s necessary to set table name accordingly to table name case in database. - -If you have a problems with encodings when using Oracle, see the corresponding [FAQ](../../../faq/integration/oracle-odbc.md) item. - -### Mysql - -Example of settings: - -``` xml - - - 3306 - clickhouse - qwerty - - example01-1 - 1 - - - example01-2 - 1 - - db_name - table_name
- id=10 - SQL_QUERY - true - SELECT id, value_1, value_2 FROM db_name.table_name -
- -``` - -or - -``` sql -SOURCE(MYSQL( - port 3306 - user 'clickhouse' - password 'qwerty' - replica(host 'example01-1' priority 1) - replica(host 'example01-2' priority 1) - db 'db_name' - table 'table_name' - where 'id=10' - invalidate_query 'SQL_QUERY' - fail_on_connection_loss 'true' - query 'SELECT id, value_1, value_2 FROM db_name.table_name' -)) -``` - -Setting fields: - -- `port` – The port on the MySQL server. You can specify it for all replicas, or for each one individually (inside ``). - -- `user` – Name of the MySQL user. You can specify it for all replicas, or for each one individually (inside ``). - -- `password` – Password of the MySQL user. You can specify it for all replicas, or for each one individually (inside ``). - -- `replica` – Section of replica configurations. There can be multiple sections. - - - `replica/host` – The MySQL host. - - `replica/priority` – The replica priority. When attempting to connect, ClickHouse traverses the replicas in order of priority. The lower the number, the higher the priority. - -- `db` – Name of the database. - -- `table` – Name of the table. - -- `where` – The selection criteria. The syntax for conditions is the same as for `WHERE` clause in MySQL, for example, `id > 10 AND id < 20`. Optional parameter. - -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md). - -- `fail_on_connection_loss` – The configuration parameter that controls behavior of the server on connection loss. If `true`, an exception is thrown immediately if the connection between client and server was lost. If `false`, the ClickHouse server retries to execute the query three times before throwing an exception. Note that retrying leads to increased response times. Default value: `false`. - -- `query` – The custom query. Optional parameter. - -:::note -The `table` or `where` fields cannot be used together with the `query` field. And either one of the `table` or `query` fields must be declared. -::: - -:::note -There is no explicit parameter `secure`. When establishing an SSL-connection security is mandatory. -::: - -MySQL can be connected to on a local host via sockets. To do this, set `host` and `socket`. - -Example of settings: - -``` xml - - - localhost - /path/to/socket/file.sock - clickhouse - qwerty - db_name - table_name
- id=10 - SQL_QUERY - true - SELECT id, value_1, value_2 FROM db_name.table_name -
- -``` - -or - -``` sql -SOURCE(MYSQL( - host 'localhost' - socket '/path/to/socket/file.sock' - user 'clickhouse' - password 'qwerty' - db 'db_name' - table 'table_name' - where 'id=10' - invalidate_query 'SQL_QUERY' - fail_on_connection_loss 'true' - query 'SELECT id, value_1, value_2 FROM db_name.table_name' -)) -``` - -### ClickHouse - -Example of settings: - -``` xml - - - example01-01-1 - 9000 - default - - default - ids
- id=10 - 1 - SELECT id, value_1, value_2 FROM default.ids -
- -``` - -or - -``` sql -SOURCE(CLICKHOUSE( - host 'example01-01-1' - port 9000 - user 'default' - password '' - db 'default' - table 'ids' - where 'id=10' - secure 1 - query 'SELECT id, value_1, value_2 FROM default.ids' -)); -``` - -Setting fields: - -- `host` – The ClickHouse host. If it is a local host, the query is processed without any network activity. To improve fault tolerance, you can create a [Distributed](../../../engines/table-engines/special/distributed.md) table and enter it in subsequent configurations. -- `port` – The port on the ClickHouse server. -- `user` – Name of the ClickHouse user. -- `password` – Password of the ClickHouse user. -- `db` – Name of the database. -- `table` – Name of the table. -- `where` – The selection criteria. May be omitted. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md). -- `secure` - Use ssl for connection. -- `query` – The custom query. Optional parameter. - -:::note -The `table` or `where` fields cannot be used together with the `query` field. And either one of the `table` or `query` fields must be declared. -::: - -### Mongodb - -Example of settings: - -``` xml - - - localhost - 27017 - - - test - dictionary_source - - -``` - -or - -``` sql -SOURCE(MONGODB( - host 'localhost' - port 27017 - user '' - password '' - db 'test' - collection 'dictionary_source' -)) -``` - -Setting fields: - -- `host` – The MongoDB host. -- `port` – The port on the MongoDB server. -- `user` – Name of the MongoDB user. -- `password` – Password of the MongoDB user. -- `db` – Name of the database. -- `collection` – Name of the collection. - -### Redis - -Example of settings: - -``` xml - - - localhost - 6379 - simple - 0 - - -``` - -or - -``` sql -SOURCE(REDIS( - host 'localhost' - port 6379 - storage_type 'simple' - db_index 0 -)) -``` - -Setting fields: - -- `host` – The Redis host. -- `port` – The port on the Redis server. -- `storage_type` – The structure of internal Redis storage using for work with keys. `simple` is for simple sources and for hashed single key sources, `hash_map` is for hashed sources with two keys. Ranged sources and cache sources with complex key are unsupported. May be omitted, default value is `simple`. -- `db_index` – The specific numeric index of Redis logical database. May be omitted, default value is 0. - -### Cassandra - -Example of settings: - -``` xml - - - localhost - 9042 - username - qwerty123 - database_name - table_name - 1 - 1 - One - "SomeColumn" = 42 - 8 - SELECT id, value_1, value_2 FROM database_name.table_name - - -``` - -Setting fields: - -- `host` – The Cassandra host or comma-separated list of hosts. -- `port` – The port on the Cassandra servers. If not specified, default port 9042 is used. -- `user` – Name of the Cassandra user. -- `password` – Password of the Cassandra user. -- `keyspace` – Name of the keyspace (database). -- `column_family` – Name of the column family (table). -- `allow_filering` – Flag to allow or not potentially expensive conditions on clustering key columns. Default value is 1. -- `partition_key_prefix` – Number of partition key columns in primary key of the Cassandra table. Required for compose key dictionaries. Order of key columns in the dictionary definition must be the same as in Cassandra. Default value is 1 (the first key column is a partition key and other key columns are clustering key). -- `consistency` – Consistency level. Possible values: `One`, `Two`, `Three`, `All`, `EachQuorum`, `Quorum`, `LocalQuorum`, `LocalOne`, `Serial`, `LocalSerial`. Default value is `One`. -- `where` – Optional selection criteria. -- `max_threads` – The maximum number of threads to use for loading data from multiple partitions in compose key dictionaries. -- `query` – The custom query. Optional parameter. - -:::note -The `column_family` or `where` fields cannot be used together with the `query` field. And either one of the `column_family` or `query` fields must be declared. -::: - -### PostgreSQL - -Example of settings: - -``` xml - - - 5432 - clickhouse - qwerty - db_name - table_name
- id=10 - SQL_QUERY - SELECT id, value_1, value_2 FROM db_name.table_name -
- -``` - -or - -``` sql -SOURCE(POSTGRESQL( - port 5432 - host 'postgresql-hostname' - user 'postgres_user' - password 'postgres_password' - db 'db_name' - table 'table_name' - replica(host 'example01-1' port 5432 priority 1) - replica(host 'example01-2' port 5432 priority 2) - where 'id=10' - invalidate_query 'SQL_QUERY' - query 'SELECT id, value_1, value_2 FROM db_name.table_name' -)) -``` - -Setting fields: - -- `host` – The host on the PostgreSQL server. You can specify it for all replicas, or for each one individually (inside ``). -- `port` – The port on the PostgreSQL server. You can specify it for all replicas, or for each one individually (inside ``). -- `user` – Name of the PostgreSQL user. You can specify it for all replicas, or for each one individually (inside ``). -- `password` – Password of the PostgreSQL user. You can specify it for all replicas, or for each one individually (inside ``). -- `replica` – Section of replica configurations. There can be multiple sections: - - `replica/host` – The PostgreSQL host. - - `replica/port` – The PostgreSQL port. - - `replica/priority` – The replica priority. When attempting to connect, ClickHouse traverses the replicas in order of priority. The lower the number, the higher the priority. -- `db` – Name of the database. -- `table` – Name of the table. -- `where` – The selection criteria. The syntax for conditions is the same as for `WHERE` clause in PostgreSQL. For example, `id > 10 AND id < 20`. Optional parameter. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md). -- `query` – The custom query. Optional parameter. - -:::note -The `table` or `where` fields cannot be used together with the `query` field. And either one of the `table` or `query` fields must be declared. -::: - -## Null - -A special source that can be used to create dummy (empty) dictionaries. Such dictionaries can useful for tests or with setups with separated data and query nodes at nodes with Distributed tables. - -``` sql -CREATE DICTIONARY null_dict ( - id UInt64, - val UInt8, - default_val UInt8 DEFAULT 123, - nullable_val Nullable(UInt8) -) -PRIMARY KEY id -SOURCE(NULL()) -LAYOUT(FLAT()) -LIFETIME(0); -``` - -## Related Content - -- [Using dictionaries to accelerate queries](https://clickhouse.com/blog/faster-queries-dictionaries-clickhouse) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md deleted file mode 100644 index 8271a342941..00000000000 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -slug: /en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure -sidebar_position: 44 -sidebar_label: Dictionary Key and Fields ---- -import CloudDetails from '@site/docs/en/sql-reference/dictionaries/external-dictionaries/_snippet_dictionary_in_cloud.md'; - -# Dictionary Key and Fields - - - -The `structure` clause describes the dictionary key and fields available for queries. - -XML description: - -``` xml - - - - Id - - - - - - - ... - - - -``` - -Attributes are described in the elements: - -- `` — [Key column](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-key). -- `` — [Data column](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-attributes). There can be a multiple number of attributes. - -DDL query: - -``` sql -CREATE DICTIONARY dict_name ( - Id UInt64, - -- attributes -) -PRIMARY KEY Id -... -``` - -Attributes are described in the query body: - -- `PRIMARY KEY` — [Key column](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-key) -- `AttrName AttrType` — [Data column](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-attributes). There can be a multiple number of attributes. - -## Key - -ClickHouse supports the following types of keys: - -- Numeric key. `UInt64`. Defined in the `` tag or using `PRIMARY KEY` keyword. -- Composite key. Set of values of different types. Defined in the tag `` or `PRIMARY KEY` keyword. - -An xml structure can contain either `` or ``. DDL-query must contain single `PRIMARY KEY`. - -:::warning -You must not describe key as an attribute. -::: - -### Numeric Key - -Type: `UInt64`. - -Configuration example: - -``` xml - - Id - -``` - -Configuration fields: - -- `name` – The name of the column with keys. - -For DDL-query: - -``` sql -CREATE DICTIONARY ( - Id UInt64, - ... -) -PRIMARY KEY Id -... -``` - -- `PRIMARY KEY` – The name of the column with keys. - -### Composite Key - -The key can be a `tuple` from any types of fields. The [layout](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md) in this case must be `complex_key_hashed` or `complex_key_cache`. - -:::tip -A composite key can consist of a single element. This makes it possible to use a string as the key, for instance. -::: - -The key structure is set in the element ``. Key fields are specified in the same format as the dictionary [attributes](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Example: - -``` xml - - - - field1 - String - - - field2 - UInt32 - - ... - -... -``` - -or - -``` sql -CREATE DICTIONARY ( - field1 String, - field2 String - ... -) -PRIMARY KEY field1, field2 -... -``` - -For a query to the `dictGet*` function, a tuple is passed as the key. Example: `dictGetString('dict_name', 'attr_name', tuple('string for field1', num_for_field2))`. - -## Attributes - -Configuration example: - -``` xml - - ... - - Name - ClickHouseDataType - - rand64() - true - true - true - - -``` - -or - -``` sql -CREATE DICTIONARY somename ( - Name ClickHouseDataType DEFAULT '' EXPRESSION rand64() HIERARCHICAL INJECTIVE IS_OBJECT_ID -) -``` - -Configuration fields: - -| Tag | Description | Required | -|------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------| -| `name` | Column name. | Yes | -| `type` | ClickHouse data type: [UInt8](../../../sql-reference/data-types/int-uint.md), [UInt16](../../../sql-reference/data-types/int-uint.md), [UInt32](../../../sql-reference/data-types/int-uint.md), [UInt64](../../../sql-reference/data-types/int-uint.md), [Int8](../../../sql-reference/data-types/int-uint.md), [Int16](../../../sql-reference/data-types/int-uint.md), [Int32](../../../sql-reference/data-types/int-uint.md), [Int64](../../../sql-reference/data-types/int-uint.md), [Float32](../../../sql-reference/data-types/float.md), [Float64](../../../sql-reference/data-types/float.md), [UUID](../../../sql-reference/data-types/uuid.md), [Decimal32](../../../sql-reference/data-types/decimal.md), [Decimal64](../../../sql-reference/data-types/decimal.md), [Decimal128](../../../sql-reference/data-types/decimal.md), [Decimal256](../../../sql-reference/data-types/decimal.md),[Date](../../../sql-reference/data-types/date), [Date32](../../../sql-reference/data-types/date32.md), [DateTime](../../../sql-reference/data-types/datetime.md), [DateTime64](../../../sql-reference/data-types/datetime64.md), [String](../../../sql-reference/data-types/string.md), [Array](../../../sql-reference/data-types/array.md).
ClickHouse tries to cast value from dictionary to the specified data type. For example, for MySQL, the field might be `TEXT`, `VARCHAR`, or `BLOB` in the MySQL source table, but it can be uploaded as `String` in ClickHouse.
[Nullable](../../../sql-reference/data-types/nullable.md) is currently supported for [Flat](external-dicts-dict-layout.md#flat), [Hashed](external-dicts-dict-layout.md#dicts-external_dicts_dict_layout-hashed), [ComplexKeyHashed](external-dicts-dict-layout.md#complex-key-hashed), [Direct](external-dicts-dict-layout.md#direct), [ComplexKeyDirect](external-dicts-dict-layout.md#complex-key-direct), [RangeHashed](external-dicts-dict-layout.md#range-hashed), [Polygon](external-dicts-dict-polygon.md), [Cache](external-dicts-dict-layout.md#cache), [ComplexKeyCache](external-dicts-dict-layout.md#complex-key-cache), [SSDCache](external-dicts-dict-layout.md#ssd-cache), [SSDComplexKeyCache](external-dicts-dict-layout.md#complex-key-ssd-cache) dictionaries. In [IPTrie](external-dicts-dict-layout.md#ip-trie) dictionaries `Nullable` types are not supported. | Yes | -| `null_value` | Default value for a non-existing element.
In the example, it is an empty string. [NULL](../../syntax.md#null-literal) value can be used only for the `Nullable` types (see the previous line with types description). | Yes | -| `expression` | [Expression](../../../sql-reference/syntax.md#syntax-expressions) that ClickHouse executes on the value.
The expression can be a column name in the remote SQL database. Thus, you can use it to create an alias for the remote column.

Default value: no expression. | No | -| `hierarchical` | If `true`, the attribute contains the value of a parent key for the current key. See [Hierarchical Dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md).

Default value: `false`. | No | -| `injective` | Flag that shows whether the `id -> attribute` image is [injective](https://en.wikipedia.org/wiki/Injective_function).
If `true`, ClickHouse can automatically place after the `GROUP BY` clause the requests to dictionaries with injection. Usually it significantly reduces the amount of such requests.

Default value: `false`. | No | -| `is_object_id` | Flag that shows whether the query is executed for a MongoDB document by `ObjectID`.

Default value: `false`. | No | - -**See Also** - -- [Functions for working with dictionaries](../../../sql-reference/functions/ext-dict-functions.md). - -## Related Content - -- [Using dictionaries to accelerate queries](https://clickhouse.com/blog/faster-queries-dictionaries-clickhouse) \ No newline at end of file diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md deleted file mode 100644 index a923511ca5e..00000000000 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md +++ /dev/null @@ -1,57 +0,0 @@ ---- -slug: /en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict -sidebar_position: 40 -sidebar_label: Configuring a Dictionary ---- -import CloudDetails from '@site/docs/en/sql-reference/dictionaries/external-dictionaries/_snippet_dictionary_in_cloud.md'; - -# Configuring a Dictionary - - - -If dictionary is configured using xml file, than dictionary configuration has the following structure: - -``` xml - - dict_name - - - - - - - - - - - - - - - - - -``` - -Corresponding [DDL-query](../../../sql-reference/statements/create/dictionary.md) has the following structure: - -``` sql -CREATE DICTIONARY dict_name -( - ... -- attributes -) -PRIMARY KEY ... -- complex or single key configuration -SOURCE(...) -- Source configuration -LAYOUT(...) -- Memory layout configuration -LIFETIME(...) -- Lifetime of dictionary in memory -``` - -- `name` – The identifier that can be used to access the dictionary. Use the characters `[a-zA-Z0-9_\-]`. -- [source](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md) — Source of the dictionary. -- [layout](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md) — Dictionary layout in memory. -- [structure](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md) — Structure of the dictionary . A key and attributes that can be retrieved by this key. -- [lifetime](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) — Frequency of dictionary updates. - -## Related Content - -- [Using dictionaries to accelerate queries](https://clickhouse.com/blog/faster-queries-dictionaries-clickhouse) \ No newline at end of file diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md deleted file mode 100644 index 8621c68b428..00000000000 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ /dev/null @@ -1,84 +0,0 @@ ---- -slug: /en/sql-reference/dictionaries/external-dictionaries/external-dicts -sidebar_position: 39 -sidebar_label: General Description ---- -import CloudDetails from '@site/docs/en/sql-reference/dictionaries/external-dictionaries/_snippet_dictionary_in_cloud.md'; - -# Dictionaries - -:::tip Tutorial -If you are getting started with Dictionaries in ClickHouse we have a tutorial that covers that topic. Take a look [here](/docs/en/tutorial.md). -::: - -You can add your own dictionaries from various data sources. The source for a dictionary can be a ClickHouse table, a local text or executable file, an HTTP(s) resource, or another DBMS. For more information, see “[Dictionary Sources](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md)”. - -ClickHouse: - -- Fully or partially stores dictionaries in RAM. -- Periodically updates dictionaries and dynamically loads missing values. In other words, dictionaries can be loaded dynamically. -- Allows creating dictionaries with xml files or [DDL queries](../../../sql-reference/statements/create/dictionary.md). - -The configuration of dictionaries can be located in one or more xml-files. The path to the configuration is specified in the [dictionaries_config](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config) parameter. - -Dictionaries can be loaded at server startup or at first use, depending on the [dictionaries_lazy_load](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load) setting. - -The [dictionaries](../../../operations/system-tables/dictionaries.md#system_tables-dictionaries) system table contains information about dictionaries configured at server. For each dictionary you can find there: - -- Status of the dictionary. -- Configuration parameters. -- Metrics like amount of RAM allocated for the dictionary or a number of queries since the dictionary was successfully loaded. - - - -## Creating a dictionary with a DDL query - -Dictionaries can be created with [DDL queries](../../../sql-reference/statements/create/dictionary.md), and this is the recommended method because with DDL created dictionaries: -- No additional records are added to server configuration files -- The dictionaries can be worked with as first-class entities, like tables or views -- Data can be read directly, using familiar SELECT rather than dictionary table functions -- The dictionaries can be easily renamed - -## Creating a dictionary with a configuration file - -:::note -Creating a dictionary with a configuration file is not applicable to ClickHouse Cloud. Please use DDL (see above), and create your dictionary as user `default`. -::: - -The dictionary configuration file has the following format: - -``` xml - - An optional element with any content. Ignored by the ClickHouse server. - - - /etc/metrika.xml - - - - - - - - -``` - -You can [configure](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md) any number of dictionaries in the same file. - - -:::note -You can convert values for a small dictionary by describing it in a `SELECT` query (see the [transform](../../../sql-reference/functions/other-functions.md) function). This functionality is not related to dictionaries. -::: - -## See Also - -- [Configuring a Dictionary](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md) -- [Storing Dictionaries in Memory](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md) -- [Dictionary Updates](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) -- [Dictionary Sources](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md) -- [Dictionary Key and Fields](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md) -- [Functions for Working with Dictionaries](../../../sql-reference/functions/ext-dict-functions.md) - -## Related Content - -- [Using dictionaries to accelerate queries](https://clickhouse.com/blog/faster-queries-dictionaries-clickhouse) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/regexp-tree.md b/docs/en/sql-reference/dictionaries/external-dictionaries/regexp-tree.md deleted file mode 100644 index 5ad15b11d07..00000000000 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/regexp-tree.md +++ /dev/null @@ -1,76 +0,0 @@ ---- -slug: /en/sql-reference/dictionaries/external-dictionaries/regexp-tree -sidebar_position: 47 -sidebar_label: RegExp Tree Dictionary -title: "RegExp Tree Dictionary" ---- -import CloudDetails from '@site/docs/en/sql-reference/dictionaries/external-dictionaries/_snippet_dictionary_in_cloud.md'; - -Regexp Tree dictionary stores multiple trees of regular expressions with attributions. Users can retrieve strings in the dictionary. If a string matches the root of the regexp tree, we will collect the corresponding attributes of the matched root and continue to walk the children. If any of the children matches the string, we will collect attributes and rewrite the old ones if conflicts occur, then continue the traverse until we reach leaf nodes. - -Example of the ddl query for creating Regexp Tree dictionary: - - - -```sql -create dictionary regexp_dict -( - regexp String, - name String, - version String -) -PRIMARY KEY(regexp) -SOURCE(YAMLRegExpTree(PATH '/var/lib/clickhouse/user_files/regexp_tree.yaml')) -LAYOUT(regexp_tree) -... -``` - -We only allow `YAMLRegExpTree` to work with regexp_tree dicitionary layout. If you want to use other sources, please set variable `regexp_dict_allow_other_sources` true. - -**Source** - -We introduce a type of source called `YAMLRegExpTree` representing the structure of Regexp Tree dictionary. An Example of a valid yaml config is like: - -```xml -- regexp: 'Linux/(\d+[\.\d]*).+tlinux' - name: 'TencentOS' - version: '\1' - -- regexp: '\d+/tclwebkit(?:\d+[\.\d]*)' - name: 'Andriod' - versions: - - regexp: '33/tclwebkit' - version: '13' - - regexp: '3[12]/tclwebkit' - version: '12' - - regexp: '30/tclwebkit' - version: '11' - - regexp: '29/tclwebkit' - version: '10' -``` - -The key `regexp` represents the regular expression of a tree node. The name of key is same as the dictionary key. The `name` and `version` is user-defined attributions in the dicitionary. The `versions` (which can be any name that not appear in attributions or the key) indicates the children nodes of this tree. - -**Back Reference** - -The value of an attribution could contain a back reference which refers to a capture group of the matched regular expression. Reference number ranges from 1 to 9 and writes as `$1` or `\1`. - -During the query execution, the back reference in the value will be replaced by the matched capture group. - -**Query** - -Due to the specialty of Regexp Tree dictionary, we only allow functions `dictGet`, `dictGetOrDefault` and `dictGetOrNull` work with it. - -Example: - -```sql -SELECT dictGet('regexp_dict', ('name', 'version'), '31/tclwebkit1024'); -``` - -Result: - -``` -┌─dictGet('regexp_dict', ('name', 'version'), '31/tclwebkit1024')─┐ -│ ('Andriod','12') │ -└─────────────────────────────────────────────────────────────────┘ -``` diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 9e6eed47d4a..2185e2b31c1 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -1,9 +1,12 @@ --- -slug: /en/sql-reference/dictionaries/ -sidebar_label: Dictionaries +slug: /en/sql-reference/dictionaries +sidebar_label: Defining Dictionaries sidebar_position: 35 --- +import SelfManaged from '@site/docs/en/_snippets/_self_managed_only_no_roadmap.md'; +import CloudDetails from '@site/docs/en/sql-reference/dictionaries/_snippet_dictionary_in_cloud.md'; + # Dictionaries A dictionary is a mapping (`key -> attributes`) that is convenient for various types of reference lists. @@ -12,5 +15,2349 @@ ClickHouse supports special functions for working with dictionaries that can be ClickHouse supports: -- [Dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md#dicts-external-dicts) with a [set of functions](../../sql-reference/functions/ext-dict-functions.md). -- [Embedded dictionaries](../../sql-reference/dictionaries/internal-dicts.md#internal_dicts) with a specific [set of functions](../../sql-reference/functions/ym-dict-functions.md). +- Dictionaries with a [set of functions](../../sql-reference/functions/ext-dict-functions.md). +- [Embedded dictionaries](#embedded_dictionaries) with a specific [set of functions](../../sql-reference/functions/ym-dict-functions.md). + + +:::tip Tutorial +If you are getting started with Dictionaries in ClickHouse we have a tutorial that covers that topic. Take a look [here](/docs/en/tutorial.md). +::: + +You can add your own dictionaries from various data sources. The source for a dictionary can be a ClickHouse table, a local text or executable file, an HTTP(s) resource, or another DBMS. For more information, see “[Dictionary Sources](#dictionary-sources)”. + +ClickHouse: + +- Fully or partially stores dictionaries in RAM. +- Periodically updates dictionaries and dynamically loads missing values. In other words, dictionaries can be loaded dynamically. +- Allows creating dictionaries with xml files or [DDL queries](../../sql-reference/statements/create/dictionary.md). + +The configuration of dictionaries can be located in one or more xml-files. The path to the configuration is specified in the [dictionaries_config](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config) parameter. + +Dictionaries can be loaded at server startup or at first use, depending on the [dictionaries_lazy_load](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load) setting. + +The [dictionaries](../../operations/system-tables/dictionaries.md#system_tables-dictionaries) system table contains information about dictionaries configured at server. For each dictionary you can find there: + +- Status of the dictionary. +- Configuration parameters. +- Metrics like amount of RAM allocated for the dictionary or a number of queries since the dictionary was successfully loaded. + + + +## Creating a dictionary with a DDL query {#creating-a-dictionary-with-a-ddl-query} + +Dictionaries can be created with [DDL queries](../../sql-reference/statements/create/dictionary.md), and this is the recommended method because with DDL created dictionaries: +- No additional records are added to server configuration files +- The dictionaries can be worked with as first-class entities, like tables or views +- Data can be read directly, using familiar SELECT rather than dictionary table functions +- The dictionaries can be easily renamed + +## Creating a dictionary with a configuration file + +:::note +Creating a dictionary with a configuration file is not applicable to ClickHouse Cloud. Please use DDL (see above), and create your dictionary as user `default`. +::: + +The dictionary configuration file has the following format: + +``` xml + + An optional element with any content. Ignored by the ClickHouse server. + + + /etc/metrika.xml + + + + + + + + +``` + +You can [configure](#configuring-a-dictionary) any number of dictionaries in the same file. + + +:::note +You can convert values for a small dictionary by describing it in a `SELECT` query (see the [transform](../../sql-reference/functions/other-functions.md) function). This functionality is not related to dictionaries. +::: + +## Configuring a Dictionary {#configuring-a-dictionary} + + + +If dictionary is configured using xml file, than dictionary configuration has the following structure: + +``` xml + + dict_name + + + + + + + + + + + + + + + + + +``` + +Corresponding [DDL-query](../../sql-reference/statements/create/dictionary.md) has the following structure: + +``` sql +CREATE DICTIONARY dict_name +( + ... -- attributes +) +PRIMARY KEY ... -- complex or single key configuration +SOURCE(...) -- Source configuration +LAYOUT(...) -- Memory layout configuration +LIFETIME(...) -- Lifetime of dictionary in memory +``` + +## Storing Dictionaries in Memory {#storig-dictionaries-in-memory} + +There are a variety of ways to store dictionaries in memory. + +We recommend [flat](#flat), [hashed](#hashed) and [complex_key_hashed](#complex_key_hashed), which provide optimal processing speed. + +Caching is not recommended because of potentially poor performance and difficulties in selecting optimal parameters. Read more in the section [cache](#cache). + +There are several ways to improve dictionary performance: + +- Call the function for working with the dictionary after `GROUP BY`. +- Mark attributes to extract as injective. An attribute is called injective if different attribute values correspond to different keys. So when `GROUP BY` uses a function that fetches an attribute value by the key, this function is automatically taken out of `GROUP BY`. + +ClickHouse generates an exception for errors with dictionaries. Examples of errors: + +- The dictionary being accessed could not be loaded. +- Error querying a `cached` dictionary. + +You can view the list of dictionaries and their statuses in the [system.dictionaries](../../operations/system-tables/dictionaries.md) table. + + + +The configuration looks like this: + +``` xml + + + ... + + + + + + ... + + +``` + +Corresponding [DDL-query](../../sql-reference/statements/create/dictionary.md): + +``` sql +CREATE DICTIONARY (...) +... +LAYOUT(LAYOUT_TYPE(param value)) -- layout settings +... +``` + +Dictionaries without word `complex-key*` in a layout have a key with [UInt64](../../sql-reference/data-types/int-uint.md) type, `complex-key*` dictionaries have a composite key (complex, with arbitrary types). + +[UInt64](../../sql-reference/data-types/int-uint.md) keys in XML dictionaries are defined with `` tag. + +Configuration example (column key_column has UInt64 type): +```xml +... + + + key_column + +... +``` + +Composite `complex` keys XML dictionaries are defined `` tag. + +Configuration example of a composite key (key has one element with [String](../../sql-reference/data-types/string.md) type): +```xml +... + + + + country_code + String + + +... +``` + +## Ways to Store Dictionaries in Memory + +- [flat](#flat) +- [hashed](#hashed) +- [sparse_hashed](#sparse_hashed) +- [complex_key_hashed](#complex_key_hashed) +- [complex_key_sparse_hashed](#complex_key_sparse_hashed) +- [hashed_array](#hashed_array) +- [complex_key_hashed_array](#complex_key_hashed_array) +- [range_hashed](#range_hashed) +- [complex_key_range_hashed](#complex_key_range_hashed) +- [cache](#cache) +- [complex_key_cache](#complex_key_cache) +- [ssd_cache](#ssd_cache) +- [complex_key_ssd_cache](#complex_key_ssd_cache) +- [direct](#direct) +- [complex_key_direct](#complex_key_direct) +- [ip_trie](#ip_trie) + +### flat + +The dictionary is completely stored in memory in the form of flat arrays. How much memory does the dictionary use? The amount is proportional to the size of the largest key (in space used). + +The dictionary key has the [UInt64](../../sql-reference/data-types/int-uint.md) type and the value is limited to `max_array_size` (by default — 500,000). If a larger key is discovered when creating the dictionary, ClickHouse throws an exception and does not create the dictionary. Dictionary flat arrays initial size is controlled by `initial_array_size` setting (by default — 1024). + +All types of sources are supported. When updating, data (from a file or from a table) is read in it entirety. + +This method provides the best performance among all available methods of storing the dictionary. + +Configuration example: + +``` xml + + + 50000 + 5000000 + + +``` + +or + +``` sql +LAYOUT(FLAT(INITIAL_ARRAY_SIZE 50000 MAX_ARRAY_SIZE 5000000)) +``` + +### hashed + +The dictionary is completely stored in memory in the form of a hash table. The dictionary can contain any number of elements with any identifiers In practice, the number of keys can reach tens of millions of items. + +The dictionary key has the [UInt64](../../sql-reference/data-types/int-uint.md) type. + +All types of sources are supported. When updating, data (from a file or from a table) is read in its entirety. + +Configuration example: + +``` xml + + + +``` + +or + +``` sql +LAYOUT(HASHED()) +``` + +If `shards` greater then 1 (default is `1`) the dictionary will load data in parallel, useful if you have huge amount of elements in one dictionary. + +Configuration example: + +``` xml + + + 10 + + 10000 + + +``` + +or + +``` sql +LAYOUT(HASHED(SHARDS 10 [SHARD_LOAD_QUEUE_BACKLOG 10000])) +``` + +### sparse_hashed + +Similar to `hashed`, but uses less memory in favor more CPU usage. + +The dictionary key has the [UInt64](../../sql-reference/data-types/int-uint.md) type. + +Configuration example: + +``` xml + + + +``` + +or + +``` sql +LAYOUT(SPARSE_HASHED()) +``` + +It is also possible to use `shards` for this type of dictionary, and again it is more important for `sparse_hashed` then for `hashed`, since `sparse_hashed` is slower. + +### complex_key_hashed + +This type of storage is for use with composite [keys](#dictionary-key-and-fields). Similar to `hashed`. + +Configuration example: + +``` xml + + + 1 + + + +``` + +or + +``` sql +LAYOUT(COMPLEX_KEY_HASHED([SHARDS 1] [SHARD_LOAD_QUEUE_BACKLOG 10000])) +``` + +### complex_key_sparse_hashed + +This type of storage is for use with composite [keys](#dictionary-key-and-fields). Similar to [sparse_hashed](#sparse_hashed). + +Configuration example: + +``` xml + + + 1 + + +``` + +or + +``` sql +LAYOUT(COMPLEX_KEY_SPARSE_HASHED([SHARDS 1] [SHARD_LOAD_QUEUE_BACKLOG 10000])) +``` + +### hashed_array + +The dictionary is completely stored in memory. Each attribute is stored in an array. The key attribute is stored in the form of a hashed table where value is an index in the attributes array. The dictionary can contain any number of elements with any identifiers. In practice, the number of keys can reach tens of millions of items. + +The dictionary key has the [UInt64](../../sql-reference/data-types/int-uint.md) type. + +All types of sources are supported. When updating, data (from a file or from a table) is read in its entirety. + +Configuration example: + +``` xml + + + + +``` + +or + +``` sql +LAYOUT(HASHED_ARRAY()) +``` + +### complex_key_hashed_array + +This type of storage is for use with composite [keys](#dictionary-key-and-fields). Similar to [hashed_array](#hashed_array). + +Configuration example: + +``` xml + + + +``` + +or + +``` sql +LAYOUT(COMPLEX_KEY_HASHED_ARRAY()) +``` + +### range_hashed {#range_hashed} + +The dictionary is stored in memory in the form of a hash table with an ordered array of ranges and their corresponding values. + +The dictionary key has the [UInt64](../../sql-reference/data-types/int-uint.md) type. +This storage method works the same way as hashed and allows using date/time (arbitrary numeric type) ranges in addition to the key. + +Example: The table contains discounts for each advertiser in the format: + +``` text +┌─advertiser_id─┬─discount_start_date─┬─discount_end_date─┬─amount─┐ +│ 123 │ 2015-01-16 │ 2015-01-31 │ 0.25 │ +│ 123 │ 2015-01-01 │ 2015-01-15 │ 0.15 │ +│ 456 │ 2015-01-01 │ 2015-01-15 │ 0.05 │ +└───────────────┴─────────────────────┴───────────────────┴────────┘ +``` + +To use a sample for date ranges, define the `range_min` and `range_max` elements in the [structure](#dictionary-key-and-fields). These elements must contain elements `name` and `type` (if `type` is not specified, the default type will be used - Date). `type` can be any numeric type (Date / DateTime / UInt64 / Int32 / others). + +:::warning +Values of `range_min` and `range_max` should fit in `Int64` type. +::: + +Example: + +``` xml + + + + min + + + + + advertiser_id + + + discount_start_date + Date + + + discount_end_date + Date + + ... +``` + +or + +``` sql +CREATE DICTIONARY discounts_dict ( + advertiser_id UInt64, + discount_start_date Date, + discount_end_date Date, + amount Float64 +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(TABLE 'discounts')) +LIFETIME(MIN 1 MAX 1000) +LAYOUT(RANGE_HASHED(range_lookup_strategy 'max')) +RANGE(MIN discount_start_date MAX discount_end_date) +``` + +To work with these dictionaries, you need to pass an additional argument to the `dictGet` function, for which a range is selected: + +``` sql +dictGet('dict_name', 'attr_name', id, date) +``` +Query example: + +``` sql +SELECT dictGet('discounts_dict', 'amount', 1, '2022-10-20'::Date); +``` + +This function returns the value for the specified `id`s and the date range that includes the passed date. + +Details of the algorithm: + +- If the `id` is not found or a range is not found for the `id`, it returns the default value of the attribute's type. +- If there are overlapping ranges and `range_lookup_strategy=min`, it returns a matching range with minimal `range_min`, if several ranges found, it returns a range with minimal `range_max`, if again several ranges found (several ranges had the same `range_min` and `range_max` it returns a random range of them. +- If there are overlapping ranges and `range_lookup_strategy=max`, it returns a matching range with maximal `range_min`, if several ranges found, it returns a range with maximal `range_max`, if again several ranges found (several ranges had the same `range_min` and `range_max` it returns a random range of them. +- If the `range_max` is `NULL`, the range is open. `NULL` is treated as maximal possible value. For the `range_min` `1970-01-01` or `0` (-MAX_INT) can be used as the open value. + +Configuration example: + +``` xml + + + ... + + + + + + + + Abcdef + + + StartTimeStamp + UInt64 + + + EndTimeStamp + UInt64 + + + XXXType + String + + + + + + +``` + +or + +``` sql +CREATE DICTIONARY somedict( + Abcdef UInt64, + StartTimeStamp UInt64, + EndTimeStamp UInt64, + XXXType String DEFAULT '' +) +PRIMARY KEY Abcdef +RANGE(MIN StartTimeStamp MAX EndTimeStamp) +``` + +Configuration example with overlapping ranges and open ranges: + +```sql +CREATE TABLE discounts +( + advertiser_id UInt64, + discount_start_date Date, + discount_end_date Nullable(Date), + amount Float64 +) +ENGINE = Memory; + +INSERT INTO discounts VALUES (1, '2015-01-01', Null, 0.1); +INSERT INTO discounts VALUES (1, '2015-01-15', Null, 0.2); +INSERT INTO discounts VALUES (2, '2015-01-01', '2015-01-15', 0.3); +INSERT INTO discounts VALUES (2, '2015-01-04', '2015-01-10', 0.4); +INSERT INTO discounts VALUES (3, '1970-01-01', '2015-01-15', 0.5); +INSERT INTO discounts VALUES (3, '1970-01-01', '2015-01-10', 0.6); + +SELECT * FROM discounts ORDER BY advertiser_id, discount_start_date; +┌─advertiser_id─┬─discount_start_date─┬─discount_end_date─┬─amount─┐ +│ 1 │ 2015-01-01 │ ᴺᵁᴸᴸ │ 0.1 │ +│ 1 │ 2015-01-15 │ ᴺᵁᴸᴸ │ 0.2 │ +│ 2 │ 2015-01-01 │ 2015-01-15 │ 0.3 │ +│ 2 │ 2015-01-04 │ 2015-01-10 │ 0.4 │ +│ 3 │ 1970-01-01 │ 2015-01-15 │ 0.5 │ +│ 3 │ 1970-01-01 │ 2015-01-10 │ 0.6 │ +└───────────────┴─────────────────────┴───────────────────┴────────┘ + +-- RANGE_LOOKUP_STRATEGY 'max' + +CREATE DICTIONARY discounts_dict +( + advertiser_id UInt64, + discount_start_date Date, + discount_end_date Nullable(Date), + amount Float64 +) +PRIMARY KEY advertiser_id +SOURCE(CLICKHOUSE(TABLE discounts)) +LIFETIME(MIN 600 MAX 900) +LAYOUT(RANGE_HASHED(RANGE_LOOKUP_STRATEGY 'max')) +RANGE(MIN discount_start_date MAX discount_end_date); + +select dictGet('discounts_dict', 'amount', 1, toDate('2015-01-14')) res; +┌─res─┐ +│ 0.1 │ -- the only one range is matching: 2015-01-01 - Null +└─────┘ + +select dictGet('discounts_dict', 'amount', 1, toDate('2015-01-16')) res; +┌─res─┐ +│ 0.2 │ -- two ranges are matching, range_min 2015-01-15 (0.2) is bigger than 2015-01-01 (0.1) +└─────┘ + +select dictGet('discounts_dict', 'amount', 2, toDate('2015-01-06')) res; +┌─res─┐ +│ 0.4 │ -- two ranges are matching, range_min 2015-01-04 (0.4) is bigger than 2015-01-01 (0.3) +└─────┘ + +select dictGet('discounts_dict', 'amount', 3, toDate('2015-01-01')) res; +┌─res─┐ +│ 0.5 │ -- two ranges are matching, range_min are equal, 2015-01-15 (0.5) is bigger than 2015-01-10 (0.6) +└─────┘ + +DROP DICTIONARY discounts_dict; + +-- RANGE_LOOKUP_STRATEGY 'min' + +CREATE DICTIONARY discounts_dict +( + advertiser_id UInt64, + discount_start_date Date, + discount_end_date Nullable(Date), + amount Float64 +) +PRIMARY KEY advertiser_id +SOURCE(CLICKHOUSE(TABLE discounts)) +LIFETIME(MIN 600 MAX 900) +LAYOUT(RANGE_HASHED(RANGE_LOOKUP_STRATEGY 'min')) +RANGE(MIN discount_start_date MAX discount_end_date); + +select dictGet('discounts_dict', 'amount', 1, toDate('2015-01-14')) res; +┌─res─┐ +│ 0.1 │ -- the only one range is matching: 2015-01-01 - Null +└─────┘ + +select dictGet('discounts_dict', 'amount', 1, toDate('2015-01-16')) res; +┌─res─┐ +│ 0.1 │ -- two ranges are matching, range_min 2015-01-01 (0.1) is less than 2015-01-15 (0.2) +└─────┘ + +select dictGet('discounts_dict', 'amount', 2, toDate('2015-01-06')) res; +┌─res─┐ +│ 0.3 │ -- two ranges are matching, range_min 2015-01-01 (0.3) is less than 2015-01-04 (0.4) +└─────┘ + +select dictGet('discounts_dict', 'amount', 3, toDate('2015-01-01')) res; +┌─res─┐ +│ 0.6 │ -- two ranges are matching, range_min are equal, 2015-01-10 (0.6) is less than 2015-01-15 (0.5) +└─────┘ +``` + +### complex_key_range_hashed + +The dictionary is stored in memory in the form of a hash table with an ordered array of ranges and their corresponding values (see [range_hashed](#range_hashed)). This type of storage is for use with composite [keys](#dictionary-key-and-fields). + +Configuration example: + +``` sql +CREATE DICTIONARY range_dictionary +( + CountryID UInt64, + CountryKey String, + StartDate Date, + EndDate Date, + Tax Float64 DEFAULT 0.2 +) +PRIMARY KEY CountryID, CountryKey +SOURCE(CLICKHOUSE(TABLE 'date_table')) +LIFETIME(MIN 1 MAX 1000) +LAYOUT(COMPLEX_KEY_RANGE_HASHED()) +RANGE(MIN StartDate MAX EndDate); +``` + +### cache + +The dictionary is stored in a cache that has a fixed number of cells. These cells contain frequently used elements. + +The dictionary key has the [UInt64](../../sql-reference/data-types/int-uint.md) type. + +When searching for a dictionary, the cache is searched first. For each block of data, all keys that are not found in the cache or are outdated are requested from the source using `SELECT attrs... FROM db.table WHERE id IN (k1, k2, ...)`. The received data is then written to the cache. + +If keys are not found in dictionary, then update cache task is created and added into update queue. Update queue properties can be controlled with settings `max_update_queue_size`, `update_queue_push_timeout_milliseconds`, `query_wait_timeout_milliseconds`, `max_threads_for_updates`. + +For cache dictionaries, the expiration [lifetime](#dictionary-updates) of data in the cache can be set. If more time than `lifetime` has passed since loading the data in a cell, the cell’s value is not used and key becomes expired. The key is re-requested the next time it needs to be used. This behaviour can be configured with setting `allow_read_expired_keys`. + +This is the least effective of all the ways to store dictionaries. The speed of the cache depends strongly on correct settings and the usage scenario. A cache type dictionary performs well only when the hit rates are high enough (recommended 99% and higher). You can view the average hit rate in the [system.dictionaries](../../operations/system-tables/dictionaries.md) table. + +If setting `allow_read_expired_keys` is set to 1, by default 0. Then dictionary can support asynchronous updates. If a client requests keys and all of them are in cache, but some of them are expired, then dictionary will return expired keys for a client and request them asynchronously from the source. + +To improve cache performance, use a subquery with `LIMIT`, and call the function with the dictionary externally. + +All types of sources are supported. + +Example of settings: + +``` xml + + + + 1000000000 + + 0 + + 100000 + + 10 + + 60000 + + 4 + + +``` + +or + +``` sql +LAYOUT(CACHE(SIZE_IN_CELLS 1000000000)) +``` + +Set a large enough cache size. You need to experiment to select the number of cells: + +1. Set some value. +2. Run queries until the cache is completely full. +3. Assess memory consumption using the `system.dictionaries` table. +4. Increase or decrease the number of cells until the required memory consumption is reached. + +:::warning +Do not use ClickHouse as a source, because it is slow to process queries with random reads. +::: + +### complex_key_cache + +This type of storage is for use with composite [keys](#dictionary-key-and-fields). Similar to `cache`. + +### ssd_cache + +Similar to `cache`, but stores data on SSD and index in RAM. All cache dictionary settings related to update queue can also be applied to SSD cache dictionaries. + +The dictionary key has the [UInt64](../../sql-reference/data-types/int-uint.md) type. + +``` xml + + + + 4096 + + 16777216 + + 131072 + + 1048576 + + /var/lib/clickhouse/user_files/test_dict + + +``` + +or + +``` sql +LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 16777216 READ_BUFFER_SIZE 1048576 + PATH '/var/lib/clickhouse/user_files/test_dict')) +``` + +### complex_key_ssd_cache + +This type of storage is for use with composite [keys](#dictionary-key-and-fields). Similar to `ssd_cache`. + +### direct + +The dictionary is not stored in memory and directly goes to the source during the processing of a request. + +The dictionary key has the [UInt64](../../sql-reference/data-types/int-uint.md) type. + +All types of [sources](#dictionary-sources), except local files, are supported. + +Configuration example: + +``` xml + + + +``` + +or + +``` sql +LAYOUT(DIRECT()) +``` + +### complex_key_direct + +This type of storage is for use with composite [keys](#dictionary-key-and-fields). Similar to `direct`. + +### ip_trie + +This type of storage is for mapping network prefixes (IP addresses) to metadata such as ASN. + +**Example** + +Suppose we have a table in ClickHouse that contains our IP prefixes and mappings: + +```sql +CREATE TABLE my_ip_addresses ( + prefix String, + asn UInt32, + cca2 String +) +ENGINE = MergeTree +PRIMARY KEY prefix; +``` + +```sql +INSERT INTO my_ip_addresses VALUES + ('202.79.32.0/20', 17501, 'NP'), + ('2620:0:870::/48', 3856, 'US'), + ('2a02:6b8:1::/48', 13238, 'RU'), + ('2001:db8::/32', 65536, 'ZZ') +; +``` + +Let's define an `ip_trie` dictionary for this table. The `ip_trie` layout requires a composite key: + +``` xml + + + + prefix + String + + + + asn + UInt32 + + + + cca2 + String + ?? + + ... + + + + + + true + + +``` + +or + +``` sql +CREATE DICTIONARY my_ip_trie_dictionary ( + prefix String, + asn UInt32, + cca2 String DEFAULT '??' +) +PRIMARY KEY prefix +SOURCE(CLICKHOUSE(TABLE 'my_ip_addresses')) +LAYOUT(IP_TRIE) +LIFETIME(3600); +``` + +The key must have only one `String` type attribute that contains an allowed IP prefix. Other types are not supported yet. + +For queries, you must use the same functions (`dictGetT` with a tuple) as for dictionaries with composite keys. The syntax is: + +``` sql +dictGetT('dict_name', 'attr_name', tuple(ip)) +``` + +The function takes either `UInt32` for IPv4, or `FixedString(16)` for IPv6. For example: + +``` sql +select dictGet('my_ip_trie_dictionary', 'asn', tuple(IPv6StringToNum('2001:db8::1'))) +``` + +Other types are not supported yet. The function returns the attribute for the prefix that corresponds to this IP address. If there are overlapping prefixes, the most specific one is returned. + +Data must completely fit into RAM. + +## Dictionary Updates {#dictionary-updates} + +ClickHouse periodically updates the dictionaries. The update interval for fully downloaded dictionaries and the invalidation interval for cached dictionaries are defined in the `lifetime` tag in seconds. + +Dictionary updates (other than loading for first use) do not block queries. During updates, the old version of a dictionary is used. If an error occurs during an update, the error is written to the server log, and queries continue using the old version of dictionaries. + +Example of settings: + + + +``` xml + + ... + 300 + ... + +``` + +or + +``` sql +CREATE DICTIONARY (...) +... +LIFETIME(300) +... +``` + +Setting `0` (`LIFETIME(0)`) prevents dictionaries from updating. + +You can set a time interval for updates, and ClickHouse will choose a uniformly random time within this range. This is necessary in order to distribute the load on the dictionary source when updating on a large number of servers. + +Example of settings: + +``` xml + + ... + + 300 + 360 + + ... + +``` + +or + +``` sql +LIFETIME(MIN 300 MAX 360) +``` + +If `0` and `0`, ClickHouse does not reload the dictionary by timeout. +In this case, ClickHouse can reload the dictionary earlier if the dictionary configuration file was changed or the `SYSTEM RELOAD DICTIONARY` command was executed. + +When updating the dictionaries, the ClickHouse server applies different logic depending on the type of [source](#dictionary-sources): + +- For a text file, it checks the time of modification. If the time differs from the previously recorded time, the dictionary is updated. +- For MySQL source, the time of modification is checked using a `SHOW TABLE STATUS` query (in case of MySQL 8 you need to disable meta-information caching in MySQL by `set global information_schema_stats_expiry=0`). +- Dictionaries from other sources are updated every time by default. + +For other sources (ODBC, PostgreSQL, ClickHouse, etc), you can set up a query that will update the dictionaries only if they really changed, rather than each time. To do this, follow these steps: + +- The dictionary table must have a field that always changes when the source data is updated. +- The settings of the source must specify a query that retrieves the changing field. The ClickHouse server interprets the query result as a row, and if this row has changed relative to its previous state, the dictionary is updated. Specify the query in the `` field in the settings for the [source](#dictionary-sources). + +Example of settings: + +``` xml + + ... + + ... + SELECT update_time FROM dictionary_source where id = 1 + + ... + +``` + +or + +``` sql +... +SOURCE(ODBC(... invalidate_query 'SELECT update_time FROM dictionary_source where id = 1')) +... +``` + +For `Cache`, `ComplexKeyCache`, `SSDCache`, and `SSDComplexKeyCache` dictionaries both synchronious and asynchronious updates are supported. + +It is also possible for `Flat`, `Hashed`, `ComplexKeyHashed` dictionaries to only request data that was changed after the previous update. If `update_field` is specified as part of the dictionary source configuration, value of the previous update time in seconds will be added to the data request. Depends on source type (Executable, HTTP, MySQL, PostgreSQL, ClickHouse, or ODBC) different logic will be applied to `update_field` before request data from an external source. + +- If the source is HTTP then `update_field` will be added as a query parameter with the last update time as the parameter value. +- If the source is Executable then `update_field` will be added as an executable script argument with the last update time as the argument value. +- If the source is ClickHouse, MySQL, PostgreSQL, ODBC there will be an additional part of `WHERE`, where `update_field` is compared as greater or equal with the last update time. + - Per default, this `WHERE`-condition is checked at the highest level of the SQL-Query. Alternatively, the condition can be checked in any other `WHERE`-clause within the query using the `{condition}`-keyword. Example: + ```sql + ... + SOURCE(CLICKHOUSE(... + update_field 'added_time' + QUERY ' + SELECT my_arr.1 AS x, my_arr.2 AS y, creation_time + FROM ( + SELECT arrayZip(x_arr, y_arr) AS my_arr, creation_time + FROM dictionary_source + WHERE {condition} + )' + )) + ... + ``` + +If `update_field` option is set, additional option `update_lag` can be set. Value of `update_lag` option is subtracted from previous update time before request updated data. + +Example of settings: + +``` xml + + ... + + ... + added_time + 15 + + ... + +``` + +or + +``` sql +... +SOURCE(CLICKHOUSE(... update_field 'added_time' update_lag 15)) +... +``` + +## Dictionary Sources {#dictionary-sources} + + + +A dictionary can be connected to ClickHouse from many different sources. + +If the dictionary is configured using an xml-file, the configuration looks like this: + +``` xml + + + ... + + + + + + ... + + ... + +``` + +In case of [DDL-query](../../sql-reference/statements/create/dictionary.md), the configuration described above will look like: + +``` sql +CREATE DICTIONARY dict_name (...) +... +SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration +... +``` + +The source is configured in the `source` section. + +For source types [Local file](#local_file), [Executable file](#executable), [HTTP(s)](#https), [ClickHouse](#clickhouse) +optional settings are available: + +``` xml + + + /opt/dictionaries/os.tsv + TabSeparated + + + 0 + + +``` + +or + +``` sql +SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated')) +SETTINGS(format_csv_allow_single_quotes = 0) +``` + +Types of sources (`source_type`): + +- [Local file](#local_file) +- [Executable File](#executable) +- [Executable Pool](#executable_pool) +- [HTTP(s)](#http) +- DBMS + - [ODBC](#odbc) + - [MySQL](#mysql) + - [ClickHouse](#clickhouse) + - [MongoDB](#mongodb) + - [Redis](#redis) + - [Cassandra](#cassandra) + - [PostgreSQL](#postgresql) + +## Local File {#local_file} + +Example of settings: + +``` xml + + + /opt/dictionaries/os.tsv + TabSeparated + + +``` + +or + +``` sql +SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated')) +``` + +Setting fields: + +- `path` – The absolute path to the file. +- `format` – The file format. All the formats described in [Formats](../../interfaces/formats.md#formats) are supported. + +When a dictionary with source `FILE` is created via DDL command (`CREATE DICTIONARY ...`), the source file needs to be located in the `user_files` directory to prevent DB users from accessing arbitrary files on the ClickHouse node. + +**See Also** + +- [Dictionary function](../../sql-reference/table-functions/dictionary.md#dictionary-function) + +## Executable File {#executable} + +Working with executable files depends on [how the dictionary is stored in memory](#storig-dictionaries-in-memory). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request to the executable file’s STDIN. Otherwise, ClickHouse starts the executable file and treats its output as dictionary data. + +Example of settings: + +``` xml + + + cat /opt/dictionaries/os.tsv + TabSeparated + false + + +``` + +Setting fields: + +- `command` — The absolute path to the executable file, or the file name (if the command's directory is in the `PATH`). +- `format` — The file format. All the formats described in [Formats](../../interfaces/formats.md#formats) are supported. +- `command_termination_timeout` — The executable script should contain a main read-write loop. After the dictionary is destroyed, the pipe is closed, and the executable file will have `command_termination_timeout` seconds to shutdown before ClickHouse will send a SIGTERM signal to the child process. `command_termination_timeout` is specified in seconds. Default value is 10. Optional parameter. +- `command_read_timeout` - Timeout for reading data from command stdout in milliseconds. Default value 10000. Optional parameter. +- `command_write_timeout` - Timeout for writing data to command stdin in milliseconds. Default value 10000. Optional parameter. +- `implicit_key` — The executable source file can return only values, and the correspondence to the requested keys is determined implicitly — by the order of rows in the result. Default value is false. +- `execute_direct` - If `execute_direct` = `1`, then `command` will be searched inside user_scripts folder specified by [user_scripts_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_scripts_path). Additional script arguments can be specified using a whitespace separator. Example: `script_name arg1 arg2`. If `execute_direct` = `0`, `command` is passed as argument for `bin/sh -c`. Default value is `0`. Optional parameter. +- `send_chunk_header` - controls whether to send row count before sending a chunk of data to process. Optional. Default value is `false`. + +That dictionary source can be configured only via XML configuration. Creating dictionaries with executable source via DDL is disabled; otherwise, the DB user would be able to execute arbitrary binaries on the ClickHouse node. + +## Executable Pool {#executable_pool} + +Executable pool allows loading data from pool of processes. This source does not work with dictionary layouts that need to load all data from source. Executable pool works if the dictionary [is stored](#ways-to-store-dictionaries-in-memory) using `cache`, `complex_key_cache`, `ssd_cache`, `complex_key_ssd_cache`, `direct`, or `complex_key_direct` layouts. + +Executable pool will spawn a pool of processes with the specified command and keep them running until they exit. The program should read data from STDIN while it is available and output the result to STDOUT. It can wait for the next block of data on STDIN. ClickHouse will not close STDIN after processing a block of data, but will pipe another chunk of data when needed. The executable script should be ready for this way of data processing — it should poll STDIN and flush data to STDOUT early. + +Example of settings: + +``` xml + + + while read key; do printf "$key\tData for key $key\n"; done + TabSeparated + 10 + 10 + false + + +``` + +Setting fields: + +- `command` — The absolute path to the executable file, or the file name (if the program directory is written to `PATH`). +- `format` — The file format. All the formats described in “[Formats](../../interfaces/formats.md#formats)” are supported. +- `pool_size` — Size of pool. If 0 is specified as `pool_size` then there is no pool size restrictions. Default value is `16`. +- `command_termination_timeout` — executable script should contain main read-write loop. After dictionary is destroyed, pipe is closed, and executable file will have `command_termination_timeout` seconds to shutdown, before ClickHouse will send SIGTERM signal to child process. Specified in seconds. Default value is 10. Optional parameter. +- `max_command_execution_time` — Maximum executable script command execution time for processing block of data. Specified in seconds. Default value is 10. Optional parameter. +- `command_read_timeout` - timeout for reading data from command stdout in milliseconds. Default value 10000. Optional parameter. +- `command_write_timeout` - timeout for writing data to command stdin in milliseconds. Default value 10000. Optional parameter. +- `implicit_key` — The executable source file can return only values, and the correspondence to the requested keys is determined implicitly — by the order of rows in the result. Default value is false. Optional parameter. +- `execute_direct` - If `execute_direct` = `1`, then `command` will be searched inside user_scripts folder specified by [user_scripts_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_scripts_path). Additional script arguments can be specified using whitespace separator. Example: `script_name arg1 arg2`. If `execute_direct` = `0`, `command` is passed as argument for `bin/sh -c`. Default value is `1`. Optional parameter. +- `send_chunk_header` - controls whether to send row count before sending a chunk of data to process. Optional. Default value is `false`. + +That dictionary source can be configured only via XML configuration. Creating dictionaries with executable source via DDL is disabled, otherwise, the DB user would be able to execute arbitrary binary on ClickHouse node. + +## Http(s) {#https} + +Working with an HTTP(s) server depends on [how the dictionary is stored in memory](#storig-dictionaries-in-memory). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request via the `POST` method. + +Example of settings: + +``` xml + + + http://[::1]/os.tsv + TabSeparated + + user + password + + +
+ API-KEY + key +
+
+
+ +``` + +or + +``` sql +SOURCE(HTTP( + url 'http://[::1]/os.tsv' + format 'TabSeparated' + credentials(user 'user' password 'password') + headers(header(name 'API-KEY' value 'key')) +)) +``` + +In order for ClickHouse to access an HTTPS resource, you must [configure openSSL](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-openssl) in the server configuration. + +Setting fields: + +- `url` – The source URL. +- `format` – The file format. All the formats described in “[Formats](../../interfaces/formats.md#formats)” are supported. +- `credentials` – Basic HTTP authentication. Optional parameter. +- `user` – Username required for the authentication. +- `password` – Password required for the authentication. +- `headers` – All custom HTTP headers entries used for the HTTP request. Optional parameter. +- `header` – Single HTTP header entry. +- `name` – Identifiant name used for the header send on the request. +- `value` – Value set for a specific identifiant name. + +When creating a dictionary using the DDL command (`CREATE DICTIONARY ...`) remote hosts for HTTP dictionaries are checked against the contents of `remote_url_allow_hosts` section from config to prevent database users to access arbitrary HTTP server. + +### Known Vulnerability of the ODBC Dictionary Functionality + +:::note +When connecting to the database through the ODBC driver connection parameter `Servername` can be substituted. In this case values of `USERNAME` and `PASSWORD` from `odbc.ini` are sent to the remote server and can be compromised. +::: + +**Example of insecure use** + +Let’s configure unixODBC for PostgreSQL. Content of `/etc/odbc.ini`: + +``` text +[gregtest] +Driver = /usr/lib/psqlodbca.so +Servername = localhost +PORT = 5432 +DATABASE = test_db +#OPTION = 3 +USERNAME = test +PASSWORD = test +``` + +If you then make a query such as + +``` sql +SELECT * FROM odbc('DSN=gregtest;Servername=some-server.com', 'test_db'); +``` + +ODBC driver will send values of `USERNAME` and `PASSWORD` from `odbc.ini` to `some-server.com`. + +### Example of Connecting Postgresql + +Ubuntu OS. + +Installing unixODBC and the ODBC driver for PostgreSQL: + +``` bash +$ sudo apt-get install -y unixodbc odbcinst odbc-postgresql +``` + +Configuring `/etc/odbc.ini` (or `~/.odbc.ini` if you signed in under a user that runs ClickHouse): + +``` text + [DEFAULT] + Driver = myconnection + + [myconnection] + Description = PostgreSQL connection to my_db + Driver = PostgreSQL Unicode + Database = my_db + Servername = 127.0.0.1 + UserName = username + Password = password + Port = 5432 + Protocol = 9.3 + ReadOnly = No + RowVersioning = No + ShowSystemTables = No + ConnSettings = +``` + +The dictionary configuration in ClickHouse: + +``` xml + + + table_name + + + + + DSN=myconnection + postgresql_table
+
+ + + 300 + 360 + + + + + + + id + + + some_column + UInt64 + 0 + + +
+
+``` + +or + +``` sql +CREATE DICTIONARY table_name ( + id UInt64, + some_column UInt64 DEFAULT 0 +) +PRIMARY KEY id +SOURCE(ODBC(connection_string 'DSN=myconnection' table 'postgresql_table')) +LAYOUT(HASHED()) +LIFETIME(MIN 300 MAX 360) +``` + +You may need to edit `odbc.ini` to specify the full path to the library with the driver `DRIVER=/usr/local/lib/psqlodbcw.so`. + +### Example of Connecting MS SQL Server + +Ubuntu OS. + +Installing the ODBC driver for connecting to MS SQL: + +``` bash +$ sudo apt-get install tdsodbc freetds-bin sqsh +``` + +Configuring the driver: + +```bash + $ cat /etc/freetds/freetds.conf + ... + + [MSSQL] + host = 192.168.56.101 + port = 1433 + tds version = 7.0 + client charset = UTF-8 + + # test TDS connection + $ sqsh -S MSSQL -D database -U user -P password + + + $ cat /etc/odbcinst.ini + + [FreeTDS] + Description = FreeTDS + Driver = /usr/lib/x86_64-linux-gnu/odbc/libtdsodbc.so + Setup = /usr/lib/x86_64-linux-gnu/odbc/libtdsS.so + FileUsage = 1 + UsageCount = 5 + + $ cat /etc/odbc.ini + # $ cat ~/.odbc.ini # if you signed in under a user that runs ClickHouse + + [MSSQL] + Description = FreeTDS + Driver = FreeTDS + Servername = MSSQL + Database = test + UID = test + PWD = test + Port = 1433 + + + # (optional) test ODBC connection (to use isql-tool install the [unixodbc](https://packages.debian.org/sid/unixodbc)-package) + $ isql -v MSSQL "user" "password" +``` + +Remarks: +- to determine the earliest TDS version that is supported by a particular SQL Server version, refer to the product documentation or look at [MS-TDS Product Behavior](https://docs.microsoft.com/en-us/openspecs/windows_protocols/ms-tds/135d0ebe-5c4c-4a94-99bf-1811eccb9f4a) + +Configuring the dictionary in ClickHouse: + +``` xml + + + test + + + dict
+ DSN=MSSQL;UID=test;PWD=test +
+ + + + 300 + 360 + + + + + + + + + k + + + s + String + + + +
+
+``` + +or + +``` sql +CREATE DICTIONARY test ( + k UInt64, + s String DEFAULT '' +) +PRIMARY KEY k +SOURCE(ODBC(table 'dict' connection_string 'DSN=MSSQL;UID=test;PWD=test')) +LAYOUT(FLAT()) +LIFETIME(MIN 300 MAX 360) +``` + +## DBMS + +### ODBC + +You can use this method to connect any database that has an ODBC driver. + +Example of settings: + +``` xml + + + DatabaseName + ShemaName.TableName
+ DSN=some_parameters + SQL_QUERY + SELECT id, value_1, value_2 FROM ShemaName.TableName +
+ +``` + +or + +``` sql +SOURCE(ODBC( + db 'DatabaseName' + table 'SchemaName.TableName' + connection_string 'DSN=some_parameters' + invalidate_query 'SQL_QUERY' + query 'SELECT id, value_1, value_2 FROM db_name.table_name' +)) +``` + +Setting fields: + +- `db` – Name of the database. Omit it if the database name is set in the `` parameters. +- `table` – Name of the table and schema if exists. +- `connection_string` – Connection string. +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](#dictionary-updates). +- `query` – The custom query. Optional parameter. + +:::note +The `table` and `query` fields cannot be used together. And either one of the `table` or `query` fields must be declared. +::: + +ClickHouse receives quoting symbols from ODBC-driver and quote all settings in queries to driver, so it’s necessary to set table name accordingly to table name case in database. + +If you have a problems with encodings when using Oracle, see the corresponding [FAQ](/knowledgebase/oracle-odbc) item. + +### Mysql + +Example of settings: + +``` xml + + + 3306 + clickhouse + qwerty + + example01-1 + 1 + + + example01-2 + 1 + + db_name + table_name
+ id=10 + SQL_QUERY + true + SELECT id, value_1, value_2 FROM db_name.table_name +
+ +``` + +or + +``` sql +SOURCE(MYSQL( + port 3306 + user 'clickhouse' + password 'qwerty' + replica(host 'example01-1' priority 1) + replica(host 'example01-2' priority 1) + db 'db_name' + table 'table_name' + where 'id=10' + invalidate_query 'SQL_QUERY' + fail_on_connection_loss 'true' + query 'SELECT id, value_1, value_2 FROM db_name.table_name' +)) +``` + +Setting fields: + +- `port` – The port on the MySQL server. You can specify it for all replicas, or for each one individually (inside ``). + +- `user` – Name of the MySQL user. You can specify it for all replicas, or for each one individually (inside ``). + +- `password` – Password of the MySQL user. You can specify it for all replicas, or for each one individually (inside ``). + +- `replica` – Section of replica configurations. There can be multiple sections. + + - `replica/host` – The MySQL host. + - `replica/priority` – The replica priority. When attempting to connect, ClickHouse traverses the replicas in order of priority. The lower the number, the higher the priority. + +- `db` – Name of the database. + +- `table` – Name of the table. + +- `where` – The selection criteria. The syntax for conditions is the same as for `WHERE` clause in MySQL, for example, `id > 10 AND id < 20`. Optional parameter. + +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](#dictionary-updates). + +- `fail_on_connection_loss` – The configuration parameter that controls behavior of the server on connection loss. If `true`, an exception is thrown immediately if the connection between client and server was lost. If `false`, the ClickHouse server retries to execute the query three times before throwing an exception. Note that retrying leads to increased response times. Default value: `false`. + +- `query` – The custom query. Optional parameter. + +:::note +The `table` or `where` fields cannot be used together with the `query` field. And either one of the `table` or `query` fields must be declared. +::: + +:::note +There is no explicit parameter `secure`. When establishing an SSL-connection security is mandatory. +::: + +MySQL can be connected to on a local host via sockets. To do this, set `host` and `socket`. + +Example of settings: + +``` xml + + + localhost + /path/to/socket/file.sock + clickhouse + qwerty + db_name + table_name
+ id=10 + SQL_QUERY + true + SELECT id, value_1, value_2 FROM db_name.table_name +
+ +``` + +or + +``` sql +SOURCE(MYSQL( + host 'localhost' + socket '/path/to/socket/file.sock' + user 'clickhouse' + password 'qwerty' + db 'db_name' + table 'table_name' + where 'id=10' + invalidate_query 'SQL_QUERY' + fail_on_connection_loss 'true' + query 'SELECT id, value_1, value_2 FROM db_name.table_name' +)) +``` + +### ClickHouse + +Example of settings: + +``` xml + + + example01-01-1 + 9000 + default + + default + ids
+ id=10 + 1 + SELECT id, value_1, value_2 FROM default.ids +
+ +``` + +or + +``` sql +SOURCE(CLICKHOUSE( + host 'example01-01-1' + port 9000 + user 'default' + password '' + db 'default' + table 'ids' + where 'id=10' + secure 1 + query 'SELECT id, value_1, value_2 FROM default.ids' +)); +``` + +Setting fields: + +- `host` – The ClickHouse host. If it is a local host, the query is processed without any network activity. To improve fault tolerance, you can create a [Distributed](../../engines/table-engines/special/distributed.md) table and enter it in subsequent configurations. +- `port` – The port on the ClickHouse server. +- `user` – Name of the ClickHouse user. +- `password` – Password of the ClickHouse user. +- `db` – Name of the database. +- `table` – Name of the table. +- `where` – The selection criteria. May be omitted. +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](#dictionary-updates). +- `secure` - Use ssl for connection. +- `query` – The custom query. Optional parameter. + +:::note +The `table` or `where` fields cannot be used together with the `query` field. And either one of the `table` or `query` fields must be declared. +::: + +### Mongodb + +Example of settings: + +``` xml + + + localhost + 27017 + + + test + dictionary_source + + +``` + +or + +``` sql +SOURCE(MONGODB( + host 'localhost' + port 27017 + user '' + password '' + db 'test' + collection 'dictionary_source' +)) +``` + +Setting fields: + +- `host` – The MongoDB host. +- `port` – The port on the MongoDB server. +- `user` – Name of the MongoDB user. +- `password` – Password of the MongoDB user. +- `db` – Name of the database. +- `collection` – Name of the collection. + +### Redis + +Example of settings: + +``` xml + + + localhost + 6379 + simple + 0 + + +``` + +or + +``` sql +SOURCE(REDIS( + host 'localhost' + port 6379 + storage_type 'simple' + db_index 0 +)) +``` + +Setting fields: + +- `host` – The Redis host. +- `port` – The port on the Redis server. +- `storage_type` – The structure of internal Redis storage using for work with keys. `simple` is for simple sources and for hashed single key sources, `hash_map` is for hashed sources with two keys. Ranged sources and cache sources with complex key are unsupported. May be omitted, default value is `simple`. +- `db_index` – The specific numeric index of Redis logical database. May be omitted, default value is 0. + +### Cassandra + +Example of settings: + +``` xml + + + localhost + 9042 + username + qwerty123 + database_name + table_name + 1 + 1 + One + "SomeColumn" = 42 + 8 + SELECT id, value_1, value_2 FROM database_name.table_name + + +``` + +Setting fields: + +- `host` – The Cassandra host or comma-separated list of hosts. +- `port` – The port on the Cassandra servers. If not specified, default port 9042 is used. +- `user` – Name of the Cassandra user. +- `password` – Password of the Cassandra user. +- `keyspace` – Name of the keyspace (database). +- `column_family` – Name of the column family (table). +- `allow_filering` – Flag to allow or not potentially expensive conditions on clustering key columns. Default value is 1. +- `partition_key_prefix` – Number of partition key columns in primary key of the Cassandra table. Required for compose key dictionaries. Order of key columns in the dictionary definition must be the same as in Cassandra. Default value is 1 (the first key column is a partition key and other key columns are clustering key). +- `consistency` – Consistency level. Possible values: `One`, `Two`, `Three`, `All`, `EachQuorum`, `Quorum`, `LocalQuorum`, `LocalOne`, `Serial`, `LocalSerial`. Default value is `One`. +- `where` – Optional selection criteria. +- `max_threads` – The maximum number of threads to use for loading data from multiple partitions in compose key dictionaries. +- `query` – The custom query. Optional parameter. + +:::note +The `column_family` or `where` fields cannot be used together with the `query` field. And either one of the `column_family` or `query` fields must be declared. +::: + +### PostgreSQL + +Example of settings: + +``` xml + + + 5432 + clickhouse + qwerty + db_name + table_name
+ id=10 + SQL_QUERY + SELECT id, value_1, value_2 FROM db_name.table_name +
+ +``` + +or + +``` sql +SOURCE(POSTGRESQL( + port 5432 + host 'postgresql-hostname' + user 'postgres_user' + password 'postgres_password' + db 'db_name' + table 'table_name' + replica(host 'example01-1' port 5432 priority 1) + replica(host 'example01-2' port 5432 priority 2) + where 'id=10' + invalidate_query 'SQL_QUERY' + query 'SELECT id, value_1, value_2 FROM db_name.table_name' +)) +``` + +Setting fields: + +- `host` – The host on the PostgreSQL server. You can specify it for all replicas, or for each one individually (inside ``). +- `port` – The port on the PostgreSQL server. You can specify it for all replicas, or for each one individually (inside ``). +- `user` – Name of the PostgreSQL user. You can specify it for all replicas, or for each one individually (inside ``). +- `password` – Password of the PostgreSQL user. You can specify it for all replicas, or for each one individually (inside ``). +- `replica` – Section of replica configurations. There can be multiple sections: + - `replica/host` – The PostgreSQL host. + - `replica/port` – The PostgreSQL port. + - `replica/priority` – The replica priority. When attempting to connect, ClickHouse traverses the replicas in order of priority. The lower the number, the higher the priority. +- `db` – Name of the database. +- `table` – Name of the table. +- `where` – The selection criteria. The syntax for conditions is the same as for `WHERE` clause in PostgreSQL. For example, `id > 10 AND id < 20`. Optional parameter. +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](#dictionary-updates). +- `query` – The custom query. Optional parameter. + +:::note +The `table` or `where` fields cannot be used together with the `query` field. And either one of the `table` or `query` fields must be declared. +::: + +## Null + +A special source that can be used to create dummy (empty) dictionaries. Such dictionaries can useful for tests or with setups with separated data and query nodes at nodes with Distributed tables. + +``` sql +CREATE DICTIONARY null_dict ( + id UInt64, + val UInt8, + default_val UInt8 DEFAULT 123, + nullable_val Nullable(UInt8) +) +PRIMARY KEY id +SOURCE(NULL()) +LAYOUT(FLAT()) +LIFETIME(0); +``` + +## Dictionary Key and Fields {#dictionary-key-and-fields} + + + +The `structure` clause describes the dictionary key and fields available for queries. + +XML description: + +``` xml + + + + Id + + + + + + + ... + + + +``` + +Attributes are described in the elements: + +- `` — Key column +- `` — Data column: there can be a multiple number of attributes. + +DDL query: + +``` sql +CREATE DICTIONARY dict_name ( + Id UInt64, + -- attributes +) +PRIMARY KEY Id +... +``` + +Attributes are described in the query body: + +- `PRIMARY KEY` — Key column +- `AttrName AttrType` — Data column. There can be a multiple number of attributes. + +## Key + +ClickHouse supports the following types of keys: + +- Numeric key. `UInt64`. Defined in the `` tag or using `PRIMARY KEY` keyword. +- Composite key. Set of values of different types. Defined in the tag `` or `PRIMARY KEY` keyword. + +An xml structure can contain either `` or ``. DDL-query must contain single `PRIMARY KEY`. + +:::warning +You must not describe key as an attribute. +::: + +### Numeric Key + +Type: `UInt64`. + +Configuration example: + +``` xml + + Id + +``` + +Configuration fields: + +- `name` – The name of the column with keys. + +For DDL-query: + +``` sql +CREATE DICTIONARY ( + Id UInt64, + ... +) +PRIMARY KEY Id +... +``` + +- `PRIMARY KEY` – The name of the column with keys. + +### Composite Key + +The key can be a `tuple` from any types of fields. The [layout](#storig-dictionaries-in-memory) in this case must be `complex_key_hashed` or `complex_key_cache`. + +:::tip +A composite key can consist of a single element. This makes it possible to use a string as the key, for instance. +::: + +The key structure is set in the element ``. Key fields are specified in the same format as the dictionary [attributes](#dictionary-key-and-fields). Example: + +``` xml + + + + field1 + String + + + field2 + UInt32 + + ... + +... +``` + +or + +``` sql +CREATE DICTIONARY ( + field1 String, + field2 String + ... +) +PRIMARY KEY field1, field2 +... +``` + +For a query to the `dictGet*` function, a tuple is passed as the key. Example: `dictGetString('dict_name', 'attr_name', tuple('string for field1', num_for_field2))`. + +## Attributes + +Configuration example: + +``` xml + + ... + + Name + ClickHouseDataType + + rand64() + true + true + true + + +``` + +or + +``` sql +CREATE DICTIONARY somename ( + Name ClickHouseDataType DEFAULT '' EXPRESSION rand64() HIERARCHICAL INJECTIVE IS_OBJECT_ID +) +``` + +Configuration fields: + +| Tag | Description | Required | +|------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------| +| `name` | Column name. | Yes | +| `type` | ClickHouse data type: [UInt8](../../sql-reference/data-types/int-uint.md), [UInt16](../../sql-reference/data-types/int-uint.md), [UInt32](../../sql-reference/data-types/int-uint.md), [UInt64](../../sql-reference/data-types/int-uint.md), [Int8](../../sql-reference/data-types/int-uint.md), [Int16](../../sql-reference/data-types/int-uint.md), [Int32](../../sql-reference/data-types/int-uint.md), [Int64](../../sql-reference/data-types/int-uint.md), [Float32](../../sql-reference/data-types/float.md), [Float64](../../sql-reference/data-types/float.md), [UUID](../../sql-reference/data-types/uuid.md), [Decimal32](../../sql-reference/data-types/decimal.md), [Decimal64](../../sql-reference/data-types/decimal.md), [Decimal128](../../sql-reference/data-types/decimal.md), [Decimal256](../../sql-reference/data-types/decimal.md),[Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md), [DateTime64](../../sql-reference/data-types/datetime64.md), [String](../../sql-reference/data-types/string.md), [Array](../../sql-reference/data-types/array.md).
ClickHouse tries to cast value from dictionary to the specified data type. For example, for MySQL, the field might be `TEXT`, `VARCHAR`, or `BLOB` in the MySQL source table, but it can be uploaded as `String` in ClickHouse.
[Nullable](../../sql-reference/data-types/nullable.md) is currently supported for [Flat](#flat), [Hashed](#hashed), [ComplexKeyHashed](#complex_key_hashed), [Direct](#direct), [ComplexKeyDirect](#complex_key_direct), [RangeHashed](#range_hashed), Polygon, [Cache](#cache), [ComplexKeyCache](#complex_key_cache), [SSDCache](#ssd_cache), [SSDComplexKeyCache](#complex_key_ssd_cache) dictionaries. In [IPTrie](#ip_trie) dictionaries `Nullable` types are not supported. | Yes | +| `null_value` | Default value for a non-existing element.
In the example, it is an empty string. [NULL](../syntax.md#null) value can be used only for the `Nullable` types (see the previous line with types description). | Yes | +| `expression` | [Expression](../../sql-reference/syntax.md#expressions) that ClickHouse executes on the value.
The expression can be a column name in the remote SQL database. Thus, you can use it to create an alias for the remote column.

Default value: no expression. | No | +| `hierarchical` | If `true`, the attribute contains the value of a parent key for the current key. See [Hierarchical Dictionaries](#hierarchical-dictionaries).

Default value: `false`. | No | +| `injective` | Flag that shows whether the `id -> attribute` image is [injective](https://en.wikipedia.org/wiki/Injective_function).
If `true`, ClickHouse can automatically place after the `GROUP BY` clause the requests to dictionaries with injection. Usually it significantly reduces the amount of such requests.

Default value: `false`. | No | +| `is_object_id` | Flag that shows whether the query is executed for a MongoDB document by `ObjectID`.

Default value: `false`. + +## Hierarchical Dictionaries {#hierarchical-dictionaries} + +ClickHouse supports hierarchical dictionaries with a [numeric key](#numeric-key). + +Look at the following hierarchical structure: + +``` text +0 (Common parent) +│ +├── 1 (Russia) +│ │ +│ └── 2 (Moscow) +│ │ +│ └── 3 (Center) +│ +└── 4 (Great Britain) + │ + └── 5 (London) +``` + +This hierarchy can be expressed as the following dictionary table. + +| region_id | parent_region | region_name | +|------------|----------------|---------------| +| 1 | 0 | Russia | +| 2 | 1 | Moscow | +| 3 | 2 | Center | +| 4 | 0 | Great Britain | +| 5 | 4 | London | + +This table contains a column `parent_region` that contains the key of the nearest parent for the element. + +ClickHouse supports the hierarchical property for external dictionary attributes. This property allows you to configure the hierarchical dictionary similar to described above. + +The [dictGetHierarchy](../../sql-reference/functions/ext-dict-functions.md#dictgethierarchy) function allows you to get the parent chain of an element. + +For our example, the structure of dictionary can be the following: + +``` xml + + + + region_id + + + + parent_region + UInt64 + 0 + true + + + + region_name + String + + + + + +``` + +## Polygon dictionaries {#polygon-dictionaries} + +Polygon dictionaries allow you to efficiently search for the polygon containing specified points. +For example: defining a city area by geographical coordinates. + +Example of a polygon dictionary configuration: + + + +``` xml + + + + + key + Array(Array(Array(Array(Float64)))) + + + + + name + String + + + + + value + UInt64 + 0 + + + + + + 1 + + + + ... + +``` + +The corresponding [DDL-query](../../sql-reference/statements/create/dictionary.md#create-dictionary-query): +``` sql +CREATE DICTIONARY polygon_dict_name ( + key Array(Array(Array(Array(Float64)))), + name String, + value UInt64 +) +PRIMARY KEY key +LAYOUT(POLYGON(STORE_POLYGON_KEY_COLUMN 1)) +... +``` + +When configuring the polygon dictionary, the key must have one of two types: + +- A simple polygon. It is an array of points. +- MultiPolygon. It is an array of polygons. Each polygon is a two-dimensional array of points. The first element of this array is the outer boundary of the polygon, and subsequent elements specify areas to be excluded from it. + +Points can be specified as an array or a tuple of their coordinates. In the current implementation, only two-dimensional points are supported. + +The user can upload their own data in all formats supported by ClickHouse. + +There are 3 types of [in-memory storage](#storig-dictionaries-in-memory) available: + +- `POLYGON_SIMPLE`. This is a naive implementation, where a linear pass through all polygons is made for each query, and membership is checked for each one without using additional indexes. + +- `POLYGON_INDEX_EACH`. A separate index is built for each polygon, which allows you to quickly check whether it belongs in most cases (optimized for geographical regions). +Also, a grid is superimposed on the area under consideration, which significantly narrows the number of polygons under consideration. +The grid is created by recursively dividing the cell into 16 equal parts and is configured with two parameters. +The division stops when the recursion depth reaches `MAX_DEPTH` or when the cell crosses no more than `MIN_INTERSECTIONS` polygons. +To respond to the query, there is a corresponding cell, and the index for the polygons stored in it is accessed alternately. + +- `POLYGON_INDEX_CELL`. This placement also creates the grid described above. The same options are available. For each sheet cell, an index is built on all pieces of polygons that fall into it, which allows you to quickly respond to a request. + +- `POLYGON`. Synonym to `POLYGON_INDEX_CELL`. + +Dictionary queries are carried out using standard [functions](../../sql-reference/functions/ext-dict-functions.md) for working with dictionaries. +An important difference is that here the keys will be the points for which you want to find the polygon containing them. + +**Example** + +Example of working with the dictionary defined above: + +``` sql +CREATE TABLE points ( + x Float64, + y Float64 +) +... +SELECT tuple(x, y) AS key, dictGet(dict_name, 'name', key), dictGet(dict_name, 'value', key) FROM points ORDER BY x, y; +``` + +As a result of executing the last command for each point in the 'points' table, a minimum area polygon containing this point will be found, and the requested attributes will be output. + +**Example** + +You can read columns from polygon dictionaries via SELECT query, just turn on the `store_polygon_key_column = 1` in the dictionary configuration or corresponding DDL-query. + +Query: + +``` sql +CREATE TABLE polygons_test_table +( + key Array(Array(Array(Tuple(Float64, Float64)))), + name String +) ENGINE = TinyLog; + +INSERT INTO polygons_test_table VALUES ([[[(3, 1), (0, 1), (0, -1), (3, -1)]]], 'Value'); + +CREATE DICTIONARY polygons_test_dictionary +( + key Array(Array(Array(Tuple(Float64, Float64)))), + name String +) +PRIMARY KEY key +SOURCE(CLICKHOUSE(TABLE 'polygons_test_table')) +LAYOUT(POLYGON(STORE_POLYGON_KEY_COLUMN 1)) +LIFETIME(0); + +SELECT * FROM polygons_test_dictionary; +``` + +Result: + +``` text +┌─key─────────────────────────────┬─name──┐ +│ [[[(3,1),(0,1),(0,-1),(3,-1)]]] │ Value │ +└─────────────────────────────────┴───────┘ +``` + +## RegExp Tree Dictionary {#regexp-tree-dictionary} + +Regexp Tree dictionary stores multiple trees of regular expressions with attributions. Users can retrieve strings in the dictionary. If a string matches the root of the regexp tree, we will collect the corresponding attributes of the matched root and continue to walk the children. If any of the children matches the string, we will collect attributes and rewrite the old ones if conflicts occur, then continue the traverse until we reach leaf nodes. + +Example of the ddl query for creating Regexp Tree dictionary: + + + +```sql +create dictionary regexp_dict +( + regexp String, + name String, + version String +) +PRIMARY KEY(regexp) +SOURCE(YAMLRegExpTree(PATH '/var/lib/clickhouse/user_files/regexp_tree.yaml')) +LAYOUT(regexp_tree) +... +``` + +We only allow `YAMLRegExpTree` to work with regexp_tree dicitionary layout. If you want to use other sources, please set variable `regexp_dict_allow_other_sources` true. + +**Source** + +We introduce a type of source called `YAMLRegExpTree` representing the structure of Regexp Tree dictionary. An Example of a valid yaml config is like: + +```xml +- regexp: 'Linux/(\d+[\.\d]*).+tlinux' + name: 'TencentOS' + version: '\1' + +- regexp: '\d+/tclwebkit(?:\d+[\.\d]*)' + name: 'Andriod' + versions: + - regexp: '33/tclwebkit' + version: '13' + - regexp: '3[12]/tclwebkit' + version: '12' + - regexp: '30/tclwebkit' + version: '11' + - regexp: '29/tclwebkit' + version: '10' +``` + +The key `regexp` represents the regular expression of a tree node. The name of key is same as the dictionary key. The `name` and `version` is user-defined attributions in the dicitionary. The `versions` (which can be any name that not appear in attributions or the key) indicates the children nodes of this tree. + +**Back Reference** + +The value of an attribution could contain a back reference which refers to a capture group of the matched regular expression. Reference number ranges from 1 to 9 and writes as `$1` or `\1`. + +During the query execution, the back reference in the value will be replaced by the matched capture group. + +**Query** + +Due to the specialty of Regexp Tree dictionary, we only allow functions `dictGet`, `dictGetOrDefault` and `dictGetOrNull` work with it. + +Example: + +```sql +SELECT dictGet('regexp_dict', ('name', 'version'), '31/tclwebkit1024'); +``` + +Result: + +``` +┌─dictGet('regexp_dict', ('name', 'version'), '31/tclwebkit1024')─┐ +│ ('Andriod','12') │ +└─────────────────────────────────────────────────────────────────┘ +``` + +## Embedded Dictionaries {#embedded-dictionaries} + + + +ClickHouse contains a built-in feature for working with a geobase. + +This allows you to: + +- Use a region’s ID to get its name in the desired language. +- Use a region’s ID to get the ID of a city, area, federal district, country, or continent. +- Check whether a region is part of another region. +- Get a chain of parent regions. + +All the functions support “translocality,” the ability to simultaneously use different perspectives on region ownership. For more information, see the section “Functions for working with web analytics dictionaries”. + +The internal dictionaries are disabled in the default package. +To enable them, uncomment the parameters `path_to_regions_hierarchy_file` and `path_to_regions_names_files` in the server configuration file. + +The geobase is loaded from text files. + +Place the `regions_hierarchy*.txt` files into the `path_to_regions_hierarchy_file` directory. This configuration parameter must contain the path to the `regions_hierarchy.txt` file (the default regional hierarchy), and the other files (`regions_hierarchy_ua.txt`) must be located in the same directory. + +Put the `regions_names_*.txt` files in the `path_to_regions_names_files` directory. + +You can also create these files yourself. The file format is as follows: + +`regions_hierarchy*.txt`: TabSeparated (no header), columns: + +- region ID (`UInt32`) +- parent region ID (`UInt32`) +- region type (`UInt8`): 1 - continent, 3 - country, 4 - federal district, 5 - region, 6 - city; other types do not have values +- population (`UInt32`) — optional column + +`regions_names_*.txt`: TabSeparated (no header), columns: + +- region ID (`UInt32`) +- region name (`String`) — Can’t contain tabs or line feeds, even escaped ones. + +A flat array is used for storing in RAM. For this reason, IDs shouldn’t be more than a million. + +Dictionaries can be updated without restarting the server. However, the set of available dictionaries is not updated. +For updates, the file modification times are checked. If a file has changed, the dictionary is updated. +The interval to check for changes is configured in the `builtin_dictionaries_reload_interval` parameter. +Dictionary updates (other than loading at first use) do not block queries. During updates, queries use the old versions of dictionaries. If an error occurs during an update, the error is written to the server log, and queries continue using the old version of dictionaries. + +We recommend periodically updating the dictionaries with the geobase. During an update, generate new files and write them to a separate location. When everything is ready, rename them to the files used by the server. + +There are also functions for working with OS identifiers and search engines, but they shouldn’t be used. + +## Embedded Dictionaries + + + +ClickHouse contains a built-in feature for working with a geobase. + +This allows you to: + +- Use a region’s ID to get its name in the desired language. +- Use a region’s ID to get the ID of a city, area, federal district, country, or continent. +- Check whether a region is part of another region. +- Get a chain of parent regions. + +All the functions support “translocality,” the ability to simultaneously use different perspectives on region ownership. For more information, see the section “Functions for working with web analytics dictionaries”. + +The internal dictionaries are disabled in the default package. +To enable them, uncomment the parameters `path_to_regions_hierarchy_file` and `path_to_regions_names_files` in the server configuration file. + +The geobase is loaded from text files. + +Place the `regions_hierarchy*.txt` files into the `path_to_regions_hierarchy_file` directory. This configuration parameter must contain the path to the `regions_hierarchy.txt` file (the default regional hierarchy), and the other files (`regions_hierarchy_ua.txt`) must be located in the same directory. + +Put the `regions_names_*.txt` files in the `path_to_regions_names_files` directory. + +You can also create these files yourself. The file format is as follows: + +`regions_hierarchy*.txt`: TabSeparated (no header), columns: + +- region ID (`UInt32`) +- parent region ID (`UInt32`) +- region type (`UInt8`): 1 - continent, 3 - country, 4 - federal district, 5 - region, 6 - city; other types do not have values +- population (`UInt32`) — optional column + +`regions_names_*.txt`: TabSeparated (no header), columns: + +- region ID (`UInt32`) +- region name (`String`) — Can’t contain tabs or line feeds, even escaped ones. + +A flat array is used for storing in RAM. For this reason, IDs shouldn’t be more than a million. + +Dictionaries can be updated without restarting the server. However, the set of available dictionaries is not updated. +For updates, the file modification times are checked. If a file has changed, the dictionary is updated. +The interval to check for changes is configured in the `builtin_dictionaries_reload_interval` parameter. +Dictionary updates (other than loading at first use) do not block queries. During updates, queries use the old versions of dictionaries. If an error occurs during an update, the error is written to the server log, and queries continue using the old version of dictionaries. + +We recommend periodically updating the dictionaries with the geobase. During an update, generate new files and write them to a separate location. When everything is ready, rename them to the files used by the server. + +There are also functions for working with OS identifiers and search engines, but they shouldn’t be used. diff --git a/docs/en/sql-reference/dictionaries/internal-dicts.md b/docs/en/sql-reference/dictionaries/internal-dicts.md deleted file mode 100644 index 11c6ee93aa6..00000000000 --- a/docs/en/sql-reference/dictionaries/internal-dicts.md +++ /dev/null @@ -1,55 +0,0 @@ ---- -slug: /en/sql-reference/dictionaries/internal-dicts -sidebar_position: 39 -sidebar_label: Embedded Dictionaries ---- -import SelfManaged from '@site/docs/en/_snippets/_self_managed_only_no_roadmap.md'; - -# Embedded Dictionaries - - - -ClickHouse contains a built-in feature for working with a geobase. - -This allows you to: - -- Use a region’s ID to get its name in the desired language. -- Use a region’s ID to get the ID of a city, area, federal district, country, or continent. -- Check whether a region is part of another region. -- Get a chain of parent regions. - -All the functions support “translocality,” the ability to simultaneously use different perspectives on region ownership. For more information, see the section “Functions for working with web analytics dictionaries”. - -The internal dictionaries are disabled in the default package. -To enable them, uncomment the parameters `path_to_regions_hierarchy_file` and `path_to_regions_names_files` in the server configuration file. - -The geobase is loaded from text files. - -Place the `regions_hierarchy*.txt` files into the `path_to_regions_hierarchy_file` directory. This configuration parameter must contain the path to the `regions_hierarchy.txt` file (the default regional hierarchy), and the other files (`regions_hierarchy_ua.txt`) must be located in the same directory. - -Put the `regions_names_*.txt` files in the `path_to_regions_names_files` directory. - -You can also create these files yourself. The file format is as follows: - -`regions_hierarchy*.txt`: TabSeparated (no header), columns: - -- region ID (`UInt32`) -- parent region ID (`UInt32`) -- region type (`UInt8`): 1 - continent, 3 - country, 4 - federal district, 5 - region, 6 - city; other types do not have values -- population (`UInt32`) — optional column - -`regions_names_*.txt`: TabSeparated (no header), columns: - -- region ID (`UInt32`) -- region name (`String`) — Can’t contain tabs or line feeds, even escaped ones. - -A flat array is used for storing in RAM. For this reason, IDs shouldn’t be more than a million. - -Dictionaries can be updated without restarting the server. However, the set of available dictionaries is not updated. -For updates, the file modification times are checked. If a file has changed, the dictionary is updated. -The interval to check for changes is configured in the `builtin_dictionaries_reload_interval` parameter. -Dictionary updates (other than loading at first use) do not block queries. During updates, queries use the old versions of dictionaries. If an error occurs during an update, the error is written to the server log, and queries continue using the old version of dictionaries. - -We recommend periodically updating the dictionaries with the geobase. During an update, generate new files and write them to a separate location. When everything is ready, rename them to the files used by the server. - -There are also functions for working with OS identifiers and search engines, but they shouldn’t be used. diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index ef0475027dd..503ef66143e 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -283,7 +283,7 @@ Result: ``` :::note -The return type of `toStartOf*`, `toLastDayOfMonth`, `toMonday`, `timeSlot` functions described below is determined by the configuration parameter [enable_extended_results_for_datetime_functions](../../operations/settings/settings#enable-extended-results-for-datetime-functions) which is `0` by default. +The return type of `toStartOf*`, `toLastDayOfMonth`, `toMonday`, `timeSlot` functions described below is determined by the configuration parameter [enable_extended_results_for_datetime_functions](../../operations/settings/settings.md#enable-extended-results-for-datetime-functions) which is `0` by default. Behavior for * `enable_extended_results_for_datetime_functions = 0`: Functions `toStartOfYear`, `toStartOfISOYear`, `toStartOfQuarter`, `toStartOfMonth`, `toStartOfWeek`, `toLastDayOfMonth`, `toMonday` return `Date` or `DateTime`. Functions `toStartOfDay`, `toStartOfHour`, `toStartOfFifteenMinutes`, `toStartOfTenMinutes`, `toStartOfFiveMinutes`, `toStartOfMinute`, `timeSlot` return `DateTime`. Though these functions can take values of the extended types `Date32` and `DateTime64` as an argument, passing them a time outside the normal range (year 1970 to 2149 for `Date` / 2106 for `DateTime`) will produce wrong results. @@ -1135,7 +1135,7 @@ SELECT ``` ```response ┌─toYYYYMM(now(), 'US/Eastern')─┐ -│ 202303 │ +│ 202303 │ └───────────────────────────────┘ ``` @@ -1335,7 +1335,7 @@ Similar to formatDateTime, except that it formats datetime in Joda style instead **Replacement fields** -Using replacement fields, you can define a pattern for the resulting string. +Using replacement fields, you can define a pattern for the resulting string. | Placeholder | Description | Presentation | Examples | diff --git a/docs/en/sql-reference/functions/ext-dict-functions.md b/docs/en/sql-reference/functions/ext-dict-functions.md index b4b7ec5ab21..07226b67601 100644 --- a/docs/en/sql-reference/functions/ext-dict-functions.md +++ b/docs/en/sql-reference/functions/ext-dict-functions.md @@ -6,11 +6,11 @@ sidebar_label: Dictionaries # Functions for Working with Dictionaries -:::note +:::note For dictionaries created with [DDL queries](../../sql-reference/statements/create/dictionary.md), the `dict_name` parameter must be fully specified, like `.`. Otherwise, the current database is used. ::: -For information on connecting and configuring dictionaries, see [Dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md). +For information on connecting and configuring dictionaries, see [Dictionaries](../../sql-reference/dictionaries/index.md). ## dictGet, dictGetOrDefault, dictGetOrNull @@ -31,7 +31,7 @@ dictGetOrNull('dict_name', attr_name, id_expr) **Returned value** -- If ClickHouse parses the attribute successfully in the [attribute’s data type](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-attributes), functions return the value of the dictionary attribute that corresponds to `id_expr`. +- If ClickHouse parses the attribute successfully in the [attribute’s data type](../../sql-reference/dictionaries/index.md#dictionary-key-and-fields#ext_dict_structure-attributes), functions return the value of the dictionary attribute that corresponds to `id_expr`. - If there is no the key, corresponding to `id_expr`, in the dictionary, then: @@ -226,7 +226,7 @@ Result: **See Also** -- [Dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) +- [Dictionaries](../../sql-reference/dictionaries/index.md) ## dictHas @@ -250,7 +250,7 @@ Type: `UInt8`. ## dictGetHierarchy -Creates an array, containing all the parents of a key in the [hierarchical dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md). +Creates an array, containing all the parents of a key in the [hierarchical dictionary](../../sql-reference/dictionaries/index.md#hierarchical-dictionaries). **Syntax** @@ -436,7 +436,7 @@ dictGet[Type]OrDefault('dict_name', 'attr_name', id_expr, default_value_expr) **Returned value** -- If ClickHouse parses the attribute successfully in the [attribute’s data type](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-attributes), functions return the value of the dictionary attribute that corresponds to `id_expr`. +- If ClickHouse parses the attribute successfully in the [attribute’s data type](../../sql-reference/dictionaries/index.md#dictionary-key-and-fields#ext_dict_structure-attributes), functions return the value of the dictionary attribute that corresponds to `id_expr`. - If there is no requested `id_expr` in the dictionary then: diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 7146484361e..011b73405c5 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -792,7 +792,7 @@ neighbor(column, offset[, default_value]) The result of the function depends on the affected data blocks and the order of data in the block. -:::warning +:::warning It can reach the neighbor rows only inside the currently processed data block. ::: @@ -902,7 +902,7 @@ Result: Calculates the difference between successive row values ​​in the data block. Returns 0 for the first row and the difference from the previous row for each subsequent row. -:::warning +:::warning It can reach the previous row only inside the currently processed data block. ::: @@ -986,7 +986,7 @@ Each event has a start time and an end time. The start time is included in the e The function calculates the total number of active (concurrent) events for each event start time. -:::warning +:::warning Events must be ordered by the start time in ascending order. If this requirement is violated the function raises an exception. Every data block is processed separately. If events from different data blocks overlap then they can not be processed correctly. ::: @@ -1674,7 +1674,7 @@ Result: Accumulates states of an aggregate function for each row of a data block. -:::warning +:::warning The state is reset for each new data block. ::: @@ -2177,7 +2177,7 @@ Number of digits. Type: [UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges). -:::note +:::note For `Decimal` values takes into account their scales: calculates result over underlying integer type which is `(value * scale)`. For example: `countDigits(42) = 2`, `countDigits(42.000) = 5`, `countDigits(0.04200) = 4`. I.e. you may check decimal overflow for `Decimal64` with `countDecimal(x) > 18`. It's a slow variant of [isDecimalOverflow](#is-decimal-overflow). ::: @@ -2260,7 +2260,7 @@ Result: ## currentProfiles -Returns a list of the current [settings profiles](../../operations/access-rights.md#settings-profiles-management) for the current user. +Returns a list of the current [settings profiles](../../guides/sre/user-management/index.md#settings-profiles-management) for the current user. The command [SET PROFILE](../../sql-reference/statements/set.md#query-set) could be used to change the current setting profile. If the command `SET PROFILE` was not used the function returns the profiles specified at the current user's definition (see [CREATE USER](../../sql-reference/statements/create/user.md#create-user-statement)). @@ -2272,7 +2272,7 @@ currentProfiles() **Returned value** -- List of the current user settings profiles. +- List of the current user settings profiles. Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). @@ -2288,7 +2288,7 @@ enabledProfiles() **Returned value** -- List of the enabled settings profiles. +- List of the enabled settings profiles. Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). @@ -2304,7 +2304,7 @@ defaultProfiles() **Returned value** -- List of the default settings profiles. +- List of the default settings profiles. Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). @@ -2320,7 +2320,7 @@ currentRoles() **Returned value** -- List of the current roles for the current user. +- List of the current roles for the current user. Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). @@ -2336,13 +2336,13 @@ enabledRoles() **Returned value** -- List of the enabled roles for the current user. +- List of the enabled roles for the current user. Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). ## defaultRoles -Returns the names of the roles which are enabled by default for the current user when he logins. Initially these are all roles granted to the current user (see [GRANT](../../sql-reference/statements/grant/#grant-select)), but that can be changed with the [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement) statement. +Returns the names of the roles which are enabled by default for the current user when he logins. Initially these are all roles granted to the current user (see [GRANT](../../sql-reference/statements/grant.md#grant-select)), but that can be changed with the [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement) statement. **Syntax** @@ -2352,7 +2352,7 @@ defaultRoles() **Returned value** -- List of the default roles for the current user. +- List of the default roles for the current user. Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). @@ -2499,7 +2499,7 @@ In the following example a configuration with two shards is used. The query is e Query: ``` sql -CREATE TABLE shard_num_example (dummy UInt8) +CREATE TABLE shard_num_example (dummy UInt8) ENGINE=Distributed(test_cluster_two_shards_localhost, system, one, dummy); SELECT dummy, shardNum(), shardCount() FROM shard_num_example; ``` diff --git a/docs/en/sql-reference/index.md b/docs/en/sql-reference/index.md new file mode 100644 index 00000000000..eddc5b204d9 --- /dev/null +++ b/docs/en/sql-reference/index.md @@ -0,0 +1,22 @@ +--- +keywords: [clickhouse, docs, sql reference, sql statements, sql, syntax] +title: SQL Reference +--- + +import { TwoColumnList } from '/src/components/two_column_list' +import { ClickableSquare } from '/src/components/clickable_square' +import { HorizontalDivide } from '/src/components/horizontal_divide' +import { ViewAllLink } from '/src/components/view_all_link' +import { VideoContainer } from '/src/components/video_container' + +import LinksDeployment from './sql-reference-links.json' + +# ClickHouse SQL Reference + +ClickHouse supports a declarative query language based on SQL that is identical to the ANSI SQL standard in many cases. + +Supported queries include GROUP BY, ORDER BY, subqueries in FROM, JOIN clause, IN operator, window functions and scalar subqueries. + + + + \ No newline at end of file diff --git a/docs/en/sql-reference/sql-reference-links.json b/docs/en/sql-reference/sql-reference-links.json new file mode 100644 index 00000000000..3811ad18462 --- /dev/null +++ b/docs/en/sql-reference/sql-reference-links.json @@ -0,0 +1,12 @@ +[ + { + "title": "Statements", + "description": "A list of available SQL statements in ClickHouse", + "url": "/docs/en/sql-reference/statements/" + }, + { + "title": "Database and Table Engines", + "description": "Engines determine where and how your data is stored", + "url": "/docs/en/engines/table-engines" + } +] diff --git a/docs/en/sql-reference/statements/alter/projection.md b/docs/en/sql-reference/statements/alter/projection.md index 908d28d7ab1..626d71709ac 100644 --- a/docs/en/sql-reference/statements/alter/projection.md +++ b/docs/en/sql-reference/statements/alter/projection.md @@ -17,7 +17,7 @@ Projections will create internally a new hidden table, this means that more IO a Example, If the projection has defined a different primary key, all the data from the original table will be duplicated. ::: -You can see more technical details about how projections work internally on this [page](/docs/en/guides/improving-query-performance/sparse-primary-indexes/sparse-primary-indexes-multiple.md/#option-3-projections). +You can see more technical details about how projections work internally on this [page](/docs/en/guides/best-practices/sparse-primary-indexes.md/#option-3-projections). ## Example filtering without using primary keys @@ -37,7 +37,7 @@ Using `ALTER TABLE`, we could add the Projection to an existing table: ``` ALTER TABLE visits_order ADD PROJECTION user_name_projection ( SELECT -* +* ORDER BY user_name ) @@ -161,6 +161,6 @@ The commands `ADD`, `DROP` and `CLEAR` are lightweight in a sense that they only Also, they are replicated, syncing projections metadata via ClickHouse Keeper or ZooKeeper. -:::note +:::note Projection manipulation is supported only for tables with [`*MergeTree`](/docs/en/engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](/docs/en/engines/table-engines/mergetree-family/replication.md) variants). ::: diff --git a/docs/en/sql-reference/statements/create/dictionary.md b/docs/en/sql-reference/statements/create/dictionary.md index e789dd9257f..29c72d62f24 100644 --- a/docs/en/sql-reference/statements/create/dictionary.md +++ b/docs/en/sql-reference/statements/create/dictionary.md @@ -5,7 +5,7 @@ sidebar_label: DICTIONARY title: "CREATE DICTIONARY" --- -Creates a new [dictionary](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) with given [structure](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md), [source](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md), [layout](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md) and [lifetime](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md). +Creates a new [dictionary](../../../sql-reference/dictionaries/index.md) with given [structure](../../../sql-reference/dictionaries/index.md#dictionary-key-and-fields), [source](../../../sql-reference/dictionaries/index.md#dictionary-sources), [layout](../../../sql-reference/dictionaries/index.md#storig-dictionaries-in-memory) and [lifetime](../../../sql-reference/dictionaries/index.md#dictionary-updates). ## Syntax @@ -29,7 +29,7 @@ The dictionary structure consists of attributes. Dictionary attributes are speci `ON CLUSTER` clause allows creating dictionary on a cluster, see [Distributed DDL](../../../sql-reference/distributed-ddl.md). -Depending on dictionary [layout](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md) one or more attributes can be specified as dictionary keys. +Depending on dictionary [layout](../../../sql-reference/dictionaries/index.md#storig-dictionaries-in-memory) one or more attributes can be specified as dictionary keys. ## SOURCE @@ -125,9 +125,9 @@ LAYOUT(HASHED()) ### Create a dictionary from another database -Please see the details in [Dictionary sources](/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md/#dbms). +Please see the details in [Dictionary sources](/docs/en/sql-reference/dictionaries/index.md#dictionary-sources/#dbms). **See Also** -- For more information, see the [Dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) section. -- [system.dictionaries](../../../operations/system-tables/dictionaries.md) — This table contains information about [Dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md). +- For more information, see the [Dictionaries](../../../sql-reference/dictionaries/index.md) section. +- [system.dictionaries](../../../operations/system-tables/dictionaries.md) — This table contains information about [Dictionaries](../../../sql-reference/dictionaries/index.md). diff --git a/docs/en/sql-reference/statements/create/quota.md b/docs/en/sql-reference/statements/create/quota.md index 3952743b480..7c31f93fff7 100644 --- a/docs/en/sql-reference/statements/create/quota.md +++ b/docs/en/sql-reference/statements/create/quota.md @@ -5,7 +5,7 @@ sidebar_label: QUOTA title: "CREATE QUOTA" --- -Creates a [quota](../../../operations/access-rights.md#quotas-management) that can be assigned to a user or a role. +Creates a [quota](../../../guides/sre/user-management/index.md#quotas-management) that can be assigned to a user or a role. Syntax: diff --git a/docs/en/sql-reference/statements/create/role.md b/docs/en/sql-reference/statements/create/role.md index 68fdd51e957..9b14e220e1f 100644 --- a/docs/en/sql-reference/statements/create/role.md +++ b/docs/en/sql-reference/statements/create/role.md @@ -5,7 +5,7 @@ sidebar_label: ROLE title: "CREATE ROLE" --- -Creates new [roles](../../../operations/access-rights.md#role-management). Role is a set of [privileges](../../../sql-reference/statements/grant.md#grant-privileges). A [user](../../../sql-reference/statements/create/user.md) assigned a role gets all the privileges of this role. +Creates new [roles](../../../guides/sre/user-management/index.md#role-management). Role is a set of [privileges](../../../sql-reference/statements/grant.md#grant-privileges). A [user](../../../sql-reference/statements/create/user.md) assigned a role gets all the privileges of this role. Syntax: @@ -22,7 +22,7 @@ User can have default roles which apply at user login. To set default roles, use To revoke a role, use the [REVOKE](../../../sql-reference/statements/revoke.md) statement. -To delete role, use the [DROP ROLE](../../../sql-reference/statements/drop#drop-role-statement) statement. The deleted role is being automatically revoked from all the users and roles to which it was assigned. +To delete role, use the [DROP ROLE](../../../sql-reference/statements/drop.md#drop-role-statement) statement. The deleted role is being automatically revoked from all the users and roles to which it was assigned. ## Examples diff --git a/docs/en/sql-reference/statements/create/row-policy.md b/docs/en/sql-reference/statements/create/row-policy.md index 31ce9221eea..56a57534234 100644 --- a/docs/en/sql-reference/statements/create/row-policy.md +++ b/docs/en/sql-reference/statements/create/row-policy.md @@ -5,9 +5,9 @@ sidebar_label: ROW POLICY title: "CREATE ROW POLICY" --- -Creates a [row policy](../../../operations/access-rights.md#row-policy-management), i.e. a filter used to determine which rows a user can read from a table. +Creates a [row policy](../../../guides/sre/user-management/index.md#row-policy-management), i.e. a filter used to determine which rows a user can read from a table. -:::warning +:::warning Row policies makes sense only for users with readonly access. If user can modify table or copy partitions between tables, it defeats the restrictions of row policies. ::: @@ -31,7 +31,7 @@ In the section `TO` you can provide a list of users and roles this policy should Keyword `ALL` means all the ClickHouse users including current user. Keyword `ALL EXCEPT` allow to exclude some users from the all users list, for example, `CREATE ROW POLICY ... TO ALL EXCEPT accountant, john@localhost` -:::note +:::note If there are no row policies defined for a table then any user can `SELECT` all the row from the table. Defining one or more row policies for the table makes the access to the table depending on the row policies no matter if those row policies are defined for the current user or not. For example, the following policy `CREATE ROW POLICY pol1 ON mydb.table1 USING b=1 TO mira, peter` diff --git a/docs/en/sql-reference/statements/create/settings-profile.md b/docs/en/sql-reference/statements/create/settings-profile.md index c2424ff6046..8e221a4d82f 100644 --- a/docs/en/sql-reference/statements/create/settings-profile.md +++ b/docs/en/sql-reference/statements/create/settings-profile.md @@ -5,7 +5,7 @@ sidebar_label: SETTINGS PROFILE title: "CREATE SETTINGS PROFILE" --- -Creates [settings profiles](../../../operations/access-rights.md#settings-profiles-management) that can be assigned to a user or a role. +Creates [settings profiles](../../../guides/sre/user-management/index.md#settings-profiles-management) that can be assigned to a user or a role. Syntax: @@ -27,7 +27,7 @@ CREATE USER robin IDENTIFIED BY 'password'; Create the `max_memory_usage_profile` settings profile with value and constraints for the `max_memory_usage` setting and assign it to user `robin`: ``` sql -CREATE -SETTINGS PROFILE max_memory_usage_profile SETTINGS max_memory_usage = 100000001 MIN 90000000 MAX 110000000 +CREATE +SETTINGS PROFILE max_memory_usage_profile SETTINGS max_memory_usage = 100000001 MIN 90000000 MAX 110000000 TO robin ``` diff --git a/docs/en/sql-reference/statements/create/user.md b/docs/en/sql-reference/statements/create/user.md index a756b3d4a0d..454195db3fa 100644 --- a/docs/en/sql-reference/statements/create/user.md +++ b/docs/en/sql-reference/statements/create/user.md @@ -5,7 +5,7 @@ sidebar_label: USER title: "CREATE USER" --- -Creates [user accounts](../../../operations/access-rights.md#user-account-management). +Creates [user accounts](../../../guides/sre/user-management/index.md#user-account-management). Syntax: diff --git a/docs/en/sql-reference/statements/delete.md b/docs/en/sql-reference/statements/delete.md index e1987e50af4..7d7b8855d51 100644 --- a/docs/en/sql-reference/statements/delete.md +++ b/docs/en/sql-reference/statements/delete.md @@ -30,12 +30,6 @@ SET allow_experimental_lightweight_delete = true; ::: -An [alternative way to delete rows](./alter/delete.md) in ClickHouse is `ALTER TABLE ... DELETE`, which might be more efficient if you do bulk deletes only occasionally and don't need the operation to be applied instantly. In most use cases the new lightweight `DELETE FROM` behavior will be considerably faster. - -:::warning -Even though deletes are becoming more lightweight in ClickHouse, they should still not be used as aggressively as on an OLTP system. Lightweight deletes are currently efficient for wide parts, but for compact parts, they can be a heavyweight operation, and it may be better to use `ALTER TABLE` for some scenarios. -::: - :::note `DELETE FROM` requires the `ALTER DELETE` privilege: ```sql @@ -51,7 +45,7 @@ The idea behind Lightweight Delete is that when a `DELETE FROM table ...` query The mask is implemented as a hidden `_row_exists` system column that stores True for all visible rows and False for deleted ones. This column is only present in a part if some rows in this part were deleted. In other words, the column is not persisted when it has all values equal to True. ## SELECT query -When the column is present `SELECT ... FROM table WHERE condition` query internally is extended by an additional predicate on `_row_exists` and becomes similar to +When the column is present `SELECT ... FROM table WHERE condition` query internally is extended by an additional predicate on `_row_exists` and becomes similar to ```sql SELECT ... FROM table PREWHERE _row_exists WHERE condition ``` diff --git a/docs/en/sql-reference/statements/detach.md b/docs/en/sql-reference/statements/detach.md index aa87b1ef613..5f1513d3f44 100644 --- a/docs/en/sql-reference/statements/detach.md +++ b/docs/en/sql-reference/statements/detach.md @@ -22,7 +22,7 @@ System log tables can be also attached back (e.g. `query_log`, `text_log`, etc). Note that you can not detach permanently the table which is already detached (temporary). But you can attach it back and then detach permanently again. -Also you can not [DROP](../../sql-reference/statements/drop#drop-table) the detached table, or [CREATE TABLE](../../sql-reference/statements/create/table.md) with the same name as detached permanently, or replace it with the other table with [RENAME TABLE](../../sql-reference/statements/rename.md) query. +Also you can not [DROP](../../sql-reference/statements/drop.md#drop-table) the detached table, or [CREATE TABLE](../../sql-reference/statements/create/table.md) with the same name as detached permanently, or replace it with the other table with [RENAME TABLE](../../sql-reference/statements/rename.md) query. The `SYNC` modifier executes the action without delay. diff --git a/docs/en/sql-reference/statements/insert-into.md b/docs/en/sql-reference/statements/insert-into.md index f2d590d196b..354ab95c598 100644 --- a/docs/en/sql-reference/statements/insert-into.md +++ b/docs/en/sql-reference/statements/insert-into.md @@ -4,7 +4,7 @@ sidebar_position: 33 sidebar_label: INSERT INTO --- -# INSERT INTO Statement +# INSERT INTO Statement Inserts data into a table. @@ -89,7 +89,7 @@ INSERT INTO t FORMAT TabSeparated 22 Qwerty ``` -You can insert data separately from the query by using the command-line client or the HTTP interface. For more information, see the section “[Interfaces](../../interfaces)”. +You can insert data separately from the query by using the [command-line client](/docs/en/integrations/sql-clients/clickhouse-client-local) or the [HTTP interface](/docs/en/interfaces/http/). :::note If you want to specify `SETTINGS` for `INSERT` query then you have to do it _before_ `FORMAT` clause since everything after `FORMAT format_name` is treated as data. For example: @@ -129,7 +129,7 @@ To insert a default value instead of `NULL` into a column with not nullable data INSERT INTO [db.]table [(c1, c2, c3)] FROM INFILE file_name [COMPRESSION type] FORMAT format_name ``` -Use the syntax above to insert data from a file, or files, stored on the **client** side. `file_name` and `type` are string literals. Input file [format](../../interfaces/formats.md) must be set in the `FORMAT` clause. +Use the syntax above to insert data from a file, or files, stored on the **client** side. `file_name` and `type` are string literals. Input file [format](../../interfaces/formats.md) must be set in the `FORMAT` clause. Compressed files are supported. The compression type is detected by the extension of the file name. Or it can be explicitly specified in a `COMPRESSION` clause. Supported types are: `'none'`, `'gzip'`, `'deflate'`, `'br'`, `'xz'`, `'zstd'`, `'lz4'`, `'bz2'`. @@ -191,7 +191,7 @@ INSERT INTO [TABLE] FUNCTION table_func ... ``` sql CREATE TABLE simple_table (id UInt32, text String) ENGINE=MergeTree() ORDER BY id; -INSERT INTO TABLE FUNCTION remote('localhost', default.simple_table) +INSERT INTO TABLE FUNCTION remote('localhost', default.simple_table) VALUES (100, 'inserted via remote()'); SELECT * FROM simple_table; ``` diff --git a/docs/en/sql-reference/statements/select/array-join.md b/docs/en/sql-reference/statements/select/array-join.md index a1b5e0cdb36..b8e6be24798 100644 --- a/docs/en/sql-reference/statements/select/array-join.md +++ b/docs/en/sql-reference/statements/select/array-join.md @@ -146,7 +146,7 @@ ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num, arrayMap(x -> x + 1, arr) AS ma └───────┴─────────┴───┴─────┴────────┘ ``` -The example below uses the [arrayEnumerate](../../../sql-reference/functions/array-functions#array_functions-arrayenumerate) function: +The example below uses the [arrayEnumerate](../../../sql-reference/functions/array-functions.md#array_functions-arrayenumerate) function: ``` sql SELECT s, arr, a, num, arrayEnumerate(arr) @@ -166,8 +166,8 @@ ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num; Multiple arrays with different sizes can be joined by using: `SETTINGS enable_unaligned_array_join = 1`. Example: ```sql -SELECT s, arr, a, b -FROM arrays_test ARRAY JOIN arr as a, [['a','b'],['c']] as b +SELECT s, arr, a, b +FROM arrays_test ARRAY JOIN arr as a, [['a','b'],['c']] as b SETTINGS enable_unaligned_array_join = 1; ``` @@ -278,7 +278,7 @@ ARRAY JOIN nest AS n; └───────┴─────┴─────┴─────────┴────────────┘ ``` -Example of using the [arrayEnumerate](../../../sql-reference/functions/array-functions#array_functions-arrayenumerate) function: +Example of using the [arrayEnumerate](../../../sql-reference/functions/array-functions.md#array_functions-arrayenumerate) function: ``` sql SELECT s, `n.x`, `n.y`, `nest.x`, `nest.y`, num diff --git a/docs/en/sql-reference/statements/select/group-by.md b/docs/en/sql-reference/statements/select/group-by.md index 2a4b06660c7..1018b24f50b 100644 --- a/docs/en/sql-reference/statements/select/group-by.md +++ b/docs/en/sql-reference/statements/select/group-by.md @@ -8,12 +8,12 @@ sidebar_label: GROUP BY `GROUP BY` clause switches the `SELECT` query into an aggregation mode, which works as follows: - `GROUP BY` clause contains a list of expressions (or a single expression, which is considered to be the list of length one). This list acts as a “grouping key”, while each individual expression will be referred to as a “key expression”. -- All the expressions in the [SELECT](../../../sql-reference/statements/select/index.md), [HAVING](../../../sql-reference/statements/select/having), and [ORDER BY](../../../sql-reference/statements/select/order-by.md) clauses **must** be calculated based on key expressions **or** on [aggregate functions](../../../sql-reference/aggregate-functions/index.md) over non-key expressions (including plain columns). In other words, each column selected from the table must be used either in a key expression or inside an aggregate function, but not both. +- All the expressions in the [SELECT](../../../sql-reference/statements/select/index.md), [HAVING](../../../sql-reference/statements/select/having.md), and [ORDER BY](../../../sql-reference/statements/select/order-by.md) clauses **must** be calculated based on key expressions **or** on [aggregate functions](../../../sql-reference/aggregate-functions/index.md) over non-key expressions (including plain columns). In other words, each column selected from the table must be used either in a key expression or inside an aggregate function, but not both. - Result of aggregating `SELECT` query will contain as many rows as there were unique values of “grouping key” in source table. Usually, this significantly reduces the row count, often by orders of magnitude, but not necessarily: row count stays the same if all “grouping key” values were distinct. When you want to group data in the table by column numbers instead of column names, enable the setting [enable_positional_arguments](../../../operations/settings/settings.md#enable-positional-arguments). -:::note +:::note There’s an additional way to run aggregation over a table. If a query contains table columns only inside aggregate functions, the `GROUP BY clause` can be omitted, and aggregation by an empty set of keys is assumed. Such queries always return exactly one row. ::: @@ -57,8 +57,8 @@ The subtotals are calculated in the reverse order: at first subtotals are calcul In the subtotals rows the values of already "grouped" key expressions are set to `0` or empty line. -:::note -Mind that [HAVING](../../../sql-reference/statements/select/having) clause can affect the subtotals results. +:::note +Mind that [HAVING](../../../sql-reference/statements/select/having.md) clause can affect the subtotals results. ::: **Example** @@ -125,8 +125,8 @@ SELECT year, month, day, count(*) FROM t GROUP BY year, month, day WITH ROLLUP; In the subtotals rows the values of all "grouped" key expressions are set to `0` or empty line. -:::note -Mind that [HAVING](../../../sql-reference/statements/select/having) clause can affect the subtotals results. +:::note +Mind that [HAVING](../../../sql-reference/statements/select/having.md) clause can affect the subtotals results. ::: **Example** @@ -226,11 +226,11 @@ This extra row is only produced in `JSON*`, `TabSeparated*`, and `Pretty*` forma - In `Template` format, the row is output according to specified template. - In the other formats it is not available. -:::note -totals is output in the results of `SELECT` queries, and is not output in `INSERT INTO ... SELECT`. +:::note +totals is output in the results of `SELECT` queries, and is not output in `INSERT INTO ... SELECT`. ::: -`WITH TOTALS` can be run in different ways when [HAVING](../../../sql-reference/statements/select/having) is present. The behavior depends on the `totals_mode` setting. +`WITH TOTALS` can be run in different ways when [HAVING](../../../sql-reference/statements/select/having.md) is present. The behavior depends on the `totals_mode` setting. ### Configuring Totals Processing diff --git a/docs/en/sql-reference/statements/select/index.md b/docs/en/sql-reference/statements/select/index.md index 5a8893f6f28..f65e40dede5 100644 --- a/docs/en/sql-reference/statements/select/index.md +++ b/docs/en/sql-reference/statements/select/index.md @@ -4,7 +4,7 @@ sidebar_position: 32 sidebar_label: SELECT --- -# SELECT Query +# SELECT Query `SELECT` queries perform data retrieval. By default, the requested data is returned to the client, while in conjunction with [INSERT INTO](../../../sql-reference/statements/insert-into.md) it can be forwarded to a different table. @@ -44,7 +44,7 @@ Specifics of each optional clause are covered in separate sections, which are li - [WHERE clause](../../../sql-reference/statements/select/where.md) - [GROUP BY clause](../../../sql-reference/statements/select/group-by.md) - [LIMIT BY clause](../../../sql-reference/statements/select/limit-by.md) -- [HAVING clause](../../../sql-reference/statements/select/having) +- [HAVING clause](../../../sql-reference/statements/select/having.md) - [LIMIT clause](../../../sql-reference/statements/select/limit.md) - [OFFSET clause](../../../sql-reference/statements/select/offset.md) - [UNION clause](../../../sql-reference/statements/select/union.md) diff --git a/docs/en/sql-reference/statements/select/join.md b/docs/en/sql-reference/statements/select/join.md index 62d3e9fd69a..49bd2672874 100644 --- a/docs/en/sql-reference/statements/select/join.md +++ b/docs/en/sql-reference/statements/select/join.md @@ -1,6 +1,6 @@ --- slug: /en/sql-reference/statements/select/join -sidebar_label: JOIN +sidebar_label: Joining Tables --- # JOIN Clause @@ -282,7 +282,7 @@ Each time a query is run with the same `JOIN`, the subquery is run again because In some cases, it is more efficient to use [IN](../../../sql-reference/operators/in.md) instead of `JOIN`. -If you need a `JOIN` for joining with dimension tables (these are relatively small tables that contain dimension properties, such as names for advertising campaigns), a `JOIN` might not be very convenient due to the fact that the right table is re-accessed for every query. For such cases, there is a “dictionaries” feature that you should use instead of `JOIN`. For more information, see the [Dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) section. +If you need a `JOIN` for joining with dimension tables (these are relatively small tables that contain dimension properties, such as names for advertising campaigns), a `JOIN` might not be very convenient due to the fact that the right table is re-accessed for every query. For such cases, there is a “dictionaries” feature that you should use instead of `JOIN`. For more information, see the [Dictionaries](../../../sql-reference/dictionaries/index.md) section. ### Memory Limitations diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index 18b019dd017..a9f0aedccdf 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -198,7 +198,7 @@ Result: ## SHOW DICTIONARIES -Displays a list of [Dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md). +Displays a list of [Dictionaries](../../sql-reference/dictionaries/index.md). ``` sql SHOW DICTIONARIES [FROM ] [LIKE ''] [LIMIT ] [INTO OUTFILE ] [FORMAT ] @@ -293,7 +293,7 @@ SHOW CREATE [SETTINGS] PROFILE name1 [, name2 ...] ## SHOW USERS -Returns a list of [user account](../../operations/access-rights.md#user-account-management) names. To view user accounts parameters, see the system table [system.users](../../operations/system-tables/users.md#system_tables-users). +Returns a list of [user account](../../guides/sre/user-management/index.md#user-account-management) names. To view user accounts parameters, see the system table [system.users](../../operations/system-tables/users.md#system_tables-users). ### Syntax @@ -303,7 +303,7 @@ SHOW USERS ## SHOW ROLES -Returns a list of [roles](../../operations/access-rights.md#role-management). To view another parameters, see system tables [system.roles](../../operations/system-tables/roles.md#system_tables-roles) and [system.role_grants](../../operations/system-tables/role-grants.md#system_tables-role_grants). +Returns a list of [roles](../../guides/sre/user-management/index.md#role-management). To view another parameters, see system tables [system.roles](../../operations/system-tables/roles.md#system_tables-roles) and [system.role_grants](../../operations/system-tables/role-grants.md#system_tables-role_grants). ### Syntax @@ -312,7 +312,7 @@ SHOW [CURRENT|ENABLED] ROLES ``` ## SHOW PROFILES -Returns a list of [setting profiles](../../operations/access-rights.md#settings-profiles-management). To view user accounts parameters, see the system table [settings_profiles](../../operations/system-tables/settings_profiles.md#system_tables-settings_profiles). +Returns a list of [setting profiles](../../guides/sre/user-management/index.md#settings-profiles-management). To view user accounts parameters, see the system table [settings_profiles](../../operations/system-tables/settings_profiles.md#system_tables-settings_profiles). ### Syntax @@ -322,7 +322,7 @@ SHOW [SETTINGS] PROFILES ## SHOW POLICIES -Returns a list of [row policies](../../operations/access-rights.md#row-policy-management) for the specified table. To view user accounts parameters, see the system table [system.row_policies](../../operations/system-tables/row_policies.md#system_tables-row_policies). +Returns a list of [row policies](../../guides/sre/user-management/index.md#row-policy-management) for the specified table. To view user accounts parameters, see the system table [system.row_policies](../../operations/system-tables/row_policies.md#system_tables-row_policies). ### Syntax @@ -332,7 +332,7 @@ SHOW [ROW] POLICIES [ON [db.]table] ## SHOW QUOTAS -Returns a list of [quotas](../../operations/access-rights.md#quotas-management). To view quotas parameters, see the system table [system.quotas](../../operations/system-tables/quotas.md#system_tables-quotas). +Returns a list of [quotas](../../guides/sre/user-management/index.md#quotas-management). To view quotas parameters, see the system table [system.quotas](../../operations/system-tables/quotas.md#system_tables-quotas). ### Syntax @@ -351,7 +351,7 @@ SHOW [CURRENT] QUOTA ``` ## SHOW ACCESS -Shows all [users](../../operations/access-rights.md#user-account-management), [roles](../../operations/access-rights.md#role-management), [profiles](../../operations/access-rights.md#settings-profiles-management), etc. and all their [grants](../../sql-reference/statements/grant.md#grant-privileges). +Shows all [users](../../guides/sre/user-management/index.md#user-account-management), [roles](../../guides/sre/user-management/index.md#role-management), [profiles](../../guides/sre/user-management/index.md#settings-profiles-management), etc. and all their [grants](../../sql-reference/statements/grant.md#grant-privileges). ### Syntax diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index f9f55acfcec..101e7c72bcb 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -8,7 +8,7 @@ sidebar_label: SYSTEM ## RELOAD EMBEDDED DICTIONARIES -Reload all [Internal dictionaries](../../sql-reference/dictionaries/internal-dicts.md). +Reload all [Internal dictionaries](../../sql-reference/dictionaries/index.md). By default, internal dictionaries are disabled. Always returns `Ok.` regardless of the result of the internal dictionary update. @@ -369,7 +369,7 @@ SYSTEM DROP FILESYSTEM CACHE It's too heavy and has potential for misuse. ::: -Will do sync syscall. +Will do sync syscall. ```sql SYSTEM SYNC FILE CACHE diff --git a/docs/en/sql-reference/table-functions/dictionary.md b/docs/en/sql-reference/table-functions/dictionary.md index 8a8cba8ff24..ab511843d63 100644 --- a/docs/en/sql-reference/table-functions/dictionary.md +++ b/docs/en/sql-reference/table-functions/dictionary.md @@ -5,7 +5,7 @@ sidebar_label: dictionary function title: dictionary --- -Displays the [dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) data as a ClickHouse table. Works the same way as [Dictionary](../../engines/table-engines/special/dictionary.md) engine. +Displays the [dictionary](../../sql-reference/dictionaries/index.md) data as a ClickHouse table. Works the same way as [Dictionary](../../engines/table-engines/special/dictionary.md) engine. **Syntax** diff --git a/docs/en/sql-reference/table-functions/executable.md b/docs/en/sql-reference/table-functions/executable.md index 635188763cf..22c74eb8cfa 100644 --- a/docs/en/sql-reference/table-functions/executable.md +++ b/docs/en/sql-reference/table-functions/executable.md @@ -85,7 +85,7 @@ The response looks like: ## Passing Query Results to a Script -Be sure to check out the example in the `Executable` table engine on [how to pass query results to a script](../../engines/table-engines/special/executable#passing-query-results-to-a-script). Here is how you execute the same script in that example using the `executable` table function: +Be sure to check out the example in the `Executable` table engine on [how to pass query results to a script](../../engines/table-engines/special/executable.md#passing-query-results-to-a-script). Here is how you execute the same script in that example using the `executable` table function: ```sql SELECT * FROM executable( diff --git a/docs/en/sql-reference/table-functions/mongodb.md b/docs/en/sql-reference/table-functions/mongodb.md index dd063ae1796..706ab68fee4 100644 --- a/docs/en/sql-reference/table-functions/mongodb.md +++ b/docs/en/sql-reference/table-functions/mongodb.md @@ -70,5 +70,5 @@ SELECT * FROM mongodb( **See Also** -- [The `MongoDB` table engine](../../engines/table-engines/integrations/mongodb.md) -- [Using MongoDB as a dictionary source](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources/#mongodb) +- [The `MongoDB` table engine](/docs/en/engines/table-engines/integrations/mongodb.md) +- [Using MongoDB as a dictionary source](/docs/en/sql-reference/dictionaries/index.md#mongodb) diff --git a/docs/en/sql-reference/table-functions/mysql.md b/docs/en/sql-reference/table-functions/mysql.md index b995319c645..64ddcd86f7f 100644 --- a/docs/en/sql-reference/table-functions/mysql.md +++ b/docs/en/sql-reference/table-functions/mysql.md @@ -56,7 +56,7 @@ SELECT name FROM mysql(`mysql1:3306|mysql2:3306|mysql3:3306`, 'mysql_database', A table object with the same columns as the original MySQL table. -:::note +:::note In the `INSERT` query to distinguish table function `mysql(...)` from table name with column names list, you must use keywords `FUNCTION` or `TABLE FUNCTION`. See examples below. ::: @@ -110,4 +110,4 @@ SELECT * FROM mysql('localhost:3306', 'test', 'test', 'bayonet', '123'); **See Also** - [The ‘MySQL’ table engine](../../engines/table-engines/integrations/mysql.md) -- [Using MySQL as a dictionary source](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-mysql) +- [Using MySQL as a dictionary source](../../sql-reference/dictionaries/index.md#dictionary-sources#dicts-external_dicts_dict_sources-mysql) diff --git a/docs/en/sql-reference/table-functions/odbc.md b/docs/en/sql-reference/table-functions/odbc.md index 7e13424bc8a..397a9ba6c89 100644 --- a/docs/en/sql-reference/table-functions/odbc.md +++ b/docs/en/sql-reference/table-functions/odbc.md @@ -101,5 +101,5 @@ SELECT * FROM odbc('DSN=mysqlconn', 'test', 'test') ## See Also -- [ODBC dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-odbc) +- [ODBC dictionaries](../../sql-reference/dictionaries/index.md#dictionary-sources#dicts-external_dicts_dict_sources-odbc) - [ODBC table engine](../../engines/table-engines/integrations/odbc.md). diff --git a/docs/en/sql-reference/table-functions/postgresql.md b/docs/en/sql-reference/table-functions/postgresql.md index 87fc6ecb234..6cd13acaa77 100644 --- a/docs/en/sql-reference/table-functions/postgresql.md +++ b/docs/en/sql-reference/table-functions/postgresql.md @@ -27,7 +27,7 @@ postgresql('host:port', 'database', 'table', 'user', 'password'[, `schema`]) A table object with the same columns as the original PostgreSQL table. -:::note +:::note In the `INSERT` query to distinguish table function `postgresql(...)` from table name with column names list you must use keywords `FUNCTION` or `TABLE FUNCTION`. See examples below. ::: @@ -43,7 +43,7 @@ All joins, aggregations, sorting, `IN [ array ]` conditions and the `LIMIT` samp PostgreSQL Array types converts into ClickHouse arrays. -:::note +:::note Be careful, in PostgreSQL an array data type column like Integer[] may contain arrays of different dimensions in different rows, but in ClickHouse it is only allowed to have multidimensional arrays of the same dimension in all rows. ::: @@ -130,7 +130,7 @@ CREATE TABLE pg_table_schema_with_dots (a UInt32) **See Also** - [The PostgreSQL table engine](../../engines/table-engines/integrations/postgresql.md) -- [Using PostgreSQL as a dictionary source](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-postgresql) +- [Using PostgreSQL as a dictionary source](../../sql-reference/dictionaries/index.md#dictionary-sources#dicts-external_dicts_dict_sources-postgresql) ## Related content - Blog: [ClickHouse and PostgreSQL - a match made in data heaven - part 1](https://clickhouse.com/blog/migrating-data-between-clickhouse-postgres) diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index c214e08dce1..df56b7a0bd6 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -97,7 +97,7 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', ### DDL-запросы {#ddl-queries} -DDL-запросы в MySQL конвертируются в соответствующие DDL-запросы в ClickHouse ([ALTER](../../sql-reference/statements/alter/index.md), [CREATE](../../sql-reference/statements/create/index.md), [DROP](../../sql-reference/statements/drop), [RENAME](../../sql-reference/statements/rename.md)). Если ClickHouse не может конвертировать какой-либо DDL-запрос, он его игнорирует. +DDL-запросы в MySQL конвертируются в соответствующие DDL-запросы в ClickHouse ([ALTER](../../sql-reference/statements/alter/index.md), [CREATE](../../sql-reference/statements/create/index.md), [DROP](../../sql-reference/statements/drop.md), [RENAME](../../sql-reference/statements/rename.md)). Если ClickHouse не может конвертировать какой-либо DDL-запрос, он его игнорирует. ### Репликация данных {#data-replication} diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 24e0f8dbbb8..ef17a370dc6 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -89,7 +89,7 @@ ORDER BY expr - `min_merge_bytes_to_use_direct_io` — минимальный объём данных при слиянии, необходимый для прямого (небуферизованного) чтения/записи (direct I/O) на диск. При слиянии частей данных ClickHouse вычисляет общий объём хранения всех данных, подлежащих слиянию. Если общий объём хранения всех данных для чтения превышает `min_bytes_to_use_direct_io` байт, тогда ClickHouse использует флаг `O_DIRECT` при чтении данных с диска. Если `min_merge_bytes_to_use_direct_io = 0`, тогда прямой ввод-вывод отключен. Значение по умолчанию: `10 * 1024 * 1024 * 1024` байтов. - `merge_with_ttl_timeout` — минимальное время в секундах перед повторным слиянием для удаления данных с истекшим TTL. По умолчанию: `14400` секунд (4 часа). - `merge_with_recompression_ttl_timeout` — минимальное время в секундах перед повторным слиянием для повторного сжатия данных с истекшим TTL. По умолчанию: `14400` секунд (4 часа). - - `try_fetch_recompressed_part_timeout` — время ожидания (в секундах) перед началом слияния с повторным сжатием. В течение этого времени ClickHouse пытается извлечь сжатую часть из реплики, которая назначила это слияние. Значение по умолчанию: `7200` секунд (2 часа). + - `try_fetch_recompressed_part_timeout` — время ожидания (в секундах) перед началом слияния с повторным сжатием. В течение этого времени ClickHouse пытается извлечь сжатую часть из реплики, которая назначила это слияние. Значение по умолчанию: `7200` секунд (2 часа). - `write_final_mark` — включает или отключает запись последней засечки индекса в конце куска данных, указывающей за последний байт. По умолчанию — 1. Не отключайте её. - `merge_max_block_size` — максимальное количество строк в блоке для операций слияния. Значение по умолчанию: 8192. - `storage_policy` — политика хранения данных. Смотрите [Хранение данных таблицы на нескольких блочных устройствах](#table_engine-mergetree-multiple-volumes). @@ -337,7 +337,7 @@ SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234 Поддерживаемые типы данных: `Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`. - Фильтром могут пользоваться функции: [equals](../../../sql-reference/functions/comparison-functions.md), [notEquals](../../../sql-reference/functions/comparison-functions.md), [in](../../../sql-reference/functions/in-functions), [notIn](../../../sql-reference/functions/in-functions), [has](../../../sql-reference/functions/array-functions#hasarr-elem), [hasAny](../../../sql-reference/functions/array-functions#hasany), [hasAll](../../../sql-reference/functions/array-functions#hasall). + Фильтром могут пользоваться функции: [equals](../../../sql-reference/functions/comparison-functions.md), [notEquals](../../../sql-reference/functions/comparison-functions.md), [in](../../../sql-reference/functions/in-functions.md), [notIn](../../../sql-reference/functions/in-functions.md), [has](../../../sql-reference/functions/array-functions.md#hasarr-elem), [hasAny](../../../sql-reference/functions/array-functions.md#hasany), [hasAll](../../../sql-reference/functions/array-functions.md#hasall). **Примеры** @@ -361,14 +361,14 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT | [startsWith](../../../sql-reference/functions/string-functions.md#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | | [endsWith](../../../sql-reference/functions/string-functions.md#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ | | [multiSearchAny](../../../sql-reference/functions/string-search-functions.md#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | -| [in](../../../sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [notIn](../../../sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [in](../../../sql-reference/functions/in-functions.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [notIn](../../../sql-reference/functions/in-functions.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | | [less (\<)](../../../sql-reference/functions/comparison-functions.md#function-less) | ✔ | ✔ | ✗ | ✗ | ✗ | | [greater (\>)](../../../sql-reference/functions/comparison-functions.md#function-greater) | ✔ | ✔ | ✗ | ✗ | ✗ | | [lessOrEquals (\<=)](../../../sql-reference/functions/comparison-functions.md#function-lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | | [greaterOrEquals (\>=)](../../../sql-reference/functions/comparison-functions.md#function-greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [empty](../../../sql-reference/functions/array-functions#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [notEmpty](../../../sql-reference/functions/array-functions#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [empty](../../../sql-reference/functions/array-functions.md#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [notEmpty](../../../sql-reference/functions/array-functions.md#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | | hasToken | ✗ | ✗ | ✗ | ✔ | ✗ | Функции с постоянным агрументом, который меньше, чем размер ngram не могут использовать индекс `ngrambf_v1` для оптимизации запроса. @@ -396,7 +396,7 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT Проекции не поддерживаются для запросов `SELECT` с модификатором [FINAL](../../../sql-reference/statements/select/from.md#select-from-final). ### Запрос проекции {#projection-query} -Запрос проекции — это то, что определяет проекцию. Такой запрос неявно выбирает данные из родительской таблицы. +Запрос проекции — это то, что определяет проекцию. Такой запрос неявно выбирает данные из родительской таблицы. **Синтаксис** ```sql @@ -406,9 +406,9 @@ SELECT [GROUP BY] [ORDER BY] Проекции можно изменить или удалить с помощью запроса [ALTER](../../../sql-reference/statements/alter/projection.md). ### Хранение проекции {#projection-storage} -Проекции хранятся в каталоге куска данных. Это похоже на хранение индексов, но используется подкаталог, в котором хранится анонимный кусок таблицы `MergeTree`. Таблица создается запросом определения проекции. -Если присутствует секция `GROUP BY`, то используется движок [AggregatingMergeTree](aggregatingmergetree.md), а все агрегатные функции преобразуются в `AggregateFunction`. -Если присутствует секция `ORDER BY`, таблица `MergeTree` использует ее в качестве выражения для первичного ключа. +Проекции хранятся в каталоге куска данных. Это похоже на хранение индексов, но используется подкаталог, в котором хранится анонимный кусок таблицы `MergeTree`. Таблица создается запросом определения проекции. +Если присутствует секция `GROUP BY`, то используется движок [AggregatingMergeTree](aggregatingmergetree.md), а все агрегатные функции преобразуются в `AggregateFunction`. +Если присутствует секция `ORDER BY`, таблица `MergeTree` использует ее в качестве выражения для первичного ключа. Во время процесса слияния кусок данных проекции объединяется с помощью процедуры слияния хранилища. Контрольная сумма куска данных родительской таблицы включает кусок данных проекции. Другие процедуры аналогичны индексам пропуска данных. ### Анализ запросов {#projection-query-analysis} @@ -499,7 +499,7 @@ TTL expr За каждым `TTL` выражением может следовать тип действия, которое выполняется после достижения времени, соответствующего результату `TTL` выражения: - `DELETE` - удалить данные (действие по умолчанию); -- `RECOMPRESS codec_name` - повторно сжать данные с помощью кодека `codec_name`; +- `RECOMPRESS codec_name` - повторно сжать данные с помощью кодека `codec_name`; - `TO DISK 'aaa'` - переместить данные на диск `aaa`; - `TO VOLUME 'bbb'` - переместить данные на том `bbb`; - `GROUP BY` - агрегировать данные. @@ -679,7 +679,7 @@ TTL d + INTERVAL 1 MONTH GROUP BY k1, k2 SET x = max(x), y = min(y); - `policy_name_N` — название политики. Названия политик должны быть уникальны. - `volume_name_N` — название тома. Названия томов должны быть уникальны. - `disk` — диск, находящийся внутри тома. -- `max_data_part_size_bytes` — максимальный размер куска данных, который может находиться на любом из дисков этого тома. Если в результате слияния размер куска ожидается больше, чем max_data_part_size_bytes, то этот кусок будет записан в следующий том. В основном эта функция позволяет хранить новые / мелкие куски на горячем (SSD) томе и перемещать их на холодный (HDD) том, когда они достигают большого размера. Не используйте этот параметр, если политика имеет только один том. +- `max_data_part_size_bytes` — максимальный размер куска данных, который может находиться на любом из дисков этого тома. Если в результате слияния размер куска ожидается больше, чем max_data_part_size_bytes, то этот кусок будет записан в следующий том. В основном эта функция позволяет хранить новые / мелкие куски на горячем (SSD) томе и перемещать их на холодный (HDD) том, когда они достигают большого размера. Не используйте этот параметр, если политика имеет только один том. - `move_factor` — доля доступного свободного места на томе, если места становится меньше, то данные начнут перемещение на следующий том, если он есть (по умолчанию 0.1). Для перемещения куски сортируются по размеру от большего к меньшему (по убыванию) и выбираются куски, совокупный размер которых достаточен для соблюдения условия `move_factor`, если совокупный размер всех партов недостаточен, будут перемещены все парты. - `prefer_not_to_merge` — Отключает слияние кусков данных, хранящихся на данном томе. Если данная настройка включена, то слияние данных, хранящихся на данном томе, не допускается. Это позволяет контролировать работу ClickHouse с медленными дисками. diff --git a/docs/ru/engines/table-engines/special/buffer.md b/docs/ru/engines/table-engines/special/buffer.md index 4987dafc11f..574d9273088 100644 --- a/docs/ru/engines/table-engines/special/buffer.md +++ b/docs/ru/engines/table-engines/special/buffer.md @@ -66,4 +66,4 @@ CREATE TABLE merge.hits_buffer AS merge.hits ENGINE = Buffer(merge, hits, 16, 10 Таблицы типа Buffer используются в тех случаях, когда от большого количества серверов поступает слишком много INSERT-ов в единицу времени, и нет возможности заранее самостоятельно буферизовать данные перед вставкой, в результате чего, INSERT-ы не успевают выполняться. -Заметим, что даже для таблиц типа Buffer не имеет смысла вставлять данные по одной строке, так как таким образом будет достигнута скорость всего лишь в несколько тысяч строк в секунду, тогда как при вставке более крупными блоками, достижимо более миллиона строк в секунду (смотрите раздел [«Производительность»](../../../introduction/performance/). +Заметим, что даже для таблиц типа Buffer не имеет смысла вставлять данные по одной строке, так как таким образом будет достигнута скорость всего лишь в несколько тысяч строк в секунду, тогда как при вставке более крупными блоками, достижимо более миллиона строк в секунду (смотрите раздел [«Производительность»](../../../introduction/performance.md). diff --git a/docs/ru/faq/operations/multi-region-replication.md b/docs/ru/faq/operations/multi-region-replication.md index bfe3231c247..eb53a69e7f6 100644 --- a/docs/ru/faq/operations/multi-region-replication.md +++ b/docs/ru/faq/operations/multi-region-replication.md @@ -10,4 +10,4 @@ The short answer is "yes". However, we recommend keeping latency between all reg Configuration-wise there's no difference compared to single-region replication, simply use hosts that are located in different locations for replicas. -For more information, see [full article on data replication](../../engines/table-engines/mergetree-family/replication/). +For more information, see [full article on data replication](../../engines/table-engines/mergetree-family/replication.md). diff --git a/docs/ru/getting-started/tutorial.md b/docs/ru/getting-started/tutorial.md index 803da2952fd..60a7463f70f 100644 --- a/docs/ru/getting-started/tutorial.md +++ b/docs/ru/getting-started/tutorial.md @@ -477,7 +477,7 @@ clickhouse-client --query "INSERT INTO tutorial.hits_v1 FORMAT TSV" --max_insert clickhouse-client --query "INSERT INTO tutorial.visits_v1 FORMAT TSV" --max_insert_block_size=100000 < visits_v1.tsv ``` -ClickHouse has a lot of [settings to tune](../operations/settings/) and one way to specify them in console client is via arguments, as we can see with `--max_insert_block_size`. The easiest way to figure out what settings are available, what do they mean and what the defaults are is to query the `system.settings` table: +ClickHouse has a lot of [settings to tune](../operations/settings/index.md) and one way to specify them in console client is via arguments, as we can see with `--max_insert_block_size`. The easiest way to figure out what settings are available, what do they mean and what the defaults are is to query the `system.settings` table: ``` sql SELECT name, value, changed, description diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 59c77d082cf..bef5c223281 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -974,7 +974,7 @@ Array представлены как длина в формате varint (unsig столбцы из входных данных будут сопоставлены со столбцами таблицы по их именам, столбцы с неизвестными именами будут пропущены, если включен параметр [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields). В противном случае первая строка будет пропущена. ::: - + ## RowBinaryWithNamesAndTypes {#rowbinarywithnamesandtypes} То же самое что [RowBinary](#rowbinary), но добавляется заголовок: @@ -1326,7 +1326,7 @@ ClickHouse поддерживает настраиваемую точность Неподдерживаемые типы данных Parquet: `TIME32`, `FIXED_SIZE_BINARY`, `JSON`, `UUID`, `ENUM`. -Типы данных столбцов в ClickHouse могут отличаться от типов данных соответствующих полей файла в формате Parquet. При вставке данных ClickHouse интерпретирует типы данных в соответствии с таблицей выше, а затем [приводит](../sql-reference/functions/type-conversion-functions/#type_conversion_function-cast) данные к тому типу, который установлен для столбца таблицы. +Типы данных столбцов в ClickHouse могут отличаться от типов данных соответствующих полей файла в формате Parquet. При вставке данных ClickHouse интерпретирует типы данных в соответствии с таблицей выше, а затем [приводит](../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) данные к тому типу, который установлен для столбца таблицы. ### Вставка и выборка данных {#inserting-and-selecting-data} @@ -1386,7 +1386,7 @@ ClickHouse поддерживает настраиваемую точность Неподдерживаемые типы данных Arrow: `TIME32`, `FIXED_SIZE_BINARY`, `JSON`, `UUID`, `ENUM`. -Типы данных столбцов в ClickHouse могут отличаться от типов данных соответствующих полей файла в формате Arrow. При вставке данных ClickHouse интерпретирует типы данных в соответствии с таблицей выше, а затем [приводит](../sql-reference/functions/type-conversion-functions/#type_conversion_function-cast) данные к тому типу, который установлен для столбца таблицы. +Типы данных столбцов в ClickHouse могут отличаться от типов данных соответствующих полей файла в формате Arrow. При вставке данных ClickHouse интерпретирует типы данных в соответствии с таблицей выше, а затем [приводит](../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) данные к тому типу, который установлен для столбца таблицы. ### Вставка данных {#inserting-data-arrow} @@ -1444,7 +1444,7 @@ ClickHouse поддерживает настраиваемую точность Неподдерживаемые типы данных ORC: `TIME32`, `FIXED_SIZE_BINARY`, `JSON`, `UUID`, `ENUM`. -Типы данных столбцов в таблицах ClickHouse могут отличаться от типов данных для соответствующих полей ORC. При вставке данных ClickHouse интерпретирует типы данных ORC согласно таблице соответствия, а затем [приводит](../sql-reference/functions/type-conversion-functions/#type_conversion_function-cast) данные к типу, установленному для столбца таблицы ClickHouse. +Типы данных столбцов в таблицах ClickHouse могут отличаться от типов данных для соответствующих полей ORC. При вставке данных ClickHouse интерпретирует типы данных ORC согласно таблице соответствия, а затем [приводит](../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) данные к типу, установленному для столбца таблицы ClickHouse. ### Вставка данных {#inserting-data-2} diff --git a/docs/ru/interfaces/http.md b/docs/ru/interfaces/http.md index 62e97e3f61d..b8c5ee77f0c 100644 --- a/docs/ru/interfaces/http.md +++ b/docs/ru/interfaces/http.md @@ -243,7 +243,7 @@ $ echo 'SELECT 1' | curl -H 'X-ClickHouse-User: user' -H 'X-ClickHouse-Key: pass Если пользователь не задан,то используется `default`. Если пароль не задан, то используется пустой пароль. Также в параметрах URL вы можете указать любые настройки, которые будут использованы для обработки одного запроса, или целые профили настроек. Пример:http://localhost:8123/?profile=web&max_rows_to_read=1000000000&query=SELECT+1 -Подробнее смотрите в разделе [Настройки](../operations/settings/). +Подробнее смотрите в разделе [Настройки](../operations/settings/index.md). ``` bash $ echo 'SELECT number FROM system.numbers LIMIT 10' | curl 'http://localhost:8123/?' --data-binary @- diff --git a/docs/ru/operations/optimizing-performance/sampling-query-profiler.md b/docs/ru/operations/optimizing-performance/sampling-query-profiler.md index c77f6a1f290..3d5ec993fdf 100644 --- a/docs/ru/operations/optimizing-performance/sampling-query-profiler.md +++ b/docs/ru/operations/optimizing-performance/sampling-query-profiler.md @@ -30,7 +30,7 @@ To analyze the `trace_log` system table: - Use the `addressToLine`, `addressToSymbol` and `demangle` [introspection functions](../../sql-reference/functions/introspection.md) to get function names and their positions in ClickHouse code. To get a profile for some query, you need to aggregate data from the `trace_log` table. You can aggregate data by individual functions or by the whole stack traces. -If you need to visualize `trace_log` info, try [flamegraph](../../interfaces/third-party/gui/#clickhouse-flamegraph) and [speedscope](https://github.com/laplab/clickhouse-speedscope). +If you need to visualize `trace_log` info, try [flamegraph](../../interfaces/third-party/gui.md#clickhouse-flamegraph) and [speedscope](https://github.com/laplab/clickhouse-speedscope). ## Example {#example} diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index e29b9def9d4..4b1d8ce717f 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -47,7 +47,7 @@ ClickHouse перезагружает встроенные словари с з - `min_part_size` - Минимальный размер части таблицы. - `min_part_size_ratio` - Отношение размера минимальной части таблицы к полному размеру таблицы. - `method` - Метод сжатия. Возможные значения: `lz4`, `lz4hc`, `zstd`,`deflate_qpl`. -- `level` – Уровень сжатия. См. [Кодеки](../../sql-reference/statements/create/table/#create-query-common-purpose-codecs). +- `level` – Уровень сжатия. См. [Кодеки](../../sql-reference/statements/create/table.md#create-query-common-purpose-codecs). Можно сконфигурировать несколько разделов ``. @@ -152,7 +152,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part ## custom_settings_prefixes {#custom_settings_prefixes} -Список префиксов для [пользовательских настроек](../../operations/settings/#custom_settings). Префиксы должны перечисляться через запятую. +Список префиксов для [пользовательских настроек](../../operations/settings/index.md#custom_settings). Префиксы должны перечисляться через запятую. **Пример** @@ -162,7 +162,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part **См. также** -- [Пользовательские настройки](../../operations/settings#custom_settings) +- [Пользовательские настройки](../../operations/settings/index.md#custom_settings) ## core_dump {#server_configuration_parameters-core_dump} diff --git a/docs/ru/operations/system-tables/information_schema.md b/docs/ru/operations/system-tables/information_schema.md index 6a9b8134dad..691fec19039 100644 --- a/docs/ru/operations/system-tables/information_schema.md +++ b/docs/ru/operations/system-tables/information_schema.md @@ -178,7 +178,7 @@ table_type: BASE TABLE - `view_definition` ([String](../../sql-reference/data-types/string.md)) — `SELECT` запрос для представления. - `check_option` ([String](../../sql-reference/data-types/string.md)) — `NONE`, нет проверки. - `is_updatable` ([Enum8](../../sql-reference/data-types/enum.md)) — `NO`, представление не обновляется. -- `is_insertable_into` ([Enum8](../../sql-reference/data-types/enum.md)) — показывает является ли представление [материализованным](../../sql-reference/statements/create/view/#materialized). Возможные значения: +- `is_insertable_into` ([Enum8](../../sql-reference/data-types/enum.md)) — показывает является ли представление [материализованным](../../sql-reference/statements/create/view.md#materialized). Возможные значения: - `NO` — создано обычное представление. - `YES` — создано материализованное представление. - `is_trigger_updatable` ([Enum8](../../sql-reference/data-types/enum.md)) — `NO`, триггер не обновляется. diff --git a/docs/ru/operations/system-tables/replicated_fetches.md b/docs/ru/operations/system-tables/replicated_fetches.md index 0b91a02cf14..c13f058aae1 100644 --- a/docs/ru/operations/system-tables/replicated_fetches.md +++ b/docs/ru/operations/system-tables/replicated_fetches.md @@ -68,4 +68,4 @@ thread_id: 54 **Смотрите также** -- [Управление таблицами ReplicatedMergeTree](../../sql-reference/statements/system/#query-language-system-replicated) +- [Управление таблицами ReplicatedMergeTree](../../sql-reference/statements/system.md#query-language-system-replicated) diff --git a/docs/ru/operations/utilities/clickhouse-benchmark.md b/docs/ru/operations/utilities/clickhouse-benchmark.md index d3185f4fcb0..73de78d1c15 100644 --- a/docs/ru/operations/utilities/clickhouse-benchmark.md +++ b/docs/ru/operations/utilities/clickhouse-benchmark.md @@ -60,7 +60,7 @@ clickhouse-benchmark [keys] < queries_file; - `--stage=WORD` — стадия обработки запроса на сервере. ClickHouse останавливает обработку запроса и возвращает ответ `clickhouse-benchmark` на заданной стадии. Возможные значения: `complete`, `fetch_columns`, `with_mergeable_state`. Значение по умолчанию: `complete`. - `--help` — показывает справку. -Если нужно применить [настройки](../../operations/settings/) для запросов, их можно передать как ключ `--= SETTING_VALUE`. Например, `--max_memory_usage=1048576`. +Если нужно применить [настройки](../../operations/settings/index.md) для запросов, их можно передать как ключ `--= SETTING_VALUE`. Например, `--max_memory_usage=1048576`. ## Вывод {#clickhouse-benchmark-output} diff --git a/docs/ru/sql-reference/data-types/datetime.md b/docs/ru/sql-reference/data-types/datetime.md index b513c51397e..e8d4a3ee9fd 100644 --- a/docs/ru/sql-reference/data-types/datetime.md +++ b/docs/ru/sql-reference/data-types/datetime.md @@ -27,9 +27,9 @@ DateTime([timezone]) Консольный клиент ClickHouse по умолчанию использует часовой пояс сервера, если для значения `DateTime` часовой пояс не был задан в явном виде при инициализации типа данных. Чтобы использовать часовой пояс клиента, запустите [clickhouse-client](../../interfaces/cli.md) с параметром `--use_client_time_zone`. -ClickHouse отображает значения в зависимости от значения параметра [date\_time\_output\_format](../../operations/settings/#settings-date_time_output_format). Текстовый формат по умолчанию `YYYY-MM-DD hh:mm:ss`. Кроме того, вы можете поменять отображение с помощью функции [formatDateTime](../../sql-reference/functions/date-time-functions.md#formatdatetime). +ClickHouse отображает значения в зависимости от значения параметра [date\_time\_output\_format](../../operations/settings/index.md#settings-date_time_output_format). Текстовый формат по умолчанию `YYYY-MM-DD hh:mm:ss`. Кроме того, вы можете поменять отображение с помощью функции [formatDateTime](../../sql-reference/functions/date-time-functions.md#formatdatetime). -При вставке данных в ClickHouse, можно использовать различные форматы даты и времени в зависимости от значения настройки [date_time_input_format](../../operations/settings/#settings-date_time_input_format). +При вставке данных в ClickHouse, можно использовать различные форматы даты и времени в зависимости от значения настройки [date_time_input_format](../../operations/settings/index.md#settings-date_time_input_format). ## Примеры {#primery} @@ -119,8 +119,8 @@ FROM dt - [Функции преобразования типов](../../sql-reference/functions/type-conversion-functions.md) - [Функции для работы с датой и временем](../../sql-reference/functions/date-time-functions.md) - [Функции для работы с массивами](../../sql-reference/functions/array-functions.md) -- [Настройка `date_time_input_format`](../../operations/settings/#settings-date_time_input_format) -- [Настройка `date_time_output_format`](../../operations/settings/) +- [Настройка `date_time_input_format`](../../operations/settings/index.md#settings-date_time_input_format) +- [Настройка `date_time_output_format`](../../operations/settings/index.md) - [Конфигурационный параметр сервера `timezone`](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) - [Операторы для работы с датой и временем](../../sql-reference/operators/index.md#operators-datetime) - [Тип данных `Date`](date.md) diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 8fbcaf9568b..a7e8a478edb 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -268,7 +268,7 @@ SELECT toUnixTimestamp('2017-11-05 08:07:47', 'Asia/Tokyo') AS unix_timestamp; ``` :::note -Тип возвращаемого значения описанными далее функциями `toStartOf*`, `toLastDayOfMonth`, `toMonday`, `timeSlot` определяется конфигурационным параметром [enable_extended_results_for_datetime_functions](../../operations/settings/settings#enable-extended-results-for-datetime-functions) имеющим по умолчанию значение `0`. +Тип возвращаемого значения описанными далее функциями `toStartOf*`, `toLastDayOfMonth`, `toMonday`, `timeSlot` определяется конфигурационным параметром [enable_extended_results_for_datetime_functions](../../operations/settings/settings.md#enable-extended-results-for-datetime-functions) имеющим по умолчанию значение `0`. Поведение для * `enable_extended_results_for_datetime_functions = 0`: Функции `toStartOf*`, `toLastDayOfMonth`, `toMonday` возвращают `Date` или `DateTime`. Функции `toStartOfDay`, `toStartOfHour`, `toStartOfFifteenMinutes`, `toStartOfTenMinutes`, `toStartOfFiveMinutes`, `toStartOfMinute`, `timeSlot` возвращают `DateTime`. Хотя эти функции могут принимать значения типа `Date32` или `DateTime64` в качестве аргумента, при обработке аргумента вне нормального диапазона значений (`1970` - `2148` для `Date` и `1970-01-01 00:00:00`-`2106-02-07 08:28:15` для `DateTime`) будет получен некорректный результат. diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index f457b54ae28..de54f1b3607 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2136,7 +2136,7 @@ countDigits(x) :::note "Примечание" Для `Decimal` значений учитывается их масштаб: вычисляется результат по базовому целочисленному типу, полученному как `(value * scale)`. Например: `countDigits(42) = 2`, `countDigits(42.000) = 5`, `countDigits(0.04200) = 4`. То есть вы можете проверить десятичное переполнение для `Decimal64` с помощью `countDecimal(x) > 18`. Это медленный вариант [isDecimalOverflow](#is-decimal-overflow). ::: - + **Пример** Запрос: @@ -2297,7 +2297,7 @@ enabledRoles() ## defaultRoles {#default-roles} -Возвращает имена ролей, которые задаются по умолчанию для текущего пользователя при входе в систему. Изначально это все роли, которые разрешено использовать текущему пользователю (см. [GRANT](../../sql-reference/statements/grant/#grant-select)). Список ролей по умолчанию может быть изменен с помощью выражения [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement). +Возвращает имена ролей, которые задаются по умолчанию для текущего пользователя при входе в систему. Изначально это все роли, которые разрешено использовать текущему пользователю (см. [GRANT](../../sql-reference/statements/grant.md#grant-select)). Список ролей по умолчанию может быть изменен с помощью выражения [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement). **Синтаксис** diff --git a/docs/ru/sql-reference/statements/select/array-join.md b/docs/ru/sql-reference/statements/select/array-join.md index 9d2dbf54a2b..6c7fcbba7cc 100644 --- a/docs/ru/sql-reference/statements/select/array-join.md +++ b/docs/ru/sql-reference/statements/select/array-join.md @@ -146,7 +146,7 @@ ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num, arrayMap(x -> x + 1, arr) AS ma └───────┴─────────┴───┴─────┴────────┘ ``` -В приведенном ниже примере используется функция [arrayEnumerate](../../../sql-reference/functions/array-functions#array_functions-arrayenumerate): +В приведенном ниже примере используется функция [arrayEnumerate](../../../sql-reference/functions/array-functions.md#array_functions-arrayenumerate): ``` sql SELECT s, arr, a, num, arrayEnumerate(arr) @@ -259,7 +259,7 @@ ARRAY JOIN nest AS n; └───────┴─────┴─────┴─────────┴────────────┘ ``` -Пример использования функции [arrayEnumerate](../../../sql-reference/functions/array-functions#array_functions-arrayenumerate): +Пример использования функции [arrayEnumerate](../../../sql-reference/functions/array-functions.md#array_functions-arrayenumerate): ``` sql SELECT s, `n.x`, `n.y`, `nest.x`, `nest.y`, num diff --git a/docs/zh/development/continuous-integration.md b/docs/zh/development/continuous-integration.md index a52d77a7a33..56e3e1dfd50 100644 --- a/docs/zh/development/continuous-integration.md +++ b/docs/zh/development/continuous-integration.md @@ -34,7 +34,7 @@ git push ## 描述信息检查 {#description-check} 检查pull请求的描述是否符合[PULL_REQUEST_TEMPLATE.md](https://github.com/ClickHouse/ClickHouse/blob/master/.github/PULL_REQUEST_TEMPLATE.md)模板. -您必须为您的更改指定一个更改日志类别(例如,Bug修复), 并且为[CHANGELOG.md](../whats-new/changelog/)编写一条用户可读的消息用来描述更改. +您必须为您的更改指定一个更改日志类别(例如,Bug修复), 并且为[CHANGELOG.md](../whats-new/changelog/index.md)编写一条用户可读的消息用来描述更改. ## 推送到DockerHub {#push-to-dockerhub} 生成用于构建和测试的docker映像, 然后将它们推送到DockerHub. diff --git a/docs/zh/engines/database-engines/index.md b/docs/zh/engines/database-engines/index.md index 0b24590686e..2839f819671 100644 --- a/docs/zh/engines/database-engines/index.md +++ b/docs/zh/engines/database-engines/index.md @@ -16,7 +16,7 @@ sidebar_position: 27 - [MaterializeMySQL](../../engines/database-engines/materialized-mysql.md) -- [Lazy](../../engines/database-engines/lazy) +- [Lazy](../../engines/database-engines/lazy.md) - [Atomic](../../engines/database-engines/atomic.md) diff --git a/docs/zh/engines/database-engines/materialize-mysql.md b/docs/zh/engines/database-engines/materialize-mysql.md index 10049017c71..b7ee3a038b8 100644 --- a/docs/zh/engines/database-engines/materialize-mysql.md +++ b/docs/zh/engines/database-engines/materialize-mysql.md @@ -38,8 +38,8 @@ ENGINE = MaterializeMySQL('host:port', ['database' | database], 'user', 'passwor - `max_wait_time_when_mysql_unavailable` — 当MySQL不可用时重试间隔(毫秒)。负值禁止重试。默认值: `1000`. - `allows_query_when_mysql_lost` — 当mysql丢失时,允许查询物化表。默认值: `0` (`false`). ``` -CREATE DATABASE mysql ENGINE = MaterializeMySQL('localhost:3306', 'db', 'user', '***') - SETTINGS +CREATE DATABASE mysql ENGINE = MaterializeMySQL('localhost:3306', 'db', 'user', '***') + SETTINGS allows_query_when_mysql_lost=true, max_wait_time_when_mysql_unavailable=10000; ``` @@ -97,7 +97,7 @@ CREATE DATABASE mysql ENGINE = MaterializeMySQL('localhost:3306', 'db', 'user', ### DDL查询 {#ddl-queries} -MySQL DDL查询转换为相应的ClickHouse DDL查询([ALTER](../../sql-reference/statements/alter/index.md), [CREATE](../../sql-reference/statements/create/index.md), [DROP](../../sql-reference/statements/drop), [RENAME](../../sql-reference/statements/rename.md))。如果ClickHouse无法解析某个DDL查询,则该查询将被忽略。 +MySQL DDL查询转换为相应的ClickHouse DDL查询([ALTER](../../sql-reference/statements/alter/index.md), [CREATE](../../sql-reference/statements/create/index.md), [DROP](../../sql-reference/statements/drop.md), [RENAME](../../sql-reference/statements/rename.md))。如果ClickHouse无法解析某个DDL查询,则该查询将被忽略。 ### Data Replication {#data-replication} @@ -148,9 +148,9 @@ mysql> SELECT * FROM test; ``` ```text -+---+------+------+ ++---+------+------+ | a | b | c | -+---+------+------+ ++---+------+------+ | 2 | 222 | Wow! | +---+------+------+ ``` @@ -177,9 +177,9 @@ SELECT * FROM mysql.test; ``` ``` text -┌─a─┬──b─┐ -│ 1 │ 11 │ -│ 2 │ 22 │ +┌─a─┬──b─┐ +│ 1 │ 11 │ +│ 2 │ 22 │ └───┴────┘ ``` @@ -190,7 +190,7 @@ SELECT * FROM mysql.test; ``` ``` text -┌─a─┬───b─┬─c────┐ -│ 2 │ 222 │ Wow! │ +┌─a─┬───b─┬─c────┐ +│ 2 │ 222 │ Wow! │ └───┴─────┴──────┘ ``` diff --git a/docs/zh/engines/database-engines/materialized-mysql.md b/docs/zh/engines/database-engines/materialized-mysql.md index c34d3a6f20d..4cc4ae58840 100644 --- a/docs/zh/engines/database-engines/materialized-mysql.md +++ b/docs/zh/engines/database-engines/materialized-mysql.md @@ -109,7 +109,7 @@ MySQL中的Time 类型,会被ClickHouse转换成微秒来存储 ### DDL Queries {#ddl-queries} -MySQL DDL 语句会被转换成对应的ClickHouse DDL 语句,比如: ([ALTER](../../sql-reference/statements/alter/index.md), [CREATE](../../sql-reference/statements/create/index.md), [DROP](../../sql-reference/statements/drop), [RENAME](../../sql-reference/statements/rename.md)). 如果ClickHouse 无法解析某些语句DDL 操作,则会跳过。 +MySQL DDL 语句会被转换成对应的ClickHouse DDL 语句,比如: ([ALTER](../../sql-reference/statements/alter/index.md), [CREATE](../../sql-reference/statements/create/index.md), [DROP](../../sql-reference/statements/drop.md), [RENAME](../../sql-reference/statements/rename.md)). 如果ClickHouse 无法解析某些语句DDL 操作,则会跳过。 ### 数据复制 {#data-replication} @@ -152,17 +152,17 @@ ClickHouse只有一个物理排序,由 `order by` 条件决定。要创建一 这些是你可以对MaterializedMySQL表重写的模式转换操作: * 修改列类型。必须与原始类型兼容,否则复制将失败。例如,可以将`UInt32`列修改为`UInt64`,不能将 `String` 列修改为 `Array(String)`。 - * 修改 [column TTL](../table-engines/mergetree-family/mergetree/#mergetree-column-ttl). + * 修改 [column TTL](../table-engines/mergetree-family/mergetree.md#mergetree-column-ttl). * 修改 [column compression codec](../../sql-reference/statements/create/table.mdx#codecs). * 增加 [ALIAS columns](../../sql-reference/statements/create/table.mdx#alias). - * 增加 [skipping indexes](../table-engines/mergetree-family/mergetree/#table_engine-mergetree-data_skipping-indexes) - * 增加 [projections](../table-engines/mergetree-family/mergetree/#projections). + * 增加 [skipping indexes](../table-engines/mergetree-family/mergetree.md#table_engine-mergetree-data_skipping-indexes) + * 增加 [projections](../table-engines/mergetree-family/mergetree.md#projections). 请注意,当使用 `SELECT ... FINAL ` (MaterializedMySQL默认是这样做的) 时,预测优化是被禁用的,所以这里是受限的, `INDEX ... TYPE hypothesis `[在v21.12的博客文章中描述]](https://clickhouse.com/blog/en/2021/clickhouse-v21.12-released/)可能在这种情况下更有用。 - * 修改 [PARTITION BY](../table-engines/mergetree-family/custom-partitioning-key/) - * 修改 [ORDER BY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) - * 修改 [PRIMARY KEY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) - * 增加 [SAMPLE BY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) - * 增加 [table TTL](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) + * 修改 [PARTITION BY](../table-engines/mergetree-family/custom-partitioning-key.md) + * 修改 [ORDER BY](../table-engines/mergetree-family/mergetree.md#mergetree-query-clauses) + * 修改 [PRIMARY KEY](../table-engines/mergetree-family/mergetree.md#mergetree-query-clauses) + * 增加 [SAMPLE BY](../table-engines/mergetree-family/mergetree.md#mergetree-query-clauses) + * 增加 [table TTL](../table-engines/mergetree-family/mergetree.md#mergetree-query-clauses) ```sql CREATE DATABASE db_name ENGINE = MaterializedMySQL(...) diff --git a/docs/zh/engines/table-engines/mergetree-family/mergetree.md b/docs/zh/engines/table-engines/mergetree-family/mergetree.md index 1fcf64fcd25..54524388650 100644 --- a/docs/zh/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/zh/engines/table-engines/mergetree-family/mergetree.md @@ -192,7 +192,7 @@ ClickHouse 会为每个数据片段创建一个索引文件来存储这些标记 ClickHouse 不要求主键唯一,所以您可以插入多条具有相同主键的行。 -您可以在`PRIMARY KEY`与`ORDER BY`条件中使用`可为空的`类型的表达式,但强烈建议不要这么做。为了启用这项功能,请打开[allow_nullable_key](../../../operations/settings/#allow-nullable-key),[NULLS_LAST](../../../sql-reference/statements/select/order-by.md/#sorting-of-special-values)规则也适用于`ORDER BY`条件中有NULL值的情况下。 +您可以在`PRIMARY KEY`与`ORDER BY`条件中使用`可为空的`类型的表达式,但强烈建议不要这么做。为了启用这项功能,请打开[allow_nullable_key](../../../operations/settings/index.md#allow-nullable-key),[NULLS_LAST](../../../sql-reference/statements/select/order-by.md#sorting-of-special-values)规则也适用于`ORDER BY`条件中有NULL值的情况下。 ### 主键的选择 {#zhu-jian-de-xuan-ze} @@ -330,7 +330,7 @@ SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234 支持的数据类型:`Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`, `Array`, `LowCardinality`, `Nullable`。 - 以下函数会用到这个索引: [equals](../../../sql-reference/functions/comparison-functions.md), [notEquals](../../../sql-reference/functions/comparison-functions.md), [in](../../../sql-reference/functions/in-functions), [notIn](../../../sql-reference/functions/in-functions), [has](../../../sql-reference/functions/array-functions) + 以下函数会用到这个索引: [equals](../../../sql-reference/functions/comparison-functions.md), [notEquals](../../../sql-reference/functions/comparison-functions.md), [in](../../../sql-reference/functions/in-functions.md), [notIn](../../../sql-reference/functions/in-functions.md), [has](../../../sql-reference/functions/array-functions.md) ``` sql INDEX sample_index (u64 * length(s)) TYPE minmax GRANULARITY 4 @@ -353,14 +353,14 @@ WHERE 子句中的条件可以包含对某列数据进行运算的函数表达 | [startsWith](../../../sql-reference/functions/string-functions.md#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | | [endsWith](../../../sql-reference/functions/string-functions.md#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ | | [multiSearchAny](../../../sql-reference/functions/string-search-functions.md#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | -| [in](../../../sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [notIn](../../../sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [in](../../../sql-reference/functions/in-functions.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [notIn](../../../sql-reference/functions/in-functions.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | | [less (\<)](../../../sql-reference/functions/comparison-functions.md#function-less) | ✔ | ✔ | ✗ | ✗ | ✗ | | [greater (\>)](../../../sql-reference/functions/comparison-functions.md#function-greater) | ✔ | ✔ | ✗ | ✗ | ✗ | | [lessOrEquals (\<=)](../../../sql-reference/functions/comparison-functions.md#function-lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | | [greaterOrEquals (\>=)](../../../sql-reference/functions/comparison-functions.md#function-greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [empty](../../../sql-reference/functions/array-functions#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [notEmpty](../../../sql-reference/functions/array-functions#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [empty](../../../sql-reference/functions/array-functions.md#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [notEmpty](../../../sql-reference/functions/array-functions.md#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | | hasToken | ✗ | ✗ | ✗ | ✔ | ✗ | 常量参数小于 ngram 大小的函数不能使用 `ngrambf_v1` 进行查询优化。 diff --git a/docs/zh/faq/general/why-clickhouse-is-so-fast.md b/docs/zh/faq/general/why-clickhouse-is-so-fast.md index a30b56adb9a..1962b8b90c2 100644 --- a/docs/zh/faq/general/why-clickhouse-is-so-fast.md +++ b/docs/zh/faq/general/why-clickhouse-is-so-fast.md @@ -9,7 +9,7 @@ sidebar_position: 8 It was designed to be fast. Query execution performance has always been a top priority during the development process, but other important characteristics like user-friendliness, scalability, and security were also considered so ClickHouse could become a real production system. -ClickHouse was initially built as a prototype to do just a single task well: to filter and aggregate data as fast as possible. That’s what needs to be done to build a typical analytical report and that’s what a typical [GROUP BY](../../sql-reference/statements/select/group-by/) query does. ClickHouse team has made several high-level decisions that combined made achieving this task possible: +ClickHouse was initially built as a prototype to do just a single task well: to filter and aggregate data as fast as possible. That’s what needs to be done to build a typical analytical report and that’s what a typical [GROUP BY](../../sql-reference/statements/select/group-by.md) query does. ClickHouse team has made several high-level decisions that combined made achieving this task possible: Column-oriented storage : Source data often contain hundreds or even thousands of columns, while a report can use just a few of them. The system needs to avoid reading unnecessary columns, or most expensive disk read operations would be wasted. diff --git a/docs/zh/faq/integration/index.md b/docs/zh/faq/integration/index.md index 6678956a0b3..3a3f97761f3 100644 --- a/docs/zh/faq/integration/index.md +++ b/docs/zh/faq/integration/index.md @@ -1,5 +1,5 @@ --- -slug: /zh/faq/integration/ +slug: /zh/faq/integration/ title: 关于集成ClickHouse和其他系统的问题 toc_hidden_folder: true sidebar_position: 4 @@ -17,6 +17,6 @@ sidebar_label: Integration !!! info "没看到你要找的东西吗?" - 查看[其他faq类别](../../faq/)或浏览左边栏中的主要文档文章。 + 查看[其他faq类别](../../faq/index.md)或浏览左边栏中的主要文档文章。 {## [原文](https://clickhouse.com/docs/en/faq/integration/) ##} \ No newline at end of file diff --git a/docs/zh/faq/integration/json-import.md b/docs/zh/faq/integration/json-import.md index 861abacc1e1..2d5c687316d 100644 --- a/docs/zh/faq/integration/json-import.md +++ b/docs/zh/faq/integration/json-import.md @@ -7,29 +7,29 @@ sidebar_position: 11 # How to Import JSON Into ClickHouse? {#how-to-import-json-into-clickhouse} -ClickHouse supports a wide range of [data formats for input and output](../../interfaces/formats/). There are multiple JSON variations among them, but the most commonly used for data ingestion is [JSONEachRow](../../interfaces/formats/#jsoneachrow). It expects one JSON object per row, each object separated by a newline. +ClickHouse supports a wide range of [data formats for input and output](../../interfaces/formats.md). There are multiple JSON variations among them, but the most commonly used for data ingestion is [JSONEachRow](../../interfaces/formats.md#jsoneachrow). It expects one JSON object per row, each object separated by a newline. ## Examples {#examples} -Using [HTTP interface](../../interfaces/http/): +Using [HTTP interface](../../interfaces/http.md): ``` bash $ echo '{"foo":"bar"}' | curl 'http://localhost:8123/?query=INSERT%20INTO%20test%20FORMAT%20JSONEachRow' --data-binary @- ``` -Using [CLI interface](../../interfaces/cli/): +Using [CLI interface](../../interfaces/cli.md): ``` bash $ echo '{"foo":"bar"}' | clickhouse-client --query="INSERT INTO test FORMAT JSONEachRow" ``` -Instead of inserting data manually, you might consider to use one of [client libraries](../../interfaces/) instead. +Instead of inserting data manually, you might consider to use one of [client libraries](../../interfaces/index.md) instead. ## Useful Settings {#useful-settings} - `input_format_skip_unknown_fields` allows to insert JSON even if there were additional fields not present in table schema (by discarding them). -- `input_format_import_nested_json` allows to insert nested JSON objects into columns of [Nested](../../sql-reference/data-types/nested-data-structures/nested/) type. +- `input_format_import_nested_json` allows to insert nested JSON objects into columns of [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) type. -:::note +:::note Settings are specified as `GET` parameters for the HTTP interface or as additional command-line arguments prefixed with `--` for the `CLI` interface. ::: \ No newline at end of file diff --git a/docs/zh/faq/operations/delete-old-data.md b/docs/zh/faq/operations/delete-old-data.md index b2229058cad..24181116bab 100644 --- a/docs/zh/faq/operations/delete-old-data.md +++ b/docs/zh/faq/operations/delete-old-data.md @@ -19,7 +19,7 @@ The key advantage of this approach is that it does not need any external system TTL can also be used to move data not only to [/dev/null](https://en.wikipedia.org/wiki/Null_device), but also between different storage systems, like from SSD to HDD. ::: -More details on [configuring TTL](../../engines/table-engines/mergetree-family/mergetree/#table_engine-mergetree-ttl). +More details on [configuring TTL](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl). ## ALTER DELETE {#alter-delete} @@ -41,4 +41,4 @@ More details on [manipulating partitions](../../sql-reference/statements/alter/p It’s rather radical to drop all data from a table, but in some cases it might be exactly what you need. -More details on [table truncation](../../sql-reference/statements/truncate/). +More details on [table truncation](../../sql-reference/statements/truncate.md). \ No newline at end of file diff --git a/docs/zh/faq/operations/index.md b/docs/zh/faq/operations/index.md index 071cc872e4e..153eda6199a 100644 --- a/docs/zh/faq/operations/index.md +++ b/docs/zh/faq/operations/index.md @@ -1,5 +1,5 @@ --- -slug: /zh/faq/operations/ +slug: /zh/faq/operations/ title: 关于操作ClickHouse服务器和集群的问题 toc_hidden_folder: true sidebar_position: 3 @@ -13,9 +13,9 @@ sidebar_label: Operations - [如果想在生产环境部署,需要用哪个版本的 ClickHouse 呢?](../../faq/operations/production.md) - [是否可能从 ClickHouse 数据表中删除所有旧的数据记录?](../../faq/operations/delete-old-data.md) - [ClickHouse支持多区域复制吗?](../../faq/operations/multi-region-replication.md) - + !!! info "没看到你要找的东西吗?" - 查看[其他faq类别](../../faq/)或浏览左边栏中的主要文档文章。 + 查看[其他faq类别](../../faq/index.md)或浏览左边栏中的主要文档文章。 {## [原文](https://clickhouse.com/docs/en/faq/production/) ##} diff --git a/docs/zh/faq/operations/production.md b/docs/zh/faq/operations/production.md index cc5cf6b9614..90db050e8d3 100644 --- a/docs/zh/faq/operations/production.md +++ b/docs/zh/faq/operations/production.md @@ -67,6 +67,6 @@ For production use, there are two key options: `stable` and `lts`. Here is some Many teams who initially thought that `lts` is the way to go, often switch to `stable` anyway because of some recent feature that’s important for their product. -:::warning -One more thing to keep in mind when upgrading ClickHouse: we’re always keeping eye on compatibility across releases, but sometimes it’s not reasonable to keep and some minor details might change. So make sure you check the [changelog](../../whats-new/changelog/) before upgrading to see if there are any notes about backward-incompatible changes. +:::warning +One more thing to keep in mind when upgrading ClickHouse: we’re always keeping eye on compatibility across releases, but sometimes it’s not reasonable to keep and some minor details might change. So make sure you check the [changelog](../../whats-new/changelog/index.md) before upgrading to see if there are any notes about backward-incompatible changes. ::: \ No newline at end of file diff --git a/docs/zh/faq/use-cases/index.md b/docs/zh/faq/use-cases/index.md index 75ef26368a3..ff0d873b37f 100644 --- a/docs/zh/faq/use-cases/index.md +++ b/docs/zh/faq/use-cases/index.md @@ -14,6 +14,6 @@ sidebar_label: 使用案例 - [我能把 ClickHouse 当做Key-value 键值存储来使用吗?](../../faq/use-cases/key-value.md) !!! info "没找到您所需要的内容?" - 请查看[其他常见问题类别](../../faq/)或浏览左侧边栏中的主要文档文章。 + 请查看[其他常见问题类别](../../faq/index.md)或浏览左侧边栏中的主要文档文章。 {## [原始文档](https://clickhouse.com/docs/en/faq/use-cases/) ##} diff --git a/docs/zh/getting-started/example-datasets/recipes.mdx b/docs/zh/getting-started/example-datasets/recipes.mdx index b7ed92962c5..b7f8fe8eafd 100644 --- a/docs/zh/getting-started/example-datasets/recipes.mdx +++ b/docs/zh/getting-started/example-datasets/recipes.mdx @@ -1,5 +1,5 @@ ---- -slug: /zh/getting-started/example-datasets/recipes +--- +slug: /zh/getting-started/example-datasets/recipes sidebar_label: 食谱数据集 title: "食谱数据集" --- @@ -8,8 +8,8 @@ RecipeNLG 数据集可在 [此处](https://recipenlg.cs.put.poznan.pl/dataset) ## 下载并解压数据集 -1. 进入下载页面[https://recipenlg.cs.put.poznan.pl/dataset](https://recipenlg.cs.put.poznan.pl/dataset)。 -2. 接受条款和条件并下载 zip 文件。 +1. 进入下载页面[https://recipenlg.cs.put.poznan.pl/dataset](https://recipenlg.cs.put.poznan.pl/dataset)。 +2. 接受条款和条件并下载 zip 文件。 3. 使用 `unzip` 解压 zip 文件,得到 `full_dataset.csv` 文件。 ## 创建表 @@ -49,13 +49,13 @@ clickhouse-client --query " 这是一个展示如何解析自定义 CSV,这其中涉及了许多调整。 -说明: -- 数据集为 CSV 格式,但在插入时需要一些预处理;使用表函数 [input](../../sql-reference/table-functions/input.md) 进行预处理; -- CSV 文件的结构在表函数 `input` 的参数中指定; -- 字段 `num`(行号)是不需要的 - 可以忽略并从文件中进行解析; -- 使用 `FORMAT CSVWithNames`,因为标题不包含第一个字段的名称,因此 CSV 中的标题将被忽略(通过命令行参数 `--input_format_with_names_use_header 0`); -- 文件仅使用双引号将 CSV 字符串括起来;一些字符串没有用双引号括起来,单引号也不能被解析为括起来的字符串 - 所以添加`--format_csv_allow_single_quote 0`参数接受文件中的单引号; -- 由于某些 CSV 的字符串的开头包含 `\M/` 因此无法被解析; CSV 中唯一可能以反斜杠开头的值是 `\N`,这个值被解析为 SQL NULL。通过添加`--input_format_allow_errors_num 10`参数,允许在导入过程中跳过 10 个格式错误; +说明: +- 数据集为 CSV 格式,但在插入时需要一些预处理;使用表函数 [input](../../sql-reference/table-functions/input.md) 进行预处理; +- CSV 文件的结构在表函数 `input` 的参数中指定; +- 字段 `num`(行号)是不需要的 - 可以忽略并从文件中进行解析; +- 使用 `FORMAT CSVWithNames`,因为标题不包含第一个字段的名称,因此 CSV 中的标题将被忽略(通过命令行参数 `--input_format_with_names_use_header 0`); +- 文件仅使用双引号将 CSV 字符串括起来;一些字符串没有用双引号括起来,单引号也不能被解析为括起来的字符串 - 所以添加`--format_csv_allow_single_quote 0`参数接受文件中的单引号; +- 由于某些 CSV 的字符串的开头包含 `\M/` 因此无法被解析; CSV 中唯一可能以反斜杠开头的值是 `\N`,这个值被解析为 SQL NULL。通过添加`--input_format_allow_errors_num 10`参数,允许在导入过程中跳过 10 个格式错误; - 在数据集中的 Ingredients、directions 和 NER 字段为数组;但这些数组并没有以一般形式表示:这些字段作为 JSON 序列化为字符串,然后放入 CSV 中 - 在导入是将它们解析为字符串,然后使用 [JSONExtract](../../sql-reference/functions/json-functions.md ) 函数将其转换为数组。 ## 验证插入的数据 @@ -80,7 +80,7 @@ SELECT count() FROM recipes; ### 按配方数量排列的顶级组件: -在此示例中,我们学习如何使用 [arrayJoin](../../sql-reference/functions/array-join/) 函数将数组扩展为行的集合。 +在此示例中,我们学习如何使用 [arrayJoin](../../sql-reference/functions/array-join.md) 函数将数组扩展为行的集合。 请求: @@ -185,7 +185,7 @@ LIMIT 10 10 rows in set. Elapsed: 0.215 sec. Processed 2.23 million rows, 1.48 GB (10.35 million rows/s., 6.86 GB/s.) ``` -在此示例中,我们使用 [has](../../sql-reference/functions/array-functions/#hasarr-elem) 函数来按过滤数组类型元素并按 directions 的数量进行排序。 +在此示例中,我们使用 [has](../../sql-reference/functions/array-functions.md#hasarr-elem) 函数来按过滤数组类型元素并按 directions 的数量进行排序。 有一个婚礼蛋糕需要整个126个步骤来制作!显示 directions: diff --git a/docs/zh/guides/improving-query-performance/skipping-indexes.md b/docs/zh/guides/improving-query-performance/skipping-indexes.md index f5889898c2c..f9f43e46927 100644 --- a/docs/zh/guides/improving-query-performance/skipping-indexes.md +++ b/docs/zh/guides/improving-query-performance/skipping-indexes.md @@ -89,7 +89,7 @@ SELECT * FROM skip_table WHERE my_value IN (125, 700) 下图是更直观的展示,这就是如何读取和选择my_value为125的4096行,以及如何跳过以下行而不从磁盘读取: -![Simple Skip](../../../en/guides/improving-query-performance/images/simple_skip.svg) +![Simple Skip](../../../en/guides/best-practices/images/simple_skip.svg) 通过在执行查询时启用跟踪,用户可以看到关于跳数索引使用情况的详细信息。在clickhouse-client中设置send_logs_level: @@ -126,7 +126,7 @@ Bloom filter是一种数据结构,它允许对集合成员进行高效的是 * 基本的**bloom_filter**接受一个可选参数,该参数表示在0到1之间允许的“假阳性”率(如果未指定,则使用.025)。 * 更专业的**tokenbf_v1**。需要三个参数,用来优化布隆过滤器:(1)过滤器的大小字节(大过滤器有更少的假阳性,有更高的存储成本),(2)哈希函数的个数(更多的散列函数可以减少假阳性)。(3)布隆过滤器哈希函数的种子。有关这些参数如何影响布隆过滤器功能的更多细节,请参阅 [这里](https://hur.st/bloomfilter/) 。此索引仅适用于String、FixedString和Map类型的数据。输入表达式被分割为由非字母数字字符分隔的字符序列。例如,列值`This is a candidate for a "full text" search`将被分割为`This` `is` `a` `candidate` `for` `full` `text` `search`。它用于LIKE、EQUALS、in、hasToken()和类似的长字符串中单词和其他值的搜索。例如,一种可能的用途是在非结构的应用程序日志行列中搜索少量的类名或行号。 - + * 更专业的**ngrambf_v1**。该索引的功能与tokenbf_v1相同。在Bloom filter设置之前需要一个额外的参数,即要索引的ngram的大小。一个ngram是长度为n的任何字符串,比如如果n是4,`A short string`会被分割为`A sh`` sho`, `shor`, `hort`, `ort s`, `or st`, `r str`, ` stri`, `trin`, `ring`。这个索引对于文本搜索也很有用,特别是没有单词间断的语言,比如中文。 ### 跳数索引函数 @@ -150,7 +150,7 @@ Bloom filter是一种数据结构,它允许对集合成员进行高效的是 考虑以下数据分布: -![Bad Skip!](../../../en/guides/improving-query-performance/images/bad_skip_1.svg) +![Bad Skip!](../../../en/guides/best-practices/images/bad_skip_1.svg) 假设主键/顺序是时间戳,并且在visitor_id上有一个索引。考虑下面的查询: diff --git a/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md b/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md index 18b23a79f86..27b6679e2c1 100644 --- a/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md +++ b/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md @@ -1,5 +1,5 @@ --- -slug: /zh/guides/improving-query-performance/sparse-primary-indexes +slug: /zh/guides/best-practices sidebar_label: 主键稀疏索引 sidebar_position: 20 --- @@ -19,21 +19,21 @@ sidebar_position: 20 :::note 这篇文章主要关注稀疏索引。 -如果想了解二级跳数索引,请查看[教程](./skipping-indexes.md). +如果想了解二级跳数索引,请查看[教程](./skipping-indexes.md). ::: -## 数据集 +## 数据集 在本文中,我们将使用一个匿名的web流量数据集。 -- 我们将使用样本数据集中的887万行(事件)的子集。 +- 我们将使用样本数据集中的887万行(事件)的子集。 - 未压缩的数据大小为887万个事件和大约700mb。当存储在ClickHouse时,压缩为200mb。 - 在我们的子集中,每行包含三列,表示在特定时间(EventTime列)单击URL (URL列)的互联网用户(UserID列)。 通过这三个列,我们已经可以制定一些典型的web分析查询,如: - + - 某个用户点击次数最多的前10个url是什么? - 点击某个URL次数最多的前10名用户是谁? - 用户点击特定URL的最频繁时间(比如一周中的几天)是什么? @@ -44,7 +44,7 @@ sidebar_position: 20 ## 全表扫描 -为了了解在没有主键的情况下如何对数据集执行查询,我们通过执行以下SQL DDL语句(使用MergeTree表引擎)创建了一个表: +为了了解在没有主键的情况下如何对数据集执行查询,我们通过执行以下SQL DDL语句(使用MergeTree表引擎)创建了一个表: ```sql CREATE TABLE hits_NoPrimaryKey @@ -70,11 +70,11 @@ FROM url('https://datasets.clickhouse.com/hits/tsv/hits_v1.tsv.xz') WHERE URL != ''; ``` 结果: -```response +```response Ok. 0 rows in set. Elapsed: 145.993 sec. Processed 8.87 million rows, 18.40 GB (60.78 thousand rows/s., 126.06 MB/s.) -``` +``` ClickHouse客户端输出了执行结果,插入了887万行数据。 @@ -102,7 +102,7 @@ ORDER BY Count DESC LIMIT 10; ``` 结果: -```response +```response ┌─URL────────────────────────────┬─Count─┐ │ http://auto.ru/chatay-barana.. │ 170 │ │ http://auto.ru/chatay-id=371...│ 52 │ @@ -117,10 +117,10 @@ LIMIT 10; └────────────────────────────────┴───────┘ 10 rows in set. Elapsed: 0.022 sec. -// highlight-next-line -Processed 8.87 million rows, +// highlight-next-line +Processed 8.87 million rows, 70.45 MB (398.53 million rows/s., 3.17 GB/s.) -``` +``` ClickHouse客户端输出表明,ClickHouse执行了一个完整的表扫描!我们的表的887万行中的每一行都被加载到ClickHouse中,这不是可扩展的。 @@ -131,7 +131,7 @@ ClickHouse客户端输出表明,ClickHouse执行了一个完整的表扫描! ## 包含主键的表 -创建一个包含联合主键UserID和URL列的表: +创建一个包含联合主键UserID和URL列的表: ```sql CREATE TABLE hits_UserID_URL @@ -141,7 +141,7 @@ CREATE TABLE hits_UserID_URL `EventTime` DateTime ) ENGINE = MergeTree -// highlight-next-line +// highlight-next-line PRIMARY KEY (UserID, URL) ORDER BY (UserID, URL, EventTime) SETTINGS index_granularity = 8192, index_granularity_bytes = 0; @@ -190,7 +190,7 @@ FROM url('https://datasets.clickhouse.com/hits/tsv/hits_v1.tsv.xz') WHERE URL != ''; ``` 结果: -```response +```response 0 rows in set. Elapsed: 149.432 sec. Processed 8.87 million rows, 18.40 GB (59.38 thousand rows/s., 123.16 MB/s.) ``` @@ -219,7 +219,7 @@ FROM system.parts WHERE (table = 'hits_UserID_URL') AND (active = 1) FORMAT Vertical; ``` - + 结果: ```response @@ -237,7 +237,7 @@ bytes_on_disk: 207.07 MiB ``` 客户端输出表明: - + - 表数据以wide format存储在一个特定目录,每个列有一个数据文件和mark文件。 - 表有887万行数据。 - 未压缩的数据有733.28 MB。 @@ -278,8 +278,8 @@ bytes_on_disk: 207.07 MiB ## 数据按照主键排序存储在磁盘上 -上面创建的表有: -- 联合主键 (UserID, URL) +上面创建的表有: +- 联合主键 (UserID, URL) - 联合排序键 (UserID, URL, EventTime)。 :::note @@ -293,7 +293,7 @@ bytes_on_disk: 207.07 MiB ::: -插入的行按照主键列(以及排序键的附加EventTime列)的字典序(从小到大)存储在磁盘上。 +插入的行按照主键列(以及排序键的附加EventTime列)的字典序(从小到大)存储在磁盘上。 :::note ClickHouse允许插入具有相同主键列的多行数据。在这种情况下(参见下图中的第1行和第2行),最终的顺序是由指定的排序键决定的,这里是EventTime列的值。 @@ -307,7 +307,7 @@ ClickHouse允许插入具有相同主键列的多行数据。在这种情况下( - 然后是URL, - 最后是EventTime: - + UserID.bin,URL.bin,和EventTime.bin是UserIDURL,和EventTime列的数据文件。
@@ -331,7 +331,7 @@ UserID.bin,URL.bin,和EventTime.bin是UserID 下图显示了如何将表中的887万行(列值)组织成1083个颗粒,这是表的DDL语句包含设置index_granularity(设置为默认值8192)的结果。 - + 第一个(根据磁盘上的物理顺序)8192行(它们的列值)在逻辑上属于颗粒0,然后下一个8192行(它们的列值)属于颗粒1,以此类推。 @@ -355,21 +355,21 @@ UserID.bin,URL.bin,和EventTime.bin是UserID 下面的图显示了索引存储了每个颗粒的最小主键列值(在上面的图中用橙色标记的值)。 例如: -- 第一个索引条目(下图中的“mark 0”)存储上图中颗粒0的主键列的最小值, -- 第二个索引条目(下图中的“mark 1”)存储上图中颗粒1的主键列的最小值,以此类推。 +- 第一个索引条目(下图中的“mark 0”)存储上图中颗粒0的主键列的最小值, +- 第二个索引条目(下图中的“mark 1”)存储上图中颗粒1的主键列的最小值,以此类推。 - + -在我们的表中,索引总共有1083个条目,887万行数据和1083个颗粒: +在我们的表中,索引总共有1083个条目,887万行数据和1083个颗粒: - + :::note - 最后一个索引条目(上图中的“mark 1082”)存储了上图中颗粒1082的主键列的最大值。 - 索引条目(索引标记)不是基于表中的特定行,而是基于颗粒。例如,对于上图中的索引条目‘mark 0’,在我们的表中没有UserID为240.923且URL为“goal://metry=10000467796a411…”的行,相反,对于该表,有一个颗粒0,在该颗粒中,最小UserID值是240.923,最小URL值是“goal://metry=10000467796a411…”,这两个值来自不同的行。 -- 主索引文件完全加载到主内存中。如果文件大于可用的空闲内存空间,则ClickHouse将发生错误。 +- 主索引文件完全加载到主内存中。如果文件大于可用的空闲内存空间,则ClickHouse将发生错误。 ::: @@ -377,11 +377,11 @@ UserID.bin,URL.bin,和EventTime.bin是UserID - UserID index marks:
主索引中存储的UserID值按升序排序。
上图中的‘mark 1’指示颗粒1中所有表行的UserID值,以及随后所有颗粒中的UserID值,都保证大于或等于4.073.710。 - + [正如我们稍后将看到的](#query-on-userid-fast), 当查询对主键的第一列进行过滤时,此全局有序使ClickHouse能够对第一个键列的索引标记使用二分查找算法。 -- URL index marks:
- 主键列UserIDURL有相同的基数,这意味着第一列之后的所有主键列的索引标记通常只表示每个颗粒的数据范围。
+- URL index marks:
+ 主键列UserIDURL有相同的基数,这意味着第一列之后的所有主键列的索引标记通常只表示每个颗粒的数据范围。
例如,‘mark 0’中的URL列所有的值都大于等于goal://metry=10000467796a411..., 然后颗粒1中的URL并不是如此,这是因为‘mark 1‘与‘mark 0‘具有不同的UserID列值。 稍后我们将更详细地讨论这对查询执行性能的影响。 @@ -401,7 +401,7 @@ GROUP BY URL ORDER BY Count DESC LIMIT 10; ``` - + 结果: @@ -420,8 +420,8 @@ LIMIT 10; └────────────────────────────────┴───────┘ 10 rows in set. Elapsed: 0.005 sec. -// highlight-next-line -Processed 8.19 thousand rows, +// highlight-next-line +Processed 8.19 thousand rows, 740.18 KB (1.53 million rows/s., 138.59 MB/s.) ``` @@ -431,13 +431,13 @@ ClickHouse客户端的输出显示,没有进行全表扫描,只有8.19万行 如果trace logging打开了,那ClickHouse服务端日志会显示ClickHouse正在对1083个UserID索引标记执行二分查找以便识别可能包含UserID列值为749927693的行的颗粒。这需要19个步骤,平均时间复杂度为O(log2 n): ```response ...Executor): Key condition: (column 0 in [749927693, 749927693]) -// highlight-next-line +// highlight-next-line ...Executor): Running binary search on index range for part all_1_9_2 (1083 marks) ...Executor): Found (LEFT) boundary mark: 176 ...Executor): Found (RIGHT) boundary mark: 177 ...Executor): Found continuous range in 19 steps ...Executor): Selected 1/1 parts by partition key, 1 parts by primary key, -// highlight-next-line +// highlight-next-line 1/1083 marks by primary key, 1 marks to read from 1 ranges ...Reading ...approx. 8192 rows starting from 1441792 ``` @@ -451,7 +451,7 @@ ClickHouse客户端的输出显示,没有进行全表扫描,只有8.19万行

-Mark 176 was identified (the 'found left boundary mark' is inclusive, the 'found right boundary mark' is exclusive), and therefore all 8192 rows from granule 176 (which starts at row 1.441.792 - we will see that later on in this article) are then streamed into ClickHouse in order to find the actual rows with a UserID column value of 749927693. +Mark 176 was identified (the 'found left boundary mark' is inclusive, the 'found right boundary mark' is exclusive), and therefore all 8192 rows from granule 176 (which starts at row 1.441.792 - we will see that later on in this article) are then streamed into ClickHouse in order to find the actual rows with a UserID column value of 749927693.

@@ -465,7 +465,7 @@ GROUP BY URL ORDER BY Count DESC LIMIT 10; ``` - + 结果如下: ```response @@ -507,15 +507,15 @@ LIMIT 10; 在**第二阶段(数据读取中)**, ClickHouse定位所选的颗粒,以便将它们的所有行流到ClickHouse引擎中,以便找到实际匹配查询的行。 -我们将在下一节更详细地讨论第二阶段。 +我们将在下一节更详细地讨论第二阶段。 ## 标记文件用来定位颗粒 -下图描述了上表主索引文件的一部分。 +下图描述了上表主索引文件的一部分。 - + 如上所述,通过对索引的1083个UserID标记进行二分搜索,确定了第176个标记。因此,它对应的颗粒176可能包含UserID列值为749.927.693的行。 @@ -537,7 +537,7 @@ LIMIT 10; 下图显示了三个标记文件UserID.mrk、URL.mrk、EventTime.mrk,为表的UserID、URL和EventTime列存储颗粒的物理位置。 - + 我们已经讨论了主索引是一个扁平的未压缩数组文件(primary.idx),其中包含从0开始编号的索引标记。 @@ -545,9 +545,9 @@ LIMIT 10; 一旦ClickHouse确定并选择了可能包含查询所需的匹配行的颗粒的索引标记,就可以在标记文件数组中查找,以获得颗粒的物理位置。 -每个特定列的标记文件条目以偏移量的形式存储两个位置: +每个特定列的标记文件条目以偏移量的形式存储两个位置: -- 第一个偏移量(上图中的'block_offset')是在包含所选颗粒的压缩版本的压缩列数据文件中定位块。这个压缩块可能包含几个压缩的颗粒。所定位的压缩文件块在读取时被解压到内存中。 +- 第一个偏移量(上图中的'block_offset')是在包含所选颗粒的压缩版本的压缩列数据文件中定位块。这个压缩块可能包含几个压缩的颗粒。所定位的压缩文件块在读取时被解压到内存中。 - 标记文件的第二个偏移量(上图中的“granule_offset”)提供了颗粒在解压数据块中的位置。 @@ -576,7 +576,7 @@ LIMIT 10; 下面的图表和文本说明了我们的查询示例,ClickHouse如何在UserID.bin数据文件中定位176颗粒。 - + 我们在本文前面讨论过,ClickHouse选择了主索引标记176,因此176颗粒可能包含查询所需的匹配行。 @@ -624,7 +624,7 @@ LIMIT 10; ``` 结果是: -```response +```response ┌─────UserID─┬─Count─┐ │ 2459550954 │ 3741 │ │ 1084649151 │ 2484 │ @@ -639,26 +639,26 @@ LIMIT 10; └────────────┴───────┘ 10 rows in set. Elapsed: 0.086 sec. -// highlight-next-line -Processed 8.81 million rows, +// highlight-next-line +Processed 8.81 million rows, 799.69 MB (102.11 million rows/s., 9.27 GB/s.) -``` +``` 客户端输出表明,尽管URL列是联合主键的一部分,ClickHouse几乎执行了一一次全表扫描!ClickHouse从表的887万行中读取881万行。 如果启用了trace日志,那么ClickHouse服务日志文件显示,ClickHouse在1083个URL索引标记上使用了通用的排除搜索,以便识别那些可能包含URL列值为"http://public_search"的行。 -```response -...Executor): Key condition: (column 1 in ['http://public_search', +```response +...Executor): Key condition: (column 1 in ['http://public_search', 'http://public_search']) -// highlight-next-line -...Executor): Used generic exclusion search over index for part all_1_9_2 +// highlight-next-line +...Executor): Used generic exclusion search over index for part all_1_9_2 with 1537 steps ...Executor): Selected 1/1 parts by partition key, 1 parts by primary key, -// highlight-next-line +// highlight-next-line 1076/1083 marks by primary key, 1076 marks to read from 5 ranges ...Executor): Reading approx. 8814592 rows with 10 streams -``` -我们可以在上面的跟踪日志示例中看到,1083个颗粒中有1076个(通过标记)被选中,因为可能包含具有匹配URL值的行。 +``` +我们可以在上面的跟踪日志示例中看到,1083个颗粒中有1076个(通过标记)被选中,因为可能包含具有匹配URL值的行。 这将导致881万行被读取到ClickHouse引擎中(通过使用10个流并行地读取),以便识别实际包含URL值"http://public_search"的行。 @@ -672,7 +672,7 @@ Processed 8.81 million rows,
- 通用排除搜索算法 + 通用排除搜索算法

@@ -693,7 +693,7 @@ Processed 8.81 million rows, 假设UserID具有较低的基数。在这种情况下,相同的UserID值很可能分布在多个表行和颗粒上,从而分布在索引标记上。对于具有相同UserID的索引标记,索引标记的URL值按升序排序(因为表行首先按UserID排序,然后按URL排序)。这使得有效的过滤如下所述: - + 在上图中,我们的抽象样本数据的颗粒选择过程有三种不同的场景: @@ -704,13 +704,13 @@ Processed 8.81 million rows, 3. 可以排除URL值大于W3的索引标记2和3,因为主索引的索引标记存储了每个颗粒的最小键列值,因此颗粒2和3不可能包含URL值W3。 - + **前缀主键高基数** 当UserID具有较高的基数时,相同的UserID值不太可能分布在多个表行和颗粒上。这意味着索引标记的URL值不是单调递增的: - + 正如在上面的图表中所看到的,所有URL值小于W3的标记都被选中,以便将其关联的颗粒的行加载到ClickHouse引擎中。 @@ -745,7 +745,7 @@ ALTER TABLE hits_UserID_URL MATERIALIZE INDEX url_skipping_index; ``` ClickHouse现在创建了一个额外的索引来存储—每组4个连续的颗粒(注意上面ALTER TABLE语句中的GRANULARITY 4子句)—最小和最大的URL值: - + 第一个索引条目(上图中的mark 0)存储属于表的前4个颗粒的行的最小和最大URL值。 @@ -786,15 +786,15 @@ ClickHouse现在创建了一个额外的索引来存储—每组4个连续的颗 当创建有不同主键的第二个表时,查询必须显式地发送给最适合查询的表版本,并且必须显式地插入新数据到两个表中,以保持表的同步: - + 在物化视图中,额外的表被隐藏,数据自动在两个表之间保持同步: - + projection方式是最透明的选项,因为除了自动保持隐藏的附加表与数据变化同步外,ClickHouse还会自动选择最有效的表版本进行查询: - + 下面我们使用真实的例子详细讨论下这三种方式。 @@ -813,7 +813,7 @@ CREATE TABLE hits_URL_UserID `EventTime` DateTime ) ENGINE = MergeTree -// highlight-next-line +// highlight-next-line PRIMARY KEY (URL, UserID) ORDER BY (URL, UserID, EventTime) SETTINGS index_granularity = 8192, index_granularity_bytes = 0; @@ -822,10 +822,10 @@ SETTINGS index_granularity = 8192, index_granularity_bytes = 0; 写入887万行源表数据: ```sql -INSERT INTO hits_URL_UserID +INSERT INTO hits_URL_UserID SELECT * from hits_UserID_URL; ``` - + 结果: ```response @@ -841,10 +841,10 @@ OPTIMIZE TABLE hits_URL_UserID FINAL; 因为我们切换了主键中列的顺序,插入的行现在以不同的字典顺序存储在磁盘上(与我们的原始表相比),因此该表的1083个颗粒也包含了与以前不同的值: - + 主键索引如下: - + 现在计算最频繁点击URL"http://public_search"的前10名用户,这时候的查询速度是明显加快的: ```sql @@ -856,7 +856,7 @@ GROUP BY UserID ORDER BY Count DESC LIMIT 10; ``` - + 结果: @@ -875,8 +875,8 @@ LIMIT 10; └────────────┴───────┘ 10 rows in set. Elapsed: 0.017 sec. -// highlight-next-line -Processed 319.49 thousand rows, +// highlight-next-line +Processed 319.49 thousand rows, 11.38 MB (18.41 million rows/s., 655.75 MB/s.) ``` @@ -887,15 +887,15 @@ Processed 319.49 thousand rows, 将URL作为主索引的第一列,ClickHouse现在对索引标记运行二分搜索。ClickHouse服务器日志文件中对应的跟踪日志: ```response -...Executor): Key condition: (column 0 in ['http://public_search', +...Executor): Key condition: (column 0 in ['http://public_search', 'http://public_search']) -// highlight-next-line +// highlight-next-line ...Executor): Running binary search on index range for part all_1_9_2 (1083 marks) ...Executor): Found (LEFT) boundary mark: 644 ...Executor): Found (RIGHT) boundary mark: 683 ...Executor): Found continuous range in 19 steps ...Executor): Selected 1/1 parts by partition key, 1 parts by primary key, -// highlight-next-line +// highlight-next-line 39/1083 marks by primary key, 39 marks to read from 1 ranges ...Executor): Reading approx. 319488 rows with 2 streams ``` @@ -920,7 +920,7 @@ GROUP BY URL ORDER BY Count DESC LIMIT 10; ``` - + 结果 ```response @@ -938,8 +938,8 @@ LIMIT 10; └────────────────────────────────┴───────┘ 10 rows in set. Elapsed: 0.024 sec. -// highlight-next-line -Processed 8.02 million rows, +// highlight-next-line +Processed 8.02 million rows, 73.04 MB (340.26 million rows/s., 3.10 GB/s.) ``` @@ -947,10 +947,10 @@ Processed 8.02 million rows, ```response ...Executor): Key condition: (column 1 in [749927693, 749927693]) // highlight-next-line -...Executor): Used generic exclusion search over index for part all_1_9_2 +...Executor): Used generic exclusion search over index for part all_1_9_2 with 1453 steps ...Executor): Selected 1/1 parts by partition key, 1 parts by primary key, -// highlight-next-line +// highlight-next-line 980/1083 marks by primary key, 980 marks to read from 23 ranges ...Executor): Reading approx. 8028160 rows with 10 streams ``` @@ -960,7 +960,7 @@ Processed 8.02 million rows, 现在我们有了两张表。优化了对UserID和URL的查询过滤,分别: - + @@ -981,7 +981,7 @@ ORDER BY (URL, UserID, EventTime) POPULATE AS SELECT * FROM hits_UserID_URL; ``` - + 结果: ```response @@ -993,20 +993,20 @@ Ok. :::note - 我们在视图的主键中切换键列的顺序(与原始表相比) - 物化视图由一个隐藏表支持,该表的行顺序和主索引基于给定的主键定义 -- 我们使用POPULATE关键字,以便用源表hits_UserID_URL中的所有887万行立即导入新的物化视图 +- 我们使用POPULATE关键字,以便用源表hits_UserID_URL中的所有887万行立即导入新的物化视图 - 如果在源表hits_UserID_URL中插入了新行,那么这些行也会自动插入到隐藏表中 - 实际上,隐式创建的隐藏表的行顺序和主索引与我们上面显式创建的辅助表相同: - + ClickHouse将隐藏表的列数据文件(.bin)、标记文件(.mrk2)和主索引(primary.idx)存储在ClickHouse服务器的数据目录的一个特殊文件夹中: - + ::: @@ -1021,7 +1021,7 @@ GROUP BY UserID ORDER BY Count DESC LIMIT 10; ``` - + 结果: ```response @@ -1039,8 +1039,8 @@ LIMIT 10; └────────────┴───────┘ 10 rows in set. Elapsed: 0.026 sec. -// highlight-next-line -Processed 335.87 thousand rows, +// highlight-next-line +Processed 335.87 thousand rows, 13.54 MB (12.91 million rows/s., 520.38 MB/s.) ``` @@ -1049,13 +1049,13 @@ Processed 335.87 thousand rows, ClickHouse服务器日志文件中相应的跟踪日志确认了ClickHouse正在对索引标记运行二分搜索: ```response -...Executor): Key condition: (column 0 in ['http://public_search', +...Executor): Key condition: (column 0 in ['http://public_search', 'http://public_search']) // highlight-next-line ...Executor): Running binary search on index range ... ... ...Executor): Selected 4/4 parts by partition key, 4 parts by primary key, -// highlight-next-line +// highlight-next-line 41/1083 marks by primary key, 41 marks to read from 4 ranges ...Executor): Reading approx. 335872 rows with 4 streams ``` @@ -1095,11 +1095,11 @@ ALTER TABLE hits_UserID_URL - 查询总是(从语法上)针对源表hits_UserID_URL,但是如果隐藏表的行顺序和主索引允许更有效地执行查询,那么将使用该隐藏表 - 实际上,隐式创建的隐藏表的行顺序和主索引与我们显式创建的辅助表相同: - + ClickHouse将隐藏表的列数据文件(.bin)、标记文件(.mrk2)和主索引(primary.idx)存储在一个特殊的文件夹中(在下面的截图中用橙色标记),紧挨着源表的数据文件、标记文件和主索引文件: - + ::: 由投影创建的隐藏表(以及它的主索引)现在可以(隐式地)用于显著加快URL列上查询过滤的执行。注意,查询在语法上针对投影的源表。 @@ -1113,7 +1113,7 @@ GROUP BY UserID ORDER BY Count DESC LIMIT 10; ``` - + 结果: ```response @@ -1130,8 +1130,8 @@ LIMIT 10; │ 765730816 │ 536 │ └────────────┴───────┘ -10 rows in set. Elapsed: 0.029 sec. -// highlight-next-line +10 rows in set. Elapsed: 0.029 sec. +// highlight-next-line Processed 319.49 thousand rows, 1 1.38 MB (11.05 million rows/s., 393.58 MB/s.) ``` @@ -1142,16 +1142,16 @@ ClickHouse服务器日志文件中跟踪日志确认了ClickHouse正在对索引 ```response -...Executor): Key condition: (column 0 in ['http://public_search', +...Executor): Key condition: (column 0 in ['http://public_search', 'http://public_search']) -// highlight-next-line +// highlight-next-line ...Executor): Running binary search on index range for part prj_url_userid (1083 marks) ...Executor): ... // highlight-next-line ...Executor): Choose complete Normal projection prj_url_userid ...Executor): projection required columns: URL, UserID ...Executor): Selected 1/1 parts by partition key, 1 parts by primary key, -// highlight-next-line +// highlight-next-line 39/1083 marks by primary key, 39 marks to read from 1 ranges ...Executor): Reading approx. 319488 rows with 2 streams ``` diff --git a/docs/zh/interfaces/http.md b/docs/zh/interfaces/http.md index e0c12193a6a..c7a0f355a92 100644 --- a/docs/zh/interfaces/http.md +++ b/docs/zh/interfaces/http.md @@ -96,7 +96,7 @@ ECT 1 , expected One of: SHOW TABLES, SHOW DATABASES, SELECT, INSERT, CREATE, ATTACH, RENAME, DROP, DETACH, USE, SET, OPTIMIZE., e.what() = DB::Exception ``` -默认情况下,返回的数据是`TabSeparated`格式的,更多信息,见[Formats](../interfaces/formats/)部分。 +默认情况下,返回的数据是`TabSeparated`格式的,更多信息,见[Formats](../interfaces/formats.md)部分。 您可以使用查询的FORMAT子句来设置其他格式。 diff --git a/docs/zh/operations/optimizing-performance/sampling-query-profiler.md b/docs/zh/operations/optimizing-performance/sampling-query-profiler.md index 4206274ec0d..5d31ab9b245 100644 --- a/docs/zh/operations/optimizing-performance/sampling-query-profiler.md +++ b/docs/zh/operations/optimizing-performance/sampling-query-profiler.md @@ -32,7 +32,7 @@ ClickHouse运行允许分析查询执行的采样探查器。 使用探查器, - 使用 `addressToLine`, `addressToSymbol` 和 `demangle` [内省功能](../../sql-reference/functions/introspection.md) 获取函数名称及其在ClickHouse代码中的位置。 要获取某些查询的配置文件,您需要从以下内容汇总数据 `trace_log` 桌子 您可以通过单个函数或整个堆栈跟踪聚合数据。 -如果你需要想象 `trace_log` 信息,尝试 [flamegraph](../../interfaces/third-party/gui/#clickhouse-flamegraph) 和 [测速镜](https://github.com/laplab/clickhouse-speedscope). +如果你需要想象 `trace_log` 信息,尝试 [flamegraph](../../interfaces/third-party/gui.md#clickhouse-flamegraph) 和 [测速镜](https://github.com/laplab/clickhouse-speedscope). ## 示例 {#example} diff --git a/docs/zh/operations/settings/settings-users.md b/docs/zh/operations/settings/settings-users.md index 3fb97bbddb2..d7fe5bad3c3 100644 --- a/docs/zh/operations/settings/settings-users.md +++ b/docs/zh/operations/settings/settings-users.md @@ -11,7 +11,7 @@ sidebar_label: "\u7528\u6237\u8BBE\u7F6E" `user.xml` 中的 `users` 配置段包含了用户配置 :::note -ClickHouse还支持 [SQL驱动的工作流](../access-rights.md#access-control) 用于管理用户。 我们建议使用它。 +ClickHouse还支持 [SQL驱动的工作流](/docs/en/operations/access-rights#access-control) 用于管理用户。 我们建议使用它。 ::: `users` 配置段的结构: @@ -79,7 +79,7 @@ ClickHouse还支持 [SQL驱动的工作流](../access-rights.md#access-control) ### access_management {#access_management-user-setting} -此设置可为用户启用或禁用 SQL-driven [访问控制和帐户管理](../access-rights.md#access-control) 。 +此设置可为用户启用或禁用 SQL-driven [访问控制和帐户管理](/docs/en/operations/access-rights#access-control) 。 可能的值: diff --git a/docs/zh/operations/system-tables/data_type_families.md b/docs/zh/operations/system-tables/data_type_families.md index 18e9455476d..f0e3a9ef896 100644 --- a/docs/zh/operations/system-tables/data_type_families.md +++ b/docs/zh/operations/system-tables/data_type_families.md @@ -3,7 +3,7 @@ slug: /zh/operations/system-tables/data_type_families --- # system.data_type_families {#system_tables-data_type_families} -包含有关受支持的[数据类型](../../sql-reference/data-types/)的信息. +包含有关受支持的[数据类型](../../sql-reference/data-types/index.md)的信息. 列字段包括: diff --git a/docs/zh/operations/system-tables/replicated_fetches.md b/docs/zh/operations/system-tables/replicated_fetches.md index 7fd517c72ab..c6c37759755 100644 --- a/docs/zh/operations/system-tables/replicated_fetches.md +++ b/docs/zh/operations/system-tables/replicated_fetches.md @@ -68,4 +68,4 @@ thread_id: 54 **另请参阅** -- [管理 ReplicatedMergeTree 表](../../sql-reference/statements/system/#query-language-system-replicated) +- [管理 ReplicatedMergeTree 表](../../sql-reference/statements/system.md#query-language-system-replicated) diff --git a/docs/zh/sql-reference/aggregate-functions/reference/grouparrayinsertat.md b/docs/zh/sql-reference/aggregate-functions/reference/grouparrayinsertat.md index 8431b5a1110..f0672d4fe45 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/grouparrayinsertat.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/grouparrayinsertat.md @@ -20,7 +20,7 @@ groupArrayInsertAt(default_x, size)(x, pos); **参数** -- `x` — 要插入的值。生成所[支持的数据类型](../../../sql-reference/data-types/index.md)(数据)的[表达式](../../../sql-reference/syntax#syntax-expressions)。 +- `x` — 要插入的值。生成所[支持的数据类型](../../../sql-reference/data-types/index.md)(数据)的[表达式](../../../sql-reference/syntax.md#syntax-expressions)。 - `pos` — 指定元素 `x` 将被插入的位置。 数组中的索引编号从零开始。 [UInt32](../../../sql-reference/data-types/int-uint.md#uint-ranges). - `default_x` — 在空位置替换的默认值。可选参数。生成 `x` 数据类型 (数据) 的[表达式](../../../sql-reference/syntax.md#syntax-expressions)。 如果 `default_x` 未定义,则 [默认值](../../../sql-reference/statements/create.md#create-default-values) 被使用。 - `size`— 结果数组的长度。可选参数。如果使用该参数,必须指定默认值 `default_x` 。 [UInt32](../../../sql-reference/data-types/int-uint.md#uint-ranges)。 diff --git a/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.mdx b/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md similarity index 59% rename from docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.mdx rename to docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md index 0c924feda73..fe70d29f8da 100644 --- a/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.mdx +++ b/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md @@ -5,6 +5,4 @@ sidebar_label: Polygon Dictionaries With Grids title: "Polygon dictionaries" --- -import Content from '@site/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md'; - - +View the [english Dictionaries doc page for details](../../../../en/sql-reference/dictionaries/index.md). diff --git a/docs/zh/sql-reference/statements/create/database.md b/docs/zh/sql-reference/statements/create/database.md index 2c6e53c0f06..3e5b71fb196 100644 --- a/docs/zh/sql-reference/statements/create/database.md +++ b/docs/zh/sql-reference/statements/create/database.md @@ -27,4 +27,4 @@ ClickHouse在指定集群的所有服务器上创建`db_name`数据库。 更多 ### ENGINE {#engine} -[MySQL](../../../engines/database-engines/mysql.md) 允许您从远程MySQL服务器检索数据. 默认情况下,ClickHouse使用自己的[database engine](../../../engines/database-engines/index.md). 还有一个[lazy](../../../engines/database-engines/lazy)引擎. +[MySQL](../../../engines/database-engines/mysql.md) 允许您从远程MySQL服务器检索数据. 默认情况下,ClickHouse使用自己的[database engine](../../../engines/database-engines/index.md). 还有一个[lazy](../../../engines/database-engines/lazy.md)引擎. diff --git a/docs/zh/sql-reference/statements/create/view.md b/docs/zh/sql-reference/statements/create/view.md index a000c69f1ef..8ce2d20a10c 100644 --- a/docs/zh/sql-reference/statements/create/view.md +++ b/docs/zh/sql-reference/statements/create/view.md @@ -63,7 +63,7 @@ ClickHouse 中的物化视图更像是插入触发器。 如果视图查询中 视图看起来与普通表相同。 例如,它们列在`SHOW TABLES`查询的结果中。 -删除视图,使用[DROP VIEW](../../../sql-reference/statements/drop#drop-view). `DROP TABLE`也适用于视图。 +删除视图,使用[DROP VIEW](../../../sql-reference/statements/drop.md#drop-view). `DROP TABLE`也适用于视图。 ## Live View (实验性) {#live-view} diff --git a/docs/zh/sql-reference/statements/index.md b/docs/zh/sql-reference/statements/index.md index cf51dadc8f1..989c368ebc4 100644 --- a/docs/zh/sql-reference/statements/index.md +++ b/docs/zh/sql-reference/statements/index.md @@ -20,7 +20,7 @@ sidebar_position: 31 - [CHECK TABLE](../../sql-reference/statements/check-table.mdx) - [DESCRIBE TABLE](../../sql-reference/statements/describe-table.mdx) - [DETACH](../../sql-reference/statements/detach.mdx) -- [DROP](../../sql-reference/statements/drop) +- [DROP](../../sql-reference/statements/drop.md) - [EXISTS](../../sql-reference/statements/exists.md) - [KILL](../../sql-reference/statements/kill.mdx) - [OPTIMIZE](../../sql-reference/statements/optimize.mdx) diff --git a/docs/zh/sql-reference/statements/select/array-join.md b/docs/zh/sql-reference/statements/select/array-join.md index b0352a7bb0a..4162a39f399 100644 --- a/docs/zh/sql-reference/statements/select/array-join.md +++ b/docs/zh/sql-reference/statements/select/array-join.md @@ -146,7 +146,7 @@ ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num, arrayMap(x -> x + 1, arr) AS ma └───────┴─────────┴───┴─────┴────────┘ ``` -下面的例子使用 [arrayEnumerate](../../../sql-reference/functions/array-functions#array_functions-arrayenumerate) 功能: +下面的例子使用 [arrayEnumerate](../../../sql-reference/functions/array-functions.md#array_functions-arrayenumerate) 功能: ``` sql SELECT s, arr, a, num, arrayEnumerate(arr) @@ -259,7 +259,7 @@ ARRAY JOIN nest AS n; └───────┴─────┴─────┴─────────┴────────────┘ ``` -使用功能 [arrayEnumerate](../../../sql-reference/functions/array-functions#array_functions-arrayenumerate) 的例子: +使用功能 [arrayEnumerate](../../../sql-reference/functions/array-functions.md#array_functions-arrayenumerate) 的例子: ``` sql SELECT s, `n.x`, `n.y`, `nest.x`, `nest.y`, num diff --git a/docs/zh/sql-reference/statements/select/group-by.md b/docs/zh/sql-reference/statements/select/group-by.md index 29c72ce7e45..86511470538 100644 --- a/docs/zh/sql-reference/statements/select/group-by.md +++ b/docs/zh/sql-reference/statements/select/group-by.md @@ -8,7 +8,7 @@ sidebar_label: GROUP BY `GROUP BY` 子句将 `SELECT` 查询结果转换为聚合模式,其工作原理如下: - `GROUP BY` 子句包含表达式列表(或单个表达式 -- 可以认为是长度为1的列表)。 这份名单充当 “grouping key”,而每个单独的表达式将被称为 “key expressions”. -- 在所有的表达式在 [SELECT](../../../sql-reference/statements/select/index.md), [HAVING](../../../sql-reference/statements/select/having),和 [ORDER BY](../../../sql-reference/statements/select/order-by.md) 子句中 **必须** 基于键表达式进行计算 **或** 上 [聚合函数](../../../sql-reference/aggregate-functions/index.md) 在非键表达式(包括纯列)上。 换句话说,从表中选择的每个列必须用于键表达式或聚合函数内,但不能同时使用。 +- 在所有的表达式在 [SELECT](../../../sql-reference/statements/select/index.md), [HAVING](../../../sql-reference/statements/select/having.md),和 [ORDER BY](../../../sql-reference/statements/select/order-by.md) 子句中 **必须** 基于键表达式进行计算 **或** 上 [聚合函数](../../../sql-reference/aggregate-functions/index.md) 在非键表达式(包括纯列)上。 换句话说,从表中选择的每个列必须用于键表达式或聚合函数内,但不能同时使用。 - 聚合结果 `SELECT` 查询将包含尽可能多的行,因为有唯一值 “grouping key” 在源表中。 通常这会显着减少行数,通常是数量级,但不一定:如果所有行数保持不变 “grouping key” 值是不同的。 :::note @@ -58,7 +58,7 @@ sidebar_label: GROUP BY - 在 `Pretty*` 格式时,该行在主结果之后作为单独的表输出。 - 在其他格式中,它不可用。 -`WITH TOTALS` 可以以不同的方式运行时 [HAVING](../../../sql-reference/statements/select/having) 是存在的。 该行为取决于 `totals_mode` 设置。 +`WITH TOTALS` 可以以不同的方式运行时 [HAVING](../../../sql-reference/statements/select/having.md) 是存在的。 该行为取决于 `totals_mode` 设置。 ### 配置总和处理 {#configuring-totals-processing} diff --git a/docs/zh/sql-reference/statements/select/index.md b/docs/zh/sql-reference/statements/select/index.md index 2d4044cbd20..fdf196e198b 100644 --- a/docs/zh/sql-reference/statements/select/index.md +++ b/docs/zh/sql-reference/statements/select/index.md @@ -41,7 +41,7 @@ SELECT [DISTINCT] expr_list - [WHERE 子句](../../../sql-reference/statements/select/where.md) - [GROUP BY 子句](../../../sql-reference/statements/select/group-by.md) - [LIMIT BY 子句](../../../sql-reference/statements/select/limit-by.md) -- [HAVING 子句](../../../sql-reference/statements/select/having) +- [HAVING 子句](../../../sql-reference/statements/select/having.md) - [SELECT 子句](#select-clause) - [DISTINCT 子句](../../../sql-reference/statements/select/distinct.md) - [LIMIT 子句](../../../sql-reference/statements/select/limit.md) From c10011ddf93794bce0f470a93b4748c8c3f4d08b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 18 Mar 2023 04:38:04 +0100 Subject: [PATCH 256/559] Fix error --- src/Functions/FunctionsBitmap.h | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionsBitmap.h b/src/Functions/FunctionsBitmap.h index cea04f1c49e..29ab0abed87 100644 --- a/src/Functions/FunctionsBitmap.h +++ b/src/Functions/FunctionsBitmap.h @@ -410,11 +410,11 @@ private: container0 = &col_agg_func->getData(); if (is_column_const[1]) - container1 = &typeid_cast(*typeid_cast(column_ptrs[1]).getDataColumnPtr()).getData(); + container1 = &typeid_cast(typeid_cast(*column_ptrs[1]).getDataColumn()).getData(); else container1 = &typeid_cast(*column_ptrs[1]).getData(); if (is_column_const[2]) - container2 = &typeid_cast(*typeid_cast(column_ptrs[2]).getDataColumnPtr()).getData(); + container2 = &typeid_cast(typeid_cast(*column_ptrs[2]).getDataColumn()).getData(); else container2 = &typeid_cast(*column_ptrs[2]).getData(); @@ -911,7 +911,7 @@ private: is_column_const[0] = isColumnConst(*column_ptrs[0]); if (is_column_const[0]) - container0 = &typeid_cast(*typeid_cast(*column_ptrs[0]).getDataColumnPtr()).getData(); + container0 = &typeid_cast(typeid_cast(*column_ptrs[0]).getDataColumn()).getData(); else container0 = &typeid_cast(*column_ptrs[0]).getData(); @@ -921,7 +921,7 @@ private: is_column_const[1] = isColumnConst(*column_ptrs[1]); if (is_column_const[1]) - container1 = &typeid_cast(*typeid_cast(*column_ptrs[1]).getDataColumnPtr()).getData(); + container1 = &typeid_cast(typeid_cast(*column_ptrs[1]).getDataColumn()).getData(); else container1 = &typeid_cast(*column_ptrs[1]).getData(); From 498b517acbebf6dee87665487ae02a2319dc7d49 Mon Sep 17 00:00:00 2001 From: clickhouse-adrianfraguela <119855513+clickhouse-adrianfraguela@users.noreply.github.com> Date: Sat, 18 Mar 2023 08:53:13 +0000 Subject: [PATCH 257/559] Updated date in Austin Meetup --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 509fd395f58..61d840ecd34 100644 --- a/README.md +++ b/README.md @@ -21,7 +21,7 @@ curl https://clickhouse.com/ | sh * [Contacts](https://clickhouse.com/company/contact) can help to get your questions answered if there are any. ## Upcoming Events -* [**ClickHouse Meetup in Austin**](https://www.meetup.com/clickhouse-austin-user-group/events/291486654/) - Mar 16 - The first ClickHouse Meetup in Austin is happening soon! Interested in speaking, let us know! +* [**ClickHouse Meetup in Austin**](https://www.meetup.com/clickhouse-austin-user-group/events/291486654/) - Mar 30 - The first ClickHouse Meetup in Austin is happening soon! Interested in speaking, let us know! * [**v23.3 Release Webinar**](https://clickhouse.com/company/events/v23-3-release-webinar?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-02) - Mar 30 - 23.3 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. ## Recent Recordings From dbfe2df1f4b8e9ed4eaf4e76d84f740beff880af Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sat, 18 Mar 2023 19:48:00 -0300 Subject: [PATCH 258/559] Update coordinates.md --- .../functions/geo/coordinates.md | 39 +++++++++++++++++-- 1 file changed, 35 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/geo/coordinates.md b/docs/en/sql-reference/functions/geo/coordinates.md index 1e023415890..01802e336bf 100644 --- a/docs/en/sql-reference/functions/geo/coordinates.md +++ b/docs/en/sql-reference/functions/geo/coordinates.md @@ -31,13 +31,13 @@ Generates an exception when the input parameter values fall outside of the range **Example** ``` sql -SELECT greatCircleDistance(55.755831, 37.617673, -55.755831, -37.617673) +SELECT greatCircleDistance(55.755831, 37.617673, -55.755831, -37.617673) AS greatCircleDistance ``` ``` text -┌─greatCircleDistance(55.755831, 37.617673, -55.755831, -37.617673)─┐ -│ 14132374.194975413 │ -└───────────────────────────────────────────────────────────────────┘ +┌─greatCircleDistance─┐ +│ 14128352 │ +└─────────────────────┘ ``` ## geoDistance @@ -47,6 +47,37 @@ The performance is the same as for `greatCircleDistance` (no performance drawbac Technical note: for close enough points we calculate the distance using planar approximation with the metric on the tangent plane at the midpoint of the coordinates. +``` sql +geoDistance(lon1Deg, lat1Deg, lon2Deg, lat2Deg) +``` + +**Input parameters** + +- `lon1Deg` — Longitude of the first point in degrees. Range: `[-180°, 180°]`. +- `lat1Deg` — Latitude of the first point in degrees. Range: `[-90°, 90°]`. +- `lon2Deg` — Longitude of the second point in degrees. Range: `[-180°, 180°]`. +- `lat2Deg` — Latitude of the second point in degrees. Range: `[-90°, 90°]`. + +Positive values correspond to North latitude and East longitude, and negative values correspond to South latitude and West longitude. + +**Returned value** + +The distance between two points on the Earth’s surface, in meters. + +Generates an exception when the input parameter values fall outside of the range. + +**Example** + +``` sql +SELECT geoDistance(38.8976, -77.0366, 39.9496, -75.1503) AS geoDistance +``` + +``` text +┌─geoDistance─┐ +│ 212458.73 │ +└─────────────┘ +``` + ## greatCircleAngle Calculates the central angle between two points on the Earth’s surface using [the great-circle formula](https://en.wikipedia.org/wiki/Great-circle_distance). From d9c7bc1859dc3b7bde81af2a40b6fec0ce4707a9 Mon Sep 17 00:00:00 2001 From: Ongkong Date: Sat, 18 Mar 2023 15:53:00 -0700 Subject: [PATCH 259/559] Fix ASOF LEFT JOIN performance degradation (#47544) --- src/Interpreters/RowRefs.cpp | 5 ++--- tests/performance/asof.xml | 30 ++++++++++++++++++++++++++++++ 2 files changed, 32 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/RowRefs.cpp b/src/Interpreters/RowRefs.cpp index 2d4f807ad46..4335cde47f9 100644 --- a/src/Interpreters/RowRefs.cpp +++ b/src/Interpreters/RowRefs.cpp @@ -74,9 +74,8 @@ class SortedLookupVector : public SortedLookupVectorBase public: - using Keys = std::vector; - using Entries = PaddedPODArray; - using RowRefs = PaddedPODArray; + using Entries = PODArrayWithStackMemory; + using RowRefs = PODArrayWithStackMemory; static constexpr bool is_descending = (inequality == ASOFJoinInequality::Greater || inequality == ASOFJoinInequality::GreaterOrEquals); static constexpr bool is_strict = (inequality == ASOFJoinInequality::Less) || (inequality == ASOFJoinInequality::Greater); diff --git a/tests/performance/asof.xml b/tests/performance/asof.xml index d9f119fae40..d00afaa26b5 100644 --- a/tests/performance/asof.xml +++ b/tests/performance/asof.xml @@ -41,4 +41,34 @@ ON (visits.fingerprint = origins.fingerprint AND visits.date >= origins.date) FORMAT Null + + + + num_rows + + 1000000 + + + + + + SELECT COUNT(*) AS count + FROM + ( + SELECT + number AS id, + number AS visitor_id + FROM system.numbers + LIMIT {num_rows} + ) AS sessions + ASOF LEFT JOIN + ( + SELECT + number AS visitor_id, + number AS starting_session_id + FROM system.numbers + LIMIT {num_rows} + ) AS visitors + ON visitors.visitor_id = sessions.visitor_id AND visitors.starting_session_id < sessions.id + From f32eec8ef66180dd93c5ddb65480f4f22b2671b1 Mon Sep 17 00:00:00 2001 From: pufit Date: Sat, 18 Mar 2023 20:32:30 -0400 Subject: [PATCH 260/559] Add tests --- .../0_stateless/01304_polygons_sym_difference.reference | 5 +++++ tests/queries/0_stateless/01304_polygons_sym_difference.sql | 4 ++++ 2 files changed, 9 insertions(+) diff --git a/tests/queries/0_stateless/01304_polygons_sym_difference.reference b/tests/queries/0_stateless/01304_polygons_sym_difference.reference index 9344410f192..1ab25a27e12 100644 --- a/tests/queries/0_stateless/01304_polygons_sym_difference.reference +++ b/tests/queries/0_stateless/01304_polygons_sym_difference.reference @@ -5,3 +5,8 @@ MULTIPOLYGON(((-20 -10.3067,-20 -20,-10 -20.8791,-10 -40,-40 -40,-40 -10,-20 -10 MULTIPOLYGON(((-10 -20.8791,-20 -20,-20 -10.3067,-10 -10,-10 -20.8791)),((10 20.8791,20 20,20 10.3067,10 10,10 20.8791)),((50 50,50 -50,-50 -50,-50 50,50 50),(20 10.3067,40 10,40 40,10 40,10 20.8791,-20 20,-20 -10.3067,-40 -10,-40 -40,-10 -40,-10 -20.8791,20 -20,20 10.3067))) -------- Polygon with Polygon with Holes MULTIPOLYGON(((-20 -10.3067,-10 -10,-10 -20.8791,-20 -20,-20 -10.3067)),((10 20.8791,20 20,20 10.3067,10 10,10 20.8791)),((50 50,50 -50,-50 -50,-50 50,50 50),(20 10.3067,40 10,40 40,10 40,10 20.8791,-20 20,-20 -10.3067,-40 -10,-40 -40,-10 -40,-10 -20.8791,20 -20,20 10.3067))) +-------- Check GROUP BY and ROLL UP +[] +[] +[[(2147483647,0),(10.0001,65535),(1,255),(1023,2147483646)]] [[[(2147483647,0),(10.0001,65535),(1023,2147483646),(2147483647,0)]]] +[[(2147483647,0),(10.0001,65535),(1,255),(1023,2147483646)]] [] diff --git a/tests/queries/0_stateless/01304_polygons_sym_difference.sql b/tests/queries/0_stateless/01304_polygons_sym_difference.sql index c4129676b26..e8b5502e00b 100644 --- a/tests/queries/0_stateless/01304_polygons_sym_difference.sql +++ b/tests/queries/0_stateless/01304_polygons_sym_difference.sql @@ -13,3 +13,7 @@ SELECT wkt(arraySort(polygonsSymDifferenceSpherical([[(50.,50.),(50.,-50.),(-50. select '-------- Polygon with Polygon with Holes'; select wkt(polygonsSymDifferenceSpherical([[(29.453587685533865,59.779570356240356),(29.393139070478895,52.276266797422124),(40.636581470703206,59.38168915000267),(41.21084331372543,59.103467777099866),(29.786055068336193,52.146627480315004),(31.23682182965546,52.16517054781818),(41.69443223416517,58.85424941916091),(42.51048853740727,58.47703162291134),(32.59691566839227,52.22075341251539),(34.289476889931414,52.22075341251539),(43.02430176537451,58.07974369546071),(43.02430176537451,57.25537683364851),(35.468224883503325,52.2022335126388),(37.16078610504247,52.23926559241349),(43.02430176537451,56.26136189644947),(43.02430176537451,55.326904361850836),(38.33953409861437,52.16517054781818),(40.09254393520848,52.16517054781818),(44.4146199116388,55.3097062225408),(44.47506852669377,59.80998197603594),(39.72985224487867,59.931351417569715),(30.23941968124846,53.67744677450975),(30.20919537372098,54.63314259659509),(38.73245009647167,59.94649146557819),(37.2816833351524,59.97675082987618),(30.23941968124846,55.2752875586599),(30.33009260383092,56.19415599955667),(36.28428118674541,59.96162460231375),(34.863738732953635,59.97675082987618),(30.178971066193498,56.97640788219866),(30.178971066193498,57.91957806959033),(33.65476643185424,59.94649146557819),(32.32489690064491,59.94649146557819),(30.481214141468342,58.85424941916091),(30.571887064050795,59.99187015036608),(29.453587685533865,59.779570356240356)]], [[(24.367675781249993,61.45977057029751),(19.577636718749993,58.67693767258692),(19.577636718749993,57.492213666700735),(19.445800781249996,55.87531083569678),(19.445800781249996,54.085173420886775),(17.468261718749996,53.014783245859235),(20.017089843749993,51.563412328675895),(21.203613281249993,50.205033264943324),(26.125488281249993,50.40151532278236),(27.22412109374999,48.980216985374994),(32.80517578124999,49.525208341974405),(35.26611328124999,48.74894534343292),(36.93603515624999,49.66762782262194),(42.56103515625,48.77791275550183),(43.92333984374999,49.8096315635631),(47.17529296875,49.152969656170455),(49.28466796875,50.54136296522162),(48.05419921875,51.17934297928929),(51.39404296875,52.48278022207825),(50.64697265625,53.014783245859235),(52.88818359375,53.93021986394004),(51.65771484374999,54.29088164657006),(52.66845703125,55.825973254619015),(50.25146484375,56.145549500679095),(51.92138671875,57.914847767009206),(49.15283203125,58.17070248348605),(49.59228515625,60.086762746260064),(47.043457031249986,59.88893689676584),(43.57177734375,61.37567331572748),(42.64892578125,60.630101766266705),(36.89208984374999,62.000904713685856),(36.01318359374999,61.143235250840576),(31.398925781249993,62.02152819100766),(30.563964843749996,61.05828537037917),(26.872558593749993,61.71070595883174),(26.652832031249993,61.10078883158897),(24.367675781249993,61.45977057029751)], [(24.455566406249993,59.42272750081452),(21.203613281249993,58.49369382056807),(21.335449218749993,56.89700392127261),(21.599121093749993,55.92458580482949),(25.202636718749993,55.998380955359636),(28.850097656249993,57.06463027327854),(27.09228515625,57.844750992890994),(28.806152343749996,59.17592824927138),(26.257324218749993,59.17592824927138),(24.455566406249993,59.42272750081452)], [(35.13427734375,59.84481485969107),(31.970214843749993,58.97266715450152),(33.20068359374999,56.776808316568406),(36.67236328125,56.41390137600675),(39.08935546874999,57.25528054528888),(42.69287109374999,58.03137242177638),(40.89111328124999,59.26588062825809),(37.28759765625,58.722598828043374),(37.11181640624999,59.66774058164964),(35.13427734375,59.84481485969107)], [(29.157714843749993,55.75184939173528),(22.565917968749993,55.128649068488784),(22.565917968749993,53.54030739150019),(22.038574218749996,51.48138289610097),(26.257324218749993,51.42661449707484),(30.124511718749993,50.54136296522162),(32.18994140624999,51.17934297928929),(30.124511718749993,53.173119202640635),(35.09033203124999,53.173119202640635),(33.11279296875,54.085173420886775),(29.597167968749993,55.50374985927513),(29.157714843749993,55.75184939173528)], [(42.82470703125,56.58369172128337),(36.584472656249986,55.329144408405085),(37.99072265625,53.592504809039355),(34.95849609374999,51.48138289610097),(36.54052734374999,50.40151532278236),(39.66064453124999,50.289339253291786),(39.79248046875,52.13348804077148),(41.77001953125,50.68079714532166),(44.49462890624999,51.97134580885171),(47.30712890624999,52.509534770327264),(44.05517578125,53.54030739150019),(46.60400390625,53.696706475303245),(47.61474609375,55.40406982700608),(45.37353515625,55.40406982700608),(42.82470703125,56.58369172128337)]])) format Null; SELECT wkt(arraySort(polygonsSymDifferenceSpherical([[(50., 50.), (50., -50.), (-50., -50.), (-50., 50.), (50., 50.)], [(10., 10.), (10., 40.), (40., 40.), (40., 10.), (10., 10.)], [(-10., -10.), (-10., -40.), (-40., -40.), (-40., -10.), (-10., -10.)]], [[(-20., -20.), (-20., 20.), (20., 20.), (20., -20.), (-20., -20.)]]))); + +SELECT '-------- Check GROUP BY and ROLL UP'; +SELECT polygonsSymDifferenceCartesian([[[(1., 1.)]] AS x], [x]) GROUP BY x WITH ROLLUP; +SELECT [[(2147483647, 0.), (10.0001, 65535), (1, 255), (1023, 2147483646)]], polygonsSymDifferenceCartesian([[[(2147483647, 0.), (10.0001, 65535), (1023, 2147483646)]]], [[[(1000.0001, 10.0001)]]]) GROUP BY [[(2147483647, 0.), (10.0001, 65535), (1023, 2147483646)]] WITH ROLLUP; From e84d1336c509d96b50ea564926efc98a4cd8aaaa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 01:55:34 +0100 Subject: [PATCH 261/559] Fix test --- tests/queries/0_stateless/02684_bson.sql | Bin 8725 -> 9049 bytes 1 file changed, 0 insertions(+), 0 deletions(-) diff --git a/tests/queries/0_stateless/02684_bson.sql b/tests/queries/0_stateless/02684_bson.sql index ef29d1a32ec72f6750a4a888772e41662832017c..577bd4ffd27dcad1ece67fe85757c10cf5528e56 100644 GIT binary patch delta 1027 zcmbR0a?@?Y8VDBA$-fp zOGW3ZLf8;e$T=r7HLt|A3`m4z=B6@;LBts+_lk*cUchb72($1ZkDnh{K%@PU?^6^M41}m*G zh6pF=0cineA5RwsFogVm}AMO_)5*gsCP|F1XPbVA; delta 681 zcmccVHq~Xr8V;rpk0#$1)@50DtZMt@`2u1b*A*sBwKBVYce5bpWk#m;4=2CknkV@m z2!xz-GE?(PT+4t&NM>#-gV?)=jFSVz#W%}{+c7e$ZGSvDQNr6i&c*YHKzL?eN`ARt zB2X1btAKw&YF=s)g71-8T#{c@>64n5UXlS;viYxsC6jnlkNVZezlB1Bd^{41GZ@a+ z82`GEq}M$;R$78%!7sO4mnRz)PM#-h#+;|}1iP~*3v%AsyjXArqY0W>We3U_&mQPJ zF5v9r>B0a8{p<`3Lg9%e$r(VAH<#-9dM3{nwq(Ap05)4`Zn{{gKu~IWW_}(}{wYw4 zP-t;#5s=+nJiW5=`R6pCWnz>h5uh&wAQ9;qz(8@N Rcb}(Wq)RI6Gb_|`0RWTtAmji5 From dc19e46cab3816dead23880757a59ca158c24386 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 02:14:41 +0100 Subject: [PATCH 262/559] Add more fuzzers --- docs/en/development/build.md | 1 - src/AggregateFunctions/CMakeLists.txt | 4 ++ src/AggregateFunctions/fuzzers/CMakeLists.txt | 2 + ..._function_state_deserialization_fuzzer.cpp | 71 +++++++++++++++++++ .../data_type_deserialization_fuzzer.cpp | 2 - 5 files changed, 77 insertions(+), 3 deletions(-) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index d52b018a5a7..804aa8a3dc5 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -159,4 +159,3 @@ The CI checks build the binaries on each commit to [ClickHouse](https://github.c 1. Find the type of package for your operating system that you need and download the files. ![build artifact check](images/find-build-artifact.png) - diff --git a/src/AggregateFunctions/CMakeLists.txt b/src/AggregateFunctions/CMakeLists.txt index 0cb38fc729a..a45adde1a36 100644 --- a/src/AggregateFunctions/CMakeLists.txt +++ b/src/AggregateFunctions/CMakeLists.txt @@ -28,3 +28,7 @@ target_link_libraries(clickhouse_aggregate_functions PRIVATE dbms PUBLIC ch_cont if(ENABLE_EXAMPLES) add_subdirectory(examples) endif() + +if (ENABLE_FUZZING) + add_subdirectory(fuzzers) +endif() diff --git a/src/AggregateFunctions/fuzzers/CMakeLists.txt b/src/AggregateFunctions/fuzzers/CMakeLists.txt index e69de29bb2d..3876ffac7ab 100644 --- a/src/AggregateFunctions/fuzzers/CMakeLists.txt +++ b/src/AggregateFunctions/fuzzers/CMakeLists.txt @@ -0,0 +1,2 @@ +clickhouse_add_executable(aggregate_function_state_deserialization_fuzzer aggregate_function_state_deserialization_fuzzer.cpp ${SRCS}) +target_link_libraries(aggregate_function_state_deserialization_fuzzer PRIVATE dbms clickhouse_aggregate_functions ${LIB_FUZZING_ENGINE}) diff --git a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp index e69de29bb2d..cf9ee89c87e 100644 --- a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp +++ b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp @@ -0,0 +1,71 @@ +#include + +#include +#include + +#include +#include + +#include + +#include + + +extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) +try +{ + using namespace DB; + + static SharedContextHolder shared_context; + static ContextMutablePtr context; + + auto initialize = [&]() mutable + { + shared_context = Context::createShared(); + context = Context::createGlobal(shared_context.get()); + context->makeGlobalContext(); + context->setApplicationType(Context::ApplicationType::LOCAL); + + registerAggregateFunctions(); + return true; + }; + + static bool initialized = initialize(); + (void) initialized; + + /// The input format is as follows: + /// - the aggregate function name on the first line, possible with parameters, then data types of the arguments, + /// example: quantile(0.5), Float64 + /// - the serialized aggregation state for the rest of the input. + + /// Compile the code as follows: + /// mkdir build_asan_fuzz + /// cd build_asan_fuzz + /// CC=clang CXX=clang++ cmake -D SANITIZE=address -D ENABLE_FUZZING=1 -D WITH_COVERAGE=1 .. + /// + /// The corpus is located here: + /// https://github.com/ClickHouse/fuzz-corpus/tree/main/aggregate_function_state_deserialization + /// + /// The fuzzer can be run as follows: + /// ../../../build_asan_fuzz/src/DataTypes/fuzzers/aggregate_function_state_deserialization corpus -jobs=64 -rss_limit_mb=8192 + + DB::ReadBufferFromMemory in(data, size); + + String args; + readStringUntilNewlineInto(args, in); + assertChar('\n', in); + + DataTypePtr type = DataTypeFactory::instance().get(fmt::format("AggregateFunction({})", args)); + AggregateFunctionPtr func = assert_cast(*type).getFunction(); + + Arena arena; + char * place = arena.alignedAlloc(func->sizeOfData(), func->alignOfData()); + func->create(place); + func->deserialize(place, in, {}, &arena); + + return 0; +} +catch (...) +{ + return 1; +} diff --git a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp index 76b8c9c0171..d744db12dc5 100644 --- a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp +++ b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp @@ -5,8 +5,6 @@ #include -#include - #include #include From 43c931c1a45d641bf40f23781f5b6efb347e9688 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 02:27:00 +0100 Subject: [PATCH 263/559] Update the fuzzer --- .../fuzzers/aggregate_function_state_deserialization_fuzzer.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp index cf9ee89c87e..53915f8b7e1 100644 --- a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp +++ b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp @@ -61,6 +61,7 @@ try Arena arena; char * place = arena.alignedAlloc(func->sizeOfData(), func->alignOfData()); func->create(place); + SCOPE_EXIT(func->destroy(place)); func->deserialize(place, in, {}, &arena); return 0; From d0090119a1cfb2e915634bc31f84c3b8089bf803 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 02:39:59 +0100 Subject: [PATCH 264/559] I hate short variable names wholeheartedly --- .../AggregateFunctionGroupBitmap.h | 26 +++--- .../AggregateFunctionGroupBitmapData.h | 93 +++++++++---------- src/Functions/FunctionsBitmap.h | 44 ++++----- 3 files changed, 81 insertions(+), 82 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmap.h b/src/AggregateFunctions/AggregateFunctionGroupBitmap.h index 5fe3128fa20..ae519a3d908 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmap.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmap.h @@ -31,22 +31,22 @@ public: void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { - this->data(place).rbs.add(assert_cast &>(*columns[0]).getData()[row_num]); + this->data(place).roaring_bitmap_with_small_set.add(assert_cast &>(*columns[0]).getData()[row_num]); } void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override { - this->data(place).rbs.merge(this->data(rhs).rbs); + this->data(place).roaring_bitmap_with_small_set.merge(this->data(rhs).roaring_bitmap_with_small_set); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).rbs.write(buf); } + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).roaring_bitmap_with_small_set.write(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).rbs.read(buf); } + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).roaring_bitmap_with_small_set.read(buf); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override { assert_cast &>(to).getData().push_back( - static_cast(this->data(place).rbs.size())); + static_cast(this->data(place).roaring_bitmap_with_small_set.size())); } }; @@ -81,7 +81,7 @@ public: if (!data_lhs.init) { data_lhs.init = true; - data_lhs.rbs.merge(data_rhs.rbs); + data_lhs.roaring_bitmap_with_small_set.merge(data_rhs.roaring_bitmap_with_small_set); } else { @@ -100,7 +100,7 @@ public: if (!data_lhs.init) { data_lhs.init = true; - data_lhs.rbs.merge(data_rhs.rbs); + data_lhs.roaring_bitmap_with_small_set.merge(data_rhs.roaring_bitmap_with_small_set); } else { @@ -128,7 +128,7 @@ public: if (*version >= 1) DB::writeBoolText(this->data(place).init, buf); - this->data(place).rbs.write(buf); + this->data(place).roaring_bitmap_with_small_set.write(buf); } void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional version, Arena *) const override @@ -138,13 +138,13 @@ public: if (*version >= 1) DB::readBoolText(this->data(place).init, buf); - this->data(place).rbs.read(buf); + this->data(place).roaring_bitmap_with_small_set.read(buf); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override { assert_cast &>(to).getData().push_back( - static_cast(this->data(place).rbs.size())); + static_cast(this->data(place).roaring_bitmap_with_small_set.size())); } }; @@ -154,7 +154,7 @@ class BitmapAndPolicy { public: static constexpr auto name = "groupBitmapAnd"; - static void apply(Data & lhs, const Data & rhs) { lhs.rbs.rb_and(rhs.rbs); } + static void apply(Data & lhs, const Data & rhs) { lhs.roaring_bitmap_with_small_set.rb_and(rhs.roaring_bitmap_with_small_set); } }; template @@ -162,7 +162,7 @@ class BitmapOrPolicy { public: static constexpr auto name = "groupBitmapOr"; - static void apply(Data & lhs, const Data & rhs) { lhs.rbs.rb_or(rhs.rbs); } + static void apply(Data & lhs, const Data & rhs) { lhs.roaring_bitmap_with_small_set.rb_or(rhs.roaring_bitmap_with_small_set); } }; template @@ -170,7 +170,7 @@ class BitmapXorPolicy { public: static constexpr auto name = "groupBitmapXor"; - static void apply(Data & lhs, const Data & rhs) { lhs.rbs.rb_xor(rhs.rbs); } + static void apply(Data & lhs, const Data & rhs) { lhs.roaring_bitmap_with_small_set.rb_xor(rhs.roaring_bitmap_with_small_set); } }; template diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index 6a4c48f0fff..df115c7fffc 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -46,20 +46,19 @@ private: using ValueBuffer = std::vector; using RoaringBitmap = std::conditional_t= 8, roaring::Roaring64Map, roaring::Roaring>; using Value = std::conditional_t= 8, UInt64, UInt32>; - std::shared_ptr rb = nullptr; + std::shared_ptr roaring_bitmap; void toLarge() { - rb = std::make_shared(); + roaring_bitmap = std::make_shared(); for (const auto & x : small) - rb->add(static_cast(x.getValue())); + roaring_bitmap->add(static_cast(x.getValue())); small.clear(); } public: - bool isLarge() const { return rb != nullptr; } - - bool isSmall() const { return rb == nullptr; } + bool isLarge() const { return roaring_bitmap != nullptr; } + bool isSmall() const { return roaring_bitmap == nullptr; } void add(T value) { @@ -72,13 +71,13 @@ public: else { toLarge(); - rb->add(static_cast(value)); + roaring_bitmap->add(static_cast(value)); } } } else { - rb->add(static_cast(value)); + roaring_bitmap->add(static_cast(value)); } } @@ -87,7 +86,7 @@ public: if (isSmall()) return small.size(); else - return rb->cardinality(); + return roaring_bitmap->cardinality(); } void merge(const RoaringBitmapWithSmallSet & r1) @@ -97,7 +96,7 @@ public: if (isSmall()) toLarge(); - *rb |= *r1.rb; + *roaring_bitmap |= *r1.roaring_bitmap; } else { @@ -123,7 +122,7 @@ public: throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size in groupBitmap."); std::unique_ptr buf(new char[size]); in.readStrict(buf.get(), size); - rb = std::make_shared(RoaringBitmap::read(buf.get())); + roaring_bitmap = std::make_shared(RoaringBitmap::read(buf.get())); } } @@ -137,10 +136,10 @@ public: } else if (BitmapKind::Bitmap == kind) { - auto size = rb->getSizeInBytes(); + auto size = roaring_bitmap->getSizeInBytes(); writeVarUInt(size, out); std::unique_ptr buf(new char[size]); - rb->write(buf.get()); + roaring_bitmap->write(buf.get()); out.write(buf.get(), size); } } @@ -181,7 +180,7 @@ public: { for (const auto & x : small) { - if (r1.rb->contains(static_cast(x.getValue()))) + if (r1.roaring_bitmap->contains(static_cast(x.getValue()))) buffer.push_back(x.getValue()); } @@ -195,8 +194,8 @@ public: } else { - std::shared_ptr new_rb = r1.isSmall() ? r1.getNewRoaringBitmapFromSmall() : r1.rb; - *rb &= *new_rb; + std::shared_ptr new_rb = r1.isSmall() ? r1.getNewRoaringBitmapFromSmall() : r1.roaring_bitmap; + *roaring_bitmap &= *new_rb; } } @@ -213,8 +212,8 @@ public: if (isSmall()) toLarge(); - std::shared_ptr new_rb = r1.isSmall() ? r1.getNewRoaringBitmapFromSmall() : r1.rb; - *rb ^= *new_rb; + std::shared_ptr new_rb = r1.isSmall() ? r1.getNewRoaringBitmapFromSmall() : r1.roaring_bitmap; + *roaring_bitmap ^= *new_rb; } /** @@ -242,7 +241,7 @@ public: { for (const auto & x : small) { - if (!r1.rb->contains(static_cast(x.getValue()))) + if (!r1.roaring_bitmap->contains(static_cast(x.getValue()))) buffer.push_back(x.getValue()); } @@ -256,8 +255,8 @@ public: } else { - std::shared_ptr new_rb = r1.isSmall() ? r1.getNewRoaringBitmapFromSmall() : r1.rb; - *rb -= *new_rb; + std::shared_ptr new_rb = r1.isSmall() ? r1.getNewRoaringBitmapFromSmall() : r1.roaring_bitmap; + *roaring_bitmap -= *new_rb; } } @@ -277,14 +276,14 @@ public: { for (const auto & x : small) { - if (r1.rb->contains(static_cast(x.getValue()))) + if (r1.roaring_bitmap->contains(static_cast(x.getValue()))) ++ret; } } else { - std::shared_ptr new_rb = r1.isSmall() ? r1.getNewRoaringBitmapFromSmall() : r1.rb; - ret = (*rb & *new_rb).cardinality(); + std::shared_ptr new_rb = r1.isSmall() ? r1.getNewRoaringBitmapFromSmall() : r1.roaring_bitmap; + ret = (*roaring_bitmap & *new_rb).cardinality(); } return ret; } @@ -329,8 +328,8 @@ public: if (isSmall()) toLarge(); - std::shared_ptr new_rb = r1.isSmall() ? r1.getNewRoaringBitmapFromSmall() : r1.rb; - return *rb == *new_rb; + std::shared_ptr new_rb = r1.isSmall() ? r1.getNewRoaringBitmapFromSmall() : r1.roaring_bitmap; + return *roaring_bitmap == *new_rb; } /** @@ -351,7 +350,7 @@ public: { for (const auto & x : small) { - if (r1.rb->contains(static_cast(x.getValue()))) + if (r1.roaring_bitmap->contains(static_cast(x.getValue()))) return 1; } } @@ -360,13 +359,13 @@ public: { for (const auto & x : r1.small) { - if (rb->contains(static_cast(x.getValue()))) + if (roaring_bitmap->contains(static_cast(x.getValue()))) return 1; } } else { - if ((*rb & *r1.rb).cardinality() > 0) + if ((*roaring_bitmap & *r1.roaring_bitmap).cardinality() > 0) return 1; } @@ -404,7 +403,7 @@ public: // greater then r1 is not a subset. for (const auto & x : small) { - if (!r1.rb->contains(static_cast(x.getValue())) && ++r1_size > small.size()) + if (!r1.roaring_bitmap->contains(static_cast(x.getValue())) && ++r1_size > small.size()) return 0; } } @@ -413,13 +412,13 @@ public: { for (const auto & x : r1.small) { - if (!rb->contains(static_cast(x.getValue()))) + if (!roaring_bitmap->contains(static_cast(x.getValue()))) return 0; } } else { - if (!r1.rb->isSubset(*rb)) + if (!r1.roaring_bitmap->isSubset(*roaring_bitmap)) return 0; } return 1; @@ -436,7 +435,7 @@ public: if (isSmall()) return small.find(static_cast(x)) != small.end(); else - return rb->contains(static_cast(x)); + return roaring_bitmap->contains(static_cast(x)); } /** @@ -450,7 +449,7 @@ public: if (isSmall()) toLarge(); - rb->remove(x); + roaring_bitmap->remove(x); } /** @@ -464,7 +463,7 @@ public: if (isSmall()) toLarge(); - rb->flip(begin, end); + roaring_bitmap->flip(begin, end); } /** @@ -475,7 +474,7 @@ public: if (isSmall()) toLarge(); - return rb->rank(x); + return roaring_bitmap->rank(x); } /** @@ -495,7 +494,7 @@ public: } else { - for (auto it = rb->begin(); it != rb->end(); ++it) + for (auto it = roaring_bitmap->begin(); it != roaring_bitmap->end(); ++it) { res.emplace_back(*it); ++count; @@ -527,7 +526,7 @@ public: } else { - for (auto it = rb->begin(); it != rb->end(); ++it) + for (auto it = roaring_bitmap->begin(); it != roaring_bitmap->end(); ++it) { if (*it < range_start) continue; @@ -577,7 +576,7 @@ public: else { UInt64 count = 0; - for (auto it = rb->begin(); it != rb->end(); ++it) + for (auto it = roaring_bitmap->begin(); it != roaring_bitmap->end(); ++it) { if (*it < range_start) continue; @@ -615,11 +614,11 @@ public: { UInt64 count = 0; UInt64 offset_count = 0; - auto it = rb->begin(); - for (;it != rb->end() && offset_count < offset; ++it) + auto it = roaring_bitmap->begin(); + for (;it != roaring_bitmap->end() && offset_count < offset; ++it) ++offset_count; - for (;it != rb->end() && count < limit; ++it, ++count) + for (;it != roaring_bitmap->end() && count < limit; ++it, ++count) r1.add(*it); return count; } @@ -641,7 +640,7 @@ public: return min_val; } else - return rb->minimum(); + return roaring_bitmap->minimum(); } UInt64 rb_max() const /// NOLINT @@ -660,7 +659,7 @@ public: return max_val; } else - return rb->maximum(); + return roaring_bitmap->maximum(); } /** @@ -676,9 +675,9 @@ public: { if (from_vals[i] == to_vals[i]) continue; - bool changed = rb->removeChecked(static_cast(from_vals[i])); + bool changed = roaring_bitmap->removeChecked(static_cast(from_vals[i])); if (changed) - rb->add(static_cast(to_vals[i])); + roaring_bitmap->add(static_cast(to_vals[i])); } } }; @@ -688,7 +687,7 @@ struct AggregateFunctionGroupBitmapData { // If false, all bitmap operations will be treated as merge to initialize the state bool init = false; - RoaringBitmapWithSmallSet rbs; + RoaringBitmapWithSmallSet roaring_bitmap_with_small_set; static const char * name() { return "groupBitmap"; } }; diff --git a/src/Functions/FunctionsBitmap.h b/src/Functions/FunctionsBitmap.h index 29ab0abed87..2292b896952 100644 --- a/src/Functions/FunctionsBitmap.h +++ b/src/Functions/FunctionsBitmap.h @@ -211,7 +211,7 @@ private: = *reinterpret_cast *>(col_to->getData()[i]); for (; pos < offsets[i]; ++pos) { - bitmap_data.rbs.add(input_data[pos]); + bitmap_data.roaring_bitmap_with_small_set.add(input_data[pos]); } } return col_to; @@ -303,7 +303,7 @@ private: { const AggregateFunctionGroupBitmapData & bitmap_data_1 = *reinterpret_cast *>(column->getData()[i]); - UInt64 count = bitmap_data_1.rbs.rb_to_array(res_data); + UInt64 count = bitmap_data_1.roaring_bitmap_with_small_set.rb_to_array(res_data); res_offset += count; res_offsets.emplace_back(res_offset); } @@ -449,7 +449,7 @@ public: UInt64 range_end, AggregateFunctionGroupBitmapData & bitmap_data_2) { - bitmap_data_0.rbs.rb_range(range_start, range_end, bitmap_data_2.rbs); + bitmap_data_0.roaring_bitmap_with_small_set.rb_range(range_start, range_end, bitmap_data_2.roaring_bitmap_with_small_set); } }; @@ -464,7 +464,7 @@ public: UInt64 range_end, AggregateFunctionGroupBitmapData & bitmap_data_2) { - bitmap_data_0.rbs.rb_limit(range_start, range_end, bitmap_data_2.rbs); + bitmap_data_0.roaring_bitmap_with_small_set.rb_limit(range_start, range_end, bitmap_data_2.roaring_bitmap_with_small_set); } }; @@ -479,7 +479,7 @@ public: UInt64 range_end, AggregateFunctionGroupBitmapData & bitmap_data_2) { - bitmap_data_0.rbs.rb_offset_limit(range_start, range_end, bitmap_data_2.rbs); + bitmap_data_0.roaring_bitmap_with_small_set.rb_offset_limit(range_start, range_end, bitmap_data_2.roaring_bitmap_with_small_set); } }; @@ -649,8 +649,8 @@ private: col_to->insertDefault(); AggregateFunctionGroupBitmapData & bitmap_data_2 = *reinterpret_cast *>(col_to->getData()[i]); - bitmap_data_2.rbs.merge(bitmap_data_0.rbs); - bitmap_data_2.rbs.rb_replace(&from_container[from_start], &to_container[to_start], from_end - from_start); + bitmap_data_2.roaring_bitmap_with_small_set.merge(bitmap_data_0.roaring_bitmap_with_small_set); + bitmap_data_2.roaring_bitmap_with_small_set.rb_replace(&from_container[from_start], &to_container[to_start], from_end - from_start); } return col_to; } @@ -740,7 +740,7 @@ public: template static UInt64 apply(const AggregateFunctionGroupBitmapData & bitmap_data) { - return bitmap_data.rbs.size(); + return bitmap_data.roaring_bitmap_with_small_set.size(); } }; @@ -751,7 +751,7 @@ public: template static UInt64 apply(const AggregateFunctionGroupBitmapData & bitmap_data) { - return bitmap_data.rbs.rb_min(); + return bitmap_data.roaring_bitmap_with_small_set.rb_min(); } }; @@ -762,7 +762,7 @@ public: template static UInt64 apply(const AggregateFunctionGroupBitmapData & bitmap_data) { - return bitmap_data.rbs.rb_max(); + return bitmap_data.roaring_bitmap_with_small_set.rb_max(); } }; @@ -773,7 +773,7 @@ struct BitmapAndCardinalityImpl static UInt64 apply(const AggregateFunctionGroupBitmapData & bitmap_data_1, const AggregateFunctionGroupBitmapData & bitmap_data_2) { // roaring_bitmap_and_cardinality( rb1, rb2 ); - return bitmap_data_1.rbs.rb_and_cardinality(bitmap_data_2.rbs); + return bitmap_data_1.roaring_bitmap_with_small_set.rb_and_cardinality(bitmap_data_2.roaring_bitmap_with_small_set); } }; @@ -785,7 +785,7 @@ struct BitmapOrCardinalityImpl static UInt64 apply(const AggregateFunctionGroupBitmapData & bitmap_data_1, const AggregateFunctionGroupBitmapData & bitmap_data_2) { // return roaring_bitmap_or_cardinality( rb1, rb2 ); - return bitmap_data_1.rbs.rb_or_cardinality(bitmap_data_2.rbs); + return bitmap_data_1.roaring_bitmap_with_small_set.rb_or_cardinality(bitmap_data_2.roaring_bitmap_with_small_set); } }; @@ -796,7 +796,7 @@ struct BitmapXorCardinalityImpl static UInt64 apply(const AggregateFunctionGroupBitmapData & bitmap_data_1, const AggregateFunctionGroupBitmapData & bitmap_data_2) { // return roaring_bitmap_xor_cardinality( rb1, rb2 ); - return bitmap_data_1.rbs.rb_xor_cardinality(bitmap_data_2.rbs); + return bitmap_data_1.roaring_bitmap_with_small_set.rb_xor_cardinality(bitmap_data_2.roaring_bitmap_with_small_set); } }; @@ -807,7 +807,7 @@ struct BitmapAndnotCardinalityImpl static UInt64 apply(const AggregateFunctionGroupBitmapData & bitmap_data_1, const AggregateFunctionGroupBitmapData & bitmap_data_2) { // roaring_bitmap_andnot_cardinality( rb1, rb2 ); - return bitmap_data_1.rbs.rb_andnot_cardinality(bitmap_data_2.rbs); + return bitmap_data_1.roaring_bitmap_with_small_set.rb_andnot_cardinality(bitmap_data_2.roaring_bitmap_with_small_set); } }; @@ -817,7 +817,7 @@ struct BitmapHasAllImpl using ReturnType = UInt8; static UInt8 apply(const AggregateFunctionGroupBitmapData & bitmap_data_1, const AggregateFunctionGroupBitmapData & bitmap_data_2) { - return bitmap_data_1.rbs.rb_is_subset(bitmap_data_2.rbs); + return bitmap_data_1.roaring_bitmap_with_small_set.rb_is_subset(bitmap_data_2.roaring_bitmap_with_small_set); } }; @@ -827,7 +827,7 @@ struct BitmapHasAnyImpl using ReturnType = UInt8; static UInt8 apply(const AggregateFunctionGroupBitmapData & bitmap_data_1, const AggregateFunctionGroupBitmapData & bitmap_data_2) { - return bitmap_data_1.rbs.rb_intersect(bitmap_data_2.rbs); + return bitmap_data_1.roaring_bitmap_with_small_set.rb_intersect(bitmap_data_2.roaring_bitmap_with_small_set); } }; @@ -931,7 +931,7 @@ private: const UInt64 data1 = is_column_const[1] ? (*container1)[0] : (*container1)[i]; const AggregateFunctionGroupBitmapData & bitmap_data_0 = *reinterpret_cast *>(data_ptr_0); - vec_to[i] = bitmap_data_0.rbs.rb_contains(data1); + vec_to[i] = bitmap_data_0.roaring_bitmap_with_small_set.rb_contains(data1); } } }; @@ -1050,7 +1050,7 @@ struct BitmapAndImpl { static void apply(AggregateFunctionGroupBitmapData & bitmap_data_1, const AggregateFunctionGroupBitmapData & bitmap_data_2) { - bitmap_data_1.rbs.rb_and(bitmap_data_2.rbs); + bitmap_data_1.roaring_bitmap_with_small_set.rb_and(bitmap_data_2.roaring_bitmap_with_small_set); } }; @@ -1059,7 +1059,7 @@ struct BitmapOrImpl { static void apply(AggregateFunctionGroupBitmapData & bitmap_data_1, const AggregateFunctionGroupBitmapData & bitmap_data_2) { - bitmap_data_1.rbs.rb_or(bitmap_data_2.rbs); + bitmap_data_1.roaring_bitmap_with_small_set.rb_or(bitmap_data_2.roaring_bitmap_with_small_set); } }; @@ -1068,7 +1068,7 @@ struct BitmapXorImpl { static void apply(AggregateFunctionGroupBitmapData & bitmap_data_1, const AggregateFunctionGroupBitmapData & bitmap_data_2) { - bitmap_data_1.rbs.rb_xor(bitmap_data_2.rbs); + bitmap_data_1.roaring_bitmap_with_small_set.rb_xor(bitmap_data_2.roaring_bitmap_with_small_set); } }; @@ -1077,7 +1077,7 @@ struct BitmapAndnotImpl { static void apply(AggregateFunctionGroupBitmapData & bitmap_data_1, const AggregateFunctionGroupBitmapData & bitmap_data_2) { - bitmap_data_1.rbs.rb_andnot(bitmap_data_2.rbs); + bitmap_data_1.roaring_bitmap_with_small_set.rb_andnot(bitmap_data_2.roaring_bitmap_with_small_set); } }; @@ -1190,7 +1190,7 @@ private: auto * bm_2 = reinterpret_cast *>(data_ptr_1); // check the name of operation (bitmapAnd) and check if it is the situation mentioned above - auto need_exchange = (name == NameBitmapAnd::name) && bm_1->rbs.isLarge() && bm_2->rbs.isSmall(); + auto need_exchange = (name == NameBitmapAnd::name) && bm_1->roaring_bitmap_with_small_set.isLarge() && bm_2->roaring_bitmap_with_small_set.isSmall(); col_to->insertFrom(need_exchange ? data_ptr_1 : data_ptr_0); AggregateFunctionGroupBitmapData & bitmap_data_1 = *reinterpret_cast *>(col_to->getData()[i]); const AggregateFunctionGroupBitmapData & bitmap_data_2 From 9a7883143ba35e0e47dcc6f6844ae981ac868c64 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 02:44:39 +0100 Subject: [PATCH 265/559] Small adjustments --- .../AggregateFunctionGroupBitmap.h | 10 +++- .../AggregateFunctionGroupBitmapData.h | 52 +++++-------------- 2 files changed, 20 insertions(+), 42 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmap.h b/src/AggregateFunctions/AggregateFunctionGroupBitmap.h index ae519a3d908..a32bb330884 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmap.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmap.h @@ -39,9 +39,15 @@ public: this->data(place).roaring_bitmap_with_small_set.merge(this->data(rhs).roaring_bitmap_with_small_set); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).roaring_bitmap_with_small_set.write(buf); } + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override + { + this->data(place).roaring_bitmap_with_small_set.write(buf); + } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).roaring_bitmap_with_small_set.read(buf); } + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override + { + this->data(place).roaring_bitmap_with_small_set.read(buf); + } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override { diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index df115c7fffc..2e3b851129b 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -67,7 +67,9 @@ public: if (small.find(value) == small.end()) { if (!small.full()) + { small.insert(value); + } else { toLarge(); @@ -109,6 +111,7 @@ public: { UInt8 kind; readBinary(kind, in); + if (BitmapKind::Small == kind) { small.read(in); @@ -117,19 +120,24 @@ public: { size_t size; readVarUInt(size, in); + static constexpr size_t max_size = 1_GiB; if (size > max_size) throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size in groupBitmap."); + std::unique_ptr buf(new char[size]); in.readStrict(buf.get(), size); roaring_bitmap = std::make_shared(RoaringBitmap::read(buf.get())); } + else + throw Exception(ErrorCodes::INCORRECT_DATA, "Unknown type of roaring bitmap"); } void write(DB::WriteBuffer & out) const { UInt8 kind = isLarge() ? BitmapKind::Bitmap : BitmapKind::Small; writeBinary(kind, out); + if (BitmapKind::Small == kind) { small.write(out); @@ -202,7 +210,10 @@ public: /** * Computes the union between two bitmaps. */ - void rb_or(const RoaringBitmapWithSmallSet & r1) { merge(r1); } /// NOLINT + void rb_or(const RoaringBitmapWithSmallSet & r1) + { + merge(r1); /// NOLINT + } /** * Computes the symmetric difference (xor) between two bitmaps. @@ -438,45 +449,6 @@ public: return roaring_bitmap->contains(static_cast(x)); } - /** - * Remove value - */ - void rb_remove(UInt64 x) /// NOLINT - { - if (!std::is_same_v && x > rb_max()) - return; - - if (isSmall()) - toLarge(); - - roaring_bitmap->remove(x); - } - - /** - * compute (in place) the negation of the roaring bitmap within a specified - * interval: [range_start, range_end). The number of negated values is - * range_end - range_start. - * Areas outside the range are passed through unchanged. - */ - void rb_flip(UInt64 begin, UInt64 end) /// NOLINT - { - if (isSmall()) - toLarge(); - - roaring_bitmap->flip(begin, end); - } - - /** - * returns the number of integers that are smaller or equal to offsetid. - */ - UInt64 rb_rank(UInt64 x) /// NOLINT - { - if (isSmall()) - toLarge(); - - return roaring_bitmap->rank(x); - } - /** * Convert elements to integer array, return number of elements */ From 8549825a74243e9d982b9f158197a0a7e208d568 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 02:55:13 +0100 Subject: [PATCH 266/559] Fix obvious error --- src/AggregateFunctions/AggregateFunctionGroupBitmapData.h | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index 2e3b851129b..1a75bd99659 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -122,12 +122,17 @@ public: readVarUInt(size, in); static constexpr size_t max_size = 1_GiB; + + if (size == 0) + throw Exception(ErrorCodes::INCORRECT_DATA, "Incorrect size (0) in groupBitmap."); if (size > max_size) throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size in groupBitmap."); + /// TODO: this is unnecessary copying - it will be better to read and deserialize in one pass. std::unique_ptr buf(new char[size]); in.readStrict(buf.get(), size); - roaring_bitmap = std::make_shared(RoaringBitmap::read(buf.get())); + + roaring_bitmap = std::make_shared(RoaringBitmap::readSafe(buf.get(), size)); } else throw Exception(ErrorCodes::INCORRECT_DATA, "Unknown type of roaring bitmap"); From ef015797267fa42a1b2786e7f9264af6ebb1ae4c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 03:19:15 +0100 Subject: [PATCH 267/559] Fix trash - the code written in C with pointers --- src/Common/SpaceSaving.h | 53 +++++++++++++++++----------------------- 1 file changed, 23 insertions(+), 30 deletions(-) diff --git a/src/Common/SpaceSaving.h b/src/Common/SpaceSaving.h index 84494e25ca5..476e107067b 100644 --- a/src/Common/SpaceSaving.h +++ b/src/Common/SpaceSaving.h @@ -160,12 +160,11 @@ public: // Key doesn't exist, but can fit in the top K if (unlikely(size() < capacity())) { - auto * c = new Counter(arena.emplace(key), increment, error, hash); - push(c); + push(std::make_unique(arena.emplace(key), increment, error, hash)); return; } - auto * min = counter_list.back(); + auto & min = counter_list.back(); // The key doesn't exist and cannot fit in the current top K, but // the new key has a bigger weight and is virtually more present // compared to the element who is less present on the set. This part @@ -173,7 +172,7 @@ public: if (increment > min->count) { destroyLastElement(); - push(new Counter(arena.emplace(key), increment, error, hash)); + push(std::make_unique(arena.emplace(key), increment, error, hash)); return; } @@ -189,7 +188,7 @@ public: alpha_map[min->hash & alpha_mask] = min->count; destroyLastElement(); - push(new Counter(arena.emplace(key), alpha + increment, alpha + error, hash)); + push(std::make_unique(arena.emplace(key), alpha + increment, alpha + error, hash)); } /* @@ -219,7 +218,7 @@ public: */ if (m2 > 0) { - for (auto * counter : counter_list) + for (auto & counter : counter_list) { counter->count += m2; counter->error += m2; @@ -227,7 +226,7 @@ public: } // The list is sorted in descending order, we have to scan in reverse - for (auto * counter : boost::adaptors::reverse(rhs.counter_list)) + for (auto & counter : boost::adaptors::reverse(rhs.counter_list)) { size_t hash = counter_map.hash(counter->key); if (auto * current = findCounter(counter->key, hash)) @@ -239,19 +238,16 @@ public: else { // Counters not monitored in S1 - counter_list.push_back(new Counter(arena.emplace(counter->key), counter->count + m1, counter->error + m1, hash)); + counter_list.push_back(std::make_unique(arena.emplace(counter->key), counter->count + m1, counter->error + m1, hash)); } } - ::sort(counter_list.begin(), counter_list.end(), [](Counter * l, Counter * r) { return *l > *r; }); + ::sort(counter_list.begin(), counter_list.end(), [](const auto & l, const auto & r) { return *l > *r; }); if (counter_list.size() > m_capacity) { for (size_t i = m_capacity; i < counter_list.size(); ++i) - { arena.free(counter_list[i]->key); - delete counter_list[i]; - } counter_list.resize(m_capacity); } @@ -263,7 +259,7 @@ public: std::vector topK(size_t k) const { std::vector res; - for (auto * counter : counter_list) + for (auto & counter : counter_list) { res.push_back(*counter); if (res.size() == k) @@ -275,7 +271,7 @@ public: void write(WriteBuffer & wb) const { writeVarUInt(size(), wb); - for (auto * counter : counter_list) + for (auto & counter : counter_list) counter->write(wb); writeVarUInt(alpha_map.size(), wb); @@ -291,10 +287,10 @@ public: for (size_t i = 0; i < count; ++i) { - auto * counter = new Counter(); + std::unique_ptr counter = std::make_unique(); counter->read(rb); counter->hash = counter_map.hash(counter->key); - push(counter); + push(std::move(counter)); } readAlphaMap(rb); @@ -313,12 +309,13 @@ public: } protected: - void push(Counter * counter) + void push(std::unique_ptr counter) { counter->slot = counter_list.size(); - counter_list.push_back(counter); - counter_map[counter->key] = counter; - percolate(counter); + auto * ptr = counter.get(); + counter_list.push_back(std::move(counter)); + counter_map[ptr->key] = ptr; + percolate(ptr); } // This is equivallent to one step of bubble sort @@ -326,7 +323,7 @@ protected: { while (counter->slot > 0) { - auto * next = counter_list[counter->slot - 1]; + auto & next = counter_list[counter->slot - 1]; if (*counter > *next) { std::swap(next->slot, counter->slot); @@ -340,11 +337,8 @@ protected: private: void destroyElements() { - for (auto * counter : counter_list) - { + for (auto & counter : counter_list) arena.free(counter->key); - delete counter; - } counter_map.clear(); counter_list.clear(); @@ -353,10 +347,9 @@ private: void destroyLastElement() { - auto last_element = counter_list.back(); + auto & last_element = counter_list.back(); counter_map.erase(last_element->key); arena.free(last_element->key); - delete last_element; counter_list.pop_back(); ++removed_keys; @@ -377,14 +370,14 @@ private: { removed_keys = 0; counter_map.clear(); - for (auto * counter : counter_list) - counter_map[counter->key] = counter; + for (auto & counter : counter_list) + counter_map[counter->key] = counter.get(); } using CounterMap = HashMapWithStackMemory; CounterMap counter_map; - std::vector> counter_list; + std::vector, AllocatorWithMemoryTracking>> counter_list; std::vector> alpha_map; SpaceSavingArena arena; size_t m_capacity; From a270f985a74150843255fb62bfe988c425507d1a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 04:35:20 +0100 Subject: [PATCH 268/559] Update roaring bitmaps, because the previous version had a bug --- contrib/croaring | 2 +- contrib/croaring-cmake/CMakeLists.txt | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/contrib/croaring b/contrib/croaring index 2c867e9f9c9..f40ed52bcdd 160000 --- a/contrib/croaring +++ b/contrib/croaring @@ -1 +1 @@ -Subproject commit 2c867e9f9c9e2a3a7032791f94c4c7ae3013f6e0 +Subproject commit f40ed52bcdd635840a79877cef4857315dba817c diff --git a/contrib/croaring-cmake/CMakeLists.txt b/contrib/croaring-cmake/CMakeLists.txt index 0bb7d0bd221..794c0426b96 100644 --- a/contrib/croaring-cmake/CMakeLists.txt +++ b/contrib/croaring-cmake/CMakeLists.txt @@ -17,7 +17,8 @@ set(SRCS "${LIBRARY_DIR}/src/containers/run.c" "${LIBRARY_DIR}/src/roaring.c" "${LIBRARY_DIR}/src/roaring_priority_queue.c" - "${LIBRARY_DIR}/src/roaring_array.c") + "${LIBRARY_DIR}/src/roaring_array.c" + "${LIBRARY_DIR}/src/memory.c") add_library(_roaring ${SRCS}) From 0ab192db8277f440debf7b55b55b29463736d21b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 05:10:35 +0100 Subject: [PATCH 269/559] Add a test --- tests/queries/0_stateless/02688_aggregate_states.reference | 0 tests/queries/0_stateless/02688_aggregate_states.sql | 4 ++++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/02688_aggregate_states.reference create mode 100644 tests/queries/0_stateless/02688_aggregate_states.sql diff --git a/tests/queries/0_stateless/02688_aggregate_states.reference b/tests/queries/0_stateless/02688_aggregate_states.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02688_aggregate_states.sql b/tests/queries/0_stateless/02688_aggregate_states.sql new file mode 100644 index 00000000000..adc1184d717 --- /dev/null +++ b/tests/queries/0_stateless/02688_aggregate_states.sql @@ -0,0 +1,4 @@ +SELECT '\x01\x00'::AggregateFunction(groupBitmap, UInt32); -- { serverError INCORRECT_DATA } +SELECT '\x01\x01\x01'::AggregateFunction(groupBitmap, UInt64); -- { serverError STD_EXCEPTION } +SELECT '\x02\x00\x0d'::AggregateFunction(topK, UInt256); -- { serverError CANNOT_READ_ALL_DATA } +SELECT unhex('bebebebebebebebebebebebebebebebebebebebebebebebebebebebebebebe0c0c3131313131313131313131313173290aee00b300')::AggregateFunction(minDistinct, Int8); -- { serverError TOO_LARGE_ARRAY_SIZE } From ee98b555fbb637c7d3ba2cc7016c3eb0008553ad Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 05:11:32 +0100 Subject: [PATCH 270/559] Limit memory in fuzzers --- .../aggregate_function_state_deserialization_fuzzer.cpp | 4 ++++ src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp | 4 ++++ src/Formats/fuzzers/format_fuzzer.cpp | 4 ++++ 3 files changed, 12 insertions(+) diff --git a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp index 53915f8b7e1..d0e13c21ddf 100644 --- a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp +++ b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp @@ -6,6 +6,8 @@ #include #include +#include + #include #include @@ -26,6 +28,8 @@ try context->makeGlobalContext(); context->setApplicationType(Context::ApplicationType::LOCAL); + total_memory_tracker.setHardLimit(1_GiB); + registerAggregateFunctions(); return true; }; diff --git a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp index d744db12dc5..09bdadaeb95 100644 --- a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp +++ b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp @@ -5,6 +5,8 @@ #include +#include + #include #include @@ -25,6 +27,8 @@ try context->makeGlobalContext(); context->setApplicationType(Context::ApplicationType::LOCAL); + total_memory_tracker.setHardLimit(1_GiB); + registerAggregateFunctions(); return true; }; diff --git a/src/Formats/fuzzers/format_fuzzer.cpp b/src/Formats/fuzzers/format_fuzzer.cpp index 9983d493e45..054a799feb4 100644 --- a/src/Formats/fuzzers/format_fuzzer.cpp +++ b/src/Formats/fuzzers/format_fuzzer.cpp @@ -12,6 +12,8 @@ #include #include +#include + #include #include @@ -33,6 +35,8 @@ try context->makeGlobalContext(); context->setApplicationType(Context::ApplicationType::LOCAL); + total_memory_tracker.setHardLimit(1_GiB); + registerAggregateFunctions(); registerFormats(); From 57a5a946c91dcf31827c61908b23f56b606e4256 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 05:34:10 +0100 Subject: [PATCH 271/559] Fix error --- .../fuzzers/aggregate_function_state_deserialization_fuzzer.cpp | 2 ++ src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp | 2 ++ src/Formats/fuzzers/format_fuzzer.cpp | 2 ++ 3 files changed, 6 insertions(+) diff --git a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp index d0e13c21ddf..08a2a50c04c 100644 --- a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp +++ b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp @@ -7,6 +7,7 @@ #include #include +#include #include @@ -28,6 +29,7 @@ try context->makeGlobalContext(); context->setApplicationType(Context::ApplicationType::LOCAL); + ThreadStatus thread_status; total_memory_tracker.setHardLimit(1_GiB); registerAggregateFunctions(); diff --git a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp index 09bdadaeb95..cba70a2bbef 100644 --- a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp +++ b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp @@ -6,6 +6,7 @@ #include #include +#include #include @@ -27,6 +28,7 @@ try context->makeGlobalContext(); context->setApplicationType(Context::ApplicationType::LOCAL); + ThreadStatus thread_status; total_memory_tracker.setHardLimit(1_GiB); registerAggregateFunctions(); diff --git a/src/Formats/fuzzers/format_fuzzer.cpp b/src/Formats/fuzzers/format_fuzzer.cpp index 054a799feb4..fde20ae466f 100644 --- a/src/Formats/fuzzers/format_fuzzer.cpp +++ b/src/Formats/fuzzers/format_fuzzer.cpp @@ -13,6 +13,7 @@ #include #include +#include #include #include @@ -35,6 +36,7 @@ try context->makeGlobalContext(); context->setApplicationType(Context::ApplicationType::LOCAL); + ThreadStatus thread_status; total_memory_tracker.setHardLimit(1_GiB); registerAggregateFunctions(); From a99f10e16ff11e23958305c16d339ecbfdb2f945 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 05:34:44 +0100 Subject: [PATCH 272/559] Better error messages --- src/Common/HashTable/HashSet.h | 3 +++ src/Common/HashTable/HashTable.h | 3 +++ src/Common/HashTable/SmallTable.h | 4 +++- src/Storages/MergeTree/RangesInDataPart.cpp | 8 ++++++++ 4 files changed, 17 insertions(+), 1 deletion(-) diff --git a/src/Common/HashTable/HashSet.h b/src/Common/HashTable/HashSet.h index bac858b16a5..8f3761599ab 100644 --- a/src/Common/HashTable/HashSet.h +++ b/src/Common/HashTable/HashSet.h @@ -16,6 +16,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int TOO_LARGE_ARRAY_SIZE; } } @@ -60,6 +61,8 @@ public: size_t new_size = 0; DB::readVarUInt(new_size, rb); + if (new_size > 100'000'000'000) + throw DB::Exception(DB::ErrorCodes::TOO_LARGE_ARRAY_SIZE, "The size of serialized hash table is suspiciously large: {}", new_size); this->resize(new_size); diff --git a/src/Common/HashTable/HashTable.h b/src/Common/HashTable/HashTable.h index 5c348f936d2..7ddcbc20b22 100644 --- a/src/Common/HashTable/HashTable.h +++ b/src/Common/HashTable/HashTable.h @@ -42,6 +42,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int NO_AVAILABLE_DATA; extern const int CANNOT_ALLOCATE_MEMORY; + extern const int TOO_LARGE_ARRAY_SIZE; } } @@ -1318,6 +1319,8 @@ public: size_t new_size = 0; DB::readVarUInt(new_size, rb); + if (new_size > 100'000'000'000) + throw DB::Exception(DB::ErrorCodes::TOO_LARGE_ARRAY_SIZE, "The size of serialized hash table is suspiciously large: {}", new_size); free(); Grower new_grower = grower; diff --git a/src/Common/HashTable/SmallTable.h b/src/Common/HashTable/SmallTable.h index f6253c3f0b9..3229e4748ea 100644 --- a/src/Common/HashTable/SmallTable.h +++ b/src/Common/HashTable/SmallTable.h @@ -9,6 +9,7 @@ namespace DB { extern const int NO_AVAILABLE_DATA; extern const int INCORRECT_DATA; + extern const int TOO_LARGE_ARRAY_SIZE; } } @@ -279,6 +280,8 @@ public: size_t new_size = 0; DB::readVarUInt(new_size, rb); + if (new_size > 1000'000) + throw DB::Exception(DB::ErrorCodes::TOO_LARGE_ARRAY_SIZE, "The size of serialized small table is suspiciously large: {}", new_size); if (new_size > capacity) throw DB::Exception(DB::ErrorCodes::INCORRECT_DATA, "Illegal size"); @@ -346,4 +349,3 @@ template size_t capacity > using SmallSet = SmallTable, capacity>; - diff --git a/src/Storages/MergeTree/RangesInDataPart.cpp b/src/Storages/MergeTree/RangesInDataPart.cpp index 29a236c9865..ab76611a507 100644 --- a/src/Storages/MergeTree/RangesInDataPart.cpp +++ b/src/Storages/MergeTree/RangesInDataPart.cpp @@ -11,6 +11,12 @@ namespace DB { +namespace ErrorCodes +{ + extern const int TOO_LARGE_ARRAY_SIZE; +} + + void RangesInDataPartDescription::serialize(WriteBuffer & out) const { info.serialize(out); @@ -50,6 +56,8 @@ void RangesInDataPartsDescription::deserialize(ReadBuffer & in) { size_t new_size = 0; readVarUInt(new_size, in); + if (new_size > 100'000'000'000) + throw DB::Exception(DB::ErrorCodes::TOO_LARGE_ARRAY_SIZE, "The size of serialized hash table is suspiciously large: {}", new_size); this->resize(new_size); for (auto & desc : *this) From 465a89ba15421cca0e6e5a55149824d988bbbfc5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 05:55:53 +0100 Subject: [PATCH 273/559] Limit memory in fuzzers --- .../aggregate_function_state_deserialization_fuzzer.cpp | 4 +++- src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp | 4 +++- src/Formats/fuzzers/format_fuzzer.cpp | 4 +++- 3 files changed, 9 insertions(+), 3 deletions(-) diff --git a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp index 08a2a50c04c..75cfdd3135c 100644 --- a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp +++ b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp @@ -29,7 +29,7 @@ try context->makeGlobalContext(); context->setApplicationType(Context::ApplicationType::LOCAL); - ThreadStatus thread_status; + MainThreadStatus::getInstance(); total_memory_tracker.setHardLimit(1_GiB); registerAggregateFunctions(); @@ -39,6 +39,8 @@ try static bool initialized = initialize(); (void) initialized; + total_memory_tracker.resetCounters(); + /// The input format is as follows: /// - the aggregate function name on the first line, possible with parameters, then data types of the arguments, /// example: quantile(0.5), Float64 diff --git a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp index cba70a2bbef..ee6540c9d30 100644 --- a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp +++ b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp @@ -28,7 +28,7 @@ try context->makeGlobalContext(); context->setApplicationType(Context::ApplicationType::LOCAL); - ThreadStatus thread_status; + MainThreadStatus::getInstance(); total_memory_tracker.setHardLimit(1_GiB); registerAggregateFunctions(); @@ -38,6 +38,8 @@ try static bool initialized = initialize(); (void) initialized; + total_memory_tracker.resetCounters(); + /// The input format is as follows: /// - data type name on the first line, /// - the data for the rest of the input. diff --git a/src/Formats/fuzzers/format_fuzzer.cpp b/src/Formats/fuzzers/format_fuzzer.cpp index fde20ae466f..f667410225c 100644 --- a/src/Formats/fuzzers/format_fuzzer.cpp +++ b/src/Formats/fuzzers/format_fuzzer.cpp @@ -36,7 +36,7 @@ try context->makeGlobalContext(); context->setApplicationType(Context::ApplicationType::LOCAL); - ThreadStatus thread_status; + MainThreadStatus::getInstance(); total_memory_tracker.setHardLimit(1_GiB); registerAggregateFunctions(); @@ -48,6 +48,8 @@ try static bool initialized = initialize(); (void) initialized; + total_memory_tracker.resetCounters(); + /// The input format is as follows: /// - format name on the first line, /// - table structure on the second line, From 637f6fdd51c455ae6c2ae68983997cac38146232 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 06:17:55 +0100 Subject: [PATCH 274/559] Limit memory in fuzzers --- .../aggregate_function_state_deserialization_fuzzer.cpp | 4 +++- src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp | 4 +++- src/Formats/fuzzers/format_fuzzer.cpp | 4 +++- 3 files changed, 9 insertions(+), 3 deletions(-) diff --git a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp index 75cfdd3135c..39f57e00c48 100644 --- a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp +++ b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp @@ -30,7 +30,6 @@ try context->setApplicationType(Context::ApplicationType::LOCAL); MainThreadStatus::getInstance(); - total_memory_tracker.setHardLimit(1_GiB); registerAggregateFunctions(); return true; @@ -40,6 +39,9 @@ try (void) initialized; total_memory_tracker.resetCounters(); + total_memory_tracker.setHardLimit(1_GiB); + CurrentThread::get().memory_tracker.resetCounters(); + CurrentThread::get().memory_tracker.setHardLimit(1_GiB); /// The input format is as follows: /// - the aggregate function name on the first line, possible with parameters, then data types of the arguments, diff --git a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp index ee6540c9d30..31e4c470ee7 100644 --- a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp +++ b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp @@ -29,7 +29,6 @@ try context->setApplicationType(Context::ApplicationType::LOCAL); MainThreadStatus::getInstance(); - total_memory_tracker.setHardLimit(1_GiB); registerAggregateFunctions(); return true; @@ -39,6 +38,9 @@ try (void) initialized; total_memory_tracker.resetCounters(); + total_memory_tracker.setHardLimit(1_GiB); + CurrentThread::get().memory_tracker.resetCounters(); + CurrentThread::get().memory_tracker.setHardLimit(1_GiB); /// The input format is as follows: /// - data type name on the first line, diff --git a/src/Formats/fuzzers/format_fuzzer.cpp b/src/Formats/fuzzers/format_fuzzer.cpp index f667410225c..e84d0913d0d 100644 --- a/src/Formats/fuzzers/format_fuzzer.cpp +++ b/src/Formats/fuzzers/format_fuzzer.cpp @@ -37,7 +37,6 @@ try context->setApplicationType(Context::ApplicationType::LOCAL); MainThreadStatus::getInstance(); - total_memory_tracker.setHardLimit(1_GiB); registerAggregateFunctions(); registerFormats(); @@ -49,6 +48,9 @@ try (void) initialized; total_memory_tracker.resetCounters(); + total_memory_tracker.setHardLimit(1_GiB); + CurrentThread::get().memory_tracker.resetCounters(); + CurrentThread::get().memory_tracker.setHardLimit(1_GiB); /// The input format is as follows: /// - format name on the first line, From 6e1b45d1d694f226b4b3b115722d348cc2027e46 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 06:48:18 +0100 Subject: [PATCH 275/559] Fix exotic trash --- src/AggregateFunctions/QuantileTiming.h | 16 +++++++++++++--- src/Common/ThreadStatus.cpp | 2 -- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/src/AggregateFunctions/QuantileTiming.h b/src/AggregateFunctions/QuantileTiming.h index 2c2e881c78f..c7ab7f00ebf 100644 --- a/src/AggregateFunctions/QuantileTiming.h +++ b/src/AggregateFunctions/QuantileTiming.h @@ -16,6 +16,7 @@ struct Settings; namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int INCORRECT_DATA; } /** Calculates quantile for time in milliseconds, less than 30 seconds. @@ -34,7 +35,7 @@ namespace ErrorCodes * -- for values from 0 to 1023 - in increments of 1; * -- for values from 1024 to 30,000 - in increments of 16; * - * NOTE: 64-bit integer weight can overflow, see also QantileExactWeighted.h::get() + * NOTE: 64-bit integer weight can overflow, see also QuantileExactWeighted.h::get() */ #define TINY_MAX_ELEMS 31 @@ -83,8 +84,12 @@ namespace detail void deserialize(ReadBuffer & buf) { - readBinary(count, buf); - buf.readStrict(reinterpret_cast(elems), count * sizeof(elems[0])); + UInt16 new_count = 0; + readBinary(new_count, buf); + if (new_count > TINY_MAX_ELEMS) + throw Exception(ErrorCodes::INCORRECT_DATA, "The number of elements {} for the 'tiny' kind of quantileTiming is exceeding the maximum of {}", new_count, TINY_MAX_ELEMS); + buf.readStrict(reinterpret_cast(elems), new_count * sizeof(elems[0])); + count = new_count; } /** This function must be called before get-functions. */ @@ -167,6 +172,9 @@ namespace detail { size_t size = 0; readBinary(size, buf); + if (size > sizeof(10'000)) + throw Exception(ErrorCodes::INCORRECT_DATA, "The number of elements {} for the 'medium' kind of quantileTiming is too large", size); + elems.resize(size); buf.readStrict(reinterpret_cast(elems.data()), size * sizeof(elems[0])); } @@ -714,6 +722,8 @@ public: tinyToLarge(); large->deserialize(buf); } + else + throw Exception(ErrorCodes::INCORRECT_DATA, "Incorrect kind of QuantileTiming"); } /// Get the value of the `level` quantile. The level must be between 0 and 1. diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 11f35bc7a6b..aa1690890d8 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -6,8 +6,6 @@ #include #include -#include -#include #include #include From c3fccefcca438c5a53162eb789c22b11384c7dfd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 06:48:41 +0100 Subject: [PATCH 276/559] Add a test --- tests/queries/0_stateless/02688_aggregate_states.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02688_aggregate_states.sql b/tests/queries/0_stateless/02688_aggregate_states.sql index adc1184d717..821c8ebca62 100644 --- a/tests/queries/0_stateless/02688_aggregate_states.sql +++ b/tests/queries/0_stateless/02688_aggregate_states.sql @@ -2,3 +2,5 @@ SELECT '\x01\x00'::AggregateFunction(groupBitmap, UInt32); -- { serverError INCO SELECT '\x01\x01\x01'::AggregateFunction(groupBitmap, UInt64); -- { serverError STD_EXCEPTION } SELECT '\x02\x00\x0d'::AggregateFunction(topK, UInt256); -- { serverError CANNOT_READ_ALL_DATA } SELECT unhex('bebebebebebebebebebebebebebebebebebebebebebebebebebebebebebebe0c0c3131313131313131313131313173290aee00b300')::AggregateFunction(minDistinct, Int8); -- { serverError TOO_LARGE_ARRAY_SIZE } +SELECT unhex('01000b0b0b0d0d0d0d7175616e74696c6554696d696e672c20496e743332000300')::AggregateFunction(quantileTiming, Int32); -- { serverError INCORRECT_DATA } +SELECT unhex('010001')::AggregateFunction(quantileTiming, Int32); -- { serverError INCORRECT_DATA } From 714015cdeae54de2a0c6c003b3c8bab9472d09f4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 06:56:25 +0100 Subject: [PATCH 277/559] Update a test --- .../02560_agg_state_deserialization_hash_table_crash.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02560_agg_state_deserialization_hash_table_crash.sql b/tests/queries/0_stateless/02560_agg_state_deserialization_hash_table_crash.sql index 5b6662faeb3..d85cacc70be 100644 --- a/tests/queries/0_stateless/02560_agg_state_deserialization_hash_table_crash.sql +++ b/tests/queries/0_stateless/02560_agg_state_deserialization_hash_table_crash.sql @@ -1,4 +1,4 @@ DROP TABLE IF EXISTS tab; create table tab (d Int64, s AggregateFunction(groupUniqArrayArray, Array(UInt64)), c SimpleAggregateFunction(groupUniqArrayArray, Array(UInt64))) engine = SummingMergeTree() order by d; -INSERT INTO tab VALUES (1, 'このコー'); -- { clientError CANNOT_ALLOCATE_MEMORY } +INSERT INTO tab VALUES (1, 'このコー'); -- { clientError 128 } DROP TABLE tab; From 100089a76cf0f15f375afbe41957b313318e6fd4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 07:17:59 +0100 Subject: [PATCH 278/559] Make the code in Join less disgusting --- src/Interpreters/HashJoin.cpp | 78 +++++++++++++++++------------------ 1 file changed, 39 insertions(+), 39 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index b4376426700..f9e61b823a7 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1203,7 +1203,7 @@ NO_INLINE IColumn::Filter joinRightColumns( AddedColumns & added_columns, JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]]) { - constexpr JoinFeatures jf; + constexpr JoinFeatures join_features; size_t rows = added_columns.rows_to_add; IColumn::Filter filter; @@ -1212,7 +1212,7 @@ NO_INLINE IColumn::Filter joinRightColumns( Arena pool; - if constexpr (jf.need_replication) + if constexpr (join_features.need_replication) added_columns.offsets_to_replicate = std::make_unique(rows); IColumn::Offset current_offset = 0; @@ -1243,7 +1243,7 @@ NO_INLINE IColumn::Filter joinRightColumns( { right_row_found = true; auto & mapped = find_result.getMapped(); - if constexpr (jf.is_asof_join) + if constexpr (join_features.is_asof_join) { const IColumn & left_asof_key = added_columns.leftAsofKey(); @@ -1252,62 +1252,62 @@ NO_INLINE IColumn::Filter joinRightColumns( { setUsed(filter, i); if constexpr (multiple_disjuncts) - used_flags.template setUsed(row_ref.block, row_ref.row_num, 0); + used_flags.template setUsed(row_ref.block, row_ref.row_num, 0); else - used_flags.template setUsed(find_result); + used_flags.template setUsed(find_result); - added_columns.appendFromBlock(*row_ref.block, row_ref.row_num); + added_columns.appendFromBlock(*row_ref.block, row_ref.row_num); } else - addNotFoundRow(added_columns, current_offset); + addNotFoundRow(added_columns, current_offset); } - else if constexpr (jf.is_all_join) + else if constexpr (join_features.is_all_join) { setUsed(filter, i); - used_flags.template setUsed(find_result); - auto used_flags_opt = jf.need_flags ? &used_flags : nullptr; - addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); + used_flags.template setUsed(find_result); + auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr; + addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); } - else if constexpr ((jf.is_any_join || jf.is_semi_join) && jf.right) + else if constexpr ((join_features.is_any_join || join_features.is_semi_join) && join_features.right) { /// Use first appeared left key + it needs left columns replication - bool used_once = used_flags.template setUsedOnce(find_result); + bool used_once = used_flags.template setUsedOnce(find_result); if (used_once) { - auto used_flags_opt = jf.need_flags ? &used_flags : nullptr; + auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr; setUsed(filter, i); - addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); + addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); } } - else if constexpr (jf.is_any_join && KIND == JoinKind::Inner) + else if constexpr (join_features.is_any_join && KIND == JoinKind::Inner) { - bool used_once = used_flags.template setUsedOnce(find_result); + bool used_once = used_flags.template setUsedOnce(find_result); /// Use first appeared left key only if (used_once) { setUsed(filter, i); - added_columns.appendFromBlock(*mapped.block, mapped.row_num); + added_columns.appendFromBlock(*mapped.block, mapped.row_num); } break; } - else if constexpr (jf.is_any_join && jf.full) + else if constexpr (join_features.is_any_join && join_features.full) { /// TODO } - else if constexpr (jf.is_anti_join) + else if constexpr (join_features.is_anti_join) { - if constexpr (jf.right && jf.need_flags) - used_flags.template setUsed(find_result); + if constexpr (join_features.right && join_features.need_flags) + used_flags.template setUsed(find_result); } else /// ANY LEFT, SEMI LEFT, old ANY (RightAny) { setUsed(filter, i); - used_flags.template setUsed(find_result); - added_columns.appendFromBlock(*mapped.block, mapped.row_num); + used_flags.template setUsed(find_result); + added_columns.appendFromBlock(*mapped.block, mapped.row_num); - if (jf.is_any_or_semi_join) + if (join_features.is_any_or_semi_join) { break; } @@ -1319,9 +1319,9 @@ NO_INLINE IColumn::Filter joinRightColumns( { if (!right_row_found && null_element_found) { - addNotFoundRow(added_columns, current_offset); + addNotFoundRow(added_columns, current_offset); - if constexpr (jf.need_replication) + if constexpr (join_features.need_replication) { (*added_columns.offsets_to_replicate)[i] = current_offset; } @@ -1331,12 +1331,12 @@ NO_INLINE IColumn::Filter joinRightColumns( if (!right_row_found) { - if constexpr (jf.is_anti_join && jf.left) + if constexpr (join_features.is_anti_join && join_features.left) setUsed(filter, i); - addNotFoundRow(added_columns, current_offset); + addNotFoundRow(added_columns, current_offset); } - if constexpr (jf.need_replication) + if constexpr (join_features.need_replication) { (*added_columns.offsets_to_replicate)[i] = current_offset; } @@ -1442,7 +1442,7 @@ void HashJoin::joinBlockImpl( const std::vector & maps_, bool is_join_get) const { - constexpr JoinFeatures jf; + constexpr JoinFeatures join_features; std::vector join_on_keys; const auto & onexprs = table_join->getClauses(); @@ -1457,7 +1457,7 @@ void HashJoin::joinBlockImpl( * Because if they are constants, then in the "not joined" rows, they may have different values * - default values, which can differ from the values of these constants. */ - if constexpr (jf.right || jf.full) + if constexpr (join_features.right || join_features.full) { materializeBlockInplace(block); } @@ -1473,11 +1473,11 @@ void HashJoin::joinBlockImpl( savedBlockSample(), *this, std::move(join_on_keys), - jf.is_asof_join, + join_features.is_asof_join, is_join_get); bool has_required_right_keys = (required_right_keys.columns() != 0); - added_columns.need_filter = jf.need_filter || has_required_right_keys; + added_columns.need_filter = join_features.need_filter || has_required_right_keys; IColumn::Filter row_filter = switchJoinRightColumns(maps_, added_columns, data->type, used_flags); @@ -1486,7 +1486,7 @@ void HashJoin::joinBlockImpl( std::vector right_keys_to_replicate [[maybe_unused]]; - if constexpr (jf.need_filter) + if constexpr (join_features.need_filter) { /// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones. for (size_t i = 0; i < existing_columns; ++i) @@ -1502,7 +1502,7 @@ void HashJoin::joinBlockImpl( const auto & left_name = required_right_keys_sources[i]; /// asof column is already in block. - if (jf.is_asof_join && right_key.name == table_join->getOnlyClause().key_names_right.back()) + if (join_features.is_asof_join && right_key.name == table_join->getOnlyClause().key_names_right.back()) continue; const auto & col = block.getByName(left_name); @@ -1534,7 +1534,7 @@ void HashJoin::joinBlockImpl( const auto & left_name = required_right_keys_sources[i]; /// asof column is already in block. - if (jf.is_asof_join && right_key.name == table_join->getOnlyClause().key_names_right.back()) + if (join_features.is_asof_join && right_key.name == table_join->getOnlyClause().key_names_right.back()) continue; const auto & col = block.getByName(left_name); @@ -1548,13 +1548,13 @@ void HashJoin::joinBlockImpl( correctNullabilityInplace(right_col, is_nullable, null_map_filter); block.insert(std::move(right_col)); - if constexpr (jf.need_replication) + if constexpr (join_features.need_replication) right_keys_to_replicate.push_back(block.getPositionByName(right_col_name)); } } } - if constexpr (jf.need_replication) + if constexpr (join_features.need_replication) { std::unique_ptr & offsets_to_replicate = added_columns.offsets_to_replicate; From 8fdbc5409cba3ccddc429d4c3500c1cb06e8150d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 07:31:18 +0100 Subject: [PATCH 279/559] Make the code in Join less disgusting --- src/Interpreters/HashJoin.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index f9e61b823a7..ad0d7625fa8 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1022,7 +1022,6 @@ private: void addColumn(const ColumnWithTypeAndName & src_column, const std::string & qualified_name) { - columns.push_back(src_column.column->cloneEmpty()); columns.back()->reserve(src_column.column->size()); type_name.emplace_back(src_column.type, src_column.name, qualified_name); From 0ec04cca5e7babefc331a703b45a1e252743b74a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 07:32:29 +0100 Subject: [PATCH 280/559] Make the code in Join less disgusting --- src/Interpreters/HashJoin.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index ad0d7625fa8..fe0244ff314 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1048,7 +1048,6 @@ struct JoinFeatures static constexpr bool add_missing = (left || full) && !is_semi_join; static constexpr bool need_flags = MapGetter::flagged; - }; template From 36556671883409efff1a3032d69a5ed408fa80f1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 07:35:56 +0100 Subject: [PATCH 281/559] Make the code in Join less disgusting --- src/Interpreters/RowRefs.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/RowRefs.h b/src/Interpreters/RowRefs.h index 294da1da571..650b2311ba7 100644 --- a/src/Interpreters/RowRefs.h +++ b/src/Interpreters/RowRefs.h @@ -63,7 +63,8 @@ struct RowRefList : RowRef return batch; } - row_refs[size++] = std::move(row_ref); + row_refs[size] = std::move(row_ref); + ++size; return this; } }; From ecff329ca1c91eb64f9ee26b46ec7a587b129a8f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 10:18:28 +0300 Subject: [PATCH 282/559] Update AggregateFunctionGroupBitmapData.h --- src/AggregateFunctions/AggregateFunctionGroupBitmapData.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index 1a75bd99659..62017251108 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -23,6 +23,7 @@ namespace DB namespace ErrorCodes { extern const int TOO_LARGE_ARRAY_SIZE; + extern const int INCORRECT_DATA; } enum BitmapKind From 2755e689810700783bf63a8257ddb01b2df9ebd3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Mar 2023 08:36:05 +0100 Subject: [PATCH 283/559] Maybe faster build --- .../AggregateFunctionStatisticsSimple.cpp | 80 ++--- .../AggregateFunctionStatisticsSimple.h | 321 +++++++++++------- src/AggregateFunctions/Moments.h | 107 ++++-- 3 files changed, 302 insertions(+), 206 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionStatisticsSimple.cpp b/src/AggregateFunctions/AggregateFunctionStatisticsSimple.cpp index d06c1619b9f..e9111ba4e1e 100644 --- a/src/AggregateFunctions/AggregateFunctionStatisticsSimple.cpp +++ b/src/AggregateFunctions/AggregateFunctionStatisticsSimple.cpp @@ -1,72 +1,36 @@ #include -#include -#include #include namespace DB { -struct Settings; -namespace ErrorCodes -{ - extern const int ILLEGAL_TYPE_OF_ARGUMENT; -} - -namespace -{ - -template