From 9c6c6d98444d160608690634cf794195fc37eaae Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 17 Jan 2023 20:12:34 +0000 Subject: [PATCH 001/418] 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 002/418] 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 003/418] 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 004/418] 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 005/418] 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 006/418] 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 007/418] 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 008/418] 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 009/418] 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 010/418] 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 011/418] 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 012/418] 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 013/418] 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 014/418] 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 015/418] 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 016/418] 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 017/418] 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 018/418] 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 019/418] 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 020/418] 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 021/418] 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 022/418] 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 023/418] 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 024/418] 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 025/418] 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 026/418] 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 027/418] 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 028/418] 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 029/418] 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 030/418] 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 031/418] 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 032/418] 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 033/418] 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 034/418] 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 035/418] 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 036/418] 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 037/418] 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 038/418] 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 039/418] 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 040/418] 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 041/418] 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 042/418] 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 043/418] 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 044/418] 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 045/418] 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 046/418] 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 047/418] 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 048/418] 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 049/418] 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 050/418] 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 051/418] 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 052/418] 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 053/418] 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 054/418] 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 055/418] 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 056/418] 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 057/418] 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 058/418] 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 059/418] 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 060/418] 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 061/418] 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 062/418] 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 063/418] 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 064/418] 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 065/418] 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 066/418] 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 067/418] 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 068/418] 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 069/418] 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 070/418] 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 071/418] 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 072/418] 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 073/418] 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 074/418] 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 075/418] 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 076/418] 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 077/418] 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 078/418] 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 079/418] 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 080/418] 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 081/418] 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 082/418] 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 083/418] 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 084/418] 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 085/418] 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 086/418] 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 087/418] 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 088/418] 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 089/418] 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 090/418] 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 091/418] 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 092/418] 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 093/418] 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 094/418] 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 095/418] 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 096/418] 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 097/418] 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 098/418] 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 099/418] 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 100/418] 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 101/418] 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 102/418] 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 103/418] 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 104/418] 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 105/418] 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 106/418] 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 107/418] 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 108/418] 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 109/418] 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 110/418] 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 111/418] 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 112/418] 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 113/418] 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 114/418] 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 115/418] 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 116/418] 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 117/418] 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 39a11854862d914033ba3f9271cd0c57da75ae11 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 10 Mar 2023 15:30:29 +0100 Subject: [PATCH 118/418] 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 6ba79c2737134eb0d50ae294a909b3b7dc900ab8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sun, 12 Mar 2023 16:05:24 +0000 Subject: [PATCH 119/418] 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 de8d0040a4452e72c7d8d00444ce787000e0ce21 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 13 Mar 2023 18:34:47 +0100 Subject: [PATCH 120/418] 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 e8f42e5fb3f8da8d792cd8d3274293cf7c59b09f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Mar 2023 11:26:35 +0000 Subject: [PATCH 121/418] 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 122/418] 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 123/418] 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 124/418] 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 125/418] 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 126/418] 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 127/418] 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 1393e9ba035c932487d82b3d0b52085b017b1954 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 15 Mar 2023 01:09:54 +0000 Subject: [PATCH 128/418] 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 129/418] 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 130/418] 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 1d06d343dc102d9c064503b9e85a35111104c436 Mon Sep 17 00:00:00 2001 From: alexX512 Date: Wed, 15 Mar 2023 11:15:50 +0000 Subject: [PATCH 131/418] 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 aa5127c2fdd7e5a0dbae8003f8e57c166a8bc4db Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 15 Mar 2023 08:33:45 +0100 Subject: [PATCH 132/418] 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 133/418] 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 134/418] 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 0088c18192225e0cfbfdc2043a30b70daf9af58e Mon Sep 17 00:00:00 2001 From: alexX512 Date: Wed, 15 Mar 2023 13:37:47 +0000 Subject: [PATCH 135/418] 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 d78a9e03ad0381049b8d0322528b6d2261bce048 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 15 Mar 2023 15:38:11 +0100 Subject: [PATCH 136/418] 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 137/418] 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 138/418] 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 1f21ac8fcf448cfa988d565c7e87902d9d0fc61c Mon Sep 17 00:00:00 2001 From: alexX512 Date: Wed, 15 Mar 2023 19:53:58 +0000 Subject: [PATCH 139/418] 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 e0954ce7beb9b38c16ebe222fc6abc354d1cd919 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 16 Mar 2023 00:22:05 +0100 Subject: [PATCH 140/418] 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 141/418] 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 142/418] 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 143/418] 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 144/418] 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 00aaa29a85acec8882bf3433cb0f4a61ad31503d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 16 Mar 2023 18:00:39 +0000 Subject: [PATCH 145/418] 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 4c9e2aecbd3ca3d797aa602c7b0e1959cd84293d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 16 Mar 2023 19:49:30 +0000 Subject: [PATCH 146/418] 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 e536c80274f343685690b455f3e27dea3602fe89 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 16 Mar 2023 20:22:25 +0000 Subject: [PATCH 147/418] 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 d7b2ea60ebc88ac1a370164661479de0b08e655b Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 17 Mar 2023 13:56:02 +0100 Subject: [PATCH 148/418] 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 149/418] 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 2958c5f0f130c6308b00af7f3c41c91357dedb72 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 17 Mar 2023 15:08:16 +0000 Subject: [PATCH 150/418] 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 4930683aa87467f21a9c25fd067857b818d549fb Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 17 Mar 2023 17:01:40 +0000 Subject: [PATCH 151/418] 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 152/418] 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 6509d9e525c89987f14ae290962c865072499406 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 17 Mar 2023 18:36:49 +0000 Subject: [PATCH 153/418] 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 154/418] 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 023c0ba46eadaed590f961bc5d168f368cc5b285 Mon Sep 17 00:00:00 2001 From: alexX512 Date: Fri, 17 Mar 2023 20:05:10 +0000 Subject: [PATCH 155/418] 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 c14c083cdff487abfff299fe2b6c08e6a5b28d07 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 17 Mar 2023 19:30:12 -0400 Subject: [PATCH 156/418] 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 f32eec8ef66180dd93c5ddb65480f4f22b2671b1 Mon Sep 17 00:00:00 2001 From: pufit Date: Sat, 18 Mar 2023 20:32:30 -0400 Subject: [PATCH 157/418] 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 87cb2e16299564c89c8caadba6c4b3acf183620d Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 19 Mar 2023 19:41:19 +0100 Subject: [PATCH 158/418] =?UTF-8?q?Support=20for=20subquery=20in=20paramet?= =?UTF-8?q?erized=20views=E2=80=A8Implementation:=20*=20Updated=20to=20pas?= =?UTF-8?q?s=20the=20parameter=20is=5Fcreate=5Fparameterized=5Fview=20to?= =?UTF-8?q?=20subquery=20processing.=20Testing:=20*=20Added=20test=20case?= =?UTF-8?q?=20with=20subquery=20for=20parameterized=20view.?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- .../InterpreterSelectWithUnionQuery.cpp | 29 +++++++------------ src/Interpreters/JoinedTables.cpp | 5 ++-- src/Interpreters/JoinedTables.h | 3 +- src/Interpreters/getTableExpressions.cpp | 10 ++++--- src/Interpreters/getTableExpressions.h | 2 +- .../02428_parameterized_view.reference | 1 + .../0_stateless/02428_parameterized_view.sh | 5 ++++ 8 files changed, 29 insertions(+), 28 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 79b073e30f3..bbd8a5744fc 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -415,7 +415,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( RewriteCountDistinctFunctionVisitor(data_rewrite_countdistinct).visit(query_ptr); } - JoinedTables joined_tables(getSubqueryContext(context), getSelectQuery(), options.with_all_cols); + JoinedTables joined_tables(getSubqueryContext(context), getSelectQuery(), options.with_all_cols, options_.is_create_parameterized_view); bool got_storage_from_query = false; if (!has_input && !storage) diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index bfa3d16bf29..8396cb32a69 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -258,12 +258,12 @@ Block InterpreterSelectWithUnionQuery::getSampleBlock(const ASTPtr & query_ptr_, { if (!context_->hasQueryContext()) { + SelectQueryOptions options; if (is_subquery) - return InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().subquery().analyze()).getSampleBlock(); - else if (is_create_parameterized_view) - return InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().createParameterizedView().analyze()).getSampleBlock(); - else - return InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().analyze()).getSampleBlock(); + options = options.subquery(); + if (is_create_parameterized_view) + options = options.createParameterizedView(); + return InterpreterSelectWithUnionQuery(query_ptr_, context_, std::move(options.analyze())).getSampleBlock(); } auto & cache = context_->getSampleBlockCache(); @@ -274,21 +274,12 @@ Block InterpreterSelectWithUnionQuery::getSampleBlock(const ASTPtr & query_ptr_, return cache[key]; } + SelectQueryOptions options; if (is_subquery) - { - return cache[key] - = InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().subquery().analyze()).getSampleBlock(); - } - else if (is_create_parameterized_view) - { - return cache[key] - = InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().createParameterizedView().analyze()) - .getSampleBlock(); - } - else - { - return cache[key] = InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().analyze()).getSampleBlock(); - } + options = options.subquery(); + if (is_create_parameterized_view) + options = options.createParameterizedView(); + return cache[key] = InterpreterSelectWithUnionQuery(query_ptr_, context_, std::move(options.analyze())).getSampleBlock(); } diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index 7c999803b44..80b2fe5302c 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -173,13 +173,14 @@ using RenameQualifiedIdentifiersVisitor = InDepthNodeVisitorgetSettingsRef(); bool include_alias_cols = include_all_columns || settings.asterisk_include_alias_columns; bool include_materialized_cols = include_all_columns || settings.asterisk_include_materialized_columns; - tables_with_columns = getDatabaseAndTablesWithColumns(table_expressions, context, include_alias_cols, include_materialized_cols); + tables_with_columns = getDatabaseAndTablesWithColumns(table_expressions, context, include_alias_cols, include_materialized_cols, is_create_parameterized_view); if (tables_with_columns.size() != table_expressions.size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected tables count"); diff --git a/src/Interpreters/JoinedTables.h b/src/Interpreters/JoinedTables.h index 7562dbc9ac5..105cfc29a7d 100644 --- a/src/Interpreters/JoinedTables.h +++ b/src/Interpreters/JoinedTables.h @@ -22,7 +22,7 @@ using StorageMetadataPtr = std::shared_ptr; class JoinedTables { public: - JoinedTables(ContextPtr context, const ASTSelectQuery & select_query_, bool include_all_columns_ = false); + JoinedTables(ContextPtr context, const ASTSelectQuery & select_query_, bool include_all_columns_ = false, bool is_create_parameterized_view_ = false); void reset(const ASTSelectQuery & select_query); @@ -53,6 +53,7 @@ private: ASTPtr left_table_expression; std::optional left_db_and_table; const ASTSelectQuery & select_query; + bool is_create_parameterized_view; }; } diff --git a/src/Interpreters/getTableExpressions.cpp b/src/Interpreters/getTableExpressions.cpp index 830f0ea4411..21f89ed2137 100644 --- a/src/Interpreters/getTableExpressions.cpp +++ b/src/Interpreters/getTableExpressions.cpp @@ -78,13 +78,14 @@ static NamesAndTypesList getColumnsFromTableExpression( ContextPtr context, NamesAndTypesList & materialized, NamesAndTypesList & aliases, - NamesAndTypesList & virtuals) + NamesAndTypesList & virtuals, + bool is_create_parameterized_view) { NamesAndTypesList names_and_type_list; if (table_expression.subquery) { const auto & subquery = table_expression.subquery->children.at(0); - names_and_type_list = InterpreterSelectWithUnionQuery::getSampleBlock(subquery, context, true).getNamesAndTypesList(); + names_and_type_list = InterpreterSelectWithUnionQuery::getSampleBlock(subquery, context, true, is_create_parameterized_view).getNamesAndTypesList(); } else if (table_expression.table_function) { @@ -117,7 +118,8 @@ TablesWithColumns getDatabaseAndTablesWithColumns( const ASTTableExprConstPtrs & table_expressions, ContextPtr context, bool include_alias_cols, - bool include_materialized_cols) + bool include_materialized_cols, + bool is_create_parameterized_view) { TablesWithColumns tables_with_columns; @@ -129,7 +131,7 @@ TablesWithColumns getDatabaseAndTablesWithColumns( NamesAndTypesList aliases; NamesAndTypesList virtuals; NamesAndTypesList names_and_types = getColumnsFromTableExpression( - *table_expression, context, materialized, aliases, virtuals); + *table_expression, context, materialized, aliases, virtuals, is_create_parameterized_view); removeDuplicateColumns(names_and_types); diff --git a/src/Interpreters/getTableExpressions.h b/src/Interpreters/getTableExpressions.h index c4ca01ee3c3..8ddfe4b32c3 100644 --- a/src/Interpreters/getTableExpressions.h +++ b/src/Interpreters/getTableExpressions.h @@ -21,6 +21,6 @@ const ASTTableExpression * getTableExpression(const ASTSelectQuery & select, siz ASTPtr extractTableExpression(const ASTSelectQuery & select, size_t table_number); TablesWithColumns getDatabaseAndTablesWithColumns( - const ASTTableExprConstPtrs & table_expressions, ContextPtr context, bool include_alias_cols, bool include_materialized_cols); + const ASTTableExprConstPtrs & table_expressions, ContextPtr context, bool include_alias_cols, bool include_materialized_cols, bool is_create_parameterized_view = false); } diff --git a/tests/queries/0_stateless/02428_parameterized_view.reference b/tests/queries/0_stateless/02428_parameterized_view.reference index 27d7c6d1956..e436075e1a8 100644 --- a/tests/queries/0_stateless/02428_parameterized_view.reference +++ b/tests/queries/0_stateless/02428_parameterized_view.reference @@ -36,3 +36,4 @@ ERROR 20 10 10 +10 diff --git a/tests/queries/0_stateless/02428_parameterized_view.sh b/tests/queries/0_stateless/02428_parameterized_view.sh index aad3aad3b22..e499b7ca389 100755 --- a/tests/queries/0_stateless/02428_parameterized_view.sh +++ b/tests/queries/0_stateless/02428_parameterized_view.sh @@ -16,6 +16,7 @@ $CLICKHOUSE_CLIENT -q "DROP VIEW IF EXISTS test_02428_pv7" $CLICKHOUSE_CLIENT -q "DROP VIEW IF EXISTS test_02428_pv8" $CLICKHOUSE_CLIENT -q "DROP VIEW IF EXISTS test_02428_pv9" $CLICKHOUSE_CLIENT -q "DROP VIEW IF EXISTS test_02428_pv10" +$CLICKHOUSE_CLIENT -q "DROP VIEW IF EXISTS test_02428_pv11" $CLICKHOUSE_CLIENT -q "DROP VIEW IF EXISTS test_02428_v1" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_02428_Catalog" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS ${CLICKHOUSE_TEST_UNIQUE_NAME}.pv1" @@ -87,6 +88,9 @@ $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02428_pv9(quantity=3)" $CLICKHOUSE_CLIENT -q "CREATE VIEW test_02428_pv10 AS SELECT Price FROM test_02428_Catalog WHERE Price={Pri:UInt64} ORDER BY Price" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02428_pv10(Pri=10)" +$CLICKHOUSE_CLIENT -q "CREATE VIEW test_02428_pv11 AS SELECT * from ( SELECT Price FROM test_02428_Catalog WHERE Price={price:UInt64} )" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02428_pv11(price=10)" + $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv1" $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv2" $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv3" @@ -96,6 +100,7 @@ $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv7" $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv8" $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv9" $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv10" +$CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv11" $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_v1" $CLICKHOUSE_CLIENT -q "DROP TABLE test_02428_Catalog" $CLICKHOUSE_CLIENT -q "DROP TABLE ${CLICKHOUSE_TEST_UNIQUE_NAME}.pv1" From dec59721944a2aeb152c7b1c5ca7de7ee1f3a107 Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Sun, 19 Mar 2023 20:35:24 +0100 Subject: [PATCH 159/418] add optimization for MemoryStorageStep --- .../optimizePrimaryKeyCondition.cpp | 1 - .../QueryPlan/ReadFromMemoryStorageStep.cpp | 155 +++++++++++++++++ .../QueryPlan/ReadFromMemoryStorageStep.h | 40 +++++ src/Storages/StorageMemory.cpp | 160 +----------------- src/Storages/StorageMemory.h | 1 + 5 files changed, 197 insertions(+), 160 deletions(-) create mode 100644 src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp create mode 100644 src/Processors/QueryPlan/ReadFromMemoryStorageStep.h diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp index 753940339ea..e98386a6ee9 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include namespace DB::QueryPlanOptimizations diff --git a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp new file mode 100644 index 00000000000..fa2414ec885 --- /dev/null +++ b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp @@ -0,0 +1,155 @@ +#include "ReadFromMemoryStorageStep.h" + +#include +#include +#include + +#include +#include +#include +#include + +#include +#include +#include + +namespace DB +{ + +class MemorySource : public ISource +{ + using InitializerFunc = std::function &)>; +public: + + MemorySource( + Names column_names_, + const StorageSnapshotPtr & storage_snapshot, + std::shared_ptr data_, + std::shared_ptr> parallel_execution_index_, + InitializerFunc initializer_func_ = {}) + : ISource(storage_snapshot->getSampleBlockForColumns(column_names_)) + , column_names_and_types(storage_snapshot->getColumnsByNames( + GetColumnsOptions(GetColumnsOptions::All).withSubcolumns().withExtendedObjects(), column_names_)) + , data(data_) + , parallel_execution_index(parallel_execution_index_) + , initializer_func(std::move(initializer_func_)) + { + } + + String getName() const override { return "Memory"; } + +protected: + Chunk generate() override + { + if (initializer_func) + { + initializer_func(data); + initializer_func = {}; + } + + size_t current_index = getAndIncrementExecutionIndex(); + + if (!data || current_index >= data->size()) + { + return {}; + } + + const Block & src = (*data)[current_index]; + + Columns columns; + size_t num_columns = column_names_and_types.size(); + columns.reserve(num_columns); + + auto name_and_type = column_names_and_types.begin(); + for (size_t i = 0; i < num_columns; ++i) + { + columns.emplace_back(tryGetColumnFromBlock(src, *name_and_type)); + ++name_and_type; + } + + fillMissingColumns(columns, src.rows(), column_names_and_types, column_names_and_types, {}, nullptr); + assert(std::all_of(columns.begin(), columns.end(), [](const auto & column) { return column != nullptr; })); + + return Chunk(std::move(columns), src.rows()); + } + +private: + size_t getAndIncrementExecutionIndex() + { + if (parallel_execution_index) + { + return (*parallel_execution_index)++; + } + else + { + return execution_index++; + } + } + + const NamesAndTypesList column_names_and_types; + size_t execution_index = 0; + std::shared_ptr data; + std::shared_ptr> parallel_execution_index; + InitializerFunc initializer_func; +}; + +ReadFromMemoryStorageStep::ReadFromMemoryStorageStep(Pipe pipe_) : + SourceStepWithFilter(DataStream{.header = pipe_.getHeader()}), + pipe(std::move(pipe_)) +{ +} + +void ReadFromMemoryStorageStep::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + // use move - make sure that the call will only be made once. + pipeline.init(std::move(pipe)); +} + +Pipe ReadFromMemoryStorageStep::makePipe(const Names & columns_to_read_, + const StorageSnapshotPtr & storage_snapshot_, + size_t num_streams_, + const bool delay_read_for_global_sub_queries_) +{ + storage_snapshot_->check(columns_to_read_); + + const auto & snapshot_data = assert_cast(*storage_snapshot_->data); + auto current_data = snapshot_data.blocks; + + if (delay_read_for_global_sub_queries_) + { + /// Note: for global subquery we use single source. + /// Mainly, the reason is that at this point table is empty, + /// and we don't know the number of blocks are going to be inserted into it. + /// + /// It may seem to be not optimal, but actually data from such table is used to fill + /// set for IN or hash table for JOIN, which can't be done concurrently. + /// Since no other manipulation with data is done, multiple sources shouldn't give any profit. + + return Pipe(std::make_shared( + columns_to_read_, + storage_snapshot_, + nullptr /* data */, + nullptr /* parallel execution index */, + [current_data](std::shared_ptr & data_to_initialize) + { + data_to_initialize = current_data; + })); + } + + size_t size = current_data->size(); + + if (num_streams_ > size) + num_streams_ = size; + + Pipes pipes; + + auto parallel_execution_index = std::make_shared>(0); + + for (size_t stream = 0; stream < num_streams_; ++stream) + { + pipes.emplace_back(std::make_shared(columns_to_read_, storage_snapshot_, current_data, parallel_execution_index)); + } + return Pipe::unitePipes(std::move(pipes)); +} + +} diff --git a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h new file mode 100644 index 00000000000..652e729a97a --- /dev/null +++ b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h @@ -0,0 +1,40 @@ +#pragma once + +#include + +#include +#include +#include + +namespace DB +{ + +class QueryPipelineBuilder; + +class ReadFromMemoryStorageStep final : public SourceStepWithFilter +{ +public: + explicit ReadFromMemoryStorageStep(Pipe pipe_); + + ReadFromMemoryStorageStep() = delete; + ReadFromMemoryStorageStep(const ReadFromMemoryStorageStep &) = delete; + ReadFromMemoryStorageStep & operator=(const ReadFromMemoryStorageStep &) = delete; + + ReadFromMemoryStorageStep(ReadFromMemoryStorageStep &&) = default; + ReadFromMemoryStorageStep & operator=(ReadFromMemoryStorageStep &&) = default; + + String getName() const override { return name; } + + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + + static Pipe makePipe(const Names & columns_to_read_, + const StorageSnapshotPtr & storage_snapshot_, + size_t num_streams_, + bool delay_read_for_global_sub_queries_); + +private: + static constexpr auto name = "ReadFromMemoryStorage"; + Pipe pipe; +}; + +} diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index e7bd7cd8e7e..11688582877 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -9,15 +9,13 @@ #include #include #include -#include #include -#include #include #include #include #include -#include +#include #include #include @@ -44,85 +42,6 @@ namespace ErrorCodes extern const int CANNOT_RESTORE_TABLE; } - -class MemorySource : public ISource -{ - using InitializerFunc = std::function &)>; -public: - - MemorySource( - Names column_names_, - const StorageSnapshotPtr & storage_snapshot, - std::shared_ptr data_, - std::shared_ptr> parallel_execution_index_, - InitializerFunc initializer_func_ = {}) - : ISource(storage_snapshot->getSampleBlockForColumns(column_names_)) - , column_names_and_types(storage_snapshot->getColumnsByNames( - GetColumnsOptions(GetColumnsOptions::All).withSubcolumns().withExtendedObjects(), column_names_)) - , data(data_) - , parallel_execution_index(parallel_execution_index_) - , initializer_func(std::move(initializer_func_)) - { - } - - String getName() const override { return "Memory"; } - -protected: - Chunk generate() override - { - if (initializer_func) - { - initializer_func(data); - initializer_func = {}; - } - - size_t current_index = getAndIncrementExecutionIndex(); - - if (!data || current_index >= data->size()) - { - return {}; - } - - const Block & src = (*data)[current_index]; - - Columns columns; - size_t num_columns = column_names_and_types.size(); - columns.reserve(num_columns); - - auto name_and_type = column_names_and_types.begin(); - for (size_t i = 0; i < num_columns; ++i) - { - columns.emplace_back(tryGetColumnFromBlock(src, *name_and_type)); - ++name_and_type; - } - - fillMissingColumns(columns, src.rows(), column_names_and_types, column_names_and_types, {}, nullptr); - assert(std::all_of(columns.begin(), columns.end(), [](const auto & column) { return column != nullptr; })); - - return Chunk(std::move(columns), src.rows()); - } - -private: - size_t getAndIncrementExecutionIndex() - { - if (parallel_execution_index) - { - return (*parallel_execution_index)++; - } - else - { - return execution_index++; - } - } - - const NamesAndTypesList column_names_and_types; - size_t execution_index = 0; - std::shared_ptr data; - std::shared_ptr> parallel_execution_index; - InitializerFunc initializer_func; -}; - - class MemorySink : public SinkToStorage { public: @@ -193,83 +112,6 @@ private: }; -class ReadFromMemoryStorageStep final : public ISourceStep -{ -public: - explicit ReadFromMemoryStorageStep(Pipe pipe_) : - ISourceStep(DataStream{.header = pipe_.getHeader()}), - pipe(std::move(pipe_)) - { - } - - ReadFromMemoryStorageStep() = delete; - ReadFromMemoryStorageStep(const ReadFromMemoryStorageStep &) = delete; - ReadFromMemoryStorageStep & operator=(const ReadFromMemoryStorageStep &) = delete; - - ReadFromMemoryStorageStep(ReadFromMemoryStorageStep &&) = default; - ReadFromMemoryStorageStep & operator=(ReadFromMemoryStorageStep &&) = default; - - String getName() const override { return name; } - - void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override - { - // use move - make sure that the call will only be made once. - pipeline.init(std::move(pipe)); - } - - static Pipe makePipe(const Names & columns_to_read_, - const StorageSnapshotPtr & storage_snapshot_, - size_t num_streams_, - const bool delay_read_for_global_subqueries_) - { - storage_snapshot_->check(columns_to_read_); - - const auto & snapshot_data = assert_cast(*storage_snapshot_->data); - auto current_data = snapshot_data.blocks; - - if (delay_read_for_global_subqueries_) - { - /// Note: for global subquery we use single source. - /// Mainly, the reason is that at this point table is empty, - /// and we don't know the number of blocks are going to be inserted into it. - /// - /// It may seem to be not optimal, but actually data from such table is used to fill - /// set for IN or hash table for JOIN, which can't be done concurrently. - /// Since no other manipulation with data is done, multiple sources shouldn't give any profit. - - return Pipe(std::make_shared( - columns_to_read_, - storage_snapshot_, - nullptr /* data */, - nullptr /* parallel execution index */, - [current_data](std::shared_ptr & data_to_initialize) - { - data_to_initialize = current_data; - })); - } - - size_t size = current_data->size(); - - if (num_streams_ > size) - num_streams_ = size; - - Pipes pipes; - - auto parallel_execution_index = std::make_shared>(0); - - for (size_t stream = 0; stream < num_streams_; ++stream) - { - pipes.emplace_back(std::make_shared(columns_to_read_, storage_snapshot_, current_data, parallel_execution_index)); - } - return Pipe::unitePipes(std::move(pipes)); - } - -private: - static constexpr auto name = "ReadFromMemoryStorage"; - Pipe pipe; -}; - - StorageMemory::StorageMemory( const StorageID & table_id_, ColumnsDescription columns_description_, diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index fcaadaff3ba..db231379df9 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -5,6 +5,7 @@ #include #include +#include #include #include From f6569f60e6f7b350d09062188df16da85c2c5473 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Sun, 19 Mar 2023 20:50:32 +0100 Subject: [PATCH 160/418] Remove constant aggregation keys --- src/Planner/PlannerExpressionAnalysis.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index 11444503c5f..7fcc626d752 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -100,6 +100,9 @@ std::optional analyzeAggregation(const QueryTreeNodeP for (auto & expression_dag_node : expression_dag_nodes) { + if (expression_dag_node->column && isColumnConst(*expression_dag_node->column)) + continue; + grouping_sets_parameters.used_keys.push_back(expression_dag_node->result_name); if (before_aggregation_actions_output_node_names.contains(expression_dag_node->result_name)) continue; @@ -146,6 +149,9 @@ std::optional analyzeAggregation(const QueryTreeNodeP for (auto & expression_dag_node : expression_dag_nodes) { + if (expression_dag_node->column && isColumnConst(*expression_dag_node->column)) + continue; + if (before_aggregation_actions_output_node_names.contains(expression_dag_node->result_name)) continue; From ac5b7bd457d1c1f3a0a5b2525cfc4eb01e8a2d72 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Sun, 19 Mar 2023 21:05:43 +0100 Subject: [PATCH 161/418] Update test --- .../0_stateless/01925_test_group_by_const_consistency.reference | 1 + .../0_stateless/01925_test_group_by_const_consistency.sql | 2 ++ 2 files changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/01925_test_group_by_const_consistency.reference b/tests/queries/0_stateless/01925_test_group_by_const_consistency.reference index 93f9e3d10db..72e8abb1428 100644 --- a/tests/queries/0_stateless/01925_test_group_by_const_consistency.reference +++ b/tests/queries/0_stateless/01925_test_group_by_const_consistency.reference @@ -1,2 +1,3 @@ 0 1 0 +1 0 diff --git a/tests/queries/0_stateless/01925_test_group_by_const_consistency.sql b/tests/queries/0_stateless/01925_test_group_by_const_consistency.sql index a73c06bbe49..176b5761ac5 100644 --- a/tests/queries/0_stateless/01925_test_group_by_const_consistency.sql +++ b/tests/queries/0_stateless/01925_test_group_by_const_consistency.sql @@ -2,3 +2,5 @@ SELECT 1 as a, count() FROM numbers(10) WHERE 0 GROUP BY a; SELECT count() FROM numbers(10) WHERE 0; SELECT 1 as a, count() FROM numbers(10) WHERE 0 GROUP BY a SETTINGS empty_result_for_aggregation_by_constant_keys_on_empty_set = 0; + +SELECT 1 as a, count() FROM numbers(10) WHERE 0 GROUP BY a SETTINGS empty_result_for_aggregation_by_constant_keys_on_empty_set = 0, optimize_trivial_count_query = 0; From f1dae287b549d58d687da5e3c094886b515fccf7 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 20 Mar 2023 07:29:06 +0100 Subject: [PATCH 162/418] Updated parameterized view test to use ReplicatedMergeTree --- tests/queries/0_stateless/02428_parameterized_view.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02428_parameterized_view.sh b/tests/queries/0_stateless/02428_parameterized_view.sh index e499b7ca389..86f131ea0f1 100755 --- a/tests/queries/0_stateless/02428_parameterized_view.sh +++ b/tests/queries/0_stateless/02428_parameterized_view.sh @@ -23,7 +23,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS ${CLICKHOUSE_TEST_UNIQUE_NAME}.pv1" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS ${CLICKHOUSE_TEST_UNIQUE_NAME}.Catalog" $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS ${CLICKHOUSE_TEST_UNIQUE_NAME}" -$CLICKHOUSE_CLIENT -q "CREATE TABLE test_02428_Catalog (Name String, Price UInt64, Quantity UInt64) ENGINE = Memory" +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_02428_Catalog (Name String, Price UInt64, Quantity UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/parameterized_view', 'r1') ORDER BY Name" $CLICKHOUSE_CLIENT -q "INSERT INTO test_02428_Catalog VALUES ('Pen', 10, 3)" $CLICKHOUSE_CLIENT -q "INSERT INTO test_02428_Catalog VALUES ('Book', 50, 2)" @@ -72,7 +72,7 @@ $CLICKHOUSE_CLIENT -q "SELECT Price FROM \`${CLICKHOUSE_TEST_UNIQUE_NAME}.pv1\`( $CLICKHOUSE_CLIENT -q "INSERT INTO test_02428_Catalog VALUES ('Book2', 30, 8)" $CLICKHOUSE_CLIENT -q "INSERT INTO test_02428_Catalog VALUES ('Book3', 30, 8)" -$CLICKHOUSE_CLIENT -q "CREATE VIEW test_02428_pv5 AS SELECT Price FROM test_02428_Catalog WHERE {price:UInt64} HAVING Quantity in (SELECT {quantity:UInt64}) LIMIT {limit:UInt64}" +$CLICKHOUSE_CLIENT -q "CREATE VIEW test_02428_pv5 AS SELECT Price FROM test_02428_Catalog WHERE Price={price:UInt64} HAVING Quantity in (SELECT {quantity:UInt64}) LIMIT {limit:UInt64}" $CLICKHOUSE_CLIENT -q "SELECT Price FROM test_02428_pv5(price=30, quantity=8, limit=1)" $CLICKHOUSE_CLIENT -q "CREATE VIEW test_02428_pv6 AS SELECT Price+{price:UInt64} FROM test_02428_Catalog GROUP BY Price+{price:UInt64} ORDER BY Price+{price:UInt64}" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02428_pv6(price=10)" From 1df2593d11292867cd4c3d1be9538bec3dea5e87 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Mar 2023 08:16:05 +0100 Subject: [PATCH 163/418] Add a test for #23162 --- ...ng_identifier_shouldnt_be_accepted.reference | 0 ...94_wrong_identifier_shouldnt_be_accepted.sql | 17 +++++++++++++++++ 2 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/02694_wrong_identifier_shouldnt_be_accepted.reference create mode 100644 tests/queries/0_stateless/02694_wrong_identifier_shouldnt_be_accepted.sql diff --git a/tests/queries/0_stateless/02694_wrong_identifier_shouldnt_be_accepted.reference b/tests/queries/0_stateless/02694_wrong_identifier_shouldnt_be_accepted.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02694_wrong_identifier_shouldnt_be_accepted.sql b/tests/queries/0_stateless/02694_wrong_identifier_shouldnt_be_accepted.sql new file mode 100644 index 00000000000..e929b1e620a --- /dev/null +++ b/tests/queries/0_stateless/02694_wrong_identifier_shouldnt_be_accepted.sql @@ -0,0 +1,17 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS s; + +CREATE TABLE t1 ( k Int64, x Int64) ENGINE = Memory; +CREATE TABLE t2 ( x Int64 ) ENGINE = Memory; + +create table s (k Int64, d DateTime) Engine=Memory; + +SELECT * FROM t1 +INNER JOIN s ON t1.k = s.k +INNER JOIN t2 ON t2.x = t1.x +WHERE (t1.d >= now()); -- { serverError UNKNOWN_IDENTIFIER } + +DROP TABLE t1; +DROP TABLE t2; +DROP TABLE s; From 9b3e57070f9efd3a2f79e838313886d83b2b1ed7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 20 Mar 2023 08:33:33 +0000 Subject: [PATCH 164/418] Fix broken cross-compiled macos builds --- CMakeLists.txt | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 1fc708e0807..49367705a08 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -353,12 +353,14 @@ if (COMPILER_CLANG) set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,-U,_inside_main") - # The LLVM MachO linker (ld64.lld) generates by default unwind info in 'compact' format which the internal unwinder doesn't support - # and the server will not come up ('invalid compact unwind encoding'). Disable it. - # You will see warning during the build "ld64.lld: warning: Option `-no_compact_unwind' is undocumented. Should lld implement it?". - # Yes, ld64.lld does not document the option, likely for compat with Apple's system ld after which ld64.lld is modeled after and - # which also does not document it. - set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,-no_compact_unwind") + # The LLVM MachO linker (ld64.lld, used in native builds) generates by default unwind info in 'compact' format which the internal + # unwinder doesn't support and the server will not come up ('invalid compact unwind encoding'). Disable it. You will see warning + # during the build "ld64.lld: warning: Option `-no_compact_unwind' is undocumented. Should lld implement it?". Yes, ld64.lld does + # not document the option, likely for compat with Apple's system ld after which ld64.lld is modeled after and which also does not + # document it. + if (NOT CMAKE_CROSSCOMPILING) + set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,-no_compact_unwind") + endif () endif() # Display absolute paths in error messages. Otherwise KDevelop fails to navigate to correct file and opens a new file instead. From 722404ca6431f1596605fab064f90a4227411739 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 20 Mar 2023 10:34:55 +0100 Subject: [PATCH 165/418] Disallow concurrency for backup restore test, added another insert block to data and setting mutations_sync --- .../test_disallow_concurrency.py | 4 +++- 1 file changed, 3 insertions(+), 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 6d0b62e79f2..3446d805d3e 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 @@ -101,6 +101,7 @@ def new_backup_name(): def create_and_fill_table(): + node0.query("SET mutations_sync=2"); node0.query( "CREATE TABLE tbl ON CLUSTER 'cluster' (" "x UInt64" @@ -108,7 +109,8 @@ 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(80000000)") + nodes[i].query(f"INSERT INTO tbl SELECT number FROM numbers(100000000)") + nodes[i].query(f"INSERT INTO tbl SELECT number+100000000 FROM numbers(100000000)") # All the tests have concurrent backup/restores with same backup names From 826dd397b1857e33d1b035b3d648a9dc0f25b994 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 20 Mar 2023 09:58:42 +0000 Subject: [PATCH 166/418] Automatic style fix --- .../test_disallow_concurrency.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 3446d805d3e..fb09920245e 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 @@ -101,7 +101,7 @@ def new_backup_name(): def create_and_fill_table(): - node0.query("SET mutations_sync=2"); + node0.query("SET mutations_sync=2") node0.query( "CREATE TABLE tbl ON CLUSTER 'cluster' (" "x UInt64" @@ -110,7 +110,9 @@ def create_and_fill_table(): ) for i in range(num_nodes): nodes[i].query(f"INSERT INTO tbl SELECT number FROM numbers(100000000)") - nodes[i].query(f"INSERT INTO tbl SELECT number+100000000 FROM numbers(100000000)") + nodes[i].query( + f"INSERT INTO tbl SELECT number+100000000 FROM numbers(100000000)" + ) # All the tests have concurrent backup/restores with same backup names From 8537ee400d36b5b477de9235d8bf9c101962e699 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 20 Mar 2023 11:22:25 +0100 Subject: [PATCH 167/418] Fix kafka test --- src/Storages/Kafka/StorageKafka.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 538b170ca59..3381561eb1b 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -848,8 +848,6 @@ void registerStorageKafka(StorageFactory & factory) } 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) { From 506e0c622ba222fe2882306801016869d739650a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 20 Mar 2023 10:31:40 +0000 Subject: [PATCH 168/418] Randomize query cache settings --- tests/clickhouse-test | 2 ++ .../0_stateless/02494_query_cache_case_agnostic_matching.sql | 3 ++- tests/queries/0_stateless/02494_query_cache_drop_cache.sql | 3 ++- .../0_stateless/02494_query_cache_eligible_queries.sql | 3 ++- tests/queries/0_stateless/02494_query_cache_events.sql | 3 ++- .../0_stateless/02494_query_cache_exception_handling.sql | 3 ++- tests/queries/0_stateless/02494_query_cache_explain.sql | 3 ++- .../0_stateless/02494_query_cache_min_query_duration.sql | 5 +++-- .../queries/0_stateless/02494_query_cache_min_query_runs.sql | 5 +++-- .../02494_query_cache_nondeterministic_functions.sql | 3 ++- .../queries/0_stateless/02494_query_cache_normalize_ast.sql | 3 ++- .../queries/0_stateless/02494_query_cache_passive_usage.sql | 3 ++- tests/queries/0_stateless/02494_query_cache_secrets.sql | 3 ++- tests/queries/0_stateless/02494_query_cache_ttl_long.sql | 3 ++- 14 files changed, 30 insertions(+), 15 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index aec52981724..185f49062f9 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -528,6 +528,8 @@ class SettingsRandomizer: "enable_memory_bound_merging_of_aggregation_results": lambda: random.randint( 0, 1 ), + "use_query_cache": lambda: random.randint(0, 1), + "enable_reads_from_query_cache": lambda: 0, } @staticmethod diff --git a/tests/queries/0_stateless/02494_query_cache_case_agnostic_matching.sql b/tests/queries/0_stateless/02494_query_cache_case_agnostic_matching.sql index 9440a1fd9c0..b827701b1d7 100644 --- a/tests/queries/0_stateless/02494_query_cache_case_agnostic_matching.sql +++ b/tests/queries/0_stateless/02494_query_cache_case_agnostic_matching.sql @@ -1,5 +1,6 @@ --- Tags: no-parallel +-- Tags: no-parallel, no-random-settings -- Tag no-parallel: Messes with internal cache +-- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; diff --git a/tests/queries/0_stateless/02494_query_cache_drop_cache.sql b/tests/queries/0_stateless/02494_query_cache_drop_cache.sql index 1f61472fcb0..8c968edbef8 100644 --- a/tests/queries/0_stateless/02494_query_cache_drop_cache.sql +++ b/tests/queries/0_stateless/02494_query_cache_drop_cache.sql @@ -1,5 +1,6 @@ --- Tags: no-parallel +-- Tags: no-parallel, no-random-settings -- Tag no-parallel: Messes with internal cache +-- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; diff --git a/tests/queries/0_stateless/02494_query_cache_eligible_queries.sql b/tests/queries/0_stateless/02494_query_cache_eligible_queries.sql index b4bc9e2c258..842770e155a 100644 --- a/tests/queries/0_stateless/02494_query_cache_eligible_queries.sql +++ b/tests/queries/0_stateless/02494_query_cache_eligible_queries.sql @@ -1,5 +1,6 @@ --- Tags: no-parallel +-- Tags: no-parallel, no-random-settings -- Tag no-parallel: Messes with internal cache +-- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; diff --git a/tests/queries/0_stateless/02494_query_cache_events.sql b/tests/queries/0_stateless/02494_query_cache_events.sql index d775467d525..1c49fdbd2e9 100644 --- a/tests/queries/0_stateless/02494_query_cache_events.sql +++ b/tests/queries/0_stateless/02494_query_cache_events.sql @@ -1,5 +1,6 @@ --- Tags: no-parallel +-- Tags: no-parallel, no-random-settings -- Tag no-parallel: Messes with internal cache +-- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; diff --git a/tests/queries/0_stateless/02494_query_cache_exception_handling.sql b/tests/queries/0_stateless/02494_query_cache_exception_handling.sql index 4d686d81ed3..a7a25f29d31 100644 --- a/tests/queries/0_stateless/02494_query_cache_exception_handling.sql +++ b/tests/queries/0_stateless/02494_query_cache_exception_handling.sql @@ -1,5 +1,6 @@ --- Tags: no-parallel +-- Tags: no-parallel, no-random-settings -- Tag no-parallel: Messes with internal cache +-- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; diff --git a/tests/queries/0_stateless/02494_query_cache_explain.sql b/tests/queries/0_stateless/02494_query_cache_explain.sql index 68b7e0005f8..ffd3c1e80e3 100644 --- a/tests/queries/0_stateless/02494_query_cache_explain.sql +++ b/tests/queries/0_stateless/02494_query_cache_explain.sql @@ -1,5 +1,6 @@ --- Tags: no-parallel +-- Tags: no-parallel, no-random-settings -- Tag no-parallel: Messes with internal cache +-- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_analyzer = 1; SET allow_experimental_query_cache = true; diff --git a/tests/queries/0_stateless/02494_query_cache_min_query_duration.sql b/tests/queries/0_stateless/02494_query_cache_min_query_duration.sql index 7d759c86130..fb2b9719189 100644 --- a/tests/queries/0_stateless/02494_query_cache_min_query_duration.sql +++ b/tests/queries/0_stateless/02494_query_cache_min_query_duration.sql @@ -1,5 +1,6 @@ --- Tags: no-parallel --- Tag no-parallel: Messes with internal cache +-- Tags: no-parallel, no-random-settings +-- Tag no-parallel: The test messes with internal cache +-- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; diff --git a/tests/queries/0_stateless/02494_query_cache_min_query_runs.sql b/tests/queries/0_stateless/02494_query_cache_min_query_runs.sql index 2401beeab13..d0317a18fdb 100644 --- a/tests/queries/0_stateless/02494_query_cache_min_query_runs.sql +++ b/tests/queries/0_stateless/02494_query_cache_min_query_runs.sql @@ -1,5 +1,6 @@ --- Tags: no-parallel --- Tag no-parallel: Messes with internal cache +-- Tags: no-parallel, no-random-settings +-- Tag no-parallel: The test messes with internal cache +-- Tag no-random-settings: The test assumes default settings for the query cache. SET allow_experimental_query_cache = true; diff --git a/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql b/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql index 534d63aa427..c11acd737f8 100644 --- a/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql +++ b/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql @@ -1,5 +1,6 @@ --- Tags: no-parallel +-- Tags: no-parallel, no-random-settings -- Tag no-parallel: Messes with internal cache +-- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; diff --git a/tests/queries/0_stateless/02494_query_cache_normalize_ast.sql b/tests/queries/0_stateless/02494_query_cache_normalize_ast.sql index 7e3cd273312..6f0b24953a6 100644 --- a/tests/queries/0_stateless/02494_query_cache_normalize_ast.sql +++ b/tests/queries/0_stateless/02494_query_cache_normalize_ast.sql @@ -1,5 +1,6 @@ --- Tags: no-parallel +-- Tags: no-parallel, no-random-settings -- Tag no-parallel: Messes with internal cache +-- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; diff --git a/tests/queries/0_stateless/02494_query_cache_passive_usage.sql b/tests/queries/0_stateless/02494_query_cache_passive_usage.sql index 8f1e3972b6d..8041cd4b417 100644 --- a/tests/queries/0_stateless/02494_query_cache_passive_usage.sql +++ b/tests/queries/0_stateless/02494_query_cache_passive_usage.sql @@ -1,5 +1,6 @@ --- Tags: no-parallel +-- Tags: no-parallel, no-random-settings -- Tag no-parallel: Messes with internal cache +-- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; diff --git a/tests/queries/0_stateless/02494_query_cache_secrets.sql b/tests/queries/0_stateless/02494_query_cache_secrets.sql index 99a972b003c..7c090fab9c8 100644 --- a/tests/queries/0_stateless/02494_query_cache_secrets.sql +++ b/tests/queries/0_stateless/02494_query_cache_secrets.sql @@ -1,6 +1,7 @@ --- Tags: no-parallel, no-fasttest +-- Tags: no-parallel, no-fasttest, no-random-settings -- Tag no-fasttest: Depends on OpenSSL -- Tag no-parallel: Messes with internal cache +-- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; diff --git a/tests/queries/0_stateless/02494_query_cache_ttl_long.sql b/tests/queries/0_stateless/02494_query_cache_ttl_long.sql index 135ddf2195c..1a99d85688d 100644 --- a/tests/queries/0_stateless/02494_query_cache_ttl_long.sql +++ b/tests/queries/0_stateless/02494_query_cache_ttl_long.sql @@ -1,7 +1,8 @@ --- Tags: no-fasttest, no-parallel, long +-- Tags: no-fasttest, no-parallel, long, no-random-settings -- Tag no-fasttest: Test runtime is > 6 sec -- Tag long: Test runtime is > 6 sec -- Tag no-parallel: Messes with internal cache +-- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; From a78b817e6fec8ba814e8d2db9d178a2d619811aa Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 20 Mar 2023 09:45:12 +0000 Subject: [PATCH 169/418] Fix logical error in evaluate constant expression --- src/Interpreters/ActionsVisitor.cpp | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 96765683abc..766740665a0 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -883,12 +883,18 @@ void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr &, Dat void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & data) { - 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"); + bool is_valid_function = node.name == "arrayJoin" || node.name == "grouping" || node.name == "indexHint" || + AggregateUtils::isAggregateFunction(node) || + node.is_window_function || + FunctionFactory::instance().has(node.name) || + UserDefinedExecutableFunctionFactory::instance().has(node.name, data.getContext()); + + if (!is_valid_function) + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unexpected function '{}'", node.name); + auto column_name = ast->getColumnName(); if (data.hasColumn(column_name)) return; From a3c4f3f8b457dfb387b77c27aba0783016c71f6e Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Mon, 20 Mar 2023 08:41:58 -0400 Subject: [PATCH 170/418] Update 00678_murmurhash.sql --- tests/queries/0_stateless/00678_murmurhash.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00678_murmurhash.sql b/tests/queries/0_stateless/00678_murmurhash.sql index cff3cc9996f..eda29fd17cd 100644 --- a/tests/queries/0_stateless/00678_murmurhash.sql +++ b/tests/queries/0_stateless/00678_murmurhash.sql @@ -32,6 +32,7 @@ SELECT gccMurmurHash('foo'); SELECT gccMurmurHash('\x01'); SELECT gccMurmurHash(1); +-- Comparison with reverse for big endian SELECT hex(murmurHash3_128('foo')) = hex(reverse(unhex('6145F501578671E2877DBA2BE487AF7E'))) or hex(murmurHash3_128('foo')) = '6145F501578671E2877DBA2BE487AF7E'; - +-- Comparison with reverse for big endian SELECT hex(murmurHash3_128('\x01')) = hex(reverse(unhex('16FE7483905CCE7A85670E43E4678877'))) or hex(murmurHash3_128('\x01')) = '16FE7483905CCE7A85670E43E4678877'; From 9b8d20e4439605522f7a3a41c0efd68775977468 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 20 Mar 2023 12:42:09 +0000 Subject: [PATCH 171/418] Revert better fix logical error in evaluate constant expression --- src/Interpreters/ActionsVisitor.cpp | 18 +++--------------- .../evaluateConstantExpression.cpp | 2 +- 2 files changed, 4 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 766740665a0..43db25e775f 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -33,8 +33,6 @@ #include -#include - #include #include #include @@ -69,7 +67,6 @@ 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; @@ -883,22 +880,13 @@ void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr &, Dat void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & data) { - if (node.name == "lambda") - throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION, "Unexpected lambda expression"); - - bool is_valid_function = node.name == "arrayJoin" || node.name == "grouping" || node.name == "indexHint" || - AggregateUtils::isAggregateFunction(node) || - node.is_window_function || - FunctionFactory::instance().has(node.name) || - UserDefinedExecutableFunctionFactory::instance().has(node.name, data.getContext()); - - if (!is_valid_function) - throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unexpected function '{}'", node.name); - auto column_name = ast->getColumnName(); if (data.hasColumn(column_name)) return; + if (node.name == "lambda") + throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION, "Unexpected lambda expression"); + /// Function arrayJoin. if (node.name == "arrayJoin") { diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index 5a333172b14..ebefa0d9ce7 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -70,6 +70,7 @@ 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. @@ -81,7 +82,6 @@ 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) From 5b036a1a3bc783086265c3070c9b868f53a9fe4c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 20 Mar 2023 12:55:03 +0000 Subject: [PATCH 172/418] More preparation for libcxx(abi), llvm, clang-tidy 16 (follow-up to #47722) --- base/base/wide_integer_to_string.h | 2 +- src/Dictionaries/RedisDictionarySource.cpp | 10 +++++ src/Dictionaries/RedisDictionarySource.h | 11 +---- src/Interpreters/ActionsVisitor.cpp | 8 ++-- src/Storages/Distributed/DistributedSink.cpp | 4 ++ src/Storages/Distributed/DistributedSink.h | 3 +- src/Storages/HDFS/ReadBufferFromHDFS.cpp | 4 +- src/Storages/MergeTree/MergeTreeSink.cpp | 28 ++++++------- src/Storages/MergeTree/MergeTreeSource.cpp | 44 ++++++++++---------- 9 files changed, 60 insertions(+), 54 deletions(-) diff --git a/base/base/wide_integer_to_string.h b/base/base/wide_integer_to_string.h index 160bf599516..c2cbe8d82e3 100644 --- a/base/base/wide_integer_to_string.h +++ b/base/base/wide_integer_to_string.h @@ -64,6 +64,6 @@ struct fmt::formatter> template auto format(const wide::integer & value, FormatContext & ctx) { - return format_to(ctx.out(), "{}", to_string(value)); + return fmt::format_to(ctx.out(), "{}", to_string(value)); } }; diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index bde4d596352..6e4c5d1d5d9 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -68,6 +68,16 @@ namespace DB factory.registerSource("redis", create_table_source); } + RedisDictionarySource::Connection::Connection(PoolPtr pool_, ClientPtr client_) + : pool(std::move(pool_)), client(std::move(client_)) + { + } + + RedisDictionarySource::Connection::~Connection() + { + pool->returnObject(std::move(client)); + } + static constexpr size_t REDIS_MAX_BLOCK_SIZE = DEFAULT_BLOCK_SIZE; static constexpr size_t REDIS_LOCK_ACQUIRE_TIMEOUT_MS = 5000; diff --git a/src/Dictionaries/RedisDictionarySource.h b/src/Dictionaries/RedisDictionarySource.h index 26f5ab2a613..8fb6f93193b 100644 --- a/src/Dictionaries/RedisDictionarySource.h +++ b/src/Dictionaries/RedisDictionarySource.h @@ -52,15 +52,8 @@ namespace DB struct Connection { - Connection(PoolPtr pool_, ClientPtr client_) - : pool(std::move(pool_)), client(std::move(client_)) - { - } - - ~Connection() - { - pool->returnObject(std::move(client)); - } + Connection(PoolPtr pool_, ClientPtr client_); + ~Connection(); PoolPtr pool; ClientPtr client; diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 43db25e775f..8a5ea1205e7 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -467,10 +467,6 @@ SetPtr makeExplicitSet( return set; } -ScopeStack::Level::~Level() = default; -ScopeStack::Level::Level() = default; -ScopeStack::Level::Level(Level &&) noexcept = default; - class ScopeStack::Index { /// Map column name -> Node. @@ -524,6 +520,10 @@ public: } }; +ScopeStack::Level::~Level() = default; +ScopeStack::Level::Level() = default; +ScopeStack::Level::Level(Level &&) noexcept = default; + ActionsMatcher::Data::Data( ContextPtr context_, SizeLimits set_size_limit_, diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 19eab0b5837..73d3aebe0d0 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -210,6 +210,10 @@ std::string DistributedSink::getCurrentStateDescription() } +DistributedSink::JobReplica::JobReplica(size_t shard_index_, size_t replica_index_, bool is_local_job_, const Block & sample_block) + : shard_index(shard_index_), replica_index(replica_index_), is_local_job(is_local_job_), current_shard_block(sample_block.cloneEmpty()) {} + + void DistributedSink::initWritingJobs(const Block & first_block, size_t start, size_t end) { const Settings & settings = context->getSettingsRef(); diff --git a/src/Storages/Distributed/DistributedSink.h b/src/Storages/Distributed/DistributedSink.h index 325d5859289..1bb4419e1a5 100644 --- a/src/Storages/Distributed/DistributedSink.h +++ b/src/Storages/Distributed/DistributedSink.h @@ -118,8 +118,7 @@ private: struct JobReplica { JobReplica() = default; - JobReplica(size_t shard_index_, size_t replica_index_, bool is_local_job_, const Block & sample_block) - : shard_index(shard_index_), replica_index(replica_index_), is_local_job(is_local_job_), current_shard_block(sample_block.cloneEmpty()) {} + JobReplica(size_t shard_index_, size_t replica_index_, bool is_local_job_, const Block & sample_block); size_t shard_index = 0; size_t replica_index = 0; diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index 3e7c27fe4f2..ee8e0764db0 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -29,8 +29,6 @@ namespace ErrorCodes } -ReadBufferFromHDFS::~ReadBufferFromHDFS() = default; - struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory { String hdfs_uri; @@ -166,6 +164,8 @@ ReadBufferFromHDFS::ReadBufferFromHDFS( { } +ReadBufferFromHDFS::~ReadBufferFromHDFS() = default; + size_t ReadBufferFromHDFS::getFileSize() { return impl->getFileSize(); diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index 1e607767f86..981eb1af280 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -13,6 +13,20 @@ namespace ProfileEvents namespace DB { +struct MergeTreeSink::DelayedChunk +{ + struct Partition + { + MergeTreeDataWriter::TemporaryPart temp_part; + UInt64 elapsed_ns; + String block_dedup_token; + ProfileEvents::Counters part_counters; + }; + + std::vector partitions; +}; + + MergeTreeSink::~MergeTreeSink() = default; MergeTreeSink::MergeTreeSink( @@ -41,20 +55,6 @@ void MergeTreeSink::onFinish() finishDelayedChunk(); } -struct MergeTreeSink::DelayedChunk -{ - struct Partition - { - MergeTreeDataWriter::TemporaryPart temp_part; - UInt64 elapsed_ns; - String block_dedup_token; - ProfileEvents::Counters part_counters; - }; - - std::vector partitions; -}; - - void MergeTreeSink::consume(Chunk chunk) { auto block = getHeader().cloneWithColumns(chunk.detachColumns()); diff --git a/src/Storages/MergeTree/MergeTreeSource.cpp b/src/Storages/MergeTree/MergeTreeSource.cpp index a37d1d3ec2c..328336ff71a 100644 --- a/src/Storages/MergeTree/MergeTreeSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSource.cpp @@ -7,28 +7,6 @@ namespace DB { -MergeTreeSource::MergeTreeSource(MergeTreeSelectAlgorithmPtr algorithm_) - : ISource(algorithm_->getHeader()) - , algorithm(std::move(algorithm_)) -{ -#if defined(OS_LINUX) - if (algorithm->getSettings().use_asynchronous_read_from_pool) - async_reading_state = std::make_unique(); -#endif -} - -MergeTreeSource::~MergeTreeSource() = default; - -std::string MergeTreeSource::getName() const -{ - return algorithm->getName(); -} - -void MergeTreeSource::onCancel() -{ - algorithm->cancel(); -} - #if defined(OS_LINUX) struct MergeTreeSource::AsyncReadingState { @@ -155,6 +133,28 @@ private: }; #endif +MergeTreeSource::MergeTreeSource(MergeTreeSelectAlgorithmPtr algorithm_) + : ISource(algorithm_->getHeader()) + , algorithm(std::move(algorithm_)) +{ +#if defined(OS_LINUX) + if (algorithm->getSettings().use_asynchronous_read_from_pool) + async_reading_state = std::make_unique(); +#endif +} + +MergeTreeSource::~MergeTreeSource() = default; + +std::string MergeTreeSource::getName() const +{ + return algorithm->getName(); +} + +void MergeTreeSource::onCancel() +{ + algorithm->cancel(); +} + ISource::Status MergeTreeSource::prepare() { #if defined(OS_LINUX) From 10980554091440304b8a235079947e685f2c3ad5 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 20 Mar 2023 13:21:10 +0000 Subject: [PATCH 173/418] Stop `wait for quorum` retries on shutdown --- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 035cbdac55e..43e7e1e3979 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -1073,13 +1073,26 @@ std::vector ReplicatedMergeTreeSinkImpl::commitPart( } }, [&zookeeper]() { zookeeper->cleanupEphemeralNodes(); }); + if (!conflict_block_ids.empty()) return conflict_block_ids; + if (isQuorumEnabled()) { ZooKeeperRetriesControl quorum_retries_ctl("waitForQuorum", zookeeper_retries_info); quorum_retries_ctl.retryLoop([&]() { + if (storage.is_readonly) + { + /// stop retries if in shutdown + if (storage.shutdown_called) + throw Exception( + ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to shutdown: replica_path={}", storage.replica_path); + + quorum_retries_ctl.setUserError(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode: replica_path={}", storage.replica_path); + return; + } + zookeeper->setKeeper(storage.getZooKeeper()); if (is_already_existing_part) From 795a1c84e1d77390dede017756af701250058039 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 20 Mar 2023 13:48:04 +0000 Subject: [PATCH 174/418] Actually stop retries in case of storage shutdown --- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 43e7e1e3979..600e2f408ef 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -649,8 +649,11 @@ std::vector ReplicatedMergeTreeSinkImpl::commitPart( { /// stop retries if in shutdown if (storage.shutdown_called) + { + retries_ctl.stopRetries(); throw Exception( ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to shutdown: replica_path={}", storage.replica_path); + } /// When we attach existing parts it's okay to be in read-only mode /// For example during RESTORE REPLICA. @@ -1086,8 +1089,11 @@ std::vector ReplicatedMergeTreeSinkImpl::commitPart( { /// stop retries if in shutdown if (storage.shutdown_called) + { + quorum_retries_ctl.stopRetries(); throw Exception( ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to shutdown: replica_path={}", storage.replica_path); + } quorum_retries_ctl.setUserError(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode: replica_path={}", storage.replica_path); return; From 1073c7f3c942949d1f5de6956fab9b3bcde8b34b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 20 Mar 2023 13:54:25 +0000 Subject: [PATCH 175/418] Revert "Actually stop retries in case of storage shutdown" This was unnecessary since, as for exceptions, retries done only on zkutil::KeeperException This reverts commit 795a1c84e1d77390dede017756af701250058039. --- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 600e2f408ef..43e7e1e3979 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -649,11 +649,8 @@ std::vector ReplicatedMergeTreeSinkImpl::commitPart( { /// stop retries if in shutdown if (storage.shutdown_called) - { - retries_ctl.stopRetries(); throw Exception( ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to shutdown: replica_path={}", storage.replica_path); - } /// When we attach existing parts it's okay to be in read-only mode /// For example during RESTORE REPLICA. @@ -1089,11 +1086,8 @@ std::vector ReplicatedMergeTreeSinkImpl::commitPart( { /// stop retries if in shutdown if (storage.shutdown_called) - { - quorum_retries_ctl.stopRetries(); throw Exception( ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to shutdown: replica_path={}", storage.replica_path); - } quorum_retries_ctl.setUserError(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode: replica_path={}", storage.replica_path); return; From 52a394168e88ef940be104eac2f8cb66a6000398 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 20 Mar 2023 14:33:27 +0000 Subject: [PATCH 176/418] fix clang-tidy --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index cee004df536..b2b4d813eee 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1585,8 +1585,8 @@ void QueryAnalyzer::collectCompoundExpressionValidIdentifiersForTypoCorrection( if (new_identifier_size == unresolved_identifier.getPartsSize()) { auto new_identifier = valid_identifier_prefix; - for (auto && part : subcolumn_indentifier) - new_identifier.emplace_back(std::move(part)); + for (const auto & part : subcolumn_indentifier) + new_identifier.push_back(part); valid_identifiers_result.insert(std::move(new_identifier)); } From 2d9088249d8770756c7e929c2a6f572e5f30a1f0 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Mon, 20 Mar 2023 08:57:13 -0600 Subject: [PATCH 177/418] Update s3.md --- docs/en/sql-reference/table-functions/s3.md | 41 ++++++++++++--------- 1 file changed, 24 insertions(+), 17 deletions(-) diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index d7199717798..6997d6c30ff 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -43,35 +43,42 @@ A table with the specified structure for reading or writing data in the specifie **Examples** -Selecting the first two rows from the table from S3 file `https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/data.csv`: +Selecting the first 5 rows from the table from S3 file `https://datasets-documentation.s3.eu-west-3.amazonaws.com/aapl_stock.csv`: ``` sql SELECT * -FROM s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/data.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32') -LIMIT 2; +FROM s3( + 'https://datasets-documentation.s3.eu-west-3.amazonaws.com/aapl_stock.csv', + 'CSVWithNames' +) +LIMIT 5; ``` -``` text -┌─column1─┬─column2─┬─column3─┐ -│ 1 │ 2 │ 3 │ -│ 3 │ 2 │ 1 │ -└─────────┴─────────┴─────────┘ +```response +┌───────Date─┬────Open─┬────High─┬─────Low─┬───Close─┬───Volume─┬─OpenInt─┐ +│ 1984-09-07 │ 0.42388 │ 0.42902 │ 0.41874 │ 0.42388 │ 23220030 │ 0 │ +│ 1984-09-10 │ 0.42388 │ 0.42516 │ 0.41366 │ 0.42134 │ 18022532 │ 0 │ +│ 1984-09-11 │ 0.42516 │ 0.43668 │ 0.42516 │ 0.42902 │ 42498199 │ 0 │ +│ 1984-09-12 │ 0.42902 │ 0.43157 │ 0.41618 │ 0.41618 │ 37125801 │ 0 │ +│ 1984-09-13 │ 0.43927 │ 0.44052 │ 0.43927 │ 0.43927 │ 57822062 │ 0 │ +└────────────┴─────────┴─────────┴─────────┴─────────┴──────────┴─────────┘ ``` -The similar but from file with `gzip` compression: +:::note +ClickHouse uses filename extensions to determine the format of the data and also the compression of the file. For example, if the file was zipped up with a `.csv.gz` extension, ClickHouse would decompress the file. + +To demonstrate, try the following command - which works the same as the example above: ``` sql SELECT * -FROM s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/data.csv.gz', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32', 'gzip') -LIMIT 2; +FROM s3( + 'https://datasets-documentation.s3.eu-west-3.amazonaws.com/aapl_stock.csv' +) +LIMIT 5; ``` -``` text -┌─column1─┬─column2─┬─column3─┐ -│ 1 │ 2 │ 3 │ -│ 3 │ 2 │ 1 │ -└─────────┴─────────┴─────────┘ -``` +::: + ## Usage From 19fb88c2c1ac2cd67fd377066251ddb5cce7f3f2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 20 Mar 2023 12:42:30 +0000 Subject: [PATCH 178/418] Simple fix logical error in evaluate constant expression --- src/Interpreters/evaluateConstantExpression.cpp | 2 +- src/Parsers/ASTFunction.cpp | 7 ++++--- .../queries/0_stateless/01715_table_function_view_fix.sql | 2 +- 3 files changed, 6 insertions(+), 5 deletions(-) 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/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index 129d3d60744..fa4874d89b6 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -28,8 +28,8 @@ namespace DB namespace ErrorCodes { - extern const int UNEXPECTED_EXPRESSION; extern const int UNEXPECTED_AST_STRUCTURE; + extern const int UNKNOWN_FUNCTION; } @@ -471,8 +471,9 @@ namespace void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const { - if (name == "view") - throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION, "Table function view cannot be used as an expression"); + /// These functions contain some unexpected ASTs in arguments (e.g. SETTINGS or even a SELECT query) + if (name == "view" || name == "viewIfPermitted" || name == "mysql" || name == "postgresql" || name == "mongodb" || name == "s3") + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Table function '{}' cannot be used as an expression", name); /// If function can be converted to literal it will be parsed as literal after formatting. /// In distributed query it may lead to mismathed column names. diff --git a/tests/queries/0_stateless/01715_table_function_view_fix.sql b/tests/queries/0_stateless/01715_table_function_view_fix.sql index b96609391b5..5c24131b438 100644 --- a/tests/queries/0_stateless/01715_table_function_view_fix.sql +++ b/tests/queries/0_stateless/01715_table_function_view_fix.sql @@ -1,3 +1,3 @@ SELECT view(SELECT 1); -- { clientError 62 } -SELECT sumIf(dummy, dummy) FROM remote('127.0.0.{1,2}', numbers(2, 100), view(SELECT CAST(NULL, 'Nullable(UInt8)') AS dummy FROM system.one)); -- { serverError 183 } +SELECT sumIf(dummy, dummy) FROM remote('127.0.0.{1,2}', numbers(2, 100), view(SELECT CAST(NULL, 'Nullable(UInt8)') AS dummy FROM system.one)); -- { serverError UNKNOWN_FUNCTION } From c77ed913198e2f4754febc5696c9c79a38f44384 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 20 Mar 2023 23:20:57 +0800 Subject: [PATCH 179/418] change as request --- src/Functions/parseDateTime.cpp | 310 ++++++++++++++++---------------- 1 file changed, 151 insertions(+), 159 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 58d9d8f317b..d43ffb376de 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -98,8 +98,6 @@ namespace 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; /// If true, year is calculated from century, range [19, 21] - 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. @@ -111,7 +109,7 @@ namespace Int32 minute = 0; /// range [0, 59] Int32 second = 0; /// range [0, 59] - bool is_am = true; /// AM -> true, PM -> false + bool is_am = true; /// If is_hour_of_half_day = true and is_am = false (i.e. pm) then add 12 hours to the result DateTime bool is_clock_hour = false; /// Whether the hour is clockhour bool is_hour_of_half_day = false; /// Whether the hour is of half day @@ -131,8 +129,6 @@ namespace day_of_year = 1; day_of_year_format = false; - century_format = false; - is_year_of_era = false; has_year = false; @@ -148,31 +144,51 @@ namespace time_zone_offset = 0; } + void setEra(String & text) // NOLINT + { + boost::to_lower(text); + if (text == "bc") + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Era BC exceeds the range of DateTime"); + else if (text != "ad") + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Unknown era {}", text); + } + void setCentury(Int32 century) { if (century < 19 || century > 21) throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "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_) + void setYear(Int32 year_, bool is_year_of_era_ = false, bool is_week_year = false) { - if (day_of_week_ < 1 || day_of_week_ > 7) - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Value {} for day of week must be in the range [1, 7]", day_of_week_); + if (year_ < minYear || year_ > maxYear) + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Value {} for year must be in the range [{}, {}]", year_, minYear, maxYear); - day_of_week = day_of_week_; - week_date_format = true; - day_of_year_format = false; - if (!has_year) + year = year_; + has_year = true; + is_year_of_era = is_year_of_era_; + if (is_week_year) { - has_year = true; - year = 2000; + week_date_format = true; + day_of_year_format = false; } } + 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::CANNOT_PARSE_DATETIME, "Value {} for year2 must be in the range [0, 99]", year_); + + setYear(year_, is_year_of_era_, is_week_year); + } + void setMonth(Int32 month_) { if (month_ < 1 || month_ > 12) @@ -188,6 +204,36 @@ namespace } } + void setWeek(Int32 week_) + { + if (week_ < 1 || week_ > 53) + 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; + day_of_year_format = false; + if (!has_year) + { + has_year = true; + year = 2000; + } + } + + void setDayOfYear(Int32 day_of_year_) + { + if (day_of_year_ < 1 || day_of_year_ > 366) + 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; + week_date_format = false; + if (!has_year) + { + has_year = true; + year = 2000; + } + } + void setDayOfMonth(Int32 day_of_month) { if (day_of_month < 1 || day_of_month > 31) @@ -203,55 +249,12 @@ namespace } } - ALWAYS_INLINE void setDayOfYear(Int32 day_of_year_) + void setDayOfWeek(Int32 day_of_week_) { - if (day_of_year_ < 1 || day_of_year_ > 366) - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Value {} for day of year must be in the range [1, 366]", day_of_year_); + if (day_of_week_ < 1 || day_of_week_ > 7) + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Value {} for day of week must be in the range [1, 7]", day_of_week_); - 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::CANNOT_PARSE_DATETIME, "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::CANNOT_PARSE_DATETIME, "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::CANNOT_PARSE_DATETIME, "Value {} for week of week year must be in the range [1, 53]", week_); - - week = week_; + day_of_week = day_of_week_; week_date_format = true; day_of_year_format = false; if (!has_year) @@ -261,32 +264,7 @@ namespace } } - void setMinute(Int32 minute_) - { - if (minute_ < 0 || minute_ > 59) - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "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::CANNOT_PARSE_DATETIME, "Value {} for second must be in the range [0, 59]", second_); - - second = second_; - } - - void setEra(String & text) // NOLINT - { - boost::to_lower(text); - if (text == "bc") - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Era BC exceeds the range of DateTime"); - else if (text != "ad") - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Unknown era {}", text); - } - - ALWAYS_INLINE void setAMPM(String & text) + void setAMPM(String & text) { boost::to_lower(text); if (text == "am") @@ -297,7 +275,7 @@ namespace 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) + void setHour(Int32 hour_, bool is_hour_of_half_day_ = false, bool is_clock_hour_ = false) { Int32 max_hour; Int32 min_hour; @@ -340,6 +318,22 @@ namespace is_clock_hour = is_clock_hour_; } + void setMinute(Int32 minute_) + { + if (minute_ < 0 || minute_ > 59) + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "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::CANNOT_PARSE_DATETIME, "Value {} for second must be in the range [0, 59]", second_); + + second = second_; + } + /// For debug [[maybe_unused]] String toString() const { @@ -676,18 +670,18 @@ namespace } template - static Pos assertChar(Pos cur, Pos end, char ch, [[maybe_unused]] const String & flag) + static Pos assertChar(Pos cur, Pos end, char expected, const String & flag) { if constexpr (need_check_space == NeedCheckSpace::Yes) checkSpace(cur, end, 1, "assertChar requires size >= 1", flag); - if (*cur != ch) + if (*cur != expected) throw Exception( ErrorCodes::CANNOT_PARSE_DATETIME, "Unable to parse flag {} from {} because char {} is expected but {} provided", flag, std::string_view(cur, end - cur), - String(ch, 1), + String(expected, 1), String(*cur, 1)); ++cur; @@ -810,7 +804,7 @@ namespace return cur; } - static Pos mysqlISO8601Year2(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos mysqlISO8601Year2(Pos cur, Pos end, const String & flag, DateTime & date) { Int32 year2; cur = readNumber2(cur, end, flag, year2); @@ -818,7 +812,7 @@ namespace return cur; } - static Pos mysqlISO8601Year4(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos mysqlISO8601Year4(Pos cur, Pos end, const String & flag, DateTime & date) { Int32 year; cur = readNumber4(cur, end, flag, year); @@ -826,7 +820,7 @@ namespace return cur; } - static Pos mysqlDayOfYear(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos mysqlDayOfYear(Pos cur, Pos end, const String & flag, DateTime & date) { Int32 day_of_year; cur = readNumber3(cur, end, flag, day_of_year); @@ -834,7 +828,7 @@ namespace return cur; } - static Pos mysqlDayOfWeek(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos mysqlDayOfWeek(Pos cur, Pos end, const String & flag, DateTime & date) { checkSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1", flag); date.setDayOfWeek(*cur - '0'); @@ -842,7 +836,7 @@ namespace return cur; } - static Pos mysqlISO8601Week(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos mysqlISO8601Week(Pos cur, Pos end, const String & flag, DateTime & date) { Int32 week; cur = readNumber2(cur, end, flag, week); @@ -850,7 +844,7 @@ namespace return cur; } - static Pos mysqlDayOfWeek0To6(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos mysqlDayOfWeek0To6(Pos cur, Pos end, const String & flag, DateTime & date) { checkSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1", flag); @@ -863,9 +857,9 @@ namespace return cur; } - static Pos mysqlDayOfWeekTextLong(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos mysqlDayOfWeekTextLong(Pos cur, Pos end, const String & flag, DateTime & date) { - checkSpace(cur, end, 6, "jodaDayOfWeekText requires the size >= 6", flag); + checkSpace(cur, end, 6, "jodaDayOfWeekText requires size >= 6", flag); String text1(cur, 3); boost::to_lower(text1); auto it = dayOfWeekMap.find(text1); @@ -878,9 +872,9 @@ namespace 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), flag); - String text2(cur, left_size); + size_t expected_remaining_size = it->second.first.size(); + checkSpace(cur, end, expected_remaining_size, "jodaDayOfWeekText requires the second parg size >= " + std::to_string(expected_remaining_size), flag); + String text2(cur, expected_remaining_size); boost::to_lower(text2); if (text2 != it->second.first) throw Exception( @@ -889,13 +883,13 @@ namespace flag, std::string_view(cur, end - cur), text1 + text2); - cur += left_size; + cur += expected_remaining_size; date.setDayOfWeek(it->second.second); return cur; } - static Pos mysqlYear2(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos mysqlYear2(Pos cur, Pos end, const String & flag, DateTime & date) { Int32 year2; cur = readNumber2(cur, end, flag, year2); @@ -903,7 +897,7 @@ namespace return cur; } - static Pos mysqlYear4(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos mysqlYear4(Pos cur, Pos end, const String & flag, DateTime & date) { Int32 year; cur = readNumber4(cur, end, flag, year); @@ -911,7 +905,7 @@ namespace return cur; } - static Pos mysqlTimezoneOffset(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos mysqlTimezoneOffset(Pos cur, Pos end, const String & flag, DateTime & date) { checkSpace(cur, end, 5, "mysqlTimezoneOffset requires size >= 5", flag); @@ -940,7 +934,7 @@ namespace return cur; } - static Pos mysqlMinute(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos mysqlMinute(Pos cur, Pos end, const String & flag, DateTime & date) { Int32 minute; cur = readNumber2(cur, end, flag, minute); @@ -948,7 +942,7 @@ namespace return cur; } - static Pos mysqlAMPM(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos mysqlAMPM(Pos cur, Pos end, const String & flag, DateTime & date) { checkSpace(cur, end, 2, "mysqlAMPM requires size >= 2", flag); @@ -958,7 +952,7 @@ namespace return cur; } - static Pos mysqlHHMM12(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos mysqlHHMM12(Pos cur, Pos end, const String & flag, DateTime & date) { checkSpace(cur, end, 8, "mysqlHHMM12 requires size >= 8", flag); @@ -976,7 +970,7 @@ namespace return cur; } - static Pos mysqlHHMM24(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos mysqlHHMM24(Pos cur, Pos end, const String & flag, DateTime & date) { checkSpace(cur, end, 5, "mysqlHHMM24 requires size >= 5", flag); @@ -991,7 +985,7 @@ namespace return cur; } - static Pos mysqlSecond(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos mysqlSecond(Pos cur, Pos end, const String & flag, DateTime & date) { Int32 second; cur = readNumber2(cur, end, flag, second); @@ -999,7 +993,7 @@ namespace return cur; } - static Pos mysqlISO8601Time(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos mysqlISO8601Time(Pos cur, Pos end, const String & flag, DateTime & date) { checkSpace(cur, end, 8, "mysqlISO8601Time requires size >= 8", flag); @@ -1018,7 +1012,7 @@ namespace return cur; } - static Pos mysqlHour12(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos mysqlHour12(Pos cur, Pos end, const String & flag, DateTime & date) { Int32 hour; cur = readNumber2(cur, end, flag, hour); @@ -1026,7 +1020,7 @@ namespace return cur; } - static Pos mysqlHour24(Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos mysqlHour24(Pos cur, Pos end, const String & flag, DateTime & date) { Int32 hour; cur = readNumber2(cur, end, flag, hour); @@ -1041,7 +1035,7 @@ namespace bool allow_plus_sign, bool is_year, int repetitions, - int max_digits_consume, + int max_digits_to_read, const String & flag, Int32 & number) { @@ -1058,7 +1052,7 @@ namespace } number = 0; - Pos start = cur; + const Pos start = cur; if (is_year && repetitions == 2) { // If abbreviated two year digit is provided in format string, try to read @@ -1068,7 +1062,7 @@ namespace // 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') + while (cur < end && cur < start + max_digits_to_read && *cur >= '0' && *cur <= '9') { number = number * 10 + (*cur - '0'); ++cur; @@ -1083,7 +1077,7 @@ namespace } else { - while (cur < end && cur < start + max_digits_consume && *cur >= '0' && *cur <= '9') + while (cur < end && cur < start + max_digits_to_read && *cur >= '0' && *cur <= '9') { number = number * 10 + (*cur - '0'); ++cur; @@ -1092,7 +1086,7 @@ namespace } else { - while (cur < end && cur < start + max_digits_consume && *cur >= '0' and *cur <= '9') + while (cur < end && cur < start + max_digits_to_read && *cur >= '0' && *cur <= '9') { number = number * 10 + (*cur - '0'); ++cur; @@ -1100,7 +1094,7 @@ namespace } /// Need to have read at least one digit. - if (cur <= start) + if (cur == start) throw Exception( ErrorCodes::CANNOT_PARSE_DATETIME, "Unable to parse flag {} from {} because read number failed", @@ -1113,7 +1107,7 @@ namespace return cur; } - static Pos jodaEra(int, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos jodaEra(int, Pos cur, Pos end, const String & flag, DateTime & date) { checkSpace(cur, end, 2, "jodaEra requires size >= 2", flag); @@ -1123,7 +1117,7 @@ namespace return cur; } - static Pos jodaCenturyOfEra(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos jodaCenturyOfEra(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) { Int32 century; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, repetitions, flag, century); @@ -1131,7 +1125,7 @@ namespace return cur; } - static Pos jodaYearOfEra(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos jodaYearOfEra(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) { Int32 year_of_era; cur = readNumberWithVariableLength(cur, end, false, false, true, repetitions, repetitions, flag, year_of_era); @@ -1139,7 +1133,7 @@ namespace return cur; } - static Pos jodaWeekYear(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos jodaWeekYear(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) { Int32 week_year; cur = readNumberWithVariableLength(cur, end, true, true, true, repetitions, repetitions, flag, week_year); @@ -1147,7 +1141,7 @@ namespace return cur; } - static Pos jodaWeekOfWeekYear(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos jodaWeekOfWeekYear(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) { Int32 week; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, week); @@ -1155,7 +1149,7 @@ namespace return cur; } - static Pos jodaDayOfWeek1Based(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos jodaDayOfWeek1Based(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) { Int32 day_of_week; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, repetitions, flag, day_of_week); @@ -1164,7 +1158,7 @@ namespace } static Pos - jodaDayOfWeekText(size_t /*min_represent_digits*/, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + jodaDayOfWeekText(size_t /*min_represent_digits*/, Pos cur, Pos end, const String & flag, DateTime & date) { checkSpace(cur, end, 3, "jodaDayOfWeekText requires size >= 3", flag); @@ -1181,21 +1175,21 @@ namespace cur += 3; date.setDayOfWeek(it->second.second); - size_t left_size = it->second.first.size(); - if (cur + left_size <= end) + size_t expected_remaining_size = it->second.first.size(); + if (cur + expected_remaining_size <= end) { - String text2(cur, left_size); + String text2(cur, expected_remaining_size); boost::to_lower(text2); if (text2 == it->second.first) { - cur += left_size; + cur += expected_remaining_size; return cur; } } return cur; } - static Pos jodaYear(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos jodaYear(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) { Int32 year; cur = readNumberWithVariableLength(cur, end, true, true, true, repetitions, repetitions, flag, year); @@ -1203,7 +1197,7 @@ namespace return cur; } - static Pos jodaDayOfYear(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos jodaDayOfYear(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) { Int32 day_of_year; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 3), flag, day_of_year); @@ -1211,7 +1205,7 @@ namespace return cur; } - static Pos jodaMonthOfYear(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos jodaMonthOfYear(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) { Int32 month; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, 2, flag, month); @@ -1219,7 +1213,7 @@ namespace return cur; } - static Pos jodaMonthOfYearText(int, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos jodaMonthOfYearText(int, Pos cur, Pos end, const String & flag, DateTime & date) { checkSpace(cur, end, 3, "jodaMonthOfYearText requires size >= 3", flag); String text1(cur, 3); @@ -1235,21 +1229,21 @@ namespace cur += 3; date.setMonth(it->second.second); - size_t left_size = it->second.first.size(); - if (cur + left_size <= end) + size_t expected_remaining_size = it->second.first.size(); + if (cur + expected_remaining_size <= end) { - String text2(cur, left_size); + String text2(cur, expected_remaining_size); boost::to_lower(text2); if (text2 == it->second.first) { - cur += left_size; + cur += expected_remaining_size; return cur; } } return cur; } - static Pos jodaDayOfMonth(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos jodaDayOfMonth(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) { Int32 day_of_month; cur = readNumberWithVariableLength( @@ -1258,7 +1252,7 @@ namespace return cur; } - static Pos jodaHalfDayOfDay(int, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos jodaHalfDayOfDay(int, Pos cur, Pos end, const String & flag, DateTime & date) { checkSpace(cur, end, 2, "jodaHalfDayOfDay requires size >= 2", flag); @@ -1268,7 +1262,7 @@ namespace return cur; } - static Pos jodaHourOfHalfDay(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos jodaHourOfHalfDay(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) { Int32 hour; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, hour); @@ -1276,7 +1270,7 @@ namespace return cur; } - static Pos jodaClockHourOfHalfDay(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos jodaClockHourOfHalfDay(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) { Int32 hour; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, hour); @@ -1284,7 +1278,7 @@ namespace return cur; } - static Pos jodaHourOfDay(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos jodaHourOfDay(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) { Int32 hour; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, hour); @@ -1292,7 +1286,7 @@ namespace return cur; } - static Pos jodaClockHourOfDay(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos jodaClockHourOfDay(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) { Int32 hour; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, hour); @@ -1300,7 +1294,7 @@ namespace return cur; } - static Pos jodaMinuteOfHour(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos jodaMinuteOfHour(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) { Int32 minute; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, minute); @@ -1308,7 +1302,7 @@ namespace return cur; } - static Pos jodaSecondOfMinute(int repetitions, Pos cur, Pos end, [[maybe_unused]] const String & flag, DateTime & date) + static Pos jodaSecondOfMinute(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) { Int32 second; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, second); @@ -1387,9 +1381,6 @@ namespace instructions.emplace_back(ACTION_ARGS(Instruction::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': @@ -1533,10 +1524,14 @@ namespace // Escaped literal characters. case '%': - instructions.emplace_back("\n"); + instructions.emplace_back("%"); break; - // Unimplemented + /// Unimplemented + + /// Fractional seconds + case 'f': + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for fractional seconds"); case 'U': throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for WEEK (Sun-Sat)"); case 'v': @@ -1545,7 +1540,6 @@ namespace 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, @@ -1572,13 +1566,13 @@ namespace { #define ACTION_ARGS_WITH_BIND(func, arg) std::bind_front(&(func), (arg)), #func, std::string_view(cur_token, repetitions) - const char * pos = format.data(); - const char * end = format.data() + format.size(); + Pos pos = format.data(); + Pos end = format.data() + format.size(); std::vector instructions; while (pos < end) { - const char * cur_token = pos; + Pos cur_token = pos; // Literal case if (*cur_token == '\'') @@ -1677,10 +1671,8 @@ namespace 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: From bb7a8ed862d6fd9c43f841ae9f6ee339177cba77 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 17 Mar 2023 15:09:38 +0000 Subject: [PATCH 180/418] Apply log_queries_cut_to_length in MergeTreeWhereOptimizer --- src/Common/SensitiveDataMasker.cpp | 7 ++++++- src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp | 3 ++- src/Storages/MergeTree/MergeTreeWhereOptimizer.h | 1 + 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Common/SensitiveDataMasker.cpp b/src/Common/SensitiveDataMasker.cpp index 5336b48524f..a5de2baca02 100644 --- a/src/Common/SensitiveDataMasker.cpp +++ b/src/Common/SensitiveDataMasker.cpp @@ -202,8 +202,13 @@ std::string wipeSensitiveDataAndCutToLength(const std::string & str, size_t max_ if (auto * masker = SensitiveDataMasker::getInstance()) masker->wipeSensitiveData(res); - if (max_length && (res.length() > max_length)) + size_t length = res.length(); + if (max_length && (length > max_length)) + { + size_t truncated_len = length - max_length; res.resize(max_length); + res += "... (truncated " + std::to_string(truncated_len) + " chars)"; + } return res; } diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index fdddc29048b..c7afe9319f9 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -44,6 +44,7 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( , log{log_} , column_sizes{std::move(column_sizes_)} , move_all_conditions_to_prewhere(context->getSettingsRef().move_all_conditions_to_prewhere) + , log_queries_cut_to_length(context->getSettingsRef().log_queries_cut_to_length) { for (const auto & name : queried_columns) { @@ -310,7 +311,7 @@ void MergeTreeWhereOptimizer::optimize(ASTSelectQuery & select) const 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()); + LOG_DEBUG(log, "MergeTreeWhereOptimizer: condition \"{}\" moved to PREWHERE", select.prewhere()->formatForLogging(log_queries_cut_to_length)); } diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index 8953923542e..ef87018f93e 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -115,6 +115,7 @@ private: UInt64 total_size_of_queried_columns = 0; NameSet array_joined_names; const bool move_all_conditions_to_prewhere = false; + UInt64 log_queries_cut_to_length = 0; }; From 34b247900ff93e24d979fdfa006ce59011edb06e Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 20 Mar 2023 16:53:54 +0100 Subject: [PATCH 181/418] Better test --- .../test_named_collections/test.py | 40 +++++++++++++++---- 1 file changed, 33 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index af5aab38264..5f5657fad54 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -199,23 +199,40 @@ def test_granular_access_show_query(cluster): assert 0 == int( node.query("SELECT count() FROM system.named_collections", user="koko") ) + assert "GRANT SELECT ON *.* TO koko" == node.query("SHOW GRANTS FOR koko;").strip() node.query("GRANT show named collections ON * TO koko") + assert ( + "GRANT SELECT ON *.* TO koko\nGRANT SHOW NAMED COLLECTIONS ON * TO koko" + == node.query("SHOW GRANTS FOR koko;").strip() + ) assert ( "collection1\ncollection2" == node.query("select name from system.named_collections", user="koko").strip() ) node.restart_clickhouse() + assert ( + "GRANT SELECT ON *.* TO koko\nGRANT SHOW NAMED COLLECTIONS ON * TO koko" + == node.query("SHOW GRANTS FOR koko;").strip() + ) assert ( "collection1\ncollection2" == node.query("select name from system.named_collections", user="koko").strip() ) node.query("REVOKE show named collections ON collection1 FROM koko;") + assert ( + "GRANT SELECT ON *.* TO koko\nGRANT SHOW NAMED COLLECTIONS ON * TO koko\nREVOKE SHOW NAMED COLLECTIONS ON collection1 FROM koko" + == node.query("SHOW GRANTS FOR koko;").strip() + ) assert ( "collection2" == node.query("select name from system.named_collections", user="koko").strip() ) node.restart_clickhouse() + assert ( + "GRANT SELECT ON *.* TO koko\nGRANT SHOW NAMED COLLECTIONS ON * TO koko\nREVOKE SHOW NAMED COLLECTIONS ON collection1 FROM koko" + == node.query("SHOW GRANTS FOR koko;").strip() + ) assert ( "collection2" == node.query("select name from system.named_collections", user="koko").strip() @@ -224,17 +241,26 @@ def test_granular_access_show_query(cluster): assert ( "" == node.query("select * from system.named_collections", user="koko").strip() ) + assert ( + "GRANT SELECT ON *.* TO koko\nGRANT SHOW NAMED COLLECTIONS ON * TO koko\nREVOKE SHOW NAMED COLLECTIONS ON collection1 FROM koko\nREVOKE SHOW NAMED COLLECTIONS ON collection2 FROM koko" + == node.query("SHOW GRANTS FOR 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 ( + "GRANT SELECT ON *.* TO koko\nGRANT SHOW NAMED COLLECTIONS ON * TO koko\nREVOKE SHOW NAMED COLLECTIONS ON collection1 FROM koko" + == node.query("SHOW GRANTS FOR koko;").strip() + ) assert ( "collection2" == node.query("select name from system.named_collections", user="koko").strip() ) node.query("REVOKE show named collections ON * FROM koko;") + assert "GRANT SELECT ON *.* TO koko" == node.query("SHOW GRANTS FOR koko;").strip() assert ( "" == node.query("select * from system.named_collections", user="koko").strip() ) @@ -250,7 +276,7 @@ def test_show_grants(cluster): 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("SHOW GRANTS FOR koko;").strip() ) node.query("DROP USER IF EXISTS koko") @@ -259,7 +285,7 @@ def test_show_grants(cluster): 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("SHOW GRANTS FOR koko;").strip() ) node.query("DROP USER IF EXISTS koko") @@ -268,7 +294,7 @@ def test_show_grants(cluster): 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("SHOW GRANTS FOR koko;").strip() ) node.query("DROP USER IF EXISTS koko") @@ -277,7 +303,7 @@ def test_show_grants(cluster): 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("SHOW GRANTS FOR koko;").strip() ) node.query("DROP USER IF EXISTS koko") @@ -286,7 +312,7 @@ def test_show_grants(cluster): 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() + == node.query("SHOW GRANTS FOR koko;").strip() ) node.query("DROP USER IF EXISTS koko") @@ -295,7 +321,7 @@ def test_show_grants(cluster): 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("SHOW GRANTS FOR koko;").strip() ) node.query("DROP USER IF EXISTS koko") @@ -304,7 +330,7 @@ def test_show_grants(cluster): node.query("GRANT CREATE NAMED COLLECTION 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("SHOW GRANTS FOR koko;").strip() ) From 8c48714236394b1069e4ae743b2d79148733ad77 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 20 Mar 2023 15:54:19 +0000 Subject: [PATCH 182/418] wipeSensitiveDataAndCutToLength take into accound suffix length --- src/Common/SensitiveDataMasker.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Common/SensitiveDataMasker.cpp b/src/Common/SensitiveDataMasker.cpp index a5de2baca02..34db78d00fb 100644 --- a/src/Common/SensitiveDataMasker.cpp +++ b/src/Common/SensitiveDataMasker.cpp @@ -205,9 +205,12 @@ std::string wipeSensitiveDataAndCutToLength(const std::string & str, size_t max_ size_t length = res.length(); if (max_length && (length > max_length)) { - size_t truncated_len = length - max_length; + constexpr size_t max_extra_msg_len = sizeof("... (truncated 18446744073709551615 characters)"); + if (max_length < max_extra_msg_len) + return "(removed " + std::to_string(length) + " characters)"; + max_length -= max_extra_msg_len; res.resize(max_length); - res += "... (truncated " + std::to_string(truncated_len) + " chars)"; + res.append("... (truncated " + std::to_string(length - max_length) + " characters)"); } return res; From 3ca789b4a21456d5cf517c294b35cb65aab64b0e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 20 Mar 2023 23:58:37 +0800 Subject: [PATCH 183/418] changes as requested --- .../functions/date-time-functions.md | 2 +- .../functions/type-conversion-functions.md | 2 +- src/Functions/formatDateTime.cpp | 2 +- src/Functions/parseDateTime.cpp | 73 ++++++++----------- 4 files changed, 35 insertions(+), 44 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 4a2820e6d89..0657f96e8fd 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1286,7 +1286,7 @@ Using replacement fields, you can define a pattern for the resulting string. “ | %T | ISO 8601 time format (HH:MM:SS), equivalent to %H:%M:%S | 22:33:44 | | %u | ISO 8601 weekday as number with Monday as 1 (1-7) | 2 | | %V | ISO 8601 week number (01-53) | 01 | -| %w | weekday as a decimal number with Sunday as 0 (0-6) | 2 | +| %w | weekday as a integer number with Sunday as 0 (0-6) | 2 | | %W | full weekday name (Monday-Sunday) | Monday | | %y | Year, last two digits (00-99) | 18 | | %Y | Year | 2018 | diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index acb094726a2..5ece8bacdac 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1354,7 +1354,7 @@ Same as for [parseDateTime64BestEffort](#parsedatetime64besteffort), except that ## 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`. +Alias: `TO_TIMESTAMP`. **Syntax** diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index 3e98c5d35c2..dbcc88eb17c 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -1007,7 +1007,7 @@ public: out_template += "00"; break; - // Weekday as a decimal number with Sunday as 0 (0-6) 4 + // Weekday as a integer number with Sunday as 0 (0-6) 4 case 'w': instructions.emplace_back(&Instruction::mysqlDayOfWeek0To6); out_template += "0"; diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index d43ffb376de..e349183d832 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -144,9 +144,9 @@ namespace time_zone_offset = 0; } - void setEra(String & text) // NOLINT + /// Input text is expected to be lowered by caller + void setEra(const String & text) // NOLINT { - boost::to_lower(text); if (text == "bc") throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Era BC exceeds the range of DateTime"); else if (text != "ad") @@ -177,7 +177,7 @@ namespace } } - void setYear2(Int32 year_, bool is_year_of_era_ = false, bool is_week_year = false) + void setYear2(Int32 year_) { if (year_ >= 70 && year_ < 100) year_ += 1900; @@ -186,7 +186,7 @@ namespace else 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); + setYear(year_, false, false); } void setMonth(Int32 month_) @@ -264,9 +264,9 @@ namespace } } - void setAMPM(String & text) + /// Input text is expected to be lowered by caller + void setAMPM(const String & text) { - boost::to_lower(text); if (text == "am") is_am = true; else if (text == "pm") @@ -446,18 +446,14 @@ namespace } }; - struct ParseDateTimeTraits + enum class ParseSyntax { - enum class ParseSyntax - { - MySQL, - Joda - }; + MySQL, + Joda }; - /// _FUNC_(str[, format, timezone]) - template + template class FunctionParseDateTimeImpl : public IFunction { public: @@ -485,21 +481,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 {}. 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 {}. 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 {}. Should be String", arguments[0].type->getName(), getName()); @@ -511,12 +507,8 @@ namespace executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, 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()); + /// It was checked before in getReturnTypeImpl + assert(!col_str); String format = getFormat(arguments); const auto & time_zone = getTimeZone(arguments); @@ -567,7 +559,7 @@ namespace }; using Func = std::conditional_t< - parse_syntax == ParseDateTimeTraits::ParseSyntax::MySQL, + parse_syntax == ParseSyntax::MySQL, Pos (*)(Pos, Pos, const String &, DateTime &), std::function>; Func func{}; @@ -947,6 +939,7 @@ namespace checkSpace(cur, end, 2, "mysqlAMPM requires size >= 2", flag); String text(cur, 2); + boost::to_lower(text); date.setAMPM(text); cur += 2; return cur; @@ -1112,6 +1105,7 @@ namespace checkSpace(cur, end, 2, "jodaEra requires size >= 2", flag); String era(cur, 2); + boost::to_lower(era); date.setEra(era); cur += 2; return cur; @@ -1257,6 +1251,7 @@ namespace checkSpace(cur, end, 2, "jodaHalfDayOfDay requires size >= 2", flag); String text(cur, 2); + boost::to_lower(text); date.setAMPM(text); cur += 2; return cur; @@ -1314,10 +1309,10 @@ namespace std::vector parseFormat(const String & format) const { static_assert( - parse_syntax == ParseDateTimeTraits::ParseSyntax::MySQL || parse_syntax == ParseDateTimeTraits::ParseSyntax::Joda, + parse_syntax == ParseSyntax::MySQL || parse_syntax == ParseSyntax::Joda, "parse syntax must be one of MySQL or Joda"); - if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::MySQL) + if constexpr (parse_syntax == ParseSyntax::MySQL) return parseMysqlFormat(format); else return parseJodaFormat(format); @@ -1342,7 +1337,8 @@ namespace pos = next_percent_pos + 1; if (pos >= end) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Sign '%' is the last in format, if you need it, use '%%'"); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "'%' must not be the last character in the format string, use '%%' instead"); switch (*pos) { @@ -1417,7 +1413,7 @@ namespace instructions.emplace_back(ACTION_ARGS(Instruction::mysqlISO8601Week)); break; - // Weekday as a decimal number with Sunday as 0 (0-6) 4 + // Weekday as a integer number with Sunday as 0 (0-6) 4 case 'w': instructions.emplace_back(ACTION_ARGS(Instruction::mysqlDayOfWeek0To6)); break; @@ -1553,6 +1549,7 @@ namespace } else { + /// Handle characters after last % if (pos < end) instructions.emplace_back(String(pos, end - pos)); break; @@ -1694,19 +1691,15 @@ namespace { if (arguments.size() < 2) { - if constexpr (parse_syntax == ParseDateTimeTraits::ParseSyntax::Joda) + if constexpr (parse_syntax == 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()); + /// It was checked before in getReturnTypeImpl + assert(!format_column); return format_column->getValue(); } @@ -1719,13 +1712,11 @@ namespace if (!col) throw Exception( ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of third ('timezone') argument of function {}. Must be constant string.", + "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); } }; @@ -1741,15 +1732,15 @@ namespace }; - using FunctionParseDateTime = FunctionParseDateTimeImpl; + using FunctionParseDateTime = FunctionParseDateTimeImpl; using FunctionParseDateTimeInJodaSyntax - = FunctionParseDateTimeImpl; + = FunctionParseDateTimeImpl; } REGISTER_FUNCTION(ParseDateTime) { factory.registerFunction(); - factory.registerAlias("TO_UNIX_TIMESTAMP", "parseDateTime"); + factory.registerAlias("TO_UNIXTIME", FunctionParseDateTime::name); factory.registerFunction(); } From 96413e528c2441f123dd72b8ba356e68f056e049 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 21 Mar 2023 00:24:06 +0800 Subject: [PATCH 184/418] commit again --- src/Functions/parseDateTime.cpp | 339 ++++++++++++++++---------------- 1 file changed, 169 insertions(+), 170 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index e349183d832..137a91584d0 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -113,7 +113,7 @@ namespace bool is_clock_hour = false; /// Whether the hour is clockhour 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. + bool has_time_zone_offset = false; /// If true, time zone offset is explicitly specified. Int64 time_zone_offset = 0; /// Offset in seconds between current timezone to UTC. void reset() @@ -384,7 +384,7 @@ namespace } } - static ALWAYS_INLINE Int32 daysSinceEpochFromWeekDate(int32_t week_year_, int32_t week_of_year_, int32_t day_of_week_) + static Int32 daysSinceEpochFromWeekDate(int32_t week_year_, int32_t week_of_year_, int32_t 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) @@ -395,7 +395,7 @@ namespace return days_since_epoch_of_jan_fourth - (first_day_of_week_year - 1) + 7 * (week_of_year_ - 1) + day_of_week_ - 1; } - static ALWAYS_INLINE Int32 daysSinceEpochFromDayOfYear(Int32 year_, Int32 day_of_year_) + static Int32 daysSinceEpochFromDayOfYear(Int32 year_, Int32 day_of_year_) { if (!isDayOfYearValid(year_, day_of_year_)) throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Invalid day of year, year:{} day of year:{}", year_, day_of_year_); @@ -405,7 +405,7 @@ namespace return res; } - static ALWAYS_INLINE Int32 daysSinceEpochFromDate(Int32 year_, Int32 month_, Int32 day_) + static Int32 daysSinceEpochFromDate(Int32 year_, Int32 month_, Int32 day_) { if (!isDateValid(year_, month_, day_)) throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Invalid date, year:{} month:{} day:{}", year_, month_, day_); @@ -416,7 +416,7 @@ namespace return res; } - Int64 checkAndGetDateTime(const DateLUTImpl & time_zone) + Int64 buildDateTime(const DateLUTImpl & time_zone) { if (is_hour_of_half_day && !is_am) hour += 12; @@ -516,7 +516,7 @@ namespace auto col_res = ColumnDateTime::create(); col_res->reserve(input_rows_count); - auto & data_res = col_res->getData(); + auto & res_data = col_res->getData(); /// Make datetime fit in a cache line. alignas(64) DateTime datetime; @@ -540,8 +540,8 @@ namespace str_ref.toView(), std::string_view(cur, end - cur)); - Int64 time = datetime.checkAndGetDateTime(time_zone); - data_res.push_back(static_cast(time)); + Int64 time = datetime.buildDateTime(time_zone); + res_data.push_back(static_cast(time)); } return col_res; @@ -562,41 +562,42 @@ namespace parse_syntax == ParseSyntax::MySQL, Pos (*)(Pos, Pos, const String &, DateTime &), std::function>; - Func func{}; - std::string func_name; - std::string literal; /// Only used when current instruction parses literal - std::string flag; /// Parsed flags in MySQL or Joda format string + const Func func{}; + const String func_name; + const String literal; /// Only used when current instruction parses literal + const String fragment; /// Parsed fragments in MySQL or Joda format string public: - 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(Func && func_, const char * func_name_, const std::string_view & fragment_) + : func(std::move(func_)), func_name(func_name_), fragment(fragment_) { } - explicit Instruction(const String & literal_) : literal(literal_), flag("LITERAL") { } - explicit Instruction(String && literal_) : literal(std::move(literal_)), flag("LITERAL") { } + explicit Instruction(const String & literal_) : literal(literal_), fragment("LITERAL") { } + explicit Instruction(String && literal_) : literal(std::move(literal_)), fragment("LITERAL") { } /// For debug [[maybe_unused]] String toString() const { if (func) - return "func:" + func_name + ",flag:" + flag; + return "func:" + func_name + ",fragment:" + fragment; else - return "literal:" + literal + ",flag:" + flag; + return "literal:" + literal + ",fragment:" + fragment; } Pos perform(Pos cur, Pos end, DateTime & date) const { if (func) - return func(cur, end, flag, date); + return func(cur, end, fragment, date); else { - checkSpace(cur, end, literal.size(), "required literal size not matched", flag); + /// literal: + checkSpace(cur, end, literal.size(), "insufficient space to parse literal", fragment); if (std::string_view(cur, literal.size()) != literal) throw Exception( ErrorCodes::CANNOT_PARSE_DATETIME, - "Unable to parse flag {} from {} because literal {} is expected but {} provided", - flag, + "Unable to parse fragment {} from {} because literal {} is expected but {} provided", + fragment, std::string_view(cur, end - cur), literal, std::string_view(cur, literal.size())); @@ -606,10 +607,10 @@ namespace } template - static Pos readNumber2(Pos cur, Pos end, [[maybe_unused]] const String & flag, T & res) + static Pos readNumber2(Pos cur, Pos end, [[maybe_unused]] const String & fragment, T & res) { if constexpr (need_check_space == NeedCheckSpace::Yes) - checkSpace(cur, end, 2, "readNumber2 requires size >= 2", flag); + checkSpace(cur, end, 2, "readNumber2 requires size >= 2", fragment); res = (*cur - '0'); ++cur; @@ -619,10 +620,10 @@ namespace } template - static Pos readNumber3(Pos cur, Pos end, [[maybe_unused]] const String & flag, T & res) + static Pos readNumber3(Pos cur, Pos end, [[maybe_unused]] const String & fragment, T & res) { if constexpr (need_check_space == NeedCheckSpace::Yes) - checkSpace(cur, end, 3, "readNumber4 requires size >= 3", flag); + checkSpace(cur, end, 3, "readNumber4 requires size >= 3", fragment); res = (*cur - '0'); ++cur; @@ -634,10 +635,10 @@ namespace } template - static Pos readNumber4(Pos cur, Pos end, [[maybe_unused]] const String & flag, T & res) + static Pos readNumber4(Pos cur, Pos end, [[maybe_unused]] const String & fragment, T & res) { if constexpr (need_check_space == NeedCheckSpace::Yes) - checkSpace(cur, end, 4, "readNumber4 requires size >= 4", flag); + checkSpace(cur, end, 4, "readNumber4 requires size >= 4", fragment); res = (*cur - '0'); ++cur; @@ -650,28 +651,28 @@ namespace return cur; } - static void checkSpace(Pos cur, Pos end, size_t len, const String & msg, const String & flag) + static void checkSpace(Pos cur, Pos end, size_t len, const String & msg, const String & fragment) { if (cur > end || cur + len > end) [[unlikely]] throw Exception( ErrorCodes::NOT_ENOUGH_SPACE, - "Unable to parse flag {} from {} because {}", - flag, + "Unable to parse fragment {} from {} because {}", + fragment, std::string_view(cur, end - cur), msg); } template - static Pos assertChar(Pos cur, Pos end, char expected, const String & flag) + static Pos assertChar(Pos cur, Pos end, char expected, const String & fragment) { if constexpr (need_check_space == NeedCheckSpace::Yes) - checkSpace(cur, end, 1, "assertChar requires size >= 1", flag); + checkSpace(cur, end, 1, "assertChar requires size >= 1", fragment); if (*cur != expected) throw Exception( ErrorCodes::CANNOT_PARSE_DATETIME, - "Unable to parse flag {} from {} because char {} is expected but {} provided", - flag, + "Unable to parse fragment {} from {} because char {} is expected but {} provided", + fragment, std::string_view(cur, end - cur), String(expected, 1), String(*cur, 1)); @@ -680,9 +681,9 @@ namespace return cur; } - static Pos mysqlDayOfWeekTextShort(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlDayOfWeekTextShort(Pos cur, Pos end, const String & fragment, DateTime & date) { - checkSpace(cur, end, 3, "mysqlDayOfWeekTextShort requires size >= 3", flag); + checkSpace(cur, end, 3, "mysqlDayOfWeekTextShort requires size >= 3", fragment); String text(cur, 3); boost::to_lower(text); @@ -690,8 +691,8 @@ namespace if (it == dayOfWeekMap.end()) throw Exception( ErrorCodes::CANNOT_PARSE_DATETIME, - "Unable to parse flag {} from {} because of unknown day of week short text {} ", - flag, + "Unable to parse fragment {} from {} because of unknown day of week short text {} ", + fragment, std::string_view(cur, end - cur), text); date.setDayOfWeek(it->second.second); @@ -699,9 +700,9 @@ namespace return cur; } - static Pos mysqlMonthOfYearTextShort(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlMonthOfYearTextShort(Pos cur, Pos end, const String & fragment, DateTime & date) { - checkSpace(cur, end, 3, "mysqlMonthOfYearTextShort requires size >= 3", flag); + checkSpace(cur, end, 3, "mysqlMonthOfYearTextShort requires size >= 3", fragment); String text(cur, 3); boost::to_lower(text); @@ -709,8 +710,8 @@ namespace if (it == monthMap.end()) throw Exception( ErrorCodes::CANNOT_PARSE_DATETIME, - "Unable to parse flag {} from {} because of unknown month of year short text {}", - flag, + "Unable to parse fragment {} from {} because of unknown month of year short text {}", + fragment, std::string_view(cur, end - cur), text); @@ -719,53 +720,53 @@ namespace return cur; } - static Pos mysqlMonth(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlMonth(Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 month; - cur = readNumber2(cur, end, flag, month); + cur = readNumber2(cur, end, fragment, month); date.setMonth(month); return cur; } - static Pos mysqlCentury(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlCentury(Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 century; - cur = readNumber2(cur, end, flag, century); + cur = readNumber2(cur, end, fragment, century); date.setCentury(century); return cur; } - static Pos mysqlDayOfMonth(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlDayOfMonth(Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 day_of_month; - cur = readNumber2(cur, end, flag, day_of_month); + cur = readNumber2(cur, end, fragment, day_of_month); date.setDayOfMonth(day_of_month); return cur; } - static Pos mysqlAmericanDate(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlAmericanDate(Pos cur, Pos end, const String & fragment, DateTime & date) { - checkSpace(cur, end, 8, "mysqlAmericanDate requires size >= 8", flag); + checkSpace(cur, end, 8, "mysqlAmericanDate requires size >= 8", fragment); Int32 month; - cur = readNumber2(cur, end, flag, month); - cur = assertChar(cur, end, '/', flag); + cur = readNumber2(cur, end, fragment, month); + cur = assertChar(cur, end, '/', fragment); date.setMonth(month); Int32 day; - cur = readNumber2(cur, end, flag, day); - cur = assertChar(cur, end, '/', flag); + cur = readNumber2(cur, end, fragment, day); + cur = assertChar(cur, end, '/', fragment); date.setDayOfMonth(day); Int32 year; - cur = readNumber2(cur, end, flag, year); + cur = readNumber2(cur, end, fragment, year); date.setYear(year); return cur; } - static Pos mysqlDayOfMonthSpacePadded(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlDayOfMonthSpacePadded(Pos cur, Pos end, const String & fragment, DateTime & date) { - checkSpace(cur, end, 2, "mysqlDayOfMonthSpacePadded requires size >= 2", flag); + checkSpace(cur, end, 2, "mysqlDayOfMonthSpacePadded requires size >= 2", fragment); Int32 day_of_month = *cur == ' ' ? 0 : (*cur - '0'); ++cur; @@ -777,18 +778,18 @@ namespace return cur; } - static Pos mysqlISO8601Date(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlISO8601Date(Pos cur, Pos end, const String & fragment, DateTime & date) { - checkSpace(cur, end, 10, "mysqlISO8601Date requires size >= 10", flag); + checkSpace(cur, end, 10, "mysqlISO8601Date requires size >= 10", fragment); Int32 year; Int32 month; Int32 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); + cur = readNumber4(cur, end, fragment, year); + cur = assertChar(cur, end, '-', fragment); + cur = readNumber2(cur, end, fragment, month); + cur = assertChar(cur, end, '-', fragment); + cur = readNumber2(cur, end, fragment, day); date.setYear(year); date.setMonth(month); @@ -796,49 +797,49 @@ namespace return cur; } - static Pos mysqlISO8601Year2(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlISO8601Year2(Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 year2; - cur = readNumber2(cur, end, flag, year2); + cur = readNumber2(cur, end, fragment, year2); date.setYear2(year2); return cur; } - static Pos mysqlISO8601Year4(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlISO8601Year4(Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 year; - cur = readNumber4(cur, end, flag, year); + cur = readNumber4(cur, end, fragment, year); date.setYear(year); return cur; } - static Pos mysqlDayOfYear(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlDayOfYear(Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 day_of_year; - cur = readNumber3(cur, end, flag, day_of_year); + cur = readNumber3(cur, end, fragment, day_of_year); date.setDayOfYear(day_of_year); return cur; } - static Pos mysqlDayOfWeek(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlDayOfWeek(Pos cur, Pos end, const String & fragment, DateTime & date) { - checkSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1", flag); + checkSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1", fragment); date.setDayOfWeek(*cur - '0'); ++cur; return cur; } - static Pos mysqlISO8601Week(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlISO8601Week(Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 week; - cur = readNumber2(cur, end, flag, week); + cur = readNumber2(cur, end, fragment, week); date.setWeek(week); return cur; } - static Pos mysqlDayOfWeek0To6(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlDayOfWeek0To6(Pos cur, Pos end, const String & fragment, DateTime & date) { - checkSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1", flag); + checkSpace(cur, end, 1, "mysqlDayOfWeek requires size >= 1", fragment); Int32 day_of_week = *cur - '0'; if (day_of_week == 0) @@ -849,30 +850,30 @@ namespace return cur; } - static Pos mysqlDayOfWeekTextLong(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlDayOfWeekTextLong(Pos cur, Pos end, const String & fragment, DateTime & date) { - checkSpace(cur, end, 6, "jodaDayOfWeekText requires size >= 6", flag); + checkSpace(cur, end, 6, "jodaDayOfWeekText requires size >= 6", fragment); String text1(cur, 3); boost::to_lower(text1); auto it = dayOfWeekMap.find(text1); if (it == dayOfWeekMap.end()) throw Exception( ErrorCodes::CANNOT_PARSE_DATETIME, - "Unable to parse first part of flag {} from {} because of unknown day of week text: {}", - flag, + "Unable to parse first part of fragment {} from {} because of unknown day of week text: {}", + fragment, std::string_view(cur, end - cur), text1); cur += 3; size_t expected_remaining_size = it->second.first.size(); - checkSpace(cur, end, expected_remaining_size, "jodaDayOfWeekText requires the second parg size >= " + std::to_string(expected_remaining_size), flag); + checkSpace(cur, end, expected_remaining_size, "jodaDayOfWeekText requires the second parg size >= " + std::to_string(expected_remaining_size), fragment); String text2(cur, expected_remaining_size); boost::to_lower(text2); if (text2 != it->second.first) throw Exception( ErrorCodes::CANNOT_PARSE_DATETIME, - "Unable to parse second part of flag {} from {} because of unknown day of week text: {}", - flag, + "Unable to parse second part of fragment {} from {} because of unknown day of week text: {}", + fragment, std::string_view(cur, end - cur), text1 + text2); cur += expected_remaining_size; @@ -881,25 +882,25 @@ namespace return cur; } - static Pos mysqlYear2(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlYear2(Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 year2; - cur = readNumber2(cur, end, flag, year2); + cur = readNumber2(cur, end, fragment, year2); date.setYear2(year2); return cur; } - static Pos mysqlYear4(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlYear4(Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 year; - cur = readNumber4(cur, end, flag, year); + cur = readNumber4(cur, end, fragment, year); date.setYear(year); return cur; } - static Pos mysqlTimezoneOffset(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlTimezoneOffset(Pos cur, Pos end, const String & fragment, DateTime & date) { - checkSpace(cur, end, 5, "mysqlTimezoneOffset requires size >= 5", flag); + checkSpace(cur, end, 5, "mysqlTimezoneOffset requires size >= 5", fragment); Int32 sign; if (*cur == '-') @@ -909,34 +910,34 @@ namespace else throw Exception( ErrorCodes::CANNOT_PARSE_DATETIME, - "Unable to parse flag {} from {} because of unknown sign time zone offset: {}", - flag, + "Unable to parse fragment {} from {} because of unknown sign time zone offset: {}", + fragment, std::string_view(cur, end - cur), std::string_view(cur, 1)); ++cur; Int32 hour; - cur = readNumber2(cur, end, flag, hour); + cur = readNumber2(cur, end, fragment, hour); Int32 minute; - cur = readNumber2(cur, end, flag, minute); + cur = readNumber2(cur, end, fragment, minute); date.has_time_zone_offset = true; date.time_zone_offset = sign * (hour * 3600 + minute * 60); return cur; } - static Pos mysqlMinute(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlMinute(Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 minute; - cur = readNumber2(cur, end, flag, minute); + cur = readNumber2(cur, end, fragment, minute); date.setMinute(minute); return cur; } - static Pos mysqlAMPM(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlAMPM(Pos cur, Pos end, const String & fragment, DateTime & date) { - checkSpace(cur, end, 2, "mysqlAMPM requires size >= 2", flag); + checkSpace(cur, end, 2, "mysqlAMPM requires size >= 2", fragment); String text(cur, 2); boost::to_lower(text); @@ -945,59 +946,59 @@ namespace return cur; } - static Pos mysqlHHMM12(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlHHMM12(Pos cur, Pos end, const String & fragment, DateTime & date) { - checkSpace(cur, end, 8, "mysqlHHMM12 requires size >= 8", flag); + checkSpace(cur, end, 8, "mysqlHHMM12 requires size >= 8", fragment); Int32 hour; - cur = readNumber2(cur, end, flag, hour); - cur = assertChar(cur, end, ':', flag); + cur = readNumber2(cur, end, fragment, hour); + cur = assertChar(cur, end, ':', fragment); date.setHour(hour, true, true); Int32 minute; - cur = readNumber2(cur, end, flag, minute); - cur = assertChar(cur, end, ' ', flag); + cur = readNumber2(cur, end, fragment, minute); + cur = assertChar(cur, end, ' ', fragment); date.setMinute(minute); - cur = mysqlAMPM(cur, end, flag, date); + cur = mysqlAMPM(cur, end, fragment, date); return cur; } - static Pos mysqlHHMM24(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlHHMM24(Pos cur, Pos end, const String & fragment, DateTime & date) { - checkSpace(cur, end, 5, "mysqlHHMM24 requires size >= 5", flag); + checkSpace(cur, end, 5, "mysqlHHMM24 requires size >= 5", fragment); Int32 hour; - cur = readNumber2(cur, end, flag, hour); - cur = assertChar(cur, end, ':', flag); + cur = readNumber2(cur, end, fragment, hour); + cur = assertChar(cur, end, ':', fragment); date.setHour(hour, false, false); Int32 minute; - cur = readNumber2(cur, end, flag, minute); + cur = readNumber2(cur, end, fragment, minute); date.setMinute(minute); return cur; } - static Pos mysqlSecond(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlSecond(Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 second; - cur = readNumber2(cur, end, flag, second); + cur = readNumber2(cur, end, fragment, second); date.setSecond(second); return cur; } - static Pos mysqlISO8601Time(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlISO8601Time(Pos cur, Pos end, const String & fragment, DateTime & date) { - checkSpace(cur, end, 8, "mysqlISO8601Time requires size >= 8", flag); + checkSpace(cur, end, 8, "mysqlISO8601Time requires size >= 8", fragment); Int32 hour; Int32 minute; Int32 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); + cur = readNumber2(cur, end, fragment, hour); + cur = assertChar(cur, end, ':', fragment); + cur = readNumber2(cur, end, fragment, minute); + cur = assertChar(cur, end, ':', fragment); + cur = readNumber2(cur, end, fragment, second); date.setHour(hour, false, false); date.setMinute(minute); @@ -1005,18 +1006,18 @@ namespace return cur; } - static Pos mysqlHour12(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlHour12(Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 hour; - cur = readNumber2(cur, end, flag, hour); + cur = readNumber2(cur, end, fragment, hour); date.setHour(hour, true, true); return cur; } - static Pos mysqlHour24(Pos cur, Pos end, const String & flag, DateTime & date) + static Pos mysqlHour24(Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 hour; - cur = readNumber2(cur, end, flag, hour); + cur = readNumber2(cur, end, fragment, hour); date.setHour(hour, false, false); return cur; } @@ -1029,7 +1030,7 @@ namespace bool is_year, int repetitions, int max_digits_to_read, - const String & flag, + const String & fragment, Int32 & number) { bool negative = false; @@ -1090,8 +1091,8 @@ namespace if (cur == start) throw Exception( ErrorCodes::CANNOT_PARSE_DATETIME, - "Unable to parse flag {} from {} because read number failed", - flag, + "Unable to parse fragment {} from {} because read number failed", + fragment, std::string_view(cur, end - cur)); if (negative) @@ -1100,9 +1101,9 @@ namespace return cur; } - static Pos jodaEra(int, Pos cur, Pos end, const String & flag, DateTime & date) + static Pos jodaEra(int, Pos cur, Pos end, const String & fragment, DateTime & date) { - checkSpace(cur, end, 2, "jodaEra requires size >= 2", flag); + checkSpace(cur, end, 2, "jodaEra requires size >= 2", fragment); String era(cur, 2); boost::to_lower(era); @@ -1111,50 +1112,50 @@ namespace return cur; } - static Pos jodaCenturyOfEra(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) + static Pos jodaCenturyOfEra(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 century; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, repetitions, flag, century); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, repetitions, fragment, century); date.setCentury(century); return cur; } - static Pos jodaYearOfEra(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) + static Pos jodaYearOfEra(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 year_of_era; - cur = readNumberWithVariableLength(cur, end, false, false, true, repetitions, repetitions, flag, year_of_era); + cur = readNumberWithVariableLength(cur, end, false, false, true, repetitions, repetitions, fragment, year_of_era); date.setYear(year_of_era, true); return cur; } - static Pos jodaWeekYear(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) + static Pos jodaWeekYear(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 week_year; - cur = readNumberWithVariableLength(cur, end, true, true, true, repetitions, repetitions, flag, week_year); + cur = readNumberWithVariableLength(cur, end, true, true, true, repetitions, repetitions, fragment, week_year); date.setYear(week_year, false, true); return cur; } - static Pos jodaWeekOfWeekYear(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) + static Pos jodaWeekOfWeekYear(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 week; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, week); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), fragment, week); date.setWeek(week); return cur; } - static Pos jodaDayOfWeek1Based(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) + static Pos jodaDayOfWeek1Based(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 day_of_week; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, repetitions, flag, day_of_week); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, repetitions, fragment, day_of_week); date.setDayOfWeek(day_of_week); return cur; } static Pos - jodaDayOfWeekText(size_t /*min_represent_digits*/, Pos cur, Pos end, const String & flag, DateTime & date) + jodaDayOfWeekText(size_t /*min_represent_digits*/, Pos cur, Pos end, const String & fragment, DateTime & date) { - checkSpace(cur, end, 3, "jodaDayOfWeekText requires size >= 3", flag); + checkSpace(cur, end, 3, "jodaDayOfWeekText requires size >= 3", fragment); String text1(cur, 3); boost::to_lower(text1); @@ -1162,8 +1163,8 @@ namespace if (it == dayOfWeekMap.end()) throw Exception( ErrorCodes::CANNOT_PARSE_DATETIME, - "Unable to parse flag {} from {} because of unknown day of week text: {}", - flag, + "Unable to parse fragment {} from {} because of unknown day of week text: {}", + fragment, std::string_view(cur, end - cur), text1); cur += 3; @@ -1183,41 +1184,41 @@ namespace return cur; } - static Pos jodaYear(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) + static Pos jodaYear(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 year; - cur = readNumberWithVariableLength(cur, end, true, true, true, repetitions, repetitions, flag, year); + cur = readNumberWithVariableLength(cur, end, true, true, true, repetitions, repetitions, fragment, year); date.setYear(year); return cur; } - static Pos jodaDayOfYear(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) + static Pos jodaDayOfYear(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 day_of_year; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 3), flag, day_of_year); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 3), fragment, day_of_year); date.setDayOfYear(day_of_year); return cur; } - static Pos jodaMonthOfYear(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) + static Pos jodaMonthOfYear(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 month; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, 2, flag, month); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, 2, fragment, month); date.setMonth(month); return cur; } - static Pos jodaMonthOfYearText(int, Pos cur, Pos end, const String & flag, DateTime & date) + static Pos jodaMonthOfYearText(int, Pos cur, Pos end, const String & fragment, DateTime & date) { - checkSpace(cur, end, 3, "jodaMonthOfYearText requires size >= 3", flag); + checkSpace(cur, end, 3, "jodaMonthOfYearText requires size >= 3", fragment); String text1(cur, 3); boost::to_lower(text1); auto it = monthMap.find(text1); if (it == monthMap.end()) throw Exception( ErrorCodes::CANNOT_PARSE_DATETIME, - "Unable to parse flag {} from {} because of unknown month of year text: {}", - flag, + "Unable to parse fragment {} from {} because of unknown month of year text: {}", + fragment, std::string_view(cur, end - cur), text1); cur += 3; @@ -1237,18 +1238,18 @@ namespace return cur; } - static Pos jodaDayOfMonth(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) + static Pos jodaDayOfMonth(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 day_of_month; cur = readNumberWithVariableLength( - cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, day_of_month); + cur, end, false, false, false, repetitions, std::max(repetitions, 2), fragment, day_of_month); date.setDayOfMonth(day_of_month); return cur; } - static Pos jodaHalfDayOfDay(int, Pos cur, Pos end, const String & flag, DateTime & date) + static Pos jodaHalfDayOfDay(int, Pos cur, Pos end, const String & fragment, DateTime & date) { - checkSpace(cur, end, 2, "jodaHalfDayOfDay requires size >= 2", flag); + checkSpace(cur, end, 2, "jodaHalfDayOfDay requires size >= 2", fragment); String text(cur, 2); boost::to_lower(text); @@ -1257,50 +1258,50 @@ namespace return cur; } - static Pos jodaHourOfHalfDay(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) + static Pos jodaHourOfHalfDay(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 hour; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, hour); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), fragment, hour); date.setHour(hour, true, false); return cur; } - static Pos jodaClockHourOfHalfDay(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) + static Pos jodaClockHourOfHalfDay(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 hour; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, hour); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), fragment, hour); date.setHour(hour, true, true); return cur; } - static Pos jodaHourOfDay(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) + static Pos jodaHourOfDay(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 hour; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, hour); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), fragment, hour); date.setHour(hour, false, false); return cur; } - static Pos jodaClockHourOfDay(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) + static Pos jodaClockHourOfDay(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 hour; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, hour); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), fragment, hour); date.setHour(hour, false, true); return cur; } - static Pos jodaMinuteOfHour(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) + static Pos jodaMinuteOfHour(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 minute; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, minute); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), fragment, minute); date.setMinute(minute); return cur; } - static Pos jodaSecondOfMinute(int repetitions, Pos cur, Pos end, const String & flag, DateTime & date) + static Pos jodaSecondOfMinute(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 second; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), flag, second); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), fragment, second); date.setSecond(second); return cur; } @@ -1443,8 +1444,6 @@ namespace 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(Instruction::mysqlMinute)); From 21f5d20b9eca19ff6e27ecd4d5af3932958a371c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 20 Mar 2023 17:30:20 +0000 Subject: [PATCH 185/418] apply black to all python scripts --- docker/test/style/Dockerfile | 1 + utils/check-style/check-black | 19 ++++++++++++++++--- 2 files changed, 17 insertions(+), 3 deletions(-) diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index e8c5e17024c..f7b2ef21c2a 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -11,6 +11,7 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ aspell \ curl \ git \ + file \ libxml2-utils \ moreutils \ python3-fuzzywuzzy \ diff --git a/utils/check-style/check-black b/utils/check-style/check-black index 141dcd1b406..33c463d1668 100755 --- a/utils/check-style/check-black +++ b/utils/check-style/check-black @@ -4,10 +4,22 @@ set -e # We check only our code, that's why we skip contrib GIT_ROOT=$(git rev-parse --show-cdup) -GIT_ROOT=${GIT_ROOT:-.} +GIT_ROOT=${GIT_ROOT:-./} tmp=$(mktemp) -# Find all *.py files in the repo except the contrib directory -find_cmd=(find "$GIT_ROOT" -name '*.py' -not -path "$GIT_ROOT/contrib/*") + +# Find all *.py, *.python files and executable files without extension +# that are determined as python scripts by 'file' util +# in the repo except the contrib directory. +find_cmd=( + find "$GIT_ROOT" -type f -not -path "${GIT_ROOT}contrib/*" + \( + -name '*.py' -or -name "*.python" -or + \( + -executable -not -name "*.*" -exec sh -c 'file {} | grep -q "Python script"' \; + \) + \) +) + if ! "${find_cmd[@]}" -exec black --check --diff {} + 1>"$tmp" 2>&1; then # Show the result only if some files need formatting cat "$tmp" @@ -16,4 +28,5 @@ if ! "${find_cmd[@]}" -exec black --check --diff {} + 1>"$tmp" 2>&1; then # Automatically add changed files to stage "${find_cmd[@]}" -exec git add -u {} + 1>/dev/null 2>&1 fi + rm "$tmp" From b75c92e6b3bff275ee02d60729f03102fc63e737 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 20 Mar 2023 18:38:53 +0100 Subject: [PATCH 186/418] May be fix window view test --- .../0_stateless/01072_window_view_multiple_columns_groupby.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 05cb973cdb5..1b66beffe7e 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 @@ -17,7 +17,7 @@ DROP TABLE IF EXISTS wv; CREATE TABLE dst(time DateTime, colA String, colB String) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(colA String, colB String) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst AS SELECT tumbleStart(w_id) AS time, colA, colB FROM mt GROUP BY tumble(now(), INTERVAL '10' SECOND, 'US/Samoa') AS w_id, colA, colB; +CREATE WINDOW VIEW wv TO dst AS SELECT tumbleStart(w_id) AS time, colA, colB FROM mt GROUP BY tumble(now('US/Samoa'), INTERVAL '10' SECOND, 'US/Samoa') AS w_id, colA, colB; INSERT INTO mt VALUES ('test1', 'test2'); EOF From ed1ff0211493ed3efe194de6dd1e835fd11496a3 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Mon, 20 Mar 2023 06:21:16 -0700 Subject: [PATCH 187/418] Fix partition ID byte order for s390x --- src/Storages/MergeTree/MergeTreePartition.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 3b28012e7d6..2c5350909d5 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -261,8 +261,11 @@ String MergeTreePartition::getID(const Block & partition_key_sample) const hash.get128(hash_data); result.resize(32); for (size_t i = 0; i < 16; ++i) +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + writeHexByteLowercase(hash_data[16 - 1 - i], &result[2 * i]); +#else writeHexByteLowercase(hash_data[i], &result[2 * i]); - +#endif return result; } From 3cafb9747a75860c7c6300401be896d68983107a Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 20 Mar 2023 19:42:11 +0100 Subject: [PATCH 188/418] Fix corner cases and TOTALS with constants --- src/Planner/PlannerExpressionAnalysis.cpp | 40 ++++++++++--------- ..._no_aggregates_and_constant_keys.reference | 2 +- .../00313_const_totals_extremes.reference | 12 +++--- 3 files changed, 29 insertions(+), 25 deletions(-) diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index 7fcc626d752..b6b6b2e2d85 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -93,16 +93,17 @@ std::optional analyzeAggregation(const QueryTreeNodeP for (auto & grouping_set_key_node : grouping_set_keys_list_node_typed.getNodes()) { - group_by_with_constant_keys |= (grouping_set_key_node->as() != nullptr); + auto is_constant_key = grouping_set_key_node->as() != nullptr; + group_by_with_constant_keys |= is_constant_key; + + if (is_constant_key && !aggregates_descriptions.empty()) + continue; auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, grouping_set_key_node); aggregation_keys.reserve(expression_dag_nodes.size()); for (auto & expression_dag_node : expression_dag_nodes) { - if (expression_dag_node->column && isColumnConst(*expression_dag_node->column)) - continue; - grouping_sets_parameters.used_keys.push_back(expression_dag_node->result_name); if (before_aggregation_actions_output_node_names.contains(expression_dag_node->result_name)) continue; @@ -142,24 +143,27 @@ std::optional analyzeAggregation(const QueryTreeNodeP else { for (auto & group_by_key_node : query_node.getGroupBy().getNodes()) - group_by_with_constant_keys |= (group_by_key_node->as() != nullptr); - - auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, query_node.getGroupByNode()); - aggregation_keys.reserve(expression_dag_nodes.size()); - - for (auto & expression_dag_node : expression_dag_nodes) { - if (expression_dag_node->column && isColumnConst(*expression_dag_node->column)) + auto is_constant_key = group_by_key_node->as() != nullptr; + group_by_with_constant_keys |= is_constant_key; + + if (is_constant_key && !aggregates_descriptions.empty()) continue; - if (before_aggregation_actions_output_node_names.contains(expression_dag_node->result_name)) - continue; + auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, group_by_key_node); + aggregation_keys.reserve(expression_dag_nodes.size()); - auto expression_type_after_aggregation = group_by_use_nulls ? makeNullableSafe(expression_dag_node->result_type) : expression_dag_node->result_type; - available_columns_after_aggregation.emplace_back(nullptr, expression_type_after_aggregation, expression_dag_node->result_name); - aggregation_keys.push_back(expression_dag_node->result_name); - before_aggregation_actions->getOutputs().push_back(expression_dag_node); - before_aggregation_actions_output_node_names.insert(expression_dag_node->result_name); + for (auto & expression_dag_node : expression_dag_nodes) + { + if (before_aggregation_actions_output_node_names.contains(expression_dag_node->result_name)) + continue; + + auto expression_type_after_aggregation = group_by_use_nulls ? makeNullableSafe(expression_dag_node->result_type) : expression_dag_node->result_type; + available_columns_after_aggregation.emplace_back(nullptr, expression_type_after_aggregation, expression_dag_node->result_name); + aggregation_keys.push_back(expression_dag_node->result_name); + before_aggregation_actions->getOutputs().push_back(expression_dag_node); + before_aggregation_actions_output_node_names.insert(expression_dag_node->result_name); + } } } } diff --git a/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference b/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference index c5e3f5b9cc9..866dc4f1894 100644 --- a/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference +++ b/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference @@ -11,7 +11,7 @@ 0 2 42 -2 0 +2 42 43 0 diff --git a/tests/queries/0_stateless/00313_const_totals_extremes.reference b/tests/queries/0_stateless/00313_const_totals_extremes.reference index b5abbfa3c4e..02ed40bc027 100644 --- a/tests/queries/0_stateless/00313_const_totals_extremes.reference +++ b/tests/queries/0_stateless/00313_const_totals_extremes.reference @@ -1,6 +1,6 @@ 1 1 -0 1 +1 1 1 1 1 1 @@ -27,7 +27,7 @@ "totals": { - "k": "0", + "k": "1234567890123", "count()": "1" }, @@ -65,7 +65,7 @@ [1.23, "1"] ], - "totals": [0, "1"], + "totals": [1.23, "1"], "extremes": { @@ -77,7 +77,7 @@ } 2010-01-01 1 -1970-01-01 1 +2010-01-01 1 2010-01-01 1 2010-01-01 1 @@ -104,7 +104,7 @@ "totals": { - "k": "1970-01-01 00:00:00", + "k": "2010-01-01 01:02:03", "count()": "1" }, @@ -142,7 +142,7 @@ [1.1, "1"] ], - "totals": [0, "1"], + "totals": [1.1, "1"], "extremes": { From dabc022289bde9986bacffa4dec8c25fe298e024 Mon Sep 17 00:00:00 2001 From: fyu Date: Tue, 21 Mar 2023 03:27:18 +0800 Subject: [PATCH 189/418] Fixed error print message while Decimal parameters is incorrect --- src/DataTypes/DataTypesDecimal.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/DataTypes/DataTypesDecimal.cpp b/src/DataTypes/DataTypesDecimal.cpp index 1ae0d8a6db1..2838f5868d4 100644 --- a/src/DataTypes/DataTypesDecimal.cpp +++ b/src/DataTypes/DataTypesDecimal.cpp @@ -89,12 +89,12 @@ static DataTypePtr createExact(const ASTPtr & arguments) { if (!arguments || arguments->children.size() != 1) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Decimal data type family must have exactly two arguments: precision and scale"); - + "Decimal32 | Decimal64 | Decimal128 | Decimal256 data type family must have exactly one arguments: scale"); const auto * scale_arg = arguments->children[0]->as(); if (!scale_arg || !(scale_arg->value.getType() == Field::Types::Int64 || scale_arg->value.getType() == Field::Types::UInt64)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Decimal data type family must have a two numbers as its arguments"); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Decimal32 | Decimal64 | Decimal128 | Decimal256 data type family must have a one number as its argument"); UInt64 precision = DecimalUtils::max_precision; UInt64 scale = scale_arg->value.get(); From 15d02f77cfcbfe5af0350e4c5b26daa2a68fb8bd Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 20 Mar 2023 20:56:11 +0100 Subject: [PATCH 190/418] Decrease scale_down ratio for faster deflation --- tests/ci/autoscale_runners_lambda/app.py | 4 ++-- tests/ci/autoscale_runners_lambda_test.py | 19 ++++++++++++++++--- 2 files changed, 18 insertions(+), 5 deletions(-) diff --git a/tests/ci/autoscale_runners_lambda/app.py b/tests/ci/autoscale_runners_lambda/app.py index 7e3af3f6779..1fcdbc40155 100644 --- a/tests/ci/autoscale_runners_lambda/app.py +++ b/tests/ci/autoscale_runners_lambda/app.py @@ -59,11 +59,11 @@ def get_scales(runner_type: str) -> Tuple[int, int]: "returns the multipliers for scaling down and up ASG by types" # Scaling down is quicker on the lack of running jobs than scaling up on # queue - scale_down = 3 + scale_down = 2 scale_up = 5 if runner_type == "style-checker": # the style checkers have so many noise, so it scales up too quickly - scale_down = 2 + scale_down = 1 scale_up = 10 return scale_down, scale_up diff --git a/tests/ci/autoscale_runners_lambda_test.py b/tests/ci/autoscale_runners_lambda_test.py index 7efa0004745..8e3828f51c0 100644 --- a/tests/ci/autoscale_runners_lambda_test.py +++ b/tests/ci/autoscale_runners_lambda_test.py @@ -70,6 +70,9 @@ class TestSetCapacity(unittest.TestCase): TestCase("w/reserve", 1, 13, 20, [Queue("queued", 17, "w/reserve")], -1), # Increase capacity TestCase("increase", 1, 13, 20, [Queue("queued", 23, "increase")], 15), + TestCase( + "style-checker", 1, 13, 20, [Queue("queued", 33, "style-checker")], 15 + ), TestCase("increase", 1, 13, 20, [Queue("queued", 18, "increase")], 14), TestCase("increase", 1, 13, 20, [Queue("queued", 183, "increase")], 20), TestCase( @@ -85,10 +88,20 @@ class TestSetCapacity(unittest.TestCase): ), TestCase("lower-min", 10, 5, 20, [Queue("queued", 5, "lower-min")], 10), # Decrease capacity - TestCase("w/reserve", 1, 13, 20, [Queue("queued", 5, "w/reserve")], 11), + TestCase("w/reserve", 1, 13, 20, [Queue("queued", 5, "w/reserve")], 9), + TestCase( + "style-checker", 1, 13, 20, [Queue("queued", 5, "style-checker")], 5 + ), TestCase("w/reserve", 1, 23, 20, [Queue("queued", 17, "w/reserve")], 20), - TestCase("decrease", 1, 13, 20, [Queue("in_progress", 3, "decrease")], 10), - TestCase("decrease", 1, 13, 20, [Queue("in_progress", 5, "decrease")], 11), + TestCase("decrease", 1, 13, 20, [Queue("in_progress", 3, "decrease")], 8), + TestCase( + "style-checker", + 1, + 13, + 20, + [Queue("in_progress", 5, "style-checker")], + 5, + ), ) for t in test_cases: self.client.data_helper(t.name, t.min_size, t.desired_capacity, t.max_size) From fbc2124c06a183d24c53ce0b30f5e783dc6fc56f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 20 Mar 2023 21:36:55 +0100 Subject: [PATCH 191/418] Update test for nested lambdas --- tests/queries/0_stateless/00897_flatten.reference | 12 ++++++++++++ tests/queries/0_stateless/00897_flatten.sql | 4 +++- 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00897_flatten.reference b/tests/queries/0_stateless/00897_flatten.reference index 6c1aa724070..591b70ee063 100644 --- a/tests/queries/0_stateless/00897_flatten.reference +++ b/tests/queries/0_stateless/00897_flatten.reference @@ -8,6 +8,18 @@ [0,0,1,0,1,0,1,0,1] [0,0,1,0,1,0,1,0,1,0,1,2,0,1,2,0,1,2,0,1,2,0,1,2,0,1,2,0,1,2,0,1,2,0,1,2] [0,0,1,0,1,0,1,0,1,0,1,2,0,1,2,0,1,2,0,1,2,0,1,2,0,1,2,0,1,2,0,1,2,0,1,2,0,1,2,3,0,1,2,3,0,1,2,3,0,1,2,3,0,1,2,3,0,1,2,3,0,1,2,3,0,1,2,3,0,1,2,3,0,1,2,3,0,1,2,3,0,1,2,3,0,1,2,3,0,1,2,3,0,1,2,3,0,1,2,3] +[] +[] +[] +[] +[0] +[0,0,0,0,1] +[] +[] +[] +[] +[0] +[0,0,0,0,1] [1,2,3,4,5,6,7,8] [] [] diff --git a/tests/queries/0_stateless/00897_flatten.sql b/tests/queries/0_stateless/00897_flatten.sql index 04c725677bd..0d67a1708fd 100644 --- a/tests/queries/0_stateless/00897_flatten.sql +++ b/tests/queries/0_stateless/00897_flatten.sql @@ -1,6 +1,8 @@ SELECT flatten(arrayJoin([[[1, 2, 3], [4, 5]], [[6], [7, 8]]])); SELECT arrayFlatten(arrayJoin([[[[]], [[1], [], [2, 3]]], [[[4]]]])); -SELECT flatten(arrayMap(x -> arrayMap(x -> arrayMap(x -> range(x), range(x)), range(x)), range(number))) FROM numbers(6); +SELECT flatten(arrayMap(x -> arrayMap(y -> arrayMap(z -> range(x), range(x)), range(x)), range(number))) FROM numbers(6); +SELECT flatten(arrayMap(x -> arrayMap(y -> arrayMap(z -> range(z), range(y)), range(x)), range(number))) FROM numbers(6); +SELECT flatten(arrayMap(x -> arrayMap(x -> arrayMap(x -> range(x), range(x)), range(x)), range(number))) FROM numbers(6) SETTINGS allow_experimental_analyzer=1; SELECT arrayFlatten([[[1, 2, 3], [4, 5]], [[6], [7, 8]]]); SELECT flatten([[[]]]); SELECT arrayFlatten([]); From 633b1809a5499339d8277aa5d974fdbe04769cc3 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 20 Mar 2023 20:08:56 +0000 Subject: [PATCH 192/418] fix reading from storage File compressed files with zlib and gzip compression --- src/IO/ZlibInflatingReadBuffer.cpp | 22 +++++++++++++++------- src/IO/ZlibInflatingReadBuffer.h | 6 ++++++ 2 files changed, 21 insertions(+), 7 deletions(-) diff --git a/src/IO/ZlibInflatingReadBuffer.cpp b/src/IO/ZlibInflatingReadBuffer.cpp index 9c2ee640cbe..8f730a06829 100644 --- a/src/IO/ZlibInflatingReadBuffer.cpp +++ b/src/IO/ZlibInflatingReadBuffer.cpp @@ -6,6 +6,7 @@ namespace DB namespace ErrorCodes { extern const int ZLIB_INFLATE_FAILED; + extern const int ARGUMENT_OUT_OF_BOUND; } ZlibInflatingReadBuffer::ZlibInflatingReadBuffer( @@ -17,6 +18,11 @@ ZlibInflatingReadBuffer::ZlibInflatingReadBuffer( : CompressedReadBufferWrapper(std::move(in_), buf_size, existing_memory, alignment) , eof_flag(false) { + if (buf_size > max_buffer_size) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, + "Zlib does not support decompression with buffer size greater than {}, got buffer size: {}", + max_buffer_size, buf_size); + zstr.zalloc = nullptr; zstr.zfree = nullptr; zstr.opaque = nullptr; @@ -31,10 +37,7 @@ ZlibInflatingReadBuffer::ZlibInflatingReadBuffer( window_bits += 16; } -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wold-style-cast" int rc = inflateInit2(&zstr, window_bits); -#pragma GCC diagnostic pop if (rc != Z_OK) throw Exception(ErrorCodes::ZLIB_INFLATE_FAILED, "inflateInit2 failed: {}; zlib version: {}.", zError(rc), ZLIB_VERSION); @@ -61,16 +64,20 @@ bool ZlibInflatingReadBuffer::nextImpl() { in->nextIfAtEnd(); zstr.next_in = reinterpret_cast(in->position()); - zstr.avail_in = static_cast(in->buffer().end() - in->position()); + zstr.avail_in = static_cast(std::min(in->buffer().end() - in->position(), static_cast(max_buffer_size))); } + /// init output bytes (place, where decompressed data will be) zstr.next_out = reinterpret_cast(internal_buffer.begin()); - zstr.avail_out = static_cast(internal_buffer.size()); + zstr.avail_out = static_cast(internal_buffer.size()); + size_t old_total_in = zstr.total_in; int rc = inflate(&zstr, Z_NO_FLUSH); /// move in stream on place, where reading stopped - in->position() = in->buffer().end() - zstr.avail_in; + size_t bytes_read = zstr.total_in - old_total_in; + in->position() += bytes_read; + /// change size of working buffer (it's size equal to internal_buffer size without unused uncompressed values) working_buffer.resize(internal_buffer.size() - zstr.avail_out); @@ -94,9 +101,10 @@ bool ZlibInflatingReadBuffer::nextImpl() return true; } } + /// If it is not end and not OK, something went wrong, throw exception if (rc != Z_OK) - throw Exception(ErrorCodes::ZLIB_INFLATE_FAILED, "inflateReset failed: {}", zError(rc)); + throw Exception(ErrorCodes::ZLIB_INFLATE_FAILED, "inflate failed: {}", zError(rc)); } while (working_buffer.empty()); diff --git a/src/IO/ZlibInflatingReadBuffer.h b/src/IO/ZlibInflatingReadBuffer.h index b534b7cb5c4..d9ca4c61268 100644 --- a/src/IO/ZlibInflatingReadBuffer.h +++ b/src/IO/ZlibInflatingReadBuffer.h @@ -4,6 +4,7 @@ #include #include +#include #include @@ -33,6 +34,11 @@ private: z_stream zstr; bool eof_flag; + + /// Limit size of buffer because zlib uses + /// UInt32 for sizes of internal buffers. + using BufferSizeType = decltype(zstr.avail_in); + static constexpr auto max_buffer_size = std::numeric_limits::max(); }; } From 18c59fb5d3c22d258bb6ec51f3d756d71a0852e4 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Mon, 20 Mar 2023 17:19:34 -0400 Subject: [PATCH 193/418] Create doc and scripts for qpl_deflate benchmark --- .../client_scripts/allin1_ssb.sh | 530 ++++++++++++++++++ .../client_scripts/client_stressing_test.py | 202 +++++++ .../client_scripts/queries_ssb.sql | 10 + .../client_scripts/run_ssb.sh | 6 + .../database_dir/deflate/config_deflate.xml | 49 ++ .../deflate_s2/config_deflate_s2.xml | 49 ++ .../database_dir/lz4/config_lz4.xml | 49 ++ .../database_dir/lz4_s2/config_lz4_s2.xml | 49 ++ .../database_dir/zstd/config_zstd.xml | 49 ++ .../database_dir/zstd_s2/config_zstd_s2.xml | 49 ++ contrib/qpl-cmake/doc/benchmark.md | 270 +++++++++ contrib/qpl-cmake/doc/build.md | 14 + .../sql-reference/statements/create/table.md | 4 +- 13 files changed, 1328 insertions(+), 2 deletions(-) create mode 100644 contrib/qpl-cmake/benchmark_sample/client_scripts/allin1_ssb.sh create mode 100644 contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py create mode 100644 contrib/qpl-cmake/benchmark_sample/client_scripts/queries_ssb.sql create mode 100644 contrib/qpl-cmake/benchmark_sample/client_scripts/run_ssb.sh create mode 100644 contrib/qpl-cmake/benchmark_sample/database_dir/deflate/config_deflate.xml create mode 100644 contrib/qpl-cmake/benchmark_sample/database_dir/deflate_s2/config_deflate_s2.xml create mode 100644 contrib/qpl-cmake/benchmark_sample/database_dir/lz4/config_lz4.xml create mode 100644 contrib/qpl-cmake/benchmark_sample/database_dir/lz4_s2/config_lz4_s2.xml create mode 100644 contrib/qpl-cmake/benchmark_sample/database_dir/zstd/config_zstd.xml create mode 100644 contrib/qpl-cmake/benchmark_sample/database_dir/zstd_s2/config_zstd_s2.xml create mode 100644 contrib/qpl-cmake/doc/benchmark.md create mode 100644 contrib/qpl-cmake/doc/build.md diff --git a/contrib/qpl-cmake/benchmark_sample/client_scripts/allin1_ssb.sh b/contrib/qpl-cmake/benchmark_sample/client_scripts/allin1_ssb.sh new file mode 100644 index 00000000000..e0880811a06 --- /dev/null +++ b/contrib/qpl-cmake/benchmark_sample/client_scripts/allin1_ssb.sh @@ -0,0 +1,530 @@ +#!/bin/bash +ckhost="localhost" +ckport=("9000" "9001" "9002" "9003") +WORKING_DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)/.." +OUTPUT_DIR="${WORKING_DIR}/output" +LOG_DIR="${OUTPUT_DIR}/log" +RAWDATA_DIR="${WORKING_DIR}/rawdata_dir" +database_dir="${WORKING_DIR}/database_dir" +CLIENT_SCRIPTS_DIR="${WORKING_DIR}/client_scripts" +LOG_PACK_FILE="$(date +%Y-%m-%d-%H-%M-%S)" +QUERY_FILE="queries_ssb.sql" +SERVER_BIND_CMD[0]="numactl -m 0 -N 0" +SERVER_BIND_CMD[1]="numactl -m 0 -N 0" +SERVER_BIND_CMD[2]="numactl -m 1 -N 1" +SERVER_BIND_CMD[3]="numactl -m 1 -N 1" +CLIENT_BIND_CMD="" +SSB_GEN_FACTOR=20 +TABLE_NAME="lineorder_flat" +TALBE_ROWS="119994608" +CODEC_CONFIG="lz4 deflate zstd" + +# define instance number +inst_num=$1 +if [ ! -n "$1" ]; then + echo "Please clarify instance number from 1,2,3 or 4" + exit 1 +else + echo "Benchmarking with instance number:$1" +fi + +if [ ! -d "$OUTPUT_DIR" ]; then +mkdir $OUTPUT_DIR +fi +if [ ! -d "$LOG_DIR" ]; then +mkdir $LOG_DIR +fi +if [ ! -d "$RAWDATA_DIR" ]; then +mkdir $RAWDATA_DIR +fi + +# define different directories +dir_server=("" "_s2" "_s3" "_s4") +ckreadSql=" + CREATE TABLE customer + ( + C_CUSTKEY UInt32, + C_NAME String, + C_ADDRESS String, + C_CITY LowCardinality(String), + C_NATION LowCardinality(String), + C_REGION LowCardinality(String), + C_PHONE String, + C_MKTSEGMENT LowCardinality(String) + ) + ENGINE = MergeTree ORDER BY (C_CUSTKEY); + + CREATE TABLE lineorder + ( + LO_ORDERKEY UInt32, + LO_LINENUMBER UInt8, + LO_CUSTKEY UInt32, + LO_PARTKEY UInt32, + LO_SUPPKEY UInt32, + LO_ORDERDATE Date, + LO_ORDERPRIORITY LowCardinality(String), + LO_SHIPPRIORITY UInt8, + LO_QUANTITY UInt8, + LO_EXTENDEDPRICE UInt32, + LO_ORDTOTALPRICE UInt32, + LO_DISCOUNT UInt8, + LO_REVENUE UInt32, + LO_SUPPLYCOST UInt32, + LO_TAX UInt8, + LO_COMMITDATE Date, + LO_SHIPMODE LowCardinality(String) + ) + ENGINE = MergeTree PARTITION BY toYear(LO_ORDERDATE) ORDER BY (LO_ORDERDATE, LO_ORDERKEY); + + CREATE TABLE part + ( + P_PARTKEY UInt32, + P_NAME String, + P_MFGR LowCardinality(String), + P_CATEGORY LowCardinality(String), + P_BRAND LowCardinality(String), + P_COLOR LowCardinality(String), + P_TYPE LowCardinality(String), + P_SIZE UInt8, + P_CONTAINER LowCardinality(String) + ) + ENGINE = MergeTree ORDER BY P_PARTKEY; + + CREATE TABLE supplier + ( + S_SUPPKEY UInt32, + S_NAME String, + S_ADDRESS String, + S_CITY LowCardinality(String), + S_NATION LowCardinality(String), + S_REGION LowCardinality(String), + S_PHONE String + ) + ENGINE = MergeTree ORDER BY S_SUPPKEY; +" +supplier_table=" + CREATE TABLE supplier + ( + S_SUPPKEY UInt32, + S_NAME String, + S_ADDRESS String, + S_CITY LowCardinality(String), + S_NATION LowCardinality(String), + S_REGION LowCardinality(String), + S_PHONE String + ) + ENGINE = MergeTree ORDER BY S_SUPPKEY; +" +part_table=" + CREATE TABLE part + ( + P_PARTKEY UInt32, + P_NAME String, + P_MFGR LowCardinality(String), + P_CATEGORY LowCardinality(String), + P_BRAND LowCardinality(String), + P_COLOR LowCardinality(String), + P_TYPE LowCardinality(String), + P_SIZE UInt8, + P_CONTAINER LowCardinality(String) + ) + ENGINE = MergeTree ORDER BY P_PARTKEY; +" +lineorder_table=" + CREATE TABLE lineorder + ( + LO_ORDERKEY UInt32, + LO_LINENUMBER UInt8, + LO_CUSTKEY UInt32, + LO_PARTKEY UInt32, + LO_SUPPKEY UInt32, + LO_ORDERDATE Date, + LO_ORDERPRIORITY LowCardinality(String), + LO_SHIPPRIORITY UInt8, + LO_QUANTITY UInt8, + LO_EXTENDEDPRICE UInt32, + LO_ORDTOTALPRICE UInt32, + LO_DISCOUNT UInt8, + LO_REVENUE UInt32, + LO_SUPPLYCOST UInt32, + LO_TAX UInt8, + LO_COMMITDATE Date, + LO_SHIPMODE LowCardinality(String) + ) + ENGINE = MergeTree PARTITION BY toYear(LO_ORDERDATE) ORDER BY (LO_ORDERDATE, LO_ORDERKEY); +" +customer_table=" + CREATE TABLE customer + ( + C_CUSTKEY UInt32, + C_NAME String, + C_ADDRESS String, + C_CITY LowCardinality(String), + C_NATION LowCardinality(String), + C_REGION LowCardinality(String), + C_PHONE String, + C_MKTSEGMENT LowCardinality(String) + ) + ENGINE = MergeTree ORDER BY (C_CUSTKEY); +" + +lineorder_flat_table=" + SET max_memory_usage = 20000000000; + CREATE TABLE lineorder_flat + ENGINE = MergeTree + PARTITION BY toYear(LO_ORDERDATE) + ORDER BY (LO_ORDERDATE, LO_ORDERKEY) AS + SELECT + l.LO_ORDERKEY AS LO_ORDERKEY, + l.LO_LINENUMBER AS LO_LINENUMBER, + l.LO_CUSTKEY AS LO_CUSTKEY, + l.LO_PARTKEY AS LO_PARTKEY, + l.LO_SUPPKEY AS LO_SUPPKEY, + l.LO_ORDERDATE AS LO_ORDERDATE, + l.LO_ORDERPRIORITY AS LO_ORDERPRIORITY, + l.LO_SHIPPRIORITY AS LO_SHIPPRIORITY, + l.LO_QUANTITY AS LO_QUANTITY, + l.LO_EXTENDEDPRICE AS LO_EXTENDEDPRICE, + l.LO_ORDTOTALPRICE AS LO_ORDTOTALPRICE, + l.LO_DISCOUNT AS LO_DISCOUNT, + l.LO_REVENUE AS LO_REVENUE, + l.LO_SUPPLYCOST AS LO_SUPPLYCOST, + l.LO_TAX AS LO_TAX, + l.LO_COMMITDATE AS LO_COMMITDATE, + l.LO_SHIPMODE AS LO_SHIPMODE, + c.C_NAME AS C_NAME, + c.C_ADDRESS AS C_ADDRESS, + c.C_CITY AS C_CITY, + c.C_NATION AS C_NATION, + c.C_REGION AS C_REGION, + c.C_PHONE AS C_PHONE, + c.C_MKTSEGMENT AS C_MKTSEGMENT, + s.S_NAME AS S_NAME, + s.S_ADDRESS AS S_ADDRESS, + s.S_CITY AS S_CITY, + s.S_NATION AS S_NATION, + s.S_REGION AS S_REGION, + s.S_PHONE AS S_PHONE, + p.P_NAME AS P_NAME, + p.P_MFGR AS P_MFGR, + p.P_CATEGORY AS P_CATEGORY, + p.P_BRAND AS P_BRAND, + p.P_COLOR AS P_COLOR, + p.P_TYPE AS P_TYPE, + p.P_SIZE AS P_SIZE, + p.P_CONTAINER AS P_CONTAINER + FROM lineorder AS l + INNER JOIN customer AS c ON c.C_CUSTKEY = l.LO_CUSTKEY + INNER JOIN supplier AS s ON s.S_SUPPKEY = l.LO_SUPPKEY + INNER JOIN part AS p ON p.P_PARTKEY = l.LO_PARTKEY; + show settings ilike 'max_memory_usage'; +" + +function insert_data(){ + echo "insert_data:$1" + create_table_prefix="clickhouse client --host ${ckhost} --port $2 --multiquery -q" + insert_data_prefix="clickhouse client --query " + case $1 in + all) + clickhouse client --host ${ckhost} --port $2 --multiquery -q"$ckreadSql" && { + ${insert_data_prefix} "INSERT INTO customer FORMAT CSV" < ${RAWDATA_DIR}/ssb-dbgen/customer.tbl --port=$2 + ${insert_data_prefix} "INSERT INTO part FORMAT CSV" < ${RAWDATA_DIR}/ssb-dbgen/part.tbl --port=$2 + ${insert_data_prefix} "INSERT INTO supplier FORMAT CSV" < ${RAWDATA_DIR}/ssb-dbgen/supplier.tbl --port=$2 + ${insert_data_prefix} "INSERT INTO lineorder FORMAT CSV" < ${RAWDATA_DIR}/ssb-dbgen/lineorder.tbl --port=$2 + } + ${create_table_prefix}"${lineorder_flat_table}" + ;; + customer) + echo ${create_table_prefix}\"${customer_table}\" + ${create_table_prefix}"${customer_table}" && { + echo "${insert_data_prefix} \"INSERT INTO $1 FORMAT CSV\" < ${RAWDATA_DIR}/ssb-dbgen/$1.tbl --port=$2" + ${insert_data_prefix} "INSERT INTO $1 FORMAT CSV" < ${RAWDATA_DIR}/ssb-dbgen/$1.tbl --port=$2 + } + ;; + part) + echo ${create_table_prefix}\"${part_table}\" + ${create_table_prefix}"${part_table}" && { + echo "${insert_data_prefix} \"INSERT INTO $1 FORMAT CSV\" < ${RAWDATA_DIR}/ssb-dbgen/$1.tbl --port=$2" + ${insert_data_prefix} "INSERT INTO $1 FORMAT CSV" < ${RAWDATA_DIR}/ssb-dbgen/$1.tbl --port=$2 + } + ;; + supplier) + echo ${create_table_prefix}"${supplier_table}" + ${create_table_prefix}"${supplier_table}" && { + echo "${insert_data_prefix} \"INSERT INTO $1 FORMAT CSV\" < ${RAWDATA_DIR}/ssb-dbgen/$1.tbl --port=$2" + ${insert_data_prefix} "INSERT INTO $1 FORMAT CSV" < ${RAWDATA_DIR}/ssb-dbgen/$1.tbl --port=$2 + } + ;; + lineorder) + echo ${create_table_prefix}"${lineorder_table}" + ${create_table_prefix}"${lineorder_table}" && { + echo "${insert_data_prefix} \"INSERT INTO $1 FORMAT CSV\" < ${RAWDATA_DIR}/ssb-dbgen/$1.tbl --port=$2" + ${insert_data_prefix} "INSERT INTO $1 FORMAT CSV" < ${RAWDATA_DIR}/ssb-dbgen/$1.tbl --port=$2 + } + ;; + lineorder_flat) + echo ${create_table_prefix}"${lineorder_flat_table}" + ${create_table_prefix}"${lineorder_flat_table}" + return 0 + ;; + *) + exit 0 + ;; + + esac +} + +function check_sql(){ + select_sql="select * from "$1" limit 1" + clickhouse client --host ${ckhost} --port $2 --multiquery -q"${select_sql}" +} + +function check_table(){ + checknum=0 + source_tables="customer part supplier lineorder lineorder_flat" + test_tables=${1:-${source_tables}} + echo "Checking table data required in server..." + for i in $(seq 0 $[inst_num-1]) + do + for j in `echo ${test_tables}` + do + check_sql $j ${ckport[i]} &> /dev/null || { + let checknum+=1 && insert_data "$j" ${ckport[i]} + } + done + done + + for i in $(seq 0 $[inst_num-1]) + do + echo "clickhouse client --host ${ckhost} --port ${ckport[i]} -m -q\"select count() from ${TABLE_NAME};\"" + var=$(clickhouse client --host ${ckhost} --port ${ckport[i]} -m -q"select count() from ${TABLE_NAME};") + if [ $var -eq $TALBE_ROWS ];then + echo "Instance_${i} Table data integrity check OK -> Rows:$var" + else + echo "Instance_${i} Table data integrity check Failed -> Rows:$var" + exit 1 + fi + done + if [ $checknum -gt 0 ];then + echo "Need sleep 10s after first table data insertion...$checknum" + sleep 10 + fi +} + +function check_instance(){ +instance_alive=0 +for i in {1..10} +do + sleep 1 + netstat -nltp | grep ${1} > /dev/null + if [ $? -ne 1 ];then + instance_alive=1 + break + fi + +done + +if [ $instance_alive -eq 0 ];then + echo "check_instance -> clickhouse server instance faild to launch due to 10s timeout!" + exit 1 +else + echo "check_instance -> clickhouse server instance launch successfully!" +fi +} + +function start_clickhouse_for_insertion(){ + echo "start_clickhouse_for_insertion" + for i in $(seq 0 $[inst_num-1]) + do + echo "cd ${database_dir}/$1${dir_server[i]}" + echo "${SERVER_BIND_CMD[i]} clickhouse server -C config_${1}${dir_server[i]}.xml >&${LOG_DIR}/${1}_${i}_server_log& > /dev/null" + + cd ${database_dir}/$1${dir_server[i]} + ${SERVER_BIND_CMD[i]} clickhouse server -C config_${1}${dir_server[i]}.xml >&${LOG_DIR}/${1}_${i}_server_log& > /dev/null + check_instance ${ckport[i]} + done +} + +function start_clickhouse_for_stressing(){ + echo "start_clickhouse_for_stressing" + for i in $(seq 0 $[inst_num-1]) + do + echo "cd ${database_dir}/$1${dir_server[i]}" + echo "${SERVER_BIND_CMD[i]} clickhouse server -C config_${1}${dir_server[i]}.xml >&/dev/null&" + + cd ${database_dir}/$1${dir_server[i]} + ${SERVER_BIND_CMD[i]} clickhouse server -C config_${1}${dir_server[i]}.xml >&/dev/null& + check_instance ${ckport[i]} + done +} +yum -y install git make gcc sudo net-tools &> /dev/null +pip3 install clickhouse_driver numpy &> /dev/null +test -d ${RAWDATA_DIR}/ssb-dbgen || git clone https://github.com/vadimtk/ssb-dbgen.git ${RAWDATA_DIR}/ssb-dbgen && cd ${RAWDATA_DIR}/ssb-dbgen + +if [ ! -f ${RAWDATA_DIR}/ssb-dbgen/dbgen ];then + make && { + test -f ${RAWDATA_DIR}/ssb-dbgen/customer.tbl || echo y |./dbgen -s ${SSB_GEN_FACTOR} -T c + test -f ${RAWDATA_DIR}/ssb-dbgen/part.tbl || echo y | ./dbgen -s ${SSB_GEN_FACTOR} -T p + test -f ${RAWDATA_DIR}/ssb-dbgen/supplier.tbl || echo y | ./dbgen -s ${SSB_GEN_FACTOR} -T s + test -f ${RAWDATA_DIR}/ssb-dbgen/date.tbl || echo y | ./dbgen -s ${SSB_GEN_FACTOR} -T d + test -f ${RAWDATA_DIR}/ssb-dbgen/lineorder.tbl || echo y | ./dbgen -s ${SSB_GEN_FACTOR} -T l + } +else + test -f ${RAWDATA_DIR}/ssb-dbgen/customer.tbl || echo y | ./dbgen -s ${SSB_GEN_FACTOR} -T c + test -f ${RAWDATA_DIR}/ssb-dbgen/part.tbl || echo y | ./dbgen -s ${SSB_GEN_FACTOR} -T p + test -f ${RAWDATA_DIR}/ssb-dbgen/supplier.tbl || echo y | ./dbgen -s ${SSB_GEN_FACTOR} -T s + test -f ${RAWDATA_DIR}/ssb-dbgen/date.tbl || echo y | ./dbgen -s ${SSB_GEN_FACTOR} -T d + test -f ${RAWDATA_DIR}/ssb-dbgen/lineorder.tbl || echo y | ./dbgen -s ${SSB_GEN_FACTOR} -T l + +fi + +filenum=`find ${RAWDATA_DIR}/ssb-dbgen/ -name "*.tbl" | wc -l` + +if [ $filenum -ne 5 ];then + echo "generate ssb data file *.tbl faild" + exit 1 +fi + +function kill_instance(){ +instance_alive=1 +for i in {1..2} +do + pkill clickhouse && sleep 5 + instance_alive=0 + for i in $(seq 0 $[inst_num-1]) + do + netstat -nltp | grep ${ckport[i]} > /dev/null + if [ $? -ne 1 ];then + instance_alive=1 + break; + fi + done + if [ $instance_alive -eq 0 ];then + break; + fi +done +if [ $instance_alive -eq 0 ];then + echo "kill_instance OK!" +else + echo "kill_instance Failed -> clickhouse server instance still alive due to 10s timeout" + exit 1 +fi +} + +function run_test(){ +is_xml=0 +for i in $(seq 0 $[inst_num-1]) +do + if [ -f ${database_dir}/${1}${dir_server[i]}/config_${1}${dir_server[i]}.xml ]; then + is_xml=$[is_xml+1] + fi +done +if [ $is_xml -eq $inst_num ];then + echo "Benchmark with $inst_num instance" + start_clickhouse_for_insertion ${1} + + for i in $(seq 0 $[inst_num-1]) + do + clickhouse client --host ${ckhost} --port ${ckport[i]} -m -q"show databases;" >/dev/null + done + + if [ $? -eq 0 ];then + check_table + fi + kill_instance + + if [ $1 == "deflate" ];then + test -f ${LOG_DIR}/${1}_server_log && deflatemsg=`cat ${LOG_DIR}/${1}_server_log | grep DeflateJobHWPool` + if [ -n "$deflatemsg" ];then + echo ------------------------------------------------------ + echo $deflatemsg + echo ------------------------------------------------------ + fi + fi + echo "Check table data required in server_${1} -> Done! " + + start_clickhouse_for_stressing ${1} + for i in $(seq 0 $[inst_num-1]) + do + clickhouse client --host ${ckhost} --port ${ckport[i]} -m -q"show databases;" >/dev/null + done + if [ $? -eq 0 ];then + test -d ${CLIENT_SCRIPTS_DIR} && cd ${CLIENT_SCRIPTS_DIR} + echo "Client stressing... " + echo "${CLIENT_BIND_CMD} python3 client_stressing_test.py ${QUERY_FILE} $inst_num &> ${LOG_DIR}/${1}.log" + ${CLIENT_BIND_CMD} python3 client_stressing_test.py ${QUERY_FILE} $inst_num &> ${LOG_DIR}/${1}.log + echo "Completed client stressing, checking log... " + finish_log=`grep "Finished" ${LOG_DIR}/${1}.log | wc -l` + if [ $finish_log -eq 1 ] ;then + kill_instance + test -f ${LOG_DIR}/${1}.log && echo "${1}.log ===> ${LOG_DIR}/${1}.log" + else + kill_instance + echo "No find 'Finished' in client log -> Performance test may fail" + exit 1 + + fi + + else + echo "${1} clickhouse server start fail" + exit 1 + fi +else + echo "clickhouse server start fail -> Please check xml files required in ${database_dir} for each instance" + exit 1 + +fi +} +function clear_log(){ + if [ -d "$LOG_DIR" ]; then + cd ${LOG_DIR} && rm -rf * + fi +} + +function gather_log_for_codec(){ + cd ${OUTPUT_DIR} && mkdir -p ${LOG_PACK_FILE}/${1} + cp -rf ${LOG_DIR} ${OUTPUT_DIR}/${LOG_PACK_FILE}/${1} +} + +function pack_log(){ + if [ -e "${OUTPUT_DIR}/run.log" ]; then + cp ${OUTPUT_DIR}/run.log ${OUTPUT_DIR}/${LOG_PACK_FILE}/ + fi + echo "Please check all log information in ${OUTPUT_DIR}/${LOG_PACK_FILE}" +} + +function setup_check(){ + + iax_dev_num=`accel-config list | grep iax | wc -l` + if [ $iax_dev_num -eq 0 ] ;then + iax_dev_num=`accel-config list | grep iax | wc -l` + if [ $iax_dev_num -eq 0 ] ;then + echo "No IAA devices available -> Please check IAA hardware setup manually!" + exit 1 + else + echo "IAA enabled devices number:$iax_dev_num" + fi + else + echo "IAA enabled devices number:$iax_dev_num" + fi + libaccel_version=`accel-config -v` + clickhouser_version=`clickhouse server --version` + kernel_dxd_log=`dmesg | grep dxd` + echo "libaccel_version:$libaccel_version" + echo "clickhouser_version:$clickhouser_version" + echo -e "idxd section in kernel log:\n$kernel_dxd_log" +} + +setup_check +export CLICKHOUSE_WATCHDOG_ENABLE=0 +for i in ${CODEC_CONFIG[@]} +do + clear_log + codec=${i} + echo "run test------------$codec" + run_test $codec + gather_log_for_codec $codec +done + +pack_log +echo "Done." \ No newline at end of file diff --git a/contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py b/contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py new file mode 100644 index 00000000000..b99a48b6a32 --- /dev/null +++ b/contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py @@ -0,0 +1,202 @@ +from operator import eq +import os +import random +import time +import sys +from clickhouse_driver import Client +import numpy as np +import subprocess +import multiprocessing +from multiprocessing import Manager + +warmup_runs = 10 +calculated_runs = 10 +seconds = 30 +max_instances_number = 8 +retest_number = 3 +retest_tolerance = 10 + +def checkInt(str): + try: + int(str) + return True + except ValueError: + return False + + +def setup_client(index): + if index<4: + port_idx = index + else: + port_idx = index + 4 + client = Client(host='localhost', database='default', user='default', password='',port='900%d'%port_idx) + union_mode_query = "SET union_default_mode='DISTINCT'" + client.execute(union_mode_query) + return client + +def warm_client(clientN, clientL, query, loop): + for c_idx in range(clientN): + for _ in range(loop): + clientL[c_idx].execute(query) + +def read_queries(queries_list): + queries = list() + queries_id = list() + with open(queries_list, 'r') as f: + for line in f: + line = line.rstrip() + line = line.split("$") + queries_id.append(line[0]) + queries.append(line[1]) + return queries_id, queries + +def run_task(client, cname, query, loop, query_latency): + start_time = time.time() + for i in range(loop): + client.execute(query) + query_latency.append(client.last_query.elapsed) + + end_time = time.time() + p95 = np.percentile(query_latency, 95) + print('CLIENT: {0} end. -> P95: %f, qps: %f'.format(cname)%(p95, loop / (end_time - start_time))) + +def run_multi_clients(clientN, clientList, query, loop): + client_pids={} + start_time = time.time() + manager = multiprocessing.Manager() + query_latency_list0 = manager.list() + query_latency_list1 = manager.list() + query_latency_list2 = manager.list() + query_latency_list3 = manager.list() + query_latency_list4 = manager.list() + query_latency_list5 = manager.list() + query_latency_list6 = manager.list() + query_latency_list7 = manager.list() + + for c_idx in range(clientN): + client_name = "Role_%d"%c_idx + if c_idx == 0: + client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list0)) + elif c_idx == 1: + client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list1)) + elif c_idx == 2: + client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list2)) + elif c_idx == 3: + client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list3)) + elif c_idx == 4: + client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list4)) + elif c_idx == 5: + client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list5)) + elif c_idx == 6: + client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list6)) + elif c_idx == 7: + client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list7)) + else: + print('ERROR: CLIENT number dismatch!!') + exit() + print('CLIENT: %s start'%client_name) + client_pids[c_idx].start() + + for c_idx in range(clientN): + client_pids[c_idx].join() + end_time = time.time() + totalT = end_time-start_time + + query_latencyTotal = list() + for item in query_latency_list0: + query_latencyTotal.append(item) + for item in query_latency_list1: + query_latencyTotal.append(item) + for item in query_latency_list2: + query_latencyTotal.append(item) + for item in query_latency_list3: + query_latencyTotal.append(item) + for item in query_latency_list4: + query_latencyTotal.append(item) + for item in query_latency_list5: + query_latencyTotal.append(item) + for item in query_latency_list6: + query_latencyTotal.append(item) + for item in query_latency_list7: + query_latencyTotal.append(item) + + totalP95 = np.percentile(query_latencyTotal, 95) * 1000 + return totalT,totalP95 + +def run_task_caculated(client, cname, query, loop): + query_latency = list() + start_time = time.time() + for i in range(loop): + client.execute(query) + query_latency.append(client.last_query.elapsed) + end_time = time.time() + p95 = np.percentile(query_latency, 95) + +def run_multi_clients_caculated(clientN, clientList, query, loop): + client_pids={} + start_time = time.time() + for c_idx in range(clientN): + client_name = "Role_%d"%c_idx + client_pids[c_idx] = multiprocessing.Process(target=run_task_caculated, args=(clientList[c_idx], client_name, query, loop)) + client_pids[c_idx].start() + for c_idx in range(clientN): + client_pids[c_idx].join() + end_time = time.time() + totalT = end_time-start_time + return totalT + +if __name__ == "__main__": + client_number = 1 + queries = list() + queries_id = list() + + if len(sys.argv) != 3: + print('usage: python3 client_stressing_test.py [queries_file_path] [client_number]') + sys.exit() + else: + queries_list = sys.argv[1] + client_number = int(sys.argv[2]) + print('queries_file_path: %s, client_number: %d'%(queries_list,client_number)) + if not os.path.isfile(queries_list) or not os.access(queries_list, os.R_OK): + print('please check the right path for queries file') + sys.exit() + if not checkInt(sys.argv[2]) or int(sys.argv[2]) > max_instances_number or int(sys.argv[2]) < 1: + print('client_number should be in [1~%d]'%max_instances_number) + sys.exit() + + client_list={} + queries_id,queries = read_queries(queries_list) + + for c_idx in range(client_number): + client_list[c_idx] = setup_client(c_idx) + #clear cache + os.system("sync; echo 3 > /proc/sys/vm/drop_caches") + + print("###Polit Run Begin") + for i in queries: + warm_client(client_number, client_list, i, 1) + print("###Polit Run End -> Start stressing....") + + query_index = 0 + for q in queries: + print("\n###START -> Index: %d, ID: %s, Query: %s" % (query_index,queries_id[query_index], q)) + warm_client(client_number, client_list, q, warmup_runs) + print("###Warm Done!") + for j in range(0,retest_number): + totalT = run_multi_clients_caculated(client_number, client_list, q, calculated_runs) + curr_loop = int (seconds * calculated_runs / totalT) + 1 + print("###Calculation Done! -> loopN: %d, expected seconds:%d" % (curr_loop, seconds)) + + print('###Stress Running! -> %d iterations......' % curr_loop) + + totalT,totalP95 = run_multi_clients(client_number, client_list, q, curr_loop) + + if totalT > (seconds - retest_tolerance) and totalT < (seconds + retest_tolerance): + break + else: + print('###totalT:%d is far way from expected seconds:%d. Run again ->j:%d!'%(totalT,seconds,j)) + + print( "###Completed! -> ID: %s, clientN: %d, totalT: %.2f s, latencyAVG: %.2f ms, P95: %.2f ms, QPS_Final: %.2f" % (queries_id[query_index], client_number, totalT, totalT * 1000/(curr_loop*client_number), totalP95, ((curr_loop*client_number)/totalT) ) ) + query_index += 1 + + print("###Finished!") \ No newline at end of file diff --git a/contrib/qpl-cmake/benchmark_sample/client_scripts/queries_ssb.sql b/contrib/qpl-cmake/benchmark_sample/client_scripts/queries_ssb.sql new file mode 100644 index 00000000000..abf2df6503a --- /dev/null +++ b/contrib/qpl-cmake/benchmark_sample/client_scripts/queries_ssb.sql @@ -0,0 +1,10 @@ +Q1.1$SELECT sum(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue FROM lineorder_flat WHERE toYear(LO_ORDERDATE) = 1993 AND LO_DISCOUNT BETWEEN 1 AND 3 AND LO_QUANTITY < 25; +Q2.1$SELECT sum(LO_REVENUE),toYear(LO_ORDERDATE) AS year,P_BRAND FROM lineorder_flat WHERE P_CATEGORY = 'MFGR#12' AND S_REGION = 'AMERICA' GROUP BY year,P_BRAND ORDER BY year,P_BRAND; +Q2.2$SELECT sum(LO_REVENUE),toYear(LO_ORDERDATE) AS year,P_BRAND FROM lineorder_flat WHERE P_BRAND >= 'MFGR#2221' AND P_BRAND <= 'MFGR#2228' AND S_REGION = 'ASIA' GROUP BY year,P_BRAND ORDER BY year,P_BRAND; +Q2.3$SELECT sum(LO_REVENUE),toYear(LO_ORDERDATE) AS year,P_BRAND FROM lineorder_flat WHERE P_BRAND = 'MFGR#2239' AND S_REGION = 'EUROPE' GROUP BY year,P_BRAND ORDER BY year,P_BRAND; +Q3.1$SELECT C_NATION,S_NATION,toYear(LO_ORDERDATE) AS year,sum(LO_REVENUE) AS revenue FROM lineorder_flat WHERE C_REGION = 'ASIA' AND S_REGION = 'ASIA' AND year >= 1992 AND year <= 1997 GROUP BY C_NATION,S_NATION,year ORDER BY year ASC,revenue DESC; +Q3.2$SELECT C_CITY,S_CITY,toYear(LO_ORDERDATE) AS year,sum(LO_REVENUE) AS revenue FROM lineorder_flat WHERE C_NATION = 'UNITED STATES' AND S_NATION = 'UNITED STATES' AND year >= 1992 AND year <= 1997 GROUP BY C_CITY,S_CITY,year ORDER BY year ASC,revenue DESC; +Q3.3$SELECT C_CITY,S_CITY,toYear(LO_ORDERDATE) AS year,sum(LO_REVENUE) AS revenue FROM lineorder_flat WHERE (C_CITY = 'UNITED KI1' OR C_CITY = 'UNITED KI5') AND (S_CITY = 'UNITED KI1' OR S_CITY = 'UNITED KI5') AND year >= 1992 AND year <= 1997 GROUP BY C_CITY,S_CITY,year ORDER BY year ASC,revenue DESC; +Q4.1$SELECT toYear(LO_ORDERDATE) AS year,C_NATION,sum(LO_REVENUE - LO_SUPPLYCOST) AS profit FROM lineorder_flat WHERE C_REGION = 'AMERICA' AND S_REGION = 'AMERICA' AND (P_MFGR = 'MFGR#1' OR P_MFGR = 'MFGR#2') GROUP BY year,C_NATION ORDER BY year ASC,C_NATION ASC; +Q4.2$SELECT toYear(LO_ORDERDATE) AS year,S_NATION,P_CATEGORY,sum(LO_REVENUE - LO_SUPPLYCOST) AS profit FROM lineorder_flat WHERE C_REGION = 'AMERICA' AND S_REGION = 'AMERICA' AND (year = 1997 OR year = 1998) AND (P_MFGR = 'MFGR#1' OR P_MFGR = 'MFGR#2') GROUP BY year,S_NATION,P_CATEGORY ORDER BY year ASC,S_NATION ASC,P_CATEGORY ASC; +Q4.3$SELECT toYear(LO_ORDERDATE) AS year,S_CITY,P_BRAND,sum(LO_REVENUE - LO_SUPPLYCOST) AS profit FROM lineorder_flat WHERE S_NATION = 'UNITED STATES' AND (year = 1997 OR year = 1998) AND P_CATEGORY = 'MFGR#14' GROUP BY year,S_CITY,P_BRAND ORDER BY year ASC,S_CITY ASC,P_BRAND ASC; diff --git a/contrib/qpl-cmake/benchmark_sample/client_scripts/run_ssb.sh b/contrib/qpl-cmake/benchmark_sample/client_scripts/run_ssb.sh new file mode 100644 index 00000000000..2bc2c5f5753 --- /dev/null +++ b/contrib/qpl-cmake/benchmark_sample/client_scripts/run_ssb.sh @@ -0,0 +1,6 @@ +WORKING_DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)/.." +if [ ! -d "${WORKING_DIR}/output" ]; then +mkdir ${WORKING_DIR}/output +fi +bash allin1_ssb.sh 2 > ${WORKING_DIR}/output/run.log +echo "Please check log in: ${WORKING_DIR}/output/run.log" diff --git a/contrib/qpl-cmake/benchmark_sample/database_dir/deflate/config_deflate.xml b/contrib/qpl-cmake/benchmark_sample/database_dir/deflate/config_deflate.xml new file mode 100644 index 00000000000..ab77a9cdcbe --- /dev/null +++ b/contrib/qpl-cmake/benchmark_sample/database_dir/deflate/config_deflate.xml @@ -0,0 +1,49 @@ + + + + + trace + true + + + 8123 + 9000 + 9004 + + ./ + + 8589934592 + 5368709120 + true + + + + deflate_qpl + + + + + + + + + ::/0 + + + default + default + 1 + + + + + + + + + + + diff --git a/contrib/qpl-cmake/benchmark_sample/database_dir/deflate_s2/config_deflate_s2.xml b/contrib/qpl-cmake/benchmark_sample/database_dir/deflate_s2/config_deflate_s2.xml new file mode 100644 index 00000000000..b71456486f5 --- /dev/null +++ b/contrib/qpl-cmake/benchmark_sample/database_dir/deflate_s2/config_deflate_s2.xml @@ -0,0 +1,49 @@ + + + + + trace + true + + + 8124 + 9001 + 9005 + + ./ + + 8589934592 + 5368709120 + true + + + + deflate_qpl + + + + + + + + + ::/0 + + + default + default + 1 + + + + + + + + + + + diff --git a/contrib/qpl-cmake/benchmark_sample/database_dir/lz4/config_lz4.xml b/contrib/qpl-cmake/benchmark_sample/database_dir/lz4/config_lz4.xml new file mode 100644 index 00000000000..f4dc59b60aa --- /dev/null +++ b/contrib/qpl-cmake/benchmark_sample/database_dir/lz4/config_lz4.xml @@ -0,0 +1,49 @@ + + + + + trace + true + + + 8123 + 9000 + 9004 + + ./ + + 8589934592 + 5368709120 + true + + + + lz4 + + + + + + + + + ::/0 + + + default + default + 1 + + + + + + + + + + + diff --git a/contrib/qpl-cmake/benchmark_sample/database_dir/lz4_s2/config_lz4_s2.xml b/contrib/qpl-cmake/benchmark_sample/database_dir/lz4_s2/config_lz4_s2.xml new file mode 100644 index 00000000000..357db8942d7 --- /dev/null +++ b/contrib/qpl-cmake/benchmark_sample/database_dir/lz4_s2/config_lz4_s2.xml @@ -0,0 +1,49 @@ + + + + + trace + true + + + 8124 + 9001 + 9005 + + ./ + + 8589934592 + 5368709120 + true + + + + lz4 + + + + + + + + + ::/0 + + + default + default + 1 + + + + + + + + + + + diff --git a/contrib/qpl-cmake/benchmark_sample/database_dir/zstd/config_zstd.xml b/contrib/qpl-cmake/benchmark_sample/database_dir/zstd/config_zstd.xml new file mode 100644 index 00000000000..1c4c738edaf --- /dev/null +++ b/contrib/qpl-cmake/benchmark_sample/database_dir/zstd/config_zstd.xml @@ -0,0 +1,49 @@ + + + + + trace + true + + + 8123 + 9000 + 9004 + + ./ + + 8589934592 + 5368709120 + true + + + + zstd + + + + + + + + + ::/0 + + + default + default + 1 + + + + + + + + + + + diff --git a/contrib/qpl-cmake/benchmark_sample/database_dir/zstd_s2/config_zstd_s2.xml b/contrib/qpl-cmake/benchmark_sample/database_dir/zstd_s2/config_zstd_s2.xml new file mode 100644 index 00000000000..f3db01b7739 --- /dev/null +++ b/contrib/qpl-cmake/benchmark_sample/database_dir/zstd_s2/config_zstd_s2.xml @@ -0,0 +1,49 @@ + + + + + trace + true + + + 8124 + 9001 + 9005 + + ./ + + 8589934592 + 5368709120 + true + + + + zstd + + + + + + + + + ::/0 + + + default + default + 1 + + + + + + + + + + + diff --git a/contrib/qpl-cmake/doc/benchmark.md b/contrib/qpl-cmake/doc/benchmark.md new file mode 100644 index 00000000000..4357b3bafb2 --- /dev/null +++ b/contrib/qpl-cmake/doc/benchmark.md @@ -0,0 +1,270 @@ +## Files listed +The folders under ./qpl-cmake/benchmark_sample give example to run benchmark with python scripts: + +"client_scripts" contains python scripts for running typical benchmark, for example: +- "client_stressing_test.py": The python script for query stress test with [1~4] server instances. +- "queries_ssb.sql": The file lists all queries for [Star Schema Benchmark](https://clickhouse.com/docs/en/getting-started/example-datasets/star-schema/) +- "allin1_ssb.sh": This shell script executes benchmark workflow all in one automatically. +"database_files" means it will store database files according to lz4/deflate/zstd codecs. + +"rawdata_dir" means it will store raw data generated by dbgen. + +## Run benchmark automatically for Start Schema: +``` bash +$ cd ./benchmark_sample/client_scripts +$ sh run_ssb.sh +``` +After complete, please check all the results in this folder:"./output/" + +In case you run into failure, please fall back to "Run benchmark manually" as below sections. + +## Definition +[CLICKHOUSE_EXE] means the path of clickhouse executable program. + +## Enviroment +- CPU: Sapphire Rapid +- OS Requirements refer to [QPL Hardware Path Requirements](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#hardware-path-requirements) +- Set up IAA devices refer to [Accelerator Configuration](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#accelerator-configuration) +- Install python modules: pip3 install clickhouse_driver numpy + +[Self-check for IAA] +``` bash +$ accel-config list | grep -P 'iax|state' +``` +Expected output like this: +``` bash + "dev":"iax1", + "state":"enabled", + "state":"enabled", +``` +If you see nothing output, it means IAA is not ready to work. Please check IAA setup again. + +## Generate raw data +``` bash +$ cd ./rawdata_dir +``` +Use [dbgen](https://clickhouse.com/docs/en/getting-started/example-datasets/star-schema) to generate 100 million rows data with the parameters: +-s 20 + +The files expected to output "./rawdata_dir": +customer.tbl +part.tbl +supplier.tbl +lineorder.tbl + +## Database setup +Set up database with LZ4 codec + +``` bash +$ cd ./database_dir/lz4 +$ [CLICKHOUSE_EXE] server -C config_lz4.xml >&/dev/null& +$ [CLICKHOUSE_EXE] client +``` +Here you should see the message "Connected to ClickHouse server " from console which means client successfuly setup connection with server. + +Complete below three steps mentioned in [Star Schema Benchmark](https://clickhouse.com/docs/en/getting-started/example-datasets/star-schema) +- Creating tables in ClickHouse +- Inserting data. Here should use ./rawdata_dir/*.tbl as input data. +- Converting “star schema” to denormalized “flat schema” + +Set up database with with IAA Deflate codec + +``` bash +$ cd ./database_dir/deflate +$ [CLICKHOUSE_EXE] server -C config_deflate.xml >&/dev/null& +$ [CLICKHOUSE_EXE] client +``` +Complete three steps same as lz4 above + +Set up database with with ZSTD codec + +``` bash +$ cd ./database_dir/zstd +$ [CLICKHOUSE_EXE] server -C config_zstd.xml >&/dev/null& +$ [CLICKHOUSE_EXE] client +``` +Complete three steps same as lz4 above + +[self-check] +For each codec(lz4/zstd/deflate), please execute below query to make sure the databases are created successfully: +$ select count() from lineorder_flat +You are expected to see below output: +```sql +┌───count()─┐ +│ 119994608 │ +└───────────┘ +``` +[Self-check for IAA Deflate codec] +At the first time you execute insertion or query from client, clickhouse server console is expected to print this log: +```text +"Hardware-assisted DeflateQpl codec is ready!" +``` +If you never find this, but see another log as below: +```text +"Initialization of hardware-assisted DeflateQpl codec failed" +``` +That means IAA devices is not ready, you need check IAA setup again. + +## Benchmark with single instance +- Before start benchmark, Please disable C6 and set CPU frequency governor to be "performance" +``` bash +$ cpupower idle-set -d 3 +$ cpupower frequency-set -g performance +``` +- To eliminate impact of memory bound on cross sockets, we use numactl to bind server on one socket and client on another socket. +- Single instance means single server connected with single client + +Now run benchmark for LZ4/Deflate/ZSTD respectively: + +LZ4: +``` bash +$ cd ./database_dir/lz4 +$ numactl -m 0 -N 0 [CLICKHOUSE_EXE] server -C config_lz4.xml >&/dev/null& +$ cd ./client_scripts +$ numactl -m 1 -N 1 python3 client_stressing_test.py queries_ssb.sql 1 > lz4.log +``` + +IAA deflate: +``` bash +$ cd ./database_dir/deflate +$ numactl -m 0 -N 0 [CLICKHOUSE_EXE] server -C config_deflate.xml >&/dev/null& +$ cd ./client_scripts +$ numactl -m 1 -N 1 python3 client_stressing_test.py queries_ssb.sql 1 > deflate.log +``` +ZSTD: +``` bash +$ cd ./database_dir/zstd +$ numactl -m 0 -N 0 [CLICKHOUSE_EXE] server -C config_zstd.xml >&/dev/null& +$ cd ./client_scripts +$ numactl -m 1 -N 1 python3 client_stressing_test.py queries_ssb.sql 1 > zstd.log +``` + +Now three logs should be output as expected: +```text +lz4.log +deflate.log +zstd.log +``` + +How to check performance metrics: + +We focus on QPS, please search the keyword: "QPS_Final" and collect statistics + +## Benchmark with multi-instances +- To reduce impact of memory bound on too much threads, We recommend run benchmark with multi-instances. +- Multi-instance means multiple(2 or 4)servers connected with respective client. +- The cores of one socket need to be divided equally and assiged to the servers respectively. +- For multi-instances, must create new folder for each codec and insert dataset by following the similiar steps as single instance. + +There are 2 differences: +- For client side, you need launch clickhouse with the assigned port during table creation and data insertion. +- For server side, you need launch clickhouse with the specific xml config file in which port has been assigned. All customized xml config files for multi-instances has been provided under ./server_config. + +Here we assume there are 60 cores per socket and take 2 instances for example. +Launch server for first instance +LZ4: +``` bash +$ cd ./database_dir/lz4 +$ numactl -C 0-29,120-149 [CLICKHOUSE_EXE] server -C config_lz4.xml >&/dev/null& +``` +ZSTD: +``` bash +$ cd ./database_dir/zstd +$ numactl -C 0-29,120-149 [CLICKHOUSE_EXE] server -C config_zstd.xml >&/dev/null& +``` +IAA Deflate: +``` bash +$ cd ./database_dir/deflate +$ numactl -C 0-29,120-149 [CLICKHOUSE_EXE] server -C config_deflate.xml >&/dev/null& +``` +[Launch server for second instance] + +LZ4: +``` bash +$ cd ./database_dir && mkdir lz4_s2 && cd lz4_s2 +$ cp ../../server_config/config_lz4_s2.xml ./ +$ numactl -C 30-59,150-179 [CLICKHOUSE_EXE] server -C config_lz4_s2.xml >&/dev/null& +``` +ZSTD: +``` bash +$ cd ./database_dir && mkdir zstd_s2 && cd zstd_s2 +$ cp ../../server_config/config_zstd_s2.xml ./ +$ numactl -C 30-59,150-179 [CLICKHOUSE_EXE] server -C config_zstd_s2.xml >&/dev/null& +``` +IAA Deflate: +``` bash +$ cd ./database_dir && mkdir deflate_s2 && cd deflate_s2 +$ cp ../../server_config/config_deflate_s2.xml ./ +$ numactl -C 30-59,150-179 [CLICKHOUSE_EXE] server -C config_deflate_s2.xml >&/dev/null& +``` + +Creating tables && Inserting data for second instance + +Creating tables: +``` bash +$ [CLICKHOUSE_EXE] client -m --port=9001 +``` +Inserting data: +``` bash +$ [CLICKHOUSE_EXE] client --query "INSERT INTO customer FORMAT CSV" < customer.tbl --port=9001 +$ [CLICKHOUSE_EXE] client --query "INSERT INTO part FORMAT CSV" < part.tbl --port=9001 +$ [CLICKHOUSE_EXE] client --query "INSERT INTO supplier FORMAT CSV" < supplier.tbl --port=9001 +$ [CLICKHOUSE_EXE] client --query "INSERT INTO lineorder FORMAT CSV" < lineorder.tbl --port=9001 +``` +Here "--port=9002" stands for the assigned port for server instance which is also defined in config_lz4_s2.xml/config_zstd_s2.xml/config_deflate_s2.xml + +For even more instances, you need replace it with the value: 9002/9003 which stand for s3/s4 instance respectively. + +if you don't assign it, the port is 9000 by default which has been used by first instance. + +Benchmarking with 2 instances + +LZ4: +``` bash +$ cd ./database_dir/lz4 +$ numactl -C 0-29,120-149 [CLICKHOUSE_EXE] server -C config_lz4.xml >&/dev/null& +$ cd ./database_dir/lz4_s2 +$ numactl -C 30-59,150-179 [CLICKHOUSE_EXE] server -C config_lz4_s2.xml >&/dev/null& +$ cd ./client_scripts +$ numactl -m 1 -N 1 python3 client_stressing_test.py queries_ssb.sql 2 > lz4_2insts.log +``` +ZSTD: +``` bash +$ cd ./database_dir/zstd +$ numactl -C 0-29,120-149 [CLICKHOUSE_EXE] server -C config_zstd.xml >&/dev/null& +$ cd ./database_dir/zstd_s2 +$ numactl -C 30-59,150-179 [CLICKHOUSE_EXE] server -C config_zstd_s2.xml >&/dev/null& +$ cd ./client_scripts +$ numactl -m 1 -N 1 python3 client_stressing_test.py queries_ssb.sql 2 > zstd_2insts.log +``` +IAA deflate +``` bash +$ cd ./database_dir/deflate +$ numactl -C 0-29,120-149 [CLICKHOUSE_EXE] server -C config_deflate.xml >&/dev/null& +$ cd ./database_dir/deflate_s2 +$ numactl -C 30-59,150-179 [CLICKHOUSE_EXE] server -C config_deflate_s2.xml >&/dev/null& +$ cd ./client_scripts +$ numactl -m 1 -N 1 python3 client_stressing_test.py queries_ssb.sql 2 > deflate_2insts.log +``` +Here the last argument: "2" of client_stressing_test.py stands for the number of instances. For more instances, you need replace it with the value: 3 or 4. This script support up to 4 instances/ + +Now three logs should be output as expected: +``` text +lz4_2insts.log +deflate_2insts.log +zstd_2insts.log +``` +How to check performance metrics: + +We focus on QPS, please search the keyword: "QPS_Final" and collect statistics + +Benchmark setup for 4 instances is similar with 2 instances above. +We recommend use 2 instances benchmark data as final report for review. + +## Tips +Each time before launch new clickhouse server, please make sure no background clickhouse process running, please check and kill old one: +``` bash +$ ps -aux| grep clickhouse +$ kill -9 [PID] +``` +By comparing the query list in ./client_scripts/queries_ssb.sql with official [Star Schema Benchmark](https://clickhouse.com/docs/en/getting-started/example-datasets/star-schema), you will find 3 queries are not included: Q1.2/Q1.3/Q3.4 . This is because cpu utilization% is very low <10% for these queries which means cannot demostrate performance difference for codecs. \ No newline at end of file diff --git a/contrib/qpl-cmake/doc/build.md b/contrib/qpl-cmake/doc/build.md new file mode 100644 index 00000000000..8ba3ed0f1b0 --- /dev/null +++ b/contrib/qpl-cmake/doc/build.md @@ -0,0 +1,14 @@ +This doc gives specfic build tips for QPL_Deflate codec based on Clickhouse generic [build instructions](https://github.com/ClickHouse/ClickHouse/blob/master/docs/en/development/build.md) + +Several tips to build Clickhouse with QPL_deflate enabled: +- The QPL build only support x86_64 platform with avx2/avx512 support. +- The QPL build requires nasm 2.15.0 or higher (e.g., can be obtained from https://www.nasm.us) +- The QPL requires C++ compiler with C++17 standard support. +- Pass the following flag to CMake. Which one to choose depends on your target hardware supported with AVX2 or AVX512. +``` bash +cmake -DENABLE_AVX2=1 -DENABLE_QPL=1 .. +``` +or +``` bash +cmake -DENABLE_AVX512=1 -DENABLE_QPL=1 .. +``` \ No newline at end of file diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index de39d960476..556d1fb9e93 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -379,8 +379,8 @@ High compression levels are useful for asymmetric scenarios, like compress once, `DEFLATE_QPL` — [Deflate compression algorithm](https://github.com/intel/qpl) implemented by Intel® Query Processing Library. Some limitations apply: - DEFLATE_QPL is experimental and can only be used after setting configuration parameter `allow_experimental_codecs=1`. -- DEFLATE_QPL only works if ClickHouse was compiled with support for AVX2 or AVX512 instructions -- DEFLATE_QPL works best if the system has a Intel® IAA (In-Memory Analytics Accelerator) offloading device +- DEFLATE_QPL only works if ClickHouse was compiled with support for AVX2 or AVX512 instructions. Refer to [Build Tips](https://github.com/ClickHouse/ClickHouse/tree/master/contrib/qpl-cmake/doc/build.md) for more details. +- DEFLATE_QPL works best if the system has a Intel® IAA (In-Memory Analytics Accelerator) offloading device. Refer to [IAA Setup](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#accelerator-configuration) and [IAA Benchmark](https://github.com/ClickHouse/ClickHouse/tree/master/contrib/qpl-cmake/doc/benchmark.md) for more details. - DEFLATE_QPL-compressed data can only be transferred between ClickHouse nodes compiled with support for AVX2/AVX512 ### Specialized Codecs From c99e74248287fe890381b766f9ecc3fa65bf6fc2 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 20 Mar 2023 20:45:31 +0100 Subject: [PATCH 194/418] impl --- src/Processors/QueryPlan/PartsSplitter.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/PartsSplitter.cpp b/src/Processors/QueryPlan/PartsSplitter.cpp index 917bea4c884..f86879e474d 100644 --- a/src/Processors/QueryPlan/PartsSplitter.cpp +++ b/src/Processors/QueryPlan/PartsSplitter.cpp @@ -94,7 +94,8 @@ std::pair, std::vector> split(RangesInDat parts_ranges_queue.push( {index_access->getValue(part_idx, range.begin), {range, part_idx}, PartsRangesIterator::EventType::RangeStart}); const auto & index_granularity = parts[part_idx].data_part->index_granularity; - if (index_granularity.hasFinalMark() && range.end + 1 == index_granularity.getMarksCount()) + if (range.end + 1 < index_granularity.getMarksCount() + || (index_granularity.hasFinalMark() && range.end + 1 == index_granularity.getMarksCount())) parts_ranges_queue.push( {index_access->getValue(part_idx, range.end), {range, part_idx}, PartsRangesIterator::EventType::RangeEnd}); } From adb83074a249bceb3a9ba6f42258dc788e9c72d5 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 20 Mar 2023 22:42:54 +0100 Subject: [PATCH 195/418] add test --- ...2681_final_excessive_reading_bug.reference | 1 + .../02681_final_excessive_reading_bug.sh | 24 +++++++++++++++++++ 2 files changed, 25 insertions(+) create mode 100644 tests/queries/0_stateless/02681_final_excessive_reading_bug.reference create mode 100755 tests/queries/0_stateless/02681_final_excessive_reading_bug.sh diff --git a/tests/queries/0_stateless/02681_final_excessive_reading_bug.reference b/tests/queries/0_stateless/02681_final_excessive_reading_bug.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02681_final_excessive_reading_bug.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02681_final_excessive_reading_bug.sh b/tests/queries/0_stateless/02681_final_excessive_reading_bug.sh new file mode 100755 index 00000000000..a795b9ec5a0 --- /dev/null +++ b/tests/queries/0_stateless/02681_final_excessive_reading_bug.sh @@ -0,0 +1,24 @@ +#!/usr/bin/env bash +# Tags: no-random-merge-tree-settings + +# shellcheck disable=SC2154 + +unset CLICKHOUSE_LOG_COMMENT + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +$CLICKHOUSE_CLIENT -q "CREATE TABLE sample_final (CounterID UInt32, EventDate Date, EventTime DateTime, UserID UInt64, Sign Int8) ENGINE = CollapsingMergeTree(Sign) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime) SAMPLE BY intHash32(UserID)" + +$CLICKHOUSE_CLIENT -q "INSERT INTO sample_final SELECT number / (8192 * 4), toDate('2019-01-01'), toDateTime('2019-01-01 00:00:01') + number, number / (8192 * 2), if((number % 3) = 1, -1, 1) FROM numbers(1000000)" + +query_id="${CLICKHOUSE_DATABASE}_final_excessive_reading_bug_$RANDOM" +$CLICKHOUSE_CLIENT --query_id="$query_id" -q "select * from sample_final FINAL SAMPLE 1/2 OFFSET 1/2 format Null settings max_threads=16" + +$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" +$CLICKHOUSE_CLIENT --param_query_id="$query_id" -q " +SELECT ProfileEvents['SelectedRows'] < 1_000_000 + FROM system.query_log + WHERE event_date >= yesterday() AND type = 'QueryFinish' AND query_id = {query_id:String} AND current_database = currentDatabase()" From d5a561a2efad0e0795c49394d3d08f7ade57ea58 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 20 Mar 2023 22:51:32 +0100 Subject: [PATCH 196/418] fix 993 and two other tests --- .../ReplicatedMergeTreePartCheckThread.cpp | 51 +++++++++++++++---- src/Storages/StorageReplicatedMergeTree.cpp | 26 ++++++---- .../02434_cancel_insert_when_client_dies.sh | 4 +- .../02435_rollback_cancelled_queries.sh | 14 ++--- 4 files changed, 67 insertions(+), 28 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index 3120205b4e3..8eafc54cb4c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -76,21 +76,50 @@ std::unique_lock ReplicatedMergeTreePartCheckThread::pausePartsCheck void ReplicatedMergeTreePartCheckThread::cancelRemovedPartsCheck(const MergeTreePartInfo & drop_range_info) { - std::lock_guard lock(parts_mutex); - for (auto it = parts_queue.begin(); it != parts_queue.end();) + Strings removed_names; { - if (drop_range_info.contains(MergeTreePartInfo::fromPartName(it->first, storage.format_version))) + std::lock_guard lock(parts_mutex); + removed_names.reserve(parts_queue.size()); /// Avoid memory limit in the middle + for (auto it = parts_queue.begin(); it != parts_queue.end();) { - /// Remove part from the queue to avoid part resurrection - /// if we will check it and enqueue fetch after DROP/REPLACE execution. - parts_set.erase(it->first); - it = parts_queue.erase(it); - } - else - { - ++it; + if (drop_range_info.contains(MergeTreePartInfo::fromPartName(it->first, storage.format_version))) + { + /// Remove part from the queue to avoid part resurrection + /// if we will check it and enqueue fetch after DROP/REPLACE execution. + removed_names.push_back(it->first); + parts_set.erase(it->first); + it = parts_queue.erase(it); + } + else + { + ++it; + } } } + + /// This filtering is not necessary + auto new_end = std::remove_if(removed_names.begin(), removed_names.end(), [this](const String & part_name) + { + auto part = storage.getPartIfExists(part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated, MergeTreeDataPartState::Deleting}); + /// The rest of parts will be removed normally + return part && !part->outdated_because_broken; + }); + removed_names.erase(new_end, removed_names.end()); + if (removed_names.empty()) + return; + + try + { + /// We have to remove parts that were not removed by removePartAndEnqueueFetch + LOG_INFO(log, "Removing broken parts from ZooKeeper: {}", fmt::join(removed_names, ", ")); + storage.removePartsFromZooKeeperWithRetries(removed_names, /* max_retries */ 100); + } + catch (...) + { + /// It's highly unlikely to happen on normal use cases. And if it happens it's easier to restart and reinitialize + LOG_FATAL(log, "Failed to remove parts [{}] from ZooKeeper: {}", fmt::join(removed_names, ", "), getCurrentExceptionMessage(/* with_stacktrace = */ true)); + std::terminate(); + } } size_t ReplicatedMergeTreePartCheckThread::size() const diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 9092f251b6d..ffe17ac7595 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2752,7 +2752,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo const String & entry_name = entry_info.parsed_entry->znode_name; const auto & entry_type = entry_info.parsed_entry->type; - if (should_ignore_log_entry(created_get_parts, part_name, fmt::format("Not copying {} {} ", entry_name, entry_type))) + if (should_ignore_log_entry(created_get_parts, part_name, fmt::format("Not copying {} {}", entry_name, entry_type))) continue; if (entry_info.parsed_entry->type == LogEntry::GET_PART) @@ -3534,7 +3534,8 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n time_t part_create_time = 0; Coordination::Stat stat; - if (zookeeper->exists(part_path, &stat)) + bool exists_in_zookeeper = zookeeper->exists(part_path, &stat); + if (exists_in_zookeeper) { /// Update version of /is_lost node to avoid race condition with cloneReplica(...). /// cloneReplica(...) expects that if some entry was executed, then its new_part_name is added to /parts, @@ -3548,13 +3549,6 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n getRemovePartFromZooKeeperOps(part_name, ops, stat.numChildren > 0); } - - LogEntryPtr log_entry = std::make_shared(); - log_entry->type = LogEntry::GET_PART; - log_entry->create_time = part_create_time; - log_entry->source_replica = ""; - log_entry->new_part_name = part_name; - if (!storage_init) { /// We use merge predicate + version check here, because DROP RANGE update log version and we are trying to avoid race with it. We must be sure, that our part @@ -3570,13 +3564,25 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n ReplicatedMergeTreeMergePredicate merge_pred = queue.getMergePredicate(zookeeper, PartitionIdsHint{broken_part_info.partition_id}); if (merge_pred.isGoingToBeDropped(broken_part_info)) { - LOG_INFO(log, "Broken part {} is covered by drop range, don't need to fetch it", part_name); + LOG_INFO(log, "Broken part {} is covered by drop range, don't need to fetch it, removing it from ZooKeeper", part_name); + + /// But we have to remove it from ZooKeeper because broken parts are not removed from ZK during Outdated parts cleanup + /// There's a chance that DROP_RANGE will remove it, but only if it was not already removed by cleanup thread + if (exists_in_zookeeper) + removePartsFromZooKeeperWithRetries({part_name}); return; } + /// Check that our version of log (and queue) is the most fresh. Otherwise don't create new entry fetch entry. ops.emplace_back(zkutil::makeCheckRequest(fs::path(zookeeper_path) / "log", merge_pred.getVersion())); } + LogEntryPtr log_entry = std::make_shared(); + log_entry->type = LogEntry::GET_PART; + log_entry->create_time = part_create_time; + log_entry->source_replica = ""; + log_entry->new_part_name = part_name; + ops.emplace_back(zkutil::makeCreateRequest( fs::path(replica_path) / "queue/queue-", log_entry->toString(), zkutil::CreateMode::PersistentSequential)); 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 2b27d88f23c..5e2da509314 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 @@ -46,9 +46,11 @@ $CLICKHOUSE_CLIENT -q 'select count() from dedup_test' function thread_insert { # supress "Killed" messages from bash + i=0 while true; do - export ID="$TEST_MARK$RANDOM-$RANDOM-$RANDOM" + export ID="$TEST_MARK$RANDOM-$RANDOM-$i" bash -c insert_data 2>&1| grep -Fav "Killed" + i=$((i + 1)) done } diff --git a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh index 640975f1ef7..25d12e75a22 100755 --- a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh +++ b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh @@ -16,7 +16,7 @@ $CLICKHOUSE_CLIENT -q 'create table dedup_test(A Int64) Engine = MergeTree order function insert_data { IMPLICIT=$(( RANDOM % 2 )) - SESSION_ID="${SESSION}_$RANDOM.$RANDOM.$RANDOM" + SESSION_ID="${SESSION}_$RANDOM.$RANDOM.$1" TXN_SETTINGS="session_id=$SESSION_ID&throw_on_unsupported_query_inside_transaction=0&implicit_transaction=$IMPLICIT" BEGIN="" COMMIT="" @@ -47,22 +47,24 @@ function insert_data fi if [[ "$IMPLICIT" -eq 0 ]]; then - $CLICKHOUSE_CURL -sS -d 'commit' "$CLICKHOUSE_URL&$TXN_SETTINGS&close_session=1" + $CLICKHOUSE_CURL -sS -d 'commit' "$CLICKHOUSE_URL&$TXN_SETTINGS&close_session=1" 2>&1| grep -Fav "Transaction is not in RUNNING state" fi } export -f insert_data ID="02435_insert_init_${CLICKHOUSE_DATABASE}_$RANDOM" -insert_data +insert_data 0 $CLICKHOUSE_CLIENT -q 'select count() from dedup_test' function thread_insert { # supress "Killed" messages from bash + i=2 while true; do - 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" + export ID="$TEST_MARK$RANDOM-$RANDOM-$i" + bash -c insert_data "$i" 2>&1| grep -Fav "Killed" | grep -Fav "SESSION_IS_LOCKED" | grep -Fav "SESSION_NOT_FOUND" + i=$((i + 1)) done } @@ -102,7 +104,7 @@ wait $CLICKHOUSE_CLIENT -q 'system flush logs' ID="02435_insert_last_${CLICKHOUSE_DATABASE}_$RANDOM" -insert_data +insert_data 1 $CLICKHOUSE_CLIENT --implicit_transaction=1 -q 'select throwIf(count() % 1000000 != 0 or count() = 0) from dedup_test' \ || $CLICKHOUSE_CLIENT -q "select name, rows, active, visible, creation_tid, creation_csn from system.parts where database=currentDatabase();" From e1d1ead9416b418dfc0374ca42932a1a4cd68962 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 8 Mar 2023 17:13:35 +0100 Subject: [PATCH 197/418] orphaned role - tests only --- programs/server/config.xml | 3 + src/Access/AccessControl.cpp | 3 +- src/Access/RoleCache.cpp | 4 +- src/Access/RoleCache.h | 2 +- .../helpers/0_common_instance_config.xml | 1 + tests/integration/test_role/test.py | 61 +++++++++++++++++++ 6 files changed, 70 insertions(+), 4 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 0ea2de18e22..09e09453660 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -703,6 +703,9 @@ actions of previous constraint (defined in other profiles) for the same specific setting, including fields that are not set by new constraint. It also enables 'changeable_in_readonly' constraint type --> false + + + 600 diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 91283da241d..dfb65721a0a 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -247,7 +247,6 @@ private: AccessControl::AccessControl() : MultipleAccessStorage("user directories"), context_access_cache(std::make_unique(*this)), - role_cache(std::make_unique(*this)), row_policy_cache(std::make_unique(*this)), quota_cache(std::make_unique(*this)), settings_profiles_cache(std::make_unique(*this)), @@ -282,6 +281,8 @@ void AccessControl::setUpFromMainConfig(const Poco::Util::AbstractConfiguration setSettingsConstraintsReplacePrevious(config_.getBool("access_control_improvements.settings_constraints_replace_previous", false)); addStoragesFromMainConfig(config_, config_path_, get_zookeeper_function_); + + role_cache = std::make_unique(*this, config_.getInt("access_control_improvements.role_cache_expiration_time", 600)); } diff --git a/src/Access/RoleCache.cpp b/src/Access/RoleCache.cpp index bfc6200929d..3f5089c540a 100644 --- a/src/Access/RoleCache.cpp +++ b/src/Access/RoleCache.cpp @@ -56,8 +56,8 @@ namespace } -RoleCache::RoleCache(const AccessControl & access_control_) - : access_control(access_control_), cache(600000 /* 10 minutes */) +RoleCache::RoleCache(const AccessControl & access_control_, int expiration_time) + : access_control(access_control_), cache(expiration_time * 1000 /* 10 minutes by default*/) { } diff --git a/src/Access/RoleCache.h b/src/Access/RoleCache.h index 24f19cb9d94..b5c7406e2c4 100644 --- a/src/Access/RoleCache.h +++ b/src/Access/RoleCache.h @@ -16,7 +16,7 @@ using RolePtr = std::shared_ptr; class RoleCache { public: - explicit RoleCache(const AccessControl & access_control_); + explicit RoleCache(const AccessControl & access_control_, int expiration_time); ~RoleCache(); std::shared_ptr getEnabledRoles( diff --git a/tests/integration/helpers/0_common_instance_config.xml b/tests/integration/helpers/0_common_instance_config.xml index 27563e47c35..e7673224849 100644 --- a/tests/integration/helpers/0_common_instance_config.xml +++ b/tests/integration/helpers/0_common_instance_config.xml @@ -25,5 +25,6 @@ true true true + 6 diff --git a/tests/integration/test_role/test.py b/tests/integration/test_role/test.py index 8e94b1620b5..0bbd62f7425 100644 --- a/tests/integration/test_role/test.py +++ b/tests/integration/test_role/test.py @@ -1,3 +1,4 @@ +import time import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster @@ -412,3 +413,63 @@ def test_function_current_roles(): ) == "['R1']\t['R1']\t['R1']\n" ) + +def test_role_expiration(): + instance.query("CREATE USER ure") + instance.query("CREATE ROLE rre") + instance.query("GRANT rre TO ure") + + instance.query("CREATE TABLE IF NOT EXISTS tre (id Int) Engine=Log") + instance.query("INSERT INTO tre VALUES (0)") + + assert "Not enough privileges" in instance.query_and_get_error( + "SELECT * FROM tre", user="ure" + ) + + instance.query("GRANT SELECT ON tre TO rre") + + assert instance.query("SELECT * FROM tre", user="ure") == "0\n" + time.sleep(10) # wait for role expiration + + instance.query("CREATE TABLE IF NOT EXISTS tre1 (id Int) Engine=Log") + instance.query("INSERT INTO tre1 VALUES (0)") + instance.query("GRANT SELECT ON tre1 TO rre") + + assert instance.query("SELECT * from tre1", user="ure") == "0\n" + + instance.query("DROP USER ure") + instance.query("DROP ROLE rre") + instance.query("DROP TABLE tre") + instance.query("DROP TABLE tre1") + +def test_two_roles_expiration(): + instance.query("CREATE USER ure") + instance.query("CREATE ROLE rre") + instance.query("GRANT rre TO ure") + + instance.query("CREATE ROLE rre_second") + + instance.query("CREATE TABLE IF NOT EXISTS tre (id Int) Engine=Log") + instance.query("INSERT INTO tre VALUES (0)") + + assert "Not enough privileges" in instance.query_and_get_error( + "SELECT * FROM tre", user="ure" + ) + + instance.query("GRANT SELECT ON tre TO rre") + + assert instance.query("SELECT * FROM tre", user="ure") == "0\n" + time.sleep(10) # wait for role expiration + instance.query("GRANT SELECT ON tre1 TO rre_second") # we expect that both rre and rre_second are gone from cache + + instance.query("CREATE TABLE IF NOT EXISTS tre1 (id Int) Engine=Log") + instance.query("INSERT INTO tre1 VALUES (0)") + instance.query("GRANT SELECT ON tre1 TO rre") + + assert instance.query("SELECT * from tre1", user="ure") == "0\n" + + instance.query("DROP USER ure") + instance.query("DROP ROLE rre") + instance.query("DROP ROLE rre_second") + instance.query("DROP TABLE tre") + instance.query("DROP TABLE tre1") From c9691b943d89b234d8ee76f8191bbb8097c3572e Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 8 Mar 2023 17:27:59 +0100 Subject: [PATCH 198/418] orphaned role - make stylecheck happy --- tests/integration/test_role/test.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_role/test.py b/tests/integration/test_role/test.py index 0bbd62f7425..0fb10989e68 100644 --- a/tests/integration/test_role/test.py +++ b/tests/integration/test_role/test.py @@ -414,6 +414,7 @@ def test_function_current_roles(): == "['R1']\t['R1']\t['R1']\n" ) + def test_role_expiration(): instance.query("CREATE USER ure") instance.query("CREATE ROLE rre") @@ -442,6 +443,7 @@ def test_role_expiration(): instance.query("DROP TABLE tre") instance.query("DROP TABLE tre1") + def test_two_roles_expiration(): instance.query("CREATE USER ure") instance.query("CREATE ROLE rre") @@ -460,8 +462,9 @@ def test_two_roles_expiration(): assert instance.query("SELECT * FROM tre", user="ure") == "0\n" time.sleep(10) # wait for role expiration - instance.query("GRANT SELECT ON tre1 TO rre_second") # we expect that both rre and rre_second are gone from cache - + instance.query( + "GRANT SELECT ON tre1 TO rre_second" + ) # we expect that both rre and rre_second are gone from cache instance.query("CREATE TABLE IF NOT EXISTS tre1 (id Int) Engine=Log") instance.query("INSERT INTO tre1 VALUES (0)") instance.query("GRANT SELECT ON tre1 TO rre") From 4166b3bb098383d9c6541aacd289c24a2f0f5789 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 8 Mar 2023 18:24:27 +0100 Subject: [PATCH 199/418] orphaned role - xml indentation fix --- tests/integration/helpers/0_common_instance_config.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/0_common_instance_config.xml b/tests/integration/helpers/0_common_instance_config.xml index e7673224849..ab685f2efe6 100644 --- a/tests/integration/helpers/0_common_instance_config.xml +++ b/tests/integration/helpers/0_common_instance_config.xml @@ -25,6 +25,6 @@ true true true - 6 + 6 From c3ce0aa3badb4107d0b542891a86c65429c5c4e1 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 8 Mar 2023 19:37:41 +0100 Subject: [PATCH 200/418] orphaned role - cache expiration time and timeout in tests decreased --- .../helpers/0_common_instance_config.xml | 2 +- tests/integration/test_role/test.py | 14 +++++++++++--- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/tests/integration/helpers/0_common_instance_config.xml b/tests/integration/helpers/0_common_instance_config.xml index ab685f2efe6..97b68fdf629 100644 --- a/tests/integration/helpers/0_common_instance_config.xml +++ b/tests/integration/helpers/0_common_instance_config.xml @@ -25,6 +25,6 @@ true true true - 6 + 2 diff --git a/tests/integration/test_role/test.py b/tests/integration/test_role/test.py index 0fb10989e68..6cf2aa76ca3 100644 --- a/tests/integration/test_role/test.py +++ b/tests/integration/test_role/test.py @@ -430,7 +430,10 @@ def test_role_expiration(): instance.query("GRANT SELECT ON tre TO rre") assert instance.query("SELECT * FROM tre", user="ure") == "0\n" - time.sleep(10) # wait for role expiration + + # access_control_improvements/role_cache_expiration_time value is 2 for the test + # so we wait >2 seconds until the role is expired + time.sleep(5) instance.query("CREATE TABLE IF NOT EXISTS tre1 (id Int) Engine=Log") instance.query("INSERT INTO tre1 VALUES (0)") @@ -461,10 +464,15 @@ def test_two_roles_expiration(): instance.query("GRANT SELECT ON tre TO rre") assert instance.query("SELECT * FROM tre", user="ure") == "0\n" - time.sleep(10) # wait for role expiration + + # access_control_improvements/role_cache_expiration_time value is 2 for the test + # so we wait >2 seconds until the roles are expired + time.sleep(5) + instance.query( "GRANT SELECT ON tre1 TO rre_second" - ) # we expect that both rre and rre_second are gone from cache + ) # we expect that both rre and rre_second are gone from cache upon this operation + instance.query("CREATE TABLE IF NOT EXISTS tre1 (id Int) Engine=Log") instance.query("INSERT INTO tre1 VALUES (0)") instance.query("GRANT SELECT ON tre1 TO rre") From 3f51101ad462aed3e6a6d59fa1a9d949d316d67d Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 9 Mar 2023 11:20:56 +0100 Subject: [PATCH 201/418] orphaned role - must initialize role cache for e.g clickhouse-local --- src/Access/AccessControl.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index dfb65721a0a..3a55137dab5 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -247,6 +247,7 @@ private: AccessControl::AccessControl() : MultipleAccessStorage("user directories"), context_access_cache(std::make_unique(*this)), + role_cache(std::make_unique(*this, 600)), row_policy_cache(std::make_unique(*this)), quota_cache(std::make_unique(*this)), settings_profiles_cache(std::make_unique(*this)), From ad0548213648f0e14011b3c713c03a477e4548f3 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 21 Mar 2023 10:36:16 +0800 Subject: [PATCH 202/418] fix fast test --- ...68_parse_datetime_in_joda_syntax.reference | 151 +++++++++++++++--- 1 file changed, 132 insertions(+), 19 deletions(-) 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 99426a8c8d8..e8cc31944c3 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,93 +1,139 @@ -- { 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 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 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 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 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 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 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 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 CANNOT_PARSE_DATETIME } select parseDateTimeInJodaSyntax('Decembr', 'MMMM', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } @@ -95,12 +141,15 @@ select parseDateTimeInJodaSyntax('Decemberary', 'MMM', 'UTC'); -- { serverError 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 CANNOT_PARSE_DATETIME } select parseDateTimeInJodaSyntax('32', 'd', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTimeInJodaSyntax('12345', 'd', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } @@ -108,126 +157,190 @@ select parseDateTimeInJodaSyntax('02-31', 'M-d', 'UTC'); -- { serverError CANNOT 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') = toDateTime('2000-02-20', 'UTC'); +1 select parseDateTimeInJodaSyntax('2 31 20 4', 'M d d M', 'UTC') = toDateTime('2000-04-20', 'UTC'); +1 --- 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 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 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 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 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 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 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 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 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 CANNOT_PARSE_DATETIME } select parseDateTimeInJodaSyntax('-1', 's', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTimeInJodaSyntax('123456789', 's', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } - --- { echoOff } From 7531840fd735465affecb57ed8d6883fb7c1d6fd Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 21 Mar 2023 02:41:51 +0000 Subject: [PATCH 203/418] Support fuzz Explain query --- src/Client/QueryFuzzer.cpp | 105 ++++++++++++++++++++++++++++++++-- src/Client/QueryFuzzer.h | 4 ++ src/Parsers/ASTExplainQuery.h | 2 + 3 files changed, 107 insertions(+), 4 deletions(-) diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index e150717db95..4383984539e 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -20,13 +20,11 @@ #include -#include -#include -#include #include #include #include #include +#include #include #include #include @@ -34,17 +32,20 @@ #include #include #include +#include #include #include #include #include #include -#include #include #include #include #include #include +#include +#include +#include namespace DB @@ -681,6 +682,98 @@ void QueryFuzzer::fuzzTableName(ASTTableExpression & table) } } +void QueryFuzzer::fuzzExplainQuery(ASTExplainQuery & explain) +{ + /// Fuzz ExplainKind + if (fuzz_rand() % 20 == 0) + { + /// Do not modify ExplainKind + } + else if (fuzz_rand() % 11 == 0) + { + explain.setExplainKind(ASTExplainQuery::ExplainKind::ParsedAST); + } + else if (fuzz_rand() % 11 == 0) + { + explain.setExplainKind(ASTExplainQuery::ExplainKind::AnalyzedSyntax); + } + else if (fuzz_rand() % 11 == 0) + { + explain.setExplainKind(ASTExplainQuery::ExplainKind::QueryTree); + } + else if (fuzz_rand() % 11 == 0) + { + explain.setExplainKind(ASTExplainQuery::ExplainKind::QueryPlan); + } + else if (fuzz_rand() % 11 == 0) + { + explain.setExplainKind(ASTExplainQuery::ExplainKind::QueryPipeline); + } + else if (fuzz_rand() % 11 == 0) + { + explain.setExplainKind(ASTExplainQuery::ExplainKind::QueryEstimates); + } + else if (fuzz_rand() % 11 == 0) + { + explain.setExplainKind(ASTExplainQuery::ExplainKind::TableOverride); + } + else if (fuzz_rand() % 11 == 0) + { + explain.setExplainKind(ASTExplainQuery::ExplainKind::CurrentTransaction); + } + + static const std::unordered_map> settings_by_kind + = {{ASTExplainQuery::ExplainKind::ParsedAST, {"graph", "optimize"}}, + {ASTExplainQuery::ExplainKind::AnalyzedSyntax, {}}, + {ASTExplainQuery::QueryTree, {"run_passes", "dump_passes", "dump_ast", "passes"}}, + {ASTExplainQuery::ExplainKind::QueryPlan, {"header, description", "actions", "indexes", "optimize", "json", "sorting"}}, + {ASTExplainQuery::ExplainKind::QueryPipeline, {"header", "graph=1", "compact"}}, + {ASTExplainQuery::ExplainKind::QueryEstimates, {}}, + {ASTExplainQuery::ExplainKind::TableOverride, {}}, + {ASTExplainQuery::ExplainKind::CurrentTransaction, {}}}; + + const auto & settings = settings_by_kind.at(explain.getKind()); + bool settings_have_fuzzed = false; + for (auto & child : explain.children) + { + if (auto * settings_ast = typeid_cast(child.get())) + { + fuzzExplainSettings(*settings_ast, settings); + settings_have_fuzzed = true; + } + /// Fuzz other child like Explain Query + else + { + fuzz(child); + } + } + + if (!settings_have_fuzzed && !settings.empty()) + { + auto settings_ast = std::make_shared(); + fuzzExplainSettings(*settings_ast, settings); + explain.setSettings(settings_ast); + } +} + +void QueryFuzzer::fuzzExplainSettings(ASTSetQuery & settings, const std::vector & names) +{ + auto & changes = settings.changes; + + if (fuzz_rand() % 50 == 0 && !changes.empty()) + { + changes.erase(changes.begin() + fuzz_rand() % changes.size()); + } + + for (const auto & name : names) + { + if (fuzz_rand() % 5 == 0) + { + changes.emplace_back(name, true); + } + } +} + static ASTPtr tryParseInsertQuery(const String & full_query) { const char * pos = full_query.data(); @@ -991,6 +1084,10 @@ void QueryFuzzer::fuzz(ASTPtr & ast) { fuzzCreateQuery(*create_query); } + else if (auto * explain_query = typeid_cast(ast.get())) + { + fuzzExplainQuery(*explain_query); + } else { fuzz(ast->children); diff --git a/src/Client/QueryFuzzer.h b/src/Client/QueryFuzzer.h index bdfdeb67663..445533258bb 100644 --- a/src/Client/QueryFuzzer.h +++ b/src/Client/QueryFuzzer.h @@ -22,6 +22,8 @@ class ASTCreateQuery; class ASTInsertQuery; class ASTColumnDeclaration; class ASTDropQuery; +class ASTExplainQuery; +class ASTSetQuery; struct ASTTableExpression; struct ASTWindowDefinition; @@ -86,6 +88,8 @@ struct QueryFuzzer void fuzzColumnLikeExpressionList(IAST * ast); void fuzzWindowFrame(ASTWindowDefinition & def); void fuzzCreateQuery(ASTCreateQuery & create); + void fuzzExplainQuery(ASTExplainQuery & explain); + void fuzzExplainSettings(ASTSetQuery & settings, const std::vector & names); void fuzzColumnDeclaration(ASTColumnDeclaration & column); void fuzzTableName(ASTTableExpression & table); void fuzz(ASTs & asts); diff --git a/src/Parsers/ASTExplainQuery.h b/src/Parsers/ASTExplainQuery.h index 3903cf42269..701bde8cebd 100644 --- a/src/Parsers/ASTExplainQuery.h +++ b/src/Parsers/ASTExplainQuery.h @@ -80,6 +80,8 @@ public: return res; } + void setExplainKind(ExplainKind kind_) { kind = kind_; } + void setExplainedQuery(ASTPtr query_) { children.emplace_back(query_); From 60ad9ebefa33c08144ad6a518f1933b9f53029c7 Mon Sep 17 00:00:00 2001 From: Rich Raposa Date: Mon, 20 Mar 2023 21:54:19 -0600 Subject: [PATCH 204/418] Update string-search-functions.md --- docs/en/sql-reference/functions/string-search-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 58006f98a4c..7de12668b1d 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -551,7 +551,7 @@ Other LIKE expressions are internally converted to a regular expression and exec ## notLike(haystack, pattern), haystack NOT LIKE pattern operator -The same thing as ‘like’, but negative. +The same thing as `like`, but negative. ## ilike @@ -613,7 +613,7 @@ Result: ## notILike(haystack, pattern), haystack NOT ILIKE pattern operator -The same thing as ‘ilike’, but negative. +The same thing as `ilike`, but negative. ## ngramDistance(haystack, needle) From d3515dd7ae7fe4eec4afd4cae5304872b755afb1 Mon Sep 17 00:00:00 2001 From: Tanya Bragin Date: Mon, 20 Mar 2023 22:05:10 -0700 Subject: [PATCH 205/418] Update query profiler docs to remove self-managed only Query profiler works in ClickHouse Cloud - update docs to reflect it and add an example. cc @alexey-milovidov --- .../sampling-query-profiler.md | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/optimizing-performance/sampling-query-profiler.md b/docs/en/operations/optimizing-performance/sampling-query-profiler.md index 72eb655101f..ab42eec4190 100644 --- a/docs/en/operations/optimizing-performance/sampling-query-profiler.md +++ b/docs/en/operations/optimizing-performance/sampling-query-profiler.md @@ -7,11 +7,23 @@ import SelfManaged from '@site/docs/en/_snippets/_self_managed_only_no_roadmap.m # Sampling Query Profiler - - ClickHouse runs sampling profiler that allows analyzing query execution. Using profiler you can find source code routines that used the most frequently during query execution. You can trace CPU time and wall-clock time spent including idle time. -To use profiler: +Query profiler is automatically enabled in ClickHouse Cloud and you can run a sample query as follows + +``` sql +SELECT + count(), + arrayStringConcat(arrayMap(x -> concat(demangle(addressToSymbol(x)), '\n ', addressToLine(x)), trace), '\n') AS sym +FROM system.trace_log +WHERE (query_id = 'ebca3574-ad0a-400a-9cbc-dca382f5998c') AND (event_date = today()) +GROUP BY trace +ORDER BY count() DESC +LIMIT 10 +SETTINGS allow_introspection_functions = 1 +``` + +In self-managed deployments, to use query profiler: - Setup the [trace_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) section of the server configuration. From 2c9761bca8821e9fbb19f5aaa497b7097244a9e8 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Mon, 20 Mar 2023 23:09:54 -0600 Subject: [PATCH 206/418] Update sparse-primary-indexes.md --- .../sparse-primary-indexes.md | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) 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 2ab9a85b7e8..54827388013 100644 --- a/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md +++ b/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md @@ -149,10 +149,10 @@ SETTINGS index_granularity = 8192, index_granularity_bytes = 0; [//]: # (
)
- + DDL详情 -

+

为了简化本文后面的讨论,并使图和结果可重现,使用DDL语句有如下说明:

    @@ -164,7 +164,7 @@ SETTINGS index_granularity = 8192, index_granularity_bytes = 0;
  • index_granularity: 显式设置为其默认值8192。这意味着对于每一组8192行,主索引将有一个索引条目,例如,如果表包含16384行,那么索引将有两个索引条目。

  • -
  • index_granularity_bytes: 设置为0表示禁止自适应索引粒度。自适应索引粒度意味着ClickHouse自动为一组n行创建一个索引条目 +
  • index_granularity_bytes: 设置为0表示禁止自适应索引粒度。自适应索引粒度意味着ClickHouse自动为一组n行创建一个索引条目
    • 如果n小于8192,但n行的合并行数据大小大于或等于10MB (index_granularity_bytes的默认值)或
    • n达到8192
    • @@ -446,10 +446,10 @@ ClickHouse客户端的输出显示,没有进行全表扫描,只有8.19千行 我们可以在上面的跟踪日志中看到,1083个现有标记中有一个满足查询。
      - + Trace Log详情 -

      +

      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.

      @@ -520,10 +520,10 @@ LIMIT 10; 如上所述,通过对索引的1083个UserID标记进行二分搜索,确定了第176个标记。因此,它对应的颗粒176可能包含UserID列值为749.927.693的行。
      - + 颗粒选择的具体过程 -

      +

      上图显示,标记176是第一个UserID值小于749.927.693的索引条目,并且下一个标记(标记177)的颗粒177的最小UserID值大于该值的索引条目。因此,只有标记176对应的颗粒176可能包含UserID列值为749.927.693的行。

      @@ -671,15 +671,15 @@ Processed 8.81 million rows, 为了说明,我们给出通用的排除搜索算法的工作原理:
      - + 通用排除搜索算法 -

      +

      -下面将演示当通过第一个列之后的任何列选择颗粒时,当前一个键列具有或高或低的基数时,ClickHouse通用排除搜索算法 是如何工作的。 +下面将演示当通过第一个列之后的任何列选择颗粒时,当前一个键列具有或高或低的基数时,ClickHouse通用排除搜索算法 是如何工作的。 作为这两种情况的例子,我们将假设: - 搜索URL值为"W3"的行。 @@ -736,9 +736,9 @@ Processed 8.81 million rows, 在我们的示例数据集中,两个键列(UserID、URL)都具有类似的高基数,并且,如前所述,当URL列的前一个键列具有较高基数时,通用排除搜索算法不是很有效。 :::note 看下跳数索引 -因为UserID和URL具有较高的基数,[根据URL过滤数据](#query-on-url)不是特别有效,对URL列创建[二级跳数索引](./skipping-indexes.md)同样也不会有太多改善。 +因为UserID和URL具有较高的基数,[根据URL过滤数据](#query-on-url)不是特别有效,对URL列创建[二级跳数索引](./skipping-indexes.md)同样也不会有太多改善。 -例如,这两个语句在我们的表的URL列上创建并填充一个minmax跳数索引。 +例如,这两个语句在我们的表的URL列上创建并填充一个minmax跳数索引。 ```sql ALTER TABLE hits_UserID_URL ADD INDEX url_skipping_index URL TYPE minmax GRANULARITY 4; ALTER TABLE hits_UserID_URL MATERIALIZE INDEX url_skipping_index; @@ -907,10 +907,10 @@ ClickHouse只选择了39个索引标记,而不是使用通用排除搜索时 点击下面了解详情:

      - + 对UserID的查询过滤性能较差 -

      +

      ```sql SELECT URL, count(URL) AS Count From 7fa9ee5452419d813a5e764b361a711d079e1476 Mon Sep 17 00:00:00 2001 From: Rich Raposa Date: Mon, 20 Mar 2023 23:20:49 -0600 Subject: [PATCH 207/418] Update s3.md --- docs/en/sql-reference/table-functions/s3.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index 6997d6c30ff..6e32afb59a1 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -186,6 +186,7 @@ INSERT INTO TABLE FUNCTION ``` As a result, the data is written into three files in different buckets: `my_bucket_1/file.csv`, `my_bucket_10/file.csv`, and `my_bucket_20/file.csv`. + **See Also** - [S3 engine](../../engines/table-engines/integrations/s3.md) From 0b9d84788248ec6ffa1a7cbd172045b4cb3a5df4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 21 Mar 2023 08:41:21 +0200 Subject: [PATCH 208/418] Fix flaky test 02417_opentelemetry_insert_on_distributed_table Signed-off-by: Azat Khuzhin --- .../02417_opentelemetry_insert_on_distributed_table.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh index 81f3e3f4ea6..edc3d06e5bf 100755 --- a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh +++ b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh @@ -20,6 +20,7 @@ function insert() -H "tracestate: $4" \ "${CLICKHOUSE_URL}" \ --data @- + ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH DISTRIBUTED ${CLICKHOUSE_DATABASE}.dist_opentelemetry" } function check_span() From 5091b4e224ec87ba4a3d7535a3372d65909896b8 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 21 Mar 2023 08:44:08 +0100 Subject: [PATCH 209/418] orphaned role - expiration_time => expiration_time_seconds --- programs/server/config.xml | 2 +- src/Access/AccessControl.cpp | 2 +- src/Access/RoleCache.cpp | 4 ++-- src/Access/RoleCache.h | 2 +- tests/integration/helpers/0_common_instance_config.xml | 2 +- tests/integration/test_role/test.py | 4 ++-- 6 files changed, 8 insertions(+), 8 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 09e09453660..2434815fa2b 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -705,7 +705,7 @@ false - 600 + 600 diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 3a55137dab5..875f4965e0b 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -283,7 +283,7 @@ void AccessControl::setUpFromMainConfig(const Poco::Util::AbstractConfiguration addStoragesFromMainConfig(config_, config_path_, get_zookeeper_function_); - role_cache = std::make_unique(*this, config_.getInt("access_control_improvements.role_cache_expiration_time", 600)); + role_cache = std::make_unique(*this, config_.getInt("access_control_improvements.role_cache_expiration_time_seconds", 600)); } diff --git a/src/Access/RoleCache.cpp b/src/Access/RoleCache.cpp index 3f5089c540a..2d94df2eea5 100644 --- a/src/Access/RoleCache.cpp +++ b/src/Access/RoleCache.cpp @@ -56,8 +56,8 @@ namespace } -RoleCache::RoleCache(const AccessControl & access_control_, int expiration_time) - : access_control(access_control_), cache(expiration_time * 1000 /* 10 minutes by default*/) +RoleCache::RoleCache(const AccessControl & access_control_, int expiration_time_seconds) + : access_control(access_control_), cache(expiration_time_seconds * 1000 /* 10 minutes by default*/) { } diff --git a/src/Access/RoleCache.h b/src/Access/RoleCache.h index b5c7406e2c4..b5712a24f46 100644 --- a/src/Access/RoleCache.h +++ b/src/Access/RoleCache.h @@ -16,7 +16,7 @@ using RolePtr = std::shared_ptr; class RoleCache { public: - explicit RoleCache(const AccessControl & access_control_, int expiration_time); + explicit RoleCache(const AccessControl & access_control_, int expiration_time_seconds); ~RoleCache(); std::shared_ptr getEnabledRoles( diff --git a/tests/integration/helpers/0_common_instance_config.xml b/tests/integration/helpers/0_common_instance_config.xml index 97b68fdf629..535cf252274 100644 --- a/tests/integration/helpers/0_common_instance_config.xml +++ b/tests/integration/helpers/0_common_instance_config.xml @@ -25,6 +25,6 @@ true true true - 2 + 2 diff --git a/tests/integration/test_role/test.py b/tests/integration/test_role/test.py index 6cf2aa76ca3..9a352f43eab 100644 --- a/tests/integration/test_role/test.py +++ b/tests/integration/test_role/test.py @@ -431,7 +431,7 @@ def test_role_expiration(): assert instance.query("SELECT * FROM tre", user="ure") == "0\n" - # access_control_improvements/role_cache_expiration_time value is 2 for the test + # access_control_improvements/role_cache_expiration_time_seconds value is 2 for the test # so we wait >2 seconds until the role is expired time.sleep(5) @@ -465,7 +465,7 @@ def test_two_roles_expiration(): assert instance.query("SELECT * FROM tre", user="ure") == "0\n" - # access_control_improvements/role_cache_expiration_time value is 2 for the test + # access_control_improvements/role_cache_expiration_time_seconds value is 2 for the test # so we wait >2 seconds until the roles are expired time.sleep(5) From 08549c0a02320a2ad5b0fd9a910f3d8567a373ee Mon Sep 17 00:00:00 2001 From: iammagicc <69392179+iammagicc@users.noreply.github.com> Date: Tue, 21 Mar 2023 15:53:45 +0800 Subject: [PATCH 210/418] =?UTF-8?q?=E5=A2=9E=E5=8A=A0=E6=9C=80=E5=90=8E?= =?UTF-8?q?=E4=B8=80=E4=B8=AA=E7=B4=A2=E5=BC=95=E6=9D=A1=E7=9B=AE=E5=AD=98?= =?UTF-8?q?=E5=82=A8=E4=BA=86=E4=B8=8A=E5=9B=BE=E4=B8=AD=E9=A2=97=E7=B2=92?= =?UTF-8?q?1082=E7=9A=84=E4=B8=BB=E9=94=AE=E5=88=97=E7=9A=84=E6=9C=80?= =?UTF-8?q?=E5=A4=A7=E5=80=BC=E4=B8=8E=E5=9B=BE=E7=89=87=E4=B8=8D=E7=AC=A6?= =?UTF-8?q?=E7=9A=84=E6=8F=8F=E8=BF=B0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 为什么图中显示的数据与结论不符合?因为图中的数据是禁用了自适应索引粒度后得到的,默认情况下索引粒度是自适应的。 https://clickhouse.com/docs/en/optimize/sparse-primary-indexes We mentioned in the beginning of this guide in the "DDL Statement Details", that we disabled adaptive index granularity (in order to simplify the discussions in this guide, as well as make the diagrams and results reproducible). For tables with adaptive index granularity (index granularity is adaptive by default) the size of some granules can be less than 8192 rows depending on the row data sizes. 我们在本指南开头的“DDL 语句详细信息”中提到,我们禁用了自适应索引粒度(为了简化本指南中的讨论,并使图表和结果可重现)。 对于具有自适应索引粒度的表(默认情况下索引粒度是自适应的),某些粒度的大小可以小于 8192 行,具体取决于行数据大小。 https://clickhouse.com/docs/en/whats-new/changelog/2019#experimental-features-1 ClickHouse Release 19.6.3.18, 2019-06-13 Experimental Features:实验性特性 Add setting index_granularity_bytes (adaptive index granularity) for MergeTree* tables family. 为合并树家族的表系列添加设置index_granularity_bytes(自适应索引粒度)。 ClickHouse Release 19.10.1.5, 2019-07-12 Performance Improvement:优化改进 Add the possibility to write the final mark at the end of MergeTree columns. It allows to avoid useless reads for keys that are out of table data range. It is enabled only if adaptive index granularity is in use. 添加在合并树列末尾写入最终标记的可能性。它允许避免对超出表数据范围的键进行无用的读取。仅当使用自适应索引粒度时,才会启用它。 --- .../sparse-primary-indexes.md | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) 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 2ab9a85b7e8..e53d9fc7829 100644 --- a/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md +++ b/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md @@ -370,6 +370,25 @@ UserID.bin,URL.bin,和EventTime.bin是UserID - 索引条目(索引标记)不是基于表中的特定行,而是基于颗粒。例如,对于上图中的索引条目‘mark 0’,在我们的表中没有UserID为240.923且URL为“goal://metry=10000467796a411…”的行,相反,对于该表,有一个颗粒0,在该颗粒中,最小UserID值是240.923,最小URL值是“goal://metry=10000467796a411…”,这两个值来自不同的行。 - 主索引文件完全加载到主内存中。如果文件大于可用的空闲内存空间,则ClickHouse将发生错误。 + +- 为什么图中显示的数据与结论不符合?因为图中的数据是禁用了自适应索引粒度后得到的,默认情况下索引粒度是自适应的。 +- 下面是英文官方文档和版本更新的描述信息。 +- https://clickhouse.com/docs/en/optimize/sparse-primary-indexes +- We mentioned in the beginning of this guide in the "DDL Statement Details", that we disabled adaptive index granularity (in order to simplify the discussions in this guide, as well as make the diagrams and results reproducible). +- For tables with adaptive index granularity (index granularity is adaptive by default) the size of some granules can be less than 8192 rows depending on the row data sizes. +- 我们在本指南开头的“DDL 语句详细信息”中提到,我们禁用了自适应索引粒度(为了简化本指南中的讨论,并使图表和结果可重现)。 +- 对于具有自适应索引粒度的表(默认情况下索引粒度是自适应的),某些粒度的大小可以小于 8192 行,具体取决于行数据大小。 + +- https://clickhouse.com/docs/en/whats-new/changelog/2019#experimental-features-1 +- ClickHouse Release 19.6.3.18, 2019-06-13 +- Experimental Features:实验性特性 +- Add setting index_granularity_bytes (adaptive index granularity) for MergeTree* tables family. +- 为合并树系列的表家族添加设置index_granularity_bytes(自适应索引粒度)。 + +- ClickHouse Release 19.10.1.5, 2019-07-12 +- Performance Improvement:优化改进 +- Add the possibility to write the final mark at the end of MergeTree columns. It allows to avoid useless reads for keys that are out of table data range. It is enabled only if adaptive index granularity is in use. +- 添加在合并树列末尾写入最终标记的可能性。它允许避免对超出表数据范围的键进行无用的读取。仅当使用自适应索引粒度时,才会启用它。 ::: From f218f3671da1a805de9a5a6a8b69660ea09f896a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 21 Mar 2023 09:40:10 +0000 Subject: [PATCH 211/418] Make 01086_odbc_roundtrip less flaky --- src/Disks/ObjectStorages/S3/diskSettings.cpp | 2 +- tests/clickhouse-test | 5 +---- tests/queries/0_stateless/01086_odbc_roundtrip.sh | 9 ++++++--- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 1c3bb857798..3abeb7c70c4 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -154,7 +154,7 @@ std::unique_ptr getClient( {}, 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 + ".expiration_window_seconds", config.getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS))); + config.getUInt64(config_prefix + ".expiration_window_seconds", config.getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS))); } } diff --git a/tests/clickhouse-test b/tests/clickhouse-test index aec52981724..cc9098b7fb4 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1076,10 +1076,7 @@ class TestCase: # >> append to stderr (but not stdout since it is not used there), # because there are also output of per test database creation - if not args.database: - pattern = "{test} > {stdout} 2> {stderr}" - else: - pattern = "{test} > {stdout} 2> {stderr}" + pattern = "{test} > {stdout} 2> {stderr}" if self.ext == ".sql": pattern = ( diff --git a/tests/queries/0_stateless/01086_odbc_roundtrip.sh b/tests/queries/0_stateless/01086_odbc_roundtrip.sh index 20066c6b34c..9e688d91b12 100755 --- a/tests/queries/0_stateless/01086_odbc_roundtrip.sh +++ b/tests/queries/0_stateless/01086_odbc_roundtrip.sh @@ -13,12 +13,15 @@ for _ in $(seq 1 10); do sleep 0.1 done -${CLICKHOUSE_CLIENT} --query "select count() > 1 as ok from (select * from odbc('DSN={ClickHouse DSN (Unicode)}','system','tables'))" +# ODBC will do HEAD request because of progress bar +# in normal situation, 501 will be returned and no Error is logged +# but sometimes we get I/O broken pipe producing an Error log but it doesn't affect the run of the test +${CLICKHOUSE_CLIENT} --query "select count() > 1 as ok from (select * from odbc('DSN={ClickHouse DSN (Unicode)}','system','tables'))" 2> >(grep -Fv "Failed to make HTTP_HEAD request") ${CLICKHOUSE_CLIENT} --query "CREATE TABLE t (x UInt8, y Float32, z String) ENGINE = Memory" ${CLICKHOUSE_CLIENT} --query "INSERT INTO t VALUES (1,0.1,'a я'),(2,0.2,'b ą'),(3,0.3,'c d')" -${CLICKHOUSE_CLIENT} --query "SELECT x, y, z FROM odbc('DSN={ClickHouse DSN (ANSI)}','$CLICKHOUSE_DATABASE','t') ORDER BY x" -${CLICKHOUSE_CLIENT} --query "SELECT x, y, z FROM odbc('DSN={ClickHouse DSN (Unicode)}','$CLICKHOUSE_DATABASE','t') ORDER BY x" +${CLICKHOUSE_CLIENT} --query "SELECT x, y, z FROM odbc('DSN={ClickHouse DSN (ANSI)}','$CLICKHOUSE_DATABASE','t') ORDER BY x" 2> >(grep -Fv "Failed to make HTTP_HEAD request") +${CLICKHOUSE_CLIENT} --query "SELECT x, y, z FROM odbc('DSN={ClickHouse DSN (Unicode)}','$CLICKHOUSE_DATABASE','t') ORDER BY x" 2> >(grep -Fv "Failed to make HTTP_HEAD request") ${CLICKHOUSE_CLIENT} --query "DROP TABLE t" From b9dca4e0dfaa08287a031e2ed85933d0c3e8adb3 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 21 Mar 2023 11:29:16 +0100 Subject: [PATCH 212/418] Place short return before big block, improve logging --- tests/ci/merge_pr.py | 81 +++++++++++++++++++++----------------------- 1 file changed, 39 insertions(+), 42 deletions(-) diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index 3656f12ed4b..2d6d81a152a 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -100,55 +100,52 @@ class Reviews: if review.state == "APPROVED" } - if approved: + if not approved: logging.info( - "The following users from %s team approved the PR: %s", + "The PR #%s is not approved by any of %s team member", + self.pr.number, TEAM_NAME, - ", ".join(user.login for user in approved.keys()), ) - # The only reliable place to get the 100% accurate last_modified - # info is when the commit was pushed to GitHub. The info is - # available as a header 'last-modified' of /{org}/{repo}/commits/{sha}. - # Unfortunately, it's formatted as 'Wed, 04 Jan 2023 11:05:13 GMT' - - commit = self.pr.head.repo.get_commit(self.pr.head.sha) - if commit.stats.last_modified is None: - logging.warning( - "Unable to get info about the commit %s", self.pr.head.sha - ) - return False - - last_changed = datetime.strptime( - commit.stats.last_modified, "%a, %d %b %Y %H:%M:%S GMT" - ) - - approved_at = max(review.submitted_at for review in approved.values()) - if approved_at == datetime.fromtimestamp(0): - logging.info( - "Unable to get `datetime.fromtimestamp(0)`, " - "here's debug info about reviews: %s", - "\n".join(pformat(review) for review in self.reviews.values()), - ) - else: - logging.info( - "The PR is approved at %s", - approved_at.isoformat(), - ) - - if approved_at < last_changed: - logging.info( - "There are changes after approve at %s", - approved_at.isoformat(), - ) - return False - return True + return False logging.info( - "The PR #%s is not approved by any of %s team member", - self.pr.number, + "The following users from %s team approved the PR: %s", TEAM_NAME, + ", ".join(user.login for user in approved.keys()), ) - return False + + # The only reliable place to get the 100% accurate last_modified + # info is when the commit was pushed to GitHub. The info is + # available as a header 'last-modified' of /{org}/{repo}/commits/{sha}. + # Unfortunately, it's formatted as 'Wed, 04 Jan 2023 11:05:13 GMT' + commit = self.pr.head.repo.get_commit(self.pr.head.sha) + if commit.stats.last_modified is None: + logging.warning("Unable to get info about the commit %s", self.pr.head.sha) + return False + + last_changed = datetime.strptime( + commit.stats.last_modified, "%a, %d %b %Y %H:%M:%S GMT" + ) + logging.info("The PR is changed at %s", last_changed.isoformat()) + + approved_at = max(review.submitted_at for review in approved.values()) + if approved_at == datetime.fromtimestamp(0): + logging.info( + "Unable to get `datetime.fromtimestamp(0)`, " + "here's debug info about reviews: %s", + "\n".join(pformat(review) for review in self.reviews.values()), + ) + else: + logging.info("The PR is approved at %s", approved_at.isoformat()) + + if approved_at < last_changed: + logging.info( + "There are changes done at %s after approval at %s", + last_changed.isoformat(), + approved_at.isoformat(), + ) + return False + return True def get_workflows_for_head(repo: Repository, head_sha: str) -> List[WorkflowRun]: From 038bfb40ab7fd66c693317f957dd4be1a43eb280 Mon Sep 17 00:00:00 2001 From: AVMusorin Date: Mon, 27 Feb 2023 15:42:38 +0100 Subject: [PATCH 213/418] dynamic update system.backups --- src/Backups/BackupUtils.cpp | 137 --------------------------- src/Backups/BackupUtils.h | 9 -- src/Backups/BackupsWorker.cpp | 171 +++++++++++++++++++++++++++++++--- src/Backups/BackupsWorker.h | 12 +++ 4 files changed, 172 insertions(+), 157 deletions(-) diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index c6a0840964b..89b75a103c2 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -1,10 +1,7 @@ #include -#include -#include #include #include #include -#include #include @@ -60,140 +57,6 @@ DDLRenamingMap makeRenamingMapFromBackupQuery(const ASTBackupQuery::Elements & e } -void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries, ThreadPool & thread_pool) -{ - size_t num_active_jobs = 0; - std::mutex mutex; - std::condition_variable event; - std::exception_ptr exception; - - bool always_single_threaded = !backup->supportsWritingInMultipleThreads(); - auto thread_group = CurrentThread::getGroup(); - - for (auto & name_and_entry : backup_entries) - { - auto & name = name_and_entry.first; - auto & entry = name_and_entry.second; - - { - std::unique_lock lock{mutex}; - if (exception) - break; - ++num_active_jobs; - } - - auto job = [&](bool async) - { - SCOPE_EXIT_SAFE( - std::lock_guard lock{mutex}; - if (!--num_active_jobs) - event.notify_all(); - if (async) - CurrentThread::detachFromGroupIfNotDetached(); - ); - - try - { - if (async && thread_group) - CurrentThread::attachToGroup(thread_group); - - if (async) - setThreadName("BackupWorker"); - - { - std::lock_guard lock{mutex}; - if (exception) - return; - } - - backup->writeFile(name, std::move(entry)); - } - catch (...) - { - std::lock_guard lock{mutex}; - if (!exception) - exception = std::current_exception(); - } - }; - - if (always_single_threaded || !thread_pool.trySchedule([job] { job(true); })) - job(false); - } - - { - std::unique_lock lock{mutex}; - event.wait(lock, [&] { return !num_active_jobs; }); - if (exception) - std::rethrow_exception(exception); - } -} - - -void restoreTablesData(DataRestoreTasks && tasks, ThreadPool & thread_pool) -{ - size_t num_active_jobs = 0; - std::mutex mutex; - std::condition_variable event; - std::exception_ptr exception; - - auto thread_group = CurrentThread::getGroup(); - - for (auto & task : tasks) - { - { - std::unique_lock lock{mutex}; - if (exception) - break; - ++num_active_jobs; - } - - auto job = [&](bool async) - { - SCOPE_EXIT_SAFE( - std::lock_guard lock{mutex}; - if (!--num_active_jobs) - event.notify_all(); - if (async) - CurrentThread::detachFromGroupIfNotDetached(); - ); - - try - { - if (async && thread_group) - CurrentThread::attachToGroup(thread_group); - - if (async) - setThreadName("RestoreWorker"); - - { - std::lock_guard lock{mutex}; - if (exception) - return; - } - - std::move(task)(); - } - catch (...) - { - std::lock_guard lock{mutex}; - if (!exception) - exception = std::current_exception(); - } - }; - - if (!thread_pool.trySchedule([job] { job(true); })) - job(false); - } - - { - std::unique_lock lock{mutex}; - event.wait(lock, [&] { return !num_active_jobs; }); - if (exception) - std::rethrow_exception(exception); - } -} - - /// Returns access required to execute BACKUP query. AccessRightsElements getRequiredAccessToBackup(const ASTBackupQuery::Elements & elements) { diff --git a/src/Backups/BackupUtils.h b/src/Backups/BackupUtils.h index cda9121b1fa..f451b003652 100644 --- a/src/Backups/BackupUtils.h +++ b/src/Backups/BackupUtils.h @@ -7,21 +7,12 @@ namespace DB { class IBackup; -using BackupMutablePtr = std::shared_ptr; -class IBackupEntry; -using BackupEntries = std::vector>>; -using DataRestoreTasks = std::vector>; class AccessRightsElements; class DDLRenamingMap; /// Initializes a DDLRenamingMap from a BACKUP or RESTORE query. DDLRenamingMap makeRenamingMapFromBackupQuery(const ASTBackupQuery::Elements & elements); -/// Write backup entries to an opened backup. -void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries, ThreadPool & thread_pool); - -/// Run data restoring tasks which insert data to tables. -void restoreTablesData(DataRestoreTasks && tasks, ThreadPool & thread_pool); /// Returns access required to execute BACKUP query. AccessRightsElements getRequiredAccessToBackup(const ASTBackupQuery::Elements & elements); diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index bdcff249e7d..58185053124 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -21,6 +21,7 @@ #include #include #include +#include namespace DB @@ -346,7 +347,7 @@ void BackupsWorker::doBackup( } /// Write the backup entries to the backup. - writeBackupEntries(backup, std::move(backup_entries), backups_thread_pool); + writeBackupEntries(backup_id, backup, std::move(backup_entries), backups_thread_pool, backup_settings.internal); /// We have written our backup entries, we need to tell other hosts (they could be waiting for it). backup_coordination->setStage(backup_settings.host_id, Stage::COMPLETED, ""); @@ -374,6 +375,7 @@ void BackupsWorker::doBackup( LOG_INFO(log, "{} {} was created successfully", (backup_settings.internal ? "Internal backup" : "Backup"), backup_name_for_logging); setStatus(backup_id, BackupStatus::BACKUP_CREATED); + /// NOTE: we need to update metadata again after backup->finalizeWriting(), because backup metadata is written there. setNumFilesAndSize(backup_id, num_files, total_size, num_entries, uncompressed_size, compressed_size, 0, 0); } catch (...) @@ -394,6 +396,88 @@ void BackupsWorker::doBackup( } +void BackupsWorker::writeBackupEntries(const OperationID & backup_id, BackupMutablePtr backup, BackupEntries && backup_entries, ThreadPool & thread_pool, bool internal) +{ + size_t num_active_jobs = 0; + std::mutex mutex; + std::condition_variable event; + std::exception_ptr exception; + + bool always_single_threaded = !backup->supportsWritingInMultipleThreads(); + auto thread_group = CurrentThread::getGroup(); + + for (auto & name_and_entry : backup_entries) + { + auto & name = name_and_entry.first; + auto & entry = name_and_entry.second; + + { + std::unique_lock lock{mutex}; + if (exception) + break; + ++num_active_jobs; + } + + auto job = [&](bool async) + { + SCOPE_EXIT_SAFE( + std::lock_guard lock{mutex}; + if (!--num_active_jobs) + event.notify_all(); + if (async) + CurrentThread::detachFromGroupIfNotDetached(); + ); + + try + { + if (async && thread_group) + CurrentThread::attachToGroup(thread_group); + + if (async) + setThreadName("BackupWorker"); + + { + std::lock_guard lock{mutex}; + if (exception) + return; + } + + backup->writeFile(name, std::move(entry)); + // Update metadata + if (!internal) + { + setNumFilesAndSize( + backup_id, + backup->getNumFiles(), + backup->getTotalSize(), + backup->getNumEntries(), + backup->getUncompressedSize(), + backup->getCompressedSize(), + 0, 0); + } + + } + catch (...) + { + std::lock_guard lock{mutex}; + if (!exception) + exception = std::current_exception(); + } + }; + + if (always_single_threaded || !thread_pool.trySchedule([job] { job(true); })) + job(false); + } + + { + std::unique_lock lock{mutex}; + event.wait(lock, [&] { return !num_active_jobs; }); + if (exception) + std::rethrow_exception(exception); + } +} + + OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePtr context) { auto restore_query = std::static_pointer_cast(query->clone()); @@ -578,7 +662,7 @@ void BackupsWorker::doRestore( } /// Execute the data restoring tasks. - restoreTablesData(std::move(data_restore_tasks), restores_thread_pool); + restoreTablesData(restore_id, backup, std::move(data_restore_tasks), restores_thread_pool); /// We have restored everything, we need to tell other hosts (they could be waiting for it). restore_coordination->setStage(restore_settings.host_id, Stage::COMPLETED, ""); @@ -586,15 +670,6 @@ void BackupsWorker::doRestore( LOG_INFO(log, "Restored from {} {} successfully", (restore_settings.internal ? "internal backup" : "backup"), backup_name_for_logging); setStatus(restore_id, BackupStatus::RESTORED); - setNumFilesAndSize( - restore_id, - backup->getNumFiles(), - backup->getTotalSize(), - backup->getNumEntries(), - backup->getUncompressedSize(), - backup->getCompressedSize(), - backup->getNumReadFiles(), - backup->getNumReadBytes()); } catch (...) { @@ -614,6 +689,80 @@ void BackupsWorker::doRestore( } +void BackupsWorker::restoreTablesData(const OperationID & restore_id, BackupPtr backup, DataRestoreTasks && tasks, ThreadPool & thread_pool) +{ + size_t num_active_jobs = 0; + std::mutex mutex; + std::condition_variable event; + std::exception_ptr exception; + + auto thread_group = CurrentThread::getGroup(); + + for (auto & task : tasks) + { + { + std::unique_lock lock{mutex}; + if (exception) + break; + ++num_active_jobs; + } + + auto job = [&](bool async) + { + SCOPE_EXIT_SAFE( + std::lock_guard lock{mutex}; + if (!--num_active_jobs) + event.notify_all(); + if (async) + CurrentThread::detachFromGroupIfNotDetached(); + ); + + try + { + if (async && thread_group) + CurrentThread::attachToGroup(thread_group); + + if (async) + setThreadName("RestoreWorker"); + + { + std::lock_guard lock{mutex}; + if (exception) + return; + } + + std::move(task)(); + setNumFilesAndSize( + restore_id, + backup->getNumFiles(), + backup->getTotalSize(), + backup->getNumEntries(), + backup->getUncompressedSize(), + backup->getCompressedSize(), + backup->getNumReadFiles(), + backup->getNumReadBytes()); + } + catch (...) + { + std::lock_guard lock{mutex}; + if (!exception) + exception = std::current_exception(); + } + }; + + if (!thread_pool.trySchedule([job] { job(true); })) + job(false); + } + + { + std::unique_lock lock{mutex}; + event.wait(lock, [&] { return !num_active_jobs; }); + if (exception) + std::rethrow_exception(exception); + } +} + + void BackupsWorker::addInfo(const OperationID & id, const String & name, bool internal, BackupStatus status) { Info info; diff --git a/src/Backups/BackupsWorker.h b/src/Backups/BackupsWorker.h index 0f5c16cd71f..c36b58da14f 100644 --- a/src/Backups/BackupsWorker.h +++ b/src/Backups/BackupsWorker.h @@ -17,6 +17,12 @@ struct RestoreSettings; struct BackupInfo; class IBackupCoordination; class IRestoreCoordination; +class IBackup; +using BackupMutablePtr = std::shared_ptr; +using BackupPtr = std::shared_ptr; +class IBackupEntry; +using BackupEntries = std::vector>>; +using DataRestoreTasks = std::vector>; /// Manager of backups and restores: executes backups and restores' threads in the background. /// Keeps information about backups and restores started in this session. @@ -99,6 +105,9 @@ private: ContextMutablePtr mutable_context, bool called_async); + /// Write backup entries to an opened backup. + void writeBackupEntries(const OperationID & backup_id, BackupMutablePtr backup, BackupEntries && backup_entries, ThreadPool & thread_pool, bool internal); + OperationID startRestoring(const ASTPtr & query, ContextMutablePtr context); void doRestore( @@ -111,6 +120,9 @@ private: ContextMutablePtr context, bool called_async); + /// Run data restoring tasks which insert data to tables. + void restoreTablesData(const OperationID & restore_id, BackupPtr backup, DataRestoreTasks && tasks, ThreadPool & thread_pool); + void addInfo(const OperationID & id, const String & name, bool internal, BackupStatus status); void setStatus(const OperationID & id, BackupStatus status, bool throw_if_error = true); void setStatusSafe(const String & id, BackupStatus status) { setStatus(id, status, false); } From 82d71a1044493b44183f36a0da073f61529e3f24 Mon Sep 17 00:00:00 2001 From: iammagicc <69392179+iammagicc@users.noreply.github.com> Date: Tue, 21 Mar 2023 19:53:00 +0800 Subject: [PATCH 214/418] image errors and adaptive index granularity Add a description of the relationship between image errors and adaptive index granularity.And create a new node with this description. --- .../sparse-primary-indexes.md | 42 ++++++++++--------- 1 file changed, 23 insertions(+), 19 deletions(-) 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 e53d9fc7829..65dff479db4 100644 --- a/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md +++ b/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md @@ -335,6 +335,29 @@ UserID.bin,URL.bin,和EventTime.bin是UserID 第一个(根据磁盘上的物理顺序)8192行(它们的列值)在逻辑上属于颗粒0,然后下一个8192行(它们的列值)属于颗粒1,以此类推。 +:::note +- 为什么上面和接下来的图片中显示的橙色数据与结论(1.最后一个颗粒我们标记的是最大的值;2.索引条目(索引标记)不是基于表中的特定行,而是基于颗粒。)不符合? +- 因为图中的数据是禁用了自适应索引粒度后得到的,默认情况下索引粒度是自适应的。 + +- 下面是英文官方文档和版本更新的描述信息。 +- https://clickhouse.com/docs/en/optimize/sparse-primary-indexes +- We mentioned in the beginning of this guide in the "DDL Statement Details", that we disabled adaptive index granularity (in order to simplify the discussions in this guide, as well as make the diagrams and results reproducible). +- For tables with adaptive index granularity (index granularity is adaptive by default) the size of some granules can be less than 8192 rows depending on the row data sizes. +- 我们在本指南开头的“DDL 语句详细信息”中提到,我们禁用了自适应索引粒度(为了简化本指南中的讨论,并使图表和结果可重现)。 +- 对于具有自适应索引粒度的表(默认情况下索引粒度是自适应的),某些粒度的大小可以小于 8192 行,具体取决于行数据大小。 + +- https://clickhouse.com/docs/en/whats-new/changelog/2019#experimental-features-1 +- ClickHouse Release 19.6.3.18, 2019-06-13 +- Experimental Features:实验性特性 +- Add setting index_granularity_bytes (adaptive index granularity) for MergeTree* tables family. +- 为合并树系列的表家族添加设置index_granularity_bytes(自适应索引粒度)。 + +- ClickHouse Release 19.10.1.5, 2019-07-12 +- Performance Improvement:优化改进 +- Add the possibility to write the final mark at the end of MergeTree columns. It allows to avoid useless reads for keys that are out of table data range. It is enabled only if adaptive index granularity is in use. +- 添加在合并树列末尾写入最终标记的可能性。它允许避免对超出表数据范围的键进行无用的读取。仅当使用自适应索引粒度时,才会启用它。 +::: + :::note - 最后一个颗粒(1082颗粒)是少于8192行的。 @@ -370,25 +393,6 @@ UserID.bin,URL.bin,和EventTime.bin是UserID - 索引条目(索引标记)不是基于表中的特定行,而是基于颗粒。例如,对于上图中的索引条目‘mark 0’,在我们的表中没有UserID为240.923且URL为“goal://metry=10000467796a411…”的行,相反,对于该表,有一个颗粒0,在该颗粒中,最小UserID值是240.923,最小URL值是“goal://metry=10000467796a411…”,这两个值来自不同的行。 - 主索引文件完全加载到主内存中。如果文件大于可用的空闲内存空间,则ClickHouse将发生错误。 - -- 为什么图中显示的数据与结论不符合?因为图中的数据是禁用了自适应索引粒度后得到的,默认情况下索引粒度是自适应的。 -- 下面是英文官方文档和版本更新的描述信息。 -- https://clickhouse.com/docs/en/optimize/sparse-primary-indexes -- We mentioned in the beginning of this guide in the "DDL Statement Details", that we disabled adaptive index granularity (in order to simplify the discussions in this guide, as well as make the diagrams and results reproducible). -- For tables with adaptive index granularity (index granularity is adaptive by default) the size of some granules can be less than 8192 rows depending on the row data sizes. -- 我们在本指南开头的“DDL 语句详细信息”中提到,我们禁用了自适应索引粒度(为了简化本指南中的讨论,并使图表和结果可重现)。 -- 对于具有自适应索引粒度的表(默认情况下索引粒度是自适应的),某些粒度的大小可以小于 8192 行,具体取决于行数据大小。 - -- https://clickhouse.com/docs/en/whats-new/changelog/2019#experimental-features-1 -- ClickHouse Release 19.6.3.18, 2019-06-13 -- Experimental Features:实验性特性 -- Add setting index_granularity_bytes (adaptive index granularity) for MergeTree* tables family. -- 为合并树系列的表家族添加设置index_granularity_bytes(自适应索引粒度)。 - -- ClickHouse Release 19.10.1.5, 2019-07-12 -- Performance Improvement:优化改进 -- Add the possibility to write the final mark at the end of MergeTree columns. It allows to avoid useless reads for keys that are out of table data range. It is enabled only if adaptive index granularity is in use. -- 添加在合并树列末尾写入最终标记的可能性。它允许避免对超出表数据范围的键进行无用的读取。仅当使用自适应索引粒度时,才会启用它。 ::: From 27ef59d46e4f6812685c4051e62ff404df0f9cab Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 21 Mar 2023 13:50:44 +0100 Subject: [PATCH 215/418] Remove wrong chassert() in UserDefinedSQLObjectsLoaderFromZooKeeper.cpp --- .../UserDefinedSQLObjectsLoaderFromZooKeeper.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp index 6655155aff9..ab6ba762103 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp @@ -279,9 +279,9 @@ bool UserDefinedSQLObjectsLoaderFromZooKeeper::getObjectDataAndSetWatch( if (response.type == Coordination::Event::CHANGED) { [[maybe_unused]] bool inserted = watch_queue->emplace(object_type, object_name); - chassert(inserted); + /// `inserted` can be false if `watch_queue` was already finalized (which happens when stopWatching() is called). } - /// NOTE: Event::DELETED is processed as child event by getChildren watch + /// Event::DELETED is processed as child event by getChildren watch }; Coordination::Stat entity_stat; @@ -340,7 +340,7 @@ Strings UserDefinedSQLObjectsLoaderFromZooKeeper::getObjectNamesAndSetWatch( auto object_list_watcher = [watch_queue = watch_queue, object_type](const Coordination::WatchResponse &) { [[maybe_unused]] bool inserted = watch_queue->emplace(object_type, ""); - chassert(inserted); + /// `inserted` can be false if `watch_queue` was already finalized (which happens when stopWatching() is called). }; Coordination::Stat stat; From 4eeab9fd25d7fee38f4c99b5d9d4c8835060b8e8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 21 Mar 2023 14:24:53 +0100 Subject: [PATCH 216/418] Revert "Add sanity checks for writing number in variable length format" --- src/IO/VarInt.h | 18 +----------------- src/Server/TCPHandler.cpp | 2 +- 2 files changed, 2 insertions(+), 18 deletions(-) diff --git a/src/IO/VarInt.h b/src/IO/VarInt.h index d026192cb7d..0869051034a 100644 --- a/src/IO/VarInt.h +++ b/src/IO/VarInt.h @@ -2,7 +2,6 @@ #include #include -#include #include #include @@ -15,19 +14,7 @@ namespace ErrorCodes } -/** 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) */ +/** Write UInt64 in variable length format (base128) NOTE Only up to 2^63 - 1 are supported. */ void writeVarUInt(UInt64 x, std::ostream & ostr); void writeVarUInt(UInt64 x, WriteBuffer & ostr); char * writeVarUInt(UInt64 x, char * ostr); @@ -199,7 +186,6 @@ 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; @@ -219,7 +205,6 @@ 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; @@ -237,7 +222,6 @@ 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; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 1f25f127b2f..5da2a9dd169 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1869,7 +1869,7 @@ void TCPHandler::sendData(const Block & block) { --unknown_packet_in_send_data; if (unknown_packet_in_send_data == 0) - writeVarUInt(VAR_UINT_MAX, *out); + writeVarUInt(UInt64(-1), *out); } writeVarUInt(Protocol::Server::Data, *out); From 840e8252827ac4fd4b1e15500eea67403b77ca26 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 21 Mar 2023 14:28:37 +0100 Subject: [PATCH 217/418] Fix test test_replicated_merge_tree_encryption_codec. --- .../test_replicated_merge_tree_encryption_codec/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_replicated_merge_tree_encryption_codec/test.py b/tests/integration/test_replicated_merge_tree_encryption_codec/test.py index 6f08daae4cf..ffe9c056f1e 100644 --- a/tests/integration/test_replicated_merge_tree_encryption_codec/test.py +++ b/tests/integration/test_replicated_merge_tree_encryption_codec/test.py @@ -64,6 +64,7 @@ def optimize_table(): def check_table(): expected = [[1, "str1"], [2, "str2"]] + node1.query("SYSTEM SYNC REPLICA ON CLUSTER 'cluster' tbl") assert node1.query("SELECT * FROM tbl ORDER BY id") == TSV(expected) assert node2.query("SELECT * FROM tbl ORDER BY id") == TSV(expected) assert node1.query("CHECK TABLE tbl") == "1\n" From 17e1c6be8e87063fb88a11c9eb6f96d81b71a6ce Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 21 Mar 2023 15:01:39 +0100 Subject: [PATCH 218/418] fix race condition on session close --- src/Interpreters/Session.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 70d4c0e6ae0..e5a8813552b 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -140,9 +140,12 @@ public: scheduleCloseSession(session, lock); } - void closeSession(const UUID & user_id, const String & session_id) + void releaseAnsCloseSession(const UUID & user_id, const String & session_id, std::shared_ptr & session_data) { std::unique_lock lock(mutex); + scheduleCloseSession(*session_data, lock); + session_data = nullptr; + Key key{user_id, session_id}; auto it = sessions.find(key); if (it == sessions.end()) @@ -559,8 +562,7 @@ void Session::closeSession(const String & session_id) if (!named_session) return; - releaseSessionID(); - NamedSessionsStorage::instance().closeSession(*user_id, session_id); + NamedSessionsStorage::instance().releaseAnsCloseSession(*user_id, session_id, named_session); } } From 214bcdda4743008fbcb0563f5182a9e052fa150a Mon Sep 17 00:00:00 2001 From: iammagicc <69392179+iammagicc@users.noreply.github.com> Date: Tue, 21 Mar 2023 22:09:10 +0800 Subject: [PATCH 219/418] =?UTF-8?q?delete=20quoat=20English=20text?= =?UTF-8?q?=EF=BC=8Cchange=20the=20translation?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit As the title says,I delete quoat English text.The Chinese translation has been modified to make it easier for readers to understand. --- .../sparse-primary-indexes.md | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) 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 d0d137d7481..e409a5295f0 100644 --- a/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md +++ b/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md @@ -337,25 +337,21 @@ UserID.bin,URL.bin,和EventTime.bin是UserID :::note - 为什么上面和接下来的图片中显示的橙色数据与结论(1.最后一个颗粒我们标记的是最大的值;2.索引条目(索引标记)不是基于表中的特定行,而是基于颗粒。)不符合? -- 因为图中的数据是禁用了自适应索引粒度后得到的,默认情况下索引粒度是自适应的。 +- 因为图中的数据是禁用了自适应索引粒度后得到的(中文文档中直接引用了英文官方文档的图片内容),默认情况下索引粒度是自适应的。而基于自适应索引粒度的表中的数据就会像上述结论中一样,下文将会分析和讲解这些问题。 -- 下面是英文官方文档和版本更新的描述信息。 +- 下面分别是英文官方文档和版本更新网页中的描述信息,您可以开启翻译找到对应的描述。 - https://clickhouse.com/docs/en/optimize/sparse-primary-indexes -- We mentioned in the beginning of this guide in the "DDL Statement Details", that we disabled adaptive index granularity (in order to simplify the discussions in this guide, as well as make the diagrams and results reproducible). -- For tables with adaptive index granularity (index granularity is adaptive by default) the size of some granules can be less than 8192 rows depending on the row data sizes. - 我们在本指南开头的“DDL 语句详细信息”中提到,我们禁用了自适应索引粒度(为了简化本指南中的讨论,并使图表和结果可重现)。 - 对于具有自适应索引粒度的表(默认情况下索引粒度是自适应的),某些粒度的大小可以小于 8192 行,具体取决于行数据大小。 - https://clickhouse.com/docs/en/whats-new/changelog/2019#experimental-features-1 - ClickHouse Release 19.6.3.18, 2019-06-13 - Experimental Features:实验性特性 -- Add setting index_granularity_bytes (adaptive index granularity) for MergeTree* tables family. -- 为合并树系列的表家族添加设置index_granularity_bytes(自适应索引粒度)。 +- 为合并树家族的表设置index_granularity_bytes(自适应索引粒度)。 - ClickHouse Release 19.10.1.5, 2019-07-12 - Performance Improvement:优化改进 -- Add the possibility to write the final mark at the end of MergeTree columns. It allows to avoid useless reads for keys that are out of table data range. It is enabled only if adaptive index granularity is in use. -- 添加在合并树列末尾写入最终标记的可能性。它允许避免对超出表数据范围的键进行无用的读取。仅当使用自适应索引粒度时,才会启用它。 +- 添加在合并树列末尾写入最终标记的可能性。它可以避免对超出表数据范围的键进行无用的读取。仅当使用自适应索引粒度时,才会启用它。 ::: :::note From db7582e0605e44c90b268bc8ba65e2a3b3a9113d Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 21 Mar 2023 14:26:39 +0000 Subject: [PATCH 220/418] AST fuzzer support fuzzing SELECT query to EXPLAIN query randomly --- src/Client/QueryFuzzer.cpp | 100 ++++++++++++++++++++++--------------- src/Client/QueryFuzzer.h | 9 ++-- 2 files changed, 64 insertions(+), 45 deletions(-) diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index 4383984539e..ba1f4e18746 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -24,7 +24,6 @@ #include #include #include -#include #include #include #include @@ -684,43 +683,75 @@ void QueryFuzzer::fuzzTableName(ASTTableExpression & table) void QueryFuzzer::fuzzExplainQuery(ASTExplainQuery & explain) { - /// Fuzz ExplainKind + explain.setExplainKind(fuzzExplainKind(explain.getKind())); + + bool settings_have_fuzzed = false; + for (auto & child : explain.children) + { + if (auto * settings_ast = typeid_cast(child.get())) + { + fuzzExplainSettings(*settings_ast, explain.getKind()); + settings_have_fuzzed = true; + } + /// Fuzzing other child like Explain Query + else + { + fuzz(child); + } + } + + if (!settings_have_fuzzed) + { + auto settings_ast = std::make_shared(); + fuzzExplainSettings(*settings_ast, explain.getKind()); + explain.setSettings(settings_ast); + } +} + +ASTExplainQuery::ExplainKind QueryFuzzer::fuzzExplainKind(ASTExplainQuery::ExplainKind kind) +{ if (fuzz_rand() % 20 == 0) { - /// Do not modify ExplainKind + return kind; } else if (fuzz_rand() % 11 == 0) { - explain.setExplainKind(ASTExplainQuery::ExplainKind::ParsedAST); + return ASTExplainQuery::ExplainKind::ParsedAST; } else if (fuzz_rand() % 11 == 0) { - explain.setExplainKind(ASTExplainQuery::ExplainKind::AnalyzedSyntax); + return ASTExplainQuery::ExplainKind::AnalyzedSyntax; } else if (fuzz_rand() % 11 == 0) { - explain.setExplainKind(ASTExplainQuery::ExplainKind::QueryTree); + return ASTExplainQuery::ExplainKind::QueryTree; } else if (fuzz_rand() % 11 == 0) { - explain.setExplainKind(ASTExplainQuery::ExplainKind::QueryPlan); + return ASTExplainQuery::ExplainKind::QueryPlan; } else if (fuzz_rand() % 11 == 0) { - explain.setExplainKind(ASTExplainQuery::ExplainKind::QueryPipeline); + return ASTExplainQuery::ExplainKind::QueryPipeline; } else if (fuzz_rand() % 11 == 0) { - explain.setExplainKind(ASTExplainQuery::ExplainKind::QueryEstimates); + return ASTExplainQuery::ExplainKind::QueryEstimates; } else if (fuzz_rand() % 11 == 0) { - explain.setExplainKind(ASTExplainQuery::ExplainKind::TableOverride); + return ASTExplainQuery::ExplainKind::TableOverride; } else if (fuzz_rand() % 11 == 0) { - explain.setExplainKind(ASTExplainQuery::ExplainKind::CurrentTransaction); + return ASTExplainQuery::ExplainKind::CurrentTransaction; } + return kind; +} + +void QueryFuzzer::fuzzExplainSettings(ASTSetQuery & settings_ast, ASTExplainQuery::ExplainKind kind) +{ + auto & changes = settings_ast.changes; static const std::unordered_map> settings_by_kind = {{ASTExplainQuery::ExplainKind::ParsedAST, {"graph", "optimize"}}, @@ -732,44 +763,17 @@ void QueryFuzzer::fuzzExplainQuery(ASTExplainQuery & explain) {ASTExplainQuery::ExplainKind::TableOverride, {}}, {ASTExplainQuery::ExplainKind::CurrentTransaction, {}}}; - const auto & settings = settings_by_kind.at(explain.getKind()); - bool settings_have_fuzzed = false; - for (auto & child : explain.children) - { - if (auto * settings_ast = typeid_cast(child.get())) - { - fuzzExplainSettings(*settings_ast, settings); - settings_have_fuzzed = true; - } - /// Fuzz other child like Explain Query - else - { - fuzz(child); - } - } - - if (!settings_have_fuzzed && !settings.empty()) - { - auto settings_ast = std::make_shared(); - fuzzExplainSettings(*settings_ast, settings); - explain.setSettings(settings_ast); - } -} - -void QueryFuzzer::fuzzExplainSettings(ASTSetQuery & settings, const std::vector & names) -{ - auto & changes = settings.changes; - + const auto & settings = settings_by_kind.at(kind); if (fuzz_rand() % 50 == 0 && !changes.empty()) { changes.erase(changes.begin() + fuzz_rand() % changes.size()); } - for (const auto & name : names) + for (const auto & setting : settings) { if (fuzz_rand() % 5 == 0) { - changes.emplace_back(name, true); + changes.emplace_back(setting, true); } } } @@ -910,6 +914,20 @@ void QueryFuzzer::fuzz(ASTPtr & ast) if (auto * with_union = typeid_cast(ast.get())) { fuzz(with_union->list_of_selects); + /// Fuzzing SELECT query to EXPLAIN query randomly + if (fuzz_rand() % 20 == 0) + { + auto explain = std::make_shared(); + + explain->setExplainKind(fuzzExplainKind()); + + auto settings_ast = std::make_shared(); + fuzzExplainSettings(*settings_ast, explain->getKind()); + explain->setSettings(settings_ast); + + explain->setExplainedQuery(ast); + ast = explain; + } } else if (auto * with_intersect_except = typeid_cast(ast.get())) { diff --git a/src/Client/QueryFuzzer.h b/src/Client/QueryFuzzer.h index 445533258bb..739c38dc380 100644 --- a/src/Client/QueryFuzzer.h +++ b/src/Client/QueryFuzzer.h @@ -7,10 +7,11 @@ #include +#include +#include +#include #include #include "Parsers/IAST_fwd.h" -#include -#include namespace DB @@ -22,7 +23,6 @@ class ASTCreateQuery; class ASTInsertQuery; class ASTColumnDeclaration; class ASTDropQuery; -class ASTExplainQuery; class ASTSetQuery; struct ASTTableExpression; struct ASTWindowDefinition; @@ -89,7 +89,8 @@ struct QueryFuzzer void fuzzWindowFrame(ASTWindowDefinition & def); void fuzzCreateQuery(ASTCreateQuery & create); void fuzzExplainQuery(ASTExplainQuery & explain); - void fuzzExplainSettings(ASTSetQuery & settings, const std::vector & names); + ASTExplainQuery::ExplainKind fuzzExplainKind(ASTExplainQuery::ExplainKind kind = ASTExplainQuery::ExplainKind::QueryPipeline); + void fuzzExplainSettings(ASTSetQuery & settings_ast, ASTExplainQuery::ExplainKind kind); void fuzzColumnDeclaration(ASTColumnDeclaration & column); void fuzzTableName(ASTTableExpression & table); void fuzz(ASTs & asts); From be4884d07a05a80b06c65645e2ce1356202e08d9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 21 Mar 2023 17:39:40 +0300 Subject: [PATCH 221/418] Update src/Interpreters/Session.cpp Co-authored-by: alesapin --- src/Interpreters/Session.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index e5a8813552b..5089656a955 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -140,7 +140,7 @@ public: scheduleCloseSession(session, lock); } - void releaseAnsCloseSession(const UUID & user_id, const String & session_id, std::shared_ptr & session_data) + void releaseAndCloseSession(const UUID & user_id, const String & session_id, std::shared_ptr & session_data) { std::unique_lock lock(mutex); scheduleCloseSession(*session_data, lock); From 9c4c7948e54b8cfe7124290cb90e1a1d12e4198c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 21 Mar 2023 17:39:45 +0300 Subject: [PATCH 222/418] Update src/Interpreters/Session.cpp Co-authored-by: alesapin --- src/Interpreters/Session.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 5089656a955..f0bb339e278 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -562,7 +562,7 @@ void Session::closeSession(const String & session_id) if (!named_session) return; - NamedSessionsStorage::instance().releaseAnsCloseSession(*user_id, session_id, named_session); + NamedSessionsStorage::instance().releaseAndCloseSession(*user_id, session_id, named_session); } } From 49ffda181b3e57aff8d63f8636241b01159bd652 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 21 Mar 2023 14:52:13 +0000 Subject: [PATCH 223/418] Update test reference --- .../00081_group_by_without_key_and_totals.reference | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/1_stateful/00081_group_by_without_key_and_totals.reference b/tests/queries/1_stateful/00081_group_by_without_key_and_totals.reference index f7b0f272bd7..fc48d81d7bb 100644 --- a/tests/queries/1_stateful/00081_group_by_without_key_and_totals.reference +++ b/tests/queries/1_stateful/00081_group_by_without_key_and_totals.reference @@ -12,16 +12,16 @@ 523264 1 523264 -0 523264 +1 523264 1 523264 -0 523264 +1 523264 1 523264 -0 523264 +1 523264 1 523264 -0 523264 +1 523264 -1 86614 0 413533 1 9984 From db31af84c3b5d30c9d47e0692586e162af294fbb Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 21 Mar 2023 15:09:10 +0000 Subject: [PATCH 224/418] fix build --- src/Client/QueryFuzzer.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index ba1f4e18746..7500f90334e 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -917,9 +917,7 @@ void QueryFuzzer::fuzz(ASTPtr & ast) /// Fuzzing SELECT query to EXPLAIN query randomly if (fuzz_rand() % 20 == 0) { - auto explain = std::make_shared(); - - explain->setExplainKind(fuzzExplainKind()); + auto explain = std::make_shared(fuzzExplainKind()); auto settings_ast = std::make_shared(); fuzzExplainSettings(*settings_ast, explain->getKind()); From d45d55969aaca495de93763b9aa3187bd5adc2c8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 21 Mar 2023 17:42:23 +0200 Subject: [PATCH 225/418] Improve empty query detection for PostgreSQL (for pgx golang driver) From PostgreSQL documentation [1]: If a completely empty (no contents other than whitespace) query string is received, the response is EmptyQueryResponse followed by ReadyForQuery. [1]: https://www.postgresql.org/docs/current/protocol-flow.html So without it it will try to process the query and send "Empty query" instead, that's why it is important. Refs: https://github.com/jackc/pgx/blob/9ae852eb583d2dced83b1d2ffe1c8803dda2c92e/conn.go#L388 Signed-off-by: Azat Khuzhin --- src/Server/PostgreSQLHandler.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Server/PostgreSQLHandler.cpp b/src/Server/PostgreSQLHandler.cpp index b017b87fcc1..36b05932979 100644 --- a/src/Server/PostgreSQLHandler.cpp +++ b/src/Server/PostgreSQLHandler.cpp @@ -318,6 +318,9 @@ bool PostgreSQLHandler::isEmptyQuery(const String & query) { if (query.empty()) return true; + /// golang driver pgx sends ";" + if (query == ";") + return true; Poco::RegularExpression regex(R"(\A\s*\z)"); return regex.match(query); From ffc8d831c56b847919cd907d42e6f9b32701480d Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 21 Mar 2023 12:00:57 -0400 Subject: [PATCH 226/418] improve doc path and structure --- contrib/qpl-cmake/doc/build.md | 14 ----- .../building_and_benchmarking_deflate_qpl | 63 ++++++++++++------- 2 files changed, 41 insertions(+), 36 deletions(-) delete mode 100644 contrib/qpl-cmake/doc/build.md rename contrib/qpl-cmake/doc/benchmark.md => docs/en/development/building_and_benchmarking_deflate_qpl (78%) diff --git a/contrib/qpl-cmake/doc/build.md b/contrib/qpl-cmake/doc/build.md deleted file mode 100644 index 8ba3ed0f1b0..00000000000 --- a/contrib/qpl-cmake/doc/build.md +++ /dev/null @@ -1,14 +0,0 @@ -This doc gives specfic build tips for QPL_Deflate codec based on Clickhouse generic [build instructions](https://github.com/ClickHouse/ClickHouse/blob/master/docs/en/development/build.md) - -Several tips to build Clickhouse with QPL_deflate enabled: -- The QPL build only support x86_64 platform with avx2/avx512 support. -- The QPL build requires nasm 2.15.0 or higher (e.g., can be obtained from https://www.nasm.us) -- The QPL requires C++ compiler with C++17 standard support. -- Pass the following flag to CMake. Which one to choose depends on your target hardware supported with AVX2 or AVX512. -``` bash -cmake -DENABLE_AVX2=1 -DENABLE_QPL=1 .. -``` -or -``` bash -cmake -DENABLE_AVX512=1 -DENABLE_QPL=1 .. -``` \ No newline at end of file diff --git a/contrib/qpl-cmake/doc/benchmark.md b/docs/en/development/building_and_benchmarking_deflate_qpl similarity index 78% rename from contrib/qpl-cmake/doc/benchmark.md rename to docs/en/development/building_and_benchmarking_deflate_qpl index 4357b3bafb2..dafeb8b9e9f 100644 --- a/contrib/qpl-cmake/doc/benchmark.md +++ b/docs/en/development/building_and_benchmarking_deflate_qpl @@ -1,30 +1,49 @@ -## Files listed -The folders under ./qpl-cmake/benchmark_sample give example to run benchmark with python scripts: +--- +slug: /en/development/contrib +sidebar_position: 73 +sidebar_label: Building and Benchmarking DEFLATE_QPL +description: How to build Clickhouse and run benchmark with DEFLATE_QPL Codec +--- +# Build +- Make sure your target machine meet the QPL required [Prerequisites](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#prerequisites) +- Pass the following flag to CMake when building ClickHouse, depending on the capabilities of your target machine: +``` bash +cmake -DENABLE_AVX2=1 -DENABLE_QPL=1 .. +``` +or +``` bash +cmake -DENABLE_AVX512=1 -DENABLE_QPL=1 .. +``` +- For generic requirements, please refer to Clickhouse generic [build instructions](/docs/en/development/build.md) -"client_scripts" contains python scripts for running typical benchmark, for example: -- "client_stressing_test.py": The python script for query stress test with [1~4] server instances. -- "queries_ssb.sql": The file lists all queries for [Star Schema Benchmark](https://clickhouse.com/docs/en/getting-started/example-datasets/star-schema/) -- "allin1_ssb.sh": This shell script executes benchmark workflow all in one automatically. -"database_files" means it will store database files according to lz4/deflate/zstd codecs. +# Benchmark +## Files list +The folders under [benchmark_sample](https://github.com/ClickHouse/ClickHouse/tree/master/contrib/qpl-cmake/benchmark_sample) give example to run benchmark with python scripts: -"rawdata_dir" means it will store raw data generated by dbgen. +`client_scripts` contains python scripts for running typical benchmark, for example: +- `client_stressing_test.py`: The python script for query stress test with [1~4] server instances. +- `queries_ssb.sql`: The file lists all queries for [Star Schema Benchmark](https://clickhouse.com/docs/en/getting-started/example-datasets/star-schema/) +- `allin1_ssb.sh`: This shell script executes benchmark workflow all in one automatically. +- `database_files` means it will store database files according to lz4/deflate/zstd codecs. -## Run benchmark automatically for Start Schema: +`rawdata_dir` means it will store raw data generated by dbgen. + +## Run benchmark automatically for Star Schema: ``` bash $ cd ./benchmark_sample/client_scripts $ sh run_ssb.sh ``` -After complete, please check all the results in this folder:"./output/" +After complete, please check all the results in this folder:`./output/` -In case you run into failure, please fall back to "Run benchmark manually" as below sections. +In case you run into failure, please mannually run benchmark as below sections. ## Definition [CLICKHOUSE_EXE] means the path of clickhouse executable program. ## Enviroment - CPU: Sapphire Rapid -- OS Requirements refer to [QPL Hardware Path Requirements](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#hardware-path-requirements) -- Set up IAA devices refer to [Accelerator Configuration](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#accelerator-configuration) +- OS Requirements refer to [System Requirements for QPL](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#system-requirements) +- IAA Setup refer to [Accelerator Configuration](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#accelerator-configuration) - Install python modules: pip3 install clickhouse_driver numpy [Self-check for IAA] @@ -46,7 +65,7 @@ $ cd ./rawdata_dir Use [dbgen](https://clickhouse.com/docs/en/getting-started/example-datasets/star-schema) to generate 100 million rows data with the parameters: -s 20 -The files expected to output "./rawdata_dir": +The files expected to output `./rawdata_dir`: customer.tbl part.tbl supplier.tbl @@ -60,7 +79,7 @@ $ cd ./database_dir/lz4 $ [CLICKHOUSE_EXE] server -C config_lz4.xml >&/dev/null& $ [CLICKHOUSE_EXE] client ``` -Here you should see the message "Connected to ClickHouse server " from console which means client successfuly setup connection with server. +Here you should see the message `Connected to ClickHouse server` from console which means client successfuly setup connection with server. Complete below three steps mentioned in [Star Schema Benchmark](https://clickhouse.com/docs/en/getting-started/example-datasets/star-schema) - Creating tables in ClickHouse @@ -97,16 +116,16 @@ You are expected to see below output: [Self-check for IAA Deflate codec] At the first time you execute insertion or query from client, clickhouse server console is expected to print this log: ```text -"Hardware-assisted DeflateQpl codec is ready!" +Hardware-assisted DeflateQpl codec is ready! ``` If you never find this, but see another log as below: ```text -"Initialization of hardware-assisted DeflateQpl codec failed" +Initialization of hardware-assisted DeflateQpl codec failed ``` That means IAA devices is not ready, you need check IAA setup again. ## Benchmark with single instance -- Before start benchmark, Please disable C6 and set CPU frequency governor to be "performance" +- Before start benchmark, Please disable C6 and set CPU frequency governor to be `performance` ``` bash $ cpupower idle-set -d 3 $ cpupower frequency-set -g performance @@ -148,7 +167,7 @@ zstd.log How to check performance metrics: -We focus on QPS, please search the keyword: "QPS_Final" and collect statistics +We focus on QPS, please search the keyword: `QPS_Final` and collect statistics ## Benchmark with multi-instances - To reduce impact of memory bound on too much threads, We recommend run benchmark with multi-instances. @@ -211,7 +230,7 @@ $ [CLICKHOUSE_EXE] client --query "INSERT INTO part FORMAT CSV" < part.tbl --po $ [CLICKHOUSE_EXE] client --query "INSERT INTO supplier FORMAT CSV" < supplier.tbl --port=9001 $ [CLICKHOUSE_EXE] client --query "INSERT INTO lineorder FORMAT CSV" < lineorder.tbl --port=9001 ``` -Here "--port=9002" stands for the assigned port for server instance which is also defined in config_lz4_s2.xml/config_zstd_s2.xml/config_deflate_s2.xml +Here `--port=9002` stands for the assigned port for server instance which is also defined in config_lz4_s2.xml/config_zstd_s2.xml/config_deflate_s2.xml For even more instances, you need replace it with the value: 9002/9003 which stand for s3/s4 instance respectively. @@ -246,7 +265,7 @@ $ numactl -C 30-59,150-179 [CLICKHOUSE_EXE] server -C config_deflate_s2.xml >&/d $ cd ./client_scripts $ numactl -m 1 -N 1 python3 client_stressing_test.py queries_ssb.sql 2 > deflate_2insts.log ``` -Here the last argument: "2" of client_stressing_test.py stands for the number of instances. For more instances, you need replace it with the value: 3 or 4. This script support up to 4 instances/ +Here the last argument: `2` of client_stressing_test.py stands for the number of instances. For more instances, you need replace it with the value: 3 or 4. This script support up to 4 instances/ Now three logs should be output as expected: ``` text @@ -256,7 +275,7 @@ zstd_2insts.log ``` How to check performance metrics: -We focus on QPS, please search the keyword: "QPS_Final" and collect statistics +We focus on QPS, please search the keyword: `QPS_Final` and collect statistics Benchmark setup for 4 instances is similar with 2 instances above. We recommend use 2 instances benchmark data as final report for review. From d65d23bf0e0b918d38bc4b00995e4a9e1477c486 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 21 Mar 2023 12:06:29 -0400 Subject: [PATCH 227/418] remove blank spaces --- .../client_scripts/allin1_ssb.sh | 6 ++-- .../client_scripts/client_stressing_test.py | 20 ++++++------ .../client_scripts/run_ssb.sh | 2 +- .../building_and_benchmarking_deflate_qpl | 32 ++++++++++--------- 4 files changed, 31 insertions(+), 29 deletions(-) diff --git a/contrib/qpl-cmake/benchmark_sample/client_scripts/allin1_ssb.sh b/contrib/qpl-cmake/benchmark_sample/client_scripts/allin1_ssb.sh index e0880811a06..31017b565b6 100644 --- a/contrib/qpl-cmake/benchmark_sample/client_scripts/allin1_ssb.sh +++ b/contrib/qpl-cmake/benchmark_sample/client_scripts/allin1_ssb.sh @@ -397,7 +397,7 @@ do if [ $? -ne 1 ];then instance_alive=1 break; - fi + fi done if [ $instance_alive -eq 0 ];then break; @@ -441,7 +441,7 @@ if [ $is_xml -eq $inst_num ];then echo ------------------------------------------------------ fi fi - echo "Check table data required in server_${1} -> Done! " + echo "Check table data required in server_${1} -> Done! " start_clickhouse_for_stressing ${1} for i in $(seq 0 $[inst_num-1]) @@ -499,7 +499,7 @@ function setup_check(){ if [ $iax_dev_num -eq 0 ] ;then iax_dev_num=`accel-config list | grep iax | wc -l` if [ $iax_dev_num -eq 0 ] ;then - echo "No IAA devices available -> Please check IAA hardware setup manually!" + echo "No IAA devices available -> Please check IAA hardware setup manually!" exit 1 else echo "IAA enabled devices number:$iax_dev_num" diff --git a/contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py b/contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py index b99a48b6a32..8a10e9d3500 100644 --- a/contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py +++ b/contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py @@ -37,7 +37,7 @@ def setup_client(index): def warm_client(clientN, clientL, query, loop): for c_idx in range(clientN): for _ in range(loop): - clientL[c_idx].execute(query) + clientL[c_idx].execute(query) def read_queries(queries_list): queries = list() @@ -84,13 +84,13 @@ def run_multi_clients(clientN, clientList, query, loop): elif c_idx == 3: client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list3)) elif c_idx == 4: - client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list4)) + client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list4)) elif c_idx == 5: - client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list5)) + client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list5)) elif c_idx == 6: - client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list6)) + client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list6)) elif c_idx == 7: - client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list7)) + client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list7)) else: print('ERROR: CLIENT number dismatch!!') exit() @@ -108,7 +108,7 @@ def run_multi_clients(clientN, clientList, query, loop): for item in query_latency_list1: query_latencyTotal.append(item) for item in query_latency_list2: - query_latencyTotal.append(item) + query_latencyTotal.append(item) for item in query_latency_list3: query_latencyTotal.append(item) for item in query_latency_list4: @@ -119,7 +119,7 @@ def run_multi_clients(clientN, clientList, query, loop): query_latencyTotal.append(item) for item in query_latency_list7: query_latencyTotal.append(item) - + totalP95 = np.percentile(query_latencyTotal, 95) * 1000 return totalT,totalP95 @@ -169,17 +169,17 @@ if __name__ == "__main__": for c_idx in range(client_number): client_list[c_idx] = setup_client(c_idx) - #clear cache + #clear cache os.system("sync; echo 3 > /proc/sys/vm/drop_caches") print("###Polit Run Begin") for i in queries: - warm_client(client_number, client_list, i, 1) + warm_client(client_number, client_list, i, 1) print("###Polit Run End -> Start stressing....") query_index = 0 for q in queries: - print("\n###START -> Index: %d, ID: %s, Query: %s" % (query_index,queries_id[query_index], q)) + print("\n###START -> Index: %d, ID: %s, Query: %s" % (query_index,queries_id[query_index], q)) warm_client(client_number, client_list, q, warmup_runs) print("###Warm Done!") for j in range(0,retest_number): diff --git a/contrib/qpl-cmake/benchmark_sample/client_scripts/run_ssb.sh b/contrib/qpl-cmake/benchmark_sample/client_scripts/run_ssb.sh index 2bc2c5f5753..6067b1058f2 100644 --- a/contrib/qpl-cmake/benchmark_sample/client_scripts/run_ssb.sh +++ b/contrib/qpl-cmake/benchmark_sample/client_scripts/run_ssb.sh @@ -3,4 +3,4 @@ if [ ! -d "${WORKING_DIR}/output" ]; then mkdir ${WORKING_DIR}/output fi bash allin1_ssb.sh 2 > ${WORKING_DIR}/output/run.log -echo "Please check log in: ${WORKING_DIR}/output/run.log" +echo "Please check log in: ${WORKING_DIR}/output/run.log" \ No newline at end of file diff --git a/docs/en/development/building_and_benchmarking_deflate_qpl b/docs/en/development/building_and_benchmarking_deflate_qpl index dafeb8b9e9f..09f4fcbee29 100644 --- a/docs/en/development/building_and_benchmarking_deflate_qpl +++ b/docs/en/development/building_and_benchmarking_deflate_qpl @@ -21,7 +21,7 @@ cmake -DENABLE_AVX512=1 -DENABLE_QPL=1 .. The folders under [benchmark_sample](https://github.com/ClickHouse/ClickHouse/tree/master/contrib/qpl-cmake/benchmark_sample) give example to run benchmark with python scripts: `client_scripts` contains python scripts for running typical benchmark, for example: -- `client_stressing_test.py`: The python script for query stress test with [1~4] server instances. +- `client_stressing_test.py`: The python script for query stress test with [1~4] server instances. - `queries_ssb.sql`: The file lists all queries for [Star Schema Benchmark](https://clickhouse.com/docs/en/getting-started/example-datasets/star-schema/) - `allin1_ssb.sh`: This shell script executes benchmark workflow all in one automatically. - `database_files` means it will store database files according to lz4/deflate/zstd codecs. @@ -30,14 +30,14 @@ The folders under [benchmark_sample](https://github.com/ClickHouse/ClickHouse/tr ## Run benchmark automatically for Star Schema: ``` bash -$ cd ./benchmark_sample/client_scripts +$ cd ./benchmark_sample/client_scripts $ sh run_ssb.sh ``` After complete, please check all the results in this folder:`./output/` In case you run into failure, please mannually run benchmark as below sections. -## Definition +## Definition [CLICKHOUSE_EXE] means the path of clickhouse executable program. ## Enviroment @@ -65,11 +65,13 @@ $ cd ./rawdata_dir Use [dbgen](https://clickhouse.com/docs/en/getting-started/example-datasets/star-schema) to generate 100 million rows data with the parameters: -s 20 -The files expected to output `./rawdata_dir`: -customer.tbl -part.tbl -supplier.tbl -lineorder.tbl +The files expected to output `./rawdata_dir`: +``` text +customer.tbl +part.tbl +supplier.tbl +lineorder.tbl +``` ## Database setup Set up database with LZ4 codec @@ -104,7 +106,7 @@ $ [CLICKHOUSE_EXE] client ``` Complete three steps same as lz4 above -[self-check] +[self-check] For each codec(lz4/zstd/deflate), please execute below query to make sure the databases are created successfully: $ select count() from lineorder_flat You are expected to see below output: @@ -113,7 +115,7 @@ You are expected to see below output: │ 119994608 │ └───────────┘ ``` -[Self-check for IAA Deflate codec] +[Self-check for IAA Deflate codec] At the first time you execute insertion or query from client, clickhouse server console is expected to print this log: ```text Hardware-assisted DeflateQpl codec is ready! @@ -151,7 +153,7 @@ $ cd ./client_scripts $ numactl -m 1 -N 1 python3 client_stressing_test.py queries_ssb.sql 1 > deflate.log ``` ZSTD: -``` bash +``` bash $ cd ./database_dir/zstd $ numactl -m 0 -N 0 [CLICKHOUSE_EXE] server -C config_zstd.xml >&/dev/null& $ cd ./client_scripts @@ -169,7 +171,7 @@ How to check performance metrics: We focus on QPS, please search the keyword: `QPS_Final` and collect statistics -## Benchmark with multi-instances +## Benchmark with multi-instances - To reduce impact of memory bound on too much threads, We recommend run benchmark with multi-instances. - Multi-instance means multiple(2 or 4)servers connected with respective client. - The cores of one socket need to be divided equally and assiged to the servers respectively. @@ -258,7 +260,7 @@ $ numactl -m 1 -N 1 python3 client_stressing_test.py queries_ssb.sql 2 > zstd_2i ``` IAA deflate ``` bash -$ cd ./database_dir/deflate +$ cd ./database_dir/deflate $ numactl -C 0-29,120-149 [CLICKHOUSE_EXE] server -C config_deflate.xml >&/dev/null& $ cd ./database_dir/deflate_s2 $ numactl -C 30-59,150-179 [CLICKHOUSE_EXE] server -C config_deflate_s2.xml >&/dev/null& @@ -278,9 +280,9 @@ How to check performance metrics: We focus on QPS, please search the keyword: `QPS_Final` and collect statistics Benchmark setup for 4 instances is similar with 2 instances above. -We recommend use 2 instances benchmark data as final report for review. +We recommend use 2 instances benchmark data as final report for review. -## Tips +## Tips Each time before launch new clickhouse server, please make sure no background clickhouse process running, please check and kill old one: ``` bash $ ps -aux| grep clickhouse From 030ee3a525917eb9ffd9423622bdf5f4801746b5 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 20 Mar 2023 12:27:53 +0000 Subject: [PATCH 228/418] Add ComparisonTupleEliminationVisitor --- .../ComparisonTupleEliminationVisitor.cpp | 108 ++++++++++++++++++ .../ComparisonTupleEliminationVisitor.h | 28 +++++ src/Interpreters/TreeRewriter.cpp | 8 ++ ...comparsion_tuple_elimination_ast.reference | 7 ++ ...02681_comparsion_tuple_elimination_ast.sql | 8 ++ 5 files changed, 159 insertions(+) create mode 100644 src/Interpreters/ComparisonTupleEliminationVisitor.cpp create mode 100644 src/Interpreters/ComparisonTupleEliminationVisitor.h create mode 100644 tests/queries/0_stateless/02681_comparsion_tuple_elimination_ast.reference create mode 100644 tests/queries/0_stateless/02681_comparsion_tuple_elimination_ast.sql diff --git a/src/Interpreters/ComparisonTupleEliminationVisitor.cpp b/src/Interpreters/ComparisonTupleEliminationVisitor.cpp new file mode 100644 index 00000000000..8aa504395a3 --- /dev/null +++ b/src/Interpreters/ComparisonTupleEliminationVisitor.cpp @@ -0,0 +1,108 @@ +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace +{ + + +ASTs splitTuple(const ASTPtr & node) +{ + if (const auto * func = node->as(); func && func->name == "tuple") + return func->arguments->children; + + if (const auto * literal = node->as(); literal && literal->value.getType() == Field::Types::Tuple) + { + ASTs result; + const auto & tuple = literal->value.get(); + for (const auto & child : tuple) + result.emplace_back(std::make_shared(child)); + return result; + } + + return {}; +} + + +ASTPtr concatWithAnd(const ASTs & nodes) +{ + if (nodes.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot concat empty list of nodes"); + + if (nodes.size() == 1) + return nodes[0]; + + auto result = makeASTFunction("and"); + result->arguments->children = nodes; + return result; +} + +void trySplitTupleComparsion(ASTPtr & expression) +{ + if (!expression) + return; + + auto * func = expression->as(); + if (!func) + return; + + if (func->name == "and" || func->name == "or" || func->name == "not" || func->name == "tuple") + { + for (auto & child : func->arguments->children) + { + trySplitTupleComparsion(child); + } + } + + if (func->name == "equals" || func->name == "notEquals") + { + if (func->arguments->children.size() != 2) + return; + + auto lhs = splitTuple(func->arguments->children[0]); + auto rhs = splitTuple(func->arguments->children[1]); + if (lhs.size() != rhs.size() || lhs.empty() || rhs.empty()) + return; + + ASTs new_args; + for (size_t i = 0; i < lhs.size(); ++i) + { + trySplitTupleComparsion(lhs[i]); + trySplitTupleComparsion(rhs[i]); + new_args.emplace_back(makeASTFunction("equals", lhs[i], rhs[i])); + } + + if (func->name == "notEquals") + expression = makeASTFunction("not", concatWithAnd(new_args)); + else + expression = concatWithAnd(new_args); + } +} + +} + +bool ComparisonTupleEliminationMatcher::needChildVisit(ASTPtr &, const ASTPtr &) +{ + return true; +} + +void ComparisonTupleEliminationMatcher::visit(ASTPtr & ast, Data &) +{ + auto * select_ast = ast->as(); + if (!select_ast || !select_ast->where()) + return; + + if (select_ast->where()) + trySplitTupleComparsion(select_ast->refWhere()); +} + +} diff --git a/src/Interpreters/ComparisonTupleEliminationVisitor.h b/src/Interpreters/ComparisonTupleEliminationVisitor.h new file mode 100644 index 00000000000..1ad7b089d5d --- /dev/null +++ b/src/Interpreters/ComparisonTupleEliminationVisitor.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class ASTSelectQuery; +struct TableWithColumnNamesAndTypes; + +/** Replaces tuple comparisons with multiple comparisons. + * + * Example: SELECT id FROM test_table WHERE (id, value) = (1, 'Value'); + * Result: SELECT id FROM test_table WHERE id = 1 AND value = 'Value'; + */ +class ComparisonTupleEliminationMatcher +{ +public: + struct Data {}; + + static bool needChildVisit(ASTPtr &, const ASTPtr &); + static void visit(ASTPtr & ast, Data & data); +}; + +using ComparisonTupleEliminationVisitor = InDepthNodeVisitor; + +} diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 4c134e175dc..716d63f03e4 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -8,6 +8,7 @@ #include #include +#include #include #include #include /// getSmallestColumn() @@ -1424,6 +1425,13 @@ void TreeRewriter::normalize( if (context_->getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY && settings.normalize_function_names) FunctionNameNormalizer().visit(query.get()); + if (settings.optimize_move_to_prewhere) + { + /// Required for PREWHERE + ComparisonTupleEliminationVisitor::Data data_comparison_tuple_elimination; + ComparisonTupleEliminationVisitor(data_comparison_tuple_elimination).visit(query); + } + /// Common subexpression elimination. Rewrite rules. QueryNormalizer::Data normalizer_data(aliases, source_columns_set, ignore_alias, settings, allow_self_aliases, is_create_parameterized_view); QueryNormalizer(normalizer_data).visit(query); diff --git a/tests/queries/0_stateless/02681_comparsion_tuple_elimination_ast.reference b/tests/queries/0_stateless/02681_comparsion_tuple_elimination_ast.reference new file mode 100644 index 00000000000..5672bd5d716 --- /dev/null +++ b/tests/queries/0_stateless/02681_comparsion_tuple_elimination_ast.reference @@ -0,0 +1,7 @@ +SELECT + a, + b, + c, + d +FROM t1 +WHERE (((a = 1) AND (b = 2)) AND ((c = 3) AND (d = 4) AND (a = 5))) OR ((a = c) AND (b = 10) AND (1000 = d)) OR (((a, b) = (c, 10)) AND (1000 = d)) diff --git a/tests/queries/0_stateless/02681_comparsion_tuple_elimination_ast.sql b/tests/queries/0_stateless/02681_comparsion_tuple_elimination_ast.sql new file mode 100644 index 00000000000..7f36b0568c7 --- /dev/null +++ b/tests/queries/0_stateless/02681_comparsion_tuple_elimination_ast.sql @@ -0,0 +1,8 @@ +SET optimize_move_to_prewhere = 1; -- works only for PREWHERE + +CREATE TABLE t1 (a UInt64, b UInt64, c UInt64, d UInt64) ENGINE = Memory; +INSERT INTO t1 SELECT number, number * 10, number * 100, number * 1000 FROM numbers(1000000); + +EXPLAIN SYNTAX +SELECT * FROM t1 +WHERE (a, b) = (1, 2) AND (c, d, a) = (3, 4, 5) OR (a, b, 1000) = (c, 10, d) OR ((a, b), 1000) = ((c, 10), d); From c25170381c533706a14cbc7b040a8de6b34aef1f Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 21 Mar 2023 16:23:09 +0000 Subject: [PATCH 229/418] Better ComparisonTupleEliminationVisitor --- .../ComparisonTupleEliminationVisitor.cpp | 44 ++++++++----------- ...comparsion_tuple_elimination_ast.reference | 2 +- 2 files changed, 19 insertions(+), 27 deletions(-) diff --git a/src/Interpreters/ComparisonTupleEliminationVisitor.cpp b/src/Interpreters/ComparisonTupleEliminationVisitor.cpp index 8aa504395a3..4f06f345b96 100644 --- a/src/Interpreters/ComparisonTupleEliminationVisitor.cpp +++ b/src/Interpreters/ComparisonTupleEliminationVisitor.cpp @@ -14,7 +14,6 @@ namespace ErrorCodes namespace { - ASTs splitTuple(const ASTPtr & node) { if (const auto * func = node->as(); func && func->name == "tuple") @@ -32,7 +31,6 @@ ASTs splitTuple(const ASTPtr & node) return {}; } - ASTPtr concatWithAnd(const ASTs & nodes) { if (nodes.empty()) @@ -46,47 +44,41 @@ ASTPtr concatWithAnd(const ASTs & nodes) return result; } -void trySplitTupleComparsion(ASTPtr & expression) +class SplitTupleComparsionExpressionMatcher { - if (!expression) - return; +public: + using Data = ComparisonTupleEliminationMatcher::Data; - auto * func = expression->as(); - if (!func) - return; - - if (func->name == "and" || func->name == "or" || func->name == "not" || func->name == "tuple") + static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } + static void visit(ASTPtr & ast, Data &) { - for (auto & child : func->arguments->children) - { - trySplitTupleComparsion(child); - } - } + auto * func = ast->as(); + if (!func || func->arguments->children.size() != 2) + return; - if (func->name == "equals" || func->name == "notEquals") - { - if (func->arguments->children.size() != 2) + if (func->name != "equals" && func->name != "notEquals") return; auto lhs = splitTuple(func->arguments->children[0]); auto rhs = splitTuple(func->arguments->children[1]); - if (lhs.size() != rhs.size() || lhs.empty() || rhs.empty()) + if (lhs.size() != rhs.size() || lhs.empty()) return; ASTs new_args; + new_args.reserve(lhs.size()); for (size_t i = 0; i < lhs.size(); ++i) { - trySplitTupleComparsion(lhs[i]); - trySplitTupleComparsion(rhs[i]); new_args.emplace_back(makeASTFunction("equals", lhs[i], rhs[i])); } if (func->name == "notEquals") - expression = makeASTFunction("not", concatWithAnd(new_args)); + ast = makeASTFunction("not", concatWithAnd(new_args)); else - expression = concatWithAnd(new_args); + ast = concatWithAnd(new_args); } -} +}; + +using SplitTupleComparsionExpressionVisitor = InDepthNodeVisitor; } @@ -95,14 +87,14 @@ bool ComparisonTupleEliminationMatcher::needChildVisit(ASTPtr &, const ASTPtr &) return true; } -void ComparisonTupleEliminationMatcher::visit(ASTPtr & ast, Data &) +void ComparisonTupleEliminationMatcher::visit(ASTPtr & ast, Data & data) { auto * select_ast = ast->as(); if (!select_ast || !select_ast->where()) return; if (select_ast->where()) - trySplitTupleComparsion(select_ast->refWhere()); + SplitTupleComparsionExpressionVisitor(data).visit(select_ast->refWhere()); } } diff --git a/tests/queries/0_stateless/02681_comparsion_tuple_elimination_ast.reference b/tests/queries/0_stateless/02681_comparsion_tuple_elimination_ast.reference index 5672bd5d716..6d848fbd4b1 100644 --- a/tests/queries/0_stateless/02681_comparsion_tuple_elimination_ast.reference +++ b/tests/queries/0_stateless/02681_comparsion_tuple_elimination_ast.reference @@ -4,4 +4,4 @@ SELECT c, d FROM t1 -WHERE (((a = 1) AND (b = 2)) AND ((c = 3) AND (d = 4) AND (a = 5))) OR ((a = c) AND (b = 10) AND (1000 = d)) OR (((a, b) = (c, 10)) AND (1000 = d)) +WHERE (((a = 1) AND (b = 2)) AND ((c = 3) AND (d = 4) AND (a = 5))) OR ((a = c) AND (b = 10) AND (1000 = d)) OR (((a = c) AND (b = 10)) AND (1000 = d)) From cd0ac7a0e1b4e4e67485a605fa7076c92e007113 Mon Sep 17 00:00:00 2001 From: Rich Raposa Date: Tue, 21 Mar 2023 11:03:44 -0600 Subject: [PATCH 230/418] Update s3.md --- docs/en/sql-reference/table-functions/s3.md | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index 6e32afb59a1..99b7832394d 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -65,9 +65,7 @@ LIMIT 5; ``` :::note -ClickHouse uses filename extensions to determine the format of the data and also the compression of the file. For example, if the file was zipped up with a `.csv.gz` extension, ClickHouse would decompress the file. - -To demonstrate, try the following command - which works the same as the example above: +ClickHouse uses filename extensions to determine the format of the data. For example, we could have run the previous command without the `CSVWithNames`: ``` sql SELECT * @@ -77,6 +75,7 @@ FROM s3( LIMIT 5; ``` +ClickHouse also can determine the compression of the file. For example, if the file was zipped up with a `.csv.gz` extension, ClickHouse would decompress the file automatically. ::: From 9fb415dd7a55181d60f2b6357c619673340ddefa Mon Sep 17 00:00:00 2001 From: Rich Raposa Date: Tue, 21 Mar 2023 11:17:04 -0600 Subject: [PATCH 231/418] Update iceberg.md --- docs/en/sql-reference/table-functions/iceberg.md | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/iceberg.md b/docs/en/sql-reference/table-functions/iceberg.md index fda4d274005..7c2648d3dcf 100644 --- a/docs/en/sql-reference/table-functions/iceberg.md +++ b/docs/en/sql-reference/table-functions/iceberg.md @@ -32,7 +32,13 @@ A table with the specified structure for reading data in the specified Iceberg t SELECT * FROM iceberg('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test') ``` -Using named collections: +:::warning +ClickHouse currently supports reading v1 (v2 support is coming soon!) of the Iceberg format via the `iceberg` table function and `Iceberg` table engine. +::: + +## Defining a named collection + +Here is an example of configuring a named collection for storing the URL and credentials: ```xml From d88f71f2b97ec76ea744fb81c17cceb9b624e202 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 21 Mar 2023 17:34:56 +0100 Subject: [PATCH 232/418] Comment stale cherry-pick PRs to resolve conflicts --- tests/ci/cherry_pick.py | 49 ++++++++++++++++++++++++++++++++++++++--- 1 file changed, 46 insertions(+), 3 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index c0fb26fd4ff..fd783192ef1 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -27,7 +27,7 @@ import argparse import logging import os from contextlib import contextmanager -from datetime import date, timedelta +from datetime import date, datetime, timedelta from subprocess import CalledProcessError from typing import List, Optional @@ -123,7 +123,9 @@ close it. # Going from the tail to keep the order and pop greater index first prs.pop(i) - def process(self, dry_run: bool) -> None: + def process( # pylint: disable=too-many-return-statements + self, dry_run: bool + ) -> None: if self.backported: return if not self.cherrypick_pr: @@ -138,6 +140,11 @@ close it. if self.cherrypick_pr is not None: # Try to merge cherrypick instantly if self.cherrypick_pr.mergeable and self.cherrypick_pr.state != "closed": + if dry_run: + logging.info( + "DRY RUN: Would merge cherry-pick PR for #%s", self.pr.number + ) + return self.cherrypick_pr.merge() # The PR needs update, since PR.merge doesn't update the object self.cherrypick_pr.update() @@ -149,7 +156,7 @@ close it. return self.create_backport() return - elif self.cherrypick_pr.state == "closed": + if self.cherrypick_pr.state == "closed": logging.info( "The cherrypick PR #%s for PR #%s is discarded", self.cherrypick_pr.number, @@ -162,6 +169,7 @@ close it. self.cherrypick_pr.number, self.pr.number, ) + self.ping_cherry_pick_assignees(dry_run) def create_cherrypick(self): # First, create backport branch: @@ -227,6 +235,7 @@ close it. assert self.cherrypick_pr is not None # Checkout the backport branch from the remote and make all changes to # apply like they are only one cherry-pick commit on top of release + logging.info("Creating backport for PR #%s", self.pr.number) git_runner(f"{self.git_prefix} checkout -f {self.backport_branch}") git_runner( f"{self.git_prefix} pull --ff-only {self.REMOTE} {self.backport_branch}" @@ -255,6 +264,40 @@ close it. self.backport_pr.add_to_labels(Labels.BACKPORT) self._assign_new_pr(self.backport_pr) + def ping_cherry_pick_assignees(self, dry_run: bool) -> None: + assert self.cherrypick_pr is not None + logging.info( + "Checking if cherry-pick PR #%s needs to be pinged", + self.cherrypick_pr.number, + ) + since_updated = datetime.now() - self.cherrypick_pr.updated_at + since_updated_str = ( + f"{since_updated.days}d{since_updated.seconds // 3600}" + f"h{since_updated.seconds // 60 % 60}m{since_updated.seconds % 60}s" + ) + if since_updated < timedelta(days=1): + logging.info( + "The cherry-pick PR was updated at %s %s ago, " + "waiting for the next running", + self.cherrypick_pr.updated_at.isoformat(), + since_updated_str, + ) + return + assignees = ", ".join(f"@{user.login}" for user in self.cherrypick_pr.assignees) + comment_body = ( + f"Dear {assignees}, the PR is not updated for {since_updated_str}. " + "Please, either resolve the conflicts, or close it to finish " + f"the backport process of #{self.pr.number}" + ) + if dry_run: + logging.info( + "DRY RUN: would comment the cherry-pick PR #%s:\n", + self.cherrypick_pr.number, + ) + return + + self.cherrypick_pr.create_issue_comment(comment_body) + def _assign_new_pr(self, new_pr: PullRequest) -> None: """Assign `new_pr` to author, merger and assignees of an original PR""" # It looks there some race when multiple .add_to_assignees are executed, From aa079efbb79a70c192365c25108b8403a0b3069b Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 21 Mar 2023 18:29:26 +0100 Subject: [PATCH 233/418] Addressed review comments --- src/Interpreters/JoinedTables.h | 2 +- src/Interpreters/getTableExpressions.cpp | 2 ++ src/Interpreters/getTableExpressions.h | 2 ++ tests/queries/0_stateless/02428_parameterized_view.reference | 1 + tests/queries/0_stateless/02428_parameterized_view.sh | 5 +++++ 5 files changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/JoinedTables.h b/src/Interpreters/JoinedTables.h index 105cfc29a7d..771f5ae6ef0 100644 --- a/src/Interpreters/JoinedTables.h +++ b/src/Interpreters/JoinedTables.h @@ -53,7 +53,7 @@ private: ASTPtr left_table_expression; std::optional left_db_and_table; const ASTSelectQuery & select_query; - bool is_create_parameterized_view; + const bool is_create_parameterized_view; }; } diff --git a/src/Interpreters/getTableExpressions.cpp b/src/Interpreters/getTableExpressions.cpp index 21f89ed2137..70e38526648 100644 --- a/src/Interpreters/getTableExpressions.cpp +++ b/src/Interpreters/getTableExpressions.cpp @@ -73,6 +73,8 @@ ASTPtr extractTableExpression(const ASTSelectQuery & select, size_t table_number return nullptr; } +/// The parameter is_create_parameterized_view is used in getSampleBlock of the subquery. +/// If it is set to true, then query parameters are allowed in the subquery, and that expression is not evaluated. static NamesAndTypesList getColumnsFromTableExpression( const ASTTableExpression & table_expression, ContextPtr context, diff --git a/src/Interpreters/getTableExpressions.h b/src/Interpreters/getTableExpressions.h index 8ddfe4b32c3..02262e5f313 100644 --- a/src/Interpreters/getTableExpressions.h +++ b/src/Interpreters/getTableExpressions.h @@ -20,6 +20,8 @@ const ASTTableExpression * getTableExpression(const ASTSelectQuery & select, siz ASTPtr extractTableExpression(const ASTSelectQuery & select, size_t table_number); +/// The parameter is_create_parameterized_view is used in getSampleBlock of the subquery. It is forwarded to getColumnsFromTableExpression. +/// If it is set to true, then query parameters are allowed in the subquery, and that expression is not evaluated. TablesWithColumns getDatabaseAndTablesWithColumns( const ASTTableExprConstPtrs & table_expressions, ContextPtr context, bool include_alias_cols, bool include_materialized_cols, bool is_create_parameterized_view = false); diff --git a/tests/queries/0_stateless/02428_parameterized_view.reference b/tests/queries/0_stateless/02428_parameterized_view.reference index e436075e1a8..5af9e75680c 100644 --- a/tests/queries/0_stateless/02428_parameterized_view.reference +++ b/tests/queries/0_stateless/02428_parameterized_view.reference @@ -37,3 +37,4 @@ ERROR 10 10 10 +10 diff --git a/tests/queries/0_stateless/02428_parameterized_view.sh b/tests/queries/0_stateless/02428_parameterized_view.sh index 86f131ea0f1..274ee43e00d 100755 --- a/tests/queries/0_stateless/02428_parameterized_view.sh +++ b/tests/queries/0_stateless/02428_parameterized_view.sh @@ -17,6 +17,7 @@ $CLICKHOUSE_CLIENT -q "DROP VIEW IF EXISTS test_02428_pv8" $CLICKHOUSE_CLIENT -q "DROP VIEW IF EXISTS test_02428_pv9" $CLICKHOUSE_CLIENT -q "DROP VIEW IF EXISTS test_02428_pv10" $CLICKHOUSE_CLIENT -q "DROP VIEW IF EXISTS test_02428_pv11" +$CLICKHOUSE_CLIENT -q "DROP VIEW IF EXISTS test_02428_pv12" $CLICKHOUSE_CLIENT -q "DROP VIEW IF EXISTS test_02428_v1" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_02428_Catalog" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS ${CLICKHOUSE_TEST_UNIQUE_NAME}.pv1" @@ -91,6 +92,9 @@ $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02428_pv10(Pri=10)" $CLICKHOUSE_CLIENT -q "CREATE VIEW test_02428_pv11 AS SELECT * from ( SELECT Price FROM test_02428_Catalog WHERE Price={price:UInt64} )" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02428_pv11(price=10)" +$CLICKHOUSE_CLIENT -q "CREATE VIEW test_02428_pv12 AS SELECT * from ( SELECT Price FROM test_02428_Catalog WHERE Price IN (SELECT number FROM numbers({price:UInt64})) )" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02428_pv12(price=11)" + $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv1" $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv2" $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv3" @@ -101,6 +105,7 @@ $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv8" $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv9" $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv10" $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv11" +$CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv12" $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_v1" $CLICKHOUSE_CLIENT -q "DROP TABLE test_02428_Catalog" $CLICKHOUSE_CLIENT -q "DROP TABLE ${CLICKHOUSE_TEST_UNIQUE_NAME}.pv1" From fbef6ec05a56c7d5753510130c01a9ef118b736f Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 15 Mar 2023 10:43:10 +0000 Subject: [PATCH 234/418] Fix test --- .../0_stateless/02676_sqlite_local_path_check.reference | 1 - .../queries/0_stateless/02676_sqlite_local_path_check.sh | 8 -------- 2 files changed, 9 deletions(-) delete mode 100644 tests/queries/0_stateless/02676_sqlite_local_path_check.reference delete 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 deleted file mode 100644 index 573541ac970..00000000000 --- a/tests/queries/0_stateless/02676_sqlite_local_path_check.reference +++ /dev/null @@ -1 +0,0 @@ -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 deleted file mode 100755 index 0cb78db0432..00000000000 --- a/tests/queries/0_stateless/02676_sqlite_local_path_check.sh +++ /dev/null @@ -1,8 +0,0 @@ -#!/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 b432fa112202864914776e16332eb0a990506b3b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 21 Mar 2023 17:56:58 +0000 Subject: [PATCH 235/418] Attempt to use new analyzer. --- .../optimizeUseAggregateProjection.cpp | 192 +++++++++++------- .../optimizeUseNormalProjection.cpp | 58 +----- .../Optimizations/projectionsCommon.cpp | 44 +++- .../Optimizations/projectionsCommon.h | 3 +- src/Storages/MergeTree/MergeTreeData.cpp | 5 +- src/Storages/MergeTree/MergeTreeData.h | 3 +- 6 files changed, 170 insertions(+), 135 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 0d18e8367cf..0eff4a3c3e8 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -10,28 +10,88 @@ #include #include +#include +#include +#include +#include +#include + #include -#include -#include +#include +#include #include +#include #include #include +#include namespace DB::QueryPlanOptimizations { +using DAGIndex = std::unordered_map; +static DAGIndex buildDAGIndex(const ActionsDAG & dag) +{ + DAGIndex index; + for (const auto * output : dag.getOutputs()) + index.emplace(output->result_name, output); + + return index; +} + /// Required analysis info from aggregate projection. struct AggregateProjectionInfo { ActionsDAGPtr before_aggregation; - NamesAndTypesList keys; + Names keys; AggregateDescriptions aggregates; + /// This field is needed for getSampleBlock only. + size_t num_virtual_keys = 0; + /// A context copy from interpreter which was used for analysis. /// Just in case it is used by some function. ContextPtr context; + + /// This is a sample block which we expect before aggregation. + /// Now, it is needed only for minmax_count projection. + Block getSampleBlock() + { + auto index = buildDAGIndex(*before_aggregation); + Block res; + size_t num_keys = keys.size() - num_virtual_keys; + for (size_t i = 0; i < num_keys; ++i) + { + const auto & key = keys[i]; + const ActionsDAG::Node & node = *index.at(key); + res.insert({node.result_type->createColumn(), node.result_type, node.result_name}); + } + + for (const auto & aggregate : aggregates) + { + size_t arguments_size = aggregate.argument_names.size(); + DataTypes argument_types(arguments_size); + for (size_t j = 0; j < arguments_size; ++j) + argument_types[j] = index.at(aggregate.argument_names[j])->result_type; + + auto type = std::make_shared(aggregate.function, argument_types, aggregate.parameters); + + res.insert({ type->createColumn(), type, aggregate.column_name }); + } + + return res; + } }; +/// This is a projection-specific. +/// We can expect specific query tree structure for projection query. +static void replaceStorageInQueryTree(QueryTreeNodePtr & query_tree, const ContextPtr & context, const StoragePtr & storage) +{ + auto replacement = std::make_shared(storage, context); + auto & query_node = query_tree->as(); + auto & join_tree = query_node.getJoinTree(); + query_tree = query_tree->cloneAndReplace(join_tree, std::move(replacement)); +} + /// Get required info from aggregate projection. /// Ideally, this should be pre-calculated and stored inside ProjectionDescription. static AggregateProjectionInfo getAggregatingProjectionInfo( @@ -42,79 +102,48 @@ static AggregateProjectionInfo getAggregatingProjectionInfo( { // 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. - InterpreterSelectQuery interpreter( - projection.query_ast, - context, - Pipe(std::make_shared(metadata_snapshot->getSampleBlock())), - SelectQueryOptions{QueryProcessingStage::WithMergeableState}); + /// This is a query tree from projection query. + /// This query does not contain source table, so it is not valid. + auto query_tree = buildQueryTree(projection.query_ast, context); + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "QueryTree : {}", query_tree->dumpTree()); - const auto & analysis_result = interpreter.getAnalysisResult(); - const auto & query_analyzer = interpreter.getQueryAnalyzer(); + /// Replace a storage so that query tree become valid and possilbe to analyze. + auto storage = std::make_shared(StorageID{"dummy", "dummy"}, metadata_snapshot->getColumns()); + replaceStorageInQueryTree(query_tree, context, storage); + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "QueryTree : {}", query_tree->dumpTree()); - AggregateProjectionInfo info; - info.context = interpreter.getContext(); - info.before_aggregation = analysis_result.before_aggregation; - info.keys = query_analyzer->aggregationKeys(); - info.aggregates = query_analyzer->aggregates(); + /// Aggregated copy-paste from InterpreterSelectQueryAnalyzer. + QueryTreePassManager query_tree_pass_manager(context); + addQueryTreePasses(query_tree_pass_manager); + query_tree_pass_manager.run(query_tree); + + auto columns = metadata_snapshot->getSampleBlock().getColumnsWithTypeAndName(); + auto mutable_context = Context::createCopy(context); + auto global_planner_context = std::make_shared(); + auto planner_context = std::make_shared(mutable_context, std::move(global_planner_context)); + PlannerQueryProcessingInfo info(QueryProcessingStage::FetchColumns, QueryProcessingStage::WithMergeableState); + + auto analysis_result = buildExpressionAnalysisResult(query_tree, columns, planner_context, info); + + const auto & aggregation = analysis_result.getAggregation(); + + AggregateProjectionInfo proj_info; + proj_info.context = planner_context->getQueryContext(); + proj_info.before_aggregation = aggregation.before_aggregation_actions; + proj_info.keys = aggregation.aggregation_keys; + proj_info.aggregates = aggregation.aggregate_descriptions; + proj_info.num_virtual_keys = key_virtual_columns.columns(); /// 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); - info.before_aggregation->getOutputs().push_back(input); - info.keys.push_back(NameAndTypePair{virt_column.name, virt_column.type}); + const auto * input = &proj_info.before_aggregation->addInput(virt_column); + proj_info.before_aggregation->getOutputs().push_back(input); + proj_info.keys.push_back(virt_column.name); } - 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) -{ - DAGIndex index; - for (const auto * output : dag.getOutputs()) - index.emplace(output->result_name, output); - - return index; + return proj_info; } static bool hasNullableOrMissingColumn(const DAGIndex & index, const Names & names) @@ -140,7 +169,7 @@ bool areAggregatesMatch( const DAGIndex & proj_index) { /// Index (projection agg function name) -> pos - std::unordered_map> projection_aggregate_functions; + 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); @@ -259,7 +288,7 @@ bool areAggregatesMatch( ActionsDAGPtr analyzeAggregateProjection( const AggregateProjectionInfo & info, - const AggregateQueryDAG & query, + const QueryDAG & query, const Names & keys, const AggregateDescriptions & aggregates) { @@ -286,7 +315,7 @@ ActionsDAGPtr analyzeAggregateProjection( for (const auto & key : info.keys) { - auto it = proj_index.find(key.name); + auto it = proj_index.find(key); /// This should not happen ideally. if (it == proj_index.end()) return {}; @@ -443,11 +472,11 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Has agg projection"); - AggregateQueryDAG dag; + QueryDAG dag; if (!dag.build(*node.children.front())) return candidates; - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", dag.dag->dumpDAG()); + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", dag.dag->dumpDAG()); candidates.has_filter = dag.filter_node; @@ -459,10 +488,22 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( // 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()); + 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; + /// Why do we need this sample block? + /// Currently, we are using InterpreterSelectQuery in order to analyze minmax_count projection. + /// This is gives different columns names after aggregation than InterpreterSelectAnalyzerQuery. + /// + /// It would be good to use InterpreterSelectAnalyzerQuery to analyze projection as well. + /// Now I can't do it cause it will breake old projection analysis which should be kept for some time. + /// + /// So, here we re-calculate the sample block the way it should be in a new analyzer. + /// Hopefully the column order is the same. + auto sample_block = candidate.info.getSampleBlock(); + + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection sample block {}", sample_block.dumpStructure()); auto block = reading.getMergeTreeData().getMinMaxCountProjectionBlock( metadata, candidate.dag->getRequiredColumnsNames(), @@ -471,7 +512,10 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( parts, minmax_projection_normal_parts, max_added_blocks.get(), - context); + context, + &sample_block); + + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection sample block 2 {}", block.dumpStructure()); if (block) { diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index 847173788dd..b46205260fe 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -7,8 +7,6 @@ #include #include #include -#include -#include #include #include @@ -72,52 +70,6 @@ static bool hasAllRequiredColumns(const ProjectionDescription * projection, cons 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) { @@ -135,8 +87,6 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) { 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; @@ -153,11 +103,17 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) if (normal_projections.empty()) return false; - NormalQueryDAG query; + QueryDAG query; { auto & clild = iter->node->children[iter->next_child - 1]; if (!query.build(*clild)) return false; + + if (query.dag) + { + query.dag->removeUnusedActions(); + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", query.dag->dumpDAG()); + } } std::list candidates; diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index b4d1e5f02ff..1cf72eec5e3 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -5,6 +5,8 @@ #include #include +#include +#include #include #include @@ -83,7 +85,7 @@ const ActionsDAG::Node * findInOutputs(ActionsDAG & dag, const std::string & nam return nullptr; } -bool QueryDAG::build(QueryPlan::Node & node) +bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs & filter_nodes) { IQueryPlanStep * step = node.step.get(); if (auto * reading = typeid_cast(step)) @@ -93,8 +95,8 @@ bool QueryDAG::build(QueryPlan::Node & node) 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); + if (const auto * filter_expression = findInOutputs(*dag, prewhere_info->row_level_column_name, false)) + filter_nodes.push_back(filter_expression); else return false; } @@ -102,8 +104,8 @@ bool QueryDAG::build(QueryPlan::Node & node) 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); + if (const auto * filter_expression = findInOutputs(*dag, prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column)) + filter_nodes.push_back(filter_expression); else return false; } @@ -114,7 +116,7 @@ bool QueryDAG::build(QueryPlan::Node & node) if (node.children.size() != 1) return false; - if (!build(*node.children.front())) + if (!buildImpl(*node.children.front(), filter_nodes)) return false; if (auto * expression = typeid_cast(step)) @@ -145,6 +147,36 @@ bool QueryDAG::build(QueryPlan::Node & node) return false; } +bool QueryDAG::build(QueryPlan::Node & node) +{ + ActionsDAG::NodeRawConstPtrs filter_nodes; + if (!buildImpl(node, filter_nodes)) + return false; + + 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"); + + auto & outputs = dag->getOutputs(); + outputs.insert(outputs.begin(), filter_node); + } + + return true; +} + bool analyzeProjectionCandidate( ProjectionCandidate & candidate, const ReadFromMergeTree & reading, diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.h b/src/Processors/QueryPlan/Optimizations/projectionsCommon.h index 04a268edab3..7233a7de640 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.h +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.h @@ -42,11 +42,12 @@ std::shared_ptr getMaxAddedBlocks(ReadFromMergeTree * rea struct QueryDAG { ActionsDAGPtr dag; - ActionsDAG::NodeRawConstPtrs filter_nodes; + const ActionsDAG::Node * filter_node = nullptr; bool build(QueryPlan::Node & node); private: + bool buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs & filter_nodes); void appendExpression(const ActionsDAGPtr & expression); }; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a290588166c..d39eca5508d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6193,14 +6193,15 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( const DataPartsVector & parts, DataPartsVector & normal_parts, const PartitionIdToMaxBlock * max_block_numbers_to_read, - ContextPtr query_context) const + ContextPtr query_context, + Block * sample_block) const { if (!metadata_snapshot->minmax_count_projection) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find the definition of minmax_count projection but it's used in current query. " "It's a bug"); - auto block = metadata_snapshot->minmax_count_projection->sample_block.cloneEmpty(); + auto block = sample_block ? *sample_block : metadata_snapshot->minmax_count_projection->sample_block.cloneEmpty(); bool need_primary_key_max_column = false; const auto & primary_key_max_column_name = metadata_snapshot->minmax_count_projection->primary_key_max_column_name; NameSet required_columns_set(required_columns.begin(), required_columns.end()); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 4a1aafe20b6..0f7479c7e7b 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -398,7 +398,8 @@ public: const DataPartsVector & parts, DataPartsVector & normal_parts, const PartitionIdToMaxBlock * max_block_numbers_to_read, - ContextPtr query_context) const; + ContextPtr query_context, + Block * sample_block = nullptr) const; std::optional getQueryProcessingStageWithAggregateProjection( ContextPtr query_context, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info) const; From 41eff365a729f5a455fc76f7ee9acf0b7df438eb Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 21 Mar 2023 13:58:10 -0400 Subject: [PATCH 236/418] update doc for deflate_qpl --- docs/en/development/building_and_benchmarking_deflate_qpl | 6 +++--- docs/en/sql-reference/statements/create/table.md | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/development/building_and_benchmarking_deflate_qpl b/docs/en/development/building_and_benchmarking_deflate_qpl index 09f4fcbee29..a3a93323eeb 100644 --- a/docs/en/development/building_and_benchmarking_deflate_qpl +++ b/docs/en/development/building_and_benchmarking_deflate_qpl @@ -1,10 +1,10 @@ --- -slug: /en/development/contrib +slug: /en/development/building_and_benchmarking_deflate_qpl sidebar_position: 73 sidebar_label: Building and Benchmarking DEFLATE_QPL description: How to build Clickhouse and run benchmark with DEFLATE_QPL Codec --- -# Build +# Build Clickhouse with DEFLATE_QPL - Make sure your target machine meet the QPL required [Prerequisites](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#prerequisites) - Pass the following flag to CMake when building ClickHouse, depending on the capabilities of your target machine: ``` bash @@ -16,7 +16,7 @@ cmake -DENABLE_AVX512=1 -DENABLE_QPL=1 .. ``` - For generic requirements, please refer to Clickhouse generic [build instructions](/docs/en/development/build.md) -# Benchmark +# Run Benchmark with DEFLATE_QPL ## Files list The folders under [benchmark_sample](https://github.com/ClickHouse/ClickHouse/tree/master/contrib/qpl-cmake/benchmark_sample) give example to run benchmark with python scripts: diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index 556d1fb9e93..220b6e885c8 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -379,8 +379,8 @@ High compression levels are useful for asymmetric scenarios, like compress once, `DEFLATE_QPL` — [Deflate compression algorithm](https://github.com/intel/qpl) implemented by Intel® Query Processing Library. Some limitations apply: - DEFLATE_QPL is experimental and can only be used after setting configuration parameter `allow_experimental_codecs=1`. -- DEFLATE_QPL only works if ClickHouse was compiled with support for AVX2 or AVX512 instructions. Refer to [Build Tips](https://github.com/ClickHouse/ClickHouse/tree/master/contrib/qpl-cmake/doc/build.md) for more details. -- DEFLATE_QPL works best if the system has a Intel® IAA (In-Memory Analytics Accelerator) offloading device. Refer to [IAA Setup](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#accelerator-configuration) and [IAA Benchmark](https://github.com/ClickHouse/ClickHouse/tree/master/contrib/qpl-cmake/doc/benchmark.md) for more details. +- DEFLATE_QPL only works if ClickHouse was compiled with support for AVX2 or AVX512 instructions. Refer to [Build Clickhouse with DEFLATE_QPL](/en/development/building_and_benchmarking_deflate_qpl.md/#Build-Clickhouse-with-DEFLATE_QPL) for more details. +- DEFLATE_QPL works best if the system has a Intel® IAA (In-Memory Analytics Accelerator) offloading device. Refer to [Accelerator Configuration](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#accelerator-configuration) and [Benchmark with DEFLATE_QPL](/en/development/building_and_benchmarking_deflate_qpl.md/#Run-Benchmark-with-DEFLATE_QPL) for more details. - DEFLATE_QPL-compressed data can only be transferred between ClickHouse nodes compiled with support for AVX2/AVX512 ### Specialized Codecs From ed23d2f3a4faec8567df54990a1aee0389b4d923 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 21 Mar 2023 14:04:16 -0400 Subject: [PATCH 237/418] add missing 'docs' --- docs/en/sql-reference/statements/create/table.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index 220b6e885c8..7e9eb41d891 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -379,8 +379,8 @@ High compression levels are useful for asymmetric scenarios, like compress once, `DEFLATE_QPL` — [Deflate compression algorithm](https://github.com/intel/qpl) implemented by Intel® Query Processing Library. Some limitations apply: - DEFLATE_QPL is experimental and can only be used after setting configuration parameter `allow_experimental_codecs=1`. -- DEFLATE_QPL only works if ClickHouse was compiled with support for AVX2 or AVX512 instructions. Refer to [Build Clickhouse with DEFLATE_QPL](/en/development/building_and_benchmarking_deflate_qpl.md/#Build-Clickhouse-with-DEFLATE_QPL) for more details. -- DEFLATE_QPL works best if the system has a Intel® IAA (In-Memory Analytics Accelerator) offloading device. Refer to [Accelerator Configuration](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#accelerator-configuration) and [Benchmark with DEFLATE_QPL](/en/development/building_and_benchmarking_deflate_qpl.md/#Run-Benchmark-with-DEFLATE_QPL) for more details. +- DEFLATE_QPL only works if ClickHouse was compiled with support for AVX2 or AVX512 instructions. Refer to [Build Clickhouse with DEFLATE_QPL](/docs/en/development/building_and_benchmarking_deflate_qpl.md/#Build-Clickhouse-with-DEFLATE_QPL) for more details. +- DEFLATE_QPL works best if the system has a Intel® IAA (In-Memory Analytics Accelerator) offloading device. Refer to [Accelerator Configuration](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#accelerator-configuration) and [Benchmark with DEFLATE_QPL](/docs/en/development/building_and_benchmarking_deflate_qpl.md/#Run-Benchmark-with-DEFLATE_QPL) for more details. - DEFLATE_QPL-compressed data can only be transferred between ClickHouse nodes compiled with support for AVX2/AVX512 ### Specialized Codecs From 049705836e86557d4d5a746208e47b315c56a395 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 21 Mar 2023 14:06:06 -0400 Subject: [PATCH 238/418] rename with .md --- ...rking_deflate_qpl => building_and_benchmarking_deflate_qpl.md} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename docs/en/development/{building_and_benchmarking_deflate_qpl => building_and_benchmarking_deflate_qpl.md} (100%) diff --git a/docs/en/development/building_and_benchmarking_deflate_qpl b/docs/en/development/building_and_benchmarking_deflate_qpl.md similarity index 100% rename from docs/en/development/building_and_benchmarking_deflate_qpl rename to docs/en/development/building_and_benchmarking_deflate_qpl.md From bb8dff8be107c33dc95cc5665e37a6e4178996c6 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 21 Mar 2023 14:11:44 -0400 Subject: [PATCH 239/418] update file path --- docs/en/development/building_and_benchmarking_deflate_qpl.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/building_and_benchmarking_deflate_qpl.md b/docs/en/development/building_and_benchmarking_deflate_qpl.md index a3a93323eeb..f46ff6dad54 100644 --- a/docs/en/development/building_and_benchmarking_deflate_qpl.md +++ b/docs/en/development/building_and_benchmarking_deflate_qpl.md @@ -18,7 +18,7 @@ cmake -DENABLE_AVX512=1 -DENABLE_QPL=1 .. # Run Benchmark with DEFLATE_QPL ## Files list -The folders under [benchmark_sample](https://github.com/ClickHouse/ClickHouse/tree/master/contrib/qpl-cmake/benchmark_sample) give example to run benchmark with python scripts: +The folders under [benchmark_sample](/contrib/qpl-cmake/benchmark_sample) give example to run benchmark with python scripts: `client_scripts` contains python scripts for running typical benchmark, for example: - `client_stressing_test.py`: The python script for query stress test with [1~4] server instances. From 03f2e5647de8d33b7560e998bfb5f13542df938a Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 21 Mar 2023 14:23:36 -0400 Subject: [PATCH 240/418] update keywords --- .../building_and_benchmarking_deflate_qpl.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/development/building_and_benchmarking_deflate_qpl.md b/docs/en/development/building_and_benchmarking_deflate_qpl.md index f46ff6dad54..6091988b9e6 100644 --- a/docs/en/development/building_and_benchmarking_deflate_qpl.md +++ b/docs/en/development/building_and_benchmarking_deflate_qpl.md @@ -24,9 +24,8 @@ The folders under [benchmark_sample](/contrib/qpl-cmake/benchmark_sample) give e - `client_stressing_test.py`: The python script for query stress test with [1~4] server instances. - `queries_ssb.sql`: The file lists all queries for [Star Schema Benchmark](https://clickhouse.com/docs/en/getting-started/example-datasets/star-schema/) - `allin1_ssb.sh`: This shell script executes benchmark workflow all in one automatically. -- `database_files` means it will store database files according to lz4/deflate/zstd codecs. -`rawdata_dir` means it will store raw data generated by dbgen. +`database_files` means it will store database files according to lz4/deflate/zstd codecs. ## Run benchmark automatically for Star Schema: ``` bash @@ -60,12 +59,13 @@ If you see nothing output, it means IAA is not ready to work. Please check IAA s ## Generate raw data ``` bash -$ cd ./rawdata_dir +$ cd ./benchmark_sample +$ mkdir rawdata_dir && cd rawdata_dir ``` Use [dbgen](https://clickhouse.com/docs/en/getting-started/example-datasets/star-schema) to generate 100 million rows data with the parameters: -s 20 -The files expected to output `./rawdata_dir`: +The files expected to output from `./benchmark_sample/rawdata_dir/ssb-dbgen`: ``` text customer.tbl part.tbl @@ -85,7 +85,7 @@ Here you should see the message `Connected to ClickHouse server` from console wh Complete below three steps mentioned in [Star Schema Benchmark](https://clickhouse.com/docs/en/getting-started/example-datasets/star-schema) - Creating tables in ClickHouse -- Inserting data. Here should use ./rawdata_dir/*.tbl as input data. +- Inserting data. Here should use `./benchmark_sample/rawdata_dir/ssb-dbgen/*.tbl` as input data. - Converting “star schema” to denormalized “flat schema” Set up database with with IAA Deflate codec From 9c822c70c5c6136fbbd7f771c357ba0ec1ac042c Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 21 Mar 2023 19:34:19 +0100 Subject: [PATCH 241/418] Updated retry_count and sleep_time for checking BACKUP_CREATED query as it might take longer to create backup due to increase in data size --- .../test_disallow_concurrency.py | 8 ++++++++ 1 file changed, 8 insertions(+) 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 fb09920245e..41876c39724 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 @@ -143,6 +143,8 @@ def test_concurrent_backups_on_same_node(): nodes[0], f"SELECT status FROM system.backups WHERE status == 'BACKUP_CREATED' AND id = '{id}'", "BACKUP_CREATED", + retry_count=100, + sleep_time=1 ) # This restore part is added to confirm creating an internal backup & restore work @@ -179,6 +181,8 @@ def test_concurrent_backups_on_different_nodes(): nodes[1], f"SELECT status FROM system.backups WHERE status == 'BACKUP_CREATED' AND id = '{id}'", "BACKUP_CREATED", + retry_count=100, + sleep_time=1 ) @@ -202,6 +206,8 @@ def test_concurrent_restores_on_same_node(): nodes[0], f"SELECT status FROM system.backups WHERE status == 'BACKUP_CREATED' AND id = '{id}'", "BACKUP_CREATED", + retry_count=100, + sleep_time=1 ) nodes[0].query( @@ -245,6 +251,8 @@ def test_concurrent_restores_on_different_node(): nodes[1], f"SELECT status FROM system.backups WHERE status == 'BACKUP_CREATED' AND id = '{id}'", "BACKUP_CREATED", + retry_count=100, + sleep_time=1 ) nodes[1].query( From 1ebbfac721079ebd02bb275738296b9ea2a23104 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 21 Mar 2023 20:35:00 +0200 Subject: [PATCH 242/418] Use restore_threads (not backup_threads) for RESTORE ASYNC Signed-off-by: Azat Khuzhin --- src/Backups/BackupsWorker.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index bdcff249e7d..7699641a974 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -441,7 +441,7 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt if (restore_settings.async) { - backups_thread_pool.scheduleOrThrowOnError( + restores_thread_pool.scheduleOrThrowOnError( [this, restore_query, restore_id, backup_name_for_logging, backup_info, restore_settings, restore_coordination, context_in_use] { doRestore( From 2d2183f54f3d17c49467da735f98620fd38907c8 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 21 Mar 2023 18:54:56 +0000 Subject: [PATCH 243/418] Automatic style fix --- .../test_disallow_concurrency.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 41876c39724..9dd0a1c40ef 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 @@ -144,7 +144,7 @@ def test_concurrent_backups_on_same_node(): f"SELECT status FROM system.backups WHERE status == 'BACKUP_CREATED' AND id = '{id}'", "BACKUP_CREATED", retry_count=100, - sleep_time=1 + sleep_time=1, ) # This restore part is added to confirm creating an internal backup & restore work @@ -182,7 +182,7 @@ def test_concurrent_backups_on_different_nodes(): f"SELECT status FROM system.backups WHERE status == 'BACKUP_CREATED' AND id = '{id}'", "BACKUP_CREATED", retry_count=100, - sleep_time=1 + sleep_time=1, ) @@ -207,7 +207,7 @@ def test_concurrent_restores_on_same_node(): f"SELECT status FROM system.backups WHERE status == 'BACKUP_CREATED' AND id = '{id}'", "BACKUP_CREATED", retry_count=100, - sleep_time=1 + sleep_time=1, ) nodes[0].query( @@ -252,7 +252,7 @@ def test_concurrent_restores_on_different_node(): f"SELECT status FROM system.backups WHERE status == 'BACKUP_CREATED' AND id = '{id}'", "BACKUP_CREATED", retry_count=100, - sleep_time=1 + sleep_time=1, ) nodes[1].query( From 0735b667d5ac84c639d46ddc7d744656ef0b5426 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 21 Mar 2023 20:24:14 +0100 Subject: [PATCH 244/418] fix bash --- .../queries/0_stateless/02435_rollback_cancelled_queries.sh | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh index 25d12e75a22..8f8e8cc7ee0 100755 --- a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh +++ b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh @@ -16,7 +16,7 @@ $CLICKHOUSE_CLIENT -q 'create table dedup_test(A Int64) Engine = MergeTree order function insert_data { IMPLICIT=$(( RANDOM % 2 )) - SESSION_ID="${SESSION}_$RANDOM.$RANDOM.$1" + SESSION_ID="${SESSION}_$RANDOM.$RANDOM.$NUM" TXN_SETTINGS="session_id=$SESSION_ID&throw_on_unsupported_query_inside_transaction=0&implicit_transaction=$IMPLICIT" BEGIN="" COMMIT="" @@ -63,7 +63,8 @@ function thread_insert i=2 while true; do export ID="$TEST_MARK$RANDOM-$RANDOM-$i" - bash -c insert_data "$i" 2>&1| grep -Fav "Killed" | grep -Fav "SESSION_IS_LOCKED" | grep -Fav "SESSION_NOT_FOUND" + export NUM="$i" + bash -c insert_data 2>&1| grep -Fav "Killed" | grep -Fav "SESSION_IS_LOCKED" | grep -Fav "SESSION_NOT_FOUND" i=$((i + 1)) done } From bdb59db8ed4fef04f85c04674e97529b046df7b8 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 21 Mar 2023 15:36:38 -0400 Subject: [PATCH 245/418] fixed spelling issues --- .../client_scripts/client_stressing_test.py | 3 +- .../building_and_benchmarking_deflate_qpl.md | 28 ++++++++++--------- 2 files changed, 16 insertions(+), 15 deletions(-) diff --git a/contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py b/contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py index 8a10e9d3500..9ab30e88f19 100644 --- a/contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py +++ b/contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py @@ -198,5 +198,4 @@ if __name__ == "__main__": print( "###Completed! -> ID: %s, clientN: %d, totalT: %.2f s, latencyAVG: %.2f ms, P95: %.2f ms, QPS_Final: %.2f" % (queries_id[query_index], client_number, totalT, totalT * 1000/(curr_loop*client_number), totalP95, ((curr_loop*client_number)/totalT) ) ) query_index += 1 - - print("###Finished!") \ No newline at end of file + print("###Finished!") diff --git a/docs/en/development/building_and_benchmarking_deflate_qpl.md b/docs/en/development/building_and_benchmarking_deflate_qpl.md index 6091988b9e6..371786cb153 100644 --- a/docs/en/development/building_and_benchmarking_deflate_qpl.md +++ b/docs/en/development/building_and_benchmarking_deflate_qpl.md @@ -18,14 +18,14 @@ cmake -DENABLE_AVX512=1 -DENABLE_QPL=1 .. # Run Benchmark with DEFLATE_QPL ## Files list -The folders under [benchmark_sample](/contrib/qpl-cmake/benchmark_sample) give example to run benchmark with python scripts: +The folders `benchmark_sample` under [qpl-cmake](https://github.com/ClickHouse/ClickHouse/tree/master/contrib/qpl-cmake) give example to run benchmark with python scripts: `client_scripts` contains python scripts for running typical benchmark, for example: - `client_stressing_test.py`: The python script for query stress test with [1~4] server instances. - `queries_ssb.sql`: The file lists all queries for [Star Schema Benchmark](https://clickhouse.com/docs/en/getting-started/example-datasets/star-schema/) - `allin1_ssb.sh`: This shell script executes benchmark workflow all in one automatically. -`database_files` means it will store database files according to lz4/deflate/zstd codecs. +`database_files` means it will store database files according to lz4/deflate/zstd codec. ## Run benchmark automatically for Star Schema: ``` bash @@ -34,17 +34,19 @@ $ sh run_ssb.sh ``` After complete, please check all the results in this folder:`./output/` -In case you run into failure, please mannually run benchmark as below sections. +In case you run into failure, please manually run benchmark as below sections. ## Definition [CLICKHOUSE_EXE] means the path of clickhouse executable program. -## Enviroment +## Environment - CPU: Sapphire Rapid - OS Requirements refer to [System Requirements for QPL](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#system-requirements) - IAA Setup refer to [Accelerator Configuration](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#accelerator-configuration) -- Install python modules: pip3 install clickhouse_driver numpy - +- Install python modules: +``` bash +pip3 install clickhouse_driver numpy +``` [Self-check for IAA] ``` bash $ accel-config list | grep -P 'iax|state' @@ -62,7 +64,7 @@ If you see nothing output, it means IAA is not ready to work. Please check IAA s $ cd ./benchmark_sample $ mkdir rawdata_dir && cd rawdata_dir ``` -Use [dbgen](https://clickhouse.com/docs/en/getting-started/example-datasets/star-schema) to generate 100 million rows data with the parameters: +Use [`dbgen`](https://clickhouse.com/docs/en/getting-started/example-datasets/star-schema) to generate 100 million rows data with the parameters: -s 20 The files expected to output from `./benchmark_sample/rawdata_dir/ssb-dbgen`: @@ -81,12 +83,12 @@ $ cd ./database_dir/lz4 $ [CLICKHOUSE_EXE] server -C config_lz4.xml >&/dev/null& $ [CLICKHOUSE_EXE] client ``` -Here you should see the message `Connected to ClickHouse server` from console which means client successfuly setup connection with server. +Here you should see the message `Connected to ClickHouse server` from console which means client successfully setup connection with server. Complete below three steps mentioned in [Star Schema Benchmark](https://clickhouse.com/docs/en/getting-started/example-datasets/star-schema) - Creating tables in ClickHouse - Inserting data. Here should use `./benchmark_sample/rawdata_dir/ssb-dbgen/*.tbl` as input data. -- Converting “star schema” to denormalized “flat schema” +- Converting “star schema” to renormalized “flat schema” Set up database with with IAA Deflate codec @@ -132,7 +134,7 @@ That means IAA devices is not ready, you need check IAA setup again. $ cpupower idle-set -d 3 $ cpupower frequency-set -g performance ``` -- To eliminate impact of memory bound on cross sockets, we use numactl to bind server on one socket and client on another socket. +- To eliminate impact of memory bound on cross sockets, we use `numactl` to bind server on one socket and client on another socket. - Single instance means single server connected with single client Now run benchmark for LZ4/Deflate/ZSTD respectively: @@ -174,8 +176,8 @@ We focus on QPS, please search the keyword: `QPS_Final` and collect statistics ## Benchmark with multi-instances - To reduce impact of memory bound on too much threads, We recommend run benchmark with multi-instances. - Multi-instance means multiple(2 or 4)servers connected with respective client. -- The cores of one socket need to be divided equally and assiged to the servers respectively. -- For multi-instances, must create new folder for each codec and insert dataset by following the similiar steps as single instance. +- The cores of one socket need to be divided equally and assigned to the servers respectively. +- For multi-instances, must create new folder for each codec and insert dataset by following the similar steps as single instance. There are 2 differences: - For client side, you need launch clickhouse with the assigned port during table creation and data insertion. @@ -288,4 +290,4 @@ Each time before launch new clickhouse server, please make sure no background cl $ ps -aux| grep clickhouse $ kill -9 [PID] ``` -By comparing the query list in ./client_scripts/queries_ssb.sql with official [Star Schema Benchmark](https://clickhouse.com/docs/en/getting-started/example-datasets/star-schema), you will find 3 queries are not included: Q1.2/Q1.3/Q3.4 . This is because cpu utilization% is very low <10% for these queries which means cannot demostrate performance difference for codecs. \ No newline at end of file +By comparing the query list in ./client_scripts/queries_ssb.sql with official [Star Schema Benchmark](https://clickhouse.com/docs/en/getting-started/example-datasets/star-schema), you will find 3 queries are not included: Q1.2/Q1.3/Q3.4 . This is because cpu utilization% is very low <10% for these queries which means cannot demonstrate performance differences. From a721c3e3925a9a82352fd23b30e5d271c901ac94 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 21 Mar 2023 16:02:03 -0400 Subject: [PATCH 246/418] remove unused space --- .../client_scripts/client_stressing_test.py | 181 +++++++++++++----- 1 file changed, 129 insertions(+), 52 deletions(-) diff --git a/contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py b/contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py index 9ab30e88f19..f02151a66bb 100644 --- a/contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py +++ b/contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py @@ -16,6 +16,7 @@ max_instances_number = 8 retest_number = 3 retest_tolerance = 10 + def checkInt(str): try: int(str) @@ -25,20 +26,28 @@ def checkInt(str): def setup_client(index): - if index<4: + if index < 4: port_idx = index else: port_idx = index + 4 - client = Client(host='localhost', database='default', user='default', password='',port='900%d'%port_idx) + client = Client( + host="localhost", + database="default", + user="default", + password="", + port="900%d" % port_idx, + ) union_mode_query = "SET union_default_mode='DISTINCT'" client.execute(union_mode_query) return client - + + def warm_client(clientN, clientL, query, loop): for c_idx in range(clientN): for _ in range(loop): clientL[c_idx].execute(query) + def read_queries(queries_list): queries = list() queries_id = list() @@ -50,6 +59,7 @@ def read_queries(queries_list): queries.append(line[1]) return queries_id, queries + def run_task(client, cname, query, loop, query_latency): start_time = time.time() for i in range(loop): @@ -58,10 +68,14 @@ def run_task(client, cname, query, loop, query_latency): end_time = time.time() p95 = np.percentile(query_latency, 95) - print('CLIENT: {0} end. -> P95: %f, qps: %f'.format(cname)%(p95, loop / (end_time - start_time))) + print( + "CLIENT: {0} end. -> P95: %f, qps: %f".format(cname) + % (p95, loop / (end_time - start_time)) + ) + def run_multi_clients(clientN, clientList, query, loop): - client_pids={} + client_pids = {} start_time = time.time() manager = multiprocessing.Manager() query_latency_list0 = manager.list() @@ -72,35 +86,59 @@ def run_multi_clients(clientN, clientList, query, loop): query_latency_list5 = manager.list() query_latency_list6 = manager.list() query_latency_list7 = manager.list() - + for c_idx in range(clientN): - client_name = "Role_%d"%c_idx + client_name = "Role_%d" % c_idx if c_idx == 0: - client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list0)) + client_pids[c_idx] = multiprocessing.Process( + target=run_task, + args=(clientList[c_idx], client_name, query, loop, query_latency_list0), + ) elif c_idx == 1: - client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list1)) + client_pids[c_idx] = multiprocessing.Process( + target=run_task, + args=(clientList[c_idx], client_name, query, loop, query_latency_list1), + ) elif c_idx == 2: - client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list2)) + client_pids[c_idx] = multiprocessing.Process( + target=run_task, + args=(clientList[c_idx], client_name, query, loop, query_latency_list2), + ) elif c_idx == 3: - client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list3)) + client_pids[c_idx] = multiprocessing.Process( + target=run_task, + args=(clientList[c_idx], client_name, query, loop, query_latency_list3), + ) elif c_idx == 4: - client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list4)) + client_pids[c_idx] = multiprocessing.Process( + target=run_task, + args=(clientList[c_idx], client_name, query, loop, query_latency_list4), + ) elif c_idx == 5: - client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list5)) + client_pids[c_idx] = multiprocessing.Process( + target=run_task, + args=(clientList[c_idx], client_name, query, loop, query_latency_list5), + ) elif c_idx == 6: - client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list6)) + client_pids[c_idx] = multiprocessing.Process( + target=run_task, + args=(clientList[c_idx], client_name, query, loop, query_latency_list6), + ) elif c_idx == 7: - client_pids[c_idx] = multiprocessing.Process(target=run_task, args=(clientList[c_idx], client_name, query, loop, query_latency_list7)) + client_pids[c_idx] = multiprocessing.Process( + target=run_task, + args=(clientList[c_idx], client_name, query, loop, query_latency_list7), + ) else: - print('ERROR: CLIENT number dismatch!!') + print("ERROR: CLIENT number dismatch!!") exit() - print('CLIENT: %s start'%client_name) + print("CLIENT: %s start" % client_name) client_pids[c_idx].start() for c_idx in range(clientN): client_pids[c_idx].join() end_time = time.time() - totalT = end_time-start_time + totalT = end_time - start_time query_latencyTotal = list() for item in query_latency_list0: @@ -112,64 +150,78 @@ def run_multi_clients(clientN, clientList, query, loop): for item in query_latency_list3: query_latencyTotal.append(item) for item in query_latency_list4: - query_latencyTotal.append(item) + query_latencyTotal.append(item) for item in query_latency_list5: - query_latencyTotal.append(item) + query_latencyTotal.append(item) for item in query_latency_list6: - query_latencyTotal.append(item) + query_latencyTotal.append(item) for item in query_latency_list7: - query_latencyTotal.append(item) + query_latencyTotal.append(item) totalP95 = np.percentile(query_latencyTotal, 95) * 1000 - return totalT,totalP95 + return totalT, totalP95 + def run_task_caculated(client, cname, query, loop): - query_latency = list() - start_time = time.time() - for i in range(loop): - client.execute(query) - query_latency.append(client.last_query.elapsed) - end_time = time.time() - p95 = np.percentile(query_latency, 95) + query_latency = list() + start_time = time.time() + for i in range(loop): + client.execute(query) + query_latency.append(client.last_query.elapsed) + end_time = time.time() + p95 = np.percentile(query_latency, 95) + def run_multi_clients_caculated(clientN, clientList, query, loop): - client_pids={} + client_pids = {} start_time = time.time() for c_idx in range(clientN): - client_name = "Role_%d"%c_idx - client_pids[c_idx] = multiprocessing.Process(target=run_task_caculated, args=(clientList[c_idx], client_name, query, loop)) + client_name = "Role_%d" % c_idx + client_pids[c_idx] = multiprocessing.Process( + target=run_task_caculated, + args=(clientList[c_idx], client_name, query, loop), + ) client_pids[c_idx].start() for c_idx in range(clientN): client_pids[c_idx].join() end_time = time.time() - totalT = end_time-start_time + totalT = end_time - start_time return totalT + if __name__ == "__main__": client_number = 1 queries = list() queries_id = list() if len(sys.argv) != 3: - print('usage: python3 client_stressing_test.py [queries_file_path] [client_number]') + print( + "usage: python3 client_stressing_test.py [queries_file_path] [client_number]" + ) sys.exit() else: queries_list = sys.argv[1] client_number = int(sys.argv[2]) - print('queries_file_path: %s, client_number: %d'%(queries_list,client_number)) + print( + "queries_file_path: %s, client_number: %d" % (queries_list, client_number) + ) if not os.path.isfile(queries_list) or not os.access(queries_list, os.R_OK): - print('please check the right path for queries file') + print("please check the right path for queries file") sys.exit() - if not checkInt(sys.argv[2]) or int(sys.argv[2]) > max_instances_number or int(sys.argv[2]) < 1: - print('client_number should be in [1~%d]'%max_instances_number) + if ( + not checkInt(sys.argv[2]) + or int(sys.argv[2]) > max_instances_number + or int(sys.argv[2]) < 1 + ): + print("client_number should be in [1~%d]" % max_instances_number) sys.exit() - client_list={} - queries_id,queries = read_queries(queries_list) + client_list = {} + queries_id, queries = read_queries(queries_list) for c_idx in range(client_number): client_list[c_idx] = setup_client(c_idx) - #clear cache + # clear cache os.system("sync; echo 3 > /proc/sys/vm/drop_caches") print("###Polit Run Begin") @@ -179,23 +231,48 @@ if __name__ == "__main__": query_index = 0 for q in queries: - print("\n###START -> Index: %d, ID: %s, Query: %s" % (query_index,queries_id[query_index], q)) + print( + "\n###START -> Index: %d, ID: %s, Query: %s" + % (query_index, queries_id[query_index], q) + ) warm_client(client_number, client_list, q, warmup_runs) print("###Warm Done!") - for j in range(0,retest_number): - totalT = run_multi_clients_caculated(client_number, client_list, q, calculated_runs) - curr_loop = int (seconds * calculated_runs / totalT) + 1 - print("###Calculation Done! -> loopN: %d, expected seconds:%d" % (curr_loop, seconds)) + for j in range(0, retest_number): + totalT = run_multi_clients_caculated( + client_number, client_list, q, calculated_runs + ) + curr_loop = int(seconds * calculated_runs / totalT) + 1 + print( + "###Calculation Done! -> loopN: %d, expected seconds:%d" + % (curr_loop, seconds) + ) - print('###Stress Running! -> %d iterations......' % curr_loop) + print("###Stress Running! -> %d iterations......" % curr_loop) - totalT,totalP95 = run_multi_clients(client_number, client_list, q, curr_loop) + totalT, totalP95 = run_multi_clients( + client_number, client_list, q, curr_loop + ) - if totalT > (seconds - retest_tolerance) and totalT < (seconds + retest_tolerance): + if totalT > (seconds - retest_tolerance) and totalT < ( + seconds + retest_tolerance + ): break else: - print('###totalT:%d is far way from expected seconds:%d. Run again ->j:%d!'%(totalT,seconds,j)) + print( + "###totalT:%d is far way from expected seconds:%d. Run again ->j:%d!" + % (totalT, seconds, j) + ) - print( "###Completed! -> ID: %s, clientN: %d, totalT: %.2f s, latencyAVG: %.2f ms, P95: %.2f ms, QPS_Final: %.2f" % (queries_id[query_index], client_number, totalT, totalT * 1000/(curr_loop*client_number), totalP95, ((curr_loop*client_number)/totalT) ) ) + print( + "###Completed! -> ID: %s, clientN: %d, totalT: %.2f s, latencyAVG: %.2f ms, P95: %.2f ms, QPS_Final: %.2f" + % ( + queries_id[query_index], + client_number, + totalT, + totalT * 1000 / (curr_loop * client_number), + totalP95, + ((curr_loop * client_number) / totalT), + ) + ) query_index += 1 print("###Finished!") From c11436187904f3f1747c133f5671cfad11a4796a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 21 Mar 2023 23:15:59 +0300 Subject: [PATCH 247/418] Update MergeTreeData.h --- src/Storages/MergeTree/MergeTreeData.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index bc5e5bc2d91..0be932ccdaf 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1210,7 +1210,7 @@ protected: { auto it = data_parts_by_info.find(part->info); if (it == data_parts_by_info.end() || (*it).get() != part.get()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} doesn't exist", part->name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} doesn't exist (info: {})", part->name, part->info.getPartNameForLogs()); if (!data_parts_by_state_and_info.modify(data_parts_indexes.project(it), getStateModifier(state))) throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't modify {}", (*it)->getNameWithState()); From 038b56ad6a752bf71ce0254877daf26f03b6d47f Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 21 Mar 2023 16:31:47 -0400 Subject: [PATCH 248/418] reformat doc --- .../client_scripts/client_stressing_test.py | 4 ++-- .../building_and_benchmarking_deflate_qpl.md | 10 ++-------- 2 files changed, 4 insertions(+), 10 deletions(-) diff --git a/contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py b/contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py index f02151a66bb..f12381a198c 100644 --- a/contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py +++ b/contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py @@ -51,7 +51,7 @@ def warm_client(clientN, clientL, query, loop): def read_queries(queries_list): queries = list() queries_id = list() - with open(queries_list, 'r') as f: + with open(queries_list, "r") as f: for line in f: line = line.rstrip() line = line.split("$") @@ -148,7 +148,7 @@ def run_multi_clients(clientN, clientList, query, loop): for item in query_latency_list2: query_latencyTotal.append(item) for item in query_latency_list3: - query_latencyTotal.append(item) + query_latencyTotal.append(item) for item in query_latency_list4: query_latencyTotal.append(item) for item in query_latency_list5: diff --git a/docs/en/development/building_and_benchmarking_deflate_qpl.md b/docs/en/development/building_and_benchmarking_deflate_qpl.md index 371786cb153..207b58cb7f8 100644 --- a/docs/en/development/building_and_benchmarking_deflate_qpl.md +++ b/docs/en/development/building_and_benchmarking_deflate_qpl.md @@ -67,13 +67,7 @@ $ mkdir rawdata_dir && cd rawdata_dir Use [`dbgen`](https://clickhouse.com/docs/en/getting-started/example-datasets/star-schema) to generate 100 million rows data with the parameters: -s 20 -The files expected to output from `./benchmark_sample/rawdata_dir/ssb-dbgen`: -``` text -customer.tbl -part.tbl -supplier.tbl -lineorder.tbl -``` +The files like `*.tbl` are expected to output under `./benchmark_sample/rawdata_dir/ssb-dbgen`: ## Database setup Set up database with LZ4 codec @@ -88,7 +82,7 @@ Here you should see the message `Connected to ClickHouse server` from console wh Complete below three steps mentioned in [Star Schema Benchmark](https://clickhouse.com/docs/en/getting-started/example-datasets/star-schema) - Creating tables in ClickHouse - Inserting data. Here should use `./benchmark_sample/rawdata_dir/ssb-dbgen/*.tbl` as input data. -- Converting “star schema” to renormalized “flat schema” +- Converting “star schema” to de-normalized “flat schema” Set up database with with IAA Deflate codec From 85d38493f670d75ee39999b5d7c1e1e0e0fda5e0 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 16 Mar 2023 19:00:51 +0100 Subject: [PATCH 249/418] Improve ThreadPool ThreadPool::setMaxThreads() can now start new threads (if there are scheduled jobs in the queue for them); ThreadPool::setMaxFreeThreads() can now finish free threads. --- src/Common/ThreadPool.cpp | 170 ++++++++++++++++++++++---------------- src/Common/ThreadPool.h | 9 ++ 2 files changed, 109 insertions(+), 70 deletions(-) diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 2843c4b1ad9..f3f8d36a24b 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -56,10 +56,20 @@ template void ThreadPoolImpl::setMaxThreads(size_t value) { std::lock_guard lock(mutex); + bool need_start_threads = (value > max_threads); + bool need_finish_free_threads = (value < max_free_threads); + max_threads = value; + max_free_threads = std::min(max_free_threads, max_threads); + /// We have to also adjust queue size, because it limits the number of scheduled and already running jobs in total. queue_size = std::max(queue_size, max_threads); jobs.reserve(queue_size); + + if (need_start_threads) + startNewThreadsNoLock(); + else if (need_finish_free_threads) + new_job_or_shutdown.notify_all(); } template @@ -73,7 +83,12 @@ template void ThreadPoolImpl::setMaxFreeThreads(size_t value) { std::lock_guard lock(mutex); + bool need_finish_free_threads = (value < max_free_threads); + max_free_threads = value; + + if (need_finish_free_threads) + new_job_or_shutdown.notify_all(); } template @@ -164,6 +179,42 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, ssize_t priority, std:: return static_cast(true); } +template +void ThreadPoolImpl::startNewThreadsNoLock() +{ + auto try_start_new_thread = [this] + { + if (threads.size() >= scheduled_jobs) + return false; /// not necessary to start more threads, already have a thread per each scheduled job + + if (threads.size() >= max_threads) + return false; /// not allowed to start more threads + + try + { + threads.emplace_front(); + } + catch (...) + { + return false; /// failed to start more threads + } + + try + { + threads.front() = Thread([this, it = threads.begin()] { worker(it); }); + } + catch (...) + { + threads.pop_front(); + return false; /// failed to start more threads + } + + return true; + }; + + while (try_start_new_thread()); +} + template void ThreadPoolImpl::scheduleOrThrowOnError(Job job, ssize_t priority) { @@ -185,20 +236,18 @@ void ThreadPoolImpl::scheduleOrThrow(Job job, ssize_t priority, uint64_t template void ThreadPoolImpl::wait() { - { - std::unique_lock lock(mutex); - /// Signal here just in case. - /// If threads are waiting on condition variables, but there are some jobs in the queue - /// then it will prevent us from deadlock. - new_job_or_shutdown.notify_all(); - job_finished.wait(lock, [this] { return scheduled_jobs == 0; }); + std::unique_lock lock(mutex); + /// Signal here just in case. + /// If threads are waiting on condition variables, but there are some jobs in the queue + /// then it will prevent us from deadlock. + new_job_or_shutdown.notify_all(); + job_finished.wait(lock, [this] { return scheduled_jobs == 0; }); - if (first_exception) - { - std::exception_ptr exception; - std::swap(exception, first_exception); - std::rethrow_exception(exception); - } + if (first_exception) + { + std::exception_ptr exception; + std::swap(exception, first_exception); + std::rethrow_exception(exception); } } @@ -216,17 +265,14 @@ ThreadPoolImpl::~ThreadPoolImpl() template void ThreadPoolImpl::finalize() { - { - std::lock_guard lock(mutex); - shutdown = true; - } + std::unique_lock lock(mutex); + shutdown = true; + new_job_or_shutdown.notify_all(); /// `shutdown` was set - new_job_or_shutdown.notify_all(); - - for (auto & thread : threads) - thread.join(); - - threads.clear(); + /// Wait for all currently running jobs to finish (we don't wait for all scheduled jobs here like the function wait() does). + /// We cannot call thread.join() for each thread here because after a thread finishes it will remove itself from `threads` + /// (see `threads.erase(thread_it)` in the worker() function). + thread_finished.wait(lock, [this] { return threads.empty(); }); } template @@ -268,39 +314,38 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ CurrentMetrics::Increment metric_all_threads( std::is_same_v ? CurrentMetrics::GlobalThread : CurrentMetrics::LocalThread); + /// Run jobs until there are scheduled jobs. while (true) { /// This is inside the loop to also reset previous thread names set inside the jobs. setThreadName("ThreadPool"); Job job; - bool need_shutdown = false; + std::exception_ptr exception_from_job; /// A copy of parent trace context DB::OpenTelemetry::TracingContextOnThread parent_thead_trace_context; { std::unique_lock lock(mutex); - new_job_or_shutdown.wait(lock, [this] { return shutdown || !jobs.empty(); }); - need_shutdown = shutdown; + new_job_or_shutdown.wait(lock, [this] { return !jobs.empty() || shutdown || (threads.size() > scheduled_jobs + max_free_threads); }); - if (!jobs.empty()) + if (shutdown || (threads.size() > scheduled_jobs + max_free_threads)) { - /// boost::priority_queue does not provide interface for getting non-const reference to an element - /// to prevent us from modifying its priority. We have to use const_cast to force move semantics on JobWithPriority::job. - job = std::move(const_cast(jobs.top().job)); - parent_thead_trace_context = std::move(const_cast(jobs.top().thread_trace_context)); - jobs.pop(); - } - else - { - /// shutdown is true, simply finish the thread. + thread_it->detach(); + threads.erase(thread_it); + thread_finished.notify_all(); return; } + chassert(!jobs.empty()); + /// boost::priority_queue does not provide interface for getting non-const reference to an element + /// to prevent us from modifying its priority. We have to use const_cast to force move semantics on JobWithPriority::job. + job = std::move(const_cast(jobs.top().job)); + parent_thead_trace_context = std::move(const_cast(jobs.top().thread_trace_context)); + jobs.pop(); } - if (!need_shutdown) { ALLOW_ALLOCATIONS_IN_SCOPE; @@ -322,50 +367,35 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ if (!thread_name.empty()) thread_trace_context.root_span.operation_name = thread_name; } - - /// job should be reset before decrementing scheduled_jobs to - /// ensure that the Job destroyed before wait() returns. - job = {}; - parent_thead_trace_context.reset(); } catch (...) { - thread_trace_context.root_span.addAttribute(std::current_exception()); - - /// job should be reset before decrementing scheduled_jobs to - /// ensure that the Job destroyed before wait() returns. - job = {}; - parent_thead_trace_context.reset(); - - { - std::lock_guard lock(mutex); - if (!first_exception) - first_exception = std::current_exception(); // NOLINT - if (shutdown_on_exception) - shutdown = true; - --scheduled_jobs; - } - - job_finished.notify_all(); - new_job_or_shutdown.notify_all(); - return; + exception_from_job = std::current_exception(); + thread_trace_context.root_span.addAttribute(exception_from_job); } } + /// job should be reset before decrementing scheduled_jobs to + /// ensure that the Job destroyed before wait() returns. + job = {}; + parent_thead_trace_context.reset(); + { std::lock_guard lock(mutex); + if (exception_from_job) + { + if (!first_exception) + first_exception = exception_from_job; + if (shutdown_on_exception) + shutdown = true; + } + --scheduled_jobs; - if (threads.size() > scheduled_jobs + max_free_threads) - { - thread_it->detach(); - threads.erase(thread_it); - job_finished.notify_all(); - return; - } + job_finished.notify_all(); + if (shutdown) + new_job_or_shutdown.notify_all(); } - - job_finished.notify_all(); } } diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 18be05f6c71..8636bc9f652 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -95,6 +95,7 @@ private: mutable std::mutex mutex; std::condition_variable job_finished; std::condition_variable new_job_or_shutdown; + std::condition_variable thread_finished; size_t max_threads; size_t max_free_threads; @@ -129,6 +130,9 @@ private: void worker(typename std::list::iterator thread_it); + /// Tries to start new threads if there are scheduled jobs and the limit `max_threads` is not reached. Must be called with `mutex` locked. + void startNewThreadsNoLock(); + void finalize(); void onDestroy(); }; @@ -260,6 +264,11 @@ public: return true; } + std::thread::id get_id() const + { + return state ? state->thread_id.load() : std::thread::id{}; + } + protected: struct State { From 0a0e9d060fd17f568370573d4565758555886463 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 17 Mar 2023 10:24:30 +0100 Subject: [PATCH 250/418] Corrections after review. --- src/Common/ThreadPool.cpp | 45 ++++++++++++++++++++------------------- 1 file changed, 23 insertions(+), 22 deletions(-) diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index f3f8d36a24b..a56d8cf878f 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -46,8 +46,8 @@ ThreadPoolImpl::ThreadPoolImpl(size_t max_threads_) template ThreadPoolImpl::ThreadPoolImpl(size_t max_threads_, size_t max_free_threads_, size_t queue_size_, bool shutdown_on_exception_) : max_threads(max_threads_) - , max_free_threads(max_free_threads_) - , queue_size(queue_size_) + , max_free_threads(std::min(max_free_threads_, max_threads)) + , queue_size(queue_size_ ? std::max(queue_size_, max_threads) : 0 /* zero means the queue is unlimited */) , shutdown_on_exception(shutdown_on_exception_) { } @@ -63,13 +63,19 @@ void ThreadPoolImpl::setMaxThreads(size_t value) max_free_threads = std::min(max_free_threads, max_threads); /// We have to also adjust queue size, because it limits the number of scheduled and already running jobs in total. - queue_size = std::max(queue_size, max_threads); + queue_size = queue_size ? std::max(queue_size, max_threads) : 0; jobs.reserve(queue_size); if (need_start_threads) + { + /// Start new threads while there are more scheduled jobs in the queue and the limit `max_threads` is not reached. startNewThreadsNoLock(); + } else if (need_finish_free_threads) + { + /// Wake up free threads so they can finish themselves. new_job_or_shutdown.notify_all(); + } } template @@ -85,17 +91,20 @@ void ThreadPoolImpl::setMaxFreeThreads(size_t value) std::lock_guard lock(mutex); bool need_finish_free_threads = (value < max_free_threads); - max_free_threads = value; + max_free_threads = std::min(value, max_threads); if (need_finish_free_threads) + { + /// Wake up free threads so they can finish themselves. new_job_or_shutdown.notify_all(); + } } template void ThreadPoolImpl::setQueueSize(size_t value) { std::lock_guard lock(mutex); - queue_size = value; + queue_size = value ? std::max(value, max_threads) : 0; /// Reserve memory to get rid of allocations jobs.reserve(queue_size); } @@ -182,21 +191,16 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, ssize_t priority, std:: template void ThreadPoolImpl::startNewThreadsNoLock() { - auto try_start_new_thread = [this] + /// Start new threads while there are more scheduled jobs in the queue and the limit `max_threads` is not reached. + while (threads.size() < std::min(scheduled_jobs, max_threads)) { - if (threads.size() >= scheduled_jobs) - return false; /// not necessary to start more threads, already have a thread per each scheduled job - - if (threads.size() >= max_threads) - return false; /// not allowed to start more threads - try { threads.emplace_front(); } catch (...) { - return false; /// failed to start more threads + break; /// failed to start more threads } try @@ -206,13 +210,9 @@ void ThreadPoolImpl::startNewThreadsNoLock() catch (...) { threads.pop_front(); - return false; /// failed to start more threads + break; /// failed to start more threads } - - return true; - }; - - while (try_start_new_thread()); + } } template @@ -314,7 +314,8 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ CurrentMetrics::Increment metric_all_threads( std::is_same_v ? CurrentMetrics::GlobalThread : CurrentMetrics::LocalThread); - /// Run jobs until there are scheduled jobs. + /// Run jobs while there are scheduled jobs and until some special event occurs (e.g. shutdown, or decreasing the number of max_threads). + /// And if `max_free_threads > 0` we keep this number of threads even when there are no jobs for them currently. while (true) { /// This is inside the loop to also reset previous thread names set inside the jobs. @@ -328,9 +329,9 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ { std::unique_lock lock(mutex); - new_job_or_shutdown.wait(lock, [this] { return !jobs.empty() || shutdown || (threads.size() > scheduled_jobs + max_free_threads); }); + new_job_or_shutdown.wait(lock, [this] { return !jobs.empty() || shutdown || (threads.size() > std::min(max_threads, scheduled_jobs + max_free_threads)); }); - if (shutdown || (threads.size() > scheduled_jobs + max_free_threads)) + if (shutdown || (threads.size() > std::min(max_threads, scheduled_jobs + max_free_threads))) { thread_it->detach(); threads.erase(thread_it); From 096d0132af96e4e0ddd598cdc90853a4ab921563 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 17 Mar 2023 21:52:05 +0100 Subject: [PATCH 251/418] More corrections. --- src/Common/ThreadPool.cpp | 88 +++++++++++++++++++++++++++++---------- src/Common/ThreadPool.h | 2 +- 2 files changed, 66 insertions(+), 24 deletions(-) diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index a56d8cf878f..d0a3f398445 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -183,6 +183,7 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, ssize_t priority, std:: ++scheduled_jobs; } + /// Wake up a free thread to run the new job. new_job_or_shutdown.notify_one(); return static_cast(true); @@ -191,6 +192,9 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, ssize_t priority, std:: template void ThreadPoolImpl::startNewThreadsNoLock() { + if (shutdown) + return; + /// Start new threads while there are more scheduled jobs in the queue and the limit `max_threads` is not reached. while (threads.size() < std::min(scheduled_jobs, max_threads)) { @@ -265,14 +269,21 @@ ThreadPoolImpl::~ThreadPoolImpl() template void ThreadPoolImpl::finalize() { - std::unique_lock lock(mutex); - shutdown = true; - new_job_or_shutdown.notify_all(); /// `shutdown` was set + { + std::lock_guard lock(mutex); + shutdown = true; + /// We don't want threads to remove themselves from `threads` anymore, otherwise `thread.join()` will go wrong below in this function. + threads_remove_themselves = false; + } + + /// Wake up threads so they can finish themselves. + new_job_or_shutdown.notify_all(); /// Wait for all currently running jobs to finish (we don't wait for all scheduled jobs here like the function wait() does). - /// We cannot call thread.join() for each thread here because after a thread finishes it will remove itself from `threads` - /// (see `threads.erase(thread_it)` in the worker() function). - thread_finished.wait(lock, [this] { return threads.empty(); }); + for (auto & thread : threads) + thread.join(); + + threads.clear(); } template @@ -314,32 +325,45 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ CurrentMetrics::Increment metric_all_threads( std::is_same_v ? CurrentMetrics::GlobalThread : CurrentMetrics::LocalThread); - /// Run jobs while there are scheduled jobs and until some special event occurs (e.g. shutdown, or decreasing the number of max_threads). + /// Remove this thread from `threads` and detach it, that must be done before exiting from this worker. + /// We can't wrap the following lambda function into `SCOPE_EXIT` because it requires `mutex` to be locked. + auto detach_thread = [this, thread_it] + { + /// `mutex` is supposed to be already locked. + if (threads_remove_themselves) + { + thread_it->detach(); + threads.erase(thread_it); + } + }; + + /// We'll run jobs in this worker while there are scheduled jobs and until some special event occurs (e.g. shutdown, or decreasing the number of max_threads). /// And if `max_free_threads > 0` we keep this number of threads even when there are no jobs for them currently. while (true) { /// This is inside the loop to also reset previous thread names set inside the jobs. setThreadName("ThreadPool"); - Job job; - std::exception_ptr exception_from_job; - /// A copy of parent trace context DB::OpenTelemetry::TracingContextOnThread parent_thead_trace_context; + /// Get a job from the queue. + Job job; + std::exception_ptr exception_from_job; + bool need_shutdown = false; + { std::unique_lock lock(mutex); - new_job_or_shutdown.wait(lock, [this] { return !jobs.empty() || shutdown || (threads.size() > std::min(max_threads, scheduled_jobs + max_free_threads)); }); + new_job_or_shutdown.wait(lock, [&] { return !jobs.empty() || shutdown || (threads.size() > std::min(max_threads, scheduled_jobs + max_free_threads)); }); + need_shutdown = shutdown; - if (shutdown || (threads.size() > std::min(max_threads, scheduled_jobs + max_free_threads))) + if (jobs.empty()) { - thread_it->detach(); - threads.erase(thread_it); - thread_finished.notify_all(); + /// No jobs and either `shutdown` is set or this thread is excessive. The worker will stop. + detach_thread(); return; } - chassert(!jobs.empty()); /// boost::priority_queue does not provide interface for getting non-const reference to an element /// to prevent us from modifying its priority. We have to use const_cast to force move semantics on JobWithPriority::job. job = std::move(const_cast(jobs.top().job)); @@ -347,6 +371,8 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ jobs.pop(); } + /// Run the job. We don't run jobs after `shutdown` is set. + if (!need_shutdown) { ALLOW_ALLOCATIONS_IN_SCOPE; @@ -368,19 +394,25 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ if (!thread_name.empty()) thread_trace_context.root_span.operation_name = thread_name; } + + /// job should be reset before decrementing scheduled_jobs to + /// ensure that the Job destroyed before wait() returns. + job = {}; } catch (...) { exception_from_job = std::current_exception(); thread_trace_context.root_span.addAttribute(exception_from_job); + + /// job should be reset before decrementing scheduled_jobs to + /// ensure that the Job destroyed before wait() returns. + job = {}; } + + parent_thead_trace_context.reset(); } - /// job should be reset before decrementing scheduled_jobs to - /// ensure that the Job destroyed before wait() returns. - job = {}; - parent_thead_trace_context.reset(); - + /// The job is done. { std::lock_guard lock(mutex); if (exception_from_job) @@ -393,9 +425,19 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ --scheduled_jobs; + if (threads.size() > std::min(max_threads, scheduled_jobs + max_free_threads)) + { + /// This thread is excessive. The worker will stop. + detach_thread(); + job_finished.notify_all(); + if (exception_from_job) + new_job_or_shutdown.notify_all(); /// `shutdown` could be just set, wake up other threads so they can finish themselves. + return; + } + job_finished.notify_all(); - if (shutdown) - new_job_or_shutdown.notify_all(); + if (exception_from_job) + new_job_or_shutdown.notify_all(); /// `shutdown` could be just set, wake up other threads so they can finish themselves. } } } diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 8636bc9f652..a1ca79a1e4b 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -95,7 +95,6 @@ private: mutable std::mutex mutex; std::condition_variable job_finished; std::condition_variable new_job_or_shutdown; - std::condition_variable thread_finished; size_t max_threads; size_t max_free_threads; @@ -103,6 +102,7 @@ private: size_t scheduled_jobs = 0; bool shutdown = false; + bool threads_remove_themselves = true; const bool shutdown_on_exception = true; struct JobWithPriority From 4318212c73e6fbcadac8d06d66c601e8300b86f9 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 20 Mar 2023 16:48:03 +0100 Subject: [PATCH 252/418] Make a condition more clear. --- src/Common/ThreadPool.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index d0a3f398445..caa32b61c65 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -430,14 +430,14 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ /// This thread is excessive. The worker will stop. detach_thread(); job_finished.notify_all(); - if (exception_from_job) - new_job_or_shutdown.notify_all(); /// `shutdown` could be just set, wake up other threads so they can finish themselves. + if (shutdown) + new_job_or_shutdown.notify_all(); /// `shutdown` was set, wake up other threads so they can finish themselves. return; } job_finished.notify_all(); - if (exception_from_job) - new_job_or_shutdown.notify_all(); /// `shutdown` could be just set, wake up other threads so they can finish themselves. + if (shutdown) + new_job_or_shutdown.notify_all(); /// `shutdown` was set, wake up other threads so they can finish themselves. } } } From 5091bd42f0980d04055b21338aa61da674511246 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 21 Mar 2023 21:24:37 +0000 Subject: [PATCH 253/418] Add REGEXP operator (Bug: 47530) --- src/Parsers/ExpressionListParsers.cpp | 1 + tests/queries/0_stateless/25401_regexp_operator.reference | 1 + tests/queries/0_stateless/25401_regexp_operator.sql | 1 + 3 files changed, 3 insertions(+) create mode 100644 tests/queries/0_stateless/25401_regexp_operator.reference create mode 100644 tests/queries/0_stateless/25401_regexp_operator.sql diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 054a22a0c3a..7b9477c5742 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2255,6 +2255,7 @@ std::vector> ParserExpressionImpl::operators_t {"ILIKE", Operator("ilike", 8, 2)}, {"NOT LIKE", Operator("notLike", 8, 2)}, {"NOT ILIKE", Operator("notILike", 8, 2)}, + {"REGEXP", Operator("match", 8, 2)}, {"IN", Operator("in", 8, 2)}, {"NOT IN", Operator("notIn", 8, 2)}, {"GLOBAL IN", Operator("globalIn", 8, 2)}, diff --git a/tests/queries/0_stateless/25401_regexp_operator.reference b/tests/queries/0_stateless/25401_regexp_operator.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/25401_regexp_operator.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/25401_regexp_operator.sql b/tests/queries/0_stateless/25401_regexp_operator.sql new file mode 100644 index 00000000000..5a5275bf1ea --- /dev/null +++ b/tests/queries/0_stateless/25401_regexp_operator.sql @@ -0,0 +1 @@ +SELECT 'ab' REGEXP 'a.*b'; From 42a2370ac6a0f5b1d4a6fbcf4a98abc5a58473b2 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 21 Mar 2023 21:33:12 +0000 Subject: [PATCH 254/418] Docs --- docs/en/sql-reference/functions/string-search-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 58006f98a4c..c5304c433c6 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -375,7 +375,7 @@ For a case-insensitive search or/and in UTF-8 format use functions `multiSearchA In all `multiSearch*` functions the number of needles should be less than 28 because of implementation specification. ::: -## match(haystack, pattern) +## match(haystack, pattern), haystack REGEXP pattern operator Checks whether string `haystack` matches the regular expression `pattern`. The pattern is an [re2 regular expression](https://github.com/google/re2/wiki/Syntax) which has a more limited syntax than Perl regular expressions. From 0a7e7637aa6f0604e94a9f1b770d707ae1f06e1f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 22 Mar 2023 00:50:47 +0100 Subject: [PATCH 255/418] make test_server_reload non-parallel --- tests/integration/parallel.json | 264 --------------------------- tests/integration/parallel_skip.json | 16 +- 2 files changed, 15 insertions(+), 265 deletions(-) delete mode 100644 tests/integration/parallel.json diff --git a/tests/integration/parallel.json b/tests/integration/parallel.json deleted file mode 100644 index 6a630bf251f..00000000000 --- a/tests/integration/parallel.json +++ /dev/null @@ -1,264 +0,0 @@ -[ - "test_atomic_drop_table/test.py::test_atomic_delete_with_stopped_zookeeper", - "test_attach_without_fetching/test.py::test_attach_without_fetching", - "test_broken_part_during_merge/test.py::test_merge_and_part_corruption", - "test_cleanup_dir_after_bad_zk_conn/test.py::test_attach_without_zk", - "test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_bad_zk_conn", - "test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_wrong_replica_name", - "test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_wrong_zk_path", - "test_consistent_parts_after_clone_replica/test.py::test_inconsistent_parts_if_drop_while_replica_not_active", - "test_cross_replication/test.py::test", - "test_ddl_worker_non_leader/test.py::test_non_leader_replica", - "test_delayed_replica_failover/test.py::test", - "test_dictionaries_update_field/test.py::test_update_field[complex_key_hashed_update_field_dictionary-HASHED]", - "test_dictionaries_update_field/test.py::test_update_field[flat_update_field_dictionary-FLAT]", - "test_dictionaries_update_field/test.py::test_update_field[simple_key_hashed_update_field_dictionary-HASHED]", - "test_dictionary_allow_read_expired_keys/test_default_reading.py::test_default_reading", - "test_dictionary_allow_read_expired_keys/test_default_string.py::test_return_real_values", - "test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py::test_simple_dict_get_or_default", - "test_dictionary_allow_read_expired_keys/test_dict_get.py::test_simple_dict_get", - "test_disabled_mysql_server/test.py::test_disabled_mysql_server", - "test_distributed_ddl_on_cross_replication/test.py::test_alter_ddl", - "test_distributed_ddl_on_cross_replication/test.py::test_atomic_database", - "test_distributed_ddl_parallel/test.py::test_all_in_parallel", - "test_distributed_ddl_parallel/test.py::test_slow_dict_load_7", - "test_distributed_ddl_parallel/test.py::test_smoke", - "test_distributed_ddl_parallel/test.py::test_smoke_parallel", - "test_distributed_ddl_parallel/test.py::test_smoke_parallel_dict_reload", - "test_distributed_ddl_parallel/test.py::test_two_in_parallel_two_queued", - "test_distributed_ddl_password/test.py::test_alter", - "test_distributed_ddl_password/test.py::test_truncate", - "test_distributed_ddl/test.py::test_allowed_databases[configs]", - "test_distributed_ddl/test.py::test_allowed_databases[configs_secure]", - "test_distributed_ddl/test.py::test_create_as_select[configs]", - "test_distributed_ddl/test.py::test_create_as_select[configs_secure]", - "test_distributed_ddl/test.py::test_create_reserved[configs]", - "test_distributed_ddl/test.py::test_create_reserved[configs_secure]", - "test_distributed_ddl/test.py::test_create_view[configs]", - "test_distributed_ddl/test.py::test_create_view[configs_secure]", - "test_distributed_ddl/test.py::test_default_database[configs]", - "test_distributed_ddl/test.py::test_default_database[configs_secure]", - "test_distributed_ddl/test.py::test_detach_query[configs]", - "test_distributed_ddl/test.py::test_detach_query[configs_secure]", - "test_distributed_ddl/test.py::test_implicit_macros[configs]", - "test_distributed_ddl/test.py::test_implicit_macros[configs_secure]", - "test_distributed_ddl/test.py::test_kill_query[configs]", - "test_distributed_ddl/test.py::test_kill_query[configs_secure]", - "test_distributed_ddl/test.py::test_macro[configs]", - "test_distributed_ddl/test.py::test_macro[configs_secure]", - "test_distributed_ddl/test.py::test_on_connection_loss[configs]", - "test_distributed_ddl/test.py::test_on_connection_loss[configs_secure]", - "test_distributed_ddl/test.py::test_on_server_fail[configs]", - "test_distributed_ddl/test.py::test_on_server_fail[configs_secure]", - "test_distributed_ddl/test.py::test_on_session_expired[configs]", - "test_distributed_ddl/test.py::test_on_session_expired[configs_secure]", - "test_distributed_ddl/test.py::test_optimize_query[configs]", - "test_distributed_ddl/test.py::test_optimize_query[configs_secure]", - "test_distributed_ddl/test.py::test_rename[configs]", - "test_distributed_ddl/test.py::test_rename[configs_secure]", - "test_distributed_ddl/test.py::test_replicated_without_arguments[configs]", - "test_distributed_ddl/test.py::test_replicated_without_arguments[configs_secure]", - "test_distributed_ddl/test.py::test_simple_alters[configs]", - "test_distributed_ddl/test.py::test_simple_alters[configs_secure]", - "test_distributed_ddl/test.py::test_socket_timeout[configs]", - "test_distributed_ddl/test.py::test_socket_timeout[configs_secure]", - "test_distributed_ddl/test_replicated_alter.py::test_replicated_alters[configs]", - "test_distributed_ddl/test_replicated_alter.py::test_replicated_alters[configs_secure]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node1-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node1-remote]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node2-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node2-remote]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node1-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node1-remote]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node2-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node2-remote]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node1-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node1-remote]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node2-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node2-remote]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node1-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node1-remote]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node2-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node2-remote]", - "test_drop_replica/test.py::test_drop_replica", - "test_hedged_requests_parallel/test.py::test_combination1", - "test_hedged_requests_parallel/test.py::test_combination2", - "test_hedged_requests_parallel/test.py::test_query_with_no_data_to_sample", - "test_hedged_requests_parallel/test.py::test_send_data", - "test_hedged_requests_parallel/test.py::test_send_table_status_sleep", - "test_hedged_requests/test.py::test_combination1", - "test_hedged_requests/test.py::test_combination2", - "test_hedged_requests/test.py::test_combination3", - "test_hedged_requests/test.py::test_combination4", - "test_hedged_requests/test.py::test_long_query", - "test_hedged_requests/test.py::test_receive_timeout1", - "test_hedged_requests/test.py::test_receive_timeout2", - "test_hedged_requests/test.py::test_send_data", - "test_hedged_requests/test.py::test_send_data2", - "test_hedged_requests/test.py::test_send_table_status_sleep", - "test_hedged_requests/test.py::test_send_table_status_sleep2", - "test_hedged_requests/test.py::test_stuck_replica", - "test_https_replication/test.py::test_both_http", - "test_https_replication/test.py::test_both_https", - "test_https_replication/test.py::test_mixed_protocol", - "test_https_replication/test.py::test_replication_after_partition", - "test_insert_into_distributed_sync_async/test.py::test_async_inserts_into_local_shard", - "test_insert_into_distributed_sync_async/test.py::test_insertion_sync", - "test_insert_into_distributed_sync_async/test.py::test_insertion_sync_fails_with_timeout", - "test_insert_into_distributed_sync_async/test.py::test_insertion_sync_with_disabled_timeout", - "test_insert_into_distributed_sync_async/test.py::test_insertion_without_sync_ignores_timeout", - "test_insert_into_distributed/test.py::test_inserts_batching", - "test_insert_into_distributed/test.py::test_inserts_local", - "test_insert_into_distributed/test.py::test_inserts_low_cardinality", - "test_insert_into_distributed/test.py::test_inserts_single_replica_internal_replication", - "test_insert_into_distributed/test.py::test_inserts_single_replica_local_internal_replication", - "test_insert_into_distributed/test.py::test_inserts_single_replica_no_internal_replication", - "test_insert_into_distributed/test.py::test_prefer_localhost_replica", - "test_insert_into_distributed/test.py::test_reconnect", - "test_insert_into_distributed/test.py::test_table_function", - "test_insert_into_distributed_through_materialized_view/test.py::test_inserts_batching SKIPPED", - "test_insert_into_distributed_through_materialized_view/test.py::test_inserts_local", - "test_insert_into_distributed_through_materialized_view/test.py::test_reconnect", - "test_keeper_multinode_blocade_leader/test.py::test_blocade_leader", - "test_keeper_multinode_blocade_leader/test.py::test_blocade_leader_twice", - "test_keeper_multinode_simple/test.py::test_follower_restart", - "test_keeper_multinode_simple/test.py::test_read_write_multinode", - "test_keeper_multinode_simple/test.py::test_session_expiration", - "test_keeper_multinode_simple/test.py::test_simple_replicated_table", - "test_keeper_multinode_simple/test.py::test_watch_on_follower", - "test_limited_replicated_fetches/test.py::test_limited_fetches", - "test_materialized_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[atomic]", - "test_materialized_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[ordinary]", - "test_materialized_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[atomic]", - "test_materialized_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[ordinary]", - "test_materialized_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[atomic]", - "test_materialized_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[ordinary]", - "test_materialized_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[atomic]", - "test_materialized_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[ordinary]", - "test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[atomic]", - "test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[ordinary]", - "test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[atomic]", - "test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[ordinary]", - "test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[atomic]", - "test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[ordinary]", - "test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[atomic]", - "test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[ordinary]", - "test_materialized_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[atomic]", - "test_materialized_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[ordinary]", - "test_materialized_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[atomic]", - "test_materialized_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[ordinary]", - "test_materialized_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[atomic]", - "test_materialized_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[ordinary]", - "test_materialized_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[atomic]", - "test_materialized_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[ordinary]", - "test_materialized_mysql_database/test.py::test_multi_table_update[clickhouse_node0]", - "test_materialized_mysql_database/test.py::test_multi_table_update[clickhouse_node1]", - "test_materialized_mysql_database/test.py::test_mysql_killed_while_insert_5_7[atomic]", - "test_materialized_mysql_database/test.py::test_mysql_killed_while_insert_5_7[ordinary]", - "test_materialized_mysql_database/test.py::test_mysql_killed_while_insert_8_0[atomic]", - "test_materialized_mysql_database/test.py::test_mysql_killed_while_insert_8_0[ordinary]", - "test_materialized_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[atomic]", - "test_materialized_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[ordinary]", - "test_materialized_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[atomic]", - "test_materialized_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[ordinary]", - "test_materialized_mysql_database/test.py::test_mysql_settings[clickhouse_node0]", - "test_materialized_mysql_database/test.py::test_mysql_settings[clickhouse_node1]", - "test_materialized_mysql_database/test.py::test_network_partition_5_7[atomic]", - "test_materialized_mysql_database/test.py::test_network_partition_5_7[ordinary]", - "test_materialized_mysql_database/test.py::test_network_partition_8_0[atomic]", - "test_materialized_mysql_database/test.py::test_network_partition_8_0[ordinary]", - "test_materialized_mysql_database/test.py::test_select_without_columns_5_7[atomic]", - "test_materialized_mysql_database/test.py::test_select_without_columns_5_7[ordinary]", - "test_materialized_mysql_database/test.py::test_select_without_columns_8_0[atomic]", - "test_materialized_mysql_database/test.py::test_select_without_columns_8_0[ordinary]", - "test_materialized_mysql_database/test.py::test_system_parts_table[clickhouse_node0]", - "test_materialized_mysql_database/test.py::test_system_parts_table[clickhouse_node1]", - "test_materialized_mysql_database/test.py::test_system_tables_table[clickhouse_node0]", - "test_materialized_mysql_database/test.py::test_system_tables_table[clickhouse_node1]", - "test_materialized_mysql_database/test.py::test_materialize_with_column_comments[clickhouse_node0]", - "test_materialized_mysql_database/test.py::test_materialize_with_column_comments[clickhouse_node1]", - "test_materialized_mysql_database/test.py::test_materialize_with_enum[clickhouse_node0]", - "test_materialized_mysql_database/test.py::test_materialize_with_enum[clickhouse_node1]", - "test_materialized_mysql_database/test.py::test_utf8mb4[clickhouse_node0]", - "test_materialized_mysql_database/test.py::test_utf8mb4[clickhouse_node1]", - "test_parts_delete_zookeeper/test.py::test_merge_doesnt_work_without_zookeeper", - "test_polymorphic_parts/test.py::test_compact_parts_only", - "test_polymorphic_parts/test.py::test_different_part_types_on_replicas[polymorphic_table_compact-Compact]", - "test_polymorphic_parts/test.py::test_different_part_types_on_replicas[polymorphic_table_wide-Wide]", - "test_polymorphic_parts/test.py::test_in_memory", - "test_polymorphic_parts/test.py::test_in_memory_alters", - "test_polymorphic_parts/test.py::test_in_memory_deduplication", - "test_polymorphic_parts/test.py::test_in_memory_wal_rotate", - "test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node0-second_node0]", - "test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node1-second_node1]", - "test_polymorphic_parts/test.py::test_polymorphic_parts_diff_versions_2 SKIPPED", - "test_polymorphic_parts/test.py::test_polymorphic_parts_diff_versions SKIPPED", - "test_polymorphic_parts/test.py::test_polymorphic_parts_index", - "test_polymorphic_parts/test.py::test_polymorphic_parts_non_adaptive", - "test_quorum_inserts_parallel/test.py::test_parallel_quorum_actually_parallel", - "test_quorum_inserts_parallel/test.py::test_parallel_quorum_actually_quorum", - "test_random_inserts/test.py::test_insert_multithreaded", - "test_random_inserts/test.py::test_random_inserts", - "test_reload_clusters_config/test.py::test_add_cluster", - "test_reload_clusters_config/test.py::test_delete_cluster", - "test_reload_clusters_config/test.py::test_simple_reload", - "test_reload_clusters_config/test.py::test_update_one_cluster", - "test_replace_partition/test.py::test_drop_failover", - "test_replace_partition/test.py::test_normal_work", - "test_replace_partition/test.py::test_replace_after_replace_failover", - "test_replicated_database/test.py::test_alters_from_different_replicas", - "test_replicated_database/test.py::test_create_replicated_table", - "test_replicated_database/test.py::test_recover_staled_replica", - "test_replicated_database/test.py::test_simple_alter_table[MergeTree]", - "test_replicated_database/test.py::test_simple_alter_table[ReplicatedMergeTree]", - "test_replicated_database/test.py::test_startup_without_zk", - "test_replicated_fetches_timeouts/test.py::test_no_stall", - "test_storage_kafka/test.py::test_bad_reschedule", - "test_storage_kafka/test.py::test_commits_of_unprocessed_messages_on_drop", - "test_storage_kafka/test.py::test_exception_from_destructor", - "test_storage_kafka/test.py::test_kafka_commit_on_block_write", - "test_storage_kafka/test.py::test_kafka_consumer_hang", - "test_storage_kafka/test.py::test_kafka_consumer_hang2", - "test_storage_kafka/test.py::test_kafka_csv_with_delimiter", - "test_storage_kafka/test.py::test_kafka_csv_with_thread_per_consumer", - "test_storage_kafka/test.py::test_kafka_duplicates_when_commit_failed", - "test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream", - "test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream_with_random_malformed_json", - "test_storage_kafka/test.py::test_kafka_flush_by_block_size", - "test_storage_kafka/test.py::test_kafka_flush_by_time", - "test_storage_kafka/test.py::test_kafka_flush_on_big_message", - "test_storage_kafka/test.py::test_kafka_formats", - "test_storage_kafka/test.py::test_kafka_formats_with_broken_message", - "test_storage_kafka/test.py::test_kafka_insert", - "test_storage_kafka/test.py::test_kafka_issue11308", - "test_storage_kafka/test.py::test_kafka_issue14202", - "test_storage_kafka/test.py::test_kafka_issue4116", - "test_storage_kafka/test.py::test_kafka_json_as_string", - "test_storage_kafka/test.py::test_kafka_json_without_delimiter", - "test_storage_kafka/test.py::test_kafka_lot_of_partitions_partial_commit_of_bulk", - "test_storage_kafka/test.py::test_kafka_many_materialized_views", - "test_storage_kafka/test.py::test_kafka_materialized_view", - "test_storage_kafka/test.py::test_kafka_materialized_view_with_subquery", - "test_storage_kafka/test.py::test_kafka_no_holes_when_write_suffix_failed", - "test_storage_kafka/test.py::test_kafka_produce_consume", - "test_storage_kafka/test.py::test_kafka_produce_key_timestamp", - "test_storage_kafka/test.py::test_kafka_protobuf", - "test_storage_kafka/test.py::test_kafka_protobuf_no_delimiter", - "test_storage_kafka/test.py::test_kafka_rebalance", - "test_storage_kafka/test.py::test_kafka_select_empty", - "test_storage_kafka/test.py::test_kafka_settings_new_syntax", - "test_storage_kafka/test.py::test_kafka_settings_old_syntax", - "test_storage_kafka/test.py::test_kafka_string_field_on_first_position_in_protobuf", - "test_storage_kafka/test.py::test_kafka_tsv_with_delimiter", - "test_storage_kafka/test.py::test_kafka_unavailable", - "test_storage_kafka/test.py::test_kafka_virtual_columns", - "test_storage_kafka/test.py::test_kafka_virtual_columns2", - "test_storage_kafka/test.py::test_kafka_virtual_columns_with_materialized_view", - "test_storage_kafka/test.py::test_librdkafka_compression", - "test_storage_kafka/test.py::test_premature_flush_on_eof", - "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string", - "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc", - "test_system_clusters_actual_information/test.py::test", - "test_system_metrics/test.py::test_readonly_metrics", - "test_system_replicated_fetches/test.py::test_system_replicated_fetches" -] diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index ba593b76bbf..5aca96467b4 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -51,5 +51,19 @@ "test_global_overcommit_tracker/test.py::test_global_overcommit", "test_user_ip_restrictions/test.py::test_ipv4", - "test_user_ip_restrictions/test.py::test_ipv6" + "test_user_ip_restrictions/test.py::test_ipv6", + + "test_server_reload/test.py::test_change_grpc_port", + "test_server_reload/test.py::test_change_http_handlers", + "test_server_reload/test.py::test_change_http_port", + "test_server_reload/test.py::test_change_listen_host", + "test_server_reload/test.py::test_change_mysql_port", + "test_server_reload/test.py::test_change_postgresql_port", + "test_server_reload/test.py::test_change_tcp_port", + "test_server_reload/test.py::test_reload_via_client", + "test_server_reload/test.py::test_remove_grpc_port", + "test_server_reload/test.py::test_remove_http_port", + "test_server_reload/test.py::test_remove_mysql_port", + "test_server_reload/test.py::test_remove_postgresql_port", + "test_server_reload/test.py::test_remove_tcp_port" ] From ef6b891f289c5248a18de60381387335a2f0a678 Mon Sep 17 00:00:00 2001 From: flynn Date: Wed, 22 Mar 2023 02:47:04 +0000 Subject: [PATCH 256/418] fix --- src/Client/QueryFuzzer.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index 7500f90334e..1938bdfabb5 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -703,6 +703,7 @@ void QueryFuzzer::fuzzExplainQuery(ASTExplainQuery & explain) if (!settings_have_fuzzed) { auto settings_ast = std::make_shared(); + settings_ast->is_standalone = false; fuzzExplainSettings(*settings_ast, explain.getKind()); explain.setSettings(settings_ast); } @@ -920,6 +921,7 @@ void QueryFuzzer::fuzz(ASTPtr & ast) auto explain = std::make_shared(fuzzExplainKind()); auto settings_ast = std::make_shared(); + settings_ast->is_standalone = false; fuzzExplainSettings(*settings_ast, explain->getKind()); explain->setSettings(settings_ast); From ad2f5ba0d6c97125f5b4ba28968a4ab896b0648a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 21 Mar 2023 18:19:09 +0000 Subject: [PATCH 257/418] Fix datetime monotonicity check for LC --- src/Functions/IFunctionDateOrDateTime.h | 7 ++++++- .../02680_datetime64_monotonic_check.reference | 1 + .../02680_datetime64_monotonic_check.sql | 13 +++++++++++++ 3 files changed, 20 insertions(+), 1 deletion(-) diff --git a/src/Functions/IFunctionDateOrDateTime.h b/src/Functions/IFunctionDateOrDateTime.h index 5d3f28bcce8..c22754eecd7 100644 --- a/src/Functions/IFunctionDateOrDateTime.h +++ b/src/Functions/IFunctionDateOrDateTime.h @@ -2,8 +2,10 @@ #include #include #include -#include #include +#include + +#include #include #include #include @@ -60,6 +62,9 @@ public: const auto * type_ptr = &type; + if (const auto * lc_type = checkAndGetDataType(type_ptr)) + type_ptr = lc_type->getDictionaryType().get(); + if (const auto * nullable_type = checkAndGetDataType(type_ptr)) type_ptr = nullable_type->getNestedType().get(); diff --git a/tests/queries/0_stateless/02680_datetime64_monotonic_check.reference b/tests/queries/0_stateless/02680_datetime64_monotonic_check.reference index 24d80c55377..7bff7e0c2e5 100644 --- a/tests/queries/0_stateless/02680_datetime64_monotonic_check.reference +++ b/tests/queries/0_stateless/02680_datetime64_monotonic_check.reference @@ -1 +1,2 @@ 22 0 1 +1970-01-01 02:00:02 diff --git a/tests/queries/0_stateless/02680_datetime64_monotonic_check.sql b/tests/queries/0_stateless/02680_datetime64_monotonic_check.sql index 63ea7a5f639..6036831d05d 100644 --- a/tests/queries/0_stateless/02680_datetime64_monotonic_check.sql +++ b/tests/queries/0_stateless/02680_datetime64_monotonic_check.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS 02680_datetime64_monotonic_check; +DROP TABLE IF EXISTS 02680_datetime_monotonic_check_lc; CREATE TABLE 02680_datetime64_monotonic_check (`t` DateTime64(3), `x` Nullable(Decimal(18, 14))) ENGINE = MergeTree @@ -13,3 +14,15 @@ WHERE toHour_Israel = 0 GROUP BY toHour_UTC, toHour_Israel; DROP TABLE 02680_datetime64_monotonic_check; + +SET allow_suspicious_low_cardinality_types = 1; +CREATE TABLE 02680_datetime_monotonic_check_lc (`timestamp` LowCardinality(UInt32)) +ENGINE = MergeTree +ORDER BY timestamp +SETTINGS index_granularity = 1; + +INSERT INTO 02680_datetime_monotonic_check_lc VALUES (2); + +SELECT toDateTime(timestamp, 'Asia/Jerusalem') FROM 02680_datetime_monotonic_check_lc WHERE toHour(toDateTime(timestamp, 'Asia/Jerusalem')) = 2; + +DROP TABLE 02680_datetime_monotonic_check_lc From 023d14a8943c1de58b3499a3357e8d3591f659e1 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 7 Mar 2023 20:39:26 +0000 Subject: [PATCH 258/418] Support transformQueryForExternalDatabase for analyzer --- src/Analyzer/ArrayJoinNode.cpp | 6 +- src/Analyzer/ArrayJoinNode.h | 2 +- src/Analyzer/ColumnNode.cpp | 4 +- src/Analyzer/ColumnNode.h | 7 +- src/Analyzer/ColumnTransformers.cpp | 12 +- src/Analyzer/ColumnTransformers.h | 6 +- src/Analyzer/ConstantNode.cpp | 5 +- src/Analyzer/ConstantNode.h | 2 +- src/Analyzer/FunctionNode.cpp | 8 +- src/Analyzer/FunctionNode.h | 2 +- src/Analyzer/IQueryTreeNode.cpp | 4 +- src/Analyzer/IQueryTreeNode.h | 13 +- src/Analyzer/IdentifierNode.cpp | 2 +- src/Analyzer/IdentifierNode.h | 2 +- src/Analyzer/InterpolateNode.cpp | 6 +- src/Analyzer/InterpolateNode.h | 2 +- src/Analyzer/JoinNode.cpp | 2 +- src/Analyzer/JoinNode.h | 2 +- src/Analyzer/LambdaNode.cpp | 6 +- src/Analyzer/LambdaNode.h | 2 +- src/Analyzer/ListNode.cpp | 4 +- src/Analyzer/ListNode.h | 2 +- src/Analyzer/MatcherNode.cpp | 4 +- src/Analyzer/MatcherNode.h | 2 +- src/Analyzer/QueryNode.cpp | 30 +-- src/Analyzer/QueryNode.h | 2 +- src/Analyzer/SortNode.cpp | 10 +- src/Analyzer/SortNode.h | 2 +- src/Analyzer/TableFunctionNode.cpp | 4 +- src/Analyzer/TableFunctionNode.h | 2 +- src/Analyzer/TableNode.cpp | 2 +- src/Analyzer/TableNode.h | 2 +- src/Analyzer/UnionNode.cpp | 4 +- src/Analyzer/UnionNode.h | 2 +- src/Analyzer/WindowNode.cpp | 10 +- src/Analyzer/WindowNode.h | 2 +- src/Analyzer/tests/gtest_query_tree_node.cpp | 2 +- src/Common/mysqlxx/Exception.cpp | 6 +- src/Common/mysqlxx/Query.cpp | 2 +- src/Common/mysqlxx/Value.cpp | 6 +- src/Common/mysqlxx/mysqlxx/Exception.h | 2 +- src/Common/tests/gtest_global_register.h | 5 + .../InterpreterSelectQueryAnalyzer.h | 7 + src/Planner/Planner.cpp | 16 +- src/Planner/Planner.h | 3 + .../Algorithms/tests/gtest_graphite.cpp | 11 - src/Processors/Sources/MySQLSource.cpp | 5 + src/Storages/SelectQueryInfo.h | 1 + src/Storages/StorageMySQL.cpp | 1 + src/Storages/StoragePostgreSQL.cpp | 4 +- src/Storages/StorageSQLite.cpp | 1 + src/Storages/StorageXDBC.cpp | 4 +- ..._transform_query_for_external_database.cpp | 223 +++++++++++++----- .../transformQueryForExternalDatabase.cpp | 97 ++++++-- .../transformQueryForExternalDatabase.h | 1 + ...nsformQueryForExternalDatabaseAnalyzer.cpp | 81 +++++++ ...ransformQueryForExternalDatabaseAnalyzer.h | 11 + .../02479_mysql_connect_to_self.reference | 40 +++- .../02479_mysql_connect_to_self.sql | 34 ++- 59 files changed, 555 insertions(+), 187 deletions(-) create mode 100644 src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp create mode 100644 src/Storages/transformQueryForExternalDatabaseAnalyzer.h diff --git a/src/Analyzer/ArrayJoinNode.cpp b/src/Analyzer/ArrayJoinNode.cpp index 490e227d617..1a7969ce64a 100644 --- a/src/Analyzer/ArrayJoinNode.cpp +++ b/src/Analyzer/ArrayJoinNode.cpp @@ -49,7 +49,7 @@ QueryTreeNodePtr ArrayJoinNode::cloneImpl() const return std::make_shared(getTableExpression(), getJoinExpressionsNode(), is_left); } -ASTPtr ArrayJoinNode::toASTImpl() const +ASTPtr ArrayJoinNode::toASTImpl(ConvertToASTOptions options) const { auto array_join_ast = std::make_shared(); array_join_ast->kind = is_left ? ASTArrayJoin::Kind::Left : ASTArrayJoin::Kind::Inner; @@ -63,9 +63,9 @@ ASTPtr ArrayJoinNode::toASTImpl() const auto * column_node = array_join_expression->as(); if (column_node && column_node->getExpression()) - array_join_expression_ast = column_node->getExpression()->toAST(); + array_join_expression_ast = column_node->getExpression()->toAST(options); else - array_join_expression_ast = array_join_expression->toAST(); + array_join_expression_ast = array_join_expression->toAST(options); array_join_expression_ast->setAlias(array_join_expression->getAlias()); array_join_expressions_ast->children.push_back(std::move(array_join_expression_ast)); diff --git a/src/Analyzer/ArrayJoinNode.h b/src/Analyzer/ArrayJoinNode.h index 50d53df465a..f19f1b67971 100644 --- a/src/Analyzer/ArrayJoinNode.h +++ b/src/Analyzer/ArrayJoinNode.h @@ -99,7 +99,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl() const override; + ASTPtr toASTImpl(ConvertToASTOptions options) const override; private: bool is_left = false; diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp index c07d7bab717..568daa321ea 100644 --- a/src/Analyzer/ColumnNode.cpp +++ b/src/Analyzer/ColumnNode.cpp @@ -91,12 +91,12 @@ QueryTreeNodePtr ColumnNode::cloneImpl() const return std::make_shared(column, getSourceWeakPointer()); } -ASTPtr ColumnNode::toASTImpl() const +ASTPtr ColumnNode::toASTImpl(ConvertToASTOptions options) const { std::vector column_identifier_parts; auto column_source = getColumnSourceOrNull(); - if (column_source) + if (column_source && options.fully_qualified_identifiers) { auto node_type = column_source->getNodeType(); if (node_type == QueryTreeNodeType::TABLE || diff --git a/src/Analyzer/ColumnNode.h b/src/Analyzer/ColumnNode.h index 79c0e23c86f..1597cc465af 100644 --- a/src/Analyzer/ColumnNode.h +++ b/src/Analyzer/ColumnNode.h @@ -103,6 +103,11 @@ public: */ QueryTreeNodePtr getColumnSource() const; + void dropColumnSource() + { + getSourceWeakPointer().reset(); + } + /** Get column source. * If column source is not valid null is returned. */ @@ -132,7 +137,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl() const override; + ASTPtr toASTImpl(ConvertToASTOptions options) const override; private: const QueryTreeNodeWeakPtr & getSourceWeakPointer() const diff --git a/src/Analyzer/ColumnTransformers.cpp b/src/Analyzer/ColumnTransformers.cpp index ee336a0e7f3..772916bbed6 100644 --- a/src/Analyzer/ColumnTransformers.cpp +++ b/src/Analyzer/ColumnTransformers.cpp @@ -91,7 +91,7 @@ QueryTreeNodePtr ApplyColumnTransformerNode::cloneImpl() const return std::make_shared(getExpressionNode()); } -ASTPtr ApplyColumnTransformerNode::toASTImpl() const +ASTPtr ApplyColumnTransformerNode::toASTImpl(ConvertToASTOptions options) const { auto ast_apply_transformer = std::make_shared(); const auto & expression_node = getExpressionNode(); @@ -100,14 +100,14 @@ ASTPtr ApplyColumnTransformerNode::toASTImpl() const { auto & function_expression = expression_node->as(); ast_apply_transformer->func_name = function_expression.getFunctionName(); - ast_apply_transformer->parameters = function_expression.getParametersNode()->toAST(); + ast_apply_transformer->parameters = function_expression.getParametersNode()->toAST(options); } else { auto & lambda_expression = expression_node->as(); if (!lambda_expression.getArgumentNames().empty()) ast_apply_transformer->lambda_arg = lambda_expression.getArgumentNames()[0]; - ast_apply_transformer->lambda = lambda_expression.toAST(); + ast_apply_transformer->lambda = lambda_expression.toAST(options); } return ast_apply_transformer; @@ -227,7 +227,7 @@ QueryTreeNodePtr ExceptColumnTransformerNode::cloneImpl() const return std::make_shared(except_column_names, is_strict); } -ASTPtr ExceptColumnTransformerNode::toASTImpl() const +ASTPtr ExceptColumnTransformerNode::toASTImpl(ConvertToASTOptions /* options */) const { auto ast_except_transformer = std::make_shared(); @@ -334,7 +334,7 @@ QueryTreeNodePtr ReplaceColumnTransformerNode::cloneImpl() const return result_replace_transformer; } -ASTPtr ReplaceColumnTransformerNode::toASTImpl() const +ASTPtr ReplaceColumnTransformerNode::toASTImpl(ConvertToASTOptions options) const { auto ast_replace_transformer = std::make_shared(); @@ -347,7 +347,7 @@ ASTPtr ReplaceColumnTransformerNode::toASTImpl() const { auto replacement_ast = std::make_shared(); replacement_ast->name = replacements_names[i]; - replacement_ast->children.push_back(replacement_expressions_nodes[i]->toAST()); + replacement_ast->children.push_back(replacement_expressions_nodes[i]->toAST(options)); ast_replace_transformer->children.push_back(std::move(replacement_ast)); } diff --git a/src/Analyzer/ColumnTransformers.h b/src/Analyzer/ColumnTransformers.h index e96e606d923..2b06e66d07e 100644 --- a/src/Analyzer/ColumnTransformers.h +++ b/src/Analyzer/ColumnTransformers.h @@ -141,7 +141,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl() const override; + ASTPtr toASTImpl(ConvertToASTOptions options) const override; private: ApplyColumnTransformerType apply_transformer_type = ApplyColumnTransformerType::LAMBDA; @@ -220,7 +220,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl() const override; + ASTPtr toASTImpl(ConvertToASTOptions options) const override; private: ExceptColumnTransformerType except_transformer_type; @@ -298,7 +298,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl() const override; + ASTPtr toASTImpl(ConvertToASTOptions options) const override; private: ListNode & getReplacements() diff --git a/src/Analyzer/ConstantNode.cpp b/src/Analyzer/ConstantNode.cpp index 79fc38cd617..8d286cc99a1 100644 --- a/src/Analyzer/ConstantNode.cpp +++ b/src/Analyzer/ConstantNode.cpp @@ -75,11 +75,14 @@ QueryTreeNodePtr ConstantNode::cloneImpl() const return std::make_shared(constant_value, source_expression); } -ASTPtr ConstantNode::toASTImpl() const +ASTPtr ConstantNode::toASTImpl(ConvertToASTOptions options) const { const auto & constant_value_literal = constant_value->getValue(); auto constant_value_ast = std::make_shared(constant_value_literal); + if (!options.add_cast_for_constants) + return constant_value_ast; + bool need_to_add_cast_function = false; auto constant_value_literal_type = constant_value_literal.getType(); WhichDataType constant_value_type(constant_value->getType()); diff --git a/src/Analyzer/ConstantNode.h b/src/Analyzer/ConstantNode.h index 6b58533a701..e7d38c5bbfa 100644 --- a/src/Analyzer/ConstantNode.h +++ b/src/Analyzer/ConstantNode.h @@ -83,7 +83,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl() const override; + ASTPtr toASTImpl(ConvertToASTOptions options) const override; private: ConstantValuePtr constant_value; diff --git a/src/Analyzer/FunctionNode.cpp b/src/Analyzer/FunctionNode.cpp index fe170c8482e..d0344c5efdf 100644 --- a/src/Analyzer/FunctionNode.cpp +++ b/src/Analyzer/FunctionNode.cpp @@ -197,7 +197,7 @@ QueryTreeNodePtr FunctionNode::cloneImpl() const return result_function; } -ASTPtr FunctionNode::toASTImpl() const +ASTPtr FunctionNode::toASTImpl(ConvertToASTOptions options) const { auto function_ast = std::make_shared(); @@ -212,12 +212,12 @@ ASTPtr FunctionNode::toASTImpl() const const auto & parameters = getParameters(); if (!parameters.getNodes().empty()) { - function_ast->children.push_back(parameters.toAST()); + function_ast->children.push_back(parameters.toAST(options)); function_ast->parameters = function_ast->children.back(); } const auto & arguments = getArguments(); - function_ast->children.push_back(arguments.toAST()); + function_ast->children.push_back(arguments.toAST(options)); function_ast->arguments = function_ast->children.back(); auto window_node = getWindowNode(); @@ -226,7 +226,7 @@ ASTPtr FunctionNode::toASTImpl() const if (auto * identifier_node = window_node->as()) function_ast->window_name = identifier_node->getIdentifier().getFullName(); else - function_ast->window_definition = window_node->toAST(); + function_ast->window_definition = window_node->toAST(options); } return function_ast; diff --git a/src/Analyzer/FunctionNode.h b/src/Analyzer/FunctionNode.h index 89a684c1d0f..2e899fe2801 100644 --- a/src/Analyzer/FunctionNode.h +++ b/src/Analyzer/FunctionNode.h @@ -209,7 +209,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl() const override; + ASTPtr toASTImpl(ConvertToASTOptions options) const override; private: String function_name; diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index ba361af0007..7603a8a1593 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -331,9 +331,9 @@ QueryTreeNodePtr IQueryTreeNode::cloneAndReplace(const QueryTreeNodePtr & node_t return cloneAndReplace(replacement_map); } -ASTPtr IQueryTreeNode::toAST() const +ASTPtr IQueryTreeNode::toAST(ConvertToASTOptions options) const { - auto converted_node = toASTImpl(); + auto converted_node = toASTImpl(options); if (auto * ast_with_alias = dynamic_cast(converted_node.get())) converted_node->setAlias(alias); diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index e344dd66fbc..18562214200 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -181,8 +181,17 @@ public: */ String formatOriginalASTForErrorMessage() const; + struct ConvertToASTOptions + { + /// Add _CAST if constant litral type is different from column type + bool add_cast_for_constants = true; + + /// Identifiers are fully qualified (`database.table.column`), otherwise names are just column names (`column`) + bool fully_qualified_identifiers = true; + }; + /// Convert query tree to AST - ASTPtr toAST() const; + ASTPtr toAST(ConvertToASTOptions options = { .add_cast_for_constants = true, .fully_qualified_identifiers = true }) const; /// Convert query tree to AST and then format it for error message. String formatConvertedASTForErrorMessage() const; @@ -258,7 +267,7 @@ protected: virtual QueryTreeNodePtr cloneImpl() const = 0; /// Subclass must convert its internal state and its children to AST - virtual ASTPtr toASTImpl() const = 0; + virtual ASTPtr toASTImpl(ConvertToASTOptions options) const = 0; QueryTreeNodes children; QueryTreeWeakNodes weak_pointers; diff --git a/src/Analyzer/IdentifierNode.cpp b/src/Analyzer/IdentifierNode.cpp index cb5d9609962..3c135a3a2bc 100644 --- a/src/Analyzer/IdentifierNode.cpp +++ b/src/Analyzer/IdentifierNode.cpp @@ -58,7 +58,7 @@ QueryTreeNodePtr IdentifierNode::cloneImpl() const return std::make_shared(identifier); } -ASTPtr IdentifierNode::toASTImpl() const +ASTPtr IdentifierNode::toASTImpl(ConvertToASTOptions /* options */) const { auto identifier_parts = identifier.getParts(); return std::make_shared(std::move(identifier_parts)); diff --git a/src/Analyzer/IdentifierNode.h b/src/Analyzer/IdentifierNode.h index 358511d1f90..ced599218b1 100644 --- a/src/Analyzer/IdentifierNode.h +++ b/src/Analyzer/IdentifierNode.h @@ -59,7 +59,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl() const override; + ASTPtr toASTImpl(ConvertToASTOptions options) const override; private: Identifier identifier; diff --git a/src/Analyzer/InterpolateNode.cpp b/src/Analyzer/InterpolateNode.cpp index c8c61b05853..9df2ac08018 100644 --- a/src/Analyzer/InterpolateNode.cpp +++ b/src/Analyzer/InterpolateNode.cpp @@ -44,11 +44,11 @@ QueryTreeNodePtr InterpolateNode::cloneImpl() const return std::make_shared(nullptr /*expression*/, nullptr /*interpolate_expression*/); } -ASTPtr InterpolateNode::toASTImpl() const +ASTPtr InterpolateNode::toASTImpl(ConvertToASTOptions options) const { auto result = std::make_shared(); - result->column = getExpression()->toAST()->getColumnName(); - result->children.push_back(getInterpolateExpression()->toAST()); + result->column = getExpression()->toAST(options)->getColumnName(); + result->children.push_back(getInterpolateExpression()->toAST(options)); result->expr = result->children.back(); return result; diff --git a/src/Analyzer/InterpolateNode.h b/src/Analyzer/InterpolateNode.h index 5764ea561c0..8ac31f0fb8f 100644 --- a/src/Analyzer/InterpolateNode.h +++ b/src/Analyzer/InterpolateNode.h @@ -59,7 +59,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl() const override; + ASTPtr toASTImpl(ConvertToASTOptions options) const override; private: static constexpr size_t expression_child_index = 0; diff --git a/src/Analyzer/JoinNode.cpp b/src/Analyzer/JoinNode.cpp index fe4dd2c5016..829db614774 100644 --- a/src/Analyzer/JoinNode.cpp +++ b/src/Analyzer/JoinNode.cpp @@ -99,7 +99,7 @@ QueryTreeNodePtr JoinNode::cloneImpl() const return std::make_shared(getLeftTableExpression(), getRightTableExpression(), getJoinExpression(), locality, strictness, kind); } -ASTPtr JoinNode::toASTImpl() const +ASTPtr JoinNode::toASTImpl(ConvertToASTOptions /* options */) const { ASTPtr tables_in_select_query_ast = std::make_shared(); diff --git a/src/Analyzer/JoinNode.h b/src/Analyzer/JoinNode.h index f58fe3f1af5..4ea08c620ef 100644 --- a/src/Analyzer/JoinNode.h +++ b/src/Analyzer/JoinNode.h @@ -148,7 +148,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl() const override; + ASTPtr toASTImpl(ConvertToASTOptions options) const override; private: JoinLocality locality = JoinLocality::Unspecified; diff --git a/src/Analyzer/LambdaNode.cpp b/src/Analyzer/LambdaNode.cpp index b60b40878ec..2df389c4029 100644 --- a/src/Analyzer/LambdaNode.cpp +++ b/src/Analyzer/LambdaNode.cpp @@ -65,17 +65,17 @@ QueryTreeNodePtr LambdaNode::cloneImpl() const return std::make_shared(argument_names, getExpression()); } -ASTPtr LambdaNode::toASTImpl() const +ASTPtr LambdaNode::toASTImpl(ConvertToASTOptions options) const { auto lambda_function_arguments_ast = std::make_shared(); auto tuple_function = std::make_shared(); tuple_function->name = "tuple"; - tuple_function->children.push_back(children[arguments_child_index]->toAST()); + tuple_function->children.push_back(children[arguments_child_index]->toAST(options)); tuple_function->arguments = tuple_function->children.back(); lambda_function_arguments_ast->children.push_back(std::move(tuple_function)); - lambda_function_arguments_ast->children.push_back(children[expression_child_index]->toAST()); + lambda_function_arguments_ast->children.push_back(children[expression_child_index]->toAST(options)); auto lambda_function_ast = std::make_shared(); lambda_function_ast->name = "lambda"; diff --git a/src/Analyzer/LambdaNode.h b/src/Analyzer/LambdaNode.h index 65b0d3de84e..f64d49fff08 100644 --- a/src/Analyzer/LambdaNode.h +++ b/src/Analyzer/LambdaNode.h @@ -98,7 +98,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl() const override; + ASTPtr toASTImpl(ConvertToASTOptions options) const override; private: Names argument_names; diff --git a/src/Analyzer/ListNode.cpp b/src/Analyzer/ListNode.cpp index 7bbb884fa7f..238ba0c8133 100644 --- a/src/Analyzer/ListNode.cpp +++ b/src/Analyzer/ListNode.cpp @@ -54,7 +54,7 @@ QueryTreeNodePtr ListNode::cloneImpl() const return std::make_shared(); } -ASTPtr ListNode::toASTImpl() const +ASTPtr ListNode::toASTImpl(ConvertToASTOptions options) const { auto expression_list_ast = std::make_shared(); @@ -62,7 +62,7 @@ ASTPtr ListNode::toASTImpl() const expression_list_ast->children.resize(children_size); for (size_t i = 0; i < children_size; ++i) - expression_list_ast->children[i] = children[i]->toAST(); + expression_list_ast->children[i] = children[i]->toAST(options); return expression_list_ast; } diff --git a/src/Analyzer/ListNode.h b/src/Analyzer/ListNode.h index 75013f7ee6a..9cf2011eb39 100644 --- a/src/Analyzer/ListNode.h +++ b/src/Analyzer/ListNode.h @@ -57,7 +57,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl() const override; + ASTPtr toASTImpl(ConvertToASTOptions options) const override; }; } diff --git a/src/Analyzer/MatcherNode.cpp b/src/Analyzer/MatcherNode.cpp index 5c8738e0504..d6bdd50562d 100644 --- a/src/Analyzer/MatcherNode.cpp +++ b/src/Analyzer/MatcherNode.cpp @@ -204,7 +204,7 @@ QueryTreeNodePtr MatcherNode::cloneImpl() const return matcher_node; } -ASTPtr MatcherNode::toASTImpl() const +ASTPtr MatcherNode::toASTImpl(ConvertToASTOptions options) const { ASTPtr result; ASTPtr transformers; @@ -216,7 +216,7 @@ ASTPtr MatcherNode::toASTImpl() const transformers = std::make_shared(); for (const auto & column_transformer : column_transformers) - transformers->children.push_back(column_transformer->toAST()); + transformers->children.push_back(column_transformer->toAST(options)); } if (matcher_type == MatcherNodeType::ASTERISK) diff --git a/src/Analyzer/MatcherNode.h b/src/Analyzer/MatcherNode.h index e79c1cb4bf2..985000f7f65 100644 --- a/src/Analyzer/MatcherNode.h +++ b/src/Analyzer/MatcherNode.h @@ -148,7 +148,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl() const override; + ASTPtr toASTImpl(ConvertToASTOptions options) const override; private: explicit MatcherNode(MatcherNodeType matcher_type_, diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 774f3376f48..c4c36c9f192 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -259,7 +259,7 @@ QueryTreeNodePtr QueryNode::cloneImpl() const return result_query_node; } -ASTPtr QueryNode::toASTImpl() const +ASTPtr QueryNode::toASTImpl(ConvertToASTOptions options) const { auto select_query = std::make_shared(); select_query->distinct = is_distinct; @@ -271,9 +271,9 @@ ASTPtr QueryNode::toASTImpl() const select_query->group_by_all = is_group_by_all; if (hasWith()) - select_query->setExpression(ASTSelectQuery::Expression::WITH, getWith().toAST()); + select_query->setExpression(ASTSelectQuery::Expression::WITH, getWith().toAST(options)); - auto projection_ast = getProjection().toAST(); + auto projection_ast = getProjection().toAST(options); auto & projection_expression_list_ast = projection_ast->as(); size_t projection_expression_list_ast_children_size = projection_expression_list_ast.children.size(); if (projection_expression_list_ast_children_size != getProjection().getNodes().size()) @@ -297,40 +297,40 @@ ASTPtr QueryNode::toASTImpl() const select_query->setExpression(ASTSelectQuery::Expression::TABLES, std::move(tables_in_select_query_ast)); if (getPrewhere()) - select_query->setExpression(ASTSelectQuery::Expression::PREWHERE, getPrewhere()->toAST()); + select_query->setExpression(ASTSelectQuery::Expression::PREWHERE, getPrewhere()->toAST(options)); if (getWhere()) - select_query->setExpression(ASTSelectQuery::Expression::WHERE, getWhere()->toAST()); + select_query->setExpression(ASTSelectQuery::Expression::WHERE, getWhere()->toAST(options)); if (!is_group_by_all && hasGroupBy()) - select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, getGroupBy().toAST()); + select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, getGroupBy().toAST(options)); if (hasHaving()) - select_query->setExpression(ASTSelectQuery::Expression::HAVING, getHaving()->toAST()); + select_query->setExpression(ASTSelectQuery::Expression::HAVING, getHaving()->toAST(options)); if (hasWindow()) - select_query->setExpression(ASTSelectQuery::Expression::WINDOW, getWindow().toAST()); + select_query->setExpression(ASTSelectQuery::Expression::WINDOW, getWindow().toAST(options)); if (hasOrderBy()) - select_query->setExpression(ASTSelectQuery::Expression::ORDER_BY, getOrderBy().toAST()); + select_query->setExpression(ASTSelectQuery::Expression::ORDER_BY, getOrderBy().toAST(options)); if (hasInterpolate()) - select_query->setExpression(ASTSelectQuery::Expression::INTERPOLATE, getInterpolate()->toAST()); + select_query->setExpression(ASTSelectQuery::Expression::INTERPOLATE, getInterpolate()->toAST(options)); if (hasLimitByLimit()) - select_query->setExpression(ASTSelectQuery::Expression::LIMIT_BY_LENGTH, getLimitByLimit()->toAST()); + select_query->setExpression(ASTSelectQuery::Expression::LIMIT_BY_LENGTH, getLimitByLimit()->toAST(options)); if (hasLimitByOffset()) - select_query->setExpression(ASTSelectQuery::Expression::LIMIT_BY_OFFSET, getLimitByOffset()->toAST()); + select_query->setExpression(ASTSelectQuery::Expression::LIMIT_BY_OFFSET, getLimitByOffset()->toAST(options)); if (hasLimitBy()) - select_query->setExpression(ASTSelectQuery::Expression::LIMIT_BY, getLimitBy().toAST()); + select_query->setExpression(ASTSelectQuery::Expression::LIMIT_BY, getLimitBy().toAST(options)); if (hasLimit()) - select_query->setExpression(ASTSelectQuery::Expression::LIMIT_LENGTH, getLimit()->toAST()); + select_query->setExpression(ASTSelectQuery::Expression::LIMIT_LENGTH, getLimit()->toAST(options)); if (hasOffset()) - select_query->setExpression(ASTSelectQuery::Expression::LIMIT_OFFSET, getOffset()->toAST()); + select_query->setExpression(ASTSelectQuery::Expression::LIMIT_OFFSET, getOffset()->toAST(options)); if (hasSettingsChanges()) { diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index 54154e1e353..add9e14a0b5 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -575,7 +575,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl() const override; + ASTPtr toASTImpl(ConvertToASTOptions options) const override; private: bool is_subquery = false; diff --git a/src/Analyzer/SortNode.cpp b/src/Analyzer/SortNode.cpp index da1c52ff0ef..6dff649186d 100644 --- a/src/Analyzer/SortNode.cpp +++ b/src/Analyzer/SortNode.cpp @@ -109,7 +109,7 @@ QueryTreeNodePtr SortNode::cloneImpl() const return std::make_shared(nullptr /*expression*/, sort_direction, nulls_sort_direction, collator, with_fill); } -ASTPtr SortNode::toASTImpl() const +ASTPtr SortNode::toASTImpl(ConvertToASTOptions options) const { auto result = std::make_shared(); result->direction = sort_direction == SortDirection::ASCENDING ? 1 : -1; @@ -120,10 +120,10 @@ ASTPtr SortNode::toASTImpl() const result->nulls_direction_was_explicitly_specified = nulls_sort_direction.has_value(); result->with_fill = with_fill; - result->fill_from = hasFillFrom() ? getFillFrom()->toAST() : nullptr; - result->fill_to = hasFillTo() ? getFillTo()->toAST() : nullptr; - result->fill_step = hasFillStep() ? getFillStep()->toAST() : nullptr; - result->children.push_back(getExpression()->toAST()); + result->fill_from = hasFillFrom() ? getFillFrom()->toAST(options) : nullptr; + result->fill_to = hasFillTo() ? getFillTo()->toAST(options) : nullptr; + result->fill_step = hasFillStep() ? getFillStep()->toAST(options) : nullptr; + result->children.push_back(getExpression()->toAST(options)); if (collator) { diff --git a/src/Analyzer/SortNode.h b/src/Analyzer/SortNode.h index 04f9fe798e1..da0996f734e 100644 --- a/src/Analyzer/SortNode.h +++ b/src/Analyzer/SortNode.h @@ -137,7 +137,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl() const override; + ASTPtr toASTImpl(ConvertToASTOptions options) const override; private: static constexpr size_t sort_expression_child_index = 0; diff --git a/src/Analyzer/TableFunctionNode.cpp b/src/Analyzer/TableFunctionNode.cpp index fb3a3af31e1..89e2cdc3ec8 100644 --- a/src/Analyzer/TableFunctionNode.cpp +++ b/src/Analyzer/TableFunctionNode.cpp @@ -113,14 +113,14 @@ QueryTreeNodePtr TableFunctionNode::cloneImpl() const return result; } -ASTPtr TableFunctionNode::toASTImpl() const +ASTPtr TableFunctionNode::toASTImpl(ConvertToASTOptions options) const { auto table_function_ast = std::make_shared(); table_function_ast->name = table_function_name; const auto & arguments = getArguments(); - table_function_ast->children.push_back(arguments.toAST()); + table_function_ast->children.push_back(arguments.toAST(options)); table_function_ast->arguments = table_function_ast->children.back(); return table_function_ast; diff --git a/src/Analyzer/TableFunctionNode.h b/src/Analyzer/TableFunctionNode.h index a88630ffd00..7e5f180bdcb 100644 --- a/src/Analyzer/TableFunctionNode.h +++ b/src/Analyzer/TableFunctionNode.h @@ -142,7 +142,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl() const override; + ASTPtr toASTImpl(ConvertToASTOptions options) const override; private: String table_function_name; diff --git a/src/Analyzer/TableNode.cpp b/src/Analyzer/TableNode.cpp index f315d372bc9..89ea98462db 100644 --- a/src/Analyzer/TableNode.cpp +++ b/src/Analyzer/TableNode.cpp @@ -86,7 +86,7 @@ QueryTreeNodePtr TableNode::cloneImpl() const return result_table_node; } -ASTPtr TableNode::toASTImpl() const +ASTPtr TableNode::toASTImpl(ConvertToASTOptions /* options */) const { if (!temporary_table_name.empty()) return std::make_shared(temporary_table_name); diff --git a/src/Analyzer/TableNode.h b/src/Analyzer/TableNode.h index 1d5ec112ee0..0b0b3a13775 100644 --- a/src/Analyzer/TableNode.h +++ b/src/Analyzer/TableNode.h @@ -106,7 +106,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl() const override; + ASTPtr toASTImpl(ConvertToASTOptions options) const override; private: StoragePtr storage; diff --git a/src/Analyzer/UnionNode.cpp b/src/Analyzer/UnionNode.cpp index 998b869cb04..fdf8880b2e1 100644 --- a/src/Analyzer/UnionNode.cpp +++ b/src/Analyzer/UnionNode.cpp @@ -140,12 +140,12 @@ QueryTreeNodePtr UnionNode::cloneImpl() const return result_union_node; } -ASTPtr UnionNode::toASTImpl() const +ASTPtr UnionNode::toASTImpl(ConvertToASTOptions options) const { auto select_with_union_query = std::make_shared(); select_with_union_query->union_mode = union_mode; select_with_union_query->is_normalized = true; - select_with_union_query->children.push_back(getQueriesNode()->toAST()); + select_with_union_query->children.push_back(getQueriesNode()->toAST(options)); select_with_union_query->list_of_selects = select_with_union_query->children.back(); if (is_subquery) diff --git a/src/Analyzer/UnionNode.h b/src/Analyzer/UnionNode.h index 5e3861da814..bd85cb802e9 100644 --- a/src/Analyzer/UnionNode.h +++ b/src/Analyzer/UnionNode.h @@ -143,7 +143,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl() const override; + ASTPtr toASTImpl(ConvertToASTOptions options) const override; private: bool is_subquery = false; diff --git a/src/Analyzer/WindowNode.cpp b/src/Analyzer/WindowNode.cpp index d516f7a58b8..8aea4f3b87c 100644 --- a/src/Analyzer/WindowNode.cpp +++ b/src/Analyzer/WindowNode.cpp @@ -107,7 +107,7 @@ QueryTreeNodePtr WindowNode::cloneImpl() const return window_node; } -ASTPtr WindowNode::toASTImpl() const +ASTPtr WindowNode::toASTImpl(ConvertToASTOptions options) const { auto window_definition = std::make_shared(); @@ -115,13 +115,13 @@ ASTPtr WindowNode::toASTImpl() const if (hasPartitionBy()) { - window_definition->children.push_back(getPartitionByNode()->toAST()); + window_definition->children.push_back(getPartitionByNode()->toAST(options)); window_definition->partition_by = window_definition->children.back(); } if (hasOrderBy()) { - window_definition->children.push_back(getOrderByNode()->toAST()); + window_definition->children.push_back(getOrderByNode()->toAST(options)); window_definition->order_by = window_definition->children.back(); } @@ -132,7 +132,7 @@ ASTPtr WindowNode::toASTImpl() const if (hasFrameBeginOffset()) { - window_definition->children.push_back(getFrameBeginOffsetNode()->toAST()); + window_definition->children.push_back(getFrameBeginOffsetNode()->toAST(options)); window_definition->frame_begin_offset = window_definition->children.back(); } @@ -140,7 +140,7 @@ ASTPtr WindowNode::toASTImpl() const window_definition->frame_end_preceding = window_frame.end_preceding; if (hasFrameEndOffset()) { - window_definition->children.push_back(getFrameEndOffsetNode()->toAST()); + window_definition->children.push_back(getFrameEndOffsetNode()->toAST(options)); window_definition->frame_end_offset = window_definition->children.back(); } diff --git a/src/Analyzer/WindowNode.h b/src/Analyzer/WindowNode.h index 9dfb3e6ef2a..3b00b8860bf 100644 --- a/src/Analyzer/WindowNode.h +++ b/src/Analyzer/WindowNode.h @@ -175,7 +175,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl() const override; + ASTPtr toASTImpl(ConvertToASTOptions options) const override; private: static constexpr size_t order_by_child_index = 0; diff --git a/src/Analyzer/tests/gtest_query_tree_node.cpp b/src/Analyzer/tests/gtest_query_tree_node.cpp index 079869b2a53..00a698071a3 100644 --- a/src/Analyzer/tests/gtest_query_tree_node.cpp +++ b/src/Analyzer/tests/gtest_query_tree_node.cpp @@ -36,7 +36,7 @@ public: return std::make_shared(); } - ASTPtr toASTImpl() const override + ASTPtr toASTImpl(ConvertToASTOptions /* options */) const override { return nullptr; } diff --git a/src/Common/mysqlxx/Exception.cpp b/src/Common/mysqlxx/Exception.cpp index 0f5320da754..ed1e2278a35 100644 --- a/src/Common/mysqlxx/Exception.cpp +++ b/src/Common/mysqlxx/Exception.cpp @@ -10,9 +10,11 @@ namespace mysqlxx { -std::string errorMessage(MYSQL * driver) +std::string errorMessage(MYSQL * driver, const std::string & query) { - return fmt::format("{} ({}:{})", mysql_error(driver), driver->host ? driver->host : "(nullptr)", driver->port); + return fmt::format("{}{} ({}:{})", mysql_error(driver), + query.empty() ? "" : " while executing query: '" + query + "'", + driver->host ? driver->host : "(nullptr)", driver->port); } void checkError(MYSQL * driver) diff --git a/src/Common/mysqlxx/Query.cpp b/src/Common/mysqlxx/Query.cpp index 97b29fa21df..42c35d26ecf 100644 --- a/src/Common/mysqlxx/Query.cpp +++ b/src/Common/mysqlxx/Query.cpp @@ -64,7 +64,7 @@ void Query::executeImpl() case CR_SERVER_LOST: throw ConnectionLost(errorMessage(mysql_driver), err_no); default: - throw BadQuery(errorMessage(mysql_driver), err_no); + throw BadQuery(errorMessage(mysql_driver, query), err_no); } } } diff --git a/src/Common/mysqlxx/Value.cpp b/src/Common/mysqlxx/Value.cpp index 6954080f864..41d717669b9 100644 --- a/src/Common/mysqlxx/Value.cpp +++ b/src/Common/mysqlxx/Value.cpp @@ -160,14 +160,16 @@ void Value::throwException(const char * text) const if (!isNull()) { - info.append(": "); + info.append(": '"); info.append(m_data, m_length); + info.append("'"); } if (res && res->getQuery()) { - info.append(", query: "); + info.append(", query: '"); info.append(res->getQuery()->str().substr(0, preview_length)); + info.append("'"); } throw CannotParseValue(info); diff --git a/src/Common/mysqlxx/mysqlxx/Exception.h b/src/Common/mysqlxx/mysqlxx/Exception.h index 7886368e747..54153a7f150 100644 --- a/src/Common/mysqlxx/mysqlxx/Exception.h +++ b/src/Common/mysqlxx/mysqlxx/Exception.h @@ -53,7 +53,7 @@ struct CannotParseValue : public Exception }; -std::string errorMessage(MYSQL * driver); +std::string errorMessage(MYSQL * driver, const std::string & query = ""); /// For internal need of library. void checkError(MYSQL * driver); diff --git a/src/Common/tests/gtest_global_register.h b/src/Common/tests/gtest_global_register.h index c4bde825109..3a7fa77c893 100644 --- a/src/Common/tests/gtest_global_register.h +++ b/src/Common/tests/gtest_global_register.h @@ -1,8 +1,13 @@ #pragma once #include +#include #include +inline void tryRegisterAggregateFunctions() +{ + static struct Register { Register() { DB::registerAggregateFunctions(); } } registered; +} inline void tryRegisterFunctions() { diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.h b/src/Interpreters/InterpreterSelectQueryAnalyzer.h index 2c8af49cf0e..f39e55072ef 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.h +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.h @@ -71,6 +71,13 @@ public: /// Set number_of_current_replica and count_participating_replicas in client_info void setProperClientInfo(size_t replica_number, size_t count_participating_replicas); + const QueryTreeNodePtr & getQueryTree() const { return query_tree; } + + SelectQueryInfo getSelectQueryInfo() + { + planner.buildQueryPlanIfNeeded(); return planner.buildSelectQueryInfo(); + } + private: ASTPtr query; ContextMutablePtr context; diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 2242bf92e6b..9e7e0e62ba0 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1159,11 +1159,7 @@ void Planner::buildPlanForQueryNode() query_node.getWhere() = {}; } - SelectQueryInfo select_query_info; - select_query_info.original_query = queryNodeToSelectQuery(query_tree); - select_query_info.query = select_query_info.original_query; - select_query_info.query_tree = query_tree; - select_query_info.planner_context = planner_context; + SelectQueryInfo select_query_info = buildSelectQueryInfo(); StorageLimitsList current_storage_limits = storage_limits; select_query_info.local_storage_limits = buildStorageLimits(*query_context, select_query_options); @@ -1454,6 +1450,16 @@ void Planner::buildPlanForQueryNode() addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute); } +SelectQueryInfo Planner::buildSelectQueryInfo() const +{ + SelectQueryInfo select_query_info; + select_query_info.original_query = queryNodeToSelectQuery(query_tree); + select_query_info.query = select_query_info.original_query; + select_query_info.query_tree = query_tree; + select_query_info.planner_context = planner_context; + return select_query_info; +} + void Planner::addStorageLimits(const StorageLimitsList & limits) { for (const auto & limit : limits) diff --git a/src/Planner/Planner.h b/src/Planner/Planner.h index 443dfa114ee..6fdce80b73a 100644 --- a/src/Planner/Planner.h +++ b/src/Planner/Planner.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB { @@ -50,6 +51,8 @@ public: return std::move(query_plan); } + SelectQueryInfo buildSelectQueryInfo() const; + void addStorageLimits(const StorageLimitsList & limits); private: diff --git a/src/Processors/Merges/Algorithms/tests/gtest_graphite.cpp b/src/Processors/Merges/Algorithms/tests/gtest_graphite.cpp index 1bb6acbcb5c..46980ceb56b 100644 --- a/src/Processors/Merges/Algorithms/tests/gtest_graphite.cpp +++ b/src/Processors/Merges/Algorithms/tests/gtest_graphite.cpp @@ -18,17 +18,6 @@ using namespace DB; -static int regAggregateFunctions = 0; - -void tryRegisterAggregateFunctions() -{ - if (!regAggregateFunctions) - { - registerAggregateFunctions(); - regAggregateFunctions = 1; - } -} - static ConfigProcessor::LoadedConfig loadConfiguration(const std::string & config_path) { ConfigProcessor config_processor(config_path, true, true); diff --git a/src/Processors/Sources/MySQLSource.cpp b/src/Processors/Sources/MySQLSource.cpp index 434d413a238..9c7e83b3869 100644 --- a/src/Processors/Sources/MySQLSource.cpp +++ b/src/Processors/Sources/MySQLSource.cpp @@ -107,6 +107,11 @@ void MySQLWithFailoverSource::onStart() throw; } } + catch (const mysqlxx::BadQuery & e) + { + LOG_ERROR(log, "Error processing query '{}': {}", query_str, e.displayText()); + throw; + } } initPositionMappingFromQueryResultStructure(); diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index db43b2fc3f8..fb895d04b8f 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -190,6 +190,7 @@ struct SelectQueryInfo PlannerContextPtr planner_context; /// Storage table expression + /// It's guaranteed to be present in JOIN TREE of `query_tree` QueryTreeNodePtr table_expression; /// Table expression modifiers for storage diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 7e133538e41..c389f7d9f7f 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -84,6 +84,7 @@ Pipe StorageMySQL::read( storage_snapshot->check(column_names_); String query = transformQueryForExternalDatabase( query_info_, + column_names_, storage_snapshot->metadata->getColumns().getOrdinary(), IdentifierQuotingStyle::BackticksMySQL, remote_database_name, diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index cf87d23bf94..8e1a799fa07 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -91,7 +91,9 @@ Pipe StoragePostgreSQL::read( /// Connection is already made to the needed database, so it should not be present in the query; /// remote_table_schema is empty if it is not specified, will access only table_name. String query = transformQueryForExternalDatabase( - query_info_, storage_snapshot->metadata->getColumns().getOrdinary(), + query_info_, + column_names_, + storage_snapshot->metadata->getColumns().getOrdinary(), IdentifierQuotingStyle::DoubleQuotes, remote_table_schema, remote_table_name, context_); LOG_TRACE(log, "Query: {}", query); diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index 706bc31122c..10eba370d26 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -66,6 +66,7 @@ Pipe StorageSQLite::read( String query = transformQueryForExternalDatabase( query_info, + column_names, storage_snapshot->metadata->getColumns().getOrdinary(), IdentifierQuotingStyle::DoubleQuotes, "", diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index cb5532e91ac..9b3e203e337 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -74,7 +74,9 @@ std::function StorageXDBC::getReadPOSTDataCallback( QueryProcessingStage::Enum & /*processed_stage*/, size_t /*max_block_size*/) const { - String query = transformQueryForExternalDatabase(query_info, + String query = transformQueryForExternalDatabase( + query_info, + column_names, columns_description.getOrdinary(), bridge_helper->getIdentifierQuotingStyle(), remote_database_name, diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index 131bc2b85e3..270af37b7de 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -9,16 +9,20 @@ #include #include #include +#include #include #include #include #include +#include +#include +#include using namespace DB; -/// NOTE How to do better? +/// TODO: use gtest fixture struct State { State(const State&) = delete; @@ -31,9 +35,9 @@ struct State return state; } - const NamesAndTypesList & getColumns() const + const NamesAndTypesList & getColumns(size_t idx = 0) const { - return tables[0].columns; + return tables[idx].columns; } std::vector getTables(size_t num = 0) const @@ -46,10 +50,10 @@ struct State private: - static DatabaseAndTableWithAlias createDBAndTable(String table_name) + static DatabaseAndTableWithAlias createDBAndTable(String table_name, String database_name = "test") { DatabaseAndTableWithAlias res; - res.database = "test"; + res.database = database_name; res.table = table_name; return res; } @@ -75,12 +79,18 @@ private: {"num", std::make_shared()}, {"attr", std::make_shared()}, }), + TableWithColumnNamesAndTypes( + createDBAndTable("external_table"), + { + {"ttt", std::make_shared()}, + }), }; explicit State() : context(Context::createCopy(getContext().context)) { tryRegisterFunctions(); + tryRegisterAggregateFunctions(); DatabasePtr database = std::make_shared("test", context); for (const auto & tab : tables) @@ -91,14 +101,28 @@ private: context, table_name, std::make_shared( - StorageID(db_name, table_name), ColumnsDescription{getColumns()}, ConstraintsDescription{}, String{})); + StorageID(db_name, table_name), ColumnsDescription{tab.columns}, ConstraintsDescription{}, String{})); } DatabaseCatalog::instance().attachDatabase(database->getDatabaseName(), database); + // DatabaseCatalog::instance().attachDatabase("system", mockSystemDatabase()); + context->setCurrentDatabase("test"); } + + DatabasePtr mockSystemDatabase() + { + DatabasePtr database = std::make_shared("system", context); + auto tab = TableWithColumnNamesAndTypes(createDBAndTable("one", "system"), { {"dummy", std::make_shared()} }); + database->attachTable(context, tab.table.table, + std::make_shared( + StorageID(tab.table.database, tab.table.table), + ColumnsDescription{tab.columns}, ConstraintsDescription{}, String{})); + + return database; + } }; -static void check( +static void checkOld( const State & state, size_t table_num, const std::string & query, @@ -109,38 +133,106 @@ static void check( SelectQueryInfo query_info; SelectQueryOptions select_options; query_info.syntax_analyzer_result - = TreeRewriter(state.context).analyzeSelect(ast, DB::TreeRewriterResult(state.getColumns()), select_options, state.getTables(table_num)); + = TreeRewriter(state.context).analyzeSelect(ast, DB::TreeRewriterResult(state.getColumns(0)), select_options, state.getTables(table_num)); query_info.query = ast; std::string transformed_query = transformQueryForExternalDatabase( - query_info, state.getColumns(), IdentifierQuotingStyle::DoubleQuotes, "test", "table", state.context); + query_info, + query_info.syntax_analyzer_result->requiredSourceColumns(), + state.getColumns(0), IdentifierQuotingStyle::DoubleQuotes, "test", "table", state.context); EXPECT_EQ(transformed_query, expected) << query; } +/// Required for transformQueryForExternalDatabase. In real life table expression is calculated via planner. +/// But in tests we can just find it in JOIN TREE. +static QueryTreeNodePtr findTableExpression(const QueryTreeNodePtr & node, const String & table_name) +{ + if (node->getNodeType() == QueryTreeNodeType::TABLE) + { + if (node->as()->getStorageID().table_name == table_name) + return node; + } + + if (node->getNodeType() == QueryTreeNodeType::JOIN) + { + if (auto res = findTableExpression(node->as()->getLeftTableExpression(), table_name)) + return res; + if (auto res = findTableExpression(node->as()->getRightTableExpression(), table_name)) + return res; + } + return nullptr; +} + +/// `column_names` - Normally it's passed to query plan step. But in test we do it manually. +static void checkNewAnalyzer( + const State & state, + const Names & column_names, + const std::string & query, + const std::string & expected) +{ + ParserSelectQuery parser; + ASTPtr ast = parseQuery(parser, query, 1000, 1000); + + SelectQueryOptions select_options; + InterpreterSelectQueryAnalyzer interpreter(ast, state.context, select_options); + SelectQueryInfo query_info = interpreter.getSelectQueryInfo(); + const auto * query_node = query_info.query_tree->as(); + if (!query_node) + throw Exception(ErrorCodes::LOGICAL_ERROR, "QueryNode expected"); + + query_info.table_expression = findTableExpression(query_node->getJoinTree(), "table"); + + std::string transformed_query = transformQueryForExternalDatabase( + query_info, column_names, state.getColumns(0), IdentifierQuotingStyle::DoubleQuotes, "test", "table", state.context); + + EXPECT_EQ(transformed_query, expected) << query; +} + +static void check( + const State & state, + size_t table_num, + const Names & column_names, + const std::string & query, + const std::string & expected, + const std::string & expected_new = "") +{ + { + SCOPED_TRACE("Old analyzer"); + checkOld(state, table_num, query, expected); + } + { + SCOPED_TRACE("New analyzer"); + checkNewAnalyzer(state, column_names, query, expected_new.empty() ? expected : expected_new); + } +} TEST(TransformQueryForExternalDatabase, InWithSingleElement) { const State & state = State::instance(); - check(state, 1, + check(state, 1, {"column"}, "SELECT column FROM test.table WHERE 1 IN (1)", - R"(SELECT "column" FROM "test"."table" WHERE 1 = 1)"); - check(state, 1, + R"(SELECT "column" FROM "test"."table" WHERE 1 = 1)", + R"(SELECT "column" FROM "test"."table")"); + + check(state, 1, {"column"}, "SELECT column FROM test.table WHERE column IN (1, 2)", R"(SELECT "column" FROM "test"."table" WHERE "column" IN (1, 2))"); - check(state, 1, - "SELECT column FROM test.table WHERE column NOT IN ('hello', 'world')", - R"(SELECT "column" FROM "test"."table" WHERE "column" NOT IN ('hello', 'world'))"); + + check(state, 1, {"field"}, + "SELECT field FROM test.table WHERE field NOT IN ('hello', 'world')", + R"(SELECT "field" FROM "test"."table" WHERE "field" NOT IN ('hello', 'world'))"); } TEST(TransformQueryForExternalDatabase, InWithMultipleColumns) { const State & state = State::instance(); - check(state, 1, + check(state, 1, {"column"}, "SELECT column FROM test.table WHERE (1,1) IN ((1,1))", - R"(SELECT "column" FROM "test"."table" WHERE 1 = 1)"); - check(state, 1, + R"(SELECT "column" FROM "test"."table" WHERE 1 = 1)", + R"(SELECT "column" FROM "test"."table")"); + check(state, 1, {"field", "value"}, "SELECT field, value FROM test.table WHERE (field, value) IN (('foo', 'bar'))", R"(SELECT "field", "value" FROM "test"."table" WHERE ("field", "value") IN (('foo', 'bar')))"); } @@ -149,17 +241,17 @@ TEST(TransformQueryForExternalDatabase, InWithTable) { const State & state = State::instance(); - check(state, 1, + check(state, 1, {"column"}, "SELECT column FROM test.table WHERE 1 IN external_table", R"(SELECT "column" FROM "test"."table")"); - check(state, 1, - "SELECT column FROM test.table WHERE 1 IN (x)", + check(state, 1, {"column"}, + "WITH x as (SELECT * FROM external_table) SELECT column FROM test.table WHERE 1 IN (x)", R"(SELECT "column" FROM "test"."table")"); - check(state, 1, - "SELECT column, field, value FROM test.table WHERE column IN (field, value)", - R"(SELECT "column", "field", "value" FROM "test"."table" WHERE "column" IN ("field", "value"))"); - check(state, 1, - "SELECT column FROM test.table WHERE column NOT IN hello AND column = 123", + check(state, 1, {"column", "field", "value"}, + "SELECT column, field, value FROM test.table WHERE column IN (1, 2)", + R"(SELECT "column", "field", "value" FROM "test"."table" WHERE "column" IN (1, 2))"); + check(state, 1, {"column"}, + "SELECT column FROM test.table WHERE column NOT IN external_table AND column = 123", R"(SELECT "column" FROM "test"."table" WHERE "column" = 123)"); } @@ -167,32 +259,32 @@ TEST(TransformQueryForExternalDatabase, Like) { const State & state = State::instance(); - check(state, 1, - "SELECT column FROM test.table WHERE column LIKE '%hello%'", - R"(SELECT "column" FROM "test"."table" WHERE "column" LIKE '%hello%')"); - check(state, 1, - "SELECT column FROM test.table WHERE column NOT LIKE 'w%rld'", - R"(SELECT "column" FROM "test"."table" WHERE "column" NOT LIKE 'w%rld')"); + check(state, 1, {"field"}, + "SELECT field FROM test.table WHERE field LIKE '%hello%'", + R"(SELECT "field" FROM "test"."table" WHERE "field" LIKE '%hello%')"); + check(state, 1, {"field"}, + "SELECT field FROM test.table WHERE field NOT LIKE 'w%rld'", + R"(SELECT "field" FROM "test"."table" WHERE "field" NOT LIKE 'w%rld')"); } TEST(TransformQueryForExternalDatabase, Substring) { const State & state = State::instance(); - check(state, 1, - "SELECT column FROM test.table WHERE left(column, 10) = RIGHT(column, 10) AND SUBSTRING(column FROM 1 FOR 2) = 'Hello'", - R"(SELECT "column" FROM "test"."table")"); + check(state, 1, {"field"}, + "SELECT field FROM test.table WHERE left(field, 10) = RIGHT(field, 10) AND SUBSTRING(field FROM 1 FOR 2) = 'Hello'", + R"(SELECT "field" FROM "test"."table")"); } TEST(TransformQueryForExternalDatabase, MultipleAndSubqueries) { const State & state = State::instance(); - check(state, 1, - "SELECT column FROM test.table WHERE 1 = 1 AND toString(column) = '42' AND column = 42 AND left(column, 10) = RIGHT(column, 10) AND column IN (1, 42) AND SUBSTRING(column FROM 1 FOR 2) = 'Hello' AND column != 4", + check(state, 1, {"column"}, + "SELECT column FROM test.table WHERE 1 = 1 AND toString(column) = '42' AND column = 42 AND left(toString(column), 10) = RIGHT(toString(column), 10) AND column IN (1, 42) AND SUBSTRING(toString(column) FROM 1 FOR 2) = 'Hello' AND column != 4", R"(SELECT "column" FROM "test"."table" WHERE 1 AND ("column" = 42) AND ("column" IN (1, 42)) AND ("column" != 4))"); - check(state, 1, - "SELECT column FROM test.table WHERE toString(column) = '42' AND left(column, 10) = RIGHT(column, 10) AND column = 42", + check(state, 1, {"column"}, + "SELECT column FROM test.table WHERE toString(column) = '42' AND left(toString(column), 10) = RIGHT(toString(column), 10) AND column = 42", R"(SELECT "column" FROM "test"."table" WHERE "column" = 42)"); } @@ -200,7 +292,7 @@ TEST(TransformQueryForExternalDatabase, Issue7245) { const State & state = State::instance(); - check(state, 1, + check(state, 1, {"apply_id", "apply_type", "apply_status", "create_time"}, "SELECT apply_id FROM test.table WHERE apply_type = 2 AND create_time > addDays(toDateTime('2019-01-01 01:02:03'),-7) AND apply_status IN (3,4)", R"(SELECT "apply_id", "apply_type", "apply_status", "create_time" FROM "test"."table" WHERE ("apply_type" = 2) AND ("create_time" > '2018-12-25 01:02:03') AND ("apply_status" IN (3, 4)))"); } @@ -209,8 +301,8 @@ TEST(TransformQueryForExternalDatabase, Aliases) { const State & state = State::instance(); - check(state, 1, - "SELECT field AS value, field AS display WHERE field NOT IN ('') AND display LIKE '%test%'", + check(state, 1, {"field"}, + "SELECT field AS value, field AS display FROM table WHERE field NOT IN ('') AND display LIKE '%test%'", R"(SELECT "field" FROM "test"."table" WHERE ("field" NOT IN ('')) AND ("field" LIKE '%test%'))"); } @@ -218,10 +310,10 @@ TEST(TransformQueryForExternalDatabase, ForeignColumnInWhere) { const State & state = State::instance(); - check(state, 2, + check(state, 2, {"column", "apply_id"}, "SELECT column FROM test.table " "JOIN test.table2 AS table2 ON (test.table.apply_id = table2.num) " - "WHERE column > 2 AND (apply_id = 1 OR table2.num = 1) AND table2.attr != ''", + "WHERE column > 2 AND apply_id = 1 AND table2.num = 1 AND table2.attr != ''", R"(SELECT "column", "apply_id" FROM "test"."table" WHERE ("column" > 2) AND ("apply_id" = 1))"); } @@ -229,7 +321,7 @@ TEST(TransformQueryForExternalDatabase, NoStrict) { const State & state = State::instance(); - check(state, 1, + check(state, 1, {"field"}, "SELECT field FROM table WHERE field IN (SELECT attr FROM table2)", R"(SELECT "field" FROM "test"."table")"); } @@ -239,37 +331,37 @@ TEST(TransformQueryForExternalDatabase, Strict) const State & state = State::instance(); state.context->setSetting("external_table_strict_query", true); - check(state, 1, + check(state, 1, {"field"}, "SELECT field FROM table WHERE field = '1'", R"(SELECT "field" FROM "test"."table" WHERE "field" = '1')"); - check(state, 1, + check(state, 1, {"field"}, "SELECT field FROM table WHERE field IN ('1', '2')", R"(SELECT "field" FROM "test"."table" WHERE "field" IN ('1', '2'))"); - check(state, 1, + check(state, 1, {"field"}, "SELECT field FROM table WHERE field LIKE '%test%'", R"(SELECT "field" FROM "test"."table" WHERE "field" LIKE '%test%')"); /// removeUnknownSubexpressionsFromWhere() takes place - EXPECT_THROW(check(state, 1, "SELECT field FROM table WHERE field IN (SELECT attr FROM table2)", ""), Exception); + EXPECT_THROW(check(state, 1, {"field"}, "SELECT field FROM table WHERE field IN (SELECT attr FROM table2)", ""), Exception); /// !isCompatible() takes place - EXPECT_THROW(check(state, 1, "SELECT column FROM test.table WHERE left(column, 10) = RIGHT(column, 10) AND SUBSTRING(column FROM 1 FOR 2) = 'Hello'", ""), Exception); + EXPECT_THROW(check(state, 1, {"column"}, "SELECT column FROM test.table WHERE left(column, 10) = RIGHT(column, 10) AND SUBSTRING(column FROM 1 FOR 2) = 'Hello'", ""), Exception); } TEST(TransformQueryForExternalDatabase, Null) { const State & state = State::instance(); - check(state, 1, + check(state, 1, {"field"}, "SELECT field FROM table WHERE field IS NULL", R"(SELECT "field" FROM "test"."table" WHERE "field" IS NULL)"); - check(state, 1, + check(state, 1, {"field"}, "SELECT field FROM table WHERE field IS NOT NULL", R"(SELECT "field" FROM "test"."table" WHERE "field" IS NOT NULL)"); - check(state, 1, + check(state, 1, {"field"}, "SELECT field FROM table WHERE isNull(field)", R"(SELECT "field" FROM "test"."table" WHERE "field" IS NULL)"); - check(state, 1, + check(state, 1, {"field"}, "SELECT field FROM table WHERE isNotNull(field)", R"(SELECT "field" FROM "test"."table" WHERE "field" IS NOT NULL)"); } @@ -278,7 +370,28 @@ TEST(TransformQueryForExternalDatabase, ToDate) { const State & state = State::instance(); - check(state, 1, + check(state, 1, {"a", "b", "foo"}, "SELECT foo FROM table WHERE a=10 AND b=toDate('2019-10-05')", R"(SELECT "a", "b", "foo" FROM "test"."table" WHERE ("a" = 10) AND ("b" = '2019-10-05'))"); } + +TEST(TransformQueryForExternalDatabase, Analyzer) +{ + const State & state = State::instance(); + + check(state, 1, {"field"}, + "SELECT count() FROM table WHERE field LIKE '%name_%'", + R"(SELECT "field" FROM "test"."table" WHERE "field" LIKE '%name_%')"); + + check(state, 1, {"column"}, + "SELECT 1 FROM table", + R"(SELECT "column" FROM "test"."table")"); + + check(state, 1, {"column"}, + "SELECT sleepEachRow(1) FROM table", + R"(SELECT "column" FROM "test"."table")"); + + check(state, 1, {"column", "apply_id", "apply_type", "apply_status", "create_time", "field", "value", "a", "b", "foo"}, + "SELECT * FROM table WHERE (column) IN (1)", + R"(SELECT "column", "apply_id", "apply_type", "apply_status", "create_time", "field", "value", "a", "b", "foo" FROM "test"."table" WHERE "column" IN (1))"); +} diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index 1ff310c3fac..da40673d979 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -14,6 +14,8 @@ #include #include +#include + namespace DB { @@ -108,9 +110,9 @@ void dropAliases(ASTPtr & node) } -bool isCompatible(IAST & node) +bool isCompatible(ASTPtr & node) { - if (auto * function = node.as()) + if (auto * function = node->as()) { if (function->parameters) /// Parametric aggregate functions return false; @@ -154,20 +156,30 @@ bool isCompatible(IAST & node) && (function->arguments->children.size() != 2 || function->arguments->children[1]->as())) return false; - for (const auto & expr : function->arguments->children) - if (!isCompatible(*expr)) + for (auto & expr : function->arguments->children) + if (!isCompatible(expr)) return false; return true; } - if (const auto * literal = node.as()) + if (const auto * literal = node->as()) { + if (literal->value.getType() == Field::Types::Tuple) + { + /// Represent a tuple with zero or one elements as (x) instead of tuple(x). + auto tuple_value = literal->value.safeGet(); + if (tuple_value.size() == 1) + { + node = makeASTFunction("", std::make_shared(tuple_value[0])); + return true; + } + } /// Foreign databases often have no support for Array. But Tuple literals are passed to support IN clause. return literal->value.getType() != Field::Types::Array; } - return node.as(); + return node->as(); } bool removeUnknownSubexpressions(ASTPtr & node, const NameSet & known_names); @@ -241,23 +253,15 @@ bool removeUnknownSubexpressionsFromWhere(ASTPtr & node, const NamesAndTypesList return removeUnknownSubexpressions(node, known_names); } -} - -String transformQueryForExternalDatabase( - const SelectQueryInfo & query_info, +String transformQueryForExternalDatabaseImpl( + ASTPtr clone_query, + Names used_columns, const NamesAndTypesList & available_columns, IdentifierQuotingStyle identifier_quoting_style, const String & database, const String & table, ContextPtr context) { - auto clone_query = query_info.query->clone(); - - /// TODO: Analyzer syntax analyzer result - if (!query_info.syntax_analyzer_result) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "transform query for external database is unsupported"); - - const Names used_columns = query_info.syntax_analyzer_result->requiredSourceColumns(); bool strict = context->getSettingsRef().external_table_strict_query; auto select = std::make_shared(); @@ -278,11 +282,12 @@ String transformQueryForExternalDatabase( ASTPtr original_where = clone_query->as().where(); bool where_has_known_columns = removeUnknownSubexpressionsFromWhere(original_where, available_columns); + if (original_where && where_has_known_columns) { replaceConstantExpressions(original_where, context, available_columns); - if (isCompatible(*original_where)) + if (isCompatible(original_where)) { select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(original_where)); } @@ -290,14 +295,14 @@ String transformQueryForExternalDatabase( { throw Exception(ErrorCodes::INCORRECT_QUERY, "Query contains non-compatible expressions (and external_table_strict_query=true)"); } - else if (const auto * function = original_where->as()) + else if (auto * function = original_where->as()) { if (function->name == "and") { auto new_function_and = makeASTFunction("and"); - for (const auto & elem : function->arguments->children) + for (auto & elem : function->arguments->children) { - if (isCompatible(*elem)) + if (isCompatible(elem)) new_function_and->arguments->children.push_back(elem); } if (new_function_and->arguments->children.size() == 1) @@ -309,7 +314,8 @@ String transformQueryForExternalDatabase( } else if (strict && original_where) { - throw Exception(ErrorCodes::INCORRECT_QUERY, "Query contains non-compatible expressions (and external_table_strict_query=true)"); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Query contains non-compatible expressions '{}' (and external_table_strict_query=true)", + original_where->formatForErrorMessage()); } auto * literal_expr = typeid_cast(original_where.get()); @@ -338,3 +344,50 @@ String transformQueryForExternalDatabase( } } + +String transformQueryForExternalDatabase( + const SelectQueryInfo & query_info, + const Names & column_names, + const NamesAndTypesList & available_columns, + IdentifierQuotingStyle identifier_quoting_style, + const String & database, + const String & table, + ContextPtr context) +{ + if (!query_info.syntax_analyzer_result) + { + if (!query_info.query_tree) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Query is not analyzed: no query tree"); + if (!query_info.planner_context) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Query is not analyzed: no planner context"); + if (!query_info.table_expression) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Query is not analyzed: no table expression"); + + if (column_names.empty()) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "No column names for query '{}' to external table '{}.{}'", + database, table, query_info.query_tree->formatASTForErrorMessage()); + + auto clone_query = getASTForExternalDatabaseFromQueryTree(query_info.query_tree); + + return transformQueryForExternalDatabaseImpl( + clone_query, + column_names, + available_columns, + identifier_quoting_style, + database, + table, + context); + } + + auto clone_query = query_info.query->clone(); + return transformQueryForExternalDatabaseImpl( + clone_query, + query_info.syntax_analyzer_result->requiredSourceColumns(), + available_columns, + identifier_quoting_style, + database, + table, + context); +} + +} diff --git a/src/Storages/transformQueryForExternalDatabase.h b/src/Storages/transformQueryForExternalDatabase.h index 3756dd97feb..0f2b0a5822f 100644 --- a/src/Storages/transformQueryForExternalDatabase.h +++ b/src/Storages/transformQueryForExternalDatabase.h @@ -28,6 +28,7 @@ class IAST; */ String transformQueryForExternalDatabase( const SelectQueryInfo & query_info, + const Names & column_names, const NamesAndTypesList & available_columns, IdentifierQuotingStyle identifier_quoting_style, const String & database, diff --git a/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp b/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp new file mode 100644 index 00000000000..782515d795e --- /dev/null +++ b/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp @@ -0,0 +1,81 @@ +#include + +#include +#include +#include +#include + +#include + +#include +#include +#include +#include +#include +#include + + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; +} + +namespace +{ + +class PrepareForExternalDatabaseVisitor : public InDepthQueryTreeVisitor +{ +public: + using Base = InDepthQueryTreeVisitor; + using Base::Base; + + void visitImpl(QueryTreeNodePtr & node) + { + auto * constant_node = node->as(); + if (constant_node) + { + auto result_type = constant_node->getResultType(); + if (isDate(result_type) || isDateTime(result_type) || isDateTime64(result_type)) + { + /// Use string representation of constant date and time values + /// The code is ugly - how to convert artbitrary Field to proper string representation? + /// (maybe we can just consider numbers as unix timestamps?) + auto result_column = result_type->createColumnConst(1, constant_node->getValue()); + const IColumn & inner_column = assert_cast(*result_column).getDataColumn(); + + WriteBufferFromOwnString out; + result_type->getDefaultSerialization()->serializeText(inner_column, 0, out, FormatSettings()); + node = std::make_shared(std::make_shared(out.str(), result_type)); + } + } + } +}; + +} + +ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tree) +{ + auto new_tree = query_tree->clone(); + + PrepareForExternalDatabaseVisitor visitor; + visitor.visit(new_tree); + const auto * query_node = new_tree->as(); + + const auto & query_node_ast = query_node->toAST({ .add_cast_for_constants = false, .fully_qualified_identifiers = false }); + + const auto * union_ast = query_node_ast->as(); + if (!union_ast) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "QueryNode AST is not a ASTSelectWithUnionQuery"); + + if (union_ast->list_of_selects->children.size() != 1) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "QueryNode AST is not a single ASTSelectQuery, got {}", union_ast->list_of_selects->children.size()); + + return union_ast->list_of_selects->children.at(0); +} + +} diff --git a/src/Storages/transformQueryForExternalDatabaseAnalyzer.h b/src/Storages/transformQueryForExternalDatabaseAnalyzer.h new file mode 100644 index 00000000000..f8983619d1f --- /dev/null +++ b/src/Storages/transformQueryForExternalDatabaseAnalyzer.h @@ -0,0 +1,11 @@ +#pragma once + +#include + + +namespace DB +{ + +ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tree); + +} diff --git a/tests/queries/0_stateless/02479_mysql_connect_to_self.reference b/tests/queries/0_stateless/02479_mysql_connect_to_self.reference index 573541ac970..07cd8dea949 100644 --- a/tests/queries/0_stateless/02479_mysql_connect_to_self.reference +++ b/tests/queries/0_stateless/02479_mysql_connect_to_self.reference @@ -1 +1,39 @@ -0 +--- +1 one -1 een +2 two -2 twee +3 three -3 drie +4 four -4 vier +5 five -5 vijf +--- +5 +--- +1 +1 +1 +1 +1 +--- +1 +2 +3 +4 +5 +--- +-5 five +-4 four +-1 one +-3 three +-2 two +--- +-3 three +-1 one +-2 two +-4 four +-5 five +--- +-1 +-3 +-4 +-5 +--- +4 diff --git a/tests/queries/0_stateless/02479_mysql_connect_to_self.sql b/tests/queries/0_stateless/02479_mysql_connect_to_self.sql index 7ff5b3e3382..832d4a6347d 100644 --- a/tests/queries/0_stateless/02479_mysql_connect_to_self.sql +++ b/tests/queries/0_stateless/02479_mysql_connect_to_self.sql @@ -1,4 +1,32 @@ -- Tags: no-fasttest -SELECT * -FROM mysql('127.0.0.1:9004', system, one, 'default', '') -SETTINGS send_logs_level = 'fatal'; -- failed connection tries are ok, if it succeeded after retry. + +SET send_logs_level = 'fatal'; -- failed connection tries are ok, if it succeeded after retry. + +CREATE TABLE foo (key UInt32, a String, b Int64, c String) ENGINE = TinyLog; +INSERT INTO foo VALUES (1, 'one', -1, 'een'), (2, 'two', -2, 'twee'), (3, 'three', -3, 'drie'), (4, 'four', -4, 'vier'), (5, 'five', -5, 'vijf'); + +SET allow_experimental_analyzer = 1; + +SELECT '---'; +SELECT * FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '') ORDER BY key; + +SELECT '---'; +SELECT count() FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', ''); + +SELECT '---'; +SELECT 1 FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', ''); + +SELECT '---'; +SELECT key FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '') ORDER BY key; + +SELECT '---'; +SELECT b, a FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '') ORDER BY a; + +SELECT '---'; +SELECT b, a FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '') ORDER BY c; + +SELECT '---'; +SELECT b FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '') WHERE c != 'twee' ORDER BY b; + +SELECT '---'; +SELECT count() FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '') WHERE c != 'twee'; From ebd71f226cbc52340e712b07bbaebc6656aa7e4b Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 13 Mar 2023 11:41:00 +0000 Subject: [PATCH 259/418] Support SETTINGS argument of table functions --- src/Analyzer/QueryTreeBuilder.cpp | 6 +++++ src/Analyzer/TableFunctionNode.cpp | 27 +++++++++++++++++++ src/Analyzer/TableFunctionNode.h | 15 +++++++++++ .../02479_mysql_connect_to_self.reference | 21 +++++++++++++++ .../02479_mysql_connect_to_self.sql | 10 +++++++ 5 files changed, 79 insertions(+) diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 7dd988619ac..4887cfcd6ea 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -838,8 +838,14 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select const auto & function_arguments_list = table_function_expression.arguments->as().children; for (const auto & argument : function_arguments_list) { + if (!node->getSettingsChanges().empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Table function '{}' has arguments after SETTINGS", + table_function_expression.formatForErrorMessage()); + if (argument->as() || argument->as() || argument->as()) node->getArguments().getNodes().push_back(buildSelectOrUnionExpression(argument, false /*is_subquery*/, {} /*cte_name*/, context)); + else if (const auto * ast_set = argument->as()) + node->setSettingsChanges(ast_set->changes); else node->getArguments().getNodes().push_back(buildExpression(argument, context)); } diff --git a/src/Analyzer/TableFunctionNode.cpp b/src/Analyzer/TableFunctionNode.cpp index 89e2cdc3ec8..41eb1d151c7 100644 --- a/src/Analyzer/TableFunctionNode.cpp +++ b/src/Analyzer/TableFunctionNode.cpp @@ -7,6 +7,7 @@ #include #include +#include #include @@ -71,6 +72,13 @@ void TableFunctionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_ buffer << '\n' << std::string(indent + 2, ' ') << "ARGUMENTS\n"; arguments.dumpTreeImpl(buffer, format_state, indent + 4); } + + if (!settings_changes.empty()) + { + buffer << '\n' << std::string(indent + 6, ' ') << "SETTINGS"; + for (const auto & change : settings_changes) + buffer << fmt::format(" {}={}", change.name, toString(change.value)); + } } bool TableFunctionNode::isEqualImpl(const IQueryTreeNode & rhs) const @@ -99,6 +107,16 @@ void TableFunctionNode::updateTreeHashImpl(HashState & state) const if (table_expression_modifiers) table_expression_modifiers->updateTreeHash(state); + + for (const auto & change : settings_changes) + { + state.update(change.name.size()); + state.update(change.name); + + const auto & value_dump = change.value.dump(); + state.update(value_dump.size()); + state.update(value_dump); + } } QueryTreeNodePtr TableFunctionNode::cloneImpl() const @@ -109,6 +127,7 @@ QueryTreeNodePtr TableFunctionNode::cloneImpl() const result->storage_id = storage_id; result->storage_snapshot = storage_snapshot; result->table_expression_modifiers = table_expression_modifiers; + result->settings_changes = settings_changes; return result; } @@ -123,6 +142,14 @@ ASTPtr TableFunctionNode::toASTImpl(ConvertToASTOptions options) const table_function_ast->children.push_back(arguments.toAST(options)); table_function_ast->arguments = table_function_ast->children.back(); + if (!settings_changes.empty()) + { + auto settings_ast = std::make_shared(); + settings_ast->changes = settings_changes; + settings_ast->is_standalone = false; + table_function_ast->arguments->children.push_back(std::move(settings_ast)); + } + return table_function_ast; } diff --git a/src/Analyzer/TableFunctionNode.h b/src/Analyzer/TableFunctionNode.h index 7e5f180bdcb..cf2830f2ecb 100644 --- a/src/Analyzer/TableFunctionNode.h +++ b/src/Analyzer/TableFunctionNode.h @@ -1,5 +1,7 @@ #pragma once +#include + #include #include #include @@ -122,6 +124,18 @@ public: return table_expression_modifiers; } + /// Get settings changes passed to table function + const SettingsChanges & getSettingsChanges() const + { + return settings_changes; + } + + /// Set settings changes passed as last argument to table function + void setSettingsChanges(SettingsChanges settings_changes_) + { + settings_changes = std::move(settings_changes_); + } + /// Set table expression modifiers void setTableExpressionModifiers(TableExpressionModifiers table_expression_modifiers_value) { @@ -151,6 +165,7 @@ private: StorageID storage_id; StorageSnapshotPtr storage_snapshot; std::optional table_expression_modifiers; + SettingsChanges settings_changes; static constexpr size_t arguments_child_index = 0; static constexpr size_t children_size = arguments_child_index + 1; diff --git a/tests/queries/0_stateless/02479_mysql_connect_to_self.reference b/tests/queries/0_stateless/02479_mysql_connect_to_self.reference index 07cd8dea949..2b9215a3517 100644 --- a/tests/queries/0_stateless/02479_mysql_connect_to_self.reference +++ b/tests/queries/0_stateless/02479_mysql_connect_to_self.reference @@ -37,3 +37,24 @@ -5 --- 4 +QUERY id: 0 + PROJECTION COLUMNS + key String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: key, result_type: String, source_id: 3 + JOIN TREE + TABLE_FUNCTION id: 3, table_function_name: mysql + ARGUMENTS + LIST id: 4, nodes: 5 + CONSTANT id: 5, constant_value: \'127.0.0.1:9004\', constant_value_type: String + CONSTANT id: 6, constant_value: \'default\', constant_value_type: String + CONSTANT id: 7, constant_value: \'foo\', constant_value_type: String + CONSTANT id: 8, constant_value: \'default\', constant_value_type: String + CONSTANT id: 9, constant_value: \'\', constant_value_type: String + SETTINGS connection_wait_timeout=123 connect_timeout=40123002 read_write_timeout=40123001 connection_pool_size=3 + +SELECT key AS key +FROM mysql(\'127.0.0.1:9004\', \'default\', \'foo\', \'default\', \'\', SETTINGS connection_wait_timeout = 123, connect_timeout = 40123002, read_write_timeout = 40123001, connection_pool_size = 3) +--- +5 diff --git a/tests/queries/0_stateless/02479_mysql_connect_to_self.sql b/tests/queries/0_stateless/02479_mysql_connect_to_self.sql index 832d4a6347d..3da2a1dd2b9 100644 --- a/tests/queries/0_stateless/02479_mysql_connect_to_self.sql +++ b/tests/queries/0_stateless/02479_mysql_connect_to_self.sql @@ -30,3 +30,13 @@ SELECT b FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '') WHE SELECT '---'; SELECT count() FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '') WHERE c != 'twee'; + +EXPLAIN QUERY TREE dump_ast = 1 +SELECT * FROM mysql( + '127.0.0.1:9004', 'default', 'foo', 'default', '', + SETTINGS connection_wait_timeout = 123, connect_timeout = 40123002, read_write_timeout = 40123001, connection_pool_size = 3 +); + +SELECT '---'; +SELECT count() FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '', SETTINGS connection_pool_size = 1); +SELECT count() FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '', SETTINGS connection_pool_size = 0); -- { serverError BAD_ARGUMENTS } From c2f0583dc66e7c358258a18ec607919a53b37e22 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 13 Mar 2023 13:59:00 +0000 Subject: [PATCH 260/418] Adjust test_mysql_protocol and others for analyzer --- tests/integration/test_mysql_protocol/test.py | 25 +++++++++++-------- tests/integration/test_part_log_table/test.py | 19 +++++++------- .../test.py | 2 +- 3 files changed, 26 insertions(+), 20 deletions(-) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 2cfb1d41ce0..96e7a087d5a 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -1,6 +1,7 @@ # coding: utf-8 import datetime +import fnmatch import math import os import time @@ -149,7 +150,7 @@ def java_container(): def test_mysql_client(started_cluster): - # type: (Container, str) -> None + # type: (ClickHouseCluster) -> None code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run( """ mysql --protocol tcp -h {host} -P {port} default -u user_with_double_sha1 --password=abacaba @@ -365,7 +366,10 @@ def test_mysql_replacement_query(started_cluster): demux=True, ) assert code == 0 - assert stdout.decode() == "currentDatabase()\ndefault\n" + assert stdout.decode().lower() in [ + "currentdatabase()\ndefault\n", + "database()\ndefault\n", + ] code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run( """ @@ -377,7 +381,10 @@ def test_mysql_replacement_query(started_cluster): demux=True, ) assert code == 0 - assert stdout.decode() == "currentDatabase()\ndefault\n" + assert stdout.decode().lower() in [ + "currentdatabase()\ndefault\n", + "database()\ndefault\n", + ] def test_mysql_select_user(started_cluster): @@ -391,7 +398,7 @@ def test_mysql_select_user(started_cluster): demux=True, ) assert code == 0 - assert stdout.decode() == "currentUser()\ndefault\n" + assert stdout.decode() in ["currentUser()\ndefault\n", "user()\ndefault\n"] def test_mysql_explain(started_cluster): @@ -568,9 +575,8 @@ def test_python_client(started_cluster): with pytest.raises(pymysql.InternalError) as exc_info: client.query("select name from tables") - assert exc_info.value.args[1].startswith( - "Code: 60. DB::Exception: Table default.tables doesn't exist" - ), exc_info.value.args[1] + resp = exc_info.value.args[1] + assert fnmatch.fnmatch(resp, "*DB::Exception:*system.part_log*UNKNOWN_TABLE*"), resp cursor = client.cursor(pymysql.cursors.DictCursor) cursor.execute("select 1 as a, 'тест' as b") @@ -602,9 +608,8 @@ def test_python_client(started_cluster): with pytest.raises(pymysql.InternalError) as exc_info: client.query("select name from tables") - assert exc_info.value.args[1].startswith( - "Code: 60. DB::Exception: Table default.tables doesn't exist" - ), exc_info.value.args[1] + resp = exc_info.value.args[1] + assert fnmatch.fnmatch(resp, "*DB::Exception:*system.part_log*UNKNOWN_TABLE*"), resp cursor = client.cursor(pymysql.cursors.DictCursor) cursor.execute("select 1 as a, 'тест' as b") diff --git a/tests/integration/test_part_log_table/test.py b/tests/integration/test_part_log_table/test.py index d81990a9d47..ea4499d5877 100644 --- a/tests/integration/test_part_log_table/test.py +++ b/tests/integration/test_part_log_table/test.py @@ -1,4 +1,5 @@ import pytest +import fnmatch from helpers.cluster import ClickHouseCluster @@ -27,20 +28,20 @@ def start_cluster(): def test_config_without_part_log(start_cluster): - assert "Table system.part_log doesn't exist" in node1.query_and_get_error( - "SELECT * FROM system.part_log" - ) + resp = node1.query_and_get_error("SELECT * FROM system.part_log") + assert fnmatch.fnmatch(resp, "*DB::Exception:*system.part_log*UNKNOWN_TABLE*"), resp + node1.query( "CREATE TABLE test_table(word String, value UInt64) ENGINE=MergeTree() ORDER BY value" ) - assert "Table system.part_log doesn't exist" in node1.query_and_get_error( - "SELECT * FROM system.part_log" - ) + resp = node1.query_and_get_error("SELECT * FROM system.part_log") + assert fnmatch.fnmatch(resp, "*DB::Exception:*system.part_log*UNKNOWN_TABLE*"), resp + node1.query("INSERT INTO test_table VALUES ('name', 1)") node1.query("SYSTEM FLUSH LOGS") - assert "Table system.part_log doesn't exist" in node1.query_and_get_error( - "SELECT * FROM system.part_log" - ) + + resp = node1.query_and_get_error("SELECT * FROM system.part_log") + assert fnmatch.fnmatch(resp, "*DB::Exception:*system.part_log*UNKNOWN_TABLE*"), resp # Note: if part_log is defined, we cannot say when the table will be created - because of metric_log, trace_log, text_log, query_log... diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 33796336550..1ba278b357e 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -203,7 +203,7 @@ def test_remove_table_from_replication(started_cluster): table_name = "postgresql_replica_4" instance.query(f"DETACH TABLE test_database.{table_name} PERMANENTLY") result = instance.query_and_get_error(f"SELECT * FROM test_database.{table_name}") - assert "doesn't exist" in result + assert "UNKNOWN_TABLE" in result result = instance.query("SHOW TABLES FROM test_database") assert ( From 2537d0ddcd6005c6dd1fa7f28c782ec97f418563 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 13 Mar 2023 14:21:47 +0000 Subject: [PATCH 261/418] Fix 02479_mysql_connect_to_self --- tests/queries/0_stateless/02479_mysql_connect_to_self.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02479_mysql_connect_to_self.sql b/tests/queries/0_stateless/02479_mysql_connect_to_self.sql index 3da2a1dd2b9..a7b10d314b8 100644 --- a/tests/queries/0_stateless/02479_mysql_connect_to_self.sql +++ b/tests/queries/0_stateless/02479_mysql_connect_to_self.sql @@ -33,7 +33,7 @@ SELECT count() FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', ' EXPLAIN QUERY TREE dump_ast = 1 SELECT * FROM mysql( - '127.0.0.1:9004', 'default', 'foo', 'default', '', + '127.0.0.1:9004', currentDatabase(), foo, 'default', '' SETTINGS connection_wait_timeout = 123, connect_timeout = 40123002, read_write_timeout = 40123001, connection_pool_size = 3 ); From a5cca40cf0004f871b09555eb775de9768571ed4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 14 Mar 2023 09:14:58 +0000 Subject: [PATCH 262/418] Pass ConvertToASTOptions by constref --- src/Analyzer/ArrayJoinNode.cpp | 2 +- src/Analyzer/ArrayJoinNode.h | 2 +- src/Analyzer/ColumnNode.cpp | 2 +- src/Analyzer/ColumnNode.h | 2 +- src/Analyzer/ColumnTransformers.cpp | 6 +++--- src/Analyzer/ColumnTransformers.h | 6 +++--- src/Analyzer/ConstantNode.cpp | 2 +- src/Analyzer/ConstantNode.h | 2 +- src/Analyzer/FunctionNode.cpp | 2 +- src/Analyzer/FunctionNode.h | 2 +- src/Analyzer/IQueryTreeNode.cpp | 2 +- src/Analyzer/IQueryTreeNode.h | 4 ++-- src/Analyzer/IdentifierNode.cpp | 2 +- src/Analyzer/IdentifierNode.h | 2 +- src/Analyzer/InterpolateNode.cpp | 2 +- src/Analyzer/InterpolateNode.h | 2 +- src/Analyzer/JoinNode.cpp | 2 +- src/Analyzer/JoinNode.h | 2 +- src/Analyzer/LambdaNode.cpp | 2 +- src/Analyzer/LambdaNode.h | 2 +- src/Analyzer/ListNode.cpp | 2 +- src/Analyzer/ListNode.h | 2 +- src/Analyzer/MatcherNode.cpp | 2 +- src/Analyzer/MatcherNode.h | 2 +- src/Analyzer/QueryNode.cpp | 2 +- src/Analyzer/QueryNode.h | 2 +- src/Analyzer/SortNode.cpp | 2 +- src/Analyzer/SortNode.h | 2 +- src/Analyzer/TableFunctionNode.cpp | 2 +- src/Analyzer/TableFunctionNode.h | 2 +- src/Analyzer/TableNode.cpp | 2 +- src/Analyzer/TableNode.h | 2 +- src/Analyzer/UnionNode.cpp | 2 +- src/Analyzer/UnionNode.h | 2 +- src/Analyzer/WindowNode.cpp | 2 +- src/Analyzer/WindowNode.h | 2 +- src/Analyzer/tests/gtest_query_tree_node.cpp | 2 +- 37 files changed, 42 insertions(+), 42 deletions(-) diff --git a/src/Analyzer/ArrayJoinNode.cpp b/src/Analyzer/ArrayJoinNode.cpp index 1a7969ce64a..4a610767341 100644 --- a/src/Analyzer/ArrayJoinNode.cpp +++ b/src/Analyzer/ArrayJoinNode.cpp @@ -49,7 +49,7 @@ QueryTreeNodePtr ArrayJoinNode::cloneImpl() const return std::make_shared(getTableExpression(), getJoinExpressionsNode(), is_left); } -ASTPtr ArrayJoinNode::toASTImpl(ConvertToASTOptions options) const +ASTPtr ArrayJoinNode::toASTImpl(const ConvertToASTOptions & options) const { auto array_join_ast = std::make_shared(); array_join_ast->kind = is_left ? ASTArrayJoin::Kind::Left : ASTArrayJoin::Kind::Inner; diff --git a/src/Analyzer/ArrayJoinNode.h b/src/Analyzer/ArrayJoinNode.h index f19f1b67971..89cb0b7b8c1 100644 --- a/src/Analyzer/ArrayJoinNode.h +++ b/src/Analyzer/ArrayJoinNode.h @@ -99,7 +99,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl(ConvertToASTOptions options) const override; + ASTPtr toASTImpl(const ConvertToASTOptions & options) const override; private: bool is_left = false; diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp index 568daa321ea..a9d47f8287d 100644 --- a/src/Analyzer/ColumnNode.cpp +++ b/src/Analyzer/ColumnNode.cpp @@ -91,7 +91,7 @@ QueryTreeNodePtr ColumnNode::cloneImpl() const return std::make_shared(column, getSourceWeakPointer()); } -ASTPtr ColumnNode::toASTImpl(ConvertToASTOptions options) const +ASTPtr ColumnNode::toASTImpl(const ConvertToASTOptions & options) const { std::vector column_identifier_parts; diff --git a/src/Analyzer/ColumnNode.h b/src/Analyzer/ColumnNode.h index 1597cc465af..fc626c62cc4 100644 --- a/src/Analyzer/ColumnNode.h +++ b/src/Analyzer/ColumnNode.h @@ -137,7 +137,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl(ConvertToASTOptions options) const override; + ASTPtr toASTImpl(const ConvertToASTOptions & options) const override; private: const QueryTreeNodeWeakPtr & getSourceWeakPointer() const diff --git a/src/Analyzer/ColumnTransformers.cpp b/src/Analyzer/ColumnTransformers.cpp index 772916bbed6..40e1e019d50 100644 --- a/src/Analyzer/ColumnTransformers.cpp +++ b/src/Analyzer/ColumnTransformers.cpp @@ -91,7 +91,7 @@ QueryTreeNodePtr ApplyColumnTransformerNode::cloneImpl() const return std::make_shared(getExpressionNode()); } -ASTPtr ApplyColumnTransformerNode::toASTImpl(ConvertToASTOptions options) const +ASTPtr ApplyColumnTransformerNode::toASTImpl(const ConvertToASTOptions & options) const { auto ast_apply_transformer = std::make_shared(); const auto & expression_node = getExpressionNode(); @@ -227,7 +227,7 @@ QueryTreeNodePtr ExceptColumnTransformerNode::cloneImpl() const return std::make_shared(except_column_names, is_strict); } -ASTPtr ExceptColumnTransformerNode::toASTImpl(ConvertToASTOptions /* options */) const +ASTPtr ExceptColumnTransformerNode::toASTImpl(const ConvertToASTOptions & /* options */) const { auto ast_except_transformer = std::make_shared(); @@ -334,7 +334,7 @@ QueryTreeNodePtr ReplaceColumnTransformerNode::cloneImpl() const return result_replace_transformer; } -ASTPtr ReplaceColumnTransformerNode::toASTImpl(ConvertToASTOptions options) const +ASTPtr ReplaceColumnTransformerNode::toASTImpl(const ConvertToASTOptions & options) const { auto ast_replace_transformer = std::make_shared(); diff --git a/src/Analyzer/ColumnTransformers.h b/src/Analyzer/ColumnTransformers.h index 2b06e66d07e..3ec6f506c3c 100644 --- a/src/Analyzer/ColumnTransformers.h +++ b/src/Analyzer/ColumnTransformers.h @@ -141,7 +141,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl(ConvertToASTOptions options) const override; + ASTPtr toASTImpl(const ConvertToASTOptions & options) const override; private: ApplyColumnTransformerType apply_transformer_type = ApplyColumnTransformerType::LAMBDA; @@ -220,7 +220,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl(ConvertToASTOptions options) const override; + ASTPtr toASTImpl(const ConvertToASTOptions & options) const override; private: ExceptColumnTransformerType except_transformer_type; @@ -298,7 +298,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl(ConvertToASTOptions options) const override; + ASTPtr toASTImpl(const ConvertToASTOptions & options) const override; private: ListNode & getReplacements() diff --git a/src/Analyzer/ConstantNode.cpp b/src/Analyzer/ConstantNode.cpp index 8d286cc99a1..768cf06bfd0 100644 --- a/src/Analyzer/ConstantNode.cpp +++ b/src/Analyzer/ConstantNode.cpp @@ -75,7 +75,7 @@ QueryTreeNodePtr ConstantNode::cloneImpl() const return std::make_shared(constant_value, source_expression); } -ASTPtr ConstantNode::toASTImpl(ConvertToASTOptions options) const +ASTPtr ConstantNode::toASTImpl(const ConvertToASTOptions & options) const { const auto & constant_value_literal = constant_value->getValue(); auto constant_value_ast = std::make_shared(constant_value_literal); diff --git a/src/Analyzer/ConstantNode.h b/src/Analyzer/ConstantNode.h index e7d38c5bbfa..d290196b48d 100644 --- a/src/Analyzer/ConstantNode.h +++ b/src/Analyzer/ConstantNode.h @@ -83,7 +83,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl(ConvertToASTOptions options) const override; + ASTPtr toASTImpl(const ConvertToASTOptions & options) const override; private: ConstantValuePtr constant_value; diff --git a/src/Analyzer/FunctionNode.cpp b/src/Analyzer/FunctionNode.cpp index d0344c5efdf..e635750569d 100644 --- a/src/Analyzer/FunctionNode.cpp +++ b/src/Analyzer/FunctionNode.cpp @@ -197,7 +197,7 @@ QueryTreeNodePtr FunctionNode::cloneImpl() const return result_function; } -ASTPtr FunctionNode::toASTImpl(ConvertToASTOptions options) const +ASTPtr FunctionNode::toASTImpl(const ConvertToASTOptions & options) const { auto function_ast = std::make_shared(); diff --git a/src/Analyzer/FunctionNode.h b/src/Analyzer/FunctionNode.h index 2e899fe2801..742374e1f0a 100644 --- a/src/Analyzer/FunctionNode.h +++ b/src/Analyzer/FunctionNode.h @@ -209,7 +209,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl(ConvertToASTOptions options) const override; + ASTPtr toASTImpl(const ConvertToASTOptions & options) const override; private: String function_name; diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index 7603a8a1593..f1056975f7f 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -331,7 +331,7 @@ QueryTreeNodePtr IQueryTreeNode::cloneAndReplace(const QueryTreeNodePtr & node_t return cloneAndReplace(replacement_map); } -ASTPtr IQueryTreeNode::toAST(ConvertToASTOptions options) const +ASTPtr IQueryTreeNode::toAST(const ConvertToASTOptions & options) const { auto converted_node = toASTImpl(options); diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index 18562214200..9266b33c867 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -191,7 +191,7 @@ public: }; /// Convert query tree to AST - ASTPtr toAST(ConvertToASTOptions options = { .add_cast_for_constants = true, .fully_qualified_identifiers = true }) const; + ASTPtr toAST(const ConvertToASTOptions & options = { .add_cast_for_constants = true, .fully_qualified_identifiers = true }) const; /// Convert query tree to AST and then format it for error message. String formatConvertedASTForErrorMessage() const; @@ -267,7 +267,7 @@ protected: virtual QueryTreeNodePtr cloneImpl() const = 0; /// Subclass must convert its internal state and its children to AST - virtual ASTPtr toASTImpl(ConvertToASTOptions options) const = 0; + virtual ASTPtr toASTImpl(const ConvertToASTOptions & options) const = 0; QueryTreeNodes children; QueryTreeWeakNodes weak_pointers; diff --git a/src/Analyzer/IdentifierNode.cpp b/src/Analyzer/IdentifierNode.cpp index 3c135a3a2bc..66a44d2ae5f 100644 --- a/src/Analyzer/IdentifierNode.cpp +++ b/src/Analyzer/IdentifierNode.cpp @@ -58,7 +58,7 @@ QueryTreeNodePtr IdentifierNode::cloneImpl() const return std::make_shared(identifier); } -ASTPtr IdentifierNode::toASTImpl(ConvertToASTOptions /* options */) const +ASTPtr IdentifierNode::toASTImpl(const ConvertToASTOptions & /* options */) const { auto identifier_parts = identifier.getParts(); return std::make_shared(std::move(identifier_parts)); diff --git a/src/Analyzer/IdentifierNode.h b/src/Analyzer/IdentifierNode.h index ced599218b1..872bb14d512 100644 --- a/src/Analyzer/IdentifierNode.h +++ b/src/Analyzer/IdentifierNode.h @@ -59,7 +59,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl(ConvertToASTOptions options) const override; + ASTPtr toASTImpl(const ConvertToASTOptions & options) const override; private: Identifier identifier; diff --git a/src/Analyzer/InterpolateNode.cpp b/src/Analyzer/InterpolateNode.cpp index 9df2ac08018..d78993c7b85 100644 --- a/src/Analyzer/InterpolateNode.cpp +++ b/src/Analyzer/InterpolateNode.cpp @@ -44,7 +44,7 @@ QueryTreeNodePtr InterpolateNode::cloneImpl() const return std::make_shared(nullptr /*expression*/, nullptr /*interpolate_expression*/); } -ASTPtr InterpolateNode::toASTImpl(ConvertToASTOptions options) const +ASTPtr InterpolateNode::toASTImpl(const ConvertToASTOptions & options) const { auto result = std::make_shared(); result->column = getExpression()->toAST(options)->getColumnName(); diff --git a/src/Analyzer/InterpolateNode.h b/src/Analyzer/InterpolateNode.h index 8ac31f0fb8f..c45800ebaaf 100644 --- a/src/Analyzer/InterpolateNode.h +++ b/src/Analyzer/InterpolateNode.h @@ -59,7 +59,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl(ConvertToASTOptions options) const override; + ASTPtr toASTImpl(const ConvertToASTOptions & options) const override; private: static constexpr size_t expression_child_index = 0; diff --git a/src/Analyzer/JoinNode.cpp b/src/Analyzer/JoinNode.cpp index 829db614774..8b6ae0289ff 100644 --- a/src/Analyzer/JoinNode.cpp +++ b/src/Analyzer/JoinNode.cpp @@ -99,7 +99,7 @@ QueryTreeNodePtr JoinNode::cloneImpl() const return std::make_shared(getLeftTableExpression(), getRightTableExpression(), getJoinExpression(), locality, strictness, kind); } -ASTPtr JoinNode::toASTImpl(ConvertToASTOptions /* options */) const +ASTPtr JoinNode::toASTImpl(const ConvertToASTOptions & /* options */) const { ASTPtr tables_in_select_query_ast = std::make_shared(); diff --git a/src/Analyzer/JoinNode.h b/src/Analyzer/JoinNode.h index 4ea08c620ef..4f071e03856 100644 --- a/src/Analyzer/JoinNode.h +++ b/src/Analyzer/JoinNode.h @@ -148,7 +148,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl(ConvertToASTOptions options) const override; + ASTPtr toASTImpl(const ConvertToASTOptions & options) const override; private: JoinLocality locality = JoinLocality::Unspecified; diff --git a/src/Analyzer/LambdaNode.cpp b/src/Analyzer/LambdaNode.cpp index 2df389c4029..e4a230de78e 100644 --- a/src/Analyzer/LambdaNode.cpp +++ b/src/Analyzer/LambdaNode.cpp @@ -65,7 +65,7 @@ QueryTreeNodePtr LambdaNode::cloneImpl() const return std::make_shared(argument_names, getExpression()); } -ASTPtr LambdaNode::toASTImpl(ConvertToASTOptions options) const +ASTPtr LambdaNode::toASTImpl(const ConvertToASTOptions & options) const { auto lambda_function_arguments_ast = std::make_shared(); diff --git a/src/Analyzer/LambdaNode.h b/src/Analyzer/LambdaNode.h index f64d49fff08..5d8e53a464c 100644 --- a/src/Analyzer/LambdaNode.h +++ b/src/Analyzer/LambdaNode.h @@ -98,7 +98,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl(ConvertToASTOptions options) const override; + ASTPtr toASTImpl(const ConvertToASTOptions & options) const override; private: Names argument_names; diff --git a/src/Analyzer/ListNode.cpp b/src/Analyzer/ListNode.cpp index 238ba0c8133..799c471d685 100644 --- a/src/Analyzer/ListNode.cpp +++ b/src/Analyzer/ListNode.cpp @@ -54,7 +54,7 @@ QueryTreeNodePtr ListNode::cloneImpl() const return std::make_shared(); } -ASTPtr ListNode::toASTImpl(ConvertToASTOptions options) const +ASTPtr ListNode::toASTImpl(const ConvertToASTOptions & options) const { auto expression_list_ast = std::make_shared(); diff --git a/src/Analyzer/ListNode.h b/src/Analyzer/ListNode.h index 9cf2011eb39..5b1abc36ae9 100644 --- a/src/Analyzer/ListNode.h +++ b/src/Analyzer/ListNode.h @@ -57,7 +57,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl(ConvertToASTOptions options) const override; + ASTPtr toASTImpl(const ConvertToASTOptions & options) const override; }; } diff --git a/src/Analyzer/MatcherNode.cpp b/src/Analyzer/MatcherNode.cpp index d6bdd50562d..9200c0f63f1 100644 --- a/src/Analyzer/MatcherNode.cpp +++ b/src/Analyzer/MatcherNode.cpp @@ -204,7 +204,7 @@ QueryTreeNodePtr MatcherNode::cloneImpl() const return matcher_node; } -ASTPtr MatcherNode::toASTImpl(ConvertToASTOptions options) const +ASTPtr MatcherNode::toASTImpl(const ConvertToASTOptions & options) const { ASTPtr result; ASTPtr transformers; diff --git a/src/Analyzer/MatcherNode.h b/src/Analyzer/MatcherNode.h index 985000f7f65..1d8f38c6f38 100644 --- a/src/Analyzer/MatcherNode.h +++ b/src/Analyzer/MatcherNode.h @@ -148,7 +148,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl(ConvertToASTOptions options) const override; + ASTPtr toASTImpl(const ConvertToASTOptions & options) const override; private: explicit MatcherNode(MatcherNodeType matcher_type_, diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index c4c36c9f192..0c586dd3fde 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -259,7 +259,7 @@ QueryTreeNodePtr QueryNode::cloneImpl() const return result_query_node; } -ASTPtr QueryNode::toASTImpl(ConvertToASTOptions options) const +ASTPtr QueryNode::toASTImpl(const ConvertToASTOptions & options) const { auto select_query = std::make_shared(); select_query->distinct = is_distinct; diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index add9e14a0b5..277d6404965 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -575,7 +575,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl(ConvertToASTOptions options) const override; + ASTPtr toASTImpl(const ConvertToASTOptions & options) const override; private: bool is_subquery = false; diff --git a/src/Analyzer/SortNode.cpp b/src/Analyzer/SortNode.cpp index 6dff649186d..750de58471a 100644 --- a/src/Analyzer/SortNode.cpp +++ b/src/Analyzer/SortNode.cpp @@ -109,7 +109,7 @@ QueryTreeNodePtr SortNode::cloneImpl() const return std::make_shared(nullptr /*expression*/, sort_direction, nulls_sort_direction, collator, with_fill); } -ASTPtr SortNode::toASTImpl(ConvertToASTOptions options) const +ASTPtr SortNode::toASTImpl(const ConvertToASTOptions & options) const { auto result = std::make_shared(); result->direction = sort_direction == SortDirection::ASCENDING ? 1 : -1; diff --git a/src/Analyzer/SortNode.h b/src/Analyzer/SortNode.h index da0996f734e..b860fd19a90 100644 --- a/src/Analyzer/SortNode.h +++ b/src/Analyzer/SortNode.h @@ -137,7 +137,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl(ConvertToASTOptions options) const override; + ASTPtr toASTImpl(const ConvertToASTOptions & options) const override; private: static constexpr size_t sort_expression_child_index = 0; diff --git a/src/Analyzer/TableFunctionNode.cpp b/src/Analyzer/TableFunctionNode.cpp index 41eb1d151c7..643383083d6 100644 --- a/src/Analyzer/TableFunctionNode.cpp +++ b/src/Analyzer/TableFunctionNode.cpp @@ -132,7 +132,7 @@ QueryTreeNodePtr TableFunctionNode::cloneImpl() const return result; } -ASTPtr TableFunctionNode::toASTImpl(ConvertToASTOptions options) const +ASTPtr TableFunctionNode::toASTImpl(const ConvertToASTOptions & options) const { auto table_function_ast = std::make_shared(); diff --git a/src/Analyzer/TableFunctionNode.h b/src/Analyzer/TableFunctionNode.h index cf2830f2ecb..7786ba62205 100644 --- a/src/Analyzer/TableFunctionNode.h +++ b/src/Analyzer/TableFunctionNode.h @@ -156,7 +156,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl(ConvertToASTOptions options) const override; + ASTPtr toASTImpl(const ConvertToASTOptions & options) const override; private: String table_function_name; diff --git a/src/Analyzer/TableNode.cpp b/src/Analyzer/TableNode.cpp index 89ea98462db..c86cbcd5a80 100644 --- a/src/Analyzer/TableNode.cpp +++ b/src/Analyzer/TableNode.cpp @@ -86,7 +86,7 @@ QueryTreeNodePtr TableNode::cloneImpl() const return result_table_node; } -ASTPtr TableNode::toASTImpl(ConvertToASTOptions /* options */) const +ASTPtr TableNode::toASTImpl(const ConvertToASTOptions & /* options */) const { if (!temporary_table_name.empty()) return std::make_shared(temporary_table_name); diff --git a/src/Analyzer/TableNode.h b/src/Analyzer/TableNode.h index 0b0b3a13775..b0bf91fa01b 100644 --- a/src/Analyzer/TableNode.h +++ b/src/Analyzer/TableNode.h @@ -106,7 +106,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl(ConvertToASTOptions options) const override; + ASTPtr toASTImpl(const ConvertToASTOptions & options) const override; private: StoragePtr storage; diff --git a/src/Analyzer/UnionNode.cpp b/src/Analyzer/UnionNode.cpp index fdf8880b2e1..2bc3daeef36 100644 --- a/src/Analyzer/UnionNode.cpp +++ b/src/Analyzer/UnionNode.cpp @@ -140,7 +140,7 @@ QueryTreeNodePtr UnionNode::cloneImpl() const return result_union_node; } -ASTPtr UnionNode::toASTImpl(ConvertToASTOptions options) const +ASTPtr UnionNode::toASTImpl(const ConvertToASTOptions & options) const { auto select_with_union_query = std::make_shared(); select_with_union_query->union_mode = union_mode; diff --git a/src/Analyzer/UnionNode.h b/src/Analyzer/UnionNode.h index bd85cb802e9..0045b0c334f 100644 --- a/src/Analyzer/UnionNode.h +++ b/src/Analyzer/UnionNode.h @@ -143,7 +143,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl(ConvertToASTOptions options) const override; + ASTPtr toASTImpl(const ConvertToASTOptions & options) const override; private: bool is_subquery = false; diff --git a/src/Analyzer/WindowNode.cpp b/src/Analyzer/WindowNode.cpp index 8aea4f3b87c..0fbe7c51bc7 100644 --- a/src/Analyzer/WindowNode.cpp +++ b/src/Analyzer/WindowNode.cpp @@ -107,7 +107,7 @@ QueryTreeNodePtr WindowNode::cloneImpl() const return window_node; } -ASTPtr WindowNode::toASTImpl(ConvertToASTOptions options) const +ASTPtr WindowNode::toASTImpl(const ConvertToASTOptions & options) const { auto window_definition = std::make_shared(); diff --git a/src/Analyzer/WindowNode.h b/src/Analyzer/WindowNode.h index 3b00b8860bf..30e1128b93c 100644 --- a/src/Analyzer/WindowNode.h +++ b/src/Analyzer/WindowNode.h @@ -175,7 +175,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; - ASTPtr toASTImpl(ConvertToASTOptions options) const override; + ASTPtr toASTImpl(const ConvertToASTOptions & options) const override; private: static constexpr size_t order_by_child_index = 0; diff --git a/src/Analyzer/tests/gtest_query_tree_node.cpp b/src/Analyzer/tests/gtest_query_tree_node.cpp index 00a698071a3..cf1ce78e423 100644 --- a/src/Analyzer/tests/gtest_query_tree_node.cpp +++ b/src/Analyzer/tests/gtest_query_tree_node.cpp @@ -36,7 +36,7 @@ public: return std::make_shared(); } - ASTPtr toASTImpl(ConvertToASTOptions /* options */) const override + ASTPtr toASTImpl(const ConvertToASTOptions & /* options */) const override { return nullptr; } From 26fdef4d495f5261c5a53d47f1fcd7b3009d4475 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 14 Mar 2023 14:56:34 +0000 Subject: [PATCH 263/418] Fix 02479_mysql_connect_to_self --- .../02479_mysql_connect_to_self.reference | 30 +++++++++++++------ .../02479_mysql_connect_to_self.sql | 2 +- 2 files changed, 22 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02479_mysql_connect_to_self.reference b/tests/queries/0_stateless/02479_mysql_connect_to_self.reference index 2b9215a3517..61e73f85ac7 100644 --- a/tests/queries/0_stateless/02479_mysql_connect_to_self.reference +++ b/tests/queries/0_stateless/02479_mysql_connect_to_self.reference @@ -40,21 +40,33 @@ QUERY id: 0 PROJECTION COLUMNS key String + a String + b String + c String PROJECTION - LIST id: 1, nodes: 1 + LIST id: 1, nodes: 4 COLUMN id: 2, column_name: key, result_type: String, source_id: 3 + COLUMN id: 4, column_name: a, result_type: String, source_id: 3 + COLUMN id: 5, column_name: b, result_type: String, source_id: 3 + COLUMN id: 6, column_name: c, result_type: String, source_id: 3 JOIN TREE TABLE_FUNCTION id: 3, table_function_name: mysql ARGUMENTS - LIST id: 4, nodes: 5 - CONSTANT id: 5, constant_value: \'127.0.0.1:9004\', constant_value_type: String - CONSTANT id: 6, constant_value: \'default\', constant_value_type: String - CONSTANT id: 7, constant_value: \'foo\', constant_value_type: String - CONSTANT id: 8, constant_value: \'default\', constant_value_type: String - CONSTANT id: 9, constant_value: \'\', constant_value_type: String + LIST id: 7, nodes: 5 + CONSTANT id: 8, constant_value: \'127.0.0.1:9004\', constant_value_type: String + CONSTANT id: 9, constant_value: \'default\', constant_value_type: String + EXPRESSION + FUNCTION id: 10, function_name: currentDatabase, function_type: ordinary, result_type: String + IDENTIFIER id: 11, identifier: foo + CONSTANT id: 12, constant_value: \'default\', constant_value_type: String + CONSTANT id: 13, constant_value: \'\', constant_value_type: String SETTINGS connection_wait_timeout=123 connect_timeout=40123002 read_write_timeout=40123001 connection_pool_size=3 -SELECT key AS key -FROM mysql(\'127.0.0.1:9004\', \'default\', \'foo\', \'default\', \'\', SETTINGS connection_wait_timeout = 123, connect_timeout = 40123002, read_write_timeout = 40123001, connection_pool_size = 3) +SELECT + key AS key, + a AS a, + b AS b, + c AS c +FROM mysql(\'127.0.0.1:9004\', \'default\', foo, \'default\', \'\', SETTINGS connection_wait_timeout = 123, connect_timeout = 40123002, read_write_timeout = 40123001, connection_pool_size = 3) --- 5 diff --git a/tests/queries/0_stateless/02479_mysql_connect_to_self.sql b/tests/queries/0_stateless/02479_mysql_connect_to_self.sql index a7b10d314b8..ff46ebf1ed2 100644 --- a/tests/queries/0_stateless/02479_mysql_connect_to_self.sql +++ b/tests/queries/0_stateless/02479_mysql_connect_to_self.sql @@ -33,7 +33,7 @@ SELECT count() FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', ' EXPLAIN QUERY TREE dump_ast = 1 SELECT * FROM mysql( - '127.0.0.1:9004', currentDatabase(), foo, 'default', '' + '127.0.0.1:9004', currentDatabase(), foo, 'default', '', SETTINGS connection_wait_timeout = 123, connect_timeout = 40123002, read_write_timeout = 40123001, connection_pool_size = 3 ); From 6919db3d46e71986b4135bcb0c4a52349eadab6c Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 15 Mar 2023 09:38:40 +0000 Subject: [PATCH 264/418] Fix test_mysql_protocol --- src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp | 2 +- tests/integration/test_mysql_protocol/test.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp b/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp index 782515d795e..291ed622521 100644 --- a/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp +++ b/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp @@ -34,7 +34,7 @@ public: using Base = InDepthQueryTreeVisitor; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + static void visitImpl(QueryTreeNodePtr & node) { auto * constant_node = node->as(); if (constant_node) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 96e7a087d5a..5e03c4883b4 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -576,7 +576,7 @@ def test_python_client(started_cluster): client.query("select name from tables") resp = exc_info.value.args[1] - assert fnmatch.fnmatch(resp, "*DB::Exception:*system.part_log*UNKNOWN_TABLE*"), resp + assert fnmatch.fnmatch(resp, "*DB::Exception:*tables*UNKNOWN_TABLE*"), resp cursor = client.cursor(pymysql.cursors.DictCursor) cursor.execute("select 1 as a, 'тест' as b") @@ -609,7 +609,7 @@ def test_python_client(started_cluster): client.query("select name from tables") resp = exc_info.value.args[1] - assert fnmatch.fnmatch(resp, "*DB::Exception:*system.part_log*UNKNOWN_TABLE*"), resp + assert fnmatch.fnmatch(resp, "*DB::Exception:*tables*UNKNOWN_TABLE*"), resp cursor = client.cursor(pymysql.cursors.DictCursor) cursor.execute("select 1 as a, 'тест' as b") From dba352274ced26b7702fb765e86d8dd8f4bad83e Mon Sep 17 00:00:00 2001 From: flynn Date: Wed, 22 Mar 2023 08:05:00 +0000 Subject: [PATCH 265/418] fix --- src/Client/QueryFuzzer.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index 1938bdfabb5..8415a990f0e 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -915,8 +915,9 @@ void QueryFuzzer::fuzz(ASTPtr & ast) if (auto * with_union = typeid_cast(ast.get())) { fuzz(with_union->list_of_selects); - /// Fuzzing SELECT query to EXPLAIN query randomly - if (fuzz_rand() % 20 == 0) + /// Fuzzing SELECT query to EXPLAIN query randomly. + /// And we only fuzzing the root query into an EXPLAIN query, not fuzzing subquery + if (fuzz_rand() % 20 == 0 && current_ast_depth <= 1) { auto explain = std::make_shared(fuzzExplainKind()); From d850c5a5265ab8e191ef7033f18949c6b13f3b40 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 22 Mar 2023 16:45:46 +0800 Subject: [PATCH 266/418] change as request --- src/Functions/parseDateTime.cpp | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 137a91584d0..a3236bbe3bc 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -507,8 +507,12 @@ namespace executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override { const auto * col_str = checkAndGetColumn(arguments[0].column.get()); - /// It was checked before in getReturnTypeImpl - assert(!col_str); + 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); @@ -1697,8 +1701,12 @@ namespace } const auto * format_column = checkAndGetColumnConst(arguments[1].column.get()); - /// It was checked before in getReturnTypeImpl - assert(!format_column); + 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(); } From db6efaa339133825cb6b06f3fd2cb6461d7e9d55 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 22 Mar 2023 16:53:47 +0800 Subject: [PATCH 267/418] fix failed uts --- src/Functions/parseDateTime.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index a3236bbe3bc..1857764557a 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -430,7 +430,7 @@ namespace else days_since_epoch = daysSinceEpochFromDate(year, month, day); - Int64 seconds_since_epoch = days_since_epoch * 86400 + hour * 3600 + minute * 60 + second; + Int64 seconds_since_epoch = days_since_epoch * 86400UZ + hour * 3600UZ + minute * 60UZ + second; /// Time zone is not specified, use local time zone if (!has_time_zone_offset) From 4522a35b0de34e5330f435cb65a5631ecae2d88e Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 22 Mar 2023 09:35:35 -0400 Subject: [PATCH 268/418] revise the spell check issue --- .../building_and_benchmarking_deflate_qpl.md | 12 +++--------- 1 file changed, 3 insertions(+), 9 deletions(-) diff --git a/docs/en/development/building_and_benchmarking_deflate_qpl.md b/docs/en/development/building_and_benchmarking_deflate_qpl.md index 207b58cb7f8..25c99bdbea5 100644 --- a/docs/en/development/building_and_benchmarking_deflate_qpl.md +++ b/docs/en/development/building_and_benchmarking_deflate_qpl.md @@ -223,16 +223,10 @@ $ [CLICKHOUSE_EXE] client -m --port=9001 ``` Inserting data: ``` bash -$ [CLICKHOUSE_EXE] client --query "INSERT INTO customer FORMAT CSV" < customer.tbl --port=9001 -$ [CLICKHOUSE_EXE] client --query "INSERT INTO part FORMAT CSV" < part.tbl --port=9001 -$ [CLICKHOUSE_EXE] client --query "INSERT INTO supplier FORMAT CSV" < supplier.tbl --port=9001 -$ [CLICKHOUSE_EXE] client --query "INSERT INTO lineorder FORMAT CSV" < lineorder.tbl --port=9001 +$ [CLICKHOUSE_EXE] client --query "INSERT INTO [TBL_FILE_NAME] FORMAT CSV" < [TBL_FILE_NAME].tbl --port=9001 ``` -Here `--port=9002` stands for the assigned port for server instance which is also defined in config_lz4_s2.xml/config_zstd_s2.xml/config_deflate_s2.xml - -For even more instances, you need replace it with the value: 9002/9003 which stand for s3/s4 instance respectively. - -if you don't assign it, the port is 9000 by default which has been used by first instance. +- [TBL_FILE_NAME] represents the name of a file named with the regular expression: *. tbl under `./benchmark_sample/rawdata_dir/ssb-dbgen`. +- `--port=9001` stands for the assigned port for server instance which is also defined in config_lz4_s2.xml/config_zstd_s2.xml/config_deflate_s2.xml. For even more instances, you need replace it with the value: 9002/9003 which stand for s3/s4 instance respectively. If you don't assign it, the port is 9000 by default which has been used by first instance. Benchmarking with 2 instances From e9015b9d2a8c3ca84c75e483b79fa11ef6598f48 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 22 Mar 2023 10:26:31 -0400 Subject: [PATCH 269/418] fixed spelling issue --- docs/en/development/building_and_benchmarking_deflate_qpl.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/en/development/building_and_benchmarking_deflate_qpl.md b/docs/en/development/building_and_benchmarking_deflate_qpl.md index 25c99bdbea5..60d6b1c7b76 100644 --- a/docs/en/development/building_and_benchmarking_deflate_qpl.md +++ b/docs/en/development/building_and_benchmarking_deflate_qpl.md @@ -104,7 +104,9 @@ Complete three steps same as lz4 above [self-check] For each codec(lz4/zstd/deflate), please execute below query to make sure the databases are created successfully: -$ select count() from lineorder_flat +```sql +select count() from lineorder_flat +``` You are expected to see below output: ```sql ┌───count()─┐ From ab44903f156ea6c1488abe3df312ad5b26141a71 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 22 Mar 2023 10:28:24 -0400 Subject: [PATCH 270/418] add to the dictionary file --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index f1bba4dc2fc..c0e180ca04c 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -542,3 +542,4 @@ zkcopy zlib znodes zstd +lineorder From 14356e3ee6d710950f4050d5973ab86c7d46d110 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 22 Mar 2023 09:17:51 +0000 Subject: [PATCH 271/418] Aspell dict: keep sorted things sorted Follow-up to #47745 --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 +- 1 file changed, 1 insertion(+), 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 b6cdd17e707..3c5af582692 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -368,6 +368,7 @@ linearizability linearizable lineasstring linefeeds +lineorder linux llvm localhost @@ -548,4 +549,3 @@ zkcopy zlib znodes zstd -lineorder From 39e4d6c0e38e5bf99607dea0b986b00027ec2ee5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 22 Mar 2023 10:23:15 +0000 Subject: [PATCH 272/418] Replace 'no-random-settings' tag by default settings per test --- .../02494_query_cache_case_agnostic_matching.sql | 7 +++++-- tests/queries/0_stateless/02494_query_cache_drop_cache.sql | 7 +++++-- .../0_stateless/02494_query_cache_eligible_queries.sql | 7 +++++-- tests/queries/0_stateless/02494_query_cache_events.sql | 7 +++++-- .../0_stateless/02494_query_cache_exception_handling.sql | 7 +++++-- tests/queries/0_stateless/02494_query_cache_explain.sql | 7 +++++-- .../0_stateless/02494_query_cache_min_query_duration.sql | 7 +++++-- .../0_stateless/02494_query_cache_min_query_runs.sql | 7 +++++-- .../02494_query_cache_nondeterministic_functions.sql | 6 +++++- .../0_stateless/02494_query_cache_normalize_ast.sql | 7 +++++-- .../0_stateless/02494_query_cache_passive_usage.sql | 7 +++++-- tests/queries/0_stateless/02494_query_cache_secrets.sql | 7 +++++-- tests/queries/0_stateless/02494_query_cache_ttl_long.sql | 7 +++++-- 13 files changed, 65 insertions(+), 25 deletions(-) diff --git a/tests/queries/0_stateless/02494_query_cache_case_agnostic_matching.sql b/tests/queries/0_stateless/02494_query_cache_case_agnostic_matching.sql index b827701b1d7..49c8f10796d 100644 --- a/tests/queries/0_stateless/02494_query_cache_case_agnostic_matching.sql +++ b/tests/queries/0_stateless/02494_query_cache_case_agnostic_matching.sql @@ -1,9 +1,12 @@ --- Tags: no-parallel, no-random-settings +-- Tags: no-parallel -- Tag no-parallel: Messes with internal cache --- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; +-- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: +SET use_query_cache = false; +SET enable_reads_from_query_cache = true; + -- Start with empty query cache (QC) and query log SYSTEM DROP QUERY CACHE; DROP TABLE system.query_log SYNC; diff --git a/tests/queries/0_stateless/02494_query_cache_drop_cache.sql b/tests/queries/0_stateless/02494_query_cache_drop_cache.sql index 8c968edbef8..33f29ea85a8 100644 --- a/tests/queries/0_stateless/02494_query_cache_drop_cache.sql +++ b/tests/queries/0_stateless/02494_query_cache_drop_cache.sql @@ -1,9 +1,12 @@ --- Tags: no-parallel, no-random-settings +-- Tags: no-parallel -- Tag no-parallel: Messes with internal cache --- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; +-- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: +SET use_query_cache = false; +SET enable_reads_from_query_cache = true; + -- Cache query result in query cache SELECT 1 SETTINGS use_query_cache = true; SELECT count(*) FROM system.query_cache; diff --git a/tests/queries/0_stateless/02494_query_cache_eligible_queries.sql b/tests/queries/0_stateless/02494_query_cache_eligible_queries.sql index 842770e155a..452ca7b4be9 100644 --- a/tests/queries/0_stateless/02494_query_cache_eligible_queries.sql +++ b/tests/queries/0_stateless/02494_query_cache_eligible_queries.sql @@ -1,9 +1,12 @@ --- Tags: no-parallel, no-random-settings +-- Tags: no-parallel -- Tag no-parallel: Messes with internal cache --- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; +-- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: +SET use_query_cache = false; +SET enable_reads_from_query_cache = true; + SYSTEM DROP QUERY CACHE; DROP TABLE IF EXISTS eligible_test; DROP TABLE IF EXISTS eligible_test2; diff --git a/tests/queries/0_stateless/02494_query_cache_events.sql b/tests/queries/0_stateless/02494_query_cache_events.sql index 1c49fdbd2e9..29feba3598a 100644 --- a/tests/queries/0_stateless/02494_query_cache_events.sql +++ b/tests/queries/0_stateless/02494_query_cache_events.sql @@ -1,9 +1,12 @@ --- Tags: no-parallel, no-random-settings +-- Tags: no-parallel -- Tag no-parallel: Messes with internal cache --- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; +-- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: +SET use_query_cache = false; +SET enable_reads_from_query_cache = true; + -- Start with empty query cache QC and query log SYSTEM DROP QUERY CACHE; DROP TABLE system.query_log SYNC; diff --git a/tests/queries/0_stateless/02494_query_cache_exception_handling.sql b/tests/queries/0_stateless/02494_query_cache_exception_handling.sql index a7a25f29d31..9b21f722b28 100644 --- a/tests/queries/0_stateless/02494_query_cache_exception_handling.sql +++ b/tests/queries/0_stateless/02494_query_cache_exception_handling.sql @@ -1,9 +1,12 @@ --- Tags: no-parallel, no-random-settings +-- Tags: no-parallel -- Tag no-parallel: Messes with internal cache --- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; +-- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: +SET use_query_cache = false; +SET enable_reads_from_query_cache = true; + SYSTEM DROP QUERY CACHE; -- If an exception is thrown during query execution, no entry must be created in the query cache diff --git a/tests/queries/0_stateless/02494_query_cache_explain.sql b/tests/queries/0_stateless/02494_query_cache_explain.sql index ffd3c1e80e3..0c1173ca275 100644 --- a/tests/queries/0_stateless/02494_query_cache_explain.sql +++ b/tests/queries/0_stateless/02494_query_cache_explain.sql @@ -1,10 +1,13 @@ --- Tags: no-parallel, no-random-settings +-- Tags: no-parallel -- Tag no-parallel: Messes with internal cache --- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_analyzer = 1; SET allow_experimental_query_cache = true; +-- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: +SET use_query_cache = false; +SET enable_reads_from_query_cache = true; + SYSTEM DROP QUERY CACHE; -- Run a silly query with a non-trivial plan and put the result into the query cache QC diff --git a/tests/queries/0_stateless/02494_query_cache_min_query_duration.sql b/tests/queries/0_stateless/02494_query_cache_min_query_duration.sql index fb2b9719189..16b15b52e15 100644 --- a/tests/queries/0_stateless/02494_query_cache_min_query_duration.sql +++ b/tests/queries/0_stateless/02494_query_cache_min_query_duration.sql @@ -1,9 +1,12 @@ --- Tags: no-parallel, no-random-settings +-- Tags: no-parallel -- Tag no-parallel: The test messes with internal cache --- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; +-- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: +SET use_query_cache = false; +SET enable_reads_from_query_cache = true; + SYSTEM DROP QUERY CACHE; -- This creates an entry in the query cache ... diff --git a/tests/queries/0_stateless/02494_query_cache_min_query_runs.sql b/tests/queries/0_stateless/02494_query_cache_min_query_runs.sql index d0317a18fdb..3a9ad86fb65 100644 --- a/tests/queries/0_stateless/02494_query_cache_min_query_runs.sql +++ b/tests/queries/0_stateless/02494_query_cache_min_query_runs.sql @@ -1,9 +1,12 @@ --- Tags: no-parallel, no-random-settings +-- Tags: no-parallel -- Tag no-parallel: The test messes with internal cache --- Tag no-random-settings: The test assumes default settings for the query cache. SET allow_experimental_query_cache = true; +-- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: +SET use_query_cache = false; +SET enable_reads_from_query_cache = true; + SYSTEM DROP QUERY CACHE; -- Cache the query after the 1st query invocation diff --git a/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql b/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql index c11acd737f8..d5e1a646311 100644 --- a/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql +++ b/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql @@ -1,9 +1,13 @@ --- Tags: no-parallel, no-random-settings +-- Tags: no-parallel -- Tag no-parallel: Messes with internal cache -- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; +-- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: +SET use_query_cache = false; +SET enable_reads_from_query_cache = true; + SYSTEM DROP QUERY CACHE; -- rand() is non-deterministic, with default settings no entry in the query cache should be created diff --git a/tests/queries/0_stateless/02494_query_cache_normalize_ast.sql b/tests/queries/0_stateless/02494_query_cache_normalize_ast.sql index 6f0b24953a6..86552e9708e 100644 --- a/tests/queries/0_stateless/02494_query_cache_normalize_ast.sql +++ b/tests/queries/0_stateless/02494_query_cache_normalize_ast.sql @@ -1,9 +1,12 @@ --- Tags: no-parallel, no-random-settings +-- Tags: no-parallel -- Tag no-parallel: Messes with internal cache --- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; +-- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: +SET use_query_cache = false; +SET enable_reads_from_query_cache = true; + -- Start with empty query cache (QC) and query log. SYSTEM DROP QUERY CACHE; DROP TABLE system.query_log SYNC; diff --git a/tests/queries/0_stateless/02494_query_cache_passive_usage.sql b/tests/queries/0_stateless/02494_query_cache_passive_usage.sql index 8041cd4b417..b5373d5cbda 100644 --- a/tests/queries/0_stateless/02494_query_cache_passive_usage.sql +++ b/tests/queries/0_stateless/02494_query_cache_passive_usage.sql @@ -1,9 +1,12 @@ --- Tags: no-parallel, no-random-settings +-- Tags: no-parallel -- Tag no-parallel: Messes with internal cache --- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; +-- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: +SET use_query_cache = false; +SET enable_reads_from_query_cache = true; + -- Start with empty query cache (QC). SYSTEM DROP QUERY CACHE; diff --git a/tests/queries/0_stateless/02494_query_cache_secrets.sql b/tests/queries/0_stateless/02494_query_cache_secrets.sql index 7c090fab9c8..b86a27bcb18 100644 --- a/tests/queries/0_stateless/02494_query_cache_secrets.sql +++ b/tests/queries/0_stateless/02494_query_cache_secrets.sql @@ -1,10 +1,13 @@ --- Tags: no-parallel, no-fasttest, no-random-settings +-- Tags: no-parallel, no-fasttest -- Tag no-fasttest: Depends on OpenSSL -- Tag no-parallel: Messes with internal cache --- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; +-- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: +SET use_query_cache = false; +SET enable_reads_from_query_cache = true; + SYSTEM DROP QUERY CACHE; -- Cache a result of a query with secret in the query cache diff --git a/tests/queries/0_stateless/02494_query_cache_ttl_long.sql b/tests/queries/0_stateless/02494_query_cache_ttl_long.sql index 1a99d85688d..1d142f9988a 100644 --- a/tests/queries/0_stateless/02494_query_cache_ttl_long.sql +++ b/tests/queries/0_stateless/02494_query_cache_ttl_long.sql @@ -1,11 +1,14 @@ --- Tags: no-fasttest, no-parallel, long, no-random-settings +-- Tags: no-fasttest, no-parallel, long -- Tag no-fasttest: Test runtime is > 6 sec -- Tag long: Test runtime is > 6 sec -- Tag no-parallel: Messes with internal cache --- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; +-- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: +SET use_query_cache = false; +SET enable_reads_from_query_cache = true; + SYSTEM DROP QUERY CACHE; -- Cache query result into query cache with a TTL of 3 sec From c58b860d081e9494df6b36f542861ab8a12b0540 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 22 Mar 2023 11:07:35 +0000 Subject: [PATCH 273/418] Revert randomization in functional tests --- tests/clickhouse-test | 2 -- .../02494_query_cache_case_agnostic_matching.sql | 4 ---- tests/queries/0_stateless/02494_query_cache_drop_cache.sql | 4 ---- .../0_stateless/02494_query_cache_eligible_queries.sql | 4 ---- tests/queries/0_stateless/02494_query_cache_events.sql | 4 ---- .../0_stateless/02494_query_cache_exception_handling.sql | 4 ---- tests/queries/0_stateless/02494_query_cache_explain.sql | 4 ---- .../0_stateless/02494_query_cache_min_query_duration.sql | 6 +----- .../0_stateless/02494_query_cache_min_query_runs.sql | 6 +----- .../02494_query_cache_nondeterministic_functions.sql | 5 ----- .../queries/0_stateless/02494_query_cache_normalize_ast.sql | 4 ---- .../queries/0_stateless/02494_query_cache_passive_usage.sql | 4 ---- tests/queries/0_stateless/02494_query_cache_secrets.sql | 4 ---- tests/queries/0_stateless/02494_query_cache_ttl_long.sql | 4 ---- 14 files changed, 2 insertions(+), 57 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 185f49062f9..aec52981724 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -528,8 +528,6 @@ class SettingsRandomizer: "enable_memory_bound_merging_of_aggregation_results": lambda: random.randint( 0, 1 ), - "use_query_cache": lambda: random.randint(0, 1), - "enable_reads_from_query_cache": lambda: 0, } @staticmethod diff --git a/tests/queries/0_stateless/02494_query_cache_case_agnostic_matching.sql b/tests/queries/0_stateless/02494_query_cache_case_agnostic_matching.sql index 49c8f10796d..9440a1fd9c0 100644 --- a/tests/queries/0_stateless/02494_query_cache_case_agnostic_matching.sql +++ b/tests/queries/0_stateless/02494_query_cache_case_agnostic_matching.sql @@ -3,10 +3,6 @@ SET allow_experimental_query_cache = true; --- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: -SET use_query_cache = false; -SET enable_reads_from_query_cache = true; - -- Start with empty query cache (QC) and query log SYSTEM DROP QUERY CACHE; DROP TABLE system.query_log SYNC; diff --git a/tests/queries/0_stateless/02494_query_cache_drop_cache.sql b/tests/queries/0_stateless/02494_query_cache_drop_cache.sql index 33f29ea85a8..1f61472fcb0 100644 --- a/tests/queries/0_stateless/02494_query_cache_drop_cache.sql +++ b/tests/queries/0_stateless/02494_query_cache_drop_cache.sql @@ -3,10 +3,6 @@ SET allow_experimental_query_cache = true; --- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: -SET use_query_cache = false; -SET enable_reads_from_query_cache = true; - -- Cache query result in query cache SELECT 1 SETTINGS use_query_cache = true; SELECT count(*) FROM system.query_cache; diff --git a/tests/queries/0_stateless/02494_query_cache_eligible_queries.sql b/tests/queries/0_stateless/02494_query_cache_eligible_queries.sql index 452ca7b4be9..b4bc9e2c258 100644 --- a/tests/queries/0_stateless/02494_query_cache_eligible_queries.sql +++ b/tests/queries/0_stateless/02494_query_cache_eligible_queries.sql @@ -3,10 +3,6 @@ SET allow_experimental_query_cache = true; --- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: -SET use_query_cache = false; -SET enable_reads_from_query_cache = true; - SYSTEM DROP QUERY CACHE; DROP TABLE IF EXISTS eligible_test; DROP TABLE IF EXISTS eligible_test2; diff --git a/tests/queries/0_stateless/02494_query_cache_events.sql b/tests/queries/0_stateless/02494_query_cache_events.sql index 29feba3598a..d775467d525 100644 --- a/tests/queries/0_stateless/02494_query_cache_events.sql +++ b/tests/queries/0_stateless/02494_query_cache_events.sql @@ -3,10 +3,6 @@ SET allow_experimental_query_cache = true; --- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: -SET use_query_cache = false; -SET enable_reads_from_query_cache = true; - -- Start with empty query cache QC and query log SYSTEM DROP QUERY CACHE; DROP TABLE system.query_log SYNC; diff --git a/tests/queries/0_stateless/02494_query_cache_exception_handling.sql b/tests/queries/0_stateless/02494_query_cache_exception_handling.sql index 9b21f722b28..4d686d81ed3 100644 --- a/tests/queries/0_stateless/02494_query_cache_exception_handling.sql +++ b/tests/queries/0_stateless/02494_query_cache_exception_handling.sql @@ -3,10 +3,6 @@ SET allow_experimental_query_cache = true; --- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: -SET use_query_cache = false; -SET enable_reads_from_query_cache = true; - SYSTEM DROP QUERY CACHE; -- If an exception is thrown during query execution, no entry must be created in the query cache diff --git a/tests/queries/0_stateless/02494_query_cache_explain.sql b/tests/queries/0_stateless/02494_query_cache_explain.sql index 0c1173ca275..68b7e0005f8 100644 --- a/tests/queries/0_stateless/02494_query_cache_explain.sql +++ b/tests/queries/0_stateless/02494_query_cache_explain.sql @@ -4,10 +4,6 @@ SET allow_experimental_analyzer = 1; SET allow_experimental_query_cache = true; --- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: -SET use_query_cache = false; -SET enable_reads_from_query_cache = true; - SYSTEM DROP QUERY CACHE; -- Run a silly query with a non-trivial plan and put the result into the query cache QC diff --git a/tests/queries/0_stateless/02494_query_cache_min_query_duration.sql b/tests/queries/0_stateless/02494_query_cache_min_query_duration.sql index 16b15b52e15..7d759c86130 100644 --- a/tests/queries/0_stateless/02494_query_cache_min_query_duration.sql +++ b/tests/queries/0_stateless/02494_query_cache_min_query_duration.sql @@ -1,12 +1,8 @@ -- Tags: no-parallel --- Tag no-parallel: The test messes with internal cache +-- Tag no-parallel: Messes with internal cache SET allow_experimental_query_cache = true; --- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: -SET use_query_cache = false; -SET enable_reads_from_query_cache = true; - SYSTEM DROP QUERY CACHE; -- This creates an entry in the query cache ... diff --git a/tests/queries/0_stateless/02494_query_cache_min_query_runs.sql b/tests/queries/0_stateless/02494_query_cache_min_query_runs.sql index 3a9ad86fb65..2401beeab13 100644 --- a/tests/queries/0_stateless/02494_query_cache_min_query_runs.sql +++ b/tests/queries/0_stateless/02494_query_cache_min_query_runs.sql @@ -1,12 +1,8 @@ -- Tags: no-parallel --- Tag no-parallel: The test messes with internal cache +-- Tag no-parallel: Messes with internal cache SET allow_experimental_query_cache = true; --- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: -SET use_query_cache = false; -SET enable_reads_from_query_cache = true; - SYSTEM DROP QUERY CACHE; -- Cache the query after the 1st query invocation diff --git a/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql b/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql index d5e1a646311..534d63aa427 100644 --- a/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql +++ b/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql @@ -1,13 +1,8 @@ -- Tags: no-parallel -- Tag no-parallel: Messes with internal cache --- Tag no-random-settings: Assumes default settings for the query cache. SET allow_experimental_query_cache = true; --- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: -SET use_query_cache = false; -SET enable_reads_from_query_cache = true; - SYSTEM DROP QUERY CACHE; -- rand() is non-deterministic, with default settings no entry in the query cache should be created diff --git a/tests/queries/0_stateless/02494_query_cache_normalize_ast.sql b/tests/queries/0_stateless/02494_query_cache_normalize_ast.sql index 86552e9708e..7e3cd273312 100644 --- a/tests/queries/0_stateless/02494_query_cache_normalize_ast.sql +++ b/tests/queries/0_stateless/02494_query_cache_normalize_ast.sql @@ -3,10 +3,6 @@ SET allow_experimental_query_cache = true; --- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: -SET use_query_cache = false; -SET enable_reads_from_query_cache = true; - -- Start with empty query cache (QC) and query log. SYSTEM DROP QUERY CACHE; DROP TABLE system.query_log SYNC; diff --git a/tests/queries/0_stateless/02494_query_cache_passive_usage.sql b/tests/queries/0_stateless/02494_query_cache_passive_usage.sql index b5373d5cbda..8f1e3972b6d 100644 --- a/tests/queries/0_stateless/02494_query_cache_passive_usage.sql +++ b/tests/queries/0_stateless/02494_query_cache_passive_usage.sql @@ -3,10 +3,6 @@ SET allow_experimental_query_cache = true; --- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: -SET use_query_cache = false; -SET enable_reads_from_query_cache = true; - -- Start with empty query cache (QC). SYSTEM DROP QUERY CACHE; diff --git a/tests/queries/0_stateless/02494_query_cache_secrets.sql b/tests/queries/0_stateless/02494_query_cache_secrets.sql index b86a27bcb18..99a972b003c 100644 --- a/tests/queries/0_stateless/02494_query_cache_secrets.sql +++ b/tests/queries/0_stateless/02494_query_cache_secrets.sql @@ -4,10 +4,6 @@ SET allow_experimental_query_cache = true; --- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: -SET use_query_cache = false; -SET enable_reads_from_query_cache = true; - SYSTEM DROP QUERY CACHE; -- Cache a result of a query with secret in the query cache diff --git a/tests/queries/0_stateless/02494_query_cache_ttl_long.sql b/tests/queries/0_stateless/02494_query_cache_ttl_long.sql index 1d142f9988a..135ddf2195c 100644 --- a/tests/queries/0_stateless/02494_query_cache_ttl_long.sql +++ b/tests/queries/0_stateless/02494_query_cache_ttl_long.sql @@ -5,10 +5,6 @@ SET allow_experimental_query_cache = true; --- The test assumes that these two settings have default values. Neutralize the effect of setting randomization: -SET use_query_cache = false; -SET enable_reads_from_query_cache = true; - SYSTEM DROP QUERY CACHE; -- Cache query result into query cache with a TTL of 3 sec From b019ed90be1e3bca4f03b249bb8383179c8a8a20 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 22 Mar 2023 11:09:30 +0000 Subject: [PATCH 274/418] Randomize query cache settings in stress tests --- tests/ci/stress.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index 5e151e6c098..f976aeec63b 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -43,6 +43,10 @@ def get_options(i, upgrade_check): client_options.append("join_algorithm='auto'") client_options.append("max_rows_in_join=1000") + if i % 3 == 1: + client_options.append("allow_experimental_query_cache=1") + client_options.append("use_query_cache=1") + if i % 5 == 1: client_options.append("memory_tracker_fault_probability=0.001") From b832840cf3cc79e39a75d489915f35e9d153965c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 22 Mar 2023 11:21:12 +0000 Subject: [PATCH 275/418] Recalculate always projections on mutations in ReplicatedMergeTree --- .../MergeTree/ReplicatedMergeTreeTableMetadata.cpp | 8 ++++++++ ..._drop_column_with_projections_replicated.reference | 0 .../02691_drop_column_with_projections_replicated.sql | 11 +++++++++++ 3 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/02691_drop_column_with_projections_replicated.reference create mode 100644 tests/queries/0_stateless/02691_drop_column_with_projections_replicated.sql diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp index f06ce725daa..41ff93e28f7 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp @@ -456,6 +456,14 @@ StorageInMemoryMetadata ReplicatedMergeTreeTableMetadata::Diff::getNewMetadata(c new_metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST( new_metadata.table_ttl.definition_ast, new_metadata.columns, context, new_metadata.primary_key); + if (!projections_changed) + { + ProjectionsDescription recalculated_projections; + for (const auto & projection : new_metadata.projections) + recalculated_projections.add(ProjectionDescription::getProjectionFromAST(projection.definition_ast, new_metadata.columns, context)); + new_metadata.projections = std::move(recalculated_projections); + } + return new_metadata; } diff --git a/tests/queries/0_stateless/02691_drop_column_with_projections_replicated.reference b/tests/queries/0_stateless/02691_drop_column_with_projections_replicated.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02691_drop_column_with_projections_replicated.sql b/tests/queries/0_stateless/02691_drop_column_with_projections_replicated.sql new file mode 100644 index 00000000000..c28c2f233dd --- /dev/null +++ b/tests/queries/0_stateless/02691_drop_column_with_projections_replicated.sql @@ -0,0 +1,11 @@ +DROP TABLE IF EXISTS 02691_drop_column_replicated; + +CREATE TABLE 02691_drop_column_replicated (col1 Int64, col2 Int64, PROJECTION 02691_drop_column_replicated (SELECT * ORDER BY col1 )) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/02691_drop_column', 'r1') +ORDER BY col1; + +INSERT INTO 02691_drop_column_replicated VALUES (1, 2); + +ALTER TABLE 02691_drop_column_replicated DROP COLUMN col2 SETTINGS alter_sync = 2; + +DROP TABLE 02691_drop_column_replicated; From c7252198aa48bca07880366aa9f3d3839e584a97 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 22 Mar 2023 19:25:37 +0800 Subject: [PATCH 276/418] fix statelsss tests --- tests/queries/0_stateless/02668_parse_datetime.reference | 8 ++++---- tests/queries/0_stateless/02668_parse_datetime.sql | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/02668_parse_datetime.reference b/tests/queries/0_stateless/02668_parse_datetime.reference index b2a42e01585..6f17041337d 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.reference +++ b/tests/queries/0_stateless/02668_parse_datetime.reference @@ -1,6 +1,6 @@ -- { echoOn } -- year -select parseDateTime('2020', '%Y') = toDateTime('2020-01-01'); +select parseDateTime('2020', '%Y', 'UTC') = toDateTime('2020-01-01', 'UTC'); 1 -- month select parseDateTime('02', '%m') = toDateTime('2000-02-01'); @@ -191,9 +191,9 @@ 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'); +select parseDateTime('2021-01-04+23:00:00', '%Y-%m-%d+%H:%i:%s', 'UTC') = toDateTime('2021-01-04 23:00:00', 'UTC'); 1 -select parseDateTime('2019-07-03 11:04:10', '%Y-%m-%d %H:%i:%s') = toDateTime('2019-07-03 11:04:10'); +select parseDateTime('2019-07-03 11:04:10', '%Y-%m-%d %H:%i:%s', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); 1 -select parseDateTime('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y') = toDateTime('2019-07-03 11:04:10'); +select parseDateTime('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); 1 diff --git a/tests/queries/0_stateless/02668_parse_datetime.sql b/tests/queries/0_stateless/02668_parse_datetime.sql index 8ff3f5a03f2..71e90165ec4 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.sql +++ b/tests/queries/0_stateless/02668_parse_datetime.sql @@ -1,6 +1,6 @@ -- { echoOn } -- year -select parseDateTime('2020', '%Y') = toDateTime('2020-01-01'); +select parseDateTime('2020', '%Y', 'UTC') = toDateTime('2020-01-01', 'UTC'); -- month select parseDateTime('02', '%m') = toDateTime('2000-02-01'); @@ -131,8 +131,8 @@ 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'); -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'); +select parseDateTime('2021-01-04+23:00:00', '%Y-%m-%d+%H:%i:%s', 'UTC') = toDateTime('2021-01-04 23:00:00', 'UTC'); +select parseDateTime('2019-07-03 11:04:10', '%Y-%m-%d %H:%i:%s', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); +select parseDateTime('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); -- { echoOff } From 301e26b23df6156137e90d4c9c2797cbb23e1a5b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 22 Mar 2023 13:05:38 +0100 Subject: [PATCH 277/418] Parallel replicas. Part 2.5 (#47858) --- docker/test/stateful/run.sh | 2 +- src/Core/Settings.h | 1 + src/Processors/Sources/RemoteSource.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 6 ++++-- tests/queries/0_stateless/02221_parallel_replicas_bug.sh | 2 +- .../02226_parallel_reading_from_replicas_benchmark.sh | 1 + tests/queries/0_stateless/02404_memory_bound_merging.sql | 1 + tests/queries/1_stateful/00177_memory_bound_merging.sh | 6 +++--- 8 files changed, 13 insertions(+), 8 deletions(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index e7a400b8216..c973b6c6ec6 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -128,7 +128,7 @@ function run_tests() set +e if [[ -n "$USE_PARALLEL_REPLICAS" ]] && [[ "$USE_PARALLEL_REPLICAS" -eq 1 ]]; then - clickhouse-test --client="clickhouse-client --use_hedged_requests=0 --allow_experimental_parallel_reading_from_replicas=1 \ + clickhouse-test --client="clickhouse-client --use_hedged_requests=0 --allow_experimental_parallel_reading_from_replicas=1 --parallel_replicas_for_non_replicated_merge_tree=1 \ --max_parallel_replicas=100 --cluster_for_parallel_replicas='parallel_replicas'" \ -j 2 --testname --shard --zookeeper --check-zookeeper-session --no-stateless --no-parallel-replicas --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \ "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3770d7f73a0..accfe387137 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -152,6 +152,7 @@ class IColumn; M(String, cluster_for_parallel_replicas, "default", "Cluster for a shard in which current server is located", 0) \ M(Bool, allow_experimental_parallel_reading_from_replicas, false, "If true, ClickHouse will send a SELECT query to all replicas of a table. It will work for any kind on MergeTree table.", 0) \ M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \ + M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \ \ M(Bool, skip_unavailable_shards, false, "If true, ClickHouse silently skips unavailable shards and nodes unresolvable through DNS. Shard is marked as unavailable when none of the replicas can be reached.", 0) \ \ diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 69964d569fa..5ffe39b1ebf 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -76,7 +76,7 @@ ISource::Status RemoteSource::prepare() return status; } - if (status == Status::PortFull) + if (status == Status::PortFull || status == Status::Ready) { /// Also push empty chunk to dependency to signal that we read data from remote source /// or answered to the incoming request from parallel replica diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index d9bb189524c..abab2b2dc68 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -214,7 +214,7 @@ void StorageMergeTree::read( size_t max_block_size, size_t num_streams) { - if (local_context->canUseParallelReplicasOnInitiator()) + if (local_context->canUseParallelReplicasOnInitiator() && local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree) { auto table_id = getStorageID(); @@ -245,10 +245,12 @@ void StorageMergeTree::read( } else { + const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower() && local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree; + if (auto plan = reader.read( column_names, storage_snapshot, query_info, local_context, max_block_size, num_streams, - processed_stage, nullptr, /*enable_parallel_reading*/local_context->canUseParallelReplicasOnFollower())) + processed_stage, nullptr, enable_parallel_reading)) query_plan = std::move(*plan); } diff --git a/tests/queries/0_stateless/02221_parallel_replicas_bug.sh b/tests/queries/0_stateless/02221_parallel_replicas_bug.sh index cce32bf8272..3c44a2a7ba7 100755 --- a/tests/queries/0_stateless/02221_parallel_replicas_bug.sh +++ b/tests/queries/0_stateless/02221_parallel_replicas_bug.sh @@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --allow_experimental_parallel_reading_from_replicas=1 -nm < "$CURDIR"/01099_parallel_distributed_insert_select.sql > /dev/null +${CLICKHOUSE_CLIENT} --allow_experimental_parallel_reading_from_replicas=1 --parallel_replicas_for_non_replicated_merge_tree=1 -nm < "$CURDIR"/01099_parallel_distributed_insert_select.sql > /dev/null diff --git a/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh b/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh index 2a163746e20..4b8f8da5480 100755 --- a/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh +++ b/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh @@ -15,6 +15,7 @@ as select * from numbers(1); # Logical error: 'Coordinator for parallel reading from replicas is not initialized'. opts=( --allow_experimental_parallel_reading_from_replicas 1 + --parallel_replicas_for_non_replicated_merge_tree 1 --max_parallel_replicas 3 --iterations 1 diff --git a/tests/queries/0_stateless/02404_memory_bound_merging.sql b/tests/queries/0_stateless/02404_memory_bound_merging.sql index b6299de9aae..23c29ca8d93 100644 --- a/tests/queries/0_stateless/02404_memory_bound_merging.sql +++ b/tests/queries/0_stateless/02404_memory_bound_merging.sql @@ -61,6 +61,7 @@ create table pr_t(a UInt64, b UInt64) engine=MergeTree order by a; insert into pr_t select number % 1000, number % 1000 from numbers_mt(1e6); set allow_experimental_parallel_reading_from_replicas = 1; +set parallel_replicas_for_non_replicated_merge_tree = 1; set max_parallel_replicas = 3; set use_hedged_requests = 0; set cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; diff --git a/tests/queries/1_stateful/00177_memory_bound_merging.sh b/tests/queries/1_stateful/00177_memory_bound_merging.sh index b01c62a7c92..008422be108 100755 --- a/tests/queries/1_stateful/00177_memory_bound_merging.sh +++ b/tests/queries/1_stateful/00177_memory_bound_merging.sh @@ -34,7 +34,7 @@ test1() { GROUP BY CounterID, URL, EventDate ORDER BY URL, EventDate LIMIT 5 OFFSET 10 - SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, use_hedged_requests = 0" + SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3, use_hedged_requests = 0" check_replicas_read_in_order $query_id } @@ -51,7 +51,7 @@ test2() { GROUP BY URL, EventDate ORDER BY URL, EventDate LIMIT 5 OFFSET 10 - SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, use_hedged_requests = 0, query_plan_aggregation_in_order = 1" + SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3, use_hedged_requests = 0, query_plan_aggregation_in_order = 1" check_replicas_read_in_order $query_id } @@ -67,7 +67,7 @@ test3() { FROM test.hits WHERE CounterID = 1704509 AND UserID = 4322253409885123546 GROUP BY URL, EventDate - SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, use_hedged_requests = 0 + SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3, use_hedged_requests = 0 ) WHERE explain LIKE '%Aggr%Transform%' OR explain LIKE '%InOrder%'" } From ea2d7e0035e462de01ad5046b49e66fd6e024bcc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 22 Mar 2023 12:46:22 +0000 Subject: [PATCH 278/418] Decouple QC enablement from 'join_use_nulls' --- tests/ci/stress.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index f976aeec63b..42433cd92fb 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -6,6 +6,7 @@ import os import argparse import logging import time +import random def get_options(i, upgrade_check): @@ -43,7 +44,7 @@ def get_options(i, upgrade_check): client_options.append("join_algorithm='auto'") client_options.append("max_rows_in_join=1000") - if i % 3 == 1: + if i > 0 and random.random() < 1/3: client_options.append("allow_experimental_query_cache=1") client_options.append("use_query_cache=1") From 8851b2b8f6d272a5ad87788c5ae32c5a804edb42 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 22 Mar 2023 12:53:22 +0000 Subject: [PATCH 279/418] Add options --link and -y to clickhouse install --- programs/install/Install.cpp | 177 +++++++++++++++++++++++------------ 1 file changed, 116 insertions(+), 61 deletions(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 80f3b0bbc63..b142159fbdf 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -222,6 +222,8 @@ int mainEntryClickHouseInstall(int argc, char ** argv) ("pid-path", po::value()->default_value("var/run/clickhouse-server"), "directory for pid file") ("user", po::value()->default_value(DEFAULT_CLICKHOUSE_SERVER_USER), "clickhouse user to create") ("group", po::value()->default_value(DEFAULT_CLICKHOUSE_SERVER_GROUP), "clickhouse group to create") + ("noninteractive,y", "run non-interactively") + ("link", "create symlink to the binary instead of copying to binary-path") ; po::variables_map options; @@ -267,8 +269,6 @@ int mainEntryClickHouseInstall(int argc, char ** argv) /// Copy binary to the destination directory. - /// TODO An option to link instead of copy - useful for developers. - fs::path prefix = options["prefix"].as(); fs::path bin_dir = prefix / options["binary-path"].as(); @@ -281,76 +281,129 @@ int mainEntryClickHouseInstall(int argc, char ** argv) bool old_binary_exists = fs::exists(main_bin_path); bool already_installed = false; - /// Check if the binary is the same file (already installed). - if (old_binary_exists && binary_self_canonical_path == fs::canonical(main_bin_path)) + if (options.count("link")) { - already_installed = true; - fmt::print("ClickHouse binary is already located at {}\n", main_bin_path.string()); - } - /// Check if binary has the same content. - else if (old_binary_exists && binary_size == fs::file_size(main_bin_path)) - { - fmt::print("Found already existing ClickHouse binary at {} having the same size. Will check its contents.\n", - main_bin_path.string()); - - if (filesEqual(binary_self_path.string(), main_bin_path.string())) + if (old_binary_exists) { - already_installed = true; - fmt::print("ClickHouse binary is already located at {} and it has the same content as {}\n", - main_bin_path.string(), binary_self_canonical_path.string()); - } - } + bool is_symlink = FS::isSymlink(main_bin_path); + fs::path points_to; + if (is_symlink) + points_to = fs::weakly_canonical(FS::readSymlink(main_bin_path)); - if (already_installed) - { - if (0 != chmod(main_bin_path.string().c_str(), S_IRUSR | S_IRGRP | S_IROTH | S_IXUSR | S_IXGRP | S_IXOTH)) - throwFromErrno(fmt::format("Cannot chmod {}", main_bin_path.string()), ErrorCodes::SYSTEM_ERROR); + if (is_symlink && points_to == binary_self_canonical_path) + { + already_installed = true; + } + else + { + if (!is_symlink) + { + fmt::print("File {} already exists but it's not a symlink. Will rename to {}.\n", + main_bin_path.string(), main_bin_old_path.string()); + fs::rename(main_bin_path, main_bin_old_path); + } + else if (points_to != main_bin_path) + { + fmt::print("Symlink {} already exists but it points to {}. Will replace the old symlink to {}.\n", + main_bin_path.string(), points_to.string(), binary_self_canonical_path.string()); + fs::remove(main_bin_path); + } + } + } + + if (!already_installed) + { + if (!fs::exists(bin_dir)) + { + fmt::print("Creating binary directory {}.\n", bin_dir.string()); + fs::create_directories(bin_dir); + } + + fmt::print("Creating symlink {} to {}.\n", main_bin_path.string(), binary_self_canonical_path.string()); + fs::create_symlink(binary_self_canonical_path, main_bin_path); + + if (0 != chmod(binary_self_canonical_path.string().c_str(), S_IRUSR | S_IRGRP | S_IROTH | S_IXUSR | S_IXGRP | S_IXOTH)) + throwFromErrno(fmt::format("Cannot chmod {}", binary_self_canonical_path.string()), ErrorCodes::SYSTEM_ERROR); + } } else { - if (!fs::exists(bin_dir)) + bool is_symlink = FS::isSymlink(main_bin_path); + + if (!is_symlink) { - fmt::print("Creating binary directory {}.\n", bin_dir.string()); - fs::create_directories(bin_dir); + /// Check if the binary is the same file (already installed). + if (old_binary_exists && binary_self_canonical_path == fs::canonical(main_bin_path)) + { + already_installed = true; + fmt::print("ClickHouse binary is already located at {}\n", main_bin_path.string()); + } + /// Check if binary has the same content. + else if (old_binary_exists && binary_size == fs::file_size(main_bin_path)) + { + fmt::print("Found already existing ClickHouse binary at {} having the same size. Will check its contents.\n", + main_bin_path.string()); + + if (filesEqual(binary_self_path.string(), main_bin_path.string())) + { + already_installed = true; + fmt::print("ClickHouse binary is already located at {} and it has the same content as {}\n", + main_bin_path.string(), binary_self_canonical_path.string()); + } + } } - size_t available_space = fs::space(bin_dir).available; - if (available_space < binary_size) - throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Not enough space for clickhouse binary in {}, required {}, available {}.", - bin_dir.string(), ReadableSize(binary_size), ReadableSize(available_space)); - - fmt::print("Copying ClickHouse binary to {}\n", main_bin_tmp_path.string()); - - try + if (already_installed) { - ReadBufferFromFile in(binary_self_path.string()); - WriteBufferFromFile out(main_bin_tmp_path.string()); - copyData(in, out); - out.sync(); - - if (0 != fchmod(out.getFD(), S_IRUSR | S_IRGRP | S_IROTH | S_IXUSR | S_IXGRP | S_IXOTH)) - throwFromErrno(fmt::format("Cannot chmod {}", main_bin_tmp_path.string()), ErrorCodes::SYSTEM_ERROR); - - out.finalize(); + if (0 != chmod(main_bin_path.string().c_str(), S_IRUSR | S_IRGRP | S_IROTH | S_IXUSR | S_IXGRP | S_IXOTH)) + throwFromErrno(fmt::format("Cannot chmod {}", main_bin_path.string()), ErrorCodes::SYSTEM_ERROR); } - catch (const Exception & e) + else { - if (e.code() == ErrorCodes::CANNOT_OPEN_FILE && geteuid() != 0) - std::cerr << "Install must be run as root: " << formatWithSudo("./clickhouse install") << '\n'; - throw; + if (!fs::exists(bin_dir)) + { + fmt::print("Creating binary directory {}.\n", bin_dir.string()); + fs::create_directories(bin_dir); + } + + size_t available_space = fs::space(bin_dir).available; + if (available_space < binary_size) + throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Not enough space for clickhouse binary in {}, required {}, available {}.", + bin_dir.string(), ReadableSize(binary_size), ReadableSize(available_space)); + + fmt::print("Copying ClickHouse binary to {}\n", main_bin_tmp_path.string()); + + try + { + ReadBufferFromFile in(binary_self_path.string()); + WriteBufferFromFile out(main_bin_tmp_path.string()); + copyData(in, out); + out.sync(); + + if (0 != fchmod(out.getFD(), S_IRUSR | S_IRGRP | S_IROTH | S_IXUSR | S_IXGRP | S_IXOTH)) + throwFromErrno(fmt::format("Cannot chmod {}", main_bin_tmp_path.string()), ErrorCodes::SYSTEM_ERROR); + + out.finalize(); + } + catch (const Exception & e) + { + if (e.code() == ErrorCodes::CANNOT_OPEN_FILE && geteuid() != 0) + std::cerr << "Install must be run as root: " << formatWithSudo("./clickhouse install") << '\n'; + throw; + } + + if (old_binary_exists) + { + fmt::print("{} already exists, will rename existing binary to {} and put the new binary in place\n", + main_bin_path.string(), main_bin_old_path.string()); + + /// There is file exchange operation in Linux but it's not portable. + fs::rename(main_bin_path, main_bin_old_path); + } + + fmt::print("Renaming {} to {}.\n", main_bin_tmp_path.string(), main_bin_path.string()); + fs::rename(main_bin_tmp_path, main_bin_path); } - - if (old_binary_exists) - { - fmt::print("{} already exists, will rename existing binary to {} and put the new binary in place\n", - main_bin_path.string(), main_bin_old_path.string()); - - /// There is file exchange operation in Linux but it's not portable. - fs::rename(main_bin_path, main_bin_old_path); - } - - fmt::print("Renaming {} to {}.\n", main_bin_tmp_path.string(), main_bin_path.string()); - fs::rename(main_bin_tmp_path, main_bin_path); } /// Create symlinks. @@ -384,7 +437,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) if (is_symlink) points_to = fs::weakly_canonical(FS::readSymlink(symlink_path)); - if (is_symlink && points_to == main_bin_path) + if (is_symlink && (points_to == main_bin_path || (options.count("link") && points_to == binary_self_canonical_path))) { need_to_create = false; } @@ -709,7 +762,9 @@ int mainEntryClickHouseInstall(int argc, char ** argv) /// dpkg or apt installers can ask for non-interactive work explicitly. const char * debian_frontend_var = getenv("DEBIAN_FRONTEND"); // NOLINT(concurrency-mt-unsafe) - bool noninteractive = debian_frontend_var && debian_frontend_var == std::string_view("noninteractive"); + bool noninteractive = (debian_frontend_var && debian_frontend_var == std::string_view("noninteractive")) + || options.count("noninteractive"); + bool is_interactive = !noninteractive && stdin_is_a_tty && stdout_is_a_tty; From d690e0f4ecce6ead141c82df9565a02995059727 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 22 Mar 2023 12:55:24 +0000 Subject: [PATCH 280/418] Automatic style fix --- tests/ci/stress.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index 42433cd92fb..953b55dbf63 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -44,7 +44,7 @@ def get_options(i, upgrade_check): client_options.append("join_algorithm='auto'") client_options.append("max_rows_in_join=1000") - if i > 0 and random.random() < 1/3: + if i > 0 and random.random() < 1 / 3: client_options.append("allow_experimental_query_cache=1") client_options.append("use_query_cache=1") From 2b9a73baa83729402f230692e246e674ea4e2cca Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 22 Mar 2023 13:58:10 +0000 Subject: [PATCH 281/418] Docs: Fix wrong bracket nesting for GenerateRandom --- docs/en/engines/table-engines/special/generate.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/special/generate.md b/docs/en/engines/table-engines/special/generate.md index 77d90082ddc..714afe3c3b5 100644 --- a/docs/en/engines/table-engines/special/generate.md +++ b/docs/en/engines/table-engines/special/generate.md @@ -15,7 +15,7 @@ Usage examples: ## Usage in ClickHouse Server {#usage-in-clickhouse-server} ``` sql -ENGINE = GenerateRandom([random_seed] [,max_string_length] [,max_array_length]) +ENGINE = GenerateRandom([random_seed [,max_string_length [,max_array_length]]]) ``` The `max_array_length` and `max_string_length` parameters specify maximum length of all From 02c5d1f3647473a825ebbc1d2340291cfc707f4b Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 22 Mar 2023 11:25:31 +0800 Subject: [PATCH 282/418] Correct exact_rows_before_limit in all senarios --- src/Interpreters/InterpreterSelectQuery.cpp | 4 +- src/Planner/Planner.cpp | 3 +- src/Processors/Formats/IOutputFormat.h | 2 +- src/Processors/IProcessor.h | 7 + src/Processors/LimitTransform.h | 2 +- .../Merges/AggregatingSortedTransform.h | 2 +- .../Merges/CollapsingSortedTransform.h | 2 +- .../FinishAggregatingInOrderTransform.h | 2 +- .../Merges/GraphiteRollupSortedTransform.h | 2 +- src/Processors/Merges/IMergingTransform.cpp | 25 ++- src/Processors/Merges/IMergingTransform.h | 13 +- .../Merges/MergingSortedTransform.cpp | 2 + .../Merges/MergingSortedTransform.h | 1 + .../Merges/ReplacingSortedTransform.h | 2 +- .../Merges/SummingSortedTransform.h | 2 +- .../Merges/VersionedCollapsingTransform.h | 2 +- src/Processors/OffsetTransform.h | 2 +- src/Processors/QueryPlan/SortingStep.cpp | 15 +- src/Processors/QueryPlan/SortingStep.h | 5 +- src/Processors/Sources/RemoteSource.cpp | 2 +- src/Processors/Sources/RemoteSource.h | 4 +- .../Transforms/ColumnGathererTransform.cpp | 2 +- .../Transforms/FinishSortingTransform.cpp | 1 + .../Transforms/MergeJoinTransform.cpp | 1 + .../Transforms/MergeSortingTransform.cpp | 1 + .../Transforms/PartialSortingTransform.h | 2 +- .../Transforms/TotalsHavingTransform.h | 2 + src/QueryPipeline/QueryPipeline.cpp | 106 ++++++++----- .../gtest_blocks_size_merging_streams.cpp | 4 +- src/Storages/MergeTree/MergeTask.cpp | 11 +- src/Storages/StorageS3Cluster.cpp | 2 + src/Storages/StorageS3Cluster.h | 1 + ...913_exact_rows_before_limit_full.reference | 144 ++++++++++++++++++ .../01913_exact_rows_before_limit_full.sql | 27 ++++ 34 files changed, 336 insertions(+), 69 deletions(-) create mode 100644 tests/queries/0_stateless/01913_exact_rows_before_limit_full.reference create mode 100644 tests/queries/0_stateless/01913_exact_rows_before_limit_full.sql diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 79b073e30f3..db46cb9f6b2 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2874,8 +2874,10 @@ void InterpreterSelectQuery::executeMergeSorted(QueryPlan & query_plan, const st SortDescription sort_description = getSortDescription(query, context); const UInt64 limit = getLimitForSorting(query, context); const auto max_block_size = context->getSettingsRef().max_block_size; + const auto exact_rows_before_limit = context->getSettingsRef().exact_rows_before_limit; - auto merging_sorted = std::make_unique(query_plan.getCurrentDataStream(), std::move(sort_description), max_block_size, limit); + auto merging_sorted = std::make_unique( + query_plan.getCurrentDataStream(), std::move(sort_description), max_block_size, limit, exact_rows_before_limit); merging_sorted->setStepDescription("Merge sorted streams " + description); query_plan.addStep(std::move(merging_sorted)); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 2242bf92e6b..b9fae6596a2 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -568,7 +568,8 @@ void addMergeSortingStep(QueryPlan & query_plan, auto merging_sorted = std::make_unique(query_plan.getCurrentDataStream(), sort_description, max_block_size, - query_analysis_result.partial_sorting_limit); + query_analysis_result.partial_sorting_limit, + settings.exact_rows_before_limit); merging_sorted->setStepDescription("Merge sorted streams " + description); query_plan.addStep(std::move(merging_sorted)); } diff --git a/src/Processors/Formats/IOutputFormat.h b/src/Processors/Formats/IOutputFormat.h index 02e91d5b28b..58700a978ff 100644 --- a/src/Processors/Formats/IOutputFormat.h +++ b/src/Processors/Formats/IOutputFormat.h @@ -39,7 +39,7 @@ public: virtual void setRowsBeforeLimit(size_t /*rows_before_limit*/) {} /// Counter to calculate rows_before_limit_at_least in processors pipeline. - void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) { rows_before_limit_counter.swap(counter); } + void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_counter.swap(counter); } /// Notify about progress. Method could be called from different threads. /// Passed value are delta, that must be summarized. diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index 6d17db69c9e..c9dd7d8d77d 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -21,6 +21,9 @@ class IQueryPlanStep; struct StorageLimits; using StorageLimitsList = std::list; +class RowsBeforeLimitCounter; +using RowsBeforeLimitCounterPtr = std::shared_ptr; + class IProcessor; using ProcessorPtr = std::shared_ptr; using Processors = std::vector; @@ -357,6 +360,10 @@ public: /// You should zero internal counters in the call, in order to make in idempotent. virtual std::optional getReadProgress() { return std::nullopt; } + /// Set rows_before_limit counter for current processor. + /// This counter is used to calculate the number of rows right before any filtration of LimitTransform. + virtual void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr /* counter */) {} + protected: virtual void onCancel() {} diff --git a/src/Processors/LimitTransform.h b/src/Processors/LimitTransform.h index 0d8c5f4ea47..791d4a44b24 100644 --- a/src/Processors/LimitTransform.h +++ b/src/Processors/LimitTransform.h @@ -66,7 +66,7 @@ public: InputPort & getInputPort() { return inputs.front(); } OutputPort & getOutputPort() { return outputs.front(); } - void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) { rows_before_limit_at_least.swap(counter); } + void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_at_least.swap(counter); } }; } diff --git a/src/Processors/Merges/AggregatingSortedTransform.h b/src/Processors/Merges/AggregatingSortedTransform.h index b0cdf4c8a3c..b7a88cca952 100644 --- a/src/Processors/Merges/AggregatingSortedTransform.h +++ b/src/Processors/Merges/AggregatingSortedTransform.h @@ -16,7 +16,7 @@ public: const Block & header, size_t num_inputs, SortDescription description_, size_t max_block_size) : IMergingTransform( - num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, + num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, /*always_read_till_end_=*/ false, header, num_inputs, std::move(description_), diff --git a/src/Processors/Merges/CollapsingSortedTransform.h b/src/Processors/Merges/CollapsingSortedTransform.h index a37e1c8402f..abe3eefb401 100644 --- a/src/Processors/Merges/CollapsingSortedTransform.h +++ b/src/Processors/Merges/CollapsingSortedTransform.h @@ -20,7 +20,7 @@ public: WriteBuffer * out_row_sources_buf_ = nullptr, bool use_average_block_sizes = false) : IMergingTransform( - num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, + num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, /*always_read_till_end_=*/ false, header, num_inputs, std::move(description_), diff --git a/src/Processors/Merges/FinishAggregatingInOrderTransform.h b/src/Processors/Merges/FinishAggregatingInOrderTransform.h index ecc88899e88..b82a103fee0 100644 --- a/src/Processors/Merges/FinishAggregatingInOrderTransform.h +++ b/src/Processors/Merges/FinishAggregatingInOrderTransform.h @@ -20,7 +20,7 @@ public: size_t max_block_size, size_t max_block_bytes) : IMergingTransform( - num_inputs, header, {}, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, + num_inputs, header, {}, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, /*always_read_till_end_=*/ false, header, num_inputs, params, diff --git a/src/Processors/Merges/GraphiteRollupSortedTransform.h b/src/Processors/Merges/GraphiteRollupSortedTransform.h index e6307c629ea..f3c391c77ce 100644 --- a/src/Processors/Merges/GraphiteRollupSortedTransform.h +++ b/src/Processors/Merges/GraphiteRollupSortedTransform.h @@ -15,7 +15,7 @@ public: SortDescription description_, size_t max_block_size, Graphite::Params params_, time_t time_of_merge_) : IMergingTransform( - num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, + num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, /*always_read_till_end_=*/ false, header, num_inputs, std::move(description_), diff --git a/src/Processors/Merges/IMergingTransform.cpp b/src/Processors/Merges/IMergingTransform.cpp index 4a6a1662f16..fbb47969b2f 100644 --- a/src/Processors/Merges/IMergingTransform.cpp +++ b/src/Processors/Merges/IMergingTransform.cpp @@ -14,10 +14,12 @@ IMergingTransformBase::IMergingTransformBase( const Block & input_header, const Block & output_header, bool have_all_inputs_, - UInt64 limit_hint_) + UInt64 limit_hint_, + bool always_read_till_end_) : IProcessor(InputPorts(num_inputs, input_header), {output_header}) , have_all_inputs(have_all_inputs_) , limit_hint(limit_hint_) + , always_read_till_end(always_read_till_end_) { } @@ -33,10 +35,12 @@ IMergingTransformBase::IMergingTransformBase( const Blocks & input_headers, const Block & output_header, bool have_all_inputs_, - UInt64 limit_hint_) + UInt64 limit_hint_, + bool always_read_till_end_) : IProcessor(createPorts(input_headers), {output_header}) , have_all_inputs(have_all_inputs_) , limit_hint(limit_hint_) + , always_read_till_end(always_read_till_end_) { } @@ -98,7 +102,7 @@ IProcessor::Status IMergingTransformBase::prepareInitializeInputs() /// (e.g. with optimized 'ORDER BY primary_key LIMIT n' and small 'n') /// we won't have to read any chunks anymore; auto chunk = input.pull(limit_hint != 0); - if (limit_hint && chunk.getNumRows() < limit_hint) + if ((limit_hint && chunk.getNumRows() < limit_hint) || always_read_till_end) input.setNeeded(); if (!chunk.hasRows()) @@ -164,6 +168,21 @@ IProcessor::Status IMergingTransformBase::prepare() if (is_port_full) return Status::PortFull; + if (always_read_till_end) + { + for (auto & input : inputs) + { + if (!input.isFinished()) + { + input.setNeeded(); + if (input.hasData()) + std::ignore = input.pull(); + + return Status::NeedData; + } + } + } + for (auto & input : inputs) input.close(); diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index 1d223179f8a..c218f622870 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -17,13 +17,15 @@ public: const Block & input_header, const Block & output_header, bool have_all_inputs_, - UInt64 limit_hint_); + UInt64 limit_hint_, + bool always_read_till_end_); IMergingTransformBase( const Blocks & input_headers, const Block & output_header, bool have_all_inputs_, - UInt64 limit_hint_); + UInt64 limit_hint_, + bool always_read_till_end_); OutputPort & getOutputPort() { return outputs.front(); } @@ -67,6 +69,7 @@ private: std::atomic have_all_inputs; bool is_initialized = false; UInt64 limit_hint = 0; + bool always_read_till_end = false; IProcessor::Status prepareInitializeInputs(); }; @@ -83,8 +86,9 @@ public: const Block & output_header, bool have_all_inputs_, UInt64 limit_hint_, + bool always_read_till_end_, Args && ... args) - : IMergingTransformBase(num_inputs, input_header, output_header, have_all_inputs_, limit_hint_) + : IMergingTransformBase(num_inputs, input_header, output_header, have_all_inputs_, limit_hint_, always_read_till_end_) , algorithm(std::forward(args) ...) { } @@ -95,9 +99,10 @@ public: const Block & output_header, bool have_all_inputs_, UInt64 limit_hint_, + bool always_read_till_end_, bool empty_chunk_on_finish_, Args && ... args) - : IMergingTransformBase(input_headers, output_header, have_all_inputs_, limit_hint_) + : IMergingTransformBase(input_headers, output_header, have_all_inputs_, limit_hint_, always_read_till_end_) , empty_chunk_on_finish(empty_chunk_on_finish_) , algorithm(std::forward(args) ...) { diff --git a/src/Processors/Merges/MergingSortedTransform.cpp b/src/Processors/Merges/MergingSortedTransform.cpp index 12cbeecff8d..2e5eda9b54b 100644 --- a/src/Processors/Merges/MergingSortedTransform.cpp +++ b/src/Processors/Merges/MergingSortedTransform.cpp @@ -14,6 +14,7 @@ MergingSortedTransform::MergingSortedTransform( size_t max_block_size, SortingQueueStrategy sorting_queue_strategy, UInt64 limit_, + bool always_read_till_end_, WriteBuffer * out_row_sources_buf_, bool quiet_, bool use_average_block_sizes, @@ -24,6 +25,7 @@ MergingSortedTransform::MergingSortedTransform( header, have_all_inputs_, limit_, + always_read_till_end_, header, num_inputs, description_, diff --git a/src/Processors/Merges/MergingSortedTransform.h b/src/Processors/Merges/MergingSortedTransform.h index 50586177c6d..3042550d5d5 100644 --- a/src/Processors/Merges/MergingSortedTransform.h +++ b/src/Processors/Merges/MergingSortedTransform.h @@ -18,6 +18,7 @@ public: size_t max_block_size, SortingQueueStrategy sorting_queue_strategy, UInt64 limit_ = 0, + bool always_read_till_end_ = false, WriteBuffer * out_row_sources_buf_ = nullptr, bool quiet_ = false, bool use_average_block_sizes = false, diff --git a/src/Processors/Merges/ReplacingSortedTransform.h b/src/Processors/Merges/ReplacingSortedTransform.h index 8284a2c3a26..8289f102cb7 100644 --- a/src/Processors/Merges/ReplacingSortedTransform.h +++ b/src/Processors/Merges/ReplacingSortedTransform.h @@ -20,7 +20,7 @@ public: bool use_average_block_sizes = false, bool cleanup = false) : IMergingTransform( - num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, + num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, /*always_read_till_end_=*/ false, header, num_inputs, std::move(description_), diff --git a/src/Processors/Merges/SummingSortedTransform.h b/src/Processors/Merges/SummingSortedTransform.h index 0530ac2e96b..204224ecf06 100644 --- a/src/Processors/Merges/SummingSortedTransform.h +++ b/src/Processors/Merges/SummingSortedTransform.h @@ -19,7 +19,7 @@ public: const Names & partition_key_columns, size_t max_block_size) : IMergingTransform( - num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, + num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, /*always_read_till_end_=*/ false, header, num_inputs, std::move(description_), diff --git a/src/Processors/Merges/VersionedCollapsingTransform.h b/src/Processors/Merges/VersionedCollapsingTransform.h index 5eced1cb58d..e7eb164f515 100644 --- a/src/Processors/Merges/VersionedCollapsingTransform.h +++ b/src/Processors/Merges/VersionedCollapsingTransform.h @@ -19,7 +19,7 @@ public: WriteBuffer * out_row_sources_buf_ = nullptr, bool use_average_block_sizes = false) : IMergingTransform( - num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, + num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, /*always_read_till_end_=*/ false, header, num_inputs, std::move(description_), diff --git a/src/Processors/OffsetTransform.h b/src/Processors/OffsetTransform.h index d24440d68ea..79a7d15fe0b 100644 --- a/src/Processors/OffsetTransform.h +++ b/src/Processors/OffsetTransform.h @@ -45,7 +45,7 @@ public: InputPort & getInputPort() { return inputs.front(); } OutputPort & getOutputPort() { return outputs.front(); } - void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) { rows_before_limit_at_least.swap(counter); } + void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_at_least.swap(counter); } }; } diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index 0ab8e091e05..db44da5a0fc 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -98,11 +98,13 @@ SortingStep::SortingStep( const DataStream & input_stream, SortDescription sort_description_, size_t max_block_size_, - UInt64 limit_) + UInt64 limit_, + bool always_read_till_end_) : ITransformingStep(input_stream, input_stream.header, getTraits(limit_)) , type(Type::MergingSorted) , result_description(std::move(sort_description_)) , limit(limit_) + , always_read_till_end(always_read_till_end_) , sort_settings(max_block_size_) { sort_settings.max_block_size = max_block_size_; @@ -175,7 +177,8 @@ void SortingStep::mergingSorted(QueryPipelineBuilder & pipeline, const SortDescr result_sort_desc, sort_settings.max_block_size, SortingQueueStrategy::Batch, - limit_); + limit_, + always_read_till_end); pipeline.addTransform(std::move(transform)); } @@ -262,7 +265,13 @@ void SortingStep::fullSort( if (pipeline.getNumStreams() > 1) { auto transform = std::make_shared( - pipeline.getHeader(), pipeline.getNumStreams(), result_sort_desc, sort_settings.max_block_size, SortingQueueStrategy::Batch, limit_); + pipeline.getHeader(), + pipeline.getNumStreams(), + result_sort_desc, + sort_settings.max_block_size, + SortingQueueStrategy::Batch, + limit_, + always_read_till_end); pipeline.addTransform(std::move(transform)); } diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h index eed1f26e110..79b4d94c2fd 100644 --- a/src/Processors/QueryPlan/SortingStep.h +++ b/src/Processors/QueryPlan/SortingStep.h @@ -53,7 +53,9 @@ public: const DataStream & input_stream, SortDescription sort_description_, size_t max_block_size_, - UInt64 limit_ = 0); + UInt64 limit_ = 0, + bool always_read_till_end_ = false + ); String getName() const override { return "Sorting"; } @@ -100,6 +102,7 @@ private: SortDescription prefix_description; const SortDescription result_description; UInt64 limit; + bool always_read_till_end; Settings sort_settings; diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 69964d569fa..e203c9e12b2 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -107,7 +107,7 @@ std::optional RemoteSource::tryGenerate() query_executor->setProfileInfoCallback([this](const ProfileInfo & info) { if (rows_before_limit && info.hasAppliedLimit()) - rows_before_limit->set(info.getRowsBeforeLimit()); + rows_before_limit->add(info.getRowsBeforeLimit()); }); query_executor->sendQuery(); diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index 8fe0114ab6f..40f1117e46e 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -3,7 +3,7 @@ #include #include #include -#include "Core/UUID.h" +#include #include namespace DB @@ -29,7 +29,7 @@ public: void connectToScheduler(InputPort & input_port); - void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) { rows_before_limit.swap(counter); } + void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit.swap(counter); } UUID getParallelReplicasGroupUUID(); diff --git a/src/Processors/Transforms/ColumnGathererTransform.cpp b/src/Processors/Transforms/ColumnGathererTransform.cpp index 2628bf7d6db..7c2b93faa91 100644 --- a/src/Processors/Transforms/ColumnGathererTransform.cpp +++ b/src/Processors/Transforms/ColumnGathererTransform.cpp @@ -126,7 +126,7 @@ ColumnGathererTransform::ColumnGathererTransform( ReadBuffer & row_sources_buf_, size_t block_preferred_size_) : IMergingTransform( - num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, + num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, /*always_read_till_end_=*/ false, num_inputs, row_sources_buf_, block_preferred_size_) , log(&Poco::Logger::get("ColumnGathererStream")) { diff --git a/src/Processors/Transforms/FinishSortingTransform.cpp b/src/Processors/Transforms/FinishSortingTransform.cpp index d8412eff588..05fddc35e15 100644 --- a/src/Processors/Transforms/FinishSortingTransform.cpp +++ b/src/Processors/Transforms/FinishSortingTransform.cpp @@ -109,6 +109,7 @@ void FinishSortingTransform::generate() generated_prefix = true; } + // TODO: Here we should also consider LIMIT optimization. generated_chunk = merge_sorter->read(); if (!generated_chunk) diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index cf5b4be4239..abeef0f9a47 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -844,6 +844,7 @@ MergeJoinTransform::MergeJoinTransform( output_header, /* have_all_inputs_= */ true, limit_hint_, + /* always_read_till_end_= */ false, /* empty_chunk_on_finish_= */ true, table_join, input_headers, max_block_size) , log(&Poco::Logger::get("MergeJoinTransform")) diff --git a/src/Processors/Transforms/MergeSortingTransform.cpp b/src/Processors/Transforms/MergeSortingTransform.cpp index efd9249066c..eebdd678a4b 100644 --- a/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/src/Processors/Transforms/MergeSortingTransform.cpp @@ -187,6 +187,7 @@ void MergeSortingTransform::consume(Chunk chunk) max_merged_block_size, SortingQueueStrategy::Batch, limit, + /*always_read_till_end_=*/ false, nullptr, quiet, use_average_block_sizes, diff --git a/src/Processors/Transforms/PartialSortingTransform.h b/src/Processors/Transforms/PartialSortingTransform.h index 6dab4497fc7..8f25c93037f 100644 --- a/src/Processors/Transforms/PartialSortingTransform.h +++ b/src/Processors/Transforms/PartialSortingTransform.h @@ -20,7 +20,7 @@ public: String getName() const override { return "PartialSortingTransform"; } - void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) { read_rows.swap(counter); } + void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { read_rows.swap(counter); } protected: void transform(Chunk & chunk) override; diff --git a/src/Processors/Transforms/TotalsHavingTransform.h b/src/Processors/Transforms/TotalsHavingTransform.h index 2567781771e..f252d683b9a 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.h +++ b/src/Processors/Transforms/TotalsHavingTransform.h @@ -42,6 +42,8 @@ public: Status prepare() override; void work() override; + bool hasFilter() const { return !filter_column_name.empty(); } + static Block transformHeader(Block block, const ActionsDAG * expression, const std::string & filter_column_name, bool remove_filter, bool final, const ColumnsMask & aggregates_mask); protected: diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index b7b18014f1f..f561f7a7396 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include @@ -129,50 +130,77 @@ static void checkCompleted(Processors & processors) static void initRowsBeforeLimit(IOutputFormat * output_format) { RowsBeforeLimitCounterPtr rows_before_limit_at_least; - - /// TODO: add setRowsBeforeLimitCounter as virtual method to IProcessor. - std::vector limits; - std::vector remote_sources; - + std::vector processors; + std::map limit_candidates; std::unordered_set visited; + bool has_limit = false; struct QueuedEntry { IProcessor * processor; - bool visited_limit; + IProcessor * limit_processor; }; std::queue queue; - queue.push({ output_format, false }); + queue.push({ output_format, nullptr }); visited.emplace(output_format); while (!queue.empty()) { auto * processor = queue.front().processor; - auto visited_limit = queue.front().visited_limit; + auto * limit_processor = queue.front().limit_processor; queue.pop(); - if (!visited_limit) + /// Set counter based on the following cases: + /// 1. Remote: Set counter on Remote + /// 2. Limit ... PartialSorting: Set counter on PartialSorting + /// 3. Limit ... TotalsHaving(with filter) ... Remote: Set counter on Limit + /// 4. Limit ... Remote: Set counter on Remote + /// 5. Limit ... : Set counter on Limit + + /// Case 1. + if (typeid_cast(processor) && !limit_processor) { - if (auto * limit = typeid_cast(processor)) + processors.emplace_back(processor); + continue; + } + + if (typeid_cast(processor)) + { + has_limit = true; + + /// Ignore child limits + if (limit_processor) + continue; + + limit_processor = processor; + limit_candidates.emplace(limit_processor, true); + } + else if (limit_processor) + { + /// Case 2. + if (typeid_cast(processor)) { - visited_limit = true; - limits.emplace_back(limit); + processors.emplace_back(processor); + limit_candidates[limit_processor] = false; + continue; } - if (auto * source = typeid_cast(processor)) - remote_sources.emplace_back(source); - } - else if (auto * sorting = typeid_cast(processor)) - { - if (!rows_before_limit_at_least) - rows_before_limit_at_least = std::make_shared(); + /// Case 3. + if (auto * having = typeid_cast(processor)) + { + if (having->hasFilter()) + continue; + } - sorting->setRowsBeforeLimitCounter(rows_before_limit_at_least); - - /// Don't go to children. Take rows_before_limit from last PartialSortingTransform. - continue; + /// Case 4. + if (typeid_cast(processor)) + { + processors.emplace_back(processor); + limit_candidates[limit_processor] = false; + continue; + } } /// Skip totals and extremes port for output format. @@ -180,7 +208,7 @@ static void initRowsBeforeLimit(IOutputFormat * output_format) { auto * child_processor = &format->getPort(IOutputFormat::PortKind::Main).getOutputPort().getProcessor(); if (visited.emplace(child_processor).second) - queue.push({ child_processor, visited_limit }); + queue.push({ child_processor, limit_processor }); continue; } @@ -189,28 +217,30 @@ static void initRowsBeforeLimit(IOutputFormat * output_format) { auto * child_processor = &child_port.getOutputPort().getProcessor(); if (visited.emplace(child_processor).second) - queue.push({ child_processor, visited_limit }); + queue.push({ child_processor, limit_processor }); } } - if (!rows_before_limit_at_least && (!limits.empty() || !remote_sources.empty())) + /// Case 5. + for (auto && [limit, valid] : limit_candidates) { - rows_before_limit_at_least = std::make_shared(); - - for (auto & limit : limits) - limit->setRowsBeforeLimitCounter(rows_before_limit_at_least); - - for (auto & source : remote_sources) - source->setRowsBeforeLimitCounter(rows_before_limit_at_least); + if (valid) + processors.push_back(limit); } - /// If there is a limit, then enable rows_before_limit_at_least - /// It is needed when zero rows is read, but we still want rows_before_limit_at_least in result. - if (!limits.empty()) - rows_before_limit_at_least->add(0); + if (!processors.empty()) + { + rows_before_limit_at_least = std::make_shared(); + for (auto & processor : processors) + processor->setRowsBeforeLimitCounter(rows_before_limit_at_least); + + /// If there is a limit, then enable rows_before_limit_at_least + /// It is needed when zero rows is read, but we still want rows_before_limit_at_least in result. + if (has_limit) + rows_before_limit_at_least->add(0); - if (rows_before_limit_at_least) output_format->setRowsBeforeLimitCounter(rows_before_limit_at_least); + } } diff --git a/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp b/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp index 2fa5873544f..d968dae3ff8 100644 --- a/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp +++ b/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp @@ -83,7 +83,7 @@ TEST(MergingSortedTest, SimpleBlockSizeTest) EXPECT_EQ(pipe.numOutputPorts(), 3); auto transform = std::make_shared(pipe.getHeader(), pipe.numOutputPorts(), sort_description, - DEFAULT_MERGE_BLOCK_SIZE, SortingQueueStrategy::Batch, 0, nullptr, false, true); + DEFAULT_MERGE_BLOCK_SIZE, SortingQueueStrategy::Batch, 0, false, nullptr, false, true); pipe.addTransform(std::move(transform)); @@ -125,7 +125,7 @@ TEST(MergingSortedTest, MoreInterestingBlockSizes) EXPECT_EQ(pipe.numOutputPorts(), 3); auto transform = std::make_shared(pipe.getHeader(), pipe.numOutputPorts(), sort_description, - DEFAULT_MERGE_BLOCK_SIZE, SortingQueueStrategy::Batch, 0, nullptr, false, true); + DEFAULT_MERGE_BLOCK_SIZE, SortingQueueStrategy::Batch, 0, false, nullptr, false, true); pipe.addTransform(std::move(transform)); diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 9d9d8420e2c..d1dfa96b87c 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -929,7 +929,16 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() { case MergeTreeData::MergingParams::Ordinary: merged_transform = std::make_shared( - header, pipes.size(), sort_description, merge_block_size, SortingQueueStrategy::Default, 0, ctx->rows_sources_write_buf.get(), true, ctx->blocks_are_granules_size); + header, + pipes.size(), + sort_description, + merge_block_size, + SortingQueueStrategy::Default, + /* limit_= */0, + /* always_read_till_end_= */false, + ctx->rows_sources_write_buf.get(), + true, + ctx->blocks_are_granules_size); break; case MergeTreeData::MergingParams::Collapsing: diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 51a4a311e93..5cefc065f21 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -49,6 +49,7 @@ StorageS3Cluster::StorageS3Cluster( ContextPtr context_, bool structure_argument_was_provided_) : IStorageCluster(table_id_) + , log(&Poco::Logger::get("StorageS3Cluster (" + table_id_.table_name + ")")) , s3_configuration{configuration_} , cluster_name(configuration_.cluster_name) , format_name(configuration_.format) @@ -156,6 +157,7 @@ Pipe StorageS3Cluster::read( processed_stage, extension); + remote_query_executor->setLogger(log); pipes.emplace_back(std::make_shared(remote_query_executor, add_agg_info, false)); } } diff --git a/src/Storages/StorageS3Cluster.h b/src/Storages/StorageS3Cluster.h index e55382c66b2..98a0bde260e 100644 --- a/src/Storages/StorageS3Cluster.h +++ b/src/Storages/StorageS3Cluster.h @@ -48,6 +48,7 @@ public: ClusterPtr getCluster(ContextPtr context) const override; private: + Poco::Logger * log; StorageS3::Configuration s3_configuration; String cluster_name; String format_name; diff --git a/tests/queries/0_stateless/01913_exact_rows_before_limit_full.reference b/tests/queries/0_stateless/01913_exact_rows_before_limit_full.reference new file mode 100644 index 00000000000..01f5c057be2 --- /dev/null +++ b/tests/queries/0_stateless/01913_exact_rows_before_limit_full.reference @@ -0,0 +1,144 @@ +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0] + ], + + "rows": 1, + + "rows_before_limit_at_least": 10000 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0] + ], + + "rows": 1, + + "rows_before_limit_at_least": 10 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [12] + ], + + "rows": 1, + + "rows_before_limit_at_least": 3 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0] + ], + + "rows": 1, + + "rows_before_limit_at_least": 20 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0] + ], + + "rows": 1, + + "rows_before_limit_at_least": 60 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0] + ], + + "rows": 1, + + "rows_before_limit_at_least": 40 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0] + ], + + "rows": 1, + + "rows_before_limit_at_least": 60 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0] + ], + + "rows": 1, + + "rows_before_limit_at_least": 40 +} diff --git a/tests/queries/0_stateless/01913_exact_rows_before_limit_full.sql b/tests/queries/0_stateless/01913_exact_rows_before_limit_full.sql new file mode 100644 index 00000000000..ec0d3d61bd4 --- /dev/null +++ b/tests/queries/0_stateless/01913_exact_rows_before_limit_full.sql @@ -0,0 +1,27 @@ +-- Tags: no-parallel, no-random-merge-tree-settings + +drop table if exists test; + +create table test (i int) engine MergeTree order by tuple(); + +insert into test select arrayJoin(range(10000)); + +set exact_rows_before_limit = 1, output_format_write_statistics = 0, max_block_size = 100; + +select * from test limit 1 FORMAT JSONCompact; + +select * from test where i < 10 group by i limit 1 FORMAT JSONCompact; + +select * from test group by i having i in (10, 11, 12) limit 1 FORMAT JSONCompact; + +select * from test where i < 20 order by i limit 1 FORMAT JSONCompact; + +set prefer_localhost_replica = 0; +select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 30 limit 1 FORMAT JSONCompact; +select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 20 order by i limit 1 FORMAT JSONCompact; + +set prefer_localhost_replica = 1; +select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 30 limit 1 FORMAT JSONCompact; +select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 20 order by i limit 1 FORMAT JSONCompact; + +drop table if exists test; From d7d3722b76123ec906a07ca60f28ee5833f977ee Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 22 Mar 2023 15:41:54 +0000 Subject: [PATCH 283/418] Revert analyzer till later. --- .../optimizeUseAggregateProjection.cpp | 118 ++++-------------- src/Storages/MergeTree/MergeTreeData.cpp | 5 +- src/Storages/MergeTree/MergeTreeData.h | 3 +- 3 files changed, 27 insertions(+), 99 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 0eff4a3c3e8..08d8ecb95b3 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -45,53 +45,11 @@ struct AggregateProjectionInfo Names keys; AggregateDescriptions aggregates; - /// This field is needed for getSampleBlock only. - size_t num_virtual_keys = 0; - /// A context copy from interpreter which was used for analysis. /// Just in case it is used by some function. ContextPtr context; - - /// This is a sample block which we expect before aggregation. - /// Now, it is needed only for minmax_count projection. - Block getSampleBlock() - { - auto index = buildDAGIndex(*before_aggregation); - Block res; - size_t num_keys = keys.size() - num_virtual_keys; - for (size_t i = 0; i < num_keys; ++i) - { - const auto & key = keys[i]; - const ActionsDAG::Node & node = *index.at(key); - res.insert({node.result_type->createColumn(), node.result_type, node.result_name}); - } - - for (const auto & aggregate : aggregates) - { - size_t arguments_size = aggregate.argument_names.size(); - DataTypes argument_types(arguments_size); - for (size_t j = 0; j < arguments_size; ++j) - argument_types[j] = index.at(aggregate.argument_names[j])->result_type; - - auto type = std::make_shared(aggregate.function, argument_types, aggregate.parameters); - - res.insert({ type->createColumn(), type, aggregate.column_name }); - } - - return res; - } }; -/// This is a projection-specific. -/// We can expect specific query tree structure for projection query. -static void replaceStorageInQueryTree(QueryTreeNodePtr & query_tree, const ContextPtr & context, const StoragePtr & storage) -{ - auto replacement = std::make_shared(storage, context); - auto & query_node = query_tree->as(); - auto & join_tree = query_node.getJoinTree(); - query_tree = query_tree->cloneAndReplace(join_tree, std::move(replacement)); -} - /// Get required info from aggregate projection. /// Ideally, this should be pre-calculated and stored inside ProjectionDescription. static AggregateProjectionInfo getAggregatingProjectionInfo( @@ -100,50 +58,34 @@ static AggregateProjectionInfo getAggregatingProjectionInfo( const StorageMetadataPtr & metadata_snapshot, const Block & key_virtual_columns) { - // 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. + InterpreterSelectQuery interpreter( + projection.query_ast, + context, + Pipe(std::make_shared(metadata_snapshot->getSampleBlock())), + SelectQueryOptions{QueryProcessingStage::WithMergeableState}); - /// This is a query tree from projection query. - /// This query does not contain source table, so it is not valid. - auto query_tree = buildQueryTree(projection.query_ast, context); - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "QueryTree : {}", query_tree->dumpTree()); + const auto & analysis_result = interpreter.getAnalysisResult(); + const auto & query_analyzer = interpreter.getQueryAnalyzer(); - /// Replace a storage so that query tree become valid and possilbe to analyze. - auto storage = std::make_shared(StorageID{"dummy", "dummy"}, metadata_snapshot->getColumns()); - replaceStorageInQueryTree(query_tree, context, storage); - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "QueryTree : {}", query_tree->dumpTree()); - - /// Aggregated copy-paste from InterpreterSelectQueryAnalyzer. - QueryTreePassManager query_tree_pass_manager(context); - addQueryTreePasses(query_tree_pass_manager); - query_tree_pass_manager.run(query_tree); - - auto columns = metadata_snapshot->getSampleBlock().getColumnsWithTypeAndName(); - auto mutable_context = Context::createCopy(context); - auto global_planner_context = std::make_shared(); - auto planner_context = std::make_shared(mutable_context, std::move(global_planner_context)); - PlannerQueryProcessingInfo info(QueryProcessingStage::FetchColumns, QueryProcessingStage::WithMergeableState); - - auto analysis_result = buildExpressionAnalysisResult(query_tree, columns, planner_context, info); - - const auto & aggregation = analysis_result.getAggregation(); - - AggregateProjectionInfo proj_info; - proj_info.context = planner_context->getQueryContext(); - proj_info.before_aggregation = aggregation.before_aggregation_actions; - proj_info.keys = aggregation.aggregation_keys; - proj_info.aggregates = aggregation.aggregate_descriptions; - proj_info.num_virtual_keys = key_virtual_columns.columns(); + AggregateProjectionInfo info; + info.context = interpreter.getContext(); + info.before_aggregation = analysis_result.before_aggregation; + info.keys = query_analyzer->aggregationKeys().getNames(); + 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 = &proj_info.before_aggregation->addInput(virt_column); - proj_info.before_aggregation->getOutputs().push_back(input); - proj_info.keys.push_back(virt_column.name); + const auto * input = &info.before_aggregation->addInput(virt_column); + info.before_aggregation->getOutputs().push_back(input); + info.keys.push_back(virt_column.name); } - return proj_info; + return info; } static bool hasNullableOrMissingColumn(const DAGIndex & index, const Names & names) @@ -488,21 +430,10 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( // 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()); + // 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; - /// Why do we need this sample block? - /// Currently, we are using InterpreterSelectQuery in order to analyze minmax_count projection. - /// This is gives different columns names after aggregation than InterpreterSelectAnalyzerQuery. - /// - /// It would be good to use InterpreterSelectAnalyzerQuery to analyze projection as well. - /// Now I can't do it cause it will breake old projection analysis which should be kept for some time. - /// - /// So, here we re-calculate the sample block the way it should be in a new analyzer. - /// Hopefully the column order is the same. - auto sample_block = candidate.info.getSampleBlock(); - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection sample block {}", sample_block.dumpStructure()); auto block = reading.getMergeTreeData().getMinMaxCountProjectionBlock( metadata, @@ -512,8 +443,7 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( parts, minmax_projection_normal_parts, max_added_blocks.get(), - context, - &sample_block); + context); // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection sample block 2 {}", block.dumpStructure()); @@ -534,12 +464,12 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( candidates.real.reserve(agg_projections.size()); for (const auto * projection : agg_projections) { - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try projection {}", projection->name); + 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()); + 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()); + 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)); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d39eca5508d..a290588166c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6193,15 +6193,14 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( const DataPartsVector & parts, DataPartsVector & normal_parts, const PartitionIdToMaxBlock * max_block_numbers_to_read, - ContextPtr query_context, - Block * sample_block) const + ContextPtr query_context) const { if (!metadata_snapshot->minmax_count_projection) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find the definition of minmax_count projection but it's used in current query. " "It's a bug"); - auto block = sample_block ? *sample_block : metadata_snapshot->minmax_count_projection->sample_block.cloneEmpty(); + auto block = metadata_snapshot->minmax_count_projection->sample_block.cloneEmpty(); bool need_primary_key_max_column = false; const auto & primary_key_max_column_name = metadata_snapshot->minmax_count_projection->primary_key_max_column_name; NameSet required_columns_set(required_columns.begin(), required_columns.end()); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 0f7479c7e7b..4a1aafe20b6 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -398,8 +398,7 @@ public: const DataPartsVector & parts, DataPartsVector & normal_parts, const PartitionIdToMaxBlock * max_block_numbers_to_read, - ContextPtr query_context, - Block * sample_block = nullptr) const; + ContextPtr query_context) const; std::optional getQueryProcessingStageWithAggregateProjection( ContextPtr query_context, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info) const; From 04a1e41233ae33b91b6b671bf1f2b2783299b9e5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 22 Mar 2023 16:01:28 +0000 Subject: [PATCH 284/418] Review fixes. --- src/Processors/QueryPlan/AggregatingStep.cpp | 4 ++-- .../optimizeUseAggregateProjection.cpp | 20 +++++++++---------- .../optimizeUseNormalProjection.cpp | 8 ++++---- .../Optimizations/projectionsCommon.cpp | 2 +- .../Optimizations/projectionsCommon.h | 2 +- 5 files changed, 18 insertions(+), 18 deletions(-) diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index ddf08d8a124..0d443f0826b 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -527,11 +527,11 @@ void AggregatingStep::requestOnlyMergeForAggregateProjection(const DataStream & if (!canUseProjection()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot aggregate from projection"); - auto cur_header = getOutputStream().header; + auto output_header = getOutputStream().header; input_streams.front() = input_stream; params.only_merge = true; updateOutputStream(); - assertBlocksHaveEqualStructure(cur_header, getOutputStream().header, "AggregatingStep"); + assertBlocksHaveEqualStructure(output_header, getOutputStream().header, "AggregatingStep"); } std::unique_ptr AggregatingStep::convertToAggregatingProjection(const DataStream & input_stream) const diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 08d8ecb95b3..ad34e9d4a09 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -464,12 +464,12 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( candidates.real.reserve(agg_projections.size()); for (const auto * projection : agg_projections) { - LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try projection {}", projection->name); + // 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()); + // 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()); + // 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)); @@ -557,7 +557,7 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & return false; QueryPlanStepPtr projection_reading; - bool has_nornal_parts; + bool has_ordinary_parts; /// Add reading from projection step. if (candidates.minmax_projection) @@ -568,8 +568,8 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & 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->normal_parts.empty(); - if (has_nornal_parts) + has_ordinary_parts = !candidates.minmax_projection->normal_parts.empty(); + if (has_ordinary_parts) reading->resetParts(std::move(candidates.minmax_projection->normal_parts)); } else @@ -601,9 +601,9 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & 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)); + has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr; + if (has_ordinary_parts) + reading->setAnalyzedResult(std::move(best_candidate->merge_tree_ordinary_select_result_ptr)); } // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}", @@ -629,7 +629,7 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & expr_or_filter_node.children.push_back(&projection_reading_node); - if (!has_nornal_parts) + if (!has_ordinary_parts) { /// All parts are taken from projection aggregating->requestOnlyMergeForAggregateProjection(expr_or_filter_node.step->getOutputStream()); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index b46205260fe..eed3707fe9c 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -197,9 +197,9 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) 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)); + bool has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr; + if (has_ordinary_parts) + reading->setAnalyzedResult(std::move(best_candidate->merge_tree_ordinary_select_result_ptr)); // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}", // projection_reading->getOutputStream().header.dumpStructure()); @@ -230,7 +230,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) next_node = &expr_or_filter_node; } - if (!has_nornal_parts) + if (!has_ordinary_parts) { /// All parts are taken from projection iter->node->children[iter->next_child - 1] = next_node; diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index 1cf72eec5e3..908d599bb06 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -232,7 +232,7 @@ bool analyzeProjectionCandidate( 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); + candidate.merge_tree_ordinary_select_result_ptr = std::move(normal_result_ptr); } } diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.h b/src/Processors/QueryPlan/Optimizations/projectionsCommon.h index 7233a7de640..1e9ab67c8fe 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.h +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.h @@ -62,7 +62,7 @@ struct ProjectionCandidate /// 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; + MergeTreeDataSelectAnalysisResultPtr merge_tree_ordinary_select_result_ptr; }; /// This function fills ProjectionCandidate structure for specified projection. From f45dac99c242614fa8d357c672e552a5bd96b5dc Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 20 Mar 2023 11:29:27 +0100 Subject: [PATCH 285/418] Revert "Revert "Analyzer planner fixes before enable by default"" --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 9 +- 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 | 169 +++++- src/Planner/CollectTableExpressionData.h | 9 +- src/Planner/Planner.cpp | 71 +-- src/Planner/PlannerActionsVisitor.cpp | 529 ++++++++++-------- src/Planner/PlannerActionsVisitor.h | 29 +- src/Planner/PlannerJoinTree.cpp | 262 +++++++-- src/Planner/TableExpressionData.h | 28 + src/Planner/Utils.cpp | 4 +- .../QueryPlan/Optimizations/Optimizations.h | 1 + .../Optimizations/optimizePrewhere.cpp | 371 ++++++++++++ .../Optimizations/optimizeReadInOrder.cpp | 2 +- .../QueryPlan/Optimizations/optimizeTree.cpp | 1 + .../QueryPlan/ReadFromMergeTree.cpp | 11 + src/Processors/QueryPlan/ReadFromMergeTree.h | 8 +- src/Storages/LiveView/StorageLiveView.cpp | 5 - .../MergeTreeBaseSelectProcessor.cpp | 43 +- .../MergeTreeSplitPrewhereIntoReadSteps.cpp | 8 +- .../MergeTree/MergeTreeWhereOptimizer.cpp | 428 +++++++------- .../MergeTree/MergeTreeWhereOptimizer.h | 78 ++- src/Storages/MergeTree/RPNBuilder.cpp | 55 +- src/Storages/MergeTree/RPNBuilder.h | 6 + .../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 + 34 files changed, 1687 insertions(+), 656 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 c1845f0ff9c..76c780f25eb 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; } @@ -6856,13 +6855,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 bbd8a5744fc..ac19d01d0e9 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..4b44374e8eb 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,60 @@ 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; + + 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()) + { + 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..b79fe9bcd46 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -79,26 +79,14 @@ 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. * - * 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 trivial count using partition predicates. + * TODO: Support trivial count for table functions. + * TODO: Support indexes for IN function. */ namespace @@ -135,37 +123,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 +1097,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 +1130,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 +1160,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..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 @@ -14,6 +20,7 @@ #include #include +#include #include #include #include @@ -22,13 +29,15 @@ #include #include #include -#include +#include +#include #include #include #include #include #include +#include #include #include #include @@ -44,10 +53,9 @@ #include #include #include +#include +#include -#include -#include -#include namespace DB { @@ -62,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 @@ -163,19 +172,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 +218,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 +317,115 @@ 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()); +} + +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, @@ -428,9 +547,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 +569,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,9 +583,67 @@ 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); + + 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(); @@ -493,7 +671,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..557b76293ea --- /dev/null +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -0,0 +1,371 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace +{ + +void matchDAGOutputNodesOrderWithHeader(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()) + { + 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.has_value()) + 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 + * enough to produce required filter output columns. + * + * Example: SELECT (a AND b) AS cond FROM test_table WHERE cond AND c; + * 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 step to calculate `and(a, b)` expression 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 = nullptr; + bool remove_filter_node = false; + + if (filter_actions) + { + filter_actions->projectInput(false); + + /// Match dag output nodes with old filter step 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()); + 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; + matchDAGOutputNodesOrderWithHeader(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 753bb070c47..668978d2605 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1397,6 +1397,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/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; diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 0ad91d84d29..a41b697f9e6 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)->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()); + } } } 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 c7afe9319f9..6e212cc1600 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,11 +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) - , log_queries_cut_to_length(context->getSettingsRef().log_queries_cut_to_length) { for (const auto & name : queried_columns) { @@ -52,180 +44,214 @@ 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)); + + UInt64 log_queries_cut_to_length = context->getSettingsRef().log_queries_cut_to_length; + LOG_DEBUG(log, "MergeTreeWhereOptimizer: condition \"{}\" moved to PREWHERE", select.prewhere()->formatForLogging(log_queries_cut_to_length)); } -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 = getColumnsSize(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) + && columnsSupportPrewhere(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(); @@ -234,18 +260,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; @@ -256,12 +293,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; @@ -278,7 +315,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) @@ -292,7 +329,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) @@ -304,129 +341,130 @@ 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()->formatForLogging(log_queries_cut_to_length)); + OptimizeResult result = {std::move(where_conditions), std::move(prewhere_conditions)}; + return result; } -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; } -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 +bool MergeTreeWhereOptimizer::isSubsetOfTableColumns(const NameSet & columns) 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) - if (!table_columns.contains(identifier)) + for (const auto & column : columns) + if (!table_columns.contains(column)) return false; 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)) || + (table_columns.contains(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 ef87018f93e..18555a72db1 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,27 +86,46 @@ 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; + UInt64 getColumnsSize(const NameSet & columns) const; - bool isExpressionOverSortingKey(const ASTPtr & ast) const; + bool columnsSupportPrewhere(const NameSet & columns) const; + + bool isExpressionOverSortingKey(const RPNBuilderTreeNode & node) const; bool isSortingKey(const String & column_name) const; 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. @@ -99,23 +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; - UInt64 log_queries_cut_to_length = 0; }; 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; 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 575c4263a3dea3865b6e87bbe54d3f64ca6faa73 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 22 Mar 2023 17:47:25 +0100 Subject: [PATCH 286/418] address comments --- src/Common/OptimizedRegularExpression.cpp | 70 +++++++++++------------ src/Common/OptimizedRegularExpression.h | 3 + src/Common/tests/gtest_optimize_re.cpp | 2 +- src/Dictionaries/RegExpTreeDictionary.cpp | 14 ++--- 4 files changed, 45 insertions(+), 44 deletions(-) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index a8aadd2fecd..77acfdda935 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -21,7 +22,7 @@ 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. + bool suffix; /// this literal string is the suffix of the whole string. void clear() { literal.clear(); @@ -35,12 +36,10 @@ using Literals = std::vector; size_t shortest_alter_length(const Literals & literals) { if (literals.empty()) return 0; - size_t shortest = ~(0); + size_t shortest = std::numeric_limits::max(); for (const auto & lit : literals) - { if (shortest > lit.literal.size()) shortest = lit.literal.size(); - } return shortest; } @@ -49,7 +48,7 @@ const char * analyzeImpl( const char * pos, Literal & required_substring, bool & is_trivial, - Literals & global_alters) + Literals & global_alternatives) { /** The expression is trivial if all the metacharacters in it are escaped. * The non-alternative string is @@ -61,10 +60,9 @@ const char * analyzeImpl( */ const char * begin = pos; const char * end = regexp.data() + regexp.size(); - bool first_call = begin == regexp.data(); + bool is_first_call = begin == regexp.data(); int depth = 0; is_trivial = true; - ///required_substring_is_prefix = false; required_substring.clear(); bool has_alternative_on_depth_0 = false; bool has_case_insensitive_flag = false; @@ -76,26 +74,26 @@ const char * analyzeImpl( Substrings trivial_substrings(1); Substring * last_substring = &trivial_substrings.back(); - Literals cur_alters; + Literals cur_alternatives; - auto finish_cur_alters = [&]() + auto finish_cur_alternatives = [&]() { - if (cur_alters.empty()) + if (cur_alternatives.empty()) return; - if (global_alters.empty()) + if (global_alternatives.empty()) { - global_alters = cur_alters; - cur_alters.clear(); + global_alternatives = cur_alternatives; + cur_alternatives.clear(); return; } /// that means current alternatives have better quality. - if (shortest_alter_length(global_alters) < shortest_alter_length(cur_alters)) + if (shortest_alter_length(global_alternatives) < shortest_alter_length(cur_alternatives)) { - global_alters.clear(); - global_alters = cur_alters; + global_alternatives.clear(); + global_alternatives = cur_alternatives; } - cur_alters.clear(); + cur_alternatives.clear(); }; auto finish_non_trivial_char = [&](bool create_new_substr = true) @@ -103,7 +101,7 @@ const char * analyzeImpl( if (depth != 0) return; - for (auto & alter : cur_alters) + for (auto & alter : cur_alternatives) { if (alter.suffix) { @@ -111,7 +109,7 @@ const char * analyzeImpl( } } - finish_cur_alters(); + finish_cur_alternatives(); if (!last_substring->first.empty() && create_new_substr) { @@ -121,9 +119,9 @@ const char * analyzeImpl( }; /// Resolve the string or alters in a group (xxxxx) - auto finish_group = [&](Literal & group_required_string, Literals & group_alters) + auto finish_group = [&](Literal & group_required_string, Literals & group_alternatives) { - for (auto & alter : group_alters) + for (auto & alter : group_alternatives) { if (alter.prefix) { @@ -146,8 +144,8 @@ const char * analyzeImpl( } /// assign group alters to current alters. - finish_cur_alters(); - cur_alters = std::move(group_alters); + finish_cur_alternatives(); + cur_alternatives = std::move(group_alternatives); }; bool in_curly_braces = false; @@ -356,7 +354,7 @@ finish: } } - if (max_length >= MIN_LENGTH_FOR_STRSTR || (!first_call && max_length > 0)) + if (max_length >= MIN_LENGTH_FOR_STRSTR || (!is_first_call && max_length > 0)) { required_substring.literal = candidate_it->first; required_substring.prefix = candidate_it->second == 0; @@ -375,8 +373,8 @@ finish: if (has_alternative_on_depth_0) { /// 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}; + if (shortest_alter_length(global_alternatives) < required_substring.literal.size()) + global_alternatives = {required_substring}; Literals next_alternatives; /// this two vals are useless, xxx|xxx cannot be trivial nor prefix. bool next_is_trivial = true; @@ -384,11 +382,11 @@ finish: /// For xxx|xxx|xxx, we only conbine the alternatives and return a empty required_substring. if (next_alternatives.empty() || shortest_alter_length(next_alternatives) < required_substring.literal.size()) { - global_alters.push_back(required_substring); + global_alternatives.push_back(required_substring); } else { - global_alters.insert(global_alters.end(), next_alternatives.begin(), next_alternatives.end()); + global_alternatives.insert(global_alternatives.end(), next_alternatives.begin(), next_alternatives.end()); } required_substring.clear(); } @@ -412,20 +410,20 @@ void OptimizedRegularExpressionImpl::analyze( 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) + Literals alternative_literals; + Literal required_literal; + analyzeImpl(regexp_, regexp_.data(), required_literal, is_trivial, alternative_literals); + required_substring = std::move(required_literal.literal); + required_substring_is_prefix = required_literal.prefix; + for (auto & lit : alternative_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_, required_substring, is_trivial, required_substring_is_prefix, alternatives); + std::vector alternativesDummy; /// this vector extracts patterns a,b,c from pattern (a|b|c). for now it's not used. + analyze(regexp_, required_substring, is_trivial, required_substring_is_prefix, alternativesDummy); /// Just three following options are supported diff --git a/src/Common/OptimizedRegularExpression.h b/src/Common/OptimizedRegularExpression.h index 566bedc5549..f6b59f0a465 100644 --- a/src/Common/OptimizedRegularExpression.h +++ b/src/Common/OptimizedRegularExpression.h @@ -95,12 +95,15 @@ public: out_required_substring_is_prefix = required_substring_is_prefix; } + /// analyze function will extract the longest string literal or multiple alternative string literals from regexp for pre-checking if + /// a string contains the string literal(s). If not, we can tell this string can never match the regexp. static void analyze( std::string_view regexp_, 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 088993cfa84..556700f1fcc 100644 --- a/src/Common/tests/gtest_optimize_re.cpp +++ b/src/Common/tests/gtest_optimize_re.cpp @@ -40,7 +40,7 @@ TEST(OptimizeRE, analyze) 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(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"}); } diff --git a/src/Dictionaries/RegExpTreeDictionary.cpp b/src/Dictionaries/RegExpTreeDictionary.cpp index 5618da3505b..c072ba78d46 100644 --- a/src/Dictionaries/RegExpTreeDictionary.cpp +++ b/src/Dictionaries/RegExpTreeDictionary.cpp @@ -242,9 +242,9 @@ void RegExpTreeDictionary::initRegexNodes(Block & block) } else if (!alternatives.empty()) { - for (auto & alter : alternatives) + for (auto & alternative : alternatives) { - simple_regexps.push_back(alter); + simple_regexps.push_back(alternative); regexp_ids.push_back(id); } } @@ -311,12 +311,12 @@ void RegExpTreeDictionary::loadData() #if USE_VECTORSCAN std::vector patterns; std::vector flags; - std::vector lens; + std::vector lengths; - for (const std::string & ref : simple_regexps) + for (const std::string & simple_regexp : simple_regexps) { - patterns.push_back(ref.data()); - lens.push_back(ref.size()); + patterns.push_back(simple_regexp.data()); + lengths.push_back(simple_regexp.size()); flags.push_back(HS_FLAG_SINGLEMATCH); } @@ -328,7 +328,7 @@ void RegExpTreeDictionary::loadData() 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); + hs_error_t err = hs_compile_lit_multi(patterns.data(), flags.data(), ids.get(), lengths.data(), static_cast(patterns.size()), HS_MODE_BLOCK, nullptr, &db, &compile_error); origin_db = (db); if (err != HS_SUCCESS) { From 02de4ad6dfaecff1ba722a449dd3ca4db058bfe1 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 22 Mar 2023 17:50:19 +0100 Subject: [PATCH 287/418] address comments --- src/Common/OptimizedRegularExpression.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index 77acfdda935..68f5b86877e 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -33,7 +33,7 @@ struct Literal using Literals = std::vector; -size_t shortest_alter_length(const Literals & literals) +size_t shortest_literal_length(const Literals & literals) { if (literals.empty()) return 0; size_t shortest = std::numeric_limits::max(); @@ -88,7 +88,7 @@ const char * analyzeImpl( return; } /// that means current alternatives have better quality. - if (shortest_alter_length(global_alternatives) < shortest_alter_length(cur_alternatives)) + if (shortest_literal_length(global_alternatives) < shortest_literal_length(cur_alternatives)) { global_alternatives.clear(); global_alternatives = cur_alternatives; @@ -373,14 +373,14 @@ finish: if (has_alternative_on_depth_0) { /// compare the quality of required substring and alternatives and choose the better one. - if (shortest_alter_length(global_alternatives) < required_substring.literal.size()) + if (shortest_literal_length(global_alternatives) < required_substring.literal.size()) global_alternatives = {required_substring}; Literals next_alternatives; /// this two vals are useless, xxx|xxx cannot be trivial nor prefix. 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() || shortest_alter_length(next_alternatives) < required_substring.literal.size()) + if (next_alternatives.empty() || shortest_literal_length(next_alternatives) < required_substring.literal.size()) { global_alternatives.push_back(required_substring); } From 426f4ec233c4ff6bcb8943b2a34f19c4ab817116 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 22 Mar 2023 16:58:34 +0000 Subject: [PATCH 288/418] Better transformQueryForExternalDatabase for analyzer --- src/Analyzer/ArrayJoinNode.cpp | 2 +- src/Analyzer/ColumnNode.h | 5 -- src/Analyzer/JoinNode.cpp | 6 +-- src/Analyzer/QueryNode.cpp | 2 +- src/Analyzer/TableFunctionNode.cpp | 6 ++- src/Analyzer/Utils.cpp | 4 +- src/Analyzer/Utils.h | 2 +- .../InterpreterSelectQueryAnalyzer.h | 13 +++-- ..._transform_query_for_external_database.cpp | 52 +++++++++++-------- .../transformQueryForExternalDatabase.cpp | 2 +- ...nsformQueryForExternalDatabaseAnalyzer.cpp | 5 -- .../02479_mysql_connect_to_self.reference | 2 +- 12 files changed, 50 insertions(+), 51 deletions(-) diff --git a/src/Analyzer/ArrayJoinNode.cpp b/src/Analyzer/ArrayJoinNode.cpp index 4a610767341..ee6bd80150d 100644 --- a/src/Analyzer/ArrayJoinNode.cpp +++ b/src/Analyzer/ArrayJoinNode.cpp @@ -75,7 +75,7 @@ ASTPtr ArrayJoinNode::toASTImpl(const ConvertToASTOptions & options) const array_join_ast->expression_list = array_join_ast->children.back(); ASTPtr tables_in_select_query_ast = std::make_shared(); - addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, children[table_expression_child_index]); + addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, children[table_expression_child_index], options); auto array_join_query_element_ast = std::make_shared(); array_join_query_element_ast->children.push_back(std::move(array_join_ast)); diff --git a/src/Analyzer/ColumnNode.h b/src/Analyzer/ColumnNode.h index fc626c62cc4..b320df788c5 100644 --- a/src/Analyzer/ColumnNode.h +++ b/src/Analyzer/ColumnNode.h @@ -103,11 +103,6 @@ public: */ QueryTreeNodePtr getColumnSource() const; - void dropColumnSource() - { - getSourceWeakPointer().reset(); - } - /** Get column source. * If column source is not valid null is returned. */ diff --git a/src/Analyzer/JoinNode.cpp b/src/Analyzer/JoinNode.cpp index 8b6ae0289ff..9b61c8b19d0 100644 --- a/src/Analyzer/JoinNode.cpp +++ b/src/Analyzer/JoinNode.cpp @@ -99,17 +99,17 @@ QueryTreeNodePtr JoinNode::cloneImpl() const return std::make_shared(getLeftTableExpression(), getRightTableExpression(), getJoinExpression(), locality, strictness, kind); } -ASTPtr JoinNode::toASTImpl(const ConvertToASTOptions & /* options */) const +ASTPtr JoinNode::toASTImpl(const ConvertToASTOptions & options) const { ASTPtr tables_in_select_query_ast = std::make_shared(); - addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, children[left_table_expression_child_index]); + addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, children[left_table_expression_child_index], options); size_t join_table_index = tables_in_select_query_ast->children.size(); auto join_ast = toASTTableJoin(); - addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, children[right_table_expression_child_index]); + addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, children[right_table_expression_child_index], options); auto & table_element = tables_in_select_query_ast->children.at(join_table_index)->as(); table_element.children.push_back(std::move(join_ast)); diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 0c586dd3fde..83e8cce8eb7 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -293,7 +293,7 @@ ASTPtr QueryNode::toASTImpl(const ConvertToASTOptions & options) const select_query->setExpression(ASTSelectQuery::Expression::SELECT, std::move(projection_ast)); ASTPtr tables_in_select_query_ast = std::make_shared(); - addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, getJoinTree()); + addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, getJoinTree(), options); select_query->setExpression(ASTSelectQuery::Expression::TABLES, std::move(tables_in_select_query_ast)); if (getPrewhere()) diff --git a/src/Analyzer/TableFunctionNode.cpp b/src/Analyzer/TableFunctionNode.cpp index 643383083d6..c130503d660 100644 --- a/src/Analyzer/TableFunctionNode.cpp +++ b/src/Analyzer/TableFunctionNode.cpp @@ -75,7 +75,7 @@ void TableFunctionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_ if (!settings_changes.empty()) { - buffer << '\n' << std::string(indent + 6, ' ') << "SETTINGS"; + buffer << '\n' << std::string(indent + 2, ' ') << "SETTINGS"; for (const auto & change : settings_changes) buffer << fmt::format(" {}={}", change.name, toString(change.value)); } @@ -90,6 +90,9 @@ bool TableFunctionNode::isEqualImpl(const IQueryTreeNode & rhs) const if (storage && rhs_typed.storage) return storage_id == rhs_typed.storage_id; + if (settings_changes != rhs_typed.settings_changes) + return false; + return table_expression_modifiers == rhs_typed.table_expression_modifiers; } @@ -108,6 +111,7 @@ void TableFunctionNode::updateTreeHashImpl(HashState & state) const if (table_expression_modifiers) table_expression_modifiers->updateTreeHash(state); + state.update(settings_changes.size()); for (const auto & change : settings_changes) { state.update(change.name.size()); diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index eb7aceef1e8..9b1bf6c5491 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -268,7 +268,7 @@ static ASTPtr convertIntoTableExpressionAST(const QueryTreeNodePtr & table_expre return result_table_expression; } -void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression) +void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression, const IQueryTreeNode::ConvertToASTOptions & convert_to_ast_options) { auto table_expression_node_type = table_expression->getNodeType(); @@ -297,7 +297,7 @@ void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_q [[fallthrough]]; case QueryTreeNodeType::JOIN: { - auto table_expression_tables_in_select_query_ast = table_expression->toAST(); + auto table_expression_tables_in_select_query_ast = table_expression->toAST(convert_to_ast_options); tables_in_select_query_ast->children.reserve(table_expression_tables_in_select_query_ast->children.size()); for (auto && table_element_ast : table_expression_tables_in_select_query_ast->children) tables_in_select_query_ast->children.push_back(std::move(table_element_ast)); diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index 5802c86c462..30c2bf65e30 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -40,7 +40,7 @@ std::optional tryExtractConstantFromConditionNode(const QueryTreeNodePtr & /** Add table expression in tables in select query children. * If table expression node is not of identifier node, table node, query node, table function node, join node or array join node type throws logical error exception. */ -void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression); +void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression, const IQueryTreeNode::ConvertToASTOptions & convert_to_ast_options); /// Extract table, table function, query, union from join tree QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node); diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.h b/src/Interpreters/InterpreterSelectQueryAnalyzer.h index f39e55072ef..8ca5cbb92d7 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.h +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.h @@ -9,6 +9,8 @@ #include #include +struct TestExternalDatabaseQueryState; + namespace DB { @@ -71,14 +73,11 @@ public: /// Set number_of_current_replica and count_participating_replicas in client_info void setProperClientInfo(size_t replica_number, size_t count_participating_replicas); - const QueryTreeNodePtr & getQueryTree() const { return query_tree; } - - SelectQueryInfo getSelectQueryInfo() - { - planner.buildQueryPlanIfNeeded(); return planner.buildSelectQueryInfo(); - } - private: + + /// Access select query info from unit tests (see gtest_transform_query_for_external_database) + friend struct ::TestExternalDatabaseQueryState; + ASTPtr query; ContextMutablePtr context; SelectQueryOptions select_query_options; diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index 270af37b7de..ed0706f3492 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -23,18 +23,24 @@ using namespace DB; /// TODO: use gtest fixture -struct State +struct TestExternalDatabaseQueryState { - State(const State&) = delete; + TestExternalDatabaseQueryState(const TestExternalDatabaseQueryState&) = delete; ContextMutablePtr context; - static const State & instance() + static const TestExternalDatabaseQueryState & instance() { - static State state; + static TestExternalDatabaseQueryState state; return state; } + static SelectQueryInfo getSelectQueryInfo(InterpreterSelectQueryAnalyzer & interpreter) + { + interpreter.planner.buildQueryPlanIfNeeded(); + return interpreter.planner.buildSelectQueryInfo(); + } + const NamesAndTypesList & getColumns(size_t idx = 0) const { return tables[idx].columns; @@ -86,7 +92,7 @@ private: }), }; - explicit State() + explicit TestExternalDatabaseQueryState() : context(Context::createCopy(getContext().context)) { tryRegisterFunctions(); @@ -123,7 +129,7 @@ private: }; static void checkOld( - const State & state, + const TestExternalDatabaseQueryState & state, size_t table_num, const std::string & query, const std::string & expected) @@ -165,7 +171,7 @@ static QueryTreeNodePtr findTableExpression(const QueryTreeNodePtr & node, const /// `column_names` - Normally it's passed to query plan step. But in test we do it manually. static void checkNewAnalyzer( - const State & state, + const TestExternalDatabaseQueryState & state, const Names & column_names, const std::string & query, const std::string & expected) @@ -175,7 +181,7 @@ static void checkNewAnalyzer( SelectQueryOptions select_options; InterpreterSelectQueryAnalyzer interpreter(ast, state.context, select_options); - SelectQueryInfo query_info = interpreter.getSelectQueryInfo(); + SelectQueryInfo query_info = state.getSelectQueryInfo(interpreter); const auto * query_node = query_info.query_tree->as(); if (!query_node) throw Exception(ErrorCodes::LOGICAL_ERROR, "QueryNode expected"); @@ -189,7 +195,7 @@ static void checkNewAnalyzer( } static void check( - const State & state, + const TestExternalDatabaseQueryState & state, size_t table_num, const Names & column_names, const std::string & query, @@ -208,7 +214,7 @@ static void check( TEST(TransformQueryForExternalDatabase, InWithSingleElement) { - const State & state = State::instance(); + const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); check(state, 1, {"column"}, "SELECT column FROM test.table WHERE 1 IN (1)", @@ -226,7 +232,7 @@ TEST(TransformQueryForExternalDatabase, InWithSingleElement) TEST(TransformQueryForExternalDatabase, InWithMultipleColumns) { - const State & state = State::instance(); + const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); check(state, 1, {"column"}, "SELECT column FROM test.table WHERE (1,1) IN ((1,1))", @@ -239,7 +245,7 @@ TEST(TransformQueryForExternalDatabase, InWithMultipleColumns) TEST(TransformQueryForExternalDatabase, InWithTable) { - const State & state = State::instance(); + const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); check(state, 1, {"column"}, "SELECT column FROM test.table WHERE 1 IN external_table", @@ -257,7 +263,7 @@ TEST(TransformQueryForExternalDatabase, InWithTable) TEST(TransformQueryForExternalDatabase, Like) { - const State & state = State::instance(); + const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); check(state, 1, {"field"}, "SELECT field FROM test.table WHERE field LIKE '%hello%'", @@ -269,7 +275,7 @@ TEST(TransformQueryForExternalDatabase, Like) TEST(TransformQueryForExternalDatabase, Substring) { - const State & state = State::instance(); + const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); check(state, 1, {"field"}, "SELECT field FROM test.table WHERE left(field, 10) = RIGHT(field, 10) AND SUBSTRING(field FROM 1 FOR 2) = 'Hello'", @@ -278,7 +284,7 @@ TEST(TransformQueryForExternalDatabase, Substring) TEST(TransformQueryForExternalDatabase, MultipleAndSubqueries) { - const State & state = State::instance(); + const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); check(state, 1, {"column"}, "SELECT column FROM test.table WHERE 1 = 1 AND toString(column) = '42' AND column = 42 AND left(toString(column), 10) = RIGHT(toString(column), 10) AND column IN (1, 42) AND SUBSTRING(toString(column) FROM 1 FOR 2) = 'Hello' AND column != 4", @@ -290,7 +296,7 @@ TEST(TransformQueryForExternalDatabase, MultipleAndSubqueries) TEST(TransformQueryForExternalDatabase, Issue7245) { - const State & state = State::instance(); + const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); check(state, 1, {"apply_id", "apply_type", "apply_status", "create_time"}, "SELECT apply_id FROM test.table WHERE apply_type = 2 AND create_time > addDays(toDateTime('2019-01-01 01:02:03'),-7) AND apply_status IN (3,4)", @@ -299,7 +305,7 @@ TEST(TransformQueryForExternalDatabase, Issue7245) TEST(TransformQueryForExternalDatabase, Aliases) { - const State & state = State::instance(); + const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); check(state, 1, {"field"}, "SELECT field AS value, field AS display FROM table WHERE field NOT IN ('') AND display LIKE '%test%'", @@ -308,7 +314,7 @@ TEST(TransformQueryForExternalDatabase, Aliases) TEST(TransformQueryForExternalDatabase, ForeignColumnInWhere) { - const State & state = State::instance(); + const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); check(state, 2, {"column", "apply_id"}, "SELECT column FROM test.table " @@ -319,7 +325,7 @@ TEST(TransformQueryForExternalDatabase, ForeignColumnInWhere) TEST(TransformQueryForExternalDatabase, NoStrict) { - const State & state = State::instance(); + const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); check(state, 1, {"field"}, "SELECT field FROM table WHERE field IN (SELECT attr FROM table2)", @@ -328,7 +334,7 @@ TEST(TransformQueryForExternalDatabase, NoStrict) TEST(TransformQueryForExternalDatabase, Strict) { - const State & state = State::instance(); + const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); state.context->setSetting("external_table_strict_query", true); check(state, 1, {"field"}, @@ -349,7 +355,7 @@ TEST(TransformQueryForExternalDatabase, Strict) TEST(TransformQueryForExternalDatabase, Null) { - const State & state = State::instance(); + const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); check(state, 1, {"field"}, "SELECT field FROM table WHERE field IS NULL", @@ -368,7 +374,7 @@ TEST(TransformQueryForExternalDatabase, Null) TEST(TransformQueryForExternalDatabase, ToDate) { - const State & state = State::instance(); + const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); check(state, 1, {"a", "b", "foo"}, "SELECT foo FROM table WHERE a=10 AND b=toDate('2019-10-05')", @@ -377,7 +383,7 @@ TEST(TransformQueryForExternalDatabase, ToDate) TEST(TransformQueryForExternalDatabase, Analyzer) { - const State & state = State::instance(); + const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); check(state, 1, {"field"}, "SELECT count() FROM table WHERE field LIKE '%name_%'", diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index da40673d979..548b55749d7 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -365,7 +365,7 @@ String transformQueryForExternalDatabase( if (column_names.empty()) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "No column names for query '{}' to external table '{}.{}'", - database, table, query_info.query_tree->formatASTForErrorMessage()); + query_info.query_tree->formatASTForErrorMessage(), database, table); auto clone_query = getASTForExternalDatabaseFromQueryTree(query_info.query_tree); diff --git a/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp b/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp index 291ed622521..513ec510d23 100644 --- a/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp +++ b/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp @@ -1,18 +1,13 @@ #include -#include #include #include -#include #include #include -#include #include #include -#include -#include #include diff --git a/tests/queries/0_stateless/02479_mysql_connect_to_self.reference b/tests/queries/0_stateless/02479_mysql_connect_to_self.reference index 61e73f85ac7..f4dd01bc184 100644 --- a/tests/queries/0_stateless/02479_mysql_connect_to_self.reference +++ b/tests/queries/0_stateless/02479_mysql_connect_to_self.reference @@ -60,7 +60,7 @@ QUERY id: 0 IDENTIFIER id: 11, identifier: foo CONSTANT id: 12, constant_value: \'default\', constant_value_type: String CONSTANT id: 13, constant_value: \'\', constant_value_type: String - SETTINGS connection_wait_timeout=123 connect_timeout=40123002 read_write_timeout=40123001 connection_pool_size=3 + SETTINGS connection_wait_timeout=123 connect_timeout=40123002 read_write_timeout=40123001 connection_pool_size=3 SELECT key AS key, From b70216f9f1877178aca03d72a0e7ea83bc7d0bbf Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 22 Mar 2023 18:12:42 +0100 Subject: [PATCH 289/418] fix for Replicated database recovery --- src/Databases/DatabaseReplicated.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 0d26fba4614..1d39796e4fc 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -928,7 +928,16 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep for (const auto & table_id : tables_to_create) { auto table_name = table_id.getTableName(); - auto create_query_string = table_name_to_metadata[table_name]; + auto metadata_it = table_name_to_metadata.find(table_name); + if (metadata_it == table_name_to_metadata.end()) + { + /// getTablesSortedByDependency() may return some not existing tables or tables from other databases + LOG_WARNING(log, "Got table name {} when resolving table dependencies, " + "but database {} does not have metadata for that table. Ignoring it", table_id.getNameForLogs(), getDatabaseName()); + continue; + } + + const auto & create_query_string = metadata_it->second; if (isTableExist(table_name, getContext())) { assert(create_query_string == readMetadataFile(table_name)); From e514723256fa48f0e875f01e909286e3840d9d7d Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 22 Mar 2023 19:06:51 +0100 Subject: [PATCH 290/418] throw exception when all retries exhausted --- src/IO/WriteBufferFromS3.cpp | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 86a4cebb5d8..dd90fd9387a 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -396,7 +396,7 @@ void WriteBufferFromS3::completeMultipartUpload() if (outcome.IsSuccess()) { LOG_TRACE(log, "Multipart upload has completed. Bucket: {}, Key: {}, Upload_id: {}, Parts: {}", bucket, key, multipart_upload_id, tags.size()); - break; + return; } else if (outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) { @@ -412,6 +412,11 @@ void WriteBufferFromS3::completeMultipartUpload() outcome.GetError().GetMessage(), key, bucket, fmt::join(tags.begin(), tags.end(), " ")); } } + + throw S3Exception( + Aws::S3::S3Errors::NO_SUCH_KEY, + "Message: Multipart upload failed with NO_SUCH_KEY error, retries {}, Key: {}, Bucket: {}", + max_retry, key, bucket); } void WriteBufferFromS3::makeSinglepartUpload() @@ -508,7 +513,7 @@ void WriteBufferFromS3::processPutRequest(const PutObjectTask & task) if (outcome.IsSuccess()) { LOG_TRACE(log, "Single part upload has completed. Bucket: {}, Key: {}, Object size: {}, WithPool: {}", bucket, key, task.req.GetContentLength(), with_pool); - break; + return; } else if (outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) { @@ -525,6 +530,11 @@ void WriteBufferFromS3::processPutRequest(const PutObjectTask & task) outcome.GetError().GetMessage(), key, bucket, task.req.GetContentLength(), with_pool); } } + + throw S3Exception( + Aws::S3::S3Errors::NO_SUCH_KEY, + "Message: Single part upload failed with NO_SUCH_KEY error, retries {}, Key: {}, Bucket: {}", + max_retry, key, bucket); } void WriteBufferFromS3::waitForReadyBackGroundTasks() From 8139d7d4b33e3a2f5b0b92be3347767ec3f88cae Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 22 Mar 2023 18:24:20 +0000 Subject: [PATCH 291/418] Fix match for constants with same values but different types. --- src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp b/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp index 02b7de60a4a..c9cf46aaeca 100644 --- a/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp +++ b/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp @@ -164,6 +164,7 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG if (frame.mapped_children[i] == nullptr) { all_children_matched = children[i]->column && isColumnConst(*children[i]->column) + && children[i]->result_type->equals(*frame.node->children[i]->result_type) && assert_cast(*children[i]->column).getField() == assert_cast(*frame.node->children[i]->column).getField(); } else From ef647d602b294848e6e60dceb7cc2881062dedfc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 22 Mar 2023 21:43:23 +0300 Subject: [PATCH 292/418] Update HTTPServerRequest.cpp --- src/Server/HTTP/HTTPServerRequest.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Server/HTTP/HTTPServerRequest.cpp b/src/Server/HTTP/HTTPServerRequest.cpp index c9ffa3a4c3b..569f763d3a0 100644 --- a/src/Server/HTTP/HTTPServerRequest.cpp +++ b/src/Server/HTTP/HTTPServerRequest.cpp @@ -63,8 +63,10 @@ HTTPServerRequest::HTTPServerRequest(HTTPContextPtr context, HTTPServerResponse } else if (getMethod() != HTTPRequest::HTTP_GET && getMethod() != HTTPRequest::HTTP_HEAD && getMethod() != HTTPRequest::HTTP_DELETE) { + /// That check for has_body may be false-negative in rare cases, but it's okay + bool has_body = stream->hasPendingData(); stream = std::move(in); - if (!startsWith(getContentType(), "multipart/form-data")) + if (!startsWith(getContentType(), "multipart/form-data") && has_body) LOG_WARNING(&Poco::Logger::get("HTTPServerRequest"), "Got an HTTP request with no content length " "and no chunked/multipart encoding, it may be impossible to distinguish graceful EOF from abnormal connection loss"); } From 577130a9f6284165acc69366aee7c64cb30a0621 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 22 Mar 2023 19:17:47 +0000 Subject: [PATCH 293/418] Fix GRANT query formatting --- src/Parsers/Access/ASTGrantQuery.cpp | 4 +++- .../02675_grant_query_formatting.reference | 1 + .../0_stateless/02675_grant_query_formatting.sh | 11 +++++++++++ 3 files changed, 15 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02675_grant_query_formatting.reference create mode 100755 tests/queries/0_stateless/02675_grant_query_formatting.sh diff --git a/src/Parsers/Access/ASTGrantQuery.cpp b/src/Parsers/Access/ASTGrantQuery.cpp index f92541ec672..c37d9f1c75f 100644 --- a/src/Parsers/Access/ASTGrantQuery.cpp +++ b/src/Parsers/Access/ASTGrantQuery.cpp @@ -111,7 +111,6 @@ ASTPtr ASTGrantQuery::clone() const void ASTGrantQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const { settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << (attach_mode ? "ATTACH " : "") - << (settings.hilite ? hilite_keyword : "") << ((!is_revoke && (replace_access || replace_granted_roles)) ? "REPLACE " : "") << (settings.hilite ? hilite_none : "") << (settings.hilite ? hilite_keyword : "") << (is_revoke ? "REVOKE" : "GRANT") << (settings.hilite ? IAST::hilite_none : ""); @@ -153,6 +152,9 @@ void ASTGrantQuery::formatImpl(const FormatSettings & settings, FormatState &, F settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH GRANT OPTION" << (settings.hilite ? hilite_none : ""); else if (admin_option) settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH ADMIN OPTION" << (settings.hilite ? hilite_none : ""); + + if (replace_access || replace_granted_roles) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH REPLACE OPTION" << (settings.hilite ? hilite_none : ""); } } diff --git a/tests/queries/0_stateless/02675_grant_query_formatting.reference b/tests/queries/0_stateless/02675_grant_query_formatting.reference new file mode 100644 index 00000000000..8cbf028e742 --- /dev/null +++ b/tests/queries/0_stateless/02675_grant_query_formatting.reference @@ -0,0 +1 @@ +GRANT SELECT ON *.* TO A WITH REPLACE OPTION diff --git a/tests/queries/0_stateless/02675_grant_query_formatting.sh b/tests/queries/0_stateless/02675_grant_query_formatting.sh new file mode 100755 index 00000000000..76c953913a9 --- /dev/null +++ b/tests/queries/0_stateless/02675_grant_query_formatting.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 + +set -e + +format="$CLICKHOUSE_FORMAT --oneline" + +echo "grant select on *.* to A with replace option" | $format From 89c43bec3e5350afabfeb748ffbdd5ddeb61733e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 22 Mar 2023 23:56:42 +0300 Subject: [PATCH 294/418] Update HTTPServerRequest.cpp --- src/Server/HTTP/HTTPServerRequest.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/HTTP/HTTPServerRequest.cpp b/src/Server/HTTP/HTTPServerRequest.cpp index 569f763d3a0..b66c6c6a62e 100644 --- a/src/Server/HTTP/HTTPServerRequest.cpp +++ b/src/Server/HTTP/HTTPServerRequest.cpp @@ -64,7 +64,7 @@ HTTPServerRequest::HTTPServerRequest(HTTPContextPtr context, HTTPServerResponse else if (getMethod() != HTTPRequest::HTTP_GET && getMethod() != HTTPRequest::HTTP_HEAD && getMethod() != HTTPRequest::HTTP_DELETE) { /// That check for has_body may be false-negative in rare cases, but it's okay - bool has_body = stream->hasPendingData(); + bool has_body = in->hasPendingData(); stream = std::move(in); if (!startsWith(getContentType(), "multipart/form-data") && has_body) LOG_WARNING(&Poco::Logger::get("HTTPServerRequest"), "Got an HTTP request with no content length " From f69c432f5f65ae07742e14f3306c6999738aba1b Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 22 Mar 2023 18:23:11 -0400 Subject: [PATCH 295/418] Second attempt to fix crash. --- src/Functions/geometryConverters.h | 10 ++-------- .../01304_polygons_sym_difference.reference | 5 ----- .../0_stateless/01304_polygons_sym_difference.sql | 4 ---- .../25401_polygons_sym_difference_rollup.reference | 6 ++++++ .../25401_polygons_sym_difference_rollup.sql | 4 ++++ 5 files changed, 12 insertions(+), 17 deletions(-) create mode 100644 tests/queries/0_stateless/25401_polygons_sym_difference_rollup.reference create mode 100644 tests/queries/0_stateless/25401_polygons_sym_difference_rollup.sql diff --git a/src/Functions/geometryConverters.h b/src/Functions/geometryConverters.h index 21cc19665dc..652c98d70d4 100644 --- a/src/Functions/geometryConverters.h +++ b/src/Functions/geometryConverters.h @@ -120,11 +120,8 @@ 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) + for (size_t iter = 0; iter < offsets.size() && iter < all_rings.size(); ++iter) { const auto current_array_size = offsets[iter] - prev_offset; answer[iter].outer() = std::move(all_rings[prev_offset]); @@ -152,10 +149,7 @@ 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 iter = 0; iter < offsets.size() && iter < all_polygons.size(); ++iter) { for (size_t polygon_iter = prev_offset; polygon_iter < offsets[iter]; ++polygon_iter) answer[iter].emplace_back(std::move(all_polygons[polygon_iter])); diff --git a/tests/queries/0_stateless/01304_polygons_sym_difference.reference b/tests/queries/0_stateless/01304_polygons_sym_difference.reference index 1ab25a27e12..9344410f192 100644 --- a/tests/queries/0_stateless/01304_polygons_sym_difference.reference +++ b/tests/queries/0_stateless/01304_polygons_sym_difference.reference @@ -5,8 +5,3 @@ 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 e8b5502e00b..c4129676b26 100644 --- a/tests/queries/0_stateless/01304_polygons_sym_difference.sql +++ b/tests/queries/0_stateless/01304_polygons_sym_difference.sql @@ -13,7 +13,3 @@ 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; diff --git a/tests/queries/0_stateless/25401_polygons_sym_difference_rollup.reference b/tests/queries/0_stateless/25401_polygons_sym_difference_rollup.reference new file mode 100644 index 00000000000..2323f050607 --- /dev/null +++ b/tests/queries/0_stateless/25401_polygons_sym_difference_rollup.reference @@ -0,0 +1,6 @@ +[] +[] +[[(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)]] [] +[[[(100.0001,1000.0001),(1000.0001,1.1920928955078125e-7),(20,-20),(20,20),(10,10),(-20,20),(100.0001,1000.0001)]]] +[[[(100.0001,1000.0001),(1000.0001,1.1920928955078125e-7),(20,-20),(20,20),(10,10),(-20,20),(100.0001,1000.0001)]]] diff --git a/tests/queries/0_stateless/25401_polygons_sym_difference_rollup.sql b/tests/queries/0_stateless/25401_polygons_sym_difference_rollup.sql new file mode 100644 index 00000000000..df8625d5915 --- /dev/null +++ b/tests/queries/0_stateless/25401_polygons_sym_difference_rollup.sql @@ -0,0 +1,4 @@ + +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; +SELECT polygonsSymDifferenceCartesian([[[(100.0001, 1000.0001), (-20., 20.), (10., 10.), (20., 20.), (20., -20.), (1000.0001, 1.1920928955078125e-7)]],[[(0.0001, 100000000000000000000.)]] AS x],[x]) GROUP BY x WITH ROLLUP; \ No newline at end of file From e3c798bae5f204208048f336fa842daad65404eb Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 22 Mar 2023 23:46:15 +0100 Subject: [PATCH 296/418] fix terminate in parts check thread --- .../ReplicatedMergeTreePartCheckThread.cpp | 73 +++++++++---------- 1 file changed, 36 insertions(+), 37 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index 8eafc54cb4c..27ab56d4773 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -76,50 +76,49 @@ std::unique_lock ReplicatedMergeTreePartCheckThread::pausePartsCheck void ReplicatedMergeTreePartCheckThread::cancelRemovedPartsCheck(const MergeTreePartInfo & drop_range_info) { - Strings removed_names; + Strings parts_to_remove; { std::lock_guard lock(parts_mutex); - removed_names.reserve(parts_queue.size()); /// Avoid memory limit in the middle - for (auto it = parts_queue.begin(); it != parts_queue.end();) - { - if (drop_range_info.contains(MergeTreePartInfo::fromPartName(it->first, storage.format_version))) - { - /// Remove part from the queue to avoid part resurrection - /// if we will check it and enqueue fetch after DROP/REPLACE execution. - removed_names.push_back(it->first); - parts_set.erase(it->first); - it = parts_queue.erase(it); - } - else - { - ++it; - } - } + for (const auto & elem : parts_queue) + if (drop_range_info.contains(MergeTreePartInfo::fromPartName(elem.first, storage.format_version))) + parts_to_remove.push_back(elem.first); } - /// This filtering is not necessary - auto new_end = std::remove_if(removed_names.begin(), removed_names.end(), [this](const String & part_name) + /// We have to remove parts that were not removed by removePartAndEnqueueFetch + LOG_INFO(log, "Removing broken parts from ZooKeeper: {}", fmt::join(parts_to_remove, ", ")); + storage.removePartsFromZooKeeperWithRetries(parts_to_remove); /// May throw + + /// Now we can remove parts from the check queue. + /// It's not atomic (because it's bad idea to hold the mutex while removing something from zk with retries), + /// but the check thread is currently paused, and no new parts in drop_range_info can by enqueued + /// while the corresponding DROP_RANGE/REPLACE_RANGE exists, so it should be okay. We will recheck it just in case. + + StringSet removed_parts; + for (auto & part : parts_to_remove) + removed_parts.emplace(std::move(part)); + size_t count = 0; + + std::lock_guard lock(parts_mutex); + for (const auto & elem : parts_queue) { - auto part = storage.getPartIfExists(part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated, MergeTreeDataPartState::Deleting}); - /// The rest of parts will be removed normally - return part && !part->outdated_because_broken; + bool is_removed = removed_parts.contains(elem.first); + bool should_have_been_removed = drop_range_info.contains(MergeTreePartInfo::fromPartName(elem.first, storage.format_version)); + if (is_removed != should_have_been_removed) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Inconsistent parts_queue: name={}, is_removed={}, should_have_been_removed={}", + elem.first, is_removed, should_have_been_removed); + count += is_removed; + } + + if (count != parts_to_remove.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected number of parts to remove from parts_queue: should be {}, got {}", + parts_to_remove.size(), count); + + auto new_end = std::remove_if(parts_queue.begin(), parts_queue.end(), [&removed_parts] (const auto & elem) + { + return removed_parts.contains(elem.first); }); - removed_names.erase(new_end, removed_names.end()); - if (removed_names.empty()) - return; - try - { - /// We have to remove parts that were not removed by removePartAndEnqueueFetch - LOG_INFO(log, "Removing broken parts from ZooKeeper: {}", fmt::join(removed_names, ", ")); - storage.removePartsFromZooKeeperWithRetries(removed_names, /* max_retries */ 100); - } - catch (...) - { - /// It's highly unlikely to happen on normal use cases. And if it happens it's easier to restart and reinitialize - LOG_FATAL(log, "Failed to remove parts [{}] from ZooKeeper: {}", fmt::join(removed_names, ", "), getCurrentExceptionMessage(/* with_stacktrace = */ true)); - std::terminate(); - } + parts_queue.erase(new_end, parts_queue.end()); } size_t ReplicatedMergeTreePartCheckThread::size() const From 14cae25bd3d8254ce22ff0f9da81ab231854b47e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 23 Mar 2023 01:48:28 +0100 Subject: [PATCH 297/418] Allow start without ZooKeeper. --- ...erDefinedSQLObjectsLoaderFromZooKeeper.cpp | 12 ++++++++- .../test.py | 25 +++++++++++++++++++ 2 files changed, 36 insertions(+), 1 deletion(-) diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp index ab6ba762103..245ad2e8fdc 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp @@ -119,10 +119,20 @@ void UserDefinedSQLObjectsLoaderFromZooKeeper::resetAfterError() void UserDefinedSQLObjectsLoaderFromZooKeeper::loadObjects() { + /// loadObjects() is called at start from Server::main(), so it's better not to stop here on no connection to ZooKeeper or any other error. + /// However the watching thread must be started anyway in case the connection will be established later. if (!objects_loaded) { - reloadObjects(); + try + { + reloadObjects(); + } + catch (...) + { + tryLogCurrentException(log, "Failed to load user-defined objects"); + } } + startWatchingThread(); } diff --git a/tests/integration/test_replicated_user_defined_functions/test.py b/tests/integration/test_replicated_user_defined_functions/test.py index 199136a75fa..c0990819bf4 100644 --- a/tests/integration/test_replicated_user_defined_functions/test.py +++ b/tests/integration/test_replicated_user_defined_functions/test.py @@ -85,6 +85,7 @@ def test_create_and_drop(): def test_create_and_replace(): node1.query("CREATE FUNCTION f1 AS (x, y) -> x + y") + assert node1.query("SELECT f1(12, 3)") == "15\n" expected_error = "User-defined function 'f1' already exists" assert expected_error in node1.query_and_get_error( @@ -253,3 +254,27 @@ def test_reload_zookeeper(): # switch to the original version of zookeeper config cluster.start_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) revert_zookeeper_config() + + +# Start without ZooKeeper must be possible, user-defined functions will be loaded after connecting to ZooKeeper. +def test_start_without_zookeeper(): + node2.stop_clickhouse() + + node1.query("CREATE FUNCTION f1 AS (x, y) -> x + y") + + cluster.stop_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) + node2.start_clickhouse() + + assert ( + node2.query("SELECT create_query FROM system.functions WHERE name='f1'") == "" + ) + + cluster.start_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) + wait_zookeeper_node_to_start(["zoo1", "zoo2", "zoo3"]) + + assert_eq_with_retry( + node2, + "SELECT create_query FROM system.functions WHERE name='f1'", + "CREATE FUNCTION f1 AS (x, y) -> (x + y)\n", + ) + node1.query("DROP FUNCTION f1") From b0fb8fecee0732a27a0907875ead0e1c4c021e4e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 23 Mar 2023 02:16:10 +0100 Subject: [PATCH 298/418] Make names in ZooKeeper consistent with non-replicated storage. --- ...erDefinedSQLObjectsLoaderFromZooKeeper.cpp | 41 ++++++++++--------- 1 file changed, 21 insertions(+), 20 deletions(-) diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp index 245ad2e8fdc..284adeb2c2d 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp @@ -26,24 +26,24 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -static String getRootNodeName(UserDefinedSQLObjectType object_type) +namespace { - switch (object_type) + std::string_view getNodePrefix(UserDefinedSQLObjectType object_type) { - case UserDefinedSQLObjectType::Function: - return "functions"; + switch (object_type) + { + case UserDefinedSQLObjectType::Function: + return "function_"; + } + UNREACHABLE(); } - UNREACHABLE(); -} -static String getRootNodePath(const String & root_path, UserDefinedSQLObjectType object_type) -{ - return root_path + "/" + getRootNodeName(object_type); -} + constexpr std::string_view sql_extension = ".sql"; -static String getNodePath(const String & root_path, UserDefinedSQLObjectType object_type, const String & object_name) -{ - return getRootNodePath(root_path, object_type) + "/" + escapeForFileName(object_name); + String getNodePath(const String & root_path, UserDefinedSQLObjectType object_type, const String & object_name) + { + return root_path + "/" + String{getNodePrefix(object_type)} + escapeForFileName(object_name) + String{sql_extension}; + } } @@ -198,7 +198,6 @@ void UserDefinedSQLObjectsLoaderFromZooKeeper::createRootNodes(const zkutil::Zoo { zookeeper->createAncestors(zookeeper_path); zookeeper->createIfNotExists(zookeeper_path, ""); - zookeeper->createIfNotExists(zookeeper_path + "/functions", ""); } bool UserDefinedSQLObjectsLoaderFromZooKeeper::storeObject( @@ -354,17 +353,19 @@ Strings UserDefinedSQLObjectsLoaderFromZooKeeper::getObjectNamesAndSetWatch( }; Coordination::Stat stat; - const auto path = getRootNodePath(zookeeper_path, object_type); - const auto node_names = zookeeper->getChildrenWatch(path, &stat, object_list_watcher); + const auto node_names = zookeeper->getChildrenWatch(zookeeper_path, &stat, object_list_watcher); + const auto prefix = getNodePrefix(object_type); 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)); + if (node_name.starts_with(prefix) && node_name.ends_with(sql_extension)) + { + String object_name = unescapeForFileName(node_name.substr(prefix.length(), node_name.length() - prefix.length() - sql_extension.length())); + if (!object_name.empty()) + object_names.push_back(std::move(object_name)); + } } return object_names; From 98b3757b42b99369db239e01b6fa67fbeba6fe50 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 23 Mar 2023 11:08:04 +0800 Subject: [PATCH 299/418] fix failed tests --- .../02668_parse_datetime.reference | 60 ++++++++--------- .../0_stateless/02668_parse_datetime.sql | 65 +++++++++---------- 2 files changed, 61 insertions(+), 64 deletions(-) diff --git a/tests/queries/0_stateless/02668_parse_datetime.reference b/tests/queries/0_stateless/02668_parse_datetime.reference index 6f17041337d..b893c1bc6e9 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.reference +++ b/tests/queries/0_stateless/02668_parse_datetime.reference @@ -3,35 +3,35 @@ select parseDateTime('2020', '%Y', 'UTC') = toDateTime('2020-01-01', 'UTC'); 1 -- month -select parseDateTime('02', '%m') = toDateTime('2000-02-01'); +select parseDateTime('02', '%m', 'UTC') = toDateTime('2000-02-01', 'UTC'); 1 -select parseDateTime('07', '%m') = toDateTime('2000-07-01'); +select parseDateTime('07', '%m', 'UTC') = toDateTime('2000-07-01', 'UTC'); 1 -select parseDateTime('11-', '%m-') = toDateTime('2000-11-01'); +select parseDateTime('11-', '%m-', 'UTC') = toDateTime('2000-11-01', 'UTC'); 1 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('02', '%c', 'UTC') = toDateTime('2000-02-01', 'UTC'); 1 -select parseDateTime('07', '%c') = toDateTime('2000-07-01'); +select parseDateTime('07', '%c', 'UTC') = toDateTime('2000-07-01', 'UTC'); 1 -select parseDateTime('11-', '%c-') = toDateTime('2000-11-01'); +select parseDateTime('11-', '%c-', 'UTC') = toDateTime('2000-11-01', 'UTC'); 1 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', 'UTC') = toDateTime('2000-06-01', 'UTC'); 1 -select parseDateTime('JUN', '%b') = toDateTime('2000-06-01'); +select parseDateTime('JUN', '%b', 'UTC') = toDateTime('2000-06-01', 'UTC'); 1 select parseDateTime('abc', '%b'); -- { serverError CANNOT_PARSE_DATETIME } -- day of month -select parseDateTime('07', '%d') = toDateTime('2000-01-07'); +select parseDateTime('07', '%d', 'UTC') = toDateTime('2000-01-07', 'UTC'); 1 -select parseDateTime('01', '%d') = toDateTime('2000-01-01'); +select parseDateTime('01', '%d', 'UTC') = toDateTime('2000-01-01', 'UTC'); 1 -select parseDateTime('/11', '/%d') = toDateTime('2000-01-11'); +select parseDateTime('/11', '/%d', 'UTC') = toDateTime('2000-01-11', 'UTC'); 1 select parseDateTime('00', '%d'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime('32', '%d'); -- { serverError CANNOT_PARSE_DATETIME } @@ -39,50 +39,50 @@ 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('01 31 20 02', '%m %d %d %m', 'UTC') = toDateTime('2000-02-20', 'UTC'); 1 -select parseDateTime('02 31 20 04', '%m %d %d %m') = toDateTime('2000-04-20'); +select parseDateTime('02 31 20 04', '%m %d %d %m', 'UTC') = toDateTime('2000-04-20', 'UTC'); 1 -select parseDateTime('02 31 01', '%m %d %m') = toDateTime('2000-01-31'); +select parseDateTime('02 31 01', '%m %d %m', 'UTC') = toDateTime('2000-01-31', 'UTC'); 1 -select parseDateTime('2000-02-29', '%Y-%m-%d') = toDateTime('2000-02-29'); +select parseDateTime('2000-02-29', '%Y-%m-%d', 'UTC') = toDateTime('2000-02-29', 'UTC'); 1 select parseDateTime('2001-02-29', '%Y-%m-%d'); -- { serverError CANNOT_PARSE_DATETIME } -- day of year -select parseDateTime('001', '%j') = toDateTime('2000-01-01'); +select parseDateTime('001', '%j', 'UTC') = toDateTime('2000-01-01', 'UTC'); 1 -select parseDateTime('007', '%j') = toDateTime('2000-01-07'); +select parseDateTime('007', '%j', 'UTC') = toDateTime('2000-01-07', 'UTC'); 1 -select parseDateTime('/031/', '/%j/') = toDateTime('2000-01-31'); +select parseDateTime('/031/', '/%j/', 'UTC') = toDateTime('2000-01-31', 'UTC'); 1 -select parseDateTime('032', '%j') = toDateTime('2000-02-01'); +select parseDateTime('032', '%j', 'UTC') = toDateTime('2000-02-01', 'UTC'); 1 -select parseDateTime('060', '%j') = toDateTime('2000-02-29'); +select parseDateTime('060', '%j', 'UTC') = toDateTime('2000-02-29', 'UTC'); 1 -select parseDateTime('365', '%j') = toDateTime('2000-12-30'); +select parseDateTime('365', '%j', 'UTC') = toDateTime('2000-12-30', 'UTC'); 1 -select parseDateTime('366', '%j') = toDateTime('2000-12-31'); +select parseDateTime('366', '%j', 'UTC') = toDateTime('2000-12-31', 'UTC'); 1 -select parseDateTime('1980 001', '%Y %j') = toDateTime('1980-01-01'); +select parseDateTime('1980 001', '%Y %j', 'UTC') = toDateTime('1980-01-01', 'UTC'); 1 -select parseDateTime('1980 007', '%Y %j') = toDateTime('1980-01-07'); +select parseDateTime('1980 007', '%Y %j', 'UTC') = toDateTime('1980-01-07', 'UTC'); 1 -select parseDateTime('1980 /007', '%Y /%j') = toDateTime('1980-01-07'); +select parseDateTime('1980 /007', '%Y /%j', 'UTC') = toDateTime('1980-01-07', 'UTC'); 1 -select parseDateTime('1980 /031/', '%Y /%j/') = toDateTime('1980-01-31'); +select parseDateTime('1980 /031/', '%Y /%j/', 'UTC') = toDateTime('1980-01-31', 'UTC'); 1 -select parseDateTime('1980 032', '%Y %j') = toDateTime('1980-02-01'); +select parseDateTime('1980 032', '%Y %j', 'UTC') = toDateTime('1980-02-01', 'UTC'); 1 -select parseDateTime('1980 060', '%Y %j') = toDateTime('1980-02-29'); +select parseDateTime('1980 060', '%Y %j', 'UTC') = toDateTime('1980-02-29', 'UTC'); 1 -select parseDateTime('1980 366', '%Y %j') = toDateTime('1980-12-31'); +select parseDateTime('1980 366', '%Y %j', 'UTC') = toDateTime('1980-12-31', 'UTC'); 1 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'); +select parseDateTime('2001 366 2000', '%Y %j %Y', 'UTC') = toDateTime('2000-12-31', 'UTC'); 1 -- hour of day select parseDateTime('07', '%H', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); diff --git a/tests/queries/0_stateless/02668_parse_datetime.sql b/tests/queries/0_stateless/02668_parse_datetime.sql index 71e90165ec4..86e8877eedc 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.sql +++ b/tests/queries/0_stateless/02668_parse_datetime.sql @@ -3,61 +3,59 @@ select parseDateTime('2020', '%Y', 'UTC') = toDateTime('2020-01-01', 'UTC'); -- month -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', '%m', 'UTC') = toDateTime('2000-02-01', 'UTC'); +select parseDateTime('07', '%m', 'UTC') = toDateTime('2000-07-01', 'UTC'); +select parseDateTime('11-', '%m-', 'UTC') = toDateTime('2000-11-01', 'UTC'); 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('02', '%c', 'UTC') = toDateTime('2000-02-01', 'UTC'); +select parseDateTime('07', '%c', 'UTC') = toDateTime('2000-07-01', 'UTC'); +select parseDateTime('11-', '%c-', 'UTC') = toDateTime('2000-11-01', 'UTC'); 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('jun', '%b', 'UTC') = toDateTime('2000-06-01', 'UTC'); +select parseDateTime('JUN', '%b', 'UTC') = toDateTime('2000-06-01', 'UTC'); 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('07', '%d', 'UTC') = toDateTime('2000-01-07', 'UTC'); +select parseDateTime('01', '%d', 'UTC') = toDateTime('2000-01-01', 'UTC'); +select parseDateTime('/11', '/%d', 'UTC') = toDateTime('2000-01-11', 'UTC'); 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('01 31 20 02', '%m %d %d %m', 'UTC') = toDateTime('2000-02-20', 'UTC'); +select parseDateTime('02 31 20 04', '%m %d %d %m', 'UTC') = toDateTime('2000-04-20', 'UTC'); +select parseDateTime('02 31 01', '%m %d %m', 'UTC') = toDateTime('2000-01-31', 'UTC'); +select parseDateTime('2000-02-29', '%Y-%m-%d', 'UTC') = toDateTime('2000-02-29', 'UTC'); select parseDateTime('2001-02-29', '%Y-%m-%d'); -- { serverError CANNOT_PARSE_DATETIME } -- 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-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'); -select parseDateTime('1980 366', '%Y %j') = toDateTime('1980-12-31'); +select parseDateTime('001', '%j', 'UTC') = toDateTime('2000-01-01', 'UTC'); +select parseDateTime('007', '%j', 'UTC') = toDateTime('2000-01-07', 'UTC'); +select parseDateTime('/031/', '/%j/', 'UTC') = toDateTime('2000-01-31', 'UTC'); +select parseDateTime('032', '%j', 'UTC') = toDateTime('2000-02-01', 'UTC'); +select parseDateTime('060', '%j', 'UTC') = toDateTime('2000-02-29', 'UTC'); +select parseDateTime('365', '%j', 'UTC') = toDateTime('2000-12-30', 'UTC'); +select parseDateTime('366', '%j', 'UTC') = toDateTime('2000-12-31', 'UTC'); +select parseDateTime('1980 001', '%Y %j', 'UTC') = toDateTime('1980-01-01', 'UTC'); +select parseDateTime('1980 007', '%Y %j', 'UTC') = toDateTime('1980-01-07', 'UTC'); +select parseDateTime('1980 /007', '%Y /%j', 'UTC') = toDateTime('1980-01-07', 'UTC'); +select parseDateTime('1980 /031/', '%Y /%j/', 'UTC') = toDateTime('1980-01-31', 'UTC'); +select parseDateTime('1980 032', '%Y %j', 'UTC') = toDateTime('1980-02-01', 'UTC'); +select parseDateTime('1980 060', '%Y %j', 'UTC') = toDateTime('1980-02-29', 'UTC'); +select parseDateTime('1980 366', '%Y %j', 'UTC') = toDateTime('1980-12-31', 'UTC'); 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'); +select parseDateTime('2001 366 2000', '%Y %j %Y', 'UTC') = toDateTime('2000-12-31', 'UTC'); -- hour of day select parseDateTime('07', '%H', 'UTC') = toDateTime('1970-01-01 07:00:00', 'UTC'); @@ -129,10 +127,9 @@ select parseDateTime('00/', '%s/', 'UTC') = toDateTime('1970-01-01 00:00:00', 'U 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', 'UTC') = toDateTime('2021-01-04 23:00:00', 'UTC'); select parseDateTime('2019-07-03 11:04:10', '%Y-%m-%d %H:%i:%s', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); select parseDateTime('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); --- { echoOff } +-- { echoOff } \ No newline at end of file From e7afe0f60fc53c2882a5109e70ffa0dfbbd2b16b Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 23 Mar 2023 04:08:36 +0000 Subject: [PATCH 300/418] fix --- src/Client/QueryFuzzer.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index 8415a990f0e..ba4246e8d0d 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -1105,7 +1105,16 @@ void QueryFuzzer::fuzz(ASTPtr & ast) } else if (auto * explain_query = typeid_cast(ast.get())) { - fuzzExplainQuery(*explain_query); + /// Fuzzing EXPLAIN query to SELECT query randomly + if (fuzz_rand() % 20 == 0 && explain_query->getExplainedQuery()->getQueryKind() == IAST::QueryKind::Select) + { + ast = explain_query->getExplainedQuery(); + fuzz(ast); + } + else + { + fuzzExplainQuery(*explain_query); + } } else { From e07114a6e3c17df9253a352438b83f07b838a7c4 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 23 Mar 2023 12:13:51 +0800 Subject: [PATCH 301/418] fix overflow issue of readNumberWithVariableLength --- src/Functions/parseDateTime.cpp | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 1857764557a..7799520b8e5 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -1035,7 +1035,7 @@ namespace int repetitions, int max_digits_to_read, const String & fragment, - Int32 & number) + Int32 & result) { bool negative = false; if (allow_negative && cur < end && *cur == '-') @@ -1049,7 +1049,7 @@ namespace ++cur; } - number = 0; + Int64 number = 0; const Pos start = cur; if (is_year && repetitions == 2) { @@ -1102,6 +1102,15 @@ namespace if (negative) number *= -1; + /// Check if number exceeds the range of Int32 + if (number < std::numeric_limits::lowest() || number > std::numeric_limits::max()) + throw Exception( + ErrorCodes::CANNOT_PARSE_DATETIME, + "Unable to parse fragment {} from {} because number is out of range of Int32", + fragment, + std::string_view(start, cur - start)); + result = static_cast(number); + return cur; } From b5e17cc2c0b6fba5e955dc4d7b9631b0993147b4 Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 23 Mar 2023 06:24:58 +0000 Subject: [PATCH 302/418] fix --- src/Client/QueryFuzzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index ba4246e8d0d..cf288456f81 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -1108,7 +1108,7 @@ void QueryFuzzer::fuzz(ASTPtr & ast) /// Fuzzing EXPLAIN query to SELECT query randomly if (fuzz_rand() % 20 == 0 && explain_query->getExplainedQuery()->getQueryKind() == IAST::QueryKind::Select) { - ast = explain_query->getExplainedQuery(); + ast = explain_query->getExplainedQuery()->clone(); fuzz(ast); } else From a41ba1e6a71bfb203479009312b50ca7ced5d2bf Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 23 Mar 2023 12:04:24 +0800 Subject: [PATCH 303/418] Fix more cases --- src/Processors/LimitTransform.cpp | 2 +- src/Processors/LimitTransform.h | 6 ++ .../QueryPlan/DistributedCreateLocalPlan.cpp | 4 ++ src/Processors/QueryPlan/SortingStep.h | 2 +- src/Processors/Sources/RemoteSource.cpp | 13 ++++- src/Processors/Sources/RemoteSource.h | 2 + src/QueryPipeline/QueryPipeline.cpp | 55 +++++++++++++------ ...913_exact_rows_before_limit_full.reference | 18 ++++++ .../01913_exact_rows_before_limit_full.sql | 2 + ...1951_distributed_push_down_limit.reference | 7 +-- 10 files changed, 86 insertions(+), 25 deletions(-) diff --git a/src/Processors/LimitTransform.cpp b/src/Processors/LimitTransform.cpp index 2feee7e65b1..5e24062d67a 100644 --- a/src/Processors/LimitTransform.cpp +++ b/src/Processors/LimitTransform.cpp @@ -183,7 +183,7 @@ LimitTransform::Status LimitTransform::preparePair(PortsData & data) auto rows = data.current_chunk.getNumRows(); - if (rows_before_limit_at_least) + if (rows_before_limit_at_least && !data.input_port_has_counter) rows_before_limit_at_least->add(rows); /// Skip block (for 'always_read_till_end' case). diff --git a/src/Processors/LimitTransform.h b/src/Processors/LimitTransform.h index 791d4a44b24..33ff968985f 100644 --- a/src/Processors/LimitTransform.h +++ b/src/Processors/LimitTransform.h @@ -41,6 +41,11 @@ private: InputPort * input_port = nullptr; OutputPort * output_port = nullptr; bool is_finished = false; + + /// This flag is used to avoid counting rows multiple times before applying a limit + /// condition, which can happen through certain input ports like PartialSortingTransform and + /// RemoteSource. + bool input_port_has_counter = false; }; std::vector ports_data; @@ -67,6 +72,7 @@ public: OutputPort & getOutputPort() { return outputs.front(); } void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_at_least.swap(counter); } + void setInputPortHasCounter(size_t pos) { ports_data[pos].input_port_has_counter = true; } }; } diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index 2bb29a0b6fe..9b9cc221ca8 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -55,6 +55,10 @@ std::unique_ptr createLocalPlan( auto query_plan = std::make_unique(); auto new_context = Context::createCopy(context); + /// Do not push down limit to local plan, as it will break `rows_before_limit_at_least` counter. + if (processed_stage == QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit) + processed_stage = QueryProcessingStage::WithMergeableStateAfterAggregation; + /// Do not apply AST optimizations, because query /// is already optimized and some optimizations /// can be applied only for non-distributed tables diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h index 79b4d94c2fd..371a24ac6f2 100644 --- a/src/Processors/QueryPlan/SortingStep.h +++ b/src/Processors/QueryPlan/SortingStep.h @@ -102,7 +102,7 @@ private: SortDescription prefix_description; const SortDescription result_description; UInt64 limit; - bool always_read_till_end; + bool always_read_till_end = false; Settings sort_settings; diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index e203c9e12b2..fbc4406ac61 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -106,8 +106,13 @@ std::optional RemoteSource::tryGenerate() /// Get rows_before_limit result for remote query from ProfileInfo packet. query_executor->setProfileInfoCallback([this](const ProfileInfo & info) { - if (rows_before_limit && info.hasAppliedLimit()) - rows_before_limit->add(info.getRowsBeforeLimit()); + if (rows_before_limit) + { + if (info.hasAppliedLimit()) + rows_before_limit->add(info.getRowsBeforeLimit()); + else + manually_add_rows_before_limit_counter = true; /// Remote subquery doesn't contain a limit + } }); query_executor->sendQuery(); @@ -146,11 +151,15 @@ std::optional RemoteSource::tryGenerate() if (!block) { + if (manually_add_rows_before_limit_counter) + rows_before_limit->add(rows); + query_executor->finish(&read_context); return {}; } UInt64 num_rows = block.rows(); + rows += num_rows; Chunk chunk(block.getColumns(), num_rows); if (add_aggregation_info) diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index 40f1117e46e..e47bf3b578f 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -58,6 +58,8 @@ private: std::unique_ptr read_context; UUID uuid; int fd = -1; + size_t rows = 0; + bool manually_add_rows_before_limit_counter = false; }; /// Totals source from RemoteQueryExecutor. diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index f561f7a7396..f060f2f508f 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -131,33 +131,35 @@ static void initRowsBeforeLimit(IOutputFormat * output_format) { RowsBeforeLimitCounterPtr rows_before_limit_at_least; std::vector processors; - std::map limit_candidates; + std::map> limit_candidates; std::unordered_set visited; bool has_limit = false; struct QueuedEntry { IProcessor * processor; - IProcessor * limit_processor; + LimitTransform * limit_processor; + ssize_t limit_input_port; }; std::queue queue; - queue.push({ output_format, nullptr }); + queue.push({ output_format, nullptr, -1 }); visited.emplace(output_format); while (!queue.empty()) { auto * processor = queue.front().processor; auto * limit_processor = queue.front().limit_processor; + auto limit_input_port = queue.front().limit_input_port; queue.pop(); /// Set counter based on the following cases: /// 1. Remote: Set counter on Remote /// 2. Limit ... PartialSorting: Set counter on PartialSorting - /// 3. Limit ... TotalsHaving(with filter) ... Remote: Set counter on Limit + /// 3. Limit ... TotalsHaving(with filter) ... Remote: Set counter on the input port of Limit /// 4. Limit ... Remote: Set counter on Remote - /// 5. Limit ... : Set counter on Limit + /// 5. Limit ... : Set counter on the input port of Limit /// Case 1. if (typeid_cast(processor) && !limit_processor) @@ -166,7 +168,7 @@ static void initRowsBeforeLimit(IOutputFormat * output_format) continue; } - if (typeid_cast(processor)) + if (auto * limit = typeid_cast(processor)) { has_limit = true; @@ -174,8 +176,8 @@ static void initRowsBeforeLimit(IOutputFormat * output_format) if (limit_processor) continue; - limit_processor = processor; - limit_candidates.emplace(limit_processor, true); + limit_processor = limit; + limit_candidates[limit_processor] = {}; } else if (limit_processor) { @@ -183,7 +185,7 @@ static void initRowsBeforeLimit(IOutputFormat * output_format) if (typeid_cast(processor)) { processors.emplace_back(processor); - limit_candidates[limit_processor] = false; + limit_candidates[limit_processor].push_back(limit_input_port); continue; } @@ -198,7 +200,7 @@ static void initRowsBeforeLimit(IOutputFormat * output_format) if (typeid_cast(processor)) { processors.emplace_back(processor); - limit_candidates[limit_processor] = false; + limit_candidates[limit_processor].push_back(limit_input_port); continue; } } @@ -208,24 +210,43 @@ static void initRowsBeforeLimit(IOutputFormat * output_format) { auto * child_processor = &format->getPort(IOutputFormat::PortKind::Main).getOutputPort().getProcessor(); if (visited.emplace(child_processor).second) - queue.push({ child_processor, limit_processor }); + queue.push({ child_processor, limit_processor, limit_input_port }); continue; } - for (auto & child_port : processor->getInputs()) + if (limit_processor == processor) { - auto * child_processor = &child_port.getOutputPort().getProcessor(); - if (visited.emplace(child_processor).second) - queue.push({ child_processor, limit_processor }); + ssize_t i = 0; + for (auto & child_port : processor->getInputs()) + { + auto * child_processor = &child_port.getOutputPort().getProcessor(); + if (visited.emplace(child_processor).second) + queue.push({ child_processor, limit_processor, i }); + ++i; + } + } + else + { + for (auto & child_port : processor->getInputs()) + { + auto * child_processor = &child_port.getOutputPort().getProcessor(); + if (visited.emplace(child_processor).second) + queue.push({ child_processor, limit_processor, limit_input_port }); + } } } /// Case 5. - for (auto && [limit, valid] : limit_candidates) + for (auto && [limit, ports] : limit_candidates) { - if (valid) + /// If there are some input ports which don't have the counter, add it to LimitTransform. + if (ports.size() < limit->getInputs().size()) + { processors.push_back(limit); + for (auto port : ports) + limit->setInputPortHasCounter(port); + } } if (!processors.empty()) diff --git a/tests/queries/0_stateless/01913_exact_rows_before_limit_full.reference b/tests/queries/0_stateless/01913_exact_rows_before_limit_full.reference index 01f5c057be2..a0f4560ca1c 100644 --- a/tests/queries/0_stateless/01913_exact_rows_before_limit_full.reference +++ b/tests/queries/0_stateless/01913_exact_rows_before_limit_full.reference @@ -142,3 +142,21 @@ "rows_before_limit_at_least": 40 } +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0] + ], + + "rows": 1, + + "rows_before_limit_at_least": 20 +} diff --git a/tests/queries/0_stateless/01913_exact_rows_before_limit_full.sql b/tests/queries/0_stateless/01913_exact_rows_before_limit_full.sql index ec0d3d61bd4..84f97090169 100644 --- a/tests/queries/0_stateless/01913_exact_rows_before_limit_full.sql +++ b/tests/queries/0_stateless/01913_exact_rows_before_limit_full.sql @@ -24,4 +24,6 @@ set prefer_localhost_replica = 1; select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 30 limit 1 FORMAT JSONCompact; select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 20 order by i limit 1 FORMAT JSONCompact; +select * from (select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 10) limit 1 FORMAT JSONCompact; + drop table if exists test; diff --git a/tests/queries/0_stateless/01951_distributed_push_down_limit.reference b/tests/queries/0_stateless/01951_distributed_push_down_limit.reference index 7f73a8c6554..b9a7d17e955 100644 --- a/tests/queries/0_stateless/01951_distributed_push_down_limit.reference +++ b/tests/queries/0_stateless/01951_distributed_push_down_limit.reference @@ -13,8 +13,7 @@ Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) Sorting (Merge sorted streams after aggregation stage for ORDER BY) Union - Limit (preliminary LIMIT (with OFFSET)) - Sorting (Sorting for ORDER BY) - Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY))))) - ReadFromStorage (SystemNumbers) + Sorting (Sorting for ORDER BY) + Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY))))) + ReadFromStorage (SystemNumbers) ReadFromRemote (Read from remote replica) From b57e82ded6796de056d7ab0d57ad741d7d1ee09b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 22 Mar 2023 12:16:55 +0000 Subject: [PATCH 304/418] Add S3 write profile events --- src/Common/ProfileEvents.cpp | 6 +- src/IO/WriteBufferFromS3.cpp | 73 +++++++++++++------ .../test_profile_events_s3/test.py | 2 + 3 files changed, 58 insertions(+), 23 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index a031221a725..3cee4a8e718 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -349,12 +349,14 @@ The server successfully detected this situation and will download merged part fr M(DiskS3PutObject, "Number of DiskS3 API PutObject calls.") \ M(DiskS3GetObject, "Number of DiskS3 API GetObject calls.") \ \ - M(ReadBufferFromS3Microseconds, "Time spend in reading from S3.") \ - M(ReadBufferFromS3InitMicroseconds, "Time spend initializing connection to S3.") \ + M(ReadBufferFromS3Microseconds, "Time spent on reading from S3.") \ + M(ReadBufferFromS3InitMicroseconds, "Time spent initializing connection to S3.") \ M(ReadBufferFromS3Bytes, "Bytes read from S3.") \ M(ReadBufferFromS3RequestsErrors, "Number of exceptions while reading from S3.") \ \ + M(WriteBufferFromS3Microseconds, "Time spent on writing to S3.") \ M(WriteBufferFromS3Bytes, "Bytes written to S3.") \ + M(WriteBufferFromS3RequestsErrors, "Number of exceptions while writing to S3.") \ \ M(QueryMemoryLimitExceeded, "Number of times when memory limit exceeded for query.") \ \ diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 86a4cebb5d8..70b2d9924e0 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -23,6 +23,8 @@ namespace ProfileEvents { extern const Event WriteBufferFromS3Bytes; + extern const Event WriteBufferFromS3Microseconds; + extern const Event WriteBufferFromS3RequestsErrors; extern const Event S3WriteBytes; extern const Event S3CreateMultipartUpload; @@ -200,7 +202,11 @@ void WriteBufferFromS3::createMultipartUpload() if (write_settings.for_object_storage) ProfileEvents::increment(ProfileEvents::DiskS3CreateMultipartUpload); + Stopwatch watch; auto outcome = client_ptr->CreateMultipartUpload(req); + watch.stop(); + + ProfileEvents::increment(ProfileEvents::WriteBufferFromS3Microseconds, watch.elapsedMicroseconds()); if (outcome.IsSuccess()) { @@ -208,7 +214,10 @@ void WriteBufferFromS3::createMultipartUpload() LOG_TRACE(log, "Multipart upload has created. Bucket: {}, Key: {}, Upload id: {}", bucket, key, multipart_upload_id); } else + { + ProfileEvents::increment(ProfileEvents::WriteBufferFromS3RequestsErrors, 1); throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType()); + } } void WriteBufferFromS3::writePart() @@ -345,9 +354,13 @@ void WriteBufferFromS3::processUploadRequest(UploadPartTask & task) ResourceCost cost = task.req.GetContentLength(); ResourceGuard rlock(write_settings.resource_link, cost); + Stopwatch watch; auto outcome = client_ptr->UploadPart(task.req); + watch.stop(); rlock.unlock(); // Avoid acquiring other locks under resource lock + ProfileEvents::increment(ProfileEvents::WriteBufferFromS3Microseconds, watch.elapsedMicroseconds()); + if (outcome.IsSuccess()) { task.tag = outcome.GetResult().GetETag(); @@ -356,6 +369,7 @@ void WriteBufferFromS3::processUploadRequest(UploadPartTask & task) } else { + ProfileEvents::increment(ProfileEvents::WriteBufferFromS3RequestsErrors, 1); write_settings.resource_link.accumulate(cost); // We assume no resource was used in case of failure throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType()); } @@ -391,25 +405,34 @@ void WriteBufferFromS3::completeMultipartUpload() if (write_settings.for_object_storage) ProfileEvents::increment(ProfileEvents::DiskS3CompleteMultipartUpload); + Stopwatch watch; auto outcome = client_ptr->CompleteMultipartUpload(req); + watch.stop(); + + ProfileEvents::increment(ProfileEvents::WriteBufferFromS3Microseconds, watch.elapsedMicroseconds()); if (outcome.IsSuccess()) { LOG_TRACE(log, "Multipart upload has completed. Bucket: {}, Key: {}, Upload_id: {}, Parts: {}", bucket, key, multipart_upload_id, tags.size()); break; } - else if (outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) - { - /// For unknown reason, at least MinIO can respond with NO_SUCH_KEY for put requests - /// BTW, NO_SUCH_UPLOAD is expected error and we shouldn't retry it - LOG_INFO(log, "Multipart upload failed with NO_SUCH_KEY error for Bucket: {}, Key: {}, Upload_id: {}, Parts: {}, will retry", bucket, key, multipart_upload_id, tags.size()); - } else { - throw S3Exception( - outcome.GetError().GetErrorType(), - "Message: {}, Key: {}, Bucket: {}, Tags: {}", - outcome.GetError().GetMessage(), key, bucket, fmt::join(tags.begin(), tags.end(), " ")); + ProfileEvents::increment(ProfileEvents::WriteBufferFromS3RequestsErrors, 1); + + if (outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) + { + /// For unknown reason, at least MinIO can respond with NO_SUCH_KEY for put requests + /// BTW, NO_SUCH_UPLOAD is expected error and we shouldn't retry it + LOG_INFO(log, "Multipart upload failed with NO_SUCH_KEY error for Bucket: {}, Key: {}, Upload_id: {}, Parts: {}, will retry", bucket, key, multipart_upload_id, tags.size()); + } + else + { + throw S3Exception( + outcome.GetError().GetErrorType(), + "Message: {}, Key: {}, Bucket: {}, Tags: {}", + outcome.GetError().GetMessage(), key, bucket, fmt::join(tags.begin(), tags.end(), " ")); + } } } } @@ -501,28 +524,36 @@ void WriteBufferFromS3::processPutRequest(const PutObjectTask & task) ResourceCost cost = task.req.GetContentLength(); ResourceGuard rlock(write_settings.resource_link, cost); + Stopwatch watch; auto outcome = client_ptr->PutObject(task.req); + watch.stop(); rlock.unlock(); + ProfileEvents::increment(ProfileEvents::WriteBufferFromS3Microseconds, watch.elapsedMicroseconds()); + bool with_pool = static_cast(schedule); if (outcome.IsSuccess()) { LOG_TRACE(log, "Single part upload has completed. Bucket: {}, Key: {}, Object size: {}, WithPool: {}", bucket, key, task.req.GetContentLength(), with_pool); break; } - else if (outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) - { - write_settings.resource_link.accumulate(cost); // We assume no resource was used in case of failure - /// For unknown reason, at least MinIO can respond with NO_SUCH_KEY for put requests - LOG_INFO(log, "Single part upload failed with NO_SUCH_KEY error for Bucket: {}, Key: {}, Object size: {}, WithPool: {}, will retry", bucket, key, task.req.GetContentLength(), with_pool); - } else { - write_settings.resource_link.accumulate(cost); // We assume no resource was used in case of failure - throw S3Exception( - outcome.GetError().GetErrorType(), - "Message: {}, Key: {}, Bucket: {}, Object size: {}, WithPool: {}", - outcome.GetError().GetMessage(), key, bucket, task.req.GetContentLength(), with_pool); + ProfileEvents::increment(ProfileEvents::WriteBufferFromS3RequestsErrors, 1); + if (outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) + { + write_settings.resource_link.accumulate(cost); // We assume no resource was used in case of failure + /// For unknown reason, at least MinIO can respond with NO_SUCH_KEY for put requests + LOG_INFO(log, "Single part upload failed with NO_SUCH_KEY error for Bucket: {}, Key: {}, Object size: {}, WithPool: {}, will retry", bucket, key, task.req.GetContentLength(), with_pool); + } + else + { + write_settings.resource_link.accumulate(cost); // We assume no resource was used in case of failure + throw S3Exception( + outcome.GetError().GetErrorType(), + "Message: {}, Key: {}, Bucket: {}, Object size: {}, WithPool: {}", + outcome.GetError().GetMessage(), key, bucket, task.req.GetContentLength(), with_pool); + } } } } diff --git a/tests/integration/test_profile_events_s3/test.py b/tests/integration/test_profile_events_s3/test.py index b5e095514cb..5c56b5c05cb 100644 --- a/tests/integration/test_profile_events_s3/test.py +++ b/tests/integration/test_profile_events_s3/test.py @@ -37,6 +37,8 @@ init_list = { "ReadBufferFromS3InitMicroseconds": 0, "ReadBufferFromS3RequestsErrors": 0, "WriteBufferFromS3Bytes": 0, + "WriteBufferFromS3Microseconds": 0, + "WriteBufferFromS3RequestsErrors": 0, "S3ReadMicroseconds": 0, "S3ReadRequestsCount": 0, "S3ReadRequestsErrorsTotal": 0, From b3260f2201c79cfa0bd0000f68311ce2b8752207 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 Mar 2023 10:24:26 +0000 Subject: [PATCH 305/418] Docs: Fixup docs on Map-data-type support in data skipping indexes --- .../table-engines/mergetree-family/mergetree.md | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index acfc5382ab1..89554ebf1b0 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -432,18 +432,13 @@ Syntax: `tokenbf_v1(size_of_bloom_filter_in_bytes, number_of_hash_functions, ran - An experimental index to support approximate nearest neighbor (ANN) search. See [here](annindexes.md) for details. - An experimental inverted index to support full-text search. See [here](invertedindexes.md) for details. -## Example of index creation for Map data type +### Example of index creation for the Map data type -``` +Data skipping indexes can be created on the key or map parts of Map columns: + +```sql INDEX map_key_index mapKeys(map_column) TYPE bloom_filter GRANULARITY 1 -INDEX map_key_index mapValues(map_column) TYPE bloom_filter GRANULARITY 1 -``` - - -``` sql -INDEX sample_index (u64 * length(s)) TYPE minmax GRANULARITY 4 -INDEX sample_index2 (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARITY 4 -INDEX sample_index3 (lower(str), str) TYPE ngrambf_v1(3, 256, 2, 0) GRANULARITY 4 +INDEX map_value_index mapValues(map_column) TYPE bloom_filter GRANULARITY 1 ``` ### Functions Support {#functions-support} From 451df6db07b2150a74d580e9de1a917ffafafbad Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 Mar 2023 10:50:40 +0000 Subject: [PATCH 306/418] More fixup --- .../mergetree-family/mergetree.md | 25 ++++++++++++------- 1 file changed, 16 insertions(+), 9 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 89554ebf1b0..ea363fd062e 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -390,6 +390,22 @@ SELECT count() FROM table WHERE s < 'z' SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234 ``` +Data skipping indexes can also be created composite columns: + +```sql +-- on columns of type Map: +INDEX map_key_index mapKeys(map_column) TYPE bloom_filter +INDEX map_value_index mapValues(map_column) TYPE bloom_filter + +-- on columns of type Tuple: +INDEX tuple_1_index tuple_column.1 TYPE bloom_filter +INDEX tuple_2_index tuple_column.2 TYPE bloom_filter + +-- on columns of type Nested: +INDEX nested_1_index col.nested_col1 TYPE bloom_filter +INDEX nested_2_index col.nested_col2 TYPE bloom_filter +``` + ### Available Types of Indices {#available-types-of-indices} #### MinMax @@ -432,15 +448,6 @@ Syntax: `tokenbf_v1(size_of_bloom_filter_in_bytes, number_of_hash_functions, ran - An experimental index to support approximate nearest neighbor (ANN) search. See [here](annindexes.md) for details. - An experimental inverted index to support full-text search. See [here](invertedindexes.md) for details. -### Example of index creation for the Map data type - -Data skipping indexes can be created on the key or map parts of Map columns: - -```sql -INDEX map_key_index mapKeys(map_column) TYPE bloom_filter GRANULARITY 1 -INDEX map_value_index mapValues(map_column) TYPE bloom_filter GRANULARITY 1 -``` - ### Functions Support {#functions-support} Conditions in the `WHERE` clause contains calls of the functions that operate with columns. If the column is a part of an index, ClickHouse tries to use this index when performing the functions. ClickHouse supports different subsets of functions for using indexes. From 3953d77059ece81b2646f34eeb57bea7e8191b86 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 Mar 2023 10:51:25 +0000 Subject: [PATCH 307/418] + on --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index ea363fd062e..9fea158b100 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -390,7 +390,7 @@ SELECT count() FROM table WHERE s < 'z' SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234 ``` -Data skipping indexes can also be created composite columns: +Data skipping indexes can also be created on composite columns: ```sql -- on columns of type Map: From 02c666d06d279ceb9f94abc5811de35d9860041c Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 23 Mar 2023 11:14:12 +0000 Subject: [PATCH 308/418] fix --- src/Client/QueryFuzzer.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index cf288456f81..adcebf8a5f4 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -1108,8 +1108,9 @@ void QueryFuzzer::fuzz(ASTPtr & ast) /// Fuzzing EXPLAIN query to SELECT query randomly if (fuzz_rand() % 20 == 0 && explain_query->getExplainedQuery()->getQueryKind() == IAST::QueryKind::Select) { - ast = explain_query->getExplainedQuery()->clone(); - fuzz(ast); + auto select_query = explain_query->getExplainedQuery()->clone(); + fuzz(select_query); + ast = select_query; } else { From 7da53bf7e806dd76931877ec8eeaa77b9ff46da5 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 23 Mar 2023 12:34:23 +0100 Subject: [PATCH 309/418] Fixed tests --- src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 6e212cc1600..25a4579c73e 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -142,7 +142,7 @@ static bool isConditionGood(const RPNBuilderTreeNode & condition, const NameSet /** 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") + if (function_node.getFunctionName() != "equals" || function_node.getArgumentsSize() != 2) return false; auto lhs_argument = function_node.getArgumentAt(0); From 54314061ab5d12534af463bb7383552b7dde845a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 22 Mar 2023 23:06:33 +0100 Subject: [PATCH 310/418] fix logical error on cancellation --- src/Storages/Distributed/DistributedSink.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 73d3aebe0d0..11b938cd722 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -58,6 +58,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int TIMEOUT_EXCEEDED; extern const int TOO_LARGE_DISTRIBUTED_DEPTH; + extern const int ABORTED; } static Block adoptBlock(const Block & header, const Block & block, Poco::Logger * log) @@ -295,6 +296,10 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si auto thread_group = CurrentThread::getGroup(); return [this, thread_group, &job, ¤t_block, num_shards]() { + /// Avoid Logical error: 'Pipeline for PushingPipelineExecutor was finished before all data was inserted' (whatever it means) + if (isCancelled()) + throw Exception(ErrorCodes::ABORTED, "Writing job was cancelled"); + SCOPE_EXIT_SAFE( if (thread_group) CurrentThread::detachFromGroupIfNotDetached(); From 51765b51f3b8ddf498e727a807dcd725a51556c2 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 23 Mar 2023 15:27:44 +0300 Subject: [PATCH 311/418] Update 01072_window_view_multiple_columns_groupby.sh --- .../0_stateless/01072_window_view_multiple_columns_groupby.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 1b66beffe7e..062ecf648a7 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,6 @@ #!/usr/bin/env bash -# Tags: no-random-settings, no-fasttest +# Tags: no-random-settings, no-fasttest, no-parallel +# For unknown reason this test is flaky without no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From eeb5a4c24138695e1b72977f4b29eb45d423f968 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 23 Mar 2023 13:48:02 +0100 Subject: [PATCH 312/418] fix an unrelated flaky test --- .../test.py | 48 +++++++++---------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/tests/integration/test_sharding_key_from_default_column/test.py b/tests/integration/test_sharding_key_from_default_column/test.py index 1ecf96305a4..5bce3ee4169 100644 --- a/tests/integration/test_sharding_key_from_default_column/test.py +++ b/tests/integration/test_sharding_key_from_default_column/test.py @@ -49,14 +49,14 @@ def test_default_column(): "INSERT INTO TABLE dist (x) VALUES (1), (2), (3), (4)", settings=settings ) node1.query("SYSTEM FLUSH DISTRIBUTED dist") - assert node1.query("SELECT x, y, z FROM local") == TSV( + assert node1.query("SELECT x, y, z FROM local ORDER BY x") == TSV( [[2, 102, 104], [4, 104, 108]] ) - assert node2.query("SELECT x, y, z FROM local") == TSV( + assert node2.query("SELECT x, y, z FROM local ORDER BY x") == TSV( [[1, 101, 102], [3, 103, 106]] ) - assert node1.query("SELECT x, y, z FROM dist") == TSV( - [[2, 102, 104], [4, 104, 108], [1, 101, 102], [3, 103, 106]] + assert node1.query("SELECT x, y, z FROM dist ORDER BY x") == TSV( + [[1, 101, 102], [2, 102, 104], [3, 103, 106], [4, 104, 108]] ) # INSERT INTO TABLE dist (x, y) @@ -66,12 +66,12 @@ def test_default_column(): settings=settings, ) node1.query("SYSTEM FLUSH DISTRIBUTED dist") - assert node1.query("SELECT x, y, z FROM local") == TSV([[2, 22, 24]]) - assert node2.query("SELECT x, y, z FROM local") == TSV( + assert node1.query("SELECT x, y, z FROM local ORDER BY x") == TSV([[2, 22, 24]]) + assert node2.query("SELECT x, y, z FROM local ORDER BY x") == TSV( [[1, 11, 12], [3, 33, 36]] ) - assert node1.query("SELECT x, y, z FROM dist") == TSV( - [[2, 22, 24], [1, 11, 12], [3, 33, 36]] + assert node1.query("SELECT x, y, z FROM dist ORDER BY x") == TSV( + [[1, 11, 12], [2, 22, 24], [3, 33, 36]] ) @@ -96,14 +96,14 @@ def test_materialized_column_allow_insert_materialized(): "INSERT INTO TABLE dist (x) VALUES (1), (2), (3), (4)", settings=settings ) node1.query("SYSTEM FLUSH DISTRIBUTED dist") - assert node1.query("SELECT x, y, z FROM local") == TSV( + assert node1.query("SELECT x, y, z FROM local ORDER BY x") == TSV( [[2, 102, 104], [4, 104, 108]] ) - assert node2.query("SELECT x, y, z FROM local") == TSV( + assert node2.query("SELECT x, y, z FROM local ORDER BY x") == TSV( [[1, 101, 102], [3, 103, 106]] ) - assert node1.query("SELECT x, y, z FROM dist") == TSV( - [[2, 102, 104], [4, 104, 108], [1, 101, 102], [3, 103, 106]] + assert node1.query("SELECT x, y, z FROM dist ORDER BY x") == TSV( + [[1, 101, 102], [2, 102, 104], [3, 103, 106], [4, 104, 108]] ) # INSERT INTO TABLE dist (x, y) @@ -113,12 +113,12 @@ def test_materialized_column_allow_insert_materialized(): settings=settings, ) node1.query("SYSTEM FLUSH DISTRIBUTED dist") - assert node1.query("SELECT x, y, z FROM local") == TSV([[2, 22, 24]]) - assert node2.query("SELECT x, y, z FROM local") == TSV( + assert node1.query("SELECT x, y, z FROM local ORDER BY x") == TSV([[2, 22, 24]]) + assert node2.query("SELECT x, y, z FROM local ORDER BY x") == TSV( [[1, 11, 12], [3, 33, 36]] ) - assert node1.query("SELECT x, y, z FROM dist") == TSV( - [[2, 22, 24], [1, 11, 12], [3, 33, 36]] + assert node1.query("SELECT x, y, z FROM dist ORDER BY x") == TSV( + [[1, 11, 12], [2, 22, 24], [3, 33, 36]] ) @@ -143,14 +143,14 @@ def test_materialized_column_disallow_insert_materialized(): "INSERT INTO TABLE dist (x) VALUES (1), (2), (3), (4)", settings=settings ) node1.query("SYSTEM FLUSH DISTRIBUTED dist") - assert node1.query("SELECT x, y, z FROM local") == TSV( + assert node1.query("SELECT x, y, z FROM local ORDER BY x") == TSV( [[2, 202, -200], [4, 204, -200]] ) - assert node2.query("SELECT x, y, z FROM local") == TSV( + assert node2.query("SELECT x, y, z FROM local ORDER BY x") == TSV( [[1, 201, -200], [3, 203, -200]] ) - assert node1.query("SELECT x, y, z FROM dist") == TSV( - [[2, 202, -200], [4, 204, -200], [1, 201, -200], [3, 203, -200]] + assert node1.query("SELECT x, y, z FROM dist ORDER BY x") == TSV( + [[1, 201, -200], [2, 202, -200], [3, 203, -200], [4, 204, -200]] ) # INSERT INTO TABLE dist (x, y) @@ -183,12 +183,12 @@ def test_materialized_column_disallow_insert_materialized_different_shards(): "INSERT INTO TABLE dist (x) VALUES (1), (2), (3), (4)", settings=settings ) node1.query("SYSTEM FLUSH DISTRIBUTED dist") - assert node1.query("SELECT x, y, z FROM local") == TSV( + assert node1.query("SELECT x, y, z FROM local ORDER BY x") == TSV( [[1, 201, -200], [3, 203, -200]] ) - assert node2.query("SELECT x, y, z FROM local") == TSV( + assert node2.query("SELECT x, y, z FROM local ORDER BY x") == TSV( [[2, 202, -200], [4, 204, -200]] ) - assert node1.query("SELECT x, y, z FROM dist") == TSV( - [[1, 201, -200], [3, 203, -200], [2, 202, -200], [4, 204, -200]] + assert node1.query("SELECT x, y, z FROM dist ORDER BY x") == TSV( + [[1, 201, -200], [2, 202, -200], [3, 203, -200], [4, 204, -200]] ) From 5e95a37c526875ad1791240895f0f5cb72632e41 Mon Sep 17 00:00:00 2001 From: MeenaRenganathan22 Date: Thu, 23 Mar 2023 07:05:51 -0700 Subject: [PATCH 313/418] Fix decimal-256 text output issue on s390x --- base/base/wide_integer_impl.h | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/base/base/wide_integer_impl.h b/base/base/wide_integer_impl.h index 4a54c0fb2a4..874a66440a7 100644 --- a/base/base/wide_integer_impl.h +++ b/base/base/wide_integer_impl.h @@ -732,9 +732,10 @@ public: if (std::numeric_limits::is_signed && (is_negative(lhs) != is_negative(rhs))) return is_negative(rhs); + integer t = rhs; for (unsigned i = 0; i < item_count; ++i) { - base_type rhs_item = get_item(rhs, big(i)); + base_type rhs_item = get_item(t, big(i)); if (lhs.items[big(i)] != rhs_item) return lhs.items[big(i)] > rhs_item; @@ -757,9 +758,10 @@ public: if (std::numeric_limits::is_signed && (is_negative(lhs) != is_negative(rhs))) return is_negative(lhs); + integer t = rhs; for (unsigned i = 0; i < item_count; ++i) { - base_type rhs_item = get_item(rhs, big(i)); + base_type rhs_item = get_item(t, big(i)); if (lhs.items[big(i)] != rhs_item) return lhs.items[big(i)] < rhs_item; @@ -779,9 +781,10 @@ public: { if constexpr (should_keep_size()) { + integer t = rhs; for (unsigned i = 0; i < item_count; ++i) { - base_type rhs_item = get_item(rhs, any(i)); + base_type rhs_item = get_item(t, any(i)); if (lhs.items[any(i)] != rhs_item) return false; From 08583c840535529d297f7eab6d72271c4b7752b6 Mon Sep 17 00:00:00 2001 From: MeenaRenganathan22 Date: Thu, 23 Mar 2023 07:17:37 -0700 Subject: [PATCH 314/418] Fixed the style issue --- base/base/wide_integer_impl.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/base/base/wide_integer_impl.h b/base/base/wide_integer_impl.h index 874a66440a7..30d08be2e4f 100644 --- a/base/base/wide_integer_impl.h +++ b/base/base/wide_integer_impl.h @@ -732,7 +732,7 @@ public: if (std::numeric_limits::is_signed && (is_negative(lhs) != is_negative(rhs))) return is_negative(rhs); - integer t = rhs; + integer t = rhs; for (unsigned i = 0; i < item_count; ++i) { base_type rhs_item = get_item(t, big(i)); @@ -758,7 +758,7 @@ public: if (std::numeric_limits::is_signed && (is_negative(lhs) != is_negative(rhs))) return is_negative(lhs); - integer t = rhs; + integer t = rhs; for (unsigned i = 0; i < item_count; ++i) { base_type rhs_item = get_item(t, big(i)); @@ -781,7 +781,7 @@ public: { if constexpr (should_keep_size()) { - integer t = rhs; + integer t = rhs; for (unsigned i = 0; i < item_count; ++i) { base_type rhs_item = get_item(t, any(i)); From 4b635afe49a83a079fec5c298fedf809db550fad Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 23 Mar 2023 15:18:37 +0100 Subject: [PATCH 315/418] Set `max_final_threads` to number of cores by default (#47915) --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index accfe387137..e948f063035 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -44,7 +44,7 @@ class IColumn; M(UInt64, max_joined_block_size_rows, DEFAULT_BLOCK_SIZE, "Maximum block size for JOIN result (if join algorithm supports it). 0 means unlimited.", 0) \ M(UInt64, max_insert_threads, 0, "The maximum number of threads to execute the INSERT SELECT query. Values 0 or 1 means that INSERT SELECT is not run in parallel. Higher values will lead to higher memory usage. Parallel INSERT SELECT has effect only if the SELECT part is run on parallel, see 'max_threads' setting.", 0) \ M(UInt64, max_insert_delayed_streams_for_parallel_write, 0, "The maximum number of streams (columns) to delay final part flush. Default - auto (1000 in case of underlying storage supports parallel write, for example S3 and disabled otherwise)", 0) \ - M(UInt64, max_final_threads, 16, "The maximum number of threads to read from table with FINAL.", 0) \ + M(MaxThreads, max_final_threads, 0, "The maximum number of threads to read from table with FINAL.", 0) \ M(MaxThreads, max_threads, 0, "The maximum number of threads to execute the request. By default, it is determined automatically.", 0) \ M(MaxThreads, max_download_threads, 4, "The maximum number of threads to download data (e.g. for URL engine).", 0) \ M(UInt64, max_download_buffer_size, 10*1024*1024, "The maximal size of buffer for parallel downloading (e.g. for URL engine) per each thread.", 0) \ From 5b263b9ffe260eb2b0f3592285842e5e39310986 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 23 Mar 2023 15:36:17 +0100 Subject: [PATCH 316/418] fix --- docker/test/stateless/run.sh | 1 + src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp | 3 +++ 2 files changed, 4 insertions(+) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index ade59224035..e509809c028 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -170,6 +170,7 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] fi rg -Fa "" /var/log/clickhouse-server/clickhouse-server.log ||: +rg -A50 -Fa "============" /var/log/clickhouse-server/stderr.log ||: zstd --threads=0 < /var/log/clickhouse-server/clickhouse-server.log > /test_output/clickhouse-server.log.zst & # Compress tables. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index 27ab56d4773..0882ff5a0bc 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -119,6 +119,9 @@ void ReplicatedMergeTreePartCheckThread::cancelRemovedPartsCheck(const MergeTree }); parts_queue.erase(new_end, parts_queue.end()); + + for (const auto & elem : removed_parts) + parts_set.erase(elem); } size_t ReplicatedMergeTreePartCheckThread::size() const From 2b439f079ef8f3410b5ace1bdba46e4ceb361ff4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 23 Mar 2023 16:44:15 +0200 Subject: [PATCH 317/418] Make backup_threads/restore_threads server settings (#47881) Those settings can be applied only at start, so it is better to have them in configuration file instead of user profile settings. Signed-off-by: Azat Khuzhin --- src/Core/ServerSettings.h | 2 ++ src/Core/Settings.h | 4 ++-- src/Interpreters/Context.cpp | 7 ++++++- 3 files changed, 10 insertions(+), 3 deletions(-) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index abc5b4d14d8..4fc721e70a2 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -24,6 +24,8 @@ namespace DB M(UInt64, max_backups_io_thread_pool_size, 1000, "The maximum number of threads that would be used for IO operations for BACKUP queries", 0) \ M(UInt64, max_backups_io_thread_pool_free_size, 0, "Max free size for backups IO thread pool.", 0) \ M(UInt64, backups_io_thread_pool_queue_size, 0, "Queue size for backups IO thread pool.", 0) \ + M(UInt64, backup_threads, 16, "The maximum number of threads to execute BACKUP requests.", 0) \ + M(UInt64, restore_threads, 16, "The maximum number of threads to execute RESTORE requests.", 0) \ M(Int32, max_connections, 1024, "Max server connections.", 0) \ M(UInt32, asynchronous_metrics_update_period_s, 1, "Period in seconds for updating asynchronous metrics.", 0) \ M(UInt32, asynchronous_heavy_metrics_update_period_s, 120, "Period in seconds for updating asynchronous metrics.", 0) \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e948f063035..689301e62b8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -414,8 +414,6 @@ class IColumn; M(UInt64, max_temporary_data_on_disk_size_for_user, 0, "The maximum amount of data consumed by temporary files on disk in bytes for all concurrently running user queries. Zero means unlimited.", 0)\ M(UInt64, max_temporary_data_on_disk_size_for_query, 0, "The maximum amount of data consumed by temporary files on disk in bytes for all concurrently running queries. Zero means unlimited.", 0)\ \ - M(UInt64, backup_threads, 16, "The maximum number of threads to execute BACKUP requests.", 0) \ - M(UInt64, restore_threads, 16, "The maximum number of threads to execute RESTORE requests.", 0) \ M(UInt64, backup_keeper_max_retries, 20, "Max retries for keeper operations during backup", 0) \ M(UInt64, backup_keeper_retry_initial_backoff_ms, 100, "Initial backoff timeout for [Zoo]Keeper operations during backup", 0) \ M(UInt64, backup_keeper_retry_max_backoff_ms, 5000, "Max backoff timeout for [Zoo]Keeper operations during backup", 0) \ @@ -761,6 +759,8 @@ class IColumn; MAKE_OBSOLETE(M, Milliseconds, async_insert_cleanup_timeout_ms, 1000) \ MAKE_OBSOLETE(M, Bool, optimize_fuse_sum_count_avg, 0) \ MAKE_OBSOLETE(M, Seconds, drain_timeout, 3) \ + MAKE_OBSOLETE(M, UInt64, backup_threads, 16) \ + MAKE_OBSOLETE(M, UInt64, restore_threads, 16) \ /** The section above is for obsolete settings. Do not add anything there. */ diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 4e882f3ab5b..5b412416747 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1918,8 +1918,13 @@ BackupsWorker & Context::getBackupsWorker() const const bool allow_concurrent_backups = this->getConfigRef().getBool("backups.allow_concurrent_backups", true); const bool allow_concurrent_restores = this->getConfigRef().getBool("backups.allow_concurrent_restores", true); + const auto & config = getConfigRef(); + const auto & settings = getSettingsRef(); + UInt64 backup_threads = config.getUInt64("backup_threads", settings.backup_threads); + UInt64 restore_threads = config.getUInt64("restore_threads", settings.restore_threads); + if (!shared->backups_worker) - shared->backups_worker.emplace(getSettingsRef().backup_threads, getSettingsRef().restore_threads, allow_concurrent_backups, allow_concurrent_restores); + shared->backups_worker.emplace(backup_threads, restore_threads, allow_concurrent_backups, allow_concurrent_restores); return *shared->backups_worker; } From 50e1eedd4766ed8c2e34339d78f000d63f4d5191 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Mar 2023 17:48:06 +0300 Subject: [PATCH 318/418] Add a test for #17756 (#47739) --- tests/queries/0_stateless/02693_multiple_joins_in.reference | 0 tests/queries/0_stateless/02693_multiple_joins_in.sql | 3 +++ 2 files changed, 3 insertions(+) create mode 100644 tests/queries/0_stateless/02693_multiple_joins_in.reference create mode 100644 tests/queries/0_stateless/02693_multiple_joins_in.sql diff --git a/tests/queries/0_stateless/02693_multiple_joins_in.reference b/tests/queries/0_stateless/02693_multiple_joins_in.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02693_multiple_joins_in.sql b/tests/queries/0_stateless/02693_multiple_joins_in.sql new file mode 100644 index 00000000000..8be52948d26 --- /dev/null +++ b/tests/queries/0_stateless/02693_multiple_joins_in.sql @@ -0,0 +1,3 @@ +create temporary table temp_table3(val0 UInt64) ENGINE=Memory(); +select * from (select 1 as id) t1 inner join (select 1 as id) t2 on t1.id=t2.id inner join (select 1 as id) t3 on t1.id=t3.id where t1.id in temp_table3; +select * from (select 1 as id) t1 inner join (select 1 as id) t2 on t1.id=t2.id where t1.id in temp_table3; From 72c6084267da929019068fae3d78741b8225efcf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Mar 2023 17:49:54 +0300 Subject: [PATCH 319/418] Add a test for #23804 (#47737) --- ...tiple_joins_backtick_identifiers.reference | 0 ...91_multiple_joins_backtick_identifiers.sql | 49 +++++++++++++++++++ 2 files changed, 49 insertions(+) create mode 100644 tests/queries/0_stateless/02691_multiple_joins_backtick_identifiers.reference create mode 100644 tests/queries/0_stateless/02691_multiple_joins_backtick_identifiers.sql diff --git a/tests/queries/0_stateless/02691_multiple_joins_backtick_identifiers.reference b/tests/queries/0_stateless/02691_multiple_joins_backtick_identifiers.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02691_multiple_joins_backtick_identifiers.sql b/tests/queries/0_stateless/02691_multiple_joins_backtick_identifiers.sql new file mode 100644 index 00000000000..4a56c12866e --- /dev/null +++ b/tests/queries/0_stateless/02691_multiple_joins_backtick_identifiers.sql @@ -0,0 +1,49 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS t3; + +CREATE TABLE t1 (`1a` Nullable(Int64), `2b` Nullable(String)) engine = Memory; +CREATE TABLE t2 (`3c` Nullable(Int64), `4d` Nullable(String)) engine = Memory; +CREATE TABLE t3 (`5e` Nullable(Int64), `6f` Nullable(String)) engine = Memory; + +SELECT + `1a`, + `2b` +FROM t1 AS tt1 +INNER JOIN +( + SELECT `3c` + FROM t2 +) AS tt2 ON tt1.`1a` = tt2.`3c` +INNER JOIN +( + SELECT `6f` + FROM t3 +) AS tt3 ON tt1.`2b` = tt3.`6f`; + +DROP TABLE t1; +DROP TABLE t2; +DROP TABLE t3; + +CREATE TABLE t1 (`a` Nullable(Int64), `b` Nullable(String)) engine = Memory; +CREATE TABLE t2 (`c` Nullable(Int64), `d` Nullable(String)) engine = Memory; +CREATE TABLE t3 (`e` Nullable(Int64), `f` Nullable(String)) engine = Memory; + +SELECT + a, + b +FROM t1 AS tt1 +INNER JOIN +( + SELECT c + FROM t2 +) AS tt2 ON tt1.a = tt2.c +INNER JOIN +( + SELECT f + FROM t3 +) AS tt3 ON tt1.b = tt3.f; + +DROP TABLE t1; +DROP TABLE t2; +DROP TABLE t3; From 8ff01429e1e75b6d773c154e743b1a61a3d6ed4c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 23 Mar 2023 15:49:59 +0100 Subject: [PATCH 320/418] Upload build reports to S3_TEST_REPORTS_BUCKET --- tests/ci/build_report_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 520051bd425..32cbaf08f07 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -247,7 +247,7 @@ def main(): str(pr_info.number) + "/" + pr_info.sha + "/" + context_name_for_path ) - url = s3_helper.upload_build_file_to_s3( + url = s3_helper.upload_test_report_to_s3( report_path, s3_path_prefix + "/report.html" ) logging.info("Report url %s", url) From 4fcc5bbea75c47496c617ce8f28b22ff13d99c7a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Mar 2023 17:50:56 +0300 Subject: [PATCH 321/418] Add a test for #18937 (#47738) --- .../02692_multiple_joins_unicode.reference | 2 ++ .../02692_multiple_joins_unicode.sql | 24 +++++++++++++++++++ 2 files changed, 26 insertions(+) create mode 100644 tests/queries/0_stateless/02692_multiple_joins_unicode.reference create mode 100644 tests/queries/0_stateless/02692_multiple_joins_unicode.sql diff --git a/tests/queries/0_stateless/02692_multiple_joins_unicode.reference b/tests/queries/0_stateless/02692_multiple_joins_unicode.reference new file mode 100644 index 00000000000..3aca4fe5c7b --- /dev/null +++ b/tests/queries/0_stateless/02692_multiple_joins_unicode.reference @@ -0,0 +1,2 @@ +2021-01-01 上海市 启用 +2021-01-02 北京市 停用 diff --git a/tests/queries/0_stateless/02692_multiple_joins_unicode.sql b/tests/queries/0_stateless/02692_multiple_joins_unicode.sql new file mode 100644 index 00000000000..d622c556e56 --- /dev/null +++ b/tests/queries/0_stateless/02692_multiple_joins_unicode.sql @@ -0,0 +1,24 @@ +DROP TABLE IF EXISTS store; +DROP TABLE IF EXISTS location; +DROP TABLE IF EXISTS sales; + +CREATE TABLE store (id UInt32, "名称" String, "状态" String) ENGINE=MergeTree() Order by id; +CREATE TABLE location (id UInt32, name String) ENGINE=MergeTree() Order by id; +CREATE TABLE sales ("日期" Date, "店铺" UInt32, "地址" UInt32, "销售额" Float32) ENGINE=MergeTree() Order by "日期"; + +INSERT INTO store VALUES (1,'店铺1','启用'),(2,'店铺2','停用'); +INSERT INTO location VALUES (1,'上海市'),(2,'北京市'); +INSERT INTO sales VALUES ('2021-01-01',1,1,10),('2021-01-02',2,2,20); + +SELECT + `日期`, + location.name, + store.`状态` +FROM sales +LEFT JOIN store ON store.id = `店铺` +LEFT JOIN location ON location.id = `地址` +ORDER BY 1, 2, 3; + +DROP TABLE store; +DROP TABLE location; +DROP TABLE sales; From 3bf2411be1f80a26b97e0ce0f555c1b7c6ab4198 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Tue, 21 Feb 2023 07:27:34 -0800 Subject: [PATCH 322/418] Make tiny improvements --- src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp | 2 +- src/Processors/ISource.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 98f47b30b1b..3bc2284a567 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -251,7 +251,7 @@ bool ValuesBlockInputFormat::tryParseExpressionUsingTemplate(MutableColumnPtr & /// Do not use this template anymore templates[column_idx].reset(); buf->rollbackToCheckpoint(); - *token_iterator = start; + token_iterator = start; /// It will deduce new template or fallback to slow SQL parser return parseExpression(*column, column_idx); diff --git a/src/Processors/ISource.cpp b/src/Processors/ISource.cpp index 221b98642f6..e6aeffa4807 100644 --- a/src/Processors/ISource.cpp +++ b/src/Processors/ISource.cpp @@ -72,7 +72,7 @@ void ISource::progress(size_t read_rows, size_t read_bytes) std::optional ISource::getReadProgress() { - if (finished && read_progress.read_bytes == 0 && read_progress.read_bytes == 0 && read_progress.total_rows_approx == 0) + if (finished && read_progress.read_bytes == 0 && read_progress.total_rows_approx == 0) return {}; ReadProgressCounters res_progress; From 531068df168b770bfe5ac6e543f40de406aa7ffa Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Thu, 23 Mar 2023 16:01:04 +0100 Subject: [PATCH 323/418] Fix exception type in arrayElement (#47909) --- src/Functions/array/arrayElement.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Functions/array/arrayElement.cpp b/src/Functions/array/arrayElement.cpp index 299f25b8292..789b2b55ca5 100644 --- a/src/Functions/array/arrayElement.cpp +++ b/src/Functions/array/arrayElement.cpp @@ -1179,12 +1179,15 @@ ColumnPtr FunctionArrayElement::perform(const ColumnsWithTypeAndName & arguments || (res = executeArgument(arguments, result_type, builder, input_rows_count)) || (res = executeArgument(arguments, result_type, builder, input_rows_count)) || (res = executeArgument(arguments, result_type, builder, input_rows_count)))) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Second argument for function {} must have UInt or Int type.", getName()); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Second argument for function {} must have UInt or Int type", getName()); } else { Field index = (*arguments[1].column)[0]; + if (index.getType() != Field::Types::UInt64 && index.getType() != Field::Types::Int64) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Second argument for function {} must have UInt or Int type", getName()); + if (builder) builder.initSink(input_rows_count); From 49079ee85c9a57091d26c69c957d28d2dd01d51c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 23 Mar 2023 15:31:36 +0000 Subject: [PATCH 324/418] fix build --- src/IO/ZlibInflatingReadBuffer.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/IO/ZlibInflatingReadBuffer.cpp b/src/IO/ZlibInflatingReadBuffer.cpp index 8f730a06829..09e4fce7c4c 100644 --- a/src/IO/ZlibInflatingReadBuffer.cpp +++ b/src/IO/ZlibInflatingReadBuffer.cpp @@ -64,7 +64,9 @@ bool ZlibInflatingReadBuffer::nextImpl() { in->nextIfAtEnd(); zstr.next_in = reinterpret_cast(in->position()); - zstr.avail_in = static_cast(std::min(in->buffer().end() - in->position(), static_cast(max_buffer_size))); + zstr.avail_in = static_cast(std::min( + static_cast(in->buffer().end() - in->position()), + static_cast(max_buffer_size))); } /// init output bytes (place, where decompressed data will be) From 0ee8dfad53fa79584dd3b9733144949e0f33ba12 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 23 Mar 2023 15:33:23 +0000 Subject: [PATCH 325/418] apply black formatter --- docker/test/stateful/s3downloader | 65 +- tests/clickhouse-test | 61 +- tests/integration/runner | 19 +- .../0_stateless/00386_long_in_pk.python | 61 +- ...411_long_accurate_number_comparison.python | 130 ++-- .../0_stateless/00646_url_engine.python | 165 ++++-- .../queries/0_stateless/00990_hasToken.python | 111 ++-- .../00991_live_view_watch_event_live.python | 51 +- .../00991_live_view_watch_http.python | 45 +- .../0_stateless/01558_ttest_scipy.python | 90 +-- .../01561_mann_whitney_scipy.python | 51 +- .../0_stateless/01626_cnf_fuzz_long.python | 18 +- .../01654_test_writer_block_sequence.python | 20 +- .../01854_HTTP_dict_decompression.python | 97 +-- .../0_stateless/02010_lc_native.python | 252 ++++---- .../queries/0_stateless/02126_url_auth.python | 140 +++-- .../02158_proportions_ztest_cmp.python | 184 ++++-- .../0_stateless/02158_ztest_cmp.python | 109 +++- .../02187_async_inserts_all_formats.python | 44 +- .../0_stateless/02205_HTTP_user_agent.python | 54 +- .../0_stateless/02233_HTTP_ranged.python | 14 +- .../0_stateless/02294_anova_cmp.python | 21 +- .../02346_read_in_order_fixed_prefix.python | 8 +- .../02403_big_http_chunk_size.python | 5 +- .../02458_insert_select_progress_tcp.python | 113 ++-- .../02473_multistep_prewhere.python | 191 ++++-- .../02473_multistep_split_prewhere.python | 161 +++-- .../02481_async_insert_dedup.python | 62 +- .../clickhouse-diagnostics | 561 ++++++++++-------- utils/s3tools/s3uploader | 140 +++-- utils/test_history/test-history | 15 +- 31 files changed, 1999 insertions(+), 1059 deletions(-) diff --git a/docker/test/stateful/s3downloader b/docker/test/stateful/s3downloader index b1302877d6a..96f2aa96dd5 100755 --- a/docker/test/stateful/s3downloader +++ b/docker/test/stateful/s3downloader @@ -10,31 +10,38 @@ import requests import tempfile -DEFAULT_URL = 'https://clickhouse-datasets.s3.amazonaws.com' +DEFAULT_URL = "https://clickhouse-datasets.s3.amazonaws.com" AVAILABLE_DATASETS = { - 'hits': 'hits_v1.tar', - 'visits': 'visits_v1.tar', + "hits": "hits_v1.tar", + "visits": "visits_v1.tar", } RETRIES_COUNT = 5 + def _get_temp_file_name(): - return os.path.join(tempfile._get_default_tempdir(), next(tempfile._get_candidate_names())) + return os.path.join( + tempfile._get_default_tempdir(), next(tempfile._get_candidate_names()) + ) + def build_url(base_url, dataset): - return os.path.join(base_url, dataset, 'partitions', AVAILABLE_DATASETS[dataset]) + return os.path.join(base_url, dataset, "partitions", AVAILABLE_DATASETS[dataset]) + def dowload_with_progress(url, path): logging.info("Downloading from %s to temp path %s", url, path) for i in range(RETRIES_COUNT): try: - with open(path, 'wb') as f: + with open(path, "wb") as f: response = requests.get(url, stream=True) response.raise_for_status() - total_length = response.headers.get('content-length') + total_length = response.headers.get("content-length") if total_length is None or int(total_length) == 0: - logging.info("No content-length, will download file without progress") + logging.info( + "No content-length, will download file without progress" + ) f.write(response.content) else: dl = 0 @@ -46,7 +53,11 @@ def dowload_with_progress(url, path): if sys.stdout.isatty(): done = int(50 * dl / total_length) percent = int(100 * float(dl) / total_length) - sys.stdout.write("\r[{}{}] {}%".format('=' * done, ' ' * (50-done), percent)) + sys.stdout.write( + "\r[{}{}] {}%".format( + "=" * done, " " * (50 - done), percent + ) + ) sys.stdout.flush() break except Exception as ex: @@ -56,14 +67,21 @@ def dowload_with_progress(url, path): if os.path.exists(path): os.remove(path) else: - raise Exception("Cannot download dataset from {}, all retries exceeded".format(url)) + raise Exception( + "Cannot download dataset from {}, all retries exceeded".format(url) + ) sys.stdout.write("\n") logging.info("Downloading finished") + def unpack_to_clickhouse_directory(tar_path, clickhouse_path): - logging.info("Will unpack data from temp path %s to clickhouse db %s", tar_path, clickhouse_path) - with tarfile.open(tar_path, 'r') as comp_file: + logging.info( + "Will unpack data from temp path %s to clickhouse db %s", + tar_path, + clickhouse_path, + ) + with tarfile.open(tar_path, "r") as comp_file: comp_file.extractall(path=clickhouse_path) logging.info("Unpack finished") @@ -72,15 +90,21 @@ if __name__ == "__main__": logging.basicConfig(level=logging.INFO) parser = argparse.ArgumentParser( - description="Simple tool for dowloading datasets for clickhouse from S3") + description="Simple tool for dowloading datasets for clickhouse from S3" + ) - parser.add_argument('--dataset-names', required=True, nargs='+', choices=list(AVAILABLE_DATASETS.keys())) - parser.add_argument('--url-prefix', default=DEFAULT_URL) - parser.add_argument('--clickhouse-data-path', default='/var/lib/clickhouse/') + parser.add_argument( + "--dataset-names", + required=True, + nargs="+", + choices=list(AVAILABLE_DATASETS.keys()), + ) + parser.add_argument("--url-prefix", default=DEFAULT_URL) + parser.add_argument("--clickhouse-data-path", default="/var/lib/clickhouse/") args = parser.parse_args() datasets = args.dataset_names - logging.info("Will fetch following datasets: %s", ', '.join(datasets)) + logging.info("Will fetch following datasets: %s", ", ".join(datasets)) for dataset in datasets: logging.info("Processing %s", dataset) temp_archive_path = _get_temp_file_name() @@ -92,10 +116,11 @@ if __name__ == "__main__": logging.info("Some exception occured %s", str(ex)) raise finally: - logging.info("Will remove downloaded file %s from filesystem if it exists", temp_archive_path) + logging.info( + "Will remove downloaded file %s from filesystem if it exists", + temp_archive_path, + ) if os.path.exists(temp_archive_path): os.remove(temp_archive_path) logging.info("Processing of %s finished", dataset) logging.info("Fetch finished, enjoy your tables!") - - diff --git a/tests/clickhouse-test b/tests/clickhouse-test index aec52981724..2de5b4e1333 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -77,7 +77,7 @@ def trim_for_log(s): return s lines = s.splitlines() if len(lines) > 10000: - separator = "-" * 40 + str(len(lines) - 10000) + " lines are hidden" + "-" * 40 + separator = "-" * 40 + str(len(lines) - 10000) + " lines are hidden" + "-" * 40 return "\n".join(lines[:5000] + [] + [separator] + [] + lines[-5000:]) else: return "\n".join(lines) @@ -95,7 +95,13 @@ class HTTPError(Exception): # Helpers to execute queries via HTTP interface. def clickhouse_execute_http( - base_args, query, timeout=30, settings=None, default_format=None, max_http_retries=5, retry_error_codes=False + base_args, + query, + timeout=30, + settings=None, + default_format=None, + max_http_retries=5, + retry_error_codes=False, ): if args.secure: client = http.client.HTTPSConnection( @@ -146,12 +152,36 @@ def clickhouse_execute_http( return data -def clickhouse_execute(base_args, query, timeout=30, settings=None, max_http_retries=5, retry_error_codes=False): - return clickhouse_execute_http(base_args, query, timeout, settings, max_http_retries=max_http_retries, retry_error_codes=retry_error_codes).strip() + +def clickhouse_execute( + base_args, + query, + timeout=30, + settings=None, + max_http_retries=5, + retry_error_codes=False, +): + return clickhouse_execute_http( + base_args, + query, + timeout, + settings, + max_http_retries=max_http_retries, + retry_error_codes=retry_error_codes, + ).strip() -def clickhouse_execute_json(base_args, query, timeout=60, settings=None, max_http_retries=5): - data = clickhouse_execute_http(base_args, query, timeout, settings, "JSONEachRow", max_http_retries=max_http_retries) +def clickhouse_execute_json( + base_args, query, timeout=60, settings=None, max_http_retries=5 +): + data = clickhouse_execute_http( + base_args, + query, + timeout, + settings, + "JSONEachRow", + max_http_retries=max_http_retries, + ) if not data: return None rows = [] @@ -648,7 +678,9 @@ class TestCase: clickhouse_execute( args, - "CREATE DATABASE IF NOT EXISTS " + database + get_db_engine(testcase_args, database), + "CREATE DATABASE IF NOT EXISTS " + + database + + get_db_engine(testcase_args, database), settings=get_create_database_settings(args, testcase_args), ) @@ -831,7 +863,8 @@ class TestCase: # TODO: remove checking "no-upgrade-check" after 23.1 elif args.upgrade_check and ( - "no-upgrade-check" in tags or "no-upgrade-check" in tags): + "no-upgrade-check" in tags or "no-upgrade-check" in tags + ): return FailureReason.NO_UPGRADE_CHECK elif tags and ("no-s3-storage" in tags) and args.s3_storage: @@ -1051,7 +1084,11 @@ class TestCase: @staticmethod def send_test_name_failed(suite: str, case: str): pid = os.getpid() - clickhouse_execute(args, f"SELECT 'Running test {suite}/{case} from pid={pid}'", retry_error_codes=True) + clickhouse_execute( + args, + f"SELECT 'Running test {suite}/{case} from pid={pid}'", + retry_error_codes=True, + ) def run_single_test( self, server_logs_level, client_options @@ -2220,6 +2257,7 @@ def find_binary(name): raise Exception(f"{name} was not found in PATH") + def find_clickhouse_command(binary, command): symlink = binary + "-" + command if os.access(symlink, os.X_OK): @@ -2228,6 +2266,7 @@ def find_clickhouse_command(binary, command): # To avoid requiring symlinks (in case you download binary from CI) return binary + " " + command + def get_additional_client_options(args): if args.client_option: return " ".join("--" + option for option in args.client_option) @@ -2569,7 +2608,9 @@ if __name__ == "__main__": "WARNING: --extract_from_config option is deprecated and will be removed the the future", file=sys.stderr, ) - args.extract_from_config = find_clickhouse_command(args.binary, "extract-from-config") + args.extract_from_config = find_clickhouse_command( + args.binary, "extract-from-config" + ) if args.configclient: args.client += " --config-file=" + args.configclient diff --git a/tests/integration/runner b/tests/integration/runner index c1b3178faa4..f658bac412b 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -243,11 +243,18 @@ if __name__ == "__main__": ) parser.add_argument( - "--no-random", action="store", dest="no_random", help="Disable tests order randomization" + "--no-random", + action="store", + dest="no_random", + help="Disable tests order randomization", ) parser.add_argument( - "--pre-pull", action="store_true", default=False, dest="pre_pull", help="Pull images for docker_compose before all other actions" + "--pre-pull", + action="store_true", + default=False, + dest="pre_pull", + help="Pull images for docker_compose before all other actions", ) parser.add_argument( @@ -306,7 +313,6 @@ if __name__ == "__main__": # if not args.no_random: # rand_args += f"--random-seed={os.getpid()}" - net = "" if args.network: net = "--net={}".format(args.network) @@ -416,8 +422,11 @@ if __name__ == "__main__": name=CONTAINER_NAME, ) - cmd = cmd_base + " " + args.command - cmd_pre_pull = cmd_base + " find /compose -name docker_compose_*.yml -exec docker-compose -f '{}' pull \;" + cmd = cmd_base + " " + args.command + cmd_pre_pull = ( + cmd_base + + " find /compose -name docker_compose_*.yml -exec docker-compose -f '{}' pull \;" + ) containers = subprocess.check_output( f"docker ps --all --quiet --filter name={CONTAINER_NAME} --format={{{{.ID}}}}", diff --git a/tests/queries/0_stateless/00386_long_in_pk.python b/tests/queries/0_stateless/00386_long_in_pk.python index e33bb254c60..c7b04102dc5 100644 --- a/tests/queries/0_stateless/00386_long_in_pk.python +++ b/tests/queries/0_stateless/00386_long_in_pk.python @@ -1,57 +1,72 @@ #!/usr/bin/env python3 + def gen_queries(): - create_template = 'create table tab_00386 (a Int8, b String, c Tuple(Int8), d Tuple(Tuple(Int8)), e Tuple(Int8, String), f Tuple(Tuple(Int8, String))) engine = MergeTree order by ({}) partition by {}' - drop_query = 'drop table if exists tab_00386' - values = ('1', "'a'", 'tuple(1)', 'tuple(tuple(1))', "(1, 'a')", "tuple((1, 'a'))") + create_template = "create table tab_00386 (a Int8, b String, c Tuple(Int8), d Tuple(Tuple(Int8)), e Tuple(Int8, String), f Tuple(Tuple(Int8, String))) engine = MergeTree order by ({}) partition by {}" + drop_query = "drop table if exists tab_00386" + values = ("1", "'a'", "tuple(1)", "tuple(tuple(1))", "(1, 'a')", "tuple((1, 'a'))") insert_query = "insert into tab_00386 values (1, 'a', tuple(1), tuple(tuple(1)), (1, 'a'), tuple((1, 'a')))" - columns = tuple('a b c d'.split()) - order_by_columns = tuple('a b c'.split()) - partition_by_columns = tuple(' tuple() a'.split()) + columns = tuple("a b c d".split()) + order_by_columns = tuple("a b c".split()) + partition_by_columns = tuple(" tuple() a".split()) for partition in partition_by_columns: for key_mask in range(1, 1 << len(order_by_columns)): - key = ','.join(order_by_columns[i] for i in range(len(order_by_columns)) if (1 << i) & key_mask != 0) + key = ",".join( + order_by_columns[i] + for i in range(len(order_by_columns)) + if (1 << i) & key_mask != 0 + ) create_query = create_template.format(key, partition) for q in (drop_query, create_query, insert_query): yield q for column, value in zip(columns, values): - yield 'select {} in {} from tab_00386'.format(column, value) - yield 'select {} in tuple({}) from tab_00386'.format(column, value) - yield 'select {} in (select {} from tab_00386) from tab_00386'.format(column, column) + yield "select {} in {} from tab_00386".format(column, value) + yield "select {} in tuple({}) from tab_00386".format(column, value) + yield "select {} in (select {} from tab_00386) from tab_00386".format( + column, column + ) for i in range(len(columns)): for j in range(i, len(columns)): - yield 'select ({}, {}) in tuple({}, {}) from tab_00386'.format(columns[i], columns[j], values[i], values[j]) - yield 'select ({}, {}) in (select {}, {} from tab_00386) from tab_00386'.format(columns[i], columns[j], columns[i], columns[j]) - yield 'select ({}, {}) in (select ({}, {}) from tab_00386) from tab_00386'.format(columns[i], columns[j], columns[i], columns[j]) + yield "select ({}, {}) in tuple({}, {}) from tab_00386".format( + columns[i], columns[j], values[i], values[j] + ) + yield "select ({}, {}) in (select {}, {} from tab_00386) from tab_00386".format( + columns[i], columns[j], columns[i], columns[j] + ) + yield "select ({}, {}) in (select ({}, {}) from tab_00386) from tab_00386".format( + columns[i], columns[j], columns[i], columns[j] + ) yield "select e in (1, 'a') from tab_00386" yield "select f in tuple((1, 'a')) from tab_00386" yield "select f in tuple(tuple((1, 'a'))) from tab_00386" - yield 'select e in (select a, b from tab_00386) from tab_00386' - yield 'select e in (select (a, b) from tab_00386) from tab_00386' - yield 'select f in (select tuple((a, b)) from tab_00386) from tab_00386' - yield 'select tuple(f) in (select tuple(tuple((a, b))) from tab_00386) from tab_00386' + yield "select e in (select a, b from tab_00386) from tab_00386" + yield "select e in (select (a, b) from tab_00386) from tab_00386" + yield "select f in (select tuple((a, b)) from tab_00386) from tab_00386" + yield "select tuple(f) in (select tuple(tuple((a, b))) from tab_00386) from tab_00386" + import requests import os + def main(): - url = os.environ['CLICKHOUSE_URL'] + url = os.environ["CLICKHOUSE_URL"] for q in gen_queries(): resp = requests.post(url, data=q) - if resp.status_code != 200 or resp.text.strip() not in ('1', ''): - print('Query:', q) - print('Code:', resp.status_code) + if resp.status_code != 200 or resp.text.strip() not in ("1", ""): + print("Query:", q) + print("Code:", resp.status_code) print(resp.text) break - requests.post(url, data='drop table tab_00386') + requests.post(url, data="drop table tab_00386") + if __name__ == "__main__": main() - diff --git a/tests/queries/0_stateless/00411_long_accurate_number_comparison.python b/tests/queries/0_stateless/00411_long_accurate_number_comparison.python index 3c8a8f2ea25..e713e8814bc 100644 --- a/tests/queries/0_stateless/00411_long_accurate_number_comparison.python +++ b/tests/queries/0_stateless/00411_long_accurate_number_comparison.python @@ -2,8 +2,20 @@ import os, itertools, urllib.request, urllib.parse, urllib.error, urllib.request, urllib.error, urllib.parse, sys + def get_ch_answer(query): - return urllib.request.urlopen(os.environ.get('CLICKHOUSE_URL', 'http://localhost:' + os.environ.get('CLICKHOUSE_PORT_HTTP', '8123') ), data=query.encode()).read().decode() + return ( + urllib.request.urlopen( + os.environ.get( + "CLICKHOUSE_URL", + "http://localhost:" + os.environ.get("CLICKHOUSE_PORT_HTTP", "8123"), + ), + data=query.encode(), + ) + .read() + .decode() + ) + def check_answers(query, answer): ch_answer = get_ch_answer(query) @@ -13,36 +25,34 @@ def check_answers(query, answer): print("Fetched answer :", ch_answer) exit(-1) + def get_values(): values = [0, 1, -1] for bits in [8, 16, 32, 64]: values += [2**bits, 2**bits - 1] - values += [2**(bits-1) - 1, 2**(bits-1), 2**(bits-1) + 1] - values += [-2**(bits-1) - 1, -2**(bits-1), -2**(bits-1) + 1] + values += [2 ** (bits - 1) - 1, 2 ** (bits - 1), 2 ** (bits - 1) + 1] + values += [-(2 ** (bits - 1)) - 1, -(2 ** (bits - 1)), -(2 ** (bits - 1)) + 1] return values + def is_valid_integer(x): - return -2**63 <= x and x <= 2**64-1 + return -(2**63) <= x and x <= 2**64 - 1 -TEST_WITH_CASTING=True -GENERATE_TEST_FILES=False +TEST_WITH_CASTING = True +GENERATE_TEST_FILES = False TYPES = { - "UInt8" : { "bits" : 8, "sign" : False, "float" : False }, - "Int8" : { "bits" : 8, "sign" : True, "float" : False }, - - "UInt16": { "bits" : 16, "sign" : False, "float" : False }, - "Int16" : { "bits" : 16, "sign" : True, "float" : False }, - - "UInt32": { "bits" : 32, "sign" : False, "float" : False }, - "Int32" : { "bits" : 32, "sign" : True, "float" : False }, - - "UInt64": { "bits" : 64, "sign" : False, "float" : False }, - "Int64" : { "bits" : 64, "sign" : True, "float" : False } - - #"Float32" : { "bits" : 32, "sign" : True, "float" : True }, - #"Float64" : { "bits" : 64, "sign" : True, "float" : True } + "UInt8": {"bits": 8, "sign": False, "float": False}, + "Int8": {"bits": 8, "sign": True, "float": False}, + "UInt16": {"bits": 16, "sign": False, "float": False}, + "Int16": {"bits": 16, "sign": True, "float": False}, + "UInt32": {"bits": 32, "sign": False, "float": False}, + "Int32": {"bits": 32, "sign": True, "float": False}, + "UInt64": {"bits": 64, "sign": False, "float": False}, + "Int64": {"bits": 64, "sign": True, "float": False} + # "Float32" : { "bits" : 32, "sign" : True, "float" : True }, + # "Float64" : { "bits" : 64, "sign" : True, "float" : True } } @@ -55,14 +65,18 @@ def inside_range(value, type_name): return True if signed: - return -2**(bits-1) <= value and value <= 2**(bits-1) - 1 + return -(2 ** (bits - 1)) <= value and value <= 2 ** (bits - 1) - 1 else: return 0 <= value and value <= 2**bits - 1 def test_operators(v1, v2, v1_passed, v2_passed): - query_str = "{v1} = {v2}, {v1} != {v2}, {v1} < {v2}, {v1} <= {v2}, {v1} > {v2}, {v1} >= {v2},\t".format(v1=v1_passed, v2=v2_passed) - query_str += "{v1} = {v2}, {v1} != {v2}, {v1} < {v2}, {v1} <= {v2}, {v1} > {v2}, {v1} >= {v2} ".format(v1=v2_passed, v2=v1_passed) + query_str = "{v1} = {v2}, {v1} != {v2}, {v1} < {v2}, {v1} <= {v2}, {v1} > {v2}, {v1} >= {v2},\t".format( + v1=v1_passed, v2=v2_passed + ) + query_str += "{v1} = {v2}, {v1} != {v2}, {v1} < {v2}, {v1} <= {v2}, {v1} > {v2}, {v1} >= {v2} ".format( + v1=v2_passed, v2=v1_passed + ) answers = [v1 == v2, v1 != v2, v1 < v2, v1 <= v2, v1 > v2, v1 >= v2] answers += [v2 == v1, v2 != v1, v2 < v1, v2 <= v1, v2 > v1, v2 >= v1] @@ -74,6 +88,7 @@ def test_operators(v1, v2, v1_passed, v2_passed): VALUES = [x for x in get_values() if is_valid_integer(x)] + def test_pair(v1, v2): query = "SELECT {}, {}, ".format(v1, v2) answers = "{}\t{}\t".format(v1, v2) @@ -87,19 +102,58 @@ def test_pair(v1, v2): if inside_range(v1, t1): for t2 in TYPES.keys(): if inside_range(v2, t2): - q, a = test_operators(v1, v2, 'to{}({})'.format(t1, v1), 'to{}({})'.format(t2, v2)) - query += ', ' + q + q, a = test_operators( + v1, v2, "to{}({})".format(t1, v1), "to{}({})".format(t2, v2) + ) + query += ", " + q answers += "\t" + a check_answers(query, answers) return query, answers -VALUES_INT = [0, -1, 1, 2**64-1, 2**63, -2**63, 2**63-1, 2**51, 2**52, 2**53-1, 2**53, 2**53+1, 2**53+2, -2**53+1, -2**53, -2**53-1, -2**53-2, 2*52, -2**52] -VALUES_FLOAT = [float(x) for x in VALUES_INT + [-0.5, 0.5, -1.5, 1.5, 2**53, 2**51 - 0.5, 2**51 + 0.5, 2**60, -2**60, -2**63 - 10000, 2**63 + 10000]] +VALUES_INT = [ + 0, + -1, + 1, + 2**64 - 1, + 2**63, + -(2**63), + 2**63 - 1, + 2**51, + 2**52, + 2**53 - 1, + 2**53, + 2**53 + 1, + 2**53 + 2, + -(2**53) + 1, + -(2**53), + -(2**53) - 1, + -(2**53) - 2, + 2 * 52, + -(2**52), +] +VALUES_FLOAT = [ + float(x) + for x in VALUES_INT + + [ + -0.5, + 0.5, + -1.5, + 1.5, + 2**53, + 2**51 - 0.5, + 2**51 + 0.5, + 2**60, + -(2**60), + -(2**63) - 10000, + 2**63 + 10000, + ] +] + def test_float_pair(i, f): - f_str = ("%.9f" % f) + f_str = "%.9f" % f query = "SELECT '{}', '{}', ".format(i, f_str) answers = "{}\t{}\t".format(i, f_str) @@ -110,8 +164,8 @@ def test_float_pair(i, f): if TEST_WITH_CASTING: for t1 in TYPES.keys(): if inside_range(i, t1): - q, a = test_operators(i, f, 'to{}({})'.format(t1, i), f_str) - query += ', ' + q + q, a = test_operators(i, f, "to{}({})".format(t1, i), f_str) + query += ", " + q answers += "\t" + a check_answers(query, answers) @@ -120,22 +174,26 @@ def test_float_pair(i, f): def main(): if GENERATE_TEST_FILES: - base_name = '00411_accurate_number_comparison' - sql_file = open(base_name + '.sql', 'wt') - ref_file = open(base_name + '.reference', 'wt') + base_name = "00411_accurate_number_comparison" + sql_file = open(base_name + ".sql", "wt") + ref_file = open(base_name + ".reference", "wt") num_int_tests = len(list(itertools.combinations(VALUES, 2))) num_parts = 4 for part in range(0, num_parts): - if 'int' + str(part + 1) in sys.argv[1:]: - for (v1, v2) in itertools.islice(itertools.combinations(VALUES, 2), part * num_int_tests // num_parts, (part + 1) * num_int_tests // num_parts): + if "int" + str(part + 1) in sys.argv[1:]: + for (v1, v2) in itertools.islice( + itertools.combinations(VALUES, 2), + part * num_int_tests // num_parts, + (part + 1) * num_int_tests // num_parts, + ): q, a = test_pair(v1, v2) if GENERATE_TEST_FILES: sql_file.write(q + ";\n") ref_file.write(a + "\n") - if 'float' in sys.argv[1:]: + if "float" in sys.argv[1:]: for (i, f) in itertools.product(VALUES_INT, VALUES_FLOAT): q, a = test_float_pair(i, f) if GENERATE_TEST_FILES: diff --git a/tests/queries/0_stateless/00646_url_engine.python b/tests/queries/0_stateless/00646_url_engine.python index 5f3b7546dd5..0a26f8039c2 100644 --- a/tests/queries/0_stateless/00646_url_engine.python +++ b/tests/queries/0_stateless/00646_url_engine.python @@ -12,6 +12,7 @@ import subprocess from io import StringIO from http.server import BaseHTTPRequestHandler, HTTPServer + def is_ipv6(host): try: socket.inet_aton(host) @@ -19,6 +20,7 @@ def is_ipv6(host): except: return True + def get_local_port(host, ipv6): if ipv6: family = socket.AF_INET6 @@ -29,8 +31,9 @@ def get_local_port(host, ipv6): fd.bind((host, 0)) return fd.getsockname()[1] -CLICKHOUSE_HOST = os.environ.get('CLICKHOUSE_HOST', '127.0.0.1') -CLICKHOUSE_PORT_HTTP = os.environ.get('CLICKHOUSE_PORT_HTTP', '8123') + +CLICKHOUSE_HOST = os.environ.get("CLICKHOUSE_HOST", "127.0.0.1") +CLICKHOUSE_PORT_HTTP = os.environ.get("CLICKHOUSE_PORT_HTTP", "8123") ##################################################################################### # This test starts an HTTP server and serves data to clickhouse url-engine based table. @@ -39,27 +42,42 @@ CLICKHOUSE_PORT_HTTP = os.environ.get('CLICKHOUSE_PORT_HTTP', '8123') ##################################################################################### # IP-address of this host accessible from the outside world. Get the first one -HTTP_SERVER_HOST = subprocess.check_output(['hostname', '-i']).decode('utf-8').strip().split()[0] +HTTP_SERVER_HOST = ( + subprocess.check_output(["hostname", "-i"]).decode("utf-8").strip().split()[0] +) IS_IPV6 = is_ipv6(HTTP_SERVER_HOST) HTTP_SERVER_PORT = get_local_port(HTTP_SERVER_HOST, IS_IPV6) # IP address and port of the HTTP server started from this script. HTTP_SERVER_ADDRESS = (HTTP_SERVER_HOST, HTTP_SERVER_PORT) if IS_IPV6: - HTTP_SERVER_URL_STR = 'http://' + f'[{str(HTTP_SERVER_ADDRESS[0])}]:{str(HTTP_SERVER_ADDRESS[1])}' + "/" + HTTP_SERVER_URL_STR = ( + "http://" + + f"[{str(HTTP_SERVER_ADDRESS[0])}]:{str(HTTP_SERVER_ADDRESS[1])}" + + "/" + ) else: - HTTP_SERVER_URL_STR = 'http://' + f'{str(HTTP_SERVER_ADDRESS[0])}:{str(HTTP_SERVER_ADDRESS[1])}' + "/" + HTTP_SERVER_URL_STR = ( + "http://" + f"{str(HTTP_SERVER_ADDRESS[0])}:{str(HTTP_SERVER_ADDRESS[1])}" + "/" + ) + +CSV_DATA = os.path.join( + tempfile._get_default_tempdir(), next(tempfile._get_candidate_names()) +) -CSV_DATA = os.path.join(tempfile._get_default_tempdir(), next(tempfile._get_candidate_names())) def get_ch_answer(query): host = CLICKHOUSE_HOST if IS_IPV6: - host = f'[{host}]' + host = f"[{host}]" - url = os.environ.get('CLICKHOUSE_URL', 'http://{host}:{port}'.format(host=CLICKHOUSE_HOST, port=CLICKHOUSE_PORT_HTTP)) + url = os.environ.get( + "CLICKHOUSE_URL", + "http://{host}:{port}".format(host=CLICKHOUSE_HOST, port=CLICKHOUSE_PORT_HTTP), + ) return urllib.request.urlopen(url, data=query.encode()).read().decode() + def check_answers(query, answer): ch_answer = get_ch_answer(query) if ch_answer.strip() != answer.strip(): @@ -68,18 +86,19 @@ def check_answers(query, answer): print("Fetched answer :", ch_answer, file=sys.stderr) raise Exception("Fail on query") + class CSVHTTPServer(BaseHTTPRequestHandler): def _set_headers(self): self.send_response(200) - self.send_header('Content-type', 'text/csv') + self.send_header("Content-type", "text/csv") self.end_headers() def do_GET(self): self._set_headers() - with open(CSV_DATA, 'r') as fl: - reader = csv.reader(fl, delimiter=',') + with open(CSV_DATA, "r") as fl: + reader = csv.reader(fl, delimiter=",") for row in reader: - self.wfile.write((', '.join(row) + '\n').encode()) + self.wfile.write((", ".join(row) + "\n").encode()) return def do_HEAD(self): @@ -87,33 +106,33 @@ class CSVHTTPServer(BaseHTTPRequestHandler): return def read_chunk(self): - msg = '' + msg = "" while True: sym = self.rfile.read(1) - if sym == '': + if sym == "": break - msg += sym.decode('utf-8') - if msg.endswith('\r\n'): + msg += sym.decode("utf-8") + if msg.endswith("\r\n"): break length = int(msg[:-2], 16) if length == 0: - return '' + return "" content = self.rfile.read(length) - self.rfile.read(2) # read sep \r\n - return content.decode('utf-8') + self.rfile.read(2) # read sep \r\n + return content.decode("utf-8") def do_POST(self): - data = '' + data = "" while True: chunk = self.read_chunk() if not chunk: break data += chunk with StringIO(data) as fl: - reader = csv.reader(fl, delimiter=',') - with open(CSV_DATA, 'a') as d: + reader = csv.reader(fl, delimiter=",") + with open(CSV_DATA, "a") as d: for row in reader: - d.write(','.join(row) + '\n') + d.write(",".join(row) + "\n") self._set_headers() self.wfile.write(b"ok") @@ -124,6 +143,7 @@ class CSVHTTPServer(BaseHTTPRequestHandler): class HTTPServerV6(HTTPServer): address_family = socket.AF_INET6 + def start_server(): if IS_IPV6: httpd = HTTPServerV6(HTTP_SERVER_ADDRESS, CSVHTTPServer) @@ -133,49 +153,76 @@ def start_server(): t = threading.Thread(target=httpd.serve_forever) return t, httpd + # test section -def test_select(table_name="", schema="str String,numuint UInt32,numint Int32,double Float64", requests=[], answers=[], test_data=""): - with open(CSV_DATA, 'w') as f: # clear file - f.write('') + +def test_select( + table_name="", + schema="str String,numuint UInt32,numint Int32,double Float64", + requests=[], + answers=[], + test_data="", +): + with open(CSV_DATA, "w") as f: # clear file + f.write("") if test_data: - with open(CSV_DATA, 'w') as f: + with open(CSV_DATA, "w") as f: f.write(test_data + "\n") if table_name: get_ch_answer("drop table if exists {}".format(table_name)) - get_ch_answer("create table {} ({}) engine=URL('{}', 'CSV')".format(table_name, schema, HTTP_SERVER_URL_STR)) + get_ch_answer( + "create table {} ({}) engine=URL('{}', 'CSV')".format( + table_name, schema, HTTP_SERVER_URL_STR + ) + ) for i in range(len(requests)): tbl = table_name if not tbl: - tbl = "url('{addr}', 'CSV', '{schema}')".format(addr=HTTP_SERVER_URL_STR, schema=schema) + tbl = "url('{addr}', 'CSV', '{schema}')".format( + addr=HTTP_SERVER_URL_STR, schema=schema + ) check_answers(requests[i].format(tbl=tbl), answers[i]) if table_name: get_ch_answer("drop table if exists {}".format(table_name)) -def test_insert(table_name="", schema="str String,numuint UInt32,numint Int32,double Float64", requests_insert=[], requests_select=[], answers=[]): - with open(CSV_DATA, 'w') as f: # flush test file - f.write('') +def test_insert( + table_name="", + schema="str String,numuint UInt32,numint Int32,double Float64", + requests_insert=[], + requests_select=[], + answers=[], +): + with open(CSV_DATA, "w") as f: # flush test file + f.write("") if table_name: get_ch_answer("drop table if exists {}".format(table_name)) - get_ch_answer("create table {} ({}) engine=URL('{}', 'CSV')".format(table_name, schema, HTTP_SERVER_URL_STR)) + get_ch_answer( + "create table {} ({}) engine=URL('{}', 'CSV')".format( + table_name, schema, HTTP_SERVER_URL_STR + ) + ) for req in requests_insert: tbl = table_name if not tbl: - tbl = "table function url('{addr}', 'CSV', '{schema}')".format(addr=HTTP_SERVER_URL_STR, schema=schema) + tbl = "table function url('{addr}', 'CSV', '{schema}')".format( + addr=HTTP_SERVER_URL_STR, schema=schema + ) get_ch_answer(req.format(tbl=tbl)) - for i in range(len(requests_select)): tbl = table_name if not tbl: - tbl = "url('{addr}', 'CSV', '{schema}')".format(addr=HTTP_SERVER_URL_STR, schema=schema) + tbl = "url('{addr}', 'CSV', '{schema}')".format( + addr=HTTP_SERVER_URL_STR, schema=schema + ) check_answers(requests_select[i].format(tbl=tbl), answers[i]) if table_name: @@ -185,9 +232,11 @@ def test_insert(table_name="", schema="str String,numuint UInt32,numint Int32,do def main(): test_data = "Hello,2,-2,7.7\nWorld,2,-5,8.8" select_only_requests = { - "select str,numuint,numint,double from {tbl}" : test_data.replace(',', '\t'), - "select numuint, count(*) from {tbl} group by numuint" : "2\t2", - "select str,numuint,numint,double from {tbl} limit 1": test_data.split("\n")[0].replace(',', '\t'), + "select str,numuint,numint,double from {tbl}": test_data.replace(",", "\t"), + "select numuint, count(*) from {tbl} group by numuint": "2\t2", + "select str,numuint,numint,double from {tbl} limit 1": test_data.split("\n")[ + 0 + ].replace(",", "\t"), } insert_requests = [ @@ -196,21 +245,41 @@ def main(): ] select_requests = { - "select distinct numuint from {tbl} order by numuint": '\n'.join([str(i) for i in range(11)]), - "select count(*) from {tbl}": '12', - 'select double, count(*) from {tbl} group by double order by double': "7.7\t2\n9.9\t10" + "select distinct numuint from {tbl} order by numuint": "\n".join( + [str(i) for i in range(11)] + ), + "select count(*) from {tbl}": "12", + "select double, count(*) from {tbl} group by double order by double": "7.7\t2\n9.9\t10", } t, httpd = start_server() t.start() # test table with url engine - test_select(table_name="test_table_select", requests=list(select_only_requests.keys()), answers=list(select_only_requests.values()), test_data=test_data) + test_select( + table_name="test_table_select", + requests=list(select_only_requests.keys()), + answers=list(select_only_requests.values()), + test_data=test_data, + ) # test table function url - test_select(requests=list(select_only_requests.keys()), answers=list(select_only_requests.values()), test_data=test_data) - #test insert into table with url engine - test_insert(table_name="test_table_insert", requests_insert=insert_requests, requests_select=list(select_requests.keys()), answers=list(select_requests.values())) - #test insert into table function url - test_insert(requests_insert=insert_requests, requests_select=list(select_requests.keys()), answers=list(select_requests.values())) + test_select( + requests=list(select_only_requests.keys()), + answers=list(select_only_requests.values()), + test_data=test_data, + ) + # test insert into table with url engine + test_insert( + table_name="test_table_insert", + requests_insert=insert_requests, + requests_select=list(select_requests.keys()), + answers=list(select_requests.values()), + ) + # test insert into table function url + test_insert( + requests_insert=insert_requests, + requests_select=list(select_requests.keys()), + answers=list(select_requests.values()), + ) httpd.shutdown() t.join() diff --git a/tests/queries/0_stateless/00990_hasToken.python b/tests/queries/0_stateless/00990_hasToken.python index 7d3775adc9d..e9bc514474a 100644 --- a/tests/queries/0_stateless/00990_hasToken.python +++ b/tests/queries/0_stateless/00990_hasToken.python @@ -12,35 +12,46 @@ HAYSTACKS = [ NEEDLE = "needle" -HAY_RE = re.compile(r'\bhay\b', re.IGNORECASE) -NEEDLE_RE = re.compile(r'\bneedle\b', re.IGNORECASE) +HAY_RE = re.compile(r"\bhay\b", re.IGNORECASE) +NEEDLE_RE = re.compile(r"\bneedle\b", re.IGNORECASE) + def replace_follow_case(replacement): def func(match): g = match.group() - if g.islower(): return replacement.lower() - if g.istitle(): return replacement.title() - if g.isupper(): return replacement.upper() + if g.islower(): + return replacement.lower() + if g.istitle(): + return replacement.title() + if g.isupper(): + return replacement.upper() return replacement + return func + def replace_separators(query, new_sep): - SEP_RE = re.compile('\\s+') + SEP_RE = re.compile("\\s+") result = SEP_RE.sub(new_sep, query) return result -def enlarge_haystack(query, times, separator=''): - return HAY_RE.sub(replace_follow_case(('hay' + separator) * times), query) + +def enlarge_haystack(query, times, separator=""): + return HAY_RE.sub(replace_follow_case(("hay" + separator) * times), query) + def small_needle(query): - return NEEDLE_RE.sub(replace_follow_case('n'), query) + return NEEDLE_RE.sub(replace_follow_case("n"), query) + def remove_needle(query): - return NEEDLE_RE.sub('', query) + return NEEDLE_RE.sub("", query) + def replace_needle(query, new_needle): return NEEDLE_RE.sub(new_needle, query) + # with str.lower, str.uppert, str.title and such def transform_needle(query, string_transformation_func): def replace_with_transformation(match): @@ -49,19 +60,21 @@ def transform_needle(query, string_transformation_func): return NEEDLE_RE.sub(replace_with_transformation, query) -def create_cases(case_sensitive_func, case_insensitive_func, table_row_template, table_query_template, const_query_template): + +def create_cases( + case_sensitive_func, + case_insensitive_func, + table_row_template, + table_query_template, + const_query_template, +): const_queries = [] table_rows = [] table_queries = set() def add_case(func, haystack, needle, match): match = int(match) - args = dict( - func = func, - haystack = haystack, - needle = needle, - match = match - ) + args = dict(func=func, haystack=haystack, needle=needle, match=match) const_queries.append(const_query_template.substitute(args)) table_queries.add(table_query_template.substitute(args)) table_rows.append(table_row_template.substitute(args)) @@ -69,14 +82,28 @@ def create_cases(case_sensitive_func, case_insensitive_func, table_row_template, def add_case_sensitive(haystack, needle, match): add_case(case_sensitive_func, haystack, needle, match) if match: - add_case(case_sensitive_func, transform_needle(haystack, str.swapcase), transform_needle(needle, str.swapcase), match) + add_case( + case_sensitive_func, + transform_needle(haystack, str.swapcase), + transform_needle(needle, str.swapcase), + match, + ) def add_case_insensitive(haystack, needle, match): add_case(case_insensitive_func, haystack, needle, match) if match: - add_case(case_insensitive_func, transform_needle(haystack, str.swapcase), needle, match) - add_case(case_insensitive_func, haystack, transform_needle(needle, str.swapcase), match) - + add_case( + case_insensitive_func, + transform_needle(haystack, str.swapcase), + needle, + match, + ) + add_case( + case_insensitive_func, + haystack, + transform_needle(needle, str.swapcase), + match, + ) # Negative cases add_case_sensitive(remove_needle(HAYSTACKS[0]), NEEDLE, False) @@ -85,7 +112,7 @@ def create_cases(case_sensitive_func, case_insensitive_func, table_row_template, for haystack in HAYSTACKS: add_case_sensitive(transform_needle(haystack, str.swapcase), NEEDLE, False) - sep = '' + sep = "" h = replace_separators(haystack, sep) add_case_sensitive(h, NEEDLE, False) @@ -102,8 +129,7 @@ def create_cases(case_sensitive_func, case_insensitive_func, table_row_template, add_case_sensitive(haystack, NEEDLE, True) add_case_insensitive(haystack, NEEDLE, True) - - for sep in list(''' ,'''): + for sep in list(""" ,"""): h = replace_separators(haystack, sep) add_case_sensitive(h, NEEDLE, True) add_case_sensitive(small_needle(h), small_needle(NEEDLE), True) @@ -114,32 +140,43 @@ def create_cases(case_sensitive_func, case_insensitive_func, table_row_template, add_case_insensitive(enlarge_haystack(h, 200, sep), NEEDLE, True) # case insesitivity works only on ASCII strings - add_case_sensitive(replace_needle(h, 'иголка'), replace_needle(NEEDLE, 'иголка'), True) - add_case_sensitive(replace_needle(h, '指针'), replace_needle(NEEDLE, '指针'), True) + add_case_sensitive( + replace_needle(h, "иголка"), replace_needle(NEEDLE, "иголка"), True + ) + add_case_sensitive( + replace_needle(h, "指针"), replace_needle(NEEDLE, "指针"), True + ) - for sep in list('''~!@$%^&*()-=+|]}[{";:/?.><\t''') + [r'\\\\']: + for sep in list("""~!@$%^&*()-=+|]}[{";:/?.><\t""") + [r"\\\\"]: h = replace_separators(HAYSTACKS[0], sep) add_case(case_sensitive_func, h, NEEDLE, True) return table_rows, table_queries, const_queries -def main(): +def main(): def query(x): print(x) - CONST_QUERY = Template("""SELECT ${func}('${haystack}', '${needle}'), ' expecting ', ${match};""") - TABLE_QUERY = Template("""WITH '${needle}' as n + CONST_QUERY = Template( + """SELECT ${func}('${haystack}', '${needle}'), ' expecting ', ${match};""" + ) + TABLE_QUERY = Template( + """WITH '${needle}' as n SELECT haystack, needle, ${func}(haystack, n) as result FROM ht - WHERE func = '${func}' AND needle = n AND result != match;""") + WHERE func = '${func}' AND needle = n AND result != match;""" + ) TABLE_ROW = Template("""('${haystack}', '${needle}', ${match}, '${func}')""") - rows, table_queries, const_queries = create_cases('hasToken', 'hasTokenCaseInsensitive', TABLE_ROW, TABLE_QUERY, CONST_QUERY) + rows, table_queries, const_queries = create_cases( + "hasToken", "hasTokenCaseInsensitive", TABLE_ROW, TABLE_QUERY, CONST_QUERY + ) for q in const_queries: query(q) - query("""DROP TABLE IF EXISTS ht; + query( + """DROP TABLE IF EXISTS ht; CREATE TABLE IF NOT EXISTS ht ( @@ -150,11 +187,15 @@ def main(): ) ENGINE MergeTree() ORDER BY haystack; -INSERT INTO ht VALUES {values};""".format(values=", ".join(rows))) +INSERT INTO ht VALUES {values};""".format( + values=", ".join(rows) + ) + ) for q in sorted(table_queries): query(q) query("""DROP TABLE ht""") -if __name__ == '__main__': + +if __name__ == "__main__": main() diff --git a/tests/queries/0_stateless/00991_live_view_watch_event_live.python b/tests/queries/0_stateless/00991_live_view_watch_event_live.python index 901d388ec01..9b7a3300c15 100644 --- a/tests/queries/0_stateless/00991_live_view_watch_event_live.python +++ b/tests/queries/0_stateless/00991_live_view_watch_event_live.python @@ -8,28 +8,32 @@ import sys import signal -CLICKHOUSE_CLIENT = os.environ.get('CLICKHOUSE_CLIENT') -CLICKHOUSE_CURL = os.environ.get('CLICKHOUSE_CURL') -CLICKHOUSE_URL = os.environ.get('CLICKHOUSE_URL') +CLICKHOUSE_CLIENT = os.environ.get("CLICKHOUSE_CLIENT") +CLICKHOUSE_CURL = os.environ.get("CLICKHOUSE_CURL") +CLICKHOUSE_URL = os.environ.get("CLICKHOUSE_URL") def send_query(query): cmd = list(CLICKHOUSE_CLIENT.split()) - cmd += ['--query', query] + cmd += ["--query", query] # print(cmd) - return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout + return subprocess.Popen( + cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT + ).stdout def send_query_in_process_group(query): cmd = list(CLICKHOUSE_CLIENT.split()) - cmd += ['--query', query] + cmd += ["--query", query] # print(cmd) - return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, preexec_fn=os.setsid) + return subprocess.Popen( + cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, preexec_fn=os.setsid + ) def read_lines_and_push_to_queue(pipe, queue): try: - for line in iter(pipe.readline, ''): + for line in iter(pipe.readline, ""): line = line.strip() print(line) sys.stdout.flush() @@ -41,41 +45,44 @@ def read_lines_and_push_to_queue(pipe, queue): def test(): - send_query('DROP TABLE IF EXISTS test.lv').read() - send_query('DROP TABLE IF EXISTS test.mt').read() - send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').read() - send_query('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').read() + send_query("DROP TABLE IF EXISTS test.lv").read() + send_query("DROP TABLE IF EXISTS test.mt").read() + send_query( + "CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()" + ).read() + send_query("CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt").read() q = queue.Queue() - p = send_query_in_process_group('WATCH test.lv') + p = send_query_in_process_group("WATCH test.lv") thread = threading.Thread(target=read_lines_and_push_to_queue, args=(p.stdout, q)) thread.start() line = q.get() print(line) - assert (line == '0\t1') + assert line == "0\t1" - send_query('INSERT INTO test.mt VALUES (1),(2),(3)').read() + send_query("INSERT INTO test.mt VALUES (1),(2),(3)").read() line = q.get() print(line) - assert (line == '6\t2') + assert line == "6\t2" - send_query('INSERT INTO test.mt VALUES (4),(5),(6)').read() + send_query("INSERT INTO test.mt VALUES (4),(5),(6)").read() line = q.get() print(line) - assert (line == '21\t3') + assert line == "21\t3" # Send Ctrl+C to client. os.killpg(os.getpgid(p.pid), signal.SIGINT) # This insert shouldn't affect lv. - send_query('INSERT INTO test.mt VALUES (7),(8),(9)').read() + send_query("INSERT INTO test.mt VALUES (7),(8),(9)").read() line = q.get() print(line) - assert (line is None) + assert line is None - send_query('DROP TABLE if exists test.lv').read() - send_query('DROP TABLE if exists test.lv').read() + send_query("DROP TABLE if exists test.lv").read() + send_query("DROP TABLE if exists test.lv").read() thread.join() + test() diff --git a/tests/queries/0_stateless/00991_live_view_watch_http.python b/tests/queries/0_stateless/00991_live_view_watch_http.python index d5a1e6e8ed9..72c07b27d82 100755 --- a/tests/queries/0_stateless/00991_live_view_watch_http.python +++ b/tests/queries/0_stateless/00991_live_view_watch_http.python @@ -7,26 +7,30 @@ import os import sys -CLICKHOUSE_CLIENT = os.environ.get('CLICKHOUSE_CLIENT') -CLICKHOUSE_CURL = os.environ.get('CLICKHOUSE_CURL') -CLICKHOUSE_URL = os.environ.get('CLICKHOUSE_URL') +CLICKHOUSE_CLIENT = os.environ.get("CLICKHOUSE_CLIENT") +CLICKHOUSE_CURL = os.environ.get("CLICKHOUSE_CURL") +CLICKHOUSE_URL = os.environ.get("CLICKHOUSE_URL") def send_query(query): cmd = list(CLICKHOUSE_CLIENT.split()) - cmd += ['--query', query] + cmd += ["--query", query] # print(cmd) - return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout + return subprocess.Popen( + cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT + ).stdout def send_http_query(query): - cmd = list(CLICKHOUSE_CURL.split()) # list(['curl', '-sSN', '--max-time', '10']) - cmd += ['-sSN', CLICKHOUSE_URL, '-d', query] - return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout + cmd = list(CLICKHOUSE_CURL.split()) # list(['curl', '-sSN', '--max-time', '10']) + cmd += ["-sSN", CLICKHOUSE_URL, "-d", query] + return subprocess.Popen( + cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT + ).stdout def read_lines_and_push_to_queue(pipe, queue): - for line in iter(pipe.readline, ''): + for line in iter(pipe.readline, ""): line = line.strip() print(line) sys.stdout.flush() @@ -36,28 +40,31 @@ def read_lines_and_push_to_queue(pipe, queue): def test(): - send_query('DROP TABLE IF EXISTS test.lv').read() - send_query('DROP TABLE IF EXISTS test.mt').read() - send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').read() - send_query('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').read() + send_query("DROP TABLE IF EXISTS test.lv").read() + send_query("DROP TABLE IF EXISTS test.mt").read() + send_query( + "CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()" + ).read() + send_query("CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt").read() q = queue.Queue() - pipe = send_http_query('WATCH test.lv') + pipe = send_http_query("WATCH test.lv") thread = threading.Thread(target=read_lines_and_push_to_queue, args=(pipe, q)) thread.start() line = q.get() print(line) - assert (line == '0\t1') + assert line == "0\t1" - send_query('INSERT INTO test.mt VALUES (1),(2),(3)').read() + send_query("INSERT INTO test.mt VALUES (1),(2),(3)").read() line = q.get() print(line) - assert (line == '6\t2') + assert line == "6\t2" - send_query('DROP TABLE if exists test.lv').read() - send_query('DROP TABLE if exists test.lv').read() + send_query("DROP TABLE if exists test.lv").read() + send_query("DROP TABLE if exists test.lv").read() thread.join() + test() diff --git a/tests/queries/0_stateless/01558_ttest_scipy.python b/tests/queries/0_stateless/01558_ttest_scipy.python index 4d913d4292f..75e1c2701b2 100644 --- a/tests/queries/0_stateless/01558_ttest_scipy.python +++ b/tests/queries/0_stateless/01558_ttest_scipy.python @@ -1,4 +1,4 @@ -#!/usr/bin/env python3 +#!/usr/bin/env python3 import os import sys from scipy import stats @@ -6,70 +6,86 @@ import pandas as pd import numpy as np CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) from pure_http_client import ClickHouseClient + def test_and_check(name, a, b, t_stat, p_value, precision=1e-2): client = ClickHouseClient() client.query("DROP TABLE IF EXISTS ttest;") - client.query("CREATE TABLE ttest (left Float64, right UInt8) ENGINE = Memory;"); - client.query("INSERT INTO ttest VALUES {};".format(", ".join(['({},{})'.format(i, 0) for i in a]))) - client.query("INSERT INTO ttest VALUES {};".format(", ".join(['({},{})'.format(j, 1) for j in b]))) + client.query("CREATE TABLE ttest (left Float64, right UInt8) ENGINE = Memory;") + client.query( + "INSERT INTO ttest VALUES {};".format( + ", ".join(["({},{})".format(i, 0) for i in a]) + ) + ) + client.query( + "INSERT INTO ttest VALUES {};".format( + ", ".join(["({},{})".format(j, 1) for j in b]) + ) + ) real = client.query_return_df( - "SELECT roundBankers({}(left, right).1, 16) as t_stat, ".format(name) + - "roundBankers({}(left, right).2, 16) as p_value ".format(name) + - "FROM ttest FORMAT TabSeparatedWithNames;") - real_t_stat = real['t_stat'][0] - real_p_value = real['p_value'][0] - assert(abs(real_t_stat - np.float64(t_stat)) < precision), "clickhouse_t_stat {}, scipy_t_stat {}".format(real_t_stat, t_stat) - assert(abs(real_p_value - np.float64(p_value)) < precision), "clickhouse_p_value {}, scipy_p_value {}".format(real_p_value, p_value) + "SELECT roundBankers({}(left, right).1, 16) as t_stat, ".format(name) + + "roundBankers({}(left, right).2, 16) as p_value ".format(name) + + "FROM ttest FORMAT TabSeparatedWithNames;" + ) + real_t_stat = real["t_stat"][0] + real_p_value = real["p_value"][0] + assert ( + abs(real_t_stat - np.float64(t_stat)) < precision + ), "clickhouse_t_stat {}, scipy_t_stat {}".format(real_t_stat, t_stat) + assert ( + abs(real_p_value - np.float64(p_value)) < precision + ), "clickhouse_p_value {}, scipy_p_value {}".format(real_p_value, p_value) client.query("DROP TABLE IF EXISTS ttest;") def test_student(): - rvs1 = np.round(stats.norm.rvs(loc=1, scale=5,size=500), 2) - rvs2 = np.round(stats.norm.rvs(loc=10, scale=5,size=500), 2) - s, p = stats.ttest_ind(rvs1, rvs2, equal_var = True) + rvs1 = np.round(stats.norm.rvs(loc=1, scale=5, size=500), 2) + rvs2 = np.round(stats.norm.rvs(loc=10, scale=5, size=500), 2) + s, p = stats.ttest_ind(rvs1, rvs2, equal_var=True) test_and_check("studentTTest", rvs1, rvs2, s, p) - rvs1 = np.round(stats.norm.rvs(loc=0, scale=5,size=500), 2) - rvs2 = np.round(stats.norm.rvs(loc=0, scale=5,size=500), 2) - s, p = stats.ttest_ind(rvs1, rvs2, equal_var = True) + rvs1 = np.round(stats.norm.rvs(loc=0, scale=5, size=500), 2) + rvs2 = np.round(stats.norm.rvs(loc=0, scale=5, size=500), 2) + s, p = stats.ttest_ind(rvs1, rvs2, equal_var=True) test_and_check("studentTTest", rvs1, rvs2, s, p) - rvs1 = np.round(stats.norm.rvs(loc=2, scale=10,size=512), 2) - rvs2 = np.round(stats.norm.rvs(loc=5, scale=20,size=1024), 2) - s, p = stats.ttest_ind(rvs1, rvs2, equal_var = True) + rvs1 = np.round(stats.norm.rvs(loc=2, scale=10, size=512), 2) + rvs2 = np.round(stats.norm.rvs(loc=5, scale=20, size=1024), 2) + s, p = stats.ttest_ind(rvs1, rvs2, equal_var=True) test_and_check("studentTTest", rvs1, rvs2, s, p) - rvs1 = np.round(stats.norm.rvs(loc=0, scale=10,size=1024), 2) - rvs2 = np.round(stats.norm.rvs(loc=0, scale=10,size=512), 2) - s, p = stats.ttest_ind(rvs1, rvs2, equal_var = True) + rvs1 = np.round(stats.norm.rvs(loc=0, scale=10, size=1024), 2) + rvs2 = np.round(stats.norm.rvs(loc=0, scale=10, size=512), 2) + s, p = stats.ttest_ind(rvs1, rvs2, equal_var=True) test_and_check("studentTTest", rvs1, rvs2, s, p) + def test_welch(): - rvs1 = np.round(stats.norm.rvs(loc=1, scale=15,size=500), 2) - rvs2 = np.round(stats.norm.rvs(loc=10, scale=5,size=500), 2) - s, p = stats.ttest_ind(rvs1, rvs2, equal_var = False) + rvs1 = np.round(stats.norm.rvs(loc=1, scale=15, size=500), 2) + rvs2 = np.round(stats.norm.rvs(loc=10, scale=5, size=500), 2) + s, p = stats.ttest_ind(rvs1, rvs2, equal_var=False) test_and_check("welchTTest", rvs1, rvs2, s, p) - rvs1 = np.round(stats.norm.rvs(loc=0, scale=7,size=500), 2) - rvs2 = np.round(stats.norm.rvs(loc=0, scale=3,size=500), 2) - s, p = stats.ttest_ind(rvs1, rvs2, equal_var = False) + rvs1 = np.round(stats.norm.rvs(loc=0, scale=7, size=500), 2) + rvs2 = np.round(stats.norm.rvs(loc=0, scale=3, size=500), 2) + s, p = stats.ttest_ind(rvs1, rvs2, equal_var=False) test_and_check("welchTTest", rvs1, rvs2, s, p) - rvs1 = np.round(stats.norm.rvs(loc=0, scale=10,size=1024), 2) - rvs2 = np.round(stats.norm.rvs(loc=5, scale=1,size=512), 2) - s, p = stats.ttest_ind(rvs1, rvs2, equal_var = False) + rvs1 = np.round(stats.norm.rvs(loc=0, scale=10, size=1024), 2) + rvs2 = np.round(stats.norm.rvs(loc=5, scale=1, size=512), 2) + s, p = stats.ttest_ind(rvs1, rvs2, equal_var=False) test_and_check("welchTTest", rvs1, rvs2, s, p) - rvs1 = np.round(stats.norm.rvs(loc=5, scale=10,size=512), 2) - rvs2 = np.round(stats.norm.rvs(loc=5, scale=10,size=1024), 2) - s, p = stats.ttest_ind(rvs1, rvs2, equal_var = False) + rvs1 = np.round(stats.norm.rvs(loc=5, scale=10, size=512), 2) + rvs2 = np.round(stats.norm.rvs(loc=5, scale=10, size=1024), 2) + s, p = stats.ttest_ind(rvs1, rvs2, equal_var=False) test_and_check("welchTTest", rvs1, rvs2, s, p) + if __name__ == "__main__": test_student() test_welch() - print("Ok.") \ No newline at end of file + print("Ok.") diff --git a/tests/queries/0_stateless/01561_mann_whitney_scipy.python b/tests/queries/0_stateless/01561_mann_whitney_scipy.python index 7958e8bbaf1..4713120287d 100644 --- a/tests/queries/0_stateless/01561_mann_whitney_scipy.python +++ b/tests/queries/0_stateless/01561_mann_whitney_scipy.python @@ -6,7 +6,7 @@ import pandas as pd import numpy as np CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) from pure_http_client import ClickHouseClient @@ -14,40 +14,51 @@ from pure_http_client import ClickHouseClient def test_and_check(name, a, b, t_stat, p_value): client = ClickHouseClient() client.query("DROP TABLE IF EXISTS mann_whitney;") - client.query("CREATE TABLE mann_whitney (left Float64, right UInt8) ENGINE = Memory;"); - client.query("INSERT INTO mann_whitney VALUES {};".format(", ".join(['({},{}), ({},{})'.format(i, 0, j, 1) for i,j in zip(a, b)]))) + client.query( + "CREATE TABLE mann_whitney (left Float64, right UInt8) ENGINE = Memory;" + ) + client.query( + "INSERT INTO mann_whitney VALUES {};".format( + ", ".join(["({},{}), ({},{})".format(i, 0, j, 1) for i, j in zip(a, b)]) + ) + ) real = client.query_return_df( - "SELECT roundBankers({}(left, right).1, 16) as t_stat, ".format(name) + - "roundBankers({}(left, right).2, 16) as p_value ".format(name) + - "FROM mann_whitney FORMAT TabSeparatedWithNames;") - real_t_stat = real['t_stat'][0] - real_p_value = real['p_value'][0] - assert(abs(real_t_stat - np.float64(t_stat) < 1e-2)), "clickhouse_t_stat {}, scipy_t_stat {}".format(real_t_stat, t_stat) - assert(abs(real_p_value - np.float64(p_value)) < 1e-2), "clickhouse_p_value {}, scipy_p_value {}".format(real_p_value, p_value) + "SELECT roundBankers({}(left, right).1, 16) as t_stat, ".format(name) + + "roundBankers({}(left, right).2, 16) as p_value ".format(name) + + "FROM mann_whitney FORMAT TabSeparatedWithNames;" + ) + real_t_stat = real["t_stat"][0] + real_p_value = real["p_value"][0] + assert abs( + real_t_stat - np.float64(t_stat) < 1e-2 + ), "clickhouse_t_stat {}, scipy_t_stat {}".format(real_t_stat, t_stat) + assert ( + abs(real_p_value - np.float64(p_value)) < 1e-2 + ), "clickhouse_p_value {}, scipy_p_value {}".format(real_p_value, p_value) client.query("DROP TABLE IF EXISTS mann_whitney;") def test_mann_whitney(): - rvs1 = np.round(stats.norm.rvs(loc=1, scale=5,size=500), 5) - rvs2 = np.round(stats.expon.rvs(scale=0.2,size=500), 5) - s, p = stats.mannwhitneyu(rvs1, rvs2, alternative='two-sided') + rvs1 = np.round(stats.norm.rvs(loc=1, scale=5, size=500), 5) + rvs2 = np.round(stats.expon.rvs(scale=0.2, size=500), 5) + s, p = stats.mannwhitneyu(rvs1, rvs2, alternative="two-sided") test_and_check("mannWhitneyUTest", rvs1, rvs2, s, p) test_and_check("mannWhitneyUTest('two-sided')", rvs1, rvs2, s, p) equal = np.round(stats.cauchy.rvs(scale=5, size=500), 5) - s, p = stats.mannwhitneyu(equal, equal, alternative='two-sided') + s, p = stats.mannwhitneyu(equal, equal, alternative="two-sided") test_and_check("mannWhitneyUTest('two-sided')", equal, equal, s, p) - s, p = stats.mannwhitneyu(equal, equal, alternative='less', use_continuity=False) + s, p = stats.mannwhitneyu(equal, equal, alternative="less", use_continuity=False) test_and_check("mannWhitneyUTest('less', 0)", equal, equal, s, p) - - rvs1 = np.round(stats.cauchy.rvs(scale=10,size=65536), 5) - rvs2 = np.round(stats.norm.rvs(loc=0, scale=10,size=65536), 5) - s, p = stats.mannwhitneyu(rvs1, rvs2, alternative='greater') + rvs1 = np.round(stats.cauchy.rvs(scale=10, size=65536), 5) + rvs2 = np.round(stats.norm.rvs(loc=0, scale=10, size=65536), 5) + s, p = stats.mannwhitneyu(rvs1, rvs2, alternative="greater") test_and_check("mannWhitneyUTest('greater')", rvs1, rvs2, s, p) + if __name__ == "__main__": test_mann_whitney() - print("Ok.") \ No newline at end of file + print("Ok.") diff --git a/tests/queries/0_stateless/01626_cnf_fuzz_long.python b/tests/queries/0_stateless/01626_cnf_fuzz_long.python index 10c12d14182..de9e4a21dbb 100644 --- a/tests/queries/0_stateless/01626_cnf_fuzz_long.python +++ b/tests/queries/0_stateless/01626_cnf_fuzz_long.python @@ -4,14 +4,18 @@ from random import randint, choices import sys CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) from pure_http_client import ClickHouseClient client = ClickHouseClient() N = 10 -create_query = "CREATE TABLE t_cnf_fuzz(" + ", ".join([f"c{i} UInt8" for i in range(N)]) + ") ENGINE = Memory" +create_query = ( + "CREATE TABLE t_cnf_fuzz(" + + ", ".join([f"c{i} UInt8" for i in range(N)]) + + ") ENGINE = Memory" +) client.query("DROP TABLE IF EXISTS t_cnf_fuzz") client.query(create_query) @@ -35,6 +39,7 @@ client.query(insert_query) MAX_CLAUSES = 10 MAX_ATOMS = 5 + def generate_dnf(): clauses = [] num_clauses = randint(1, MAX_CLAUSES) @@ -42,12 +47,17 @@ def generate_dnf(): num_atoms = randint(1, MAX_ATOMS) atom_ids = choices(range(N), k=num_atoms) negates = choices([0, 1], k=num_atoms) - atoms = [f"(NOT c{i})" if neg else f"c{i}" for (i, neg) in zip(atom_ids, negates)] + atoms = [ + f"(NOT c{i})" if neg else f"c{i}" for (i, neg) in zip(atom_ids, negates) + ] clauses.append("(" + " AND ".join(atoms) + ")") return " OR ".join(clauses) -select_query = "SELECT count() FROM t_cnf_fuzz WHERE {} SETTINGS convert_query_to_cnf = {}" + +select_query = ( + "SELECT count() FROM t_cnf_fuzz WHERE {} SETTINGS convert_query_to_cnf = {}" +) fail_report = """ Failed query: '{}'. diff --git a/tests/queries/0_stateless/01654_test_writer_block_sequence.python b/tests/queries/0_stateless/01654_test_writer_block_sequence.python index e80cc273076..bc4e3da9ed5 100644 --- a/tests/queries/0_stateless/01654_test_writer_block_sequence.python +++ b/tests/queries/0_stateless/01654_test_writer_block_sequence.python @@ -5,15 +5,20 @@ import random import string CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) from pure_http_client import ClickHouseClient + def get_random_string(length): - return ''.join(random.choice(string.ascii_uppercase + string.digits) for _ in range(length)) + return "".join( + random.choice(string.ascii_uppercase + string.digits) for _ in range(length) + ) + client = ClickHouseClient() + def insert_block(table_name, block_granularity_rows, block_rows): global client block_data = [] @@ -25,9 +30,12 @@ def insert_block(table_name, block_granularity_rows, block_rows): values_row = ", ".join("(1, '" + row + "')" for row in block_data) client.query("INSERT INTO {} VALUES {}".format(table_name, values_row)) + try: client.query("DROP TABLE IF EXISTS t") - client.query("CREATE TABLE t (v UInt8, data String) ENGINE = MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0") + client.query( + "CREATE TABLE t (v UInt8, data String) ENGINE = MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0" + ) client.query("SYSTEM STOP MERGES t") @@ -53,6 +61,10 @@ try: client.query("SYSTEM START MERGES t") client.query("OPTIMIZE TABLE t FINAL") - print(client.query_return_df("SELECT COUNT() as C FROM t FORMAT TabSeparatedWithNames")['C'][0]) + print( + client.query_return_df( + "SELECT COUNT() as C FROM t FORMAT TabSeparatedWithNames" + )["C"][0] + ) finally: client.query("DROP TABLE IF EXISTS t") diff --git a/tests/queries/0_stateless/01854_HTTP_dict_decompression.python b/tests/queries/0_stateless/01854_HTTP_dict_decompression.python index 4f6878665aa..7d98a24e83e 100644 --- a/tests/queries/0_stateless/01854_HTTP_dict_decompression.python +++ b/tests/queries/0_stateless/01854_HTTP_dict_decompression.python @@ -1,6 +1,6 @@ #!/usr/bin/env python3 -from http.server import SimpleHTTPRequestHandler,HTTPServer +from http.server import SimpleHTTPRequestHandler, HTTPServer import socket import csv import sys @@ -21,6 +21,7 @@ def is_ipv6(host): except: return True + def get_local_port(host, ipv6): if ipv6: family = socket.AF_INET6 @@ -31,8 +32,9 @@ def get_local_port(host, ipv6): fd.bind((host, 0)) return fd.getsockname()[1] -CLICKHOUSE_HOST = os.environ.get('CLICKHOUSE_HOST', 'localhost') -CLICKHOUSE_PORT_HTTP = os.environ.get('CLICKHOUSE_PORT_HTTP', '8123') + +CLICKHOUSE_HOST = os.environ.get("CLICKHOUSE_HOST", "localhost") +CLICKHOUSE_PORT_HTTP = os.environ.get("CLICKHOUSE_PORT_HTTP", "8123") ##################################################################################### # This test starts an HTTP server and serves data to clickhouse url-engine based table. @@ -42,16 +44,24 @@ CLICKHOUSE_PORT_HTTP = os.environ.get('CLICKHOUSE_PORT_HTTP', '8123') ##################################################################################### # IP-address of this host accessible from the outside world. Get the first one -HTTP_SERVER_HOST = subprocess.check_output(['hostname', '-i']).decode('utf-8').strip().split()[0] +HTTP_SERVER_HOST = ( + subprocess.check_output(["hostname", "-i"]).decode("utf-8").strip().split()[0] +) IS_IPV6 = is_ipv6(HTTP_SERVER_HOST) HTTP_SERVER_PORT = get_local_port(HTTP_SERVER_HOST, IS_IPV6) # IP address and port of the HTTP server started from this script. HTTP_SERVER_ADDRESS = (HTTP_SERVER_HOST, HTTP_SERVER_PORT) if IS_IPV6: - HTTP_SERVER_URL_STR = 'http://' + f'[{str(HTTP_SERVER_ADDRESS[0])}]:{str(HTTP_SERVER_ADDRESS[1])}' + "/" + HTTP_SERVER_URL_STR = ( + "http://" + + f"[{str(HTTP_SERVER_ADDRESS[0])}]:{str(HTTP_SERVER_ADDRESS[1])}" + + "/" + ) else: - HTTP_SERVER_URL_STR = 'http://' + f'{str(HTTP_SERVER_ADDRESS[0])}:{str(HTTP_SERVER_ADDRESS[1])}' + "/" + HTTP_SERVER_URL_STR = ( + "http://" + f"{str(HTTP_SERVER_ADDRESS[0])}:{str(HTTP_SERVER_ADDRESS[1])}" + "/" + ) # Because we need to check the content of file.csv we can create this content and avoid reading csv CSV_DATA = "Hello, 1\nWorld, 2\nThis, 152\nis, 9283\ntesting, 2313213\ndata, 555\n" @@ -59,19 +69,24 @@ CSV_DATA = "Hello, 1\nWorld, 2\nThis, 152\nis, 9283\ntesting, 2313213\ndata, 555 # Choose compression method # (Will change during test, need to check standard data sending, to make sure that nothing broke) -COMPRESS_METHOD = 'none' -ADDING_ENDING = '' -ENDINGS = ['.gz', '.xz'] +COMPRESS_METHOD = "none" +ADDING_ENDING = "" +ENDINGS = [".gz", ".xz"] SEND_ENCODING = True + def get_ch_answer(query): host = CLICKHOUSE_HOST if IS_IPV6: - host = f'[{host}]' + host = f"[{host}]" - url = os.environ.get('CLICKHOUSE_URL', 'http://{host}:{port}'.format(host=CLICKHOUSE_HOST, port=CLICKHOUSE_PORT_HTTP)) + url = os.environ.get( + "CLICKHOUSE_URL", + "http://{host}:{port}".format(host=CLICKHOUSE_HOST, port=CLICKHOUSE_PORT_HTTP), + ) return urllib.request.urlopen(url, data=query.encode()).read().decode() + def check_answers(query, answer): ch_answer = get_ch_answer(query) if ch_answer.strip() != answer.strip(): @@ -80,18 +95,19 @@ def check_answers(query, answer): print("Fetched answer :", ch_answer, file=sys.stderr) raise Exception("Fail on query") + # Server with head method which is useful for debuging by hands class HttpProcessor(SimpleHTTPRequestHandler): def _set_headers(self): self.send_response(200) if SEND_ENCODING: - self.send_header('Content-Encoding', COMPRESS_METHOD) - if COMPRESS_METHOD == 'none': - self.send_header('Content-Length', len(CSV_DATA.encode())) + self.send_header("Content-Encoding", COMPRESS_METHOD) + if COMPRESS_METHOD == "none": + self.send_header("Content-Length", len(CSV_DATA.encode())) else: self.compress_data() - self.send_header('Content-Length', len(self.data)) - self.send_header('Content-Type', 'text/csv') + self.send_header("Content-Length", len(self.data)) + self.send_header("Content-Type", "text/csv") self.end_headers() def do_HEAD(self): @@ -99,18 +115,17 @@ class HttpProcessor(SimpleHTTPRequestHandler): return def compress_data(self): - if COMPRESS_METHOD == 'gzip': + if COMPRESS_METHOD == "gzip": self.data = gzip.compress((CSV_DATA).encode()) - elif COMPRESS_METHOD == 'lzma': + elif COMPRESS_METHOD == "lzma": self.data = lzma.compress((CSV_DATA).encode()) else: - self.data = 'WRONG CONVERSATION'.encode() - + self.data = "WRONG CONVERSATION".encode() def do_GET(self): self._set_headers() - if COMPRESS_METHOD == 'none': + if COMPRESS_METHOD == "none": self.wfile.write(CSV_DATA.encode()) else: self.wfile.write(self.data) @@ -119,9 +134,11 @@ class HttpProcessor(SimpleHTTPRequestHandler): def log_message(self, format, *args): return + class HTTPServerV6(HTTPServer): address_family = socket.AF_INET6 + def start_server(requests_amount): if IS_IPV6: httpd = HTTPServerV6(HTTP_SERVER_ADDRESS, HttpProcessor) @@ -135,52 +152,60 @@ def start_server(requests_amount): t = threading.Thread(target=real_func) return t + ##################################################################### # Testing area. ##################################################################### -def test_select(dict_name="", schema="word String, counter UInt32", requests=[], answers=[], test_data=""): + +def test_select( + dict_name="", + schema="word String, counter UInt32", + requests=[], + answers=[], + test_data="", +): global ADDING_ENDING global SEND_ENCODING global COMPRESS_METHOD for i in range(len(requests)): if i > 2: - ADDING_ENDING = ENDINGS[i-3] + ADDING_ENDING = ENDINGS[i - 3] SEND_ENCODING = False if dict_name: get_ch_answer("drop dictionary if exists {}".format(dict_name)) - get_ch_answer('''CREATE DICTIONARY {} ({}) + get_ch_answer( + """CREATE DICTIONARY {} ({}) PRIMARY KEY word SOURCE(HTTP(url '{}' format 'CSV')) LAYOUT(complex_key_hashed()) - LIFETIME(0)'''.format(dict_name, schema, HTTP_SERVER_URL_STR + '/test.csv' + ADDING_ENDING)) + LIFETIME(0)""".format( + dict_name, schema, HTTP_SERVER_URL_STR + "/test.csv" + ADDING_ENDING + ) + ) COMPRESS_METHOD = requests[i] print(i, COMPRESS_METHOD, ADDING_ENDING, SEND_ENCODING) check_answers("SELECT * FROM {} ORDER BY word".format(dict_name), answers[i]) + def main(): # first three for encoding, second three for url - insert_requests = [ - 'none', - 'gzip', - 'lzma', - 'gzip', - 'lzma' - ] + insert_requests = ["none", "gzip", "lzma", "gzip", "lzma"] # This answers got experemently in non compressed mode and they are correct - answers = ['''Hello 1\nThis 152\nWorld 2\ndata 555\nis 9283\ntesting 2313213'''] * 5 + answers = ["""Hello 1\nThis 152\nWorld 2\ndata 555\nis 9283\ntesting 2313213"""] * 5 t = start_server(len(insert_requests)) t.start() - test_select(dict_name="test_table_select", requests=insert_requests, answers=answers) + test_select( + dict_name="test_table_select", requests=insert_requests, answers=answers + ) t.join() print("PASSED") - if __name__ == "__main__": try: main() @@ -191,5 +216,3 @@ if __name__ == "__main__": sys.stderr.flush() os._exit(1) - - diff --git a/tests/queries/0_stateless/02010_lc_native.python b/tests/queries/0_stateless/02010_lc_native.python index e6d6f9e1317..09ac60405e7 100755 --- a/tests/queries/0_stateless/02010_lc_native.python +++ b/tests/queries/0_stateless/02010_lc_native.python @@ -5,9 +5,10 @@ import socket import os import uuid -CLICKHOUSE_HOST = os.environ.get('CLICKHOUSE_HOST', '127.0.0.1') -CLICKHOUSE_PORT = int(os.environ.get('CLICKHOUSE_PORT_TCP', '900000')) -CLICKHOUSE_DATABASE = os.environ.get('CLICKHOUSE_DATABASE', 'default') +CLICKHOUSE_HOST = os.environ.get("CLICKHOUSE_HOST", "127.0.0.1") +CLICKHOUSE_PORT = int(os.environ.get("CLICKHOUSE_PORT_TCP", "900000")) +CLICKHOUSE_DATABASE = os.environ.get("CLICKHOUSE_DATABASE", "default") + def writeVarUInt(x, ba): for _ in range(0, 9): @@ -24,12 +25,12 @@ def writeVarUInt(x, ba): def writeStringBinary(s, ba): - b = bytes(s, 'utf-8') + b = bytes(s, "utf-8") writeVarUInt(len(s), ba) ba.extend(b) -def readStrict(s, size = 1): +def readStrict(s, size=1): res = bytearray() while size: cur = s.recv(size) @@ -48,18 +49,23 @@ def readUInt(s, size=1): val += res[i] << (i * 8) return val + def readUInt8(s): return readUInt(s) + def readUInt16(s): return readUInt(s, 2) + def readUInt32(s): return readUInt(s, 4) + def readUInt64(s): return readUInt(s, 8) + def readVarUInt(s): x = 0 for i in range(9): @@ -75,25 +81,25 @@ def readVarUInt(s): def readStringBinary(s): size = readVarUInt(s) s = readStrict(s, size) - return s.decode('utf-8') + return s.decode("utf-8") def sendHello(s): ba = bytearray() - writeVarUInt(0, ba) # Hello - writeStringBinary('simple native protocol', ba) + writeVarUInt(0, ba) # Hello + writeStringBinary("simple native protocol", ba) writeVarUInt(21, ba) writeVarUInt(9, ba) writeVarUInt(54449, ba) - writeStringBinary('default', ba) # database - writeStringBinary('default', ba) # user - writeStringBinary('', ba) # pwd + writeStringBinary("default", ba) # database + writeStringBinary("default", ba) # user + writeStringBinary("", ba) # pwd s.sendall(ba) def receiveHello(s): p_type = readVarUInt(s) - assert (p_type == 0) # Hello + assert p_type == 0 # Hello server_name = readStringBinary(s) # print("Server name: ", server_name) server_version_major = readVarUInt(s) @@ -111,78 +117,79 @@ def receiveHello(s): def serializeClientInfo(ba, query_id): - writeStringBinary('default', ba) # initial_user - writeStringBinary(query_id, ba) # initial_query_id - writeStringBinary('127.0.0.1:9000', ba) # initial_address - ba.extend([0] * 8) # initial_query_start_time_microseconds - ba.append(1) # TCP - writeStringBinary('os_user', ba) # os_user - writeStringBinary('client_hostname', ba) # client_hostname - writeStringBinary('client_name', ba) # client_name + writeStringBinary("default", ba) # initial_user + writeStringBinary(query_id, ba) # initial_query_id + writeStringBinary("127.0.0.1:9000", ba) # initial_address + ba.extend([0] * 8) # initial_query_start_time_microseconds + ba.append(1) # TCP + writeStringBinary("os_user", ba) # os_user + writeStringBinary("client_hostname", ba) # client_hostname + writeStringBinary("client_name", ba) # client_name writeVarUInt(21, ba) writeVarUInt(9, ba) writeVarUInt(54449, ba) - writeStringBinary('', ba) # quota_key - writeVarUInt(0, ba) # distributed_depth - writeVarUInt(1, ba) # client_version_patch - ba.append(0) # No telemetry + writeStringBinary("", ba) # quota_key + writeVarUInt(0, ba) # distributed_depth + writeVarUInt(1, ba) # client_version_patch + ba.append(0) # No telemetry def sendQuery(s, query): ba = bytearray() query_id = uuid.uuid4().hex - writeVarUInt(1, ba) # query + writeVarUInt(1, ba) # query writeStringBinary(query_id, ba) - ba.append(1) # INITIAL_QUERY + ba.append(1) # INITIAL_QUERY # client info serializeClientInfo(ba, query_id) - writeStringBinary('', ba) # No settings - writeStringBinary('', ba) # No interserver secret - writeVarUInt(2, ba) # Stage - Complete - ba.append(0) # No compression - writeStringBinary(query, ba) # query, finally + writeStringBinary("", ba) # No settings + writeStringBinary("", ba) # No interserver secret + writeVarUInt(2, ba) # Stage - Complete + ba.append(0) # No compression + writeStringBinary(query, ba) # query, finally s.sendall(ba) def serializeBlockInfo(ba): - writeVarUInt(1, ba) # 1 - ba.append(0) # is_overflows - writeVarUInt(2, ba) # 2 - writeVarUInt(0, ba) # 0 - ba.extend([0] * 4) # bucket_num + writeVarUInt(1, ba) # 1 + ba.append(0) # is_overflows + writeVarUInt(2, ba) # 2 + writeVarUInt(0, ba) # 0 + ba.extend([0] * 4) # bucket_num def sendEmptyBlock(s): ba = bytearray() - writeVarUInt(2, ba) # Data - writeStringBinary('', ba) + writeVarUInt(2, ba) # Data + writeStringBinary("", ba) serializeBlockInfo(ba) - writeVarUInt(0, ba) # rows - writeVarUInt(0, ba) # columns + writeVarUInt(0, ba) # rows + writeVarUInt(0, ba) # columns s.sendall(ba) def assertPacket(packet, expected): - assert(packet == expected), packet + assert packet == expected, packet + def readHeader(s): packet_type = readVarUInt(s) - if packet_type == 2: # Exception + if packet_type == 2: # Exception raise RuntimeError(readException(s)) - assertPacket(packet_type, 1) # Data + assertPacket(packet_type, 1) # Data - readStringBinary(s) # external table name + readStringBinary(s) # external table name # BlockInfo - assertPacket(readVarUInt(s), 1) # 1 - assertPacket(readUInt8(s), 0) # is_overflows - assertPacket(readVarUInt(s), 2) # 2 - assertPacket(readUInt32(s), 4294967295) # bucket_num - assertPacket(readVarUInt(s), 0) # 0 - columns = readVarUInt(s) # rows - rows = readVarUInt(s) # columns + assertPacket(readVarUInt(s), 1) # 1 + assertPacket(readUInt8(s), 0) # is_overflows + assertPacket(readVarUInt(s), 2) # 2 + assertPacket(readUInt32(s), 4294967295) # bucket_num + assertPacket(readVarUInt(s), 0) # 0 + columns = readVarUInt(s) # rows + rows = readVarUInt(s) # columns print("Rows {} Columns {}".format(rows, columns)) for _ in range(columns): col_name = readStringBinary(s) @@ -194,9 +201,9 @@ def readException(s): code = readUInt32(s) name = readStringBinary(s) text = readStringBinary(s) - readStringBinary(s) # trace - assertPacket(readUInt8(s), 0) # has_nested - return "code {}: {}".format(code, text.replace('DB::Exception:', '')) + readStringBinary(s) # trace + assertPacket(readUInt8(s), 0) # has_nested + return "code {}: {}".format(code, text.replace("DB::Exception:", "")) def insertValidLowCardinalityRow(): @@ -205,7 +212,12 @@ def insertValidLowCardinalityRow(): s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT)) sendHello(s) receiveHello(s) - sendQuery(s, 'insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV'.format(CLICKHOUSE_DATABASE)) + sendQuery( + s, + "insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV".format( + CLICKHOUSE_DATABASE + ), + ) # external tables sendEmptyBlock(s) @@ -213,25 +225,27 @@ def insertValidLowCardinalityRow(): # Data ba = bytearray() - writeVarUInt(2, ba) # Data - writeStringBinary('', ba) + writeVarUInt(2, ba) # Data + writeStringBinary("", ba) serializeBlockInfo(ba) - writeVarUInt(1, ba) # rows - writeVarUInt(1, ba) # columns - writeStringBinary('x', ba) - writeStringBinary('LowCardinality(String)', ba) - ba.extend([1] + [0] * 7) # SharedDictionariesWithAdditionalKeys - ba.extend([3, 2] + [0] * 6) # indexes type: UInt64 [3], with additional keys [2] - ba.extend([1] + [0] * 7) # num_keys in dict - writeStringBinary('hello', ba) # key - ba.extend([1] + [0] * 7) # num_indexes - ba.extend([0] * 8) # UInt64 index (0 for 'hello') + writeVarUInt(1, ba) # rows + writeVarUInt(1, ba) # columns + writeStringBinary("x", ba) + writeStringBinary("LowCardinality(String)", ba) + ba.extend([1] + [0] * 7) # SharedDictionariesWithAdditionalKeys + ba.extend( + [3, 2] + [0] * 6 + ) # indexes type: UInt64 [3], with additional keys [2] + ba.extend([1] + [0] * 7) # num_keys in dict + writeStringBinary("hello", ba) # key + ba.extend([1] + [0] * 7) # num_indexes + ba.extend([0] * 8) # UInt64 index (0 for 'hello') s.sendall(ba) # Fin block sendEmptyBlock(s) - assertPacket(readVarUInt(s), 5) # End of stream + assertPacket(readVarUInt(s), 5) # End of stream s.close() @@ -241,7 +255,12 @@ def insertLowCardinalityRowWithIndexOverflow(): s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT)) sendHello(s) receiveHello(s) - sendQuery(s, 'insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV'.format(CLICKHOUSE_DATABASE)) + sendQuery( + s, + "insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV".format( + CLICKHOUSE_DATABASE + ), + ) # external tables sendEmptyBlock(s) @@ -249,19 +268,21 @@ def insertLowCardinalityRowWithIndexOverflow(): # Data ba = bytearray() - writeVarUInt(2, ba) # Data - writeStringBinary('', ba) + writeVarUInt(2, ba) # Data + writeStringBinary("", ba) serializeBlockInfo(ba) - writeVarUInt(1, ba) # rows - writeVarUInt(1, ba) # columns - writeStringBinary('x', ba) - writeStringBinary('LowCardinality(String)', ba) - ba.extend([1] + [0] * 7) # SharedDictionariesWithAdditionalKeys - ba.extend([3, 2] + [0] * 6) # indexes type: UInt64 [3], with additional keys [2] - ba.extend([1] + [0] * 7) # num_keys in dict - writeStringBinary('hello', ba) # key - ba.extend([1] + [0] * 7) # num_indexes - ba.extend([0] * 7 + [1]) # UInt64 index (overflow) + writeVarUInt(1, ba) # rows + writeVarUInt(1, ba) # columns + writeStringBinary("x", ba) + writeStringBinary("LowCardinality(String)", ba) + ba.extend([1] + [0] * 7) # SharedDictionariesWithAdditionalKeys + ba.extend( + [3, 2] + [0] * 6 + ) # indexes type: UInt64 [3], with additional keys [2] + ba.extend([1] + [0] * 7) # num_keys in dict + writeStringBinary("hello", ba) # key + ba.extend([1] + [0] * 7) # num_indexes + ba.extend([0] * 7 + [1]) # UInt64 index (overflow) s.sendall(ba) assertPacket(readVarUInt(s), 2) @@ -275,7 +296,12 @@ def insertLowCardinalityRowWithIncorrectDictType(): s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT)) sendHello(s) receiveHello(s) - sendQuery(s, 'insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV'.format(CLICKHOUSE_DATABASE)) + sendQuery( + s, + "insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV".format( + CLICKHOUSE_DATABASE + ), + ) # external tables sendEmptyBlock(s) @@ -283,32 +309,40 @@ def insertLowCardinalityRowWithIncorrectDictType(): # Data ba = bytearray() - writeVarUInt(2, ba) # Data - writeStringBinary('', ba) + writeVarUInt(2, ba) # Data + writeStringBinary("", ba) serializeBlockInfo(ba) - writeVarUInt(1, ba) # rows - writeVarUInt(1, ba) # columns - writeStringBinary('x', ba) - writeStringBinary('LowCardinality(String)', ba) - ba.extend([1] + [0] * 7) # SharedDictionariesWithAdditionalKeys - ba.extend([3, 3] + [0] * 6) # indexes type: UInt64 [3], with global dict and add keys [1 + 2] - ba.extend([1] + [0] * 7) # num_keys in dict - writeStringBinary('hello', ba) # key - ba.extend([1] + [0] * 7) # num_indexes - ba.extend([0] * 8) # UInt64 index (overflow) + writeVarUInt(1, ba) # rows + writeVarUInt(1, ba) # columns + writeStringBinary("x", ba) + writeStringBinary("LowCardinality(String)", ba) + ba.extend([1] + [0] * 7) # SharedDictionariesWithAdditionalKeys + ba.extend( + [3, 3] + [0] * 6 + ) # indexes type: UInt64 [3], with global dict and add keys [1 + 2] + ba.extend([1] + [0] * 7) # num_keys in dict + writeStringBinary("hello", ba) # key + ba.extend([1] + [0] * 7) # num_indexes + ba.extend([0] * 8) # UInt64 index (overflow) s.sendall(ba) assertPacket(readVarUInt(s), 2) print(readException(s)) s.close() + def insertLowCardinalityRowWithIncorrectAdditionalKeys(): with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as s: s.settimeout(30) s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT)) sendHello(s) receiveHello(s) - sendQuery(s, 'insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV'.format(CLICKHOUSE_DATABASE)) + sendQuery( + s, + "insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV".format( + CLICKHOUSE_DATABASE + ), + ) # external tables sendEmptyBlock(s) @@ -316,30 +350,34 @@ def insertLowCardinalityRowWithIncorrectAdditionalKeys(): # Data ba = bytearray() - writeVarUInt(2, ba) # Data - writeStringBinary('', ba) + writeVarUInt(2, ba) # Data + writeStringBinary("", ba) serializeBlockInfo(ba) - writeVarUInt(1, ba) # rows - writeVarUInt(1, ba) # columns - writeStringBinary('x', ba) - writeStringBinary('LowCardinality(String)', ba) - ba.extend([1] + [0] * 7) # SharedDictionariesWithAdditionalKeys - ba.extend([3, 0] + [0] * 6) # indexes type: UInt64 [3], with NO additional keys [0] - ba.extend([1] + [0] * 7) # num_keys in dict - writeStringBinary('hello', ba) # key - ba.extend([1] + [0] * 7) # num_indexes - ba.extend([0] * 8) # UInt64 index (0 for 'hello') + writeVarUInt(1, ba) # rows + writeVarUInt(1, ba) # columns + writeStringBinary("x", ba) + writeStringBinary("LowCardinality(String)", ba) + ba.extend([1] + [0] * 7) # SharedDictionariesWithAdditionalKeys + ba.extend( + [3, 0] + [0] * 6 + ) # indexes type: UInt64 [3], with NO additional keys [0] + ba.extend([1] + [0] * 7) # num_keys in dict + writeStringBinary("hello", ba) # key + ba.extend([1] + [0] * 7) # num_indexes + ba.extend([0] * 8) # UInt64 index (0 for 'hello') s.sendall(ba) assertPacket(readVarUInt(s), 2) print(readException(s)) s.close() + def main(): insertValidLowCardinalityRow() insertLowCardinalityRowWithIndexOverflow() insertLowCardinalityRowWithIncorrectDictType() insertLowCardinalityRowWithIncorrectAdditionalKeys() + if __name__ == "__main__": main() diff --git a/tests/queries/0_stateless/02126_url_auth.python b/tests/queries/0_stateless/02126_url_auth.python index 57b16fb413e..9b2e68a017d 100644 --- a/tests/queries/0_stateless/02126_url_auth.python +++ b/tests/queries/0_stateless/02126_url_auth.python @@ -12,6 +12,7 @@ import subprocess from io import StringIO from http.server import BaseHTTPRequestHandler, HTTPServer + def is_ipv6(host): try: socket.inet_aton(host) @@ -19,6 +20,7 @@ def is_ipv6(host): except: return True + def get_local_port(host, ipv6): if ipv6: family = socket.AF_INET6 @@ -29,8 +31,9 @@ def get_local_port(host, ipv6): fd.bind((host, 0)) return fd.getsockname()[1] -CLICKHOUSE_HOST = os.environ.get('CLICKHOUSE_HOST', '127.0.0.1') -CLICKHOUSE_PORT_HTTP = os.environ.get('CLICKHOUSE_PORT_HTTP', '8123') + +CLICKHOUSE_HOST = os.environ.get("CLICKHOUSE_HOST", "127.0.0.1") +CLICKHOUSE_PORT_HTTP = os.environ.get("CLICKHOUSE_PORT_HTTP", "8123") ##################################################################################### # This test starts an HTTP server and serves data to clickhouse url-engine based table. @@ -39,27 +42,42 @@ CLICKHOUSE_PORT_HTTP = os.environ.get('CLICKHOUSE_PORT_HTTP', '8123') ##################################################################################### # IP-address of this host accessible from the outside world. Get the first one -HTTP_SERVER_HOST = subprocess.check_output(['hostname', '-i']).decode('utf-8').strip().split()[0] +HTTP_SERVER_HOST = ( + subprocess.check_output(["hostname", "-i"]).decode("utf-8").strip().split()[0] +) IS_IPV6 = is_ipv6(HTTP_SERVER_HOST) HTTP_SERVER_PORT = get_local_port(HTTP_SERVER_HOST, IS_IPV6) # IP address and port of the HTTP server started from this script. HTTP_SERVER_ADDRESS = (HTTP_SERVER_HOST, HTTP_SERVER_PORT) if IS_IPV6: - HTTP_SERVER_URL_STR = 'http://' + f'[{str(HTTP_SERVER_ADDRESS[0])}]:{str(HTTP_SERVER_ADDRESS[1])}' + "/" + HTTP_SERVER_URL_STR = ( + "http://" + + f"[{str(HTTP_SERVER_ADDRESS[0])}]:{str(HTTP_SERVER_ADDRESS[1])}" + + "/" + ) else: - HTTP_SERVER_URL_STR = 'http://' + f'{str(HTTP_SERVER_ADDRESS[0])}:{str(HTTP_SERVER_ADDRESS[1])}' + "/" + HTTP_SERVER_URL_STR = ( + "http://" + f"{str(HTTP_SERVER_ADDRESS[0])}:{str(HTTP_SERVER_ADDRESS[1])}" + "/" + ) + +CSV_DATA = os.path.join( + tempfile._get_default_tempdir(), next(tempfile._get_candidate_names()) +) -CSV_DATA = os.path.join(tempfile._get_default_tempdir(), next(tempfile._get_candidate_names())) def get_ch_answer(query): host = CLICKHOUSE_HOST if IS_IPV6: - host = f'[{host}]' + host = f"[{host}]" - url = os.environ.get('CLICKHOUSE_URL', 'http://{host}:{port}'.format(host=CLICKHOUSE_HOST, port=CLICKHOUSE_PORT_HTTP)) + url = os.environ.get( + "CLICKHOUSE_URL", + "http://{host}:{port}".format(host=CLICKHOUSE_HOST, port=CLICKHOUSE_PORT_HTTP), + ) return urllib.request.urlopen(url, data=query.encode()).read().decode() + def check_answers(query, answer): ch_answer = get_ch_answer(query) if ch_answer.strip() != answer.strip(): @@ -68,15 +86,16 @@ def check_answers(query, answer): print("Fetched answer :", ch_answer, file=sys.stderr) raise Exception("Fail on query") + class CSVHTTPServer(BaseHTTPRequestHandler): def _set_headers(self): self.send_response(200) - self.send_header('Content-type', 'text/csv') + self.send_header("Content-type", "text/csv") self.end_headers() def do_GET(self): self._set_headers() - self.wfile.write(('hello, world').encode()) + self.wfile.write(("hello, world").encode()) # with open(CSV_DATA, 'r') as fl: # reader = csv.reader(fl, delimiter=',') # for row in reader: @@ -84,33 +103,33 @@ class CSVHTTPServer(BaseHTTPRequestHandler): return def read_chunk(self): - msg = '' + msg = "" while True: sym = self.rfile.read(1) - if sym == '': + if sym == "": break - msg += sym.decode('utf-8') - if msg.endswith('\r\n'): + msg += sym.decode("utf-8") + if msg.endswith("\r\n"): break length = int(msg[:-2], 16) if length == 0: - return '' + return "" content = self.rfile.read(length) - self.rfile.read(2) # read sep \r\n - return content.decode('utf-8') + self.rfile.read(2) # read sep \r\n + return content.decode("utf-8") def do_POST(self): - data = '' + data = "" while True: chunk = self.read_chunk() if not chunk: break data += chunk with StringIO(data) as fl: - reader = csv.reader(fl, delimiter=',') - with open(CSV_DATA, 'a') as d: + reader = csv.reader(fl, delimiter=",") + with open(CSV_DATA, "a") as d: for row in reader: - d.write(','.join(row) + '\n') + d.write(",".join(row) + "\n") self._set_headers() self.wfile.write(b"ok") @@ -121,6 +140,7 @@ class CSVHTTPServer(BaseHTTPRequestHandler): class HTTPServerV6(HTTPServer): address_family = socket.AF_INET6 + def start_server(): if IS_IPV6: httpd = HTTPServerV6(HTTP_SERVER_ADDRESS, CSVHTTPServer) @@ -130,57 +150,87 @@ def start_server(): t = threading.Thread(target=httpd.serve_forever) return t, httpd + # test section -def test_select(table_name="", schema="str String,numuint UInt32,numint Int32,double Float64", requests=[], answers=[], test_data=""): - with open(CSV_DATA, 'w') as f: # clear file - f.write('') + +def test_select( + table_name="", + schema="str String,numuint UInt32,numint Int32,double Float64", + requests=[], + answers=[], + test_data="", +): + with open(CSV_DATA, "w") as f: # clear file + f.write("") if test_data: - with open(CSV_DATA, 'w') as f: + with open(CSV_DATA, "w") as f: f.write(test_data + "\n") if table_name: get_ch_answer("drop table if exists {}".format(table_name)) - get_ch_answer("create table {} ({}) engine=URL('{}', 'CSV')".format(table_name, schema, HTTP_SERVER_URL_STR)) + get_ch_answer( + "create table {} ({}) engine=URL('{}', 'CSV')".format( + table_name, schema, HTTP_SERVER_URL_STR + ) + ) for i in range(len(requests)): tbl = table_name if not tbl: - tbl = "url('{addr}', 'CSV', '{schema}')".format(addr=HTTP_SERVER_URL_STR, schema=schema) + tbl = "url('{addr}', 'CSV', '{schema}')".format( + addr=HTTP_SERVER_URL_STR, schema=schema + ) check_answers(requests[i].format(tbl=tbl), answers[i]) if table_name: get_ch_answer("drop table if exists {}".format(table_name)) -def test_insert(table_name="", schema="str String,numuint UInt32,numint Int32,double Float64", requests_insert=[], requests_select=[], answers=[]): - with open(CSV_DATA, 'w') as f: # flush test file - f.write('') + +def test_insert( + table_name="", + schema="str String,numuint UInt32,numint Int32,double Float64", + requests_insert=[], + requests_select=[], + answers=[], +): + with open(CSV_DATA, "w") as f: # flush test file + f.write("") if table_name: get_ch_answer("drop table if exists {}".format(table_name)) - get_ch_answer("create table {} ({}) engine=URL('{}', 'CSV')".format(table_name, schema, HTTP_SERVER_URL_STR)) + get_ch_answer( + "create table {} ({}) engine=URL('{}', 'CSV')".format( + table_name, schema, HTTP_SERVER_URL_STR + ) + ) for req in requests_insert: tbl = table_name if not tbl: - tbl = "table function url('{addr}', 'CSV', '{schema}')".format(addr=HTTP_SERVER_URL_STR, schema=schema) + tbl = "table function url('{addr}', 'CSV', '{schema}')".format( + addr=HTTP_SERVER_URL_STR, schema=schema + ) get_ch_answer(req.format(tbl=tbl)) - for i in range(len(requests_select)): tbl = table_name if not tbl: - tbl = "url('{addr}', 'CSV', '{schema}')".format(addr=HTTP_SERVER_URL_STR, schema=schema) + tbl = "url('{addr}', 'CSV', '{schema}')".format( + addr=HTTP_SERVER_URL_STR, schema=schema + ) check_answers(requests_select[i].format(tbl=tbl), answers[i]) if table_name: get_ch_answer("drop table if exists {}".format(table_name)) + def test_select_url_engine(requests=[], answers=[], test_data=""): for i in range(len(requests)): check_answers(requests[i], answers[i]) + def main(): test_data = "Hello,2,-2,7.7\nWorld,2,-5,8.8" """ @@ -203,19 +253,29 @@ def main(): """ if IS_IPV6: - query = "select * from url('http://guest:guest@" + f'[{str(HTTP_SERVER_ADDRESS[0])}]:{str(HTTP_SERVER_ADDRESS[1])}' + "/', 'RawBLOB', 'a String')" + query = ( + "select * from url('http://guest:guest@" + + f"[{str(HTTP_SERVER_ADDRESS[0])}]:{str(HTTP_SERVER_ADDRESS[1])}" + + "/', 'RawBLOB', 'a String')" + ) else: - query = "select * from url('http://guest:guest@" + f'{str(HTTP_SERVER_ADDRESS[0])}:{str(HTTP_SERVER_ADDRESS[1])}' + "/', 'RawBLOB', 'a String')" - - + query = ( + "select * from url('http://guest:guest@" + + f"{str(HTTP_SERVER_ADDRESS[0])}:{str(HTTP_SERVER_ADDRESS[1])}" + + "/', 'RawBLOB', 'a String')" + ) select_requests_url_auth = { - query : 'hello, world', + query: "hello, world", } t, httpd = start_server() t.start() - test_select(requests=list(select_requests_url_auth.keys()), answers=list(select_requests_url_auth.values()), test_data=test_data) + test_select( + requests=list(select_requests_url_auth.keys()), + answers=list(select_requests_url_auth.values()), + test_data=test_data, + ) httpd.shutdown() t.join() print("PASSED") diff --git a/tests/queries/0_stateless/02158_proportions_ztest_cmp.python b/tests/queries/0_stateless/02158_proportions_ztest_cmp.python index d622004db28..0555f8c36ec 100644 --- a/tests/queries/0_stateless/02158_proportions_ztest_cmp.python +++ b/tests/queries/0_stateless/02158_proportions_ztest_cmp.python @@ -1,4 +1,4 @@ -#!/usr/bin/env python3 +#!/usr/bin/env python3 import os import sys from math import sqrt, nan @@ -8,7 +8,7 @@ import pandas as pd import numpy as np CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) from pure_http_client import ClickHouseClient @@ -25,7 +25,7 @@ def twosample_proportion_ztest(s1, s2, t1, t2, alpha): return nan, nan, nan, nan z_stat = (p1 - p2) / se - one_side = 1 - stats.norm.cdf(abs(z_stat)) + one_side = 1 - stats.norm.cdf(abs(z_stat)) p_value = one_side * 2 z = stats.norm.ppf(1 - 0.5 * alpha) @@ -38,71 +38,171 @@ def twosample_proportion_ztest(s1, s2, t1, t2, alpha): def test_and_check(name, z_stat, p_value, ci_lower, ci_upper, precision=1e-2): client = ClickHouseClient() real = client.query_return_df( - "SELECT roundBankers({}.1, 16) as z_stat, ".format(name) + - "roundBankers({}.2, 16) as p_value, ".format(name) + - "roundBankers({}.3, 16) as ci_lower, ".format(name) + - "roundBankers({}.4, 16) as ci_upper ".format(name) + - "FORMAT TabSeparatedWithNames;") - real_z_stat = real['z_stat'][0] - real_p_value = real['p_value'][0] - real_ci_lower = real['ci_lower'][0] - real_ci_upper = real['ci_upper'][0] - assert((np.isnan(real_z_stat) and np.isnan(z_stat)) or abs(real_z_stat - np.float64(z_stat)) < precision), "clickhouse_z_stat {}, py_z_stat {}".format(real_z_stat, z_stat) - assert((np.isnan(real_p_value) and np.isnan(p_value)) or abs(real_p_value - np.float64(p_value)) < precision), "clickhouse_p_value {}, py_p_value {}".format(real_p_value, p_value) - assert((np.isnan(real_ci_lower) and np.isnan(ci_lower)) or abs(real_ci_lower - np.float64(ci_lower)) < precision), "clickhouse_ci_lower {}, py_ci_lower {}".format(real_ci_lower, ci_lower) - assert((np.isnan(real_ci_upper) and np.isnan(ci_upper)) or abs(real_ci_upper - np.float64(ci_upper)) < precision), "clickhouse_ci_upper {}, py_ci_upper {}".format(real_ci_upper, ci_upper) + "SELECT roundBankers({}.1, 16) as z_stat, ".format(name) + + "roundBankers({}.2, 16) as p_value, ".format(name) + + "roundBankers({}.3, 16) as ci_lower, ".format(name) + + "roundBankers({}.4, 16) as ci_upper ".format(name) + + "FORMAT TabSeparatedWithNames;" + ) + real_z_stat = real["z_stat"][0] + real_p_value = real["p_value"][0] + real_ci_lower = real["ci_lower"][0] + real_ci_upper = real["ci_upper"][0] + assert (np.isnan(real_z_stat) and np.isnan(z_stat)) or abs( + real_z_stat - np.float64(z_stat) + ) < precision, "clickhouse_z_stat {}, py_z_stat {}".format(real_z_stat, z_stat) + assert (np.isnan(real_p_value) and np.isnan(p_value)) or abs( + real_p_value - np.float64(p_value) + ) < precision, "clickhouse_p_value {}, py_p_value {}".format(real_p_value, p_value) + assert (np.isnan(real_ci_lower) and np.isnan(ci_lower)) or abs( + real_ci_lower - np.float64(ci_lower) + ) < precision, "clickhouse_ci_lower {}, py_ci_lower {}".format( + real_ci_lower, ci_lower + ) + assert (np.isnan(real_ci_upper) and np.isnan(ci_upper)) or abs( + real_ci_upper - np.float64(ci_upper) + ) < precision, "clickhouse_ci_upper {}, py_ci_upper {}".format( + real_ci_upper, ci_upper + ) def test_mean_ztest(): counts = [0, 0] nobs = [0, 0] - z_stat, p_value, ci_lower, ci_upper = twosample_proportion_ztest(counts[0], counts[1], nobs[0], nobs[1], 0.05) - test_and_check("proportionsZTest(%d, %d, %d, %d, 0.95, 'unpooled')" % (counts[0], counts[1], nobs[0], nobs[1]), z_stat, p_value, ci_lower, ci_upper) - z_stat, p_value, ci_lower, ci_upper = twosample_proportion_ztest(10, 10, 10, 10, 0.05) + z_stat, p_value, ci_lower, ci_upper = twosample_proportion_ztest( + counts[0], counts[1], nobs[0], nobs[1], 0.05 + ) + test_and_check( + "proportionsZTest(%d, %d, %d, %d, 0.95, 'unpooled')" + % (counts[0], counts[1], nobs[0], nobs[1]), + z_stat, + p_value, + ci_lower, + ci_upper, + ) + z_stat, p_value, ci_lower, ci_upper = twosample_proportion_ztest( + 10, 10, 10, 10, 0.05 + ) counts = [10, 10] nobs = [10, 10] - z_stat, p_value, ci_lower, ci_upper = twosample_proportion_ztest(counts[0], counts[1], nobs[0], nobs[1], 0.05) - test_and_check("proportionsZTest(%d, %d, %d, %d, 0.95, 'unpooled')" % (counts[0], counts[1], nobs[0], nobs[1]), z_stat, p_value, ci_lower, ci_upper) - z_stat, p_value, ci_lower, ci_upper = twosample_proportion_ztest(10, 10, 10, 10, 0.05) + z_stat, p_value, ci_lower, ci_upper = twosample_proportion_ztest( + counts[0], counts[1], nobs[0], nobs[1], 0.05 + ) + test_and_check( + "proportionsZTest(%d, %d, %d, %d, 0.95, 'unpooled')" + % (counts[0], counts[1], nobs[0], nobs[1]), + z_stat, + p_value, + ci_lower, + ci_upper, + ) + z_stat, p_value, ci_lower, ci_upper = twosample_proportion_ztest( + 10, 10, 10, 10, 0.05 + ) counts = [16, 16] nobs = [16, 18] - z_stat, p_value, ci_lower, ci_upper = twosample_proportion_ztest(counts[0], counts[1], nobs[0], nobs[1], 0.05) - test_and_check("proportionsZTest(%d, %d, %d, %d, 0.95, 'unpooled')" % (counts[0], counts[1], nobs[0], nobs[1]), z_stat, p_value, ci_lower, ci_upper) + z_stat, p_value, ci_lower, ci_upper = twosample_proportion_ztest( + counts[0], counts[1], nobs[0], nobs[1], 0.05 + ) + test_and_check( + "proportionsZTest(%d, %d, %d, %d, 0.95, 'unpooled')" + % (counts[0], counts[1], nobs[0], nobs[1]), + z_stat, + p_value, + ci_lower, + ci_upper, + ) counts = [10, 20] nobs = [30, 40] - z_stat, p_value, ci_lower, ci_upper = twosample_proportion_ztest(counts[0], counts[1], nobs[0], nobs[1], 0.05) - test_and_check("proportionsZTest(%d, %d, %d, %d, 0.95, 'unpooled')" % (counts[0], counts[1], nobs[0], nobs[1]), z_stat, p_value, ci_lower, ci_upper) + z_stat, p_value, ci_lower, ci_upper = twosample_proportion_ztest( + counts[0], counts[1], nobs[0], nobs[1], 0.05 + ) + test_and_check( + "proportionsZTest(%d, %d, %d, %d, 0.95, 'unpooled')" + % (counts[0], counts[1], nobs[0], nobs[1]), + z_stat, + p_value, + ci_lower, + ci_upper, + ) counts = [20, 10] nobs = [40, 30] - z_stat, p_value, ci_lower, ci_upper = twosample_proportion_ztest(counts[0], counts[1], nobs[0], nobs[1], 0.05) - test_and_check("proportionsZTest(%d, %d, %d, %d, 0.95, 'unpooled')" % (counts[0], counts[1], nobs[0], nobs[1]), z_stat, p_value, ci_lower, ci_upper) + z_stat, p_value, ci_lower, ci_upper = twosample_proportion_ztest( + counts[0], counts[1], nobs[0], nobs[1], 0.05 + ) + test_and_check( + "proportionsZTest(%d, %d, %d, %d, 0.95, 'unpooled')" + % (counts[0], counts[1], nobs[0], nobs[1]), + z_stat, + p_value, + ci_lower, + ci_upper, + ) - counts = [randrange(10,20), randrange(10,20)] - nobs = [randrange(counts[0] + 1, counts[0] * 2), randrange(counts[1], counts[1] * 2)] - z_stat, p_value, ci_lower, ci_upper = twosample_proportion_ztest(counts[0], counts[1], nobs[0], nobs[1], 0.05) - test_and_check("proportionsZTest(%d, %d, %d, %d, 0.95, 'unpooled')" % (counts[0], counts[1], nobs[0], nobs[1]), z_stat, p_value, ci_lower, ci_upper) + counts = [randrange(10, 20), randrange(10, 20)] + nobs = [ + randrange(counts[0] + 1, counts[0] * 2), + randrange(counts[1], counts[1] * 2), + ] + z_stat, p_value, ci_lower, ci_upper = twosample_proportion_ztest( + counts[0], counts[1], nobs[0], nobs[1], 0.05 + ) + test_and_check( + "proportionsZTest(%d, %d, %d, %d, 0.95, 'unpooled')" + % (counts[0], counts[1], nobs[0], nobs[1]), + z_stat, + p_value, + ci_lower, + ci_upper, + ) - counts = [randrange(1,100), randrange(1,200)] + counts = [randrange(1, 100), randrange(1, 200)] nobs = [randrange(counts[0], counts[0] * 2), randrange(counts[1], counts[1] * 3)] - z_stat, p_value, ci_lower, ci_upper = twosample_proportion_ztest(counts[0], counts[1], nobs[0], nobs[1], 0.05) - test_and_check("proportionsZTest(%d, %d, %d, %d, 0.95, 'unpooled')" % (counts[0], counts[1], nobs[0], nobs[1]), z_stat, p_value, ci_lower, ci_upper) + z_stat, p_value, ci_lower, ci_upper = twosample_proportion_ztest( + counts[0], counts[1], nobs[0], nobs[1], 0.05 + ) + test_and_check( + "proportionsZTest(%d, %d, %d, %d, 0.95, 'unpooled')" + % (counts[0], counts[1], nobs[0], nobs[1]), + z_stat, + p_value, + ci_lower, + ci_upper, + ) - counts = [randrange(1,200), randrange(1,100)] + counts = [randrange(1, 200), randrange(1, 100)] nobs = [randrange(counts[0], counts[0] * 3), randrange(counts[1], counts[1] * 2)] - z_stat, p_value, ci_lower, ci_upper = twosample_proportion_ztest(counts[0], counts[1], nobs[0], nobs[1], 0.05) - test_and_check("proportionsZTest(%d, %d, %d, %d, 0.95, 'unpooled')" % (counts[0], counts[1], nobs[0], nobs[1]), z_stat, p_value, ci_lower, ci_upper) + z_stat, p_value, ci_lower, ci_upper = twosample_proportion_ztest( + counts[0], counts[1], nobs[0], nobs[1], 0.05 + ) + test_and_check( + "proportionsZTest(%d, %d, %d, %d, 0.95, 'unpooled')" + % (counts[0], counts[1], nobs[0], nobs[1]), + z_stat, + p_value, + ci_lower, + ci_upper, + ) - counts = [randrange(1,1000), randrange(1,1000)] + counts = [randrange(1, 1000), randrange(1, 1000)] nobs = [randrange(counts[0], counts[0] * 2), randrange(counts[1], counts[1] * 2)] - z_stat, p_value, ci_lower, ci_upper = twosample_proportion_ztest(counts[0], counts[1], nobs[0], nobs[1], 0.05) - test_and_check("proportionsZTest(%d, %d, %d, %d, 0.95, 'unpooled')" % (counts[0], counts[1], nobs[0], nobs[1]), z_stat, p_value, ci_lower, ci_upper) + z_stat, p_value, ci_lower, ci_upper = twosample_proportion_ztest( + counts[0], counts[1], nobs[0], nobs[1], 0.05 + ) + test_and_check( + "proportionsZTest(%d, %d, %d, %d, 0.95, 'unpooled')" + % (counts[0], counts[1], nobs[0], nobs[1]), + z_stat, + p_value, + ci_lower, + ci_upper, + ) if __name__ == "__main__": test_mean_ztest() print("Ok.") - diff --git a/tests/queries/0_stateless/02158_ztest_cmp.python b/tests/queries/0_stateless/02158_ztest_cmp.python index 8fc22d78e74..9591a150337 100644 --- a/tests/queries/0_stateless/02158_ztest_cmp.python +++ b/tests/queries/0_stateless/02158_ztest_cmp.python @@ -1,4 +1,4 @@ -#!/usr/bin/env python3 +#!/usr/bin/env python3 import os import sys from statistics import variance @@ -7,7 +7,7 @@ import pandas as pd import numpy as np CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) from pure_http_client import ClickHouseClient @@ -30,46 +30,95 @@ def twosample_mean_ztest(rvs1, rvs2, alpha=0.05): def test_and_check(name, a, b, t_stat, p_value, ci_low, ci_high, precision=1e-2): client = ClickHouseClient() client.query("DROP TABLE IF EXISTS ztest;") - client.query("CREATE TABLE ztest (left Float64, right UInt8) ENGINE = Memory;"); - client.query("INSERT INTO ztest VALUES {};".format(", ".join(['({},{})'.format(i, 0) for i in a]))) - client.query("INSERT INTO ztest VALUES {};".format(", ".join(['({},{})'.format(j, 1) for j in b]))) + client.query("CREATE TABLE ztest (left Float64, right UInt8) ENGINE = Memory;") + client.query( + "INSERT INTO ztest VALUES {};".format( + ", ".join(["({},{})".format(i, 0) for i in a]) + ) + ) + client.query( + "INSERT INTO ztest VALUES {};".format( + ", ".join(["({},{})".format(j, 1) for j in b]) + ) + ) real = client.query_return_df( - "SELECT roundBankers({}(left, right).1, 16) as t_stat, ".format(name) + - "roundBankers({}(left, right).2, 16) as p_value, ".format(name) + - "roundBankers({}(left, right).3, 16) as ci_low, ".format(name) + - "roundBankers({}(left, right).4, 16) as ci_high ".format(name) + - "FROM ztest FORMAT TabSeparatedWithNames;") - real_t_stat = real['t_stat'][0] - real_p_value = real['p_value'][0] - real_ci_low = real['ci_low'][0] - real_ci_high = real['ci_high'][0] - assert(abs(real_t_stat - np.float64(t_stat)) < precision), "clickhouse_t_stat {}, py_t_stat {}".format(real_t_stat, t_stat) - assert(abs(real_p_value - np.float64(p_value)) < precision), "clickhouse_p_value {}, py_p_value {}".format(real_p_value, p_value) - assert(abs(real_ci_low - np.float64(ci_low)) < precision), "clickhouse_ci_low {}, py_ci_low {}".format(real_ci_low, ci_low) - assert(abs(real_ci_high - np.float64(ci_high)) < precision), "clickhouse_ci_high {}, py_ci_high {}".format(real_ci_high, ci_high) + "SELECT roundBankers({}(left, right).1, 16) as t_stat, ".format(name) + + "roundBankers({}(left, right).2, 16) as p_value, ".format(name) + + "roundBankers({}(left, right).3, 16) as ci_low, ".format(name) + + "roundBankers({}(left, right).4, 16) as ci_high ".format(name) + + "FROM ztest FORMAT TabSeparatedWithNames;" + ) + real_t_stat = real["t_stat"][0] + real_p_value = real["p_value"][0] + real_ci_low = real["ci_low"][0] + real_ci_high = real["ci_high"][0] + assert ( + abs(real_t_stat - np.float64(t_stat)) < precision + ), "clickhouse_t_stat {}, py_t_stat {}".format(real_t_stat, t_stat) + assert ( + abs(real_p_value - np.float64(p_value)) < precision + ), "clickhouse_p_value {}, py_p_value {}".format(real_p_value, p_value) + assert ( + abs(real_ci_low - np.float64(ci_low)) < precision + ), "clickhouse_ci_low {}, py_ci_low {}".format(real_ci_low, ci_low) + assert ( + abs(real_ci_high - np.float64(ci_high)) < precision + ), "clickhouse_ci_high {}, py_ci_high {}".format(real_ci_high, ci_high) client.query("DROP TABLE IF EXISTS ztest;") def test_mean_ztest(): - rvs1 = np.round(stats.norm.rvs(loc=1, scale=5,size=500), 2) - rvs2 = np.round(stats.norm.rvs(loc=10, scale=5,size=500), 2) + rvs1 = np.round(stats.norm.rvs(loc=1, scale=5, size=500), 2) + rvs2 = np.round(stats.norm.rvs(loc=10, scale=5, size=500), 2) s, p, cl, ch = twosample_mean_ztest(rvs1, rvs2) - test_and_check("meanZTest(%f, %f, 0.95)" % (variance(rvs1), variance(rvs2)), rvs1, rvs2, s, p, cl, ch) + test_and_check( + "meanZTest(%f, %f, 0.95)" % (variance(rvs1), variance(rvs2)), + rvs1, + rvs2, + s, + p, + cl, + ch, + ) - rvs1 = np.round(stats.norm.rvs(loc=0, scale=5,size=500), 2) - rvs2 = np.round(stats.norm.rvs(loc=0, scale=5,size=500), 2) + rvs1 = np.round(stats.norm.rvs(loc=0, scale=5, size=500), 2) + rvs2 = np.round(stats.norm.rvs(loc=0, scale=5, size=500), 2) s, p, cl, ch = twosample_mean_ztest(rvs1, rvs2) - test_and_check("meanZTest(%f, %f, 0.95)" % (variance(rvs1), variance(rvs2)), rvs1, rvs2, s, p, cl, ch) + test_and_check( + "meanZTest(%f, %f, 0.95)" % (variance(rvs1), variance(rvs2)), + rvs1, + rvs2, + s, + p, + cl, + ch, + ) - rvs1 = np.round(stats.norm.rvs(loc=2, scale=10,size=512), 2) - rvs2 = np.round(stats.norm.rvs(loc=5, scale=20,size=1024), 2) + rvs1 = np.round(stats.norm.rvs(loc=2, scale=10, size=512), 2) + rvs2 = np.round(stats.norm.rvs(loc=5, scale=20, size=1024), 2) s, p, cl, ch = twosample_mean_ztest(rvs1, rvs2) - test_and_check("meanZTest(%f, %f, 0.95)" % (variance(rvs1), variance(rvs2)), rvs1, rvs2, s, p, cl, ch) + test_and_check( + "meanZTest(%f, %f, 0.95)" % (variance(rvs1), variance(rvs2)), + rvs1, + rvs2, + s, + p, + cl, + ch, + ) - rvs1 = np.round(stats.norm.rvs(loc=0, scale=10,size=1024), 2) - rvs2 = np.round(stats.norm.rvs(loc=0, scale=10,size=512), 2) + rvs1 = np.round(stats.norm.rvs(loc=0, scale=10, size=1024), 2) + rvs2 = np.round(stats.norm.rvs(loc=0, scale=10, size=512), 2) s, p, cl, ch = twosample_mean_ztest(rvs1, rvs2) - test_and_check("meanZTest(%f, %f, 0.95)" % (variance(rvs1), variance(rvs2)), rvs1, rvs2, s, p, cl, ch) + test_and_check( + "meanZTest(%f, %f, 0.95)" % (variance(rvs1), variance(rvs2)), + rvs1, + rvs2, + s, + p, + cl, + ch, + ) if __name__ == "__main__": diff --git a/tests/queries/0_stateless/02187_async_inserts_all_formats.python b/tests/queries/0_stateless/02187_async_inserts_all_formats.python index 65a323ef9db..fa555c78f8b 100644 --- a/tests/queries/0_stateless/02187_async_inserts_all_formats.python +++ b/tests/queries/0_stateless/02187_async_inserts_all_formats.python @@ -3,47 +3,71 @@ import os import sys CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) -CLICKHOUSE_URL = os.environ.get('CLICKHOUSE_URL') -CLICKHOUSE_TMP = os.environ.get('CLICKHOUSE_TMP') +CLICKHOUSE_URL = os.environ.get("CLICKHOUSE_URL") +CLICKHOUSE_TMP = os.environ.get("CLICKHOUSE_TMP") from pure_http_client import ClickHouseClient client = ClickHouseClient() + def run_test(data_format, gen_data_template, settings): print(data_format) client.query("TRUNCATE TABLE t_async_insert") expected = client.query(gen_data_template.format("TSV")).strip() - data = client.query(gen_data_template.format(data_format), settings=settings,binary_result=True) + data = client.query( + gen_data_template.format(data_format), settings=settings, binary_result=True + ) insert_query = "INSERT INTO t_async_insert FORMAT {}".format(data_format) client.query_with_data(insert_query, data, settings=settings) result = client.query("SELECT * FROM t_async_insert FORMAT TSV").strip() if result != expected: - print("Failed for format {}.\nExpected:\n{}\nGot:\n{}\n".format(data_format, expected, result)) + print( + "Failed for format {}.\nExpected:\n{}\nGot:\n{}\n".format( + data_format, expected, result + ) + ) exit(1) -formats = client.query("SELECT name FROM system.formats WHERE is_input AND is_output \ - AND name NOT IN ('CapnProto', 'RawBLOB', 'Template', 'ProtobufSingle', 'LineAsString', 'Protobuf', 'ProtobufList') ORDER BY name").strip().split('\n') + +formats = ( + client.query( + "SELECT name FROM system.formats WHERE is_input AND is_output \ + AND name NOT IN ('CapnProto', 'RawBLOB', 'Template', 'ProtobufSingle', 'LineAsString', 'Protobuf', 'ProtobufList') ORDER BY name" + ) + .strip() + .split("\n") +) # Generic formats client.query("DROP TABLE IF EXISTS t_async_insert") -client.query("CREATE TABLE t_async_insert (id UInt64, s String, arr Array(UInt64)) ENGINE = Memory") +client.query( + "CREATE TABLE t_async_insert (id UInt64, s String, arr Array(UInt64)) ENGINE = Memory" +) gen_data_query = "SELECT number AS id, toString(number) AS s, range(number) AS arr FROM numbers(10) FORMAT {}" for data_format in formats: - run_test(data_format, gen_data_query, settings={"async_insert": 1, "wait_for_async_insert": 1}) + run_test( + data_format, + gen_data_query, + settings={"async_insert": 1, "wait_for_async_insert": 1}, + ) # LineAsString client.query("DROP TABLE IF EXISTS t_async_insert") client.query("CREATE TABLE t_async_insert (s String) ENGINE = Memory") gen_data_query = "SELECT toString(number) AS s FROM numbers(10) FORMAT {}" -run_test('LineAsString', gen_data_query, settings={"async_insert": 1, "wait_for_async_insert": 1}) +run_test( + "LineAsString", + gen_data_query, + settings={"async_insert": 1, "wait_for_async_insert": 1}, +) # TODO: add CapnProto and Protobuf diff --git a/tests/queries/0_stateless/02205_HTTP_user_agent.python b/tests/queries/0_stateless/02205_HTTP_user_agent.python index 0d3a563c094..5787ae186ab 100644 --- a/tests/queries/0_stateless/02205_HTTP_user_agent.python +++ b/tests/queries/0_stateless/02205_HTTP_user_agent.python @@ -1,6 +1,6 @@ #!/usr/bin/env python3 -from http.server import SimpleHTTPRequestHandler,HTTPServer +from http.server import SimpleHTTPRequestHandler, HTTPServer import socket import sys import threading @@ -17,6 +17,7 @@ def is_ipv6(host): except: return True + def get_local_port(host, ipv6): if ipv6: family = socket.AF_INET6 @@ -27,20 +28,19 @@ def get_local_port(host, ipv6): fd.bind((host, 0)) return fd.getsockname()[1] -CLICKHOUSE_HOST = os.environ.get('CLICKHOUSE_HOST', 'localhost') -CLICKHOUSE_PORT_HTTP = os.environ.get('CLICKHOUSE_PORT_HTTP', '8123') + +CLICKHOUSE_HOST = os.environ.get("CLICKHOUSE_HOST", "localhost") +CLICKHOUSE_PORT_HTTP = os.environ.get("CLICKHOUSE_PORT_HTTP", "8123") # Server returns this JSON response. -SERVER_JSON_RESPONSE = \ -'''{ +SERVER_JSON_RESPONSE = """{ "login": "ClickHouse", "id": 54801242, "name": "ClickHouse", "company": null -}''' +}""" -EXPECTED_ANSWER = \ -'''{\\n\\t"login": "ClickHouse",\\n\\t"id": 54801242,\\n\\t"name": "ClickHouse",\\n\\t"company": null\\n}''' +EXPECTED_ANSWER = """{\\n\\t"login": "ClickHouse",\\n\\t"id": 54801242,\\n\\t"name": "ClickHouse",\\n\\t"company": null\\n}""" ##################################################################################### # This test starts an HTTP server and serves data to clickhouse url-engine based table. @@ -51,26 +51,38 @@ EXPECTED_ANSWER = \ ##################################################################################### # IP-address of this host accessible from the outside world. Get the first one -HTTP_SERVER_HOST = subprocess.check_output(['hostname', '-i']).decode('utf-8').strip().split()[0] +HTTP_SERVER_HOST = ( + subprocess.check_output(["hostname", "-i"]).decode("utf-8").strip().split()[0] +) IS_IPV6 = is_ipv6(HTTP_SERVER_HOST) HTTP_SERVER_PORT = get_local_port(HTTP_SERVER_HOST, IS_IPV6) # IP address and port of the HTTP server started from this script. HTTP_SERVER_ADDRESS = (HTTP_SERVER_HOST, HTTP_SERVER_PORT) if IS_IPV6: - HTTP_SERVER_URL_STR = 'http://' + f'[{str(HTTP_SERVER_ADDRESS[0])}]:{str(HTTP_SERVER_ADDRESS[1])}' + "/" + HTTP_SERVER_URL_STR = ( + "http://" + + f"[{str(HTTP_SERVER_ADDRESS[0])}]:{str(HTTP_SERVER_ADDRESS[1])}" + + "/" + ) else: - HTTP_SERVER_URL_STR = 'http://' + f'{str(HTTP_SERVER_ADDRESS[0])}:{str(HTTP_SERVER_ADDRESS[1])}' + "/" + HTTP_SERVER_URL_STR = ( + "http://" + f"{str(HTTP_SERVER_ADDRESS[0])}:{str(HTTP_SERVER_ADDRESS[1])}" + "/" + ) def get_ch_answer(query): host = CLICKHOUSE_HOST if IS_IPV6: - host = f'[{host}]' + host = f"[{host}]" - url = os.environ.get('CLICKHOUSE_URL', 'http://{host}:{port}'.format(host=CLICKHOUSE_HOST, port=CLICKHOUSE_PORT_HTTP)) + url = os.environ.get( + "CLICKHOUSE_URL", + "http://{host}:{port}".format(host=CLICKHOUSE_HOST, port=CLICKHOUSE_PORT_HTTP), + ) return urllib.request.urlopen(url, data=query.encode()).read().decode() + def check_answers(query, answer): ch_answer = get_ch_answer(query) if ch_answer.strip() != answer.strip(): @@ -79,16 +91,17 @@ def check_answers(query, answer): print("Fetched answer :", ch_answer, file=sys.stderr) raise Exception("Fail on query") + # Server with check for User-Agent headers. class HttpProcessor(SimpleHTTPRequestHandler): def _set_headers(self): - user_agent = self.headers.get('User-Agent') - if user_agent and user_agent.startswith('ClickHouse/'): + user_agent = self.headers.get("User-Agent") + if user_agent and user_agent.startswith("ClickHouse/"): self.send_response(200) else: self.send_response(403) - self.send_header('Content-Type', 'text/csv') + self.send_header("Content-Type", "text/csv") self.end_headers() def do_GET(self): @@ -98,9 +111,11 @@ class HttpProcessor(SimpleHTTPRequestHandler): def log_message(self, format, *args): return + class HTTPServerV6(HTTPServer): address_family = socket.AF_INET6 + def start_server(requests_amount): if IS_IPV6: httpd = HTTPServerV6(HTTP_SERVER_ADDRESS, HttpProcessor) @@ -114,15 +129,18 @@ def start_server(requests_amount): t = threading.Thread(target=real_func) return t + ##################################################################### # Testing area. ##################################################################### + def test_select(): global HTTP_SERVER_URL_STR - query = 'SELECT * FROM url(\'{}\',\'JSONAsString\');'.format(HTTP_SERVER_URL_STR) + query = "SELECT * FROM url('{}','JSONAsString');".format(HTTP_SERVER_URL_STR) check_answers(query, EXPECTED_ANSWER) + def main(): # HEAD + GET t = start_server(3) @@ -131,6 +149,7 @@ def main(): t.join() print("PASSED") + if __name__ == "__main__": try: main() @@ -141,4 +160,3 @@ if __name__ == "__main__": sys.stderr.flush() os._exit(1) - diff --git a/tests/queries/0_stateless/02233_HTTP_ranged.python b/tests/queries/0_stateless/02233_HTTP_ranged.python index e74d494edf5..66ef3304098 100644 --- a/tests/queries/0_stateless/02233_HTTP_ranged.python +++ b/tests/queries/0_stateless/02233_HTTP_ranged.python @@ -122,7 +122,7 @@ class HttpProcessor(BaseHTTPRequestHandler): get_call_num = 0 responses_to_get = [] - def send_head(self, from_get = False): + def send_head(self, from_get=False): if self.headers["Range"] and HttpProcessor.allow_range: try: self.range = parse_byte_range(self.headers["Range"]) @@ -146,7 +146,9 @@ class HttpProcessor(BaseHTTPRequestHandler): self.send_error(416, "Requested Range Not Satisfiable") return None - retry_range_request = first != 0 and from_get is True and len(HttpProcessor.responses_to_get) > 0 + retry_range_request = ( + first != 0 and from_get is True and len(HttpProcessor.responses_to_get) > 0 + ) if retry_range_request: code = HttpProcessor.responses_to_get.pop() if code not in HttpProcessor.responses: @@ -244,7 +246,9 @@ def run_test(allow_range, settings, check_retries=False): raise Exception("HTTP Range was not used when supported") if check_retries and len(HttpProcessor.responses_to_get) > 0: - raise Exception("Expected to get http response 500, which had to be retried, but 200 ok returned and then retried") + raise Exception( + "Expected to get http response 500, which had to be retried, but 200 ok returned and then retried" + ) if retries_num > 0: expected_get_call_num += retries_num - 1 @@ -263,7 +267,7 @@ def run_test(allow_range, settings, check_retries=False): def main(): - settings = {"max_download_buffer_size" : 20} + settings = {"max_download_buffer_size": 20} # Test Accept-Ranges=False run_test(allow_range=False, settings=settings) @@ -271,7 +275,7 @@ def main(): run_test(allow_range=True, settings=settings) # Test Accept-Ranges=True, parallel download is used - settings = {"max_download_buffer_size" : 10} + settings = {"max_download_buffer_size": 10} run_test(allow_range=True, settings=settings) # Test Accept-Ranges=True, parallel download is not used, diff --git a/tests/queries/0_stateless/02294_anova_cmp.python b/tests/queries/0_stateless/02294_anova_cmp.python index 7597b3712d1..2212a887b2f 100644 --- a/tests/queries/0_stateless/02294_anova_cmp.python +++ b/tests/queries/0_stateless/02294_anova_cmp.python @@ -7,7 +7,7 @@ import pandas as pd import numpy as np CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) from pure_http_client import ClickHouseClient @@ -22,15 +22,22 @@ def test_and_check(rvs, n_groups, f_stat, p_value, precision=1e-2): client.query("DROP TABLE IF EXISTS anova;") client.query("CREATE TABLE anova (left Float64, right UInt64) ENGINE = Memory;") for group in range(n_groups): - client.query(f'''INSERT INTO anova VALUES {", ".join([f'({i},{group})' for i in rvs[group]])};''') + client.query( + f"""INSERT INTO anova VALUES {", ".join([f'({i},{group})' for i in rvs[group]])};""" + ) real = client.query_return_df( - '''SELECT roundBankers(a.1, 16) as f_stat, roundBankers(a.2, 16) as p_value FROM (SELECT anova(left, right) as a FROM anova) FORMAT TabSeparatedWithNames;''') + """SELECT roundBankers(a.1, 16) as f_stat, roundBankers(a.2, 16) as p_value FROM (SELECT anova(left, right) as a FROM anova) FORMAT TabSeparatedWithNames;""" + ) - real_f_stat = real['f_stat'][0] - real_p_value = real['p_value'][0] - assert(abs(real_f_stat - np.float64(f_stat)) < precision), f"clickhouse_f_stat {real_f_stat}, py_f_stat {f_stat}" - assert(abs(real_p_value - np.float64(p_value)) < precision), f"clickhouse_p_value {real_p_value}, py_p_value {p_value}" + real_f_stat = real["f_stat"][0] + real_p_value = real["p_value"][0] + assert ( + abs(real_f_stat - np.float64(f_stat)) < precision + ), f"clickhouse_f_stat {real_f_stat}, py_f_stat {f_stat}" + assert ( + abs(real_p_value - np.float64(p_value)) < precision + ), f"clickhouse_p_value {real_p_value}, py_p_value {p_value}" client.query("DROP TABLE IF EXISTS anova;") diff --git a/tests/queries/0_stateless/02346_read_in_order_fixed_prefix.python b/tests/queries/0_stateless/02346_read_in_order_fixed_prefix.python index 399533480a9..7f52daeb408 100644 --- a/tests/queries/0_stateless/02346_read_in_order_fixed_prefix.python +++ b/tests/queries/0_stateless/02346_read_in_order_fixed_prefix.python @@ -123,10 +123,14 @@ Uses FinishSortingTransform: {} for query in queries: check_query(query["where"], query["order_by"], query["optimize"], False) - check_query(query["where"], query["order_by"] + ["e"], query["optimize"], query["optimize"]) + check_query( + query["where"], query["order_by"] + ["e"], query["optimize"], query["optimize"] + ) where_columns = [f"bitNot({col})" for col in query["where"]] check_query(where_columns, query["order_by"], query["optimize"], False) - check_query(where_columns, query["order_by"] + ["e"], query["optimize"], query["optimize"]) + check_query( + where_columns, query["order_by"] + ["e"], query["optimize"], query["optimize"] + ) print("OK") diff --git a/tests/queries/0_stateless/02403_big_http_chunk_size.python b/tests/queries/0_stateless/02403_big_http_chunk_size.python index 4e2e97e487b..4d2f01db55b 100644 --- a/tests/queries/0_stateless/02403_big_http_chunk_size.python +++ b/tests/queries/0_stateless/02403_big_http_chunk_size.python @@ -8,8 +8,8 @@ TRANSFER_ENCODING_HEADER = "Transfer-Encoding" def main(): - host = os.environ['CLICKHOUSE_HOST'] - port = int(os.environ['CLICKHOUSE_PORT_HTTP']) + host = os.environ["CLICKHOUSE_HOST"] + port = int(os.environ["CLICKHOUSE_PORT_HTTP"]) sock = socket(AF_INET, SOCK_STREAM) sock.connect((host, port)) @@ -47,4 +47,3 @@ def main(): if __name__ == "__main__": main() - diff --git a/tests/queries/0_stateless/02458_insert_select_progress_tcp.python b/tests/queries/0_stateless/02458_insert_select_progress_tcp.python index c638b3d2040..9d64201afd9 100644 --- a/tests/queries/0_stateless/02458_insert_select_progress_tcp.python +++ b/tests/queries/0_stateless/02458_insert_select_progress_tcp.python @@ -5,9 +5,10 @@ import os import uuid import json -CLICKHOUSE_HOST = os.environ.get('CLICKHOUSE_HOST', '127.0.0.1') -CLICKHOUSE_PORT = int(os.environ.get('CLICKHOUSE_PORT_TCP', '900000')) -CLICKHOUSE_DATABASE = os.environ.get('CLICKHOUSE_DATABASE', 'default') +CLICKHOUSE_HOST = os.environ.get("CLICKHOUSE_HOST", "127.0.0.1") +CLICKHOUSE_PORT = int(os.environ.get("CLICKHOUSE_PORT_TCP", "900000")) +CLICKHOUSE_DATABASE = os.environ.get("CLICKHOUSE_DATABASE", "default") + def writeVarUInt(x, ba): for _ in range(0, 9): @@ -24,12 +25,12 @@ def writeVarUInt(x, ba): def writeStringBinary(s, ba): - b = bytes(s, 'utf-8') + b = bytes(s, "utf-8") writeVarUInt(len(s), ba) ba.extend(b) -def readStrict(s, size = 1): +def readStrict(s, size=1): res = bytearray() while size: cur = s.recv(size) @@ -48,18 +49,23 @@ def readUInt(s, size=1): val += res[i] << (i * 8) return val + def readUInt8(s): return readUInt(s) + def readUInt16(s): return readUInt(s, 2) + def readUInt32(s): return readUInt(s, 4) + def readUInt64(s): return readUInt(s, 8) + def readVarUInt(s): x = 0 for i in range(9): @@ -75,25 +81,25 @@ def readVarUInt(s): def readStringBinary(s): size = readVarUInt(s) s = readStrict(s, size) - return s.decode('utf-8') + return s.decode("utf-8") def sendHello(s): ba = bytearray() - writeVarUInt(0, ba) # Hello - writeStringBinary('simple native protocol', ba) + writeVarUInt(0, ba) # Hello + writeStringBinary("simple native protocol", ba) writeVarUInt(21, ba) writeVarUInt(9, ba) writeVarUInt(54449, ba) - writeStringBinary(CLICKHOUSE_DATABASE, ba) # database - writeStringBinary('default', ba) # user - writeStringBinary('', ba) # pwd + writeStringBinary(CLICKHOUSE_DATABASE, ba) # database + writeStringBinary("default", ba) # user + writeStringBinary("", ba) # pwd s.sendall(ba) def receiveHello(s): p_type = readVarUInt(s) - assert (p_type == 0) # Hello + assert p_type == 0 # Hello server_name = readStringBinary(s) # print("Server name: ", server_name) server_version_major = readVarUInt(s) @@ -111,65 +117,65 @@ def receiveHello(s): def serializeClientInfo(ba, query_id): - writeStringBinary('default', ba) # initial_user - writeStringBinary(query_id, ba) # initial_query_id - writeStringBinary('127.0.0.1:9000', ba) # initial_address - ba.extend([0] * 8) # initial_query_start_time_microseconds - ba.append(1) # TCP - writeStringBinary('os_user', ba) # os_user - writeStringBinary('client_hostname', ba) # client_hostname - writeStringBinary('client_name', ba) # client_name + writeStringBinary("default", ba) # initial_user + writeStringBinary(query_id, ba) # initial_query_id + writeStringBinary("127.0.0.1:9000", ba) # initial_address + ba.extend([0] * 8) # initial_query_start_time_microseconds + ba.append(1) # TCP + writeStringBinary("os_user", ba) # os_user + writeStringBinary("client_hostname", ba) # client_hostname + writeStringBinary("client_name", ba) # client_name writeVarUInt(21, ba) writeVarUInt(9, ba) writeVarUInt(54449, ba) - writeStringBinary('', ba) # quota_key - writeVarUInt(0, ba) # distributed_depth - writeVarUInt(1, ba) # client_version_patch - ba.append(0) # No telemetry + writeStringBinary("", ba) # quota_key + writeVarUInt(0, ba) # distributed_depth + writeVarUInt(1, ba) # client_version_patch + ba.append(0) # No telemetry def sendQuery(s, query): ba = bytearray() query_id = uuid.uuid4().hex - writeVarUInt(1, ba) # query + writeVarUInt(1, ba) # query writeStringBinary(query_id, ba) - ba.append(1) # INITIAL_QUERY + ba.append(1) # INITIAL_QUERY # client info serializeClientInfo(ba, query_id) - writeStringBinary('', ba) # No settings - writeStringBinary('', ba) # No interserver secret - writeVarUInt(2, ba) # Stage - Complete - ba.append(0) # No compression - writeStringBinary(query, ba) # query, finally + writeStringBinary("", ba) # No settings + writeStringBinary("", ba) # No interserver secret + writeVarUInt(2, ba) # Stage - Complete + ba.append(0) # No compression + writeStringBinary(query, ba) # query, finally s.sendall(ba) def serializeBlockInfo(ba): - writeVarUInt(1, ba) # 1 - ba.append(0) # is_overflows - writeVarUInt(2, ba) # 2 - writeVarUInt(0, ba) # 0 - ba.extend([0] * 4) # bucket_num + writeVarUInt(1, ba) # 1 + ba.append(0) # is_overflows + writeVarUInt(2, ba) # 2 + writeVarUInt(0, ba) # 0 + ba.extend([0] * 4) # bucket_num def sendEmptyBlock(s): ba = bytearray() - writeVarUInt(2, ba) # Data - writeStringBinary('', ba) + writeVarUInt(2, ba) # Data + writeStringBinary("", ba) serializeBlockInfo(ba) - writeVarUInt(0, ba) # rows - writeVarUInt(0, ba) # columns + writeVarUInt(0, ba) # rows + writeVarUInt(0, ba) # columns s.sendall(ba) def assertPacket(packet, expected): - assert(packet == expected), packet + assert packet == expected, packet -class Progress(): +class Progress: def __init__(self): # NOTE: this is done in ctor to initialize __dict__ self.read_rows = 0 @@ -198,11 +204,12 @@ class Progress(): def __bool__(self): return ( - self.read_rows > 0 or - self.read_bytes > 0 or - self.total_rows_to_read > 0 or - self.written_rows > 0 or - self.written_bytes > 0) + self.read_rows > 0 + or self.read_bytes > 0 + or self.total_rows_to_read > 0 + or self.written_rows > 0 + or self.written_bytes > 0 + ) def readProgress(s): @@ -219,13 +226,14 @@ def readProgress(s): progress.readPacket(s) return progress + def readException(s): code = readUInt32(s) name = readStringBinary(s) text = readStringBinary(s) - readStringBinary(s) # trace - assertPacket(readUInt8(s), 0) # has_nested - return "code {}: {}".format(code, text.replace('DB::Exception:', '')) + readStringBinary(s) # trace + assertPacket(readUInt8(s), 0) # has_nested + return "code {}: {}".format(code, text.replace("DB::Exception:", "")) def main(): @@ -236,7 +244,10 @@ def main(): receiveHello(s) # For 1 second sleep and 1000ms of interactive_delay we definitelly should have non zero progress packet. # NOTE: interactive_delay=0 cannot be used since in this case CompletedPipelineExecutor will not call cancelled callback. - sendQuery(s, "insert into function null('_ Int') select sleep(1) from numbers(2) settings max_block_size=1, interactive_delay=1000") + sendQuery( + s, + "insert into function null('_ Int') select sleep(1) from numbers(2) settings max_block_size=1, interactive_delay=1000", + ) # external tables sendEmptyBlock(s) diff --git a/tests/queries/0_stateless/02473_multistep_prewhere.python b/tests/queries/0_stateless/02473_multistep_prewhere.python index 37a7280dac2..a942568233c 100644 --- a/tests/queries/0_stateless/02473_multistep_prewhere.python +++ b/tests/queries/0_stateless/02473_multistep_prewhere.python @@ -4,18 +4,19 @@ import os import sys CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) from pure_http_client import ClickHouseClient class Tester: - ''' + """ - Creates test table - Deletes the specified range of rows - Masks another range using row-level policy - Runs some read queries and checks that the results - ''' + """ + def __init__(self, session, url, index_granularity, total_rows): self.session = session self.url = url @@ -25,10 +26,10 @@ class Tester: self.repro_queries = [] def report_error(self): - print('Repro steps:', '\n\n\t'.join(self.repro_queries)) + print("Repro steps:", "\n\n\t".join(self.repro_queries)) exit(1) - def query(self, query_text, include_in_repro_steps = True, expected_data = None): + def query(self, query_text, include_in_repro_steps=True, expected_data=None): self.repro_queries.append(query_text) resp = self.session.post(self.url, data=query_text) if resp.status_code != 200: @@ -36,113 +37,187 @@ class Tester: error = resp.text[0:40] if error not in self.reported_errors: self.reported_errors.add(error) - print('Code:', resp.status_code) - print('Result:', resp.text) + print("Code:", resp.status_code) + print("Result:", resp.text) self.report_error() result = resp.text # Check that the result is as expected - if ((not expected_data is None) and (int(result) != len(expected_data))): - print('Expected {} rows, got {}'.format(len(expected_data), result)) - print('Expected data:' + str(expected_data)) + if (not expected_data is None) and (int(result) != len(expected_data)): + print("Expected {} rows, got {}".format(len(expected_data), result)) + print("Expected data:" + str(expected_data)) self.report_error() if not include_in_repro_steps: self.repro_queries.pop() - - def check_data(self, all_data, delete_range_start, delete_range_end, row_level_policy_range_start, row_level_policy_range_end): + def check_data( + self, + all_data, + delete_range_start, + delete_range_end, + row_level_policy_range_start, + row_level_policy_range_end, + ): all_data_after_delete = all_data[ - ~((all_data.a == 0) & - (all_data.b > delete_range_start) & - (all_data.b <= delete_range_end))] + ~( + (all_data.a == 0) + & (all_data.b > delete_range_start) + & (all_data.b <= delete_range_end) + ) + ] all_data_after_row_policy = all_data_after_delete[ - (all_data_after_delete.b <= row_level_policy_range_start) | - (all_data_after_delete.b > row_level_policy_range_end)] + (all_data_after_delete.b <= row_level_policy_range_start) + | (all_data_after_delete.b > row_level_policy_range_end) + ] - for to_select in ['count()', 'sum(d)']: # Test reading with and without column with default value - self.query('SELECT {} FROM tab_02473;'.format(to_select), False, all_data_after_row_policy) + for to_select in [ + "count()", + "sum(d)", + ]: # Test reading with and without column with default value + self.query( + "SELECT {} FROM tab_02473;".format(to_select), + False, + all_data_after_row_policy, + ) delta = 10 for query_range_start in [0, delta]: - for query_range_end in [self.total_rows - delta]: #, self.total_rows]: + for query_range_end in [self.total_rows - delta]: # , self.total_rows]: expected = all_data_after_row_policy[ - (all_data_after_row_policy.a == 0) & - (all_data_after_row_policy.b > query_range_start) & - (all_data_after_row_policy.b <= query_range_end)] - self.query('SELECT {} from tab_02473 PREWHERE b > {} AND b <= {} WHERE a == 0;'.format( - to_select, query_range_start, query_range_end), False, expected) + (all_data_after_row_policy.a == 0) + & (all_data_after_row_policy.b > query_range_start) + & (all_data_after_row_policy.b <= query_range_end) + ] + self.query( + "SELECT {} from tab_02473 PREWHERE b > {} AND b <= {} WHERE a == 0;".format( + to_select, query_range_start, query_range_end + ), + False, + expected, + ) expected = all_data_after_row_policy[ - (all_data_after_row_policy.a == 0) & - (all_data_after_row_policy.c > query_range_start) & - (all_data_after_row_policy.c <= query_range_end)] - self.query('SELECT {} from tab_02473 PREWHERE c > {} AND c <= {} WHERE a == 0;'.format( - to_select, query_range_start, query_range_end), False, expected) + (all_data_after_row_policy.a == 0) + & (all_data_after_row_policy.c > query_range_start) + & (all_data_after_row_policy.c <= query_range_end) + ] + self.query( + "SELECT {} from tab_02473 PREWHERE c > {} AND c <= {} WHERE a == 0;".format( + to_select, query_range_start, query_range_end + ), + False, + expected, + ) expected = all_data_after_row_policy[ - (all_data_after_row_policy.a == 0) & - ((all_data_after_row_policy.c <= query_range_start) | - (all_data_after_row_policy.c > query_range_end))] - self.query('SELECT {} from tab_02473 PREWHERE c <= {} OR c > {} WHERE a == 0;'.format( - to_select, query_range_start, query_range_end), False, expected) + (all_data_after_row_policy.a == 0) + & ( + (all_data_after_row_policy.c <= query_range_start) + | (all_data_after_row_policy.c > query_range_end) + ) + ] + self.query( + "SELECT {} from tab_02473 PREWHERE c <= {} OR c > {} WHERE a == 0;".format( + to_select, query_range_start, query_range_end + ), + False, + expected, + ) - - def run_test(self, delete_range_start, delete_range_end, row_level_policy_range_start, row_level_policy_range_end): + def run_test( + self, + delete_range_start, + delete_range_end, + row_level_policy_range_start, + row_level_policy_range_end, + ): self.repro_queries = [] - self.query(''' + self.query( + """ CREATE TABLE tab_02473 (a Int8, b Int32, c Int32, PRIMARY KEY (a)) ENGINE = MergeTree() ORDER BY (a, b) - SETTINGS min_bytes_for_wide_part = 0, index_granularity = {};'''.format(self.index_granularity)) + SETTINGS min_bytes_for_wide_part = 0, index_granularity = {};""".format( + self.index_granularity + ) + ) - self.query('INSERT INTO tab_02473 select 0, number+1, number+1 FROM numbers({});'.format(self.total_rows)) + self.query( + "INSERT INTO tab_02473 select 0, number+1, number+1 FROM numbers({});".format( + self.total_rows + ) + ) client = ClickHouseClient() - all_data = client.query_return_df("SELECT a, b, c, 1 as d FROM tab_02473 FORMAT TabSeparatedWithNames;") + all_data = client.query_return_df( + "SELECT a, b, c, 1 as d FROM tab_02473 FORMAT TabSeparatedWithNames;" + ) - self.query('OPTIMIZE TABLE tab_02473 FINAL SETTINGS mutations_sync=2;') + self.query("OPTIMIZE TABLE tab_02473 FINAL SETTINGS mutations_sync=2;") # After all data has been written add a column with default value - self.query('ALTER TABLE tab_02473 ADD COLUMN d Int64 DEFAULT 1;') + self.query("ALTER TABLE tab_02473 ADD COLUMN d Int64 DEFAULT 1;") self.check_data(all_data, -100, -100, -100, -100) - self.query('DELETE FROM tab_02473 WHERE a = 0 AND b > {} AND b <= {};'.format( - delete_range_start, delete_range_end)) + self.query( + "DELETE FROM tab_02473 WHERE a = 0 AND b > {} AND b <= {};".format( + delete_range_start, delete_range_end + ) + ) self.check_data(all_data, delete_range_start, delete_range_end, -100, -100) - self.query('CREATE ROW POLICY policy_tab_02473 ON tab_02473 FOR SELECT USING b <= {} OR b > {} TO default;'.format( - row_level_policy_range_start, row_level_policy_range_end)) + self.query( + "CREATE ROW POLICY policy_tab_02473 ON tab_02473 FOR SELECT USING b <= {} OR b > {} TO default;".format( + row_level_policy_range_start, row_level_policy_range_end + ) + ) - self.check_data(all_data, delete_range_start, delete_range_end, row_level_policy_range_start, row_level_policy_range_end) + self.check_data( + all_data, + delete_range_start, + delete_range_end, + row_level_policy_range_start, + row_level_policy_range_end, + ) - self.query('DROP POLICY policy_tab_02473 ON tab_02473;') - - self.query('DROP TABLE tab_02473;') + self.query("DROP POLICY policy_tab_02473 ON tab_02473;") + self.query("DROP TABLE tab_02473;") def main(): # Set mutations to synchronous mode and enable lightweight DELETE's - url = os.environ['CLICKHOUSE_URL'] + '&max_threads=1' + url = os.environ["CLICKHOUSE_URL"] + "&max_threads=1" - default_index_granularity = 10; + default_index_granularity = 10 total_rows = 8 * default_index_granularity step = default_index_granularity session = requests.Session() - for index_granularity in [default_index_granularity-1, default_index_granularity]: # [default_index_granularity-1, default_index_granularity+1, default_index_granularity]: + for index_granularity in [ + default_index_granularity - 1, + default_index_granularity, + ]: # [default_index_granularity-1, default_index_granularity+1, default_index_granularity]: tester = Tester(session, url, index_granularity, total_rows) # Test combinations of ranges of various size masked by lightweight DELETES # along with ranges of various size masked by row-level policies for delete_range_start in range(0, total_rows, 3 * step): - for delete_range_end in range(delete_range_start + 3 * step, total_rows, 2 * step): + for delete_range_end in range( + delete_range_start + 3 * step, total_rows, 2 * step + ): for row_level_policy_range_start in range(0, total_rows, 3 * step): - for row_level_policy_range_end in range(row_level_policy_range_start + 3 * step, total_rows, 2 * step): - tester.run_test(delete_range_start, delete_range_end, row_level_policy_range_start, row_level_policy_range_end) + for row_level_policy_range_end in range( + row_level_policy_range_start + 3 * step, total_rows, 2 * step + ): + tester.run_test( + delete_range_start, + delete_range_end, + row_level_policy_range_start, + row_level_policy_range_end, + ) if __name__ == "__main__": main() - diff --git a/tests/queries/0_stateless/02473_multistep_split_prewhere.python b/tests/queries/0_stateless/02473_multistep_split_prewhere.python index 41d8a746e11..19444994fd2 100644 --- a/tests/queries/0_stateless/02473_multistep_split_prewhere.python +++ b/tests/queries/0_stateless/02473_multistep_split_prewhere.python @@ -4,16 +4,17 @@ import os import sys CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) from pure_http_client import ClickHouseClient class Tester: - ''' + """ - Creates test table with multiple integer columns - Runs read queries with multiple range conditions on different columns in PREWHERE and check that the result is correct - ''' + """ + def __init__(self, session, url, index_granularity, total_rows): self.session = session self.url = url @@ -23,10 +24,10 @@ class Tester: self.repro_queries = [] def report_error(self): - print('Repro steps:', '\n\n\t'.join(self.repro_queries)) + print("Repro steps:", "\n\n\t".join(self.repro_queries)) exit(1) - def query(self, query_text, include_in_repro_steps = True, expected_data = None): + def query(self, query_text, include_in_repro_steps=True, expected_data=None): self.repro_queries.append(query_text) resp = self.session.post(self.url, data=query_text) if resp.status_code != 200: @@ -34,98 +35,150 @@ class Tester: error = resp.text[0:40] if error not in self.reported_errors: self.reported_errors.add(error) - print('Code:', resp.status_code) - print('Result:', resp.text) + print("Code:", resp.status_code) + print("Result:", resp.text) self.report_error() result = resp.text # Check that the result is as expected - if ((not expected_data is None) and (int(result) != len(expected_data))): - print('Expected {} rows, got {}'.format(len(expected_data), result)) - print('Expected data:' + str(expected_data)) + if (not expected_data is None) and (int(result) != len(expected_data)): + print("Expected {} rows, got {}".format(len(expected_data), result)) + print("Expected data:" + str(expected_data)) self.report_error() if not include_in_repro_steps: self.repro_queries.pop() - - def check_data(self, all_data, c_range_start, c_range_end, d_range_start, d_range_end): - for to_select in ['count()', 'sum(e)']: # Test reading with and without column with default value - self.query('SELECT {} FROM tab_02473;'.format(to_select), False, all_data) + def check_data( + self, all_data, c_range_start, c_range_end, d_range_start, d_range_end + ): + for to_select in [ + "count()", + "sum(e)", + ]: # Test reading with and without column with default value + self.query("SELECT {} FROM tab_02473;".format(to_select), False, all_data) delta = 10 for b_range_start in [0, delta]: - for b_range_end in [self.total_rows - delta]: #, self.total_rows]: + for b_range_end in [self.total_rows - delta]: # , self.total_rows]: expected = all_data[ - (all_data.a == 0) & - (all_data.b > b_range_start) & - (all_data.b <= b_range_end)] - self.query('SELECT {} from tab_02473 PREWHERE b > {} AND b <= {} WHERE a == 0;'.format( - to_select, b_range_start, b_range_end), False, expected) + (all_data.a == 0) + & (all_data.b > b_range_start) + & (all_data.b <= b_range_end) + ] + self.query( + "SELECT {} from tab_02473 PREWHERE b > {} AND b <= {} WHERE a == 0;".format( + to_select, b_range_start, b_range_end + ), + False, + expected, + ) expected = all_data[ - (all_data.a == 0) & - (all_data.b > b_range_start) & - (all_data.b <= b_range_end) & - (all_data.c > c_range_start) & - (all_data.c <= c_range_end)] - self.query('SELECT {} from tab_02473 PREWHERE b > {} AND b <= {} AND c > {} AND c <= {} WHERE a == 0;'.format( - to_select, b_range_start, b_range_end, c_range_start, c_range_end), False, expected) + (all_data.a == 0) + & (all_data.b > b_range_start) + & (all_data.b <= b_range_end) + & (all_data.c > c_range_start) + & (all_data.c <= c_range_end) + ] + self.query( + "SELECT {} from tab_02473 PREWHERE b > {} AND b <= {} AND c > {} AND c <= {} WHERE a == 0;".format( + to_select, + b_range_start, + b_range_end, + c_range_start, + c_range_end, + ), + False, + expected, + ) expected = all_data[ - (all_data.a == 0) & - (all_data.b > b_range_start) & - (all_data.b <= b_range_end) & - (all_data.c > c_range_start) & - (all_data.c <= c_range_end) & - (all_data.d > d_range_start) & - (all_data.d <= d_range_end)] - self.query('SELECT {} from tab_02473 PREWHERE b > {} AND b <= {} AND c > {} AND c <= {} AND d > {} AND d <= {} WHERE a == 0;'.format( - to_select, b_range_start, b_range_end, c_range_start, c_range_end, d_range_start, d_range_end), False, expected) - + (all_data.a == 0) + & (all_data.b > b_range_start) + & (all_data.b <= b_range_end) + & (all_data.c > c_range_start) + & (all_data.c <= c_range_end) + & (all_data.d > d_range_start) + & (all_data.d <= d_range_end) + ] + self.query( + "SELECT {} from tab_02473 PREWHERE b > {} AND b <= {} AND c > {} AND c <= {} AND d > {} AND d <= {} WHERE a == 0;".format( + to_select, + b_range_start, + b_range_end, + c_range_start, + c_range_end, + d_range_start, + d_range_end, + ), + False, + expected, + ) def run_test(self, c_range_start, c_range_end, d_range_start, d_range_end): self.repro_queries = [] - self.query(''' + self.query( + """ CREATE TABLE tab_02473 (a Int8, b Int32, c Int32, d Int32, PRIMARY KEY (a)) ENGINE = MergeTree() ORDER BY (a, b) - SETTINGS min_bytes_for_wide_part = 0, index_granularity = {};'''.format(self.index_granularity)) + SETTINGS min_bytes_for_wide_part = 0, index_granularity = {};""".format( + self.index_granularity + ) + ) - self.query('INSERT INTO tab_02473 select 0, number+1, number+1, number+1 FROM numbers({});'.format(self.total_rows)) + self.query( + "INSERT INTO tab_02473 select 0, number+1, number+1, number+1 FROM numbers({});".format( + self.total_rows + ) + ) client = ClickHouseClient() - all_data = client.query_return_df("SELECT a, b, c, d, 1 as e FROM tab_02473 FORMAT TabSeparatedWithNames;") + all_data = client.query_return_df( + "SELECT a, b, c, d, 1 as e FROM tab_02473 FORMAT TabSeparatedWithNames;" + ) - self.query('OPTIMIZE TABLE tab_02473 FINAL SETTINGS mutations_sync=2;') + self.query("OPTIMIZE TABLE tab_02473 FINAL SETTINGS mutations_sync=2;") # After all data has been written add a column with default value - self.query('ALTER TABLE tab_02473 ADD COLUMN e Int64 DEFAULT 1;') + self.query("ALTER TABLE tab_02473 ADD COLUMN e Int64 DEFAULT 1;") - self.check_data(all_data, c_range_start, c_range_end, d_range_start, d_range_end) - - self.query('DROP TABLE tab_02473;') + self.check_data( + all_data, c_range_start, c_range_end, d_range_start, d_range_end + ) + self.query("DROP TABLE tab_02473;") def main(): # Enable multiple prewhere read steps - url = os.environ['CLICKHOUSE_URL'] + '&enable_multiple_prewhere_read_steps=1&move_all_conditions_to_prewhere=0&max_threads=1' + url = ( + os.environ["CLICKHOUSE_URL"] + + "&enable_multiple_prewhere_read_steps=1&move_all_conditions_to_prewhere=0&max_threads=1" + ) - default_index_granularity = 10; + default_index_granularity = 10 total_rows = 8 * default_index_granularity step = default_index_granularity session = requests.Session() - for index_granularity in [default_index_granularity-1, default_index_granularity]: + for index_granularity in [default_index_granularity - 1, default_index_granularity]: tester = Tester(session, url, index_granularity, total_rows) # Test combinations of ranges of columns c and d for c_range_start in range(0, total_rows, int(2.3 * step)): - for c_range_end in range(c_range_start + 3 * step, total_rows, int(2.1 * step)): - for d_range_start in range(int(0.5 * step), total_rows, int(2.7 * step)): - for d_range_end in range(d_range_start + 3 * step, total_rows, int(2.2 * step)): - tester.run_test(c_range_start, c_range_end, d_range_start, d_range_end) + for c_range_end in range( + c_range_start + 3 * step, total_rows, int(2.1 * step) + ): + for d_range_start in range( + int(0.5 * step), total_rows, int(2.7 * step) + ): + for d_range_end in range( + d_range_start + 3 * step, total_rows, int(2.2 * step) + ): + tester.run_test( + c_range_start, c_range_end, d_range_start, d_range_end + ) if __name__ == "__main__": main() - diff --git a/tests/queries/0_stateless/02481_async_insert_dedup.python b/tests/queries/0_stateless/02481_async_insert_dedup.python index 0cea7301ce5..1bf0edcbd05 100644 --- a/tests/queries/0_stateless/02481_async_insert_dedup.python +++ b/tests/queries/0_stateless/02481_async_insert_dedup.python @@ -8,7 +8,7 @@ import time from threading import Thread CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) from pure_http_client import ClickHouseClient @@ -16,14 +16,23 @@ client = ClickHouseClient() # test table without partition client.query("DROP TABLE IF EXISTS t_async_insert_dedup_no_part NO DELAY") -client.query(''' +client.query( + """ CREATE TABLE t_async_insert_dedup_no_part ( KeyID UInt32 ) Engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/{database}/t_async_insert_dedup', '{replica}') ORDER BY (KeyID) -''') +""" +) -client.query("insert into t_async_insert_dedup_no_part values (1), (2), (3), (4), (5)", settings = {"async_insert": 1, "wait_for_async_insert": 1, "insert_keeper_fault_injection_probability": 0}) +client.query( + "insert into t_async_insert_dedup_no_part values (1), (2), (3), (4), (5)", + settings={ + "async_insert": 1, + "wait_for_async_insert": 1, + "insert_keeper_fault_injection_probability": 0, + }, +) result = client.query("select count(*) from t_async_insert_dedup_no_part") print(result, flush=True) client.query("DROP TABLE IF EXISTS t_async_insert_dedup_no_part NO DELAY") @@ -32,13 +41,13 @@ client.query("DROP TABLE IF EXISTS t_async_insert_dedup_no_part NO DELAY") def generate_data(q, total_number): old_data = [] max_chunk_size = 30 - partitions = ['2022-11-11 10:10:10', '2022-12-12 10:10:10'] + partitions = ["2022-11-11 10:10:10", "2022-12-12 10:10:10"] last_number = 0 while True: - dup_simulate = random.randint(0,3) + dup_simulate = random.randint(0, 3) # insert old data randomly. 25% of them are dup. if dup_simulate == 0: - last_idx = len(old_data)-1 + last_idx = len(old_data) - 1 if last_idx < 0: continue idx = last_idx - random.randint(0, 50) @@ -53,7 +62,7 @@ def generate_data(q, total_number): end = start + chunk_size if end > total_number: end = total_number - for i in range(start, end+1): + for i in range(start, end + 1): partition = partitions[random.randint(0, 1)] insert_stmt += "('{}', {}),".format(partition, i) insert_stmt = insert_stmt[:-1] @@ -65,33 +74,46 @@ def generate_data(q, total_number): # wait all the tasks is done. q.join() + def fetch_and_insert_data(q, client): while True: insert = q.get() - client.query(insert, settings = {"async_insert": 1, "async_insert_deduplicate": 1, "wait_for_async_insert": 0, "async_insert_busy_timeout_ms": 1500, "insert_keeper_fault_injection_probability": 0}) + client.query( + insert, + settings={ + "async_insert": 1, + "async_insert_deduplicate": 1, + "wait_for_async_insert": 0, + "async_insert_busy_timeout_ms": 1500, + "insert_keeper_fault_injection_probability": 0, + }, + ) q.task_done() sleep_time = random.randint(50, 500) - time.sleep(sleep_time/1000.0) + time.sleep(sleep_time / 1000.0) + # main process client.query("DROP TABLE IF EXISTS t_async_insert_dedup NO DELAY") -client.query(''' +client.query( + """ CREATE TABLE t_async_insert_dedup ( EventDate DateTime, KeyID UInt32 ) Engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/{database}/t_async_insert_dedup', '{replica}') PARTITION BY toYYYYMM(EventDate) ORDER BY (KeyID, EventDate) SETTINGS use_async_block_ids_cache = 1 -''') +""" +) q = queue.Queue(100) total_number = 10000 -gen = Thread(target = generate_data, args = [q, total_number]) +gen = Thread(target=generate_data, args=[q, total_number]) gen.start() for i in range(3): - insert = Thread(target = fetch_and_insert_data, args = [q, client]) + insert = Thread(target=fetch_and_insert_data, args=[q, client]) insert.start() gen.join() @@ -109,7 +131,7 @@ while True: errMsg = f"the size of result is {len(result)}. we expect {total_number}." else: for i in range(total_number): - expect = str(i+1) + expect = str(i + 1) real = result[i] if expect != real: err = True @@ -117,7 +139,7 @@ while True: break # retry several times to get stable results. if err and retry >= 5: - print (errMsg, flush=True) + print(errMsg, flush=True) elif err: retry += 1 continue @@ -125,11 +147,15 @@ while True: print(len(result), flush=True) break -result = client.query("SELECT value FROM system.metrics where metric = 'AsyncInsertCacheSize'") +result = client.query( + "SELECT value FROM system.metrics where metric = 'AsyncInsertCacheSize'" +) result = int(result.split()[0]) if result <= 0: raise Exception(f"AsyncInsertCacheSize should > 0, but got {result}") -result = client.query("SELECT value FROM system.events where event = 'AsyncInsertCacheHits'") +result = client.query( + "SELECT value FROM system.events where event = 'AsyncInsertCacheHits'" +) result = int(result.split()[0]) if result <= 0: raise Exception(f"AsyncInsertCacheHits should > 0, but got {result}") diff --git a/utils/clickhouse-diagnostics/clickhouse-diagnostics b/utils/clickhouse-diagnostics/clickhouse-diagnostics index cf65e4efbfb..5cacbf1d4d4 100755 --- a/utils/clickhouse-diagnostics/clickhouse-diagnostics +++ b/utils/clickhouse-diagnostics/clickhouse-diagnostics @@ -19,9 +19,9 @@ import tenacity import xmltodict import yaml -SELECT_VERSION = r'SELECT version()' +SELECT_VERSION = r"SELECT version()" -SELECT_UPTIME = r''' +SELECT_UPTIME = r""" {% if version_ge('21.3') -%} SELECT formatReadableTimeDelta(uptime()) {% else -%} @@ -29,18 +29,18 @@ SELECT toString(floor(uptime() / 3600 / 24)) || ' days ' || toString(floor(uptime() % (24 * 3600) / 3600, 1)) || ' hours' {% endif -%} -''' +""" SELECT_SYSTEM_TABLES = "SELECT name FROM system.tables WHERE database = 'system'" -SELECT_DATABASE_ENGINES = r'''SELECT +SELECT_DATABASE_ENGINES = r"""SELECT engine, count() "count" FROM system.databases GROUP BY engine -''' +""" -SELECT_DATABASES = r'''SELECT +SELECT_DATABASES = r"""SELECT name, engine, tables, @@ -62,17 +62,17 @@ LEFT JOIN ) AS db_stats ON db.name = db_stats.database ORDER BY bytes_on_disk DESC LIMIT 10 -''' +""" -SELECT_TABLE_ENGINES = r'''SELECT +SELECT_TABLE_ENGINES = r"""SELECT engine, count() "count" FROM system.tables WHERE database != 'system' GROUP BY engine -''' +""" -SELECT_DICTIONARIES = r'''SELECT +SELECT_DICTIONARIES = r"""SELECT source, type, status, @@ -80,13 +80,13 @@ SELECT_DICTIONARIES = r'''SELECT FROM system.dictionaries GROUP BY source, type, status ORDER BY status DESC, source -''' +""" SELECT_ACCESS = "SHOW ACCESS" SELECT_QUOTA_USAGE = "SHOW QUOTA" -SELECT_REPLICAS = r'''SELECT +SELECT_REPLICAS = r"""SELECT database, table, is_leader, @@ -98,9 +98,9 @@ SELECT_REPLICAS = r'''SELECT FROM system.replicas ORDER BY absolute_delay DESC LIMIT 10 -''' +""" -SELECT_REPLICATION_QUEUE = r'''SELECT +SELECT_REPLICATION_QUEUE = r"""SELECT database, table, replica_name, @@ -121,9 +121,9 @@ SELECT_REPLICATION_QUEUE = r'''SELECT FROM system.replication_queue ORDER BY create_time ASC LIMIT 20 -''' +""" -SELECT_REPLICATED_FETCHES = r'''SELECT +SELECT_REPLICATED_FETCHES = r"""SELECT database, table, round(elapsed, 1) "elapsed", @@ -140,9 +140,9 @@ SELECT_REPLICATED_FETCHES = r'''SELECT to_detached, thread_id FROM system.replicated_fetches -''' +""" -SELECT_PARTS_PER_TABLE = r'''SELECT +SELECT_PARTS_PER_TABLE = r"""SELECT database, table, count() "partitions", @@ -162,9 +162,9 @@ FROM GROUP BY database, table ORDER BY max_parts_per_partition DESC LIMIT 10 -''' +""" -SELECT_MERGES = r'''SELECT +SELECT_MERGES = r"""SELECT database, table, round(elapsed, 1) "elapsed", @@ -187,9 +187,9 @@ SELECT_MERGES = r'''SELECT formatReadableSize(memory_usage) "memory_usage" {% endif -%} FROM system.merges -''' +""" -SELECT_MUTATIONS = r'''SELECT +SELECT_MUTATIONS = r"""SELECT database, table, mutation_id, @@ -206,9 +206,9 @@ SELECT_MUTATIONS = r'''SELECT FROM system.mutations WHERE NOT is_done ORDER BY create_time DESC -''' +""" -SELECT_RECENT_DATA_PARTS = r'''SELECT +SELECT_RECENT_DATA_PARTS = r"""SELECT database, table, engine, @@ -242,9 +242,9 @@ SELECT_RECENT_DATA_PARTS = r'''SELECT FROM system.parts WHERE modification_time > now() - INTERVAL 3 MINUTE ORDER BY modification_time DESC -''' +""" -SELECT_DETACHED_DATA_PARTS = r'''SELECT +SELECT_DETACHED_DATA_PARTS = r"""SELECT database, table, partition_id, @@ -255,9 +255,9 @@ SELECT_DETACHED_DATA_PARTS = r'''SELECT max_block_number, level FROM system.detached_parts -''' +""" -SELECT_PROCESSES = r'''SELECT +SELECT_PROCESSES = r"""SELECT elapsed, query_id, {% if normalize_queries -%} @@ -285,9 +285,9 @@ SELECT_PROCESSES = r'''SELECT {% endif -%} FROM system.processes ORDER BY elapsed DESC -''' +""" -SELECT_TOP_QUERIES_BY_DURATION = r'''SELECT +SELECT_TOP_QUERIES_BY_DURATION = r"""SELECT type, query_start_time, query_duration_ms, @@ -339,9 +339,9 @@ WHERE type != 'QueryStart' AND event_time >= now() - INTERVAL 1 DAY ORDER BY query_duration_ms DESC LIMIT 10 -''' +""" -SELECT_TOP_QUERIES_BY_MEMORY_USAGE = r'''SELECT +SELECT_TOP_QUERIES_BY_MEMORY_USAGE = r"""SELECT type, query_start_time, query_duration_ms, @@ -393,9 +393,9 @@ WHERE type != 'QueryStart' AND event_time >= now() - INTERVAL 1 DAY ORDER BY memory_usage DESC LIMIT 10 -''' +""" -SELECT_FAILED_QUERIES = r'''SELECT +SELECT_FAILED_QUERIES = r"""SELECT type, query_start_time, query_duration_ms, @@ -448,9 +448,9 @@ WHERE type != 'QueryStart' AND exception != '' ORDER BY query_start_time DESC LIMIT 10 -''' +""" -SELECT_STACK_TRACES = r'''SELECT +SELECT_STACK_TRACES = r"""SELECT '\n' || arrayStringConcat( arrayMap( x, @@ -459,9 +459,9 @@ SELECT_STACK_TRACES = r'''SELECT arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') AS trace FROM system.stack_trace -''' +""" -SELECT_CRASH_LOG = r'''SELECT +SELECT_CRASH_LOG = r"""SELECT event_time, signal, thread_id, @@ -470,7 +470,7 @@ SELECT_CRASH_LOG = r'''SELECT version FROM system.crash_log ORDER BY event_time DESC -''' +""" def retry(exception_types, max_attempts=5, max_interval=5): @@ -481,7 +481,8 @@ def retry(exception_types, max_attempts=5, max_interval=5): retry=tenacity.retry_if_exception_type(exception_types), wait=tenacity.wait_random_exponential(multiplier=0.5, max=max_interval), stop=tenacity.stop_after_attempt(max_attempts), - reraise=True) + reraise=True, + ) class ClickhouseError(Exception): @@ -502,9 +503,9 @@ class ClickhouseClient: def __init__(self, *, host="localhost", port=8123, user="default", password): self._session = requests.Session() if user: - self._session.headers['X-ClickHouse-User'] = user - self._session.headers['X-ClickHouse-Key'] = password - self._url = f'http://{host}:{port}' + self._session.headers["X-ClickHouse-User"] = user + self._session.headers["X-ClickHouse-Key"] = password + self._url = f"http://{host}:{port}" self._timeout = 60 self._ch_version = None @@ -516,7 +517,16 @@ class ClickhouseClient: return self._ch_version @retry(requests.exceptions.ConnectionError) - def query(self, query, query_args=None, format=None, post_data=None, timeout=None, echo=False, dry_run=False): + def query( + self, + query, + query_args=None, + format=None, + post_data=None, + timeout=None, + echo=False, + dry_run=False, + ): """ Execute query. """ @@ -524,28 +534,30 @@ class ClickhouseClient: query = self.render_query(query, **query_args) if format: - query += f' FORMAT {format}' + query += f" FORMAT {format}" if timeout is None: timeout = self._timeout if echo: - print(sqlparse.format(query, reindent=True), '\n') + print(sqlparse.format(query, reindent=True), "\n") if dry_run: return None try: - response = self._session.post(self._url, - params={ - 'query': query, - }, - json=post_data, - timeout=timeout) + response = self._session.post( + self._url, + params={ + "query": query, + }, + json=post_data, + timeout=timeout, + ) response.raise_for_status() - if format in ('JSON', 'JSONCompact'): + if format in ("JSON", "JSONCompact"): return response.json() return response.text.strip() @@ -555,7 +567,9 @@ class ClickhouseClient: def render_query(self, query, **kwargs): env = jinja2.Environment() - env.globals['version_ge'] = lambda version: version_ge(self.clickhouse_version, version) + env.globals["version_ge"] = lambda version: version_ge( + self.clickhouse_version, version + ) template = env.from_string(query) return template.render(kwargs) @@ -578,11 +592,13 @@ class ClickhouseConfig: @classmethod def load(cls): - return ClickhouseConfig(cls._load_config('/var/lib/clickhouse/preprocessed_configs/config.xml')) + return ClickhouseConfig( + cls._load_config("/var/lib/clickhouse/preprocessed_configs/config.xml") + ) @staticmethod def _load_config(config_path): - with open(config_path, 'r') as file: + with open(config_path, "r") as file: return xmltodict.parse(file.read()) @classmethod @@ -591,8 +607,8 @@ class ClickhouseConfig: for key, value in list(config.items()): if isinstance(value, MutableMapping): cls._mask_secrets(config[key]) - elif key in ('password', 'secret_access_key', 'header', 'identity'): - config[key] = '*****' + elif key in ("password", "secret_access_key", "header", "identity"): + config[key] = "*****" class DiagnosticsData: @@ -603,53 +619,53 @@ class DiagnosticsData: def __init__(self, args): self.args = args self.host = args.host - self._sections = [{'section': None, 'data': {}}] + self._sections = [{"section": None, "data": {}}] def add_string(self, name, value, section=None): self._section(section)[name] = { - 'type': 'string', - 'value': value, + "type": "string", + "value": value, } def add_xml_document(self, name, document, section=None): self._section(section)[name] = { - 'type': 'xml', - 'value': document, + "type": "xml", + "value": document, } def add_query(self, name, query, result, section=None): self._section(section)[name] = { - 'type': 'query', - 'query': query, - 'result': result, + "type": "query", + "query": query, + "result": result, } def add_command(self, name, command, result, section=None): self._section(section)[name] = { - 'type': 'command', - 'command': command, - 'result': result, + "type": "command", + "command": command, + "result": result, } def dump(self, format): - if format.startswith('json'): + if format.startswith("json"): result = self._dump_json() - elif format.startswith('yaml'): + elif format.startswith("yaml"): result = self._dump_yaml() else: result = self._dump_wiki() - if format.endswith('.gz'): - compressor = gzip.GzipFile(mode='wb', fileobj=sys.stdout.buffer) + if format.endswith(".gz"): + compressor = gzip.GzipFile(mode="wb", fileobj=sys.stdout.buffer) compressor.write(result.encode()) else: print(result) def _section(self, name=None): - if self._sections[-1]['section'] != name: - self._sections.append({'section': name, 'data': {}}) + if self._sections[-1]["section"] != name: + self._sections.append({"section": name, "data": {}}) - return self._sections[-1]['data'] + return self._sections[-1]["data"] def _dump_json(self): """ @@ -669,85 +685,85 @@ class DiagnosticsData: """ def _write_title(buffer, value): - buffer.write(f'### {value}\n') + buffer.write(f"### {value}\n") def _write_subtitle(buffer, value): - buffer.write(f'#### {value}\n') + buffer.write(f"#### {value}\n") def _write_string_item(buffer, name, item): - value = item['value'] - if value != '': - value = f'**{value}**' - buffer.write(f'{name}: {value}\n') + value = item["value"] + if value != "": + value = f"**{value}**" + buffer.write(f"{name}: {value}\n") def _write_xml_item(buffer, section_name, name, item): if section_name: - buffer.write(f'##### {name}\n') + buffer.write(f"##### {name}\n") else: _write_subtitle(buffer, name) - _write_result(buffer, item['value'], format='XML') + _write_result(buffer, item["value"], format="XML") def _write_query_item(buffer, section_name, name, item): if section_name: - buffer.write(f'##### {name}\n') + buffer.write(f"##### {name}\n") else: _write_subtitle(buffer, name) - _write_query(buffer, item['query']) - _write_result(buffer, item['result']) + _write_query(buffer, item["query"]) + _write_result(buffer, item["result"]) def _write_command_item(buffer, section_name, name, item): if section_name: - buffer.write(f'##### {name}\n') + buffer.write(f"##### {name}\n") else: _write_subtitle(buffer, name) - _write_command(buffer, item['command']) - _write_result(buffer, item['result']) + _write_command(buffer, item["command"]) + _write_result(buffer, item["result"]) def _write_unknown_item(buffer, section_name, name, item): if section_name: - buffer.write(f'**{name}**\n') + buffer.write(f"**{name}**\n") else: _write_subtitle(buffer, name) json.dump(item, buffer, indent=2) def _write_query(buffer, query): - buffer.write('**query**\n') - buffer.write('```sql\n') + buffer.write("**query**\n") + buffer.write("```sql\n") buffer.write(query) - buffer.write('\n```\n') + buffer.write("\n```\n") def _write_command(buffer, command): - buffer.write('**command**\n') - buffer.write('```\n') + buffer.write("**command**\n") + buffer.write("```\n") buffer.write(command) - buffer.write('\n```\n') + buffer.write("\n```\n") def _write_result(buffer, result, format=None): - buffer.write('**result**\n') - buffer.write(f'```{format}\n' if format else '```\n') + buffer.write("**result**\n") + buffer.write(f"```{format}\n" if format else "```\n") buffer.write(result) - buffer.write('\n```\n') + buffer.write("\n```\n") buffer = io.StringIO() - _write_title(buffer, f'Diagnostics data for host {self.host}') + _write_title(buffer, f"Diagnostics data for host {self.host}") for section in self._sections: - section_name = section['section'] + section_name = section["section"] if section_name: _write_subtitle(buffer, section_name) - for name, item in section['data'].items(): - if item['type'] == 'string': + for name, item in section["data"].items(): + if item["type"] == "string": _write_string_item(buffer, name, item) - elif item['type'] == 'query': + elif item["type"] == "query": _write_query_item(buffer, section_name, name, item) - elif item['type'] == 'command': + elif item["type"] == "command": _write_command_item(buffer, section_name, name, item) - elif item['type'] == 'xml': + elif item["type"] == "xml": _write_xml_item(buffer, section_name, name, item) else: _write_unknown_item(buffer, section_name, name, item) @@ -760,126 +776,196 @@ def main(): Program entry point. """ args = parse_args() - timestamp = datetime.strftime(datetime.now(), '%Y-%m-%d %H:%M:%S') - client = ClickhouseClient(host=args.host, port=args.port, user=args.user, password=args.password) + timestamp = datetime.strftime(datetime.now(), "%Y-%m-%d %H:%M:%S") + client = ClickhouseClient( + host=args.host, port=args.port, user=args.user, password=args.password + ) ch_config = ClickhouseConfig.load() version = client.clickhouse_version - system_tables = [row[0] for row in execute_query(client, SELECT_SYSTEM_TABLES, format='JSONCompact')['data']] + system_tables = [ + row[0] + for row in execute_query(client, SELECT_SYSTEM_TABLES, format="JSONCompact")[ + "data" + ] + ] diagnostics = DiagnosticsData(args) - diagnostics.add_string('Version', version) - diagnostics.add_string('Timestamp', timestamp) - diagnostics.add_string('Uptime', execute_query(client, SELECT_UPTIME)) + diagnostics.add_string("Version", version) + diagnostics.add_string("Timestamp", timestamp) + diagnostics.add_string("Uptime", execute_query(client, SELECT_UPTIME)) - diagnostics.add_xml_document('ClickHouse configuration', ch_config.dump()) + diagnostics.add_xml_document("ClickHouse configuration", ch_config.dump()) - if version_ge(version, '20.8'): - add_query(diagnostics, 'Access configuration', - client=client, - query=SELECT_ACCESS, - format='TSVRaw') - add_query(diagnostics, 'Quotas', - client=client, - query=SELECT_QUOTA_USAGE, - format='Vertical') + if version_ge(version, "20.8"): + add_query( + diagnostics, + "Access configuration", + client=client, + query=SELECT_ACCESS, + format="TSVRaw", + ) + add_query( + diagnostics, + "Quotas", + client=client, + query=SELECT_QUOTA_USAGE, + format="Vertical", + ) - add_query(diagnostics, 'Database engines', - client=client, - query=SELECT_DATABASE_ENGINES, - format='PrettyCompactNoEscapes', - section='Schema') - add_query(diagnostics, 'Databases (top 10 by size)', - client=client, - query=SELECT_DATABASES, - format='PrettyCompactNoEscapes', - section='Schema') - add_query(diagnostics, 'Table engines', - client=client, - query=SELECT_TABLE_ENGINES, - format='PrettyCompactNoEscapes', - section='Schema') - add_query(diagnostics, 'Dictionaries', - client=client, - query=SELECT_DICTIONARIES, - format='PrettyCompactNoEscapes', - section='Schema') + add_query( + diagnostics, + "Database engines", + client=client, + query=SELECT_DATABASE_ENGINES, + format="PrettyCompactNoEscapes", + section="Schema", + ) + add_query( + diagnostics, + "Databases (top 10 by size)", + client=client, + query=SELECT_DATABASES, + format="PrettyCompactNoEscapes", + section="Schema", + ) + add_query( + diagnostics, + "Table engines", + client=client, + query=SELECT_TABLE_ENGINES, + format="PrettyCompactNoEscapes", + section="Schema", + ) + add_query( + diagnostics, + "Dictionaries", + client=client, + query=SELECT_DICTIONARIES, + format="PrettyCompactNoEscapes", + section="Schema", + ) - add_query(diagnostics, 'Replicated tables (top 10 by absolute delay)', - client=client, - query=SELECT_REPLICAS, - format='PrettyCompactNoEscapes', - section='Replication') - add_query(diagnostics, 'Replication queue (top 20 oldest tasks)', - client=client, - query=SELECT_REPLICATION_QUEUE, - format='Vertical', - section='Replication') - if version_ge(version, '21.3'): - add_query(diagnostics, 'Replicated fetches', - client=client, - query=SELECT_REPLICATED_FETCHES, - format='Vertical', - section='Replication') + add_query( + diagnostics, + "Replicated tables (top 10 by absolute delay)", + client=client, + query=SELECT_REPLICAS, + format="PrettyCompactNoEscapes", + section="Replication", + ) + add_query( + diagnostics, + "Replication queue (top 20 oldest tasks)", + client=client, + query=SELECT_REPLICATION_QUEUE, + format="Vertical", + section="Replication", + ) + if version_ge(version, "21.3"): + add_query( + diagnostics, + "Replicated fetches", + client=client, + query=SELECT_REPLICATED_FETCHES, + format="Vertical", + section="Replication", + ) - add_query(diagnostics, 'Top 10 tables by max parts per partition', - client=client, - query=SELECT_PARTS_PER_TABLE, - format='PrettyCompactNoEscapes') - add_query(diagnostics, 'Merges in progress', - client=client, - query=SELECT_MERGES, - format='Vertical') - add_query(diagnostics, 'Mutations in progress', - client=client, - query=SELECT_MUTATIONS, - format='Vertical') - add_query(diagnostics, 'Recent data parts (modification time within last 3 minutes)', - client=client, - query=SELECT_RECENT_DATA_PARTS, - format='Vertical') + add_query( + diagnostics, + "Top 10 tables by max parts per partition", + client=client, + query=SELECT_PARTS_PER_TABLE, + format="PrettyCompactNoEscapes", + ) + add_query( + diagnostics, + "Merges in progress", + client=client, + query=SELECT_MERGES, + format="Vertical", + ) + add_query( + diagnostics, + "Mutations in progress", + client=client, + query=SELECT_MUTATIONS, + format="Vertical", + ) + add_query( + diagnostics, + "Recent data parts (modification time within last 3 minutes)", + client=client, + query=SELECT_RECENT_DATA_PARTS, + format="Vertical", + ) - add_query(diagnostics, 'system.detached_parts', - client=client, - query=SELECT_DETACHED_DATA_PARTS, - format='PrettyCompactNoEscapes', - section='Detached data') - add_command(diagnostics, 'Disk space usage', - command='du -sh -L -c /var/lib/clickhouse/data/*/*/detached/* | sort -rsh', - section='Detached data') + add_query( + diagnostics, + "system.detached_parts", + client=client, + query=SELECT_DETACHED_DATA_PARTS, + format="PrettyCompactNoEscapes", + section="Detached data", + ) + add_command( + diagnostics, + "Disk space usage", + command="du -sh -L -c /var/lib/clickhouse/data/*/*/detached/* | sort -rsh", + section="Detached data", + ) - add_query(diagnostics, 'Queries in progress (process list)', - client=client, - query=SELECT_PROCESSES, - format='Vertical', - section='Queries') - add_query(diagnostics, 'Top 10 queries by duration', - client=client, - query=SELECT_TOP_QUERIES_BY_DURATION, - format='Vertical', - section='Queries') - add_query(diagnostics, 'Top 10 queries by memory usage', - client=client, - query=SELECT_TOP_QUERIES_BY_MEMORY_USAGE, - format='Vertical', - section='Queries') - add_query(diagnostics, 'Last 10 failed queries', - client=client, - query=SELECT_FAILED_QUERIES, - format='Vertical', - section='Queries') + add_query( + diagnostics, + "Queries in progress (process list)", + client=client, + query=SELECT_PROCESSES, + format="Vertical", + section="Queries", + ) + add_query( + diagnostics, + "Top 10 queries by duration", + client=client, + query=SELECT_TOP_QUERIES_BY_DURATION, + format="Vertical", + section="Queries", + ) + add_query( + diagnostics, + "Top 10 queries by memory usage", + client=client, + query=SELECT_TOP_QUERIES_BY_MEMORY_USAGE, + format="Vertical", + section="Queries", + ) + add_query( + diagnostics, + "Last 10 failed queries", + client=client, + query=SELECT_FAILED_QUERIES, + format="Vertical", + section="Queries", + ) - add_query(diagnostics, 'Stack traces', - client=client, - query=SELECT_STACK_TRACES, - format='Vertical') + add_query( + diagnostics, + "Stack traces", + client=client, + query=SELECT_STACK_TRACES, + format="Vertical", + ) - if 'crash_log' in system_tables: - add_query(diagnostics, 'Crash log', - client=client, - query=SELECT_CRASH_LOG, - format='Vertical') + if "crash_log" in system_tables: + add_query( + diagnostics, + "Crash log", + client=client, + query=SELECT_CRASH_LOG, + format="Vertical", + ) - add_command(diagnostics, 'uname', 'uname -a') + add_command(diagnostics, "uname", "uname -a") diagnostics.dump(args.format) @@ -889,29 +975,34 @@ def parse_args(): Parse command-line arguments. """ parser = argparse.ArgumentParser() - parser.add_argument('--format', - choices=['json', 'yaml', 'json.gz', 'yaml.gz', 'wiki', 'wiki.gz'], - default='wiki') - parser.add_argument('--normalize-queries', - action='store_true', - default=False) - parser.add_argument('--host', dest="host", help="clickhouse host") - parser.add_argument('--port', dest="port", default=8123, help="clickhouse http port") - parser.add_argument('--user', dest="user", default="default", help="clickhouse user") - parser.add_argument('--password', dest="password", help="clickhouse password") + parser.add_argument( + "--format", + choices=["json", "yaml", "json.gz", "yaml.gz", "wiki", "wiki.gz"], + default="wiki", + ) + parser.add_argument("--normalize-queries", action="store_true", default=False) + parser.add_argument("--host", dest="host", help="clickhouse host") + parser.add_argument( + "--port", dest="port", default=8123, help="clickhouse http port" + ) + parser.add_argument( + "--user", dest="user", default="default", help="clickhouse user" + ) + parser.add_argument("--password", dest="password", help="clickhouse password") return parser.parse_args() def add_query(diagnostics, name, client, query, format, section=None): query_args = { - 'normalize_queries': diagnostics.args.normalize_queries, + "normalize_queries": diagnostics.args.normalize_queries, } query = client.render_query(query, **query_args) diagnostics.add_query( name=name, query=query, result=execute_query(client, query, render_query=False, format=format), - section=section) + section=section, + ) def execute_query(client, query, render_query=True, format=None): @@ -926,14 +1017,18 @@ def execute_query(client, query, render_query=True, format=None): def add_command(diagnostics, name, command, section=None): diagnostics.add_command( - name=name, - command=command, - result=execute_command(command), - section=section) + name=name, command=command, result=execute_command(command), section=section + ) def execute_command(command, input=None): - proc = subprocess.Popen(command, shell=True, stdin=subprocess.PIPE, stdout=subprocess.PIPE, stderr=subprocess.PIPE) + proc = subprocess.Popen( + command, + shell=True, + stdin=subprocess.PIPE, + stdout=subprocess.PIPE, + stderr=subprocess.PIPE, + ) if isinstance(input, str): input = input.encode() @@ -941,7 +1036,7 @@ def execute_command(command, input=None): stdout, stderr = proc.communicate(input=input) if proc.returncode: - return f'failed with exit code {proc.returncode}\n{stderr.decode()}' + return f"failed with exit code {proc.returncode}\n{stderr.decode()}" return stdout.decode() @@ -957,8 +1052,8 @@ def parse_version(version): """ Parse version string. """ - return [int(x) for x in version.strip().split('.') if x.isnumeric()] + return [int(x) for x in version.strip().split(".") if x.isnumeric()] -if __name__ == '__main__': +if __name__ == "__main__": main() diff --git a/utils/s3tools/s3uploader b/utils/s3tools/s3uploader index 33db76f57f4..4855bdb3f96 100755 --- a/utils/s3tools/s3uploader +++ b/utils/s3tools/s3uploader @@ -28,39 +28,48 @@ class S3API(object): bucket = self.connection.get_bucket(bucket_name) key = bucket.initiate_multipart_upload(s3_path) logging.info("Will upload to s3 path %s", s3_path) - chunksize = 1024 * 1024 * 1024 # 1 GB + chunksize = 1024 * 1024 * 1024 # 1 GB filesize = os.stat(file_path).st_size logging.info("File size is %s", filesize) chunkcount = int(math.ceil(filesize / chunksize)) def call_back(x, y): print("Uploaded {}/{} bytes".format(x, y)) + try: for i in range(chunkcount + 1): logging.info("Uploading chunk %s of %s", i, chunkcount + 1) offset = chunksize * i bytes_size = min(chunksize, filesize - offset) - with open(file_path, 'r') as fp: + with open(file_path, "r") as fp: fp.seek(offset) - key.upload_part_from_file(fp=fp, part_num=i+1, - size=bytes_size, cb=call_back, - num_cb=100) + key.upload_part_from_file( + fp=fp, part_num=i + 1, size=bytes_size, cb=call_back, num_cb=100 + ) key.complete_upload() except Exception as ex: key.cancel_upload() raise ex logging.info("Contents were set") return "https://{bucket}.{mds_url}/{path}".format( - bucket=bucket_name, mds_url=self.mds_url, path=s3_path) + bucket=bucket_name, mds_url=self.mds_url, path=s3_path + ) def set_file_contents(self, bucket, local_file_path, s3_file_path): key = Key(bucket) key.key = s3_file_path file_size = os.stat(local_file_path).st_size - logging.info("Uploading file `%s` to `%s`. Size is %s", local_file_path, s3_file_path, file_size) + logging.info( + "Uploading file `%s` to `%s`. Size is %s", + local_file_path, + s3_file_path, + file_size, + ) + def call_back(x, y): print("Uploaded {}/{} bytes".format(x, y)) + key.set_contents_from_filename(local_file_path, cb=call_back) def upload_data_for_static_files_disk(self, bucket_name, directory_path, s3_path): @@ -74,12 +83,14 @@ class S3API(object): path = root.split(os.sep) for file in files: local_file_path = os.path.join(root, file) - s3_file = local_file_path[len(directory_path) + 1:] + s3_file = local_file_path[len(directory_path) + 1 :] s3_file_path = os.path.join(s3_path, s3_file) self.set_file_contents(bucket, local_file_path, s3_file_path) logging.info("Uploading finished") - return "https://{bucket}.{mds_url}/{path}".format(bucket=bucket_name, mds_url=self.mds_url, path=s3_path) + return "https://{bucket}.{mds_url}/{path}".format( + bucket=bucket_name, mds_url=self.mds_url, path=s3_path + ) def list_bucket_keys(self, bucket_name): bucket = self.connection.get_bucket(bucket_name) @@ -91,100 +102,121 @@ class S3API(object): bucket.get_all_keys() for obj in bucket.get_all_keys(): if obj.key.startswith(folder_path): - print('Removing ' + obj.key) + print("Removing " + obj.key) obj.delete() -def make_tar_file_for_table(clickhouse_data_path, db_name, table_name, - tmp_prefix): +def make_tar_file_for_table(clickhouse_data_path, db_name, table_name, tmp_prefix): - relative_data_path = os.path.join('data', db_name, table_name) - relative_meta_path = os.path.join('metadata', db_name, table_name + '.sql') + relative_data_path = os.path.join("data", db_name, table_name) + relative_meta_path = os.path.join("metadata", db_name, table_name + ".sql") path_to_data = os.path.join(clickhouse_data_path, relative_data_path) path_to_metadata = os.path.join(clickhouse_data_path, relative_meta_path) - temporary_file_name = tmp_prefix + '/{tname}.tar'.format(tname=table_name) + temporary_file_name = tmp_prefix + "/{tname}.tar".format(tname=table_name) with tarfile.open(temporary_file_name, "w") as bundle: bundle.add(path_to_data, arcname=relative_data_path) bundle.add(path_to_metadata, arcname=relative_meta_path) return temporary_file_name -USAGE_EXAMPLES = ''' +USAGE_EXAMPLES = """ examples: \t./s3uploader --dataset-name some_ds --access-key-id XXX --secret-access-key YYY --clickhouse-data-path /opt/clickhouse/ --table-name default.some_tbl --bucket-name some-bucket \t./s3uploader --dataset-name some_ds --access-key-id XXX --secret-access-key YYY --file-path some_ds.tsv.xz --bucket-name some-bucket --s3-path /path/to/ -''' +""" if __name__ == "__main__": - logging.basicConfig(level=logging.INFO, format='%(asctime)s %(message)s') + logging.basicConfig(level=logging.INFO, format="%(asctime)s %(message)s") parser = argparse.ArgumentParser( description="Simple tool for uploading datasets to clickhouse S3", - usage='%(prog)s [options] {}'.format(USAGE_EXAMPLES)) - parser.add_argument('--s3-api-url', default='s3.amazonaws.com') - parser.add_argument('--s3-common-url', default='s3.amazonaws.com') - parser.add_argument('--bucket-name', default='clickhouse-datasets') - parser.add_argument('--dataset-name', required=True, - help='Name of dataset, will be used in uploaded path') - parser.add_argument('--access-key-id', required=True) - parser.add_argument('--secret-access-key', required=True) - parser.add_argument('--clickhouse-data-path', - default='/var/lib/clickhouse/', - help='Path to clickhouse database on filesystem') - parser.add_argument('--s3-path', help='Path in s3, where to upload file') - parser.add_argument('--tmp-prefix', default='/tmp', - help='Prefix to store temporary downloaded file') + usage="%(prog)s [options] {}".format(USAGE_EXAMPLES), + ) + parser.add_argument("--s3-api-url", default="s3.amazonaws.com") + parser.add_argument("--s3-common-url", default="s3.amazonaws.com") + parser.add_argument("--bucket-name", default="clickhouse-datasets") + parser.add_argument( + "--dataset-name", + required=True, + help="Name of dataset, will be used in uploaded path", + ) + parser.add_argument("--access-key-id", required=True) + parser.add_argument("--secret-access-key", required=True) + parser.add_argument( + "--clickhouse-data-path", + default="/var/lib/clickhouse/", + help="Path to clickhouse database on filesystem", + ) + parser.add_argument("--s3-path", help="Path in s3, where to upload file") + parser.add_argument( + "--tmp-prefix", default="/tmp", help="Prefix to store temporary downloaded file" + ) data_group = parser.add_mutually_exclusive_group(required=True) - table_name_argument = data_group.add_argument('--table-name', - help='Name of table with database, if you are uploading partitions') - data_group.add_argument('--file-path', - help='Name of file, if you are uploading') - data_group.add_argument('--directory-path', help='Path to directory with files to upload') - data_group.add_argument('--list-directory', help='List s3 directory by --directory-path') - data_group.add_argument('--remove-directory', help='Remove s3 directory by --directory-path') + table_name_argument = data_group.add_argument( + "--table-name", + help="Name of table with database, if you are uploading partitions", + ) + data_group.add_argument("--file-path", help="Name of file, if you are uploading") + data_group.add_argument( + "--directory-path", help="Path to directory with files to upload" + ) + data_group.add_argument( + "--list-directory", help="List s3 directory by --directory-path" + ) + data_group.add_argument( + "--remove-directory", help="Remove s3 directory by --directory-path" + ) args = parser.parse_args() if args.table_name is not None and args.clickhouse_data_path is None: - raise argparse.ArgumentError(table_name_argument, - "You should specify --clickhouse-data-path to upload --table") + raise argparse.ArgumentError( + table_name_argument, + "You should specify --clickhouse-data-path to upload --table", + ) s3_conn = S3API( - args.access_key_id, args.secret_access_key, - args.s3_api_url, args.s3_common_url) + args.access_key_id, args.secret_access_key, args.s3_api_url, args.s3_common_url + ) - file_path = '' + file_path = "" directory_path = args.directory_path s3_path = args.s3_path if args.list_directory: s3_conn.list_bucket_keys(args.bucket_name) elif args.remove_directory: - print('Removing s3 path: ' + args.remove_directory) + print("Removing s3 path: " + args.remove_directory) s3_conn.remove_folder_from_bucket(args.bucket_name, args.remove_directory) elif args.directory_path is not None: - url = s3_conn.upload_data_for_static_files_disk(args.bucket_name, directory_path, s3_path) + url = s3_conn.upload_data_for_static_files_disk( + args.bucket_name, directory_path, s3_path + ) logging.info("Data uploaded: %s", url) else: if args.table_name is not None: - if '.' not in args.table_name: - db_name = 'default' + if "." not in args.table_name: + db_name = "default" else: - db_name, table_name = args.table_name.split('.') + db_name, table_name = args.table_name.split(".") file_path = make_tar_file_for_table( - args.clickhouse_data_path, db_name, table_name, args.tmp_prefix) + args.clickhouse_data_path, db_name, table_name, args.tmp_prefix + ) else: file_path = args.file_path - if 'tsv' in file_path: + if "tsv" in file_path: s3_path = os.path.join( - args.dataset_name, 'tsv', os.path.basename(file_path)) + args.dataset_name, "tsv", os.path.basename(file_path) + ) if args.table_name is not None: s3_path = os.path.join( - args.dataset_name, 'partitions', os.path.basename(file_path)) + args.dataset_name, "partitions", os.path.basename(file_path) + ) elif args.s3_path is not None: s3_path = os.path.join( - args.dataset_name, args.s3_path, os.path.basename(file_path)) + args.dataset_name, args.s3_path, os.path.basename(file_path) + ) else: raise Exception("Don't know s3-path to upload") diff --git a/utils/test_history/test-history b/utils/test_history/test-history index fdd6c36e9dc..5f031af1d3a 100755 --- a/utils/test_history/test-history +++ b/utils/test_history/test-history @@ -11,13 +11,14 @@ from termcolor import colored import sys COLORMAP = { - "success": colored("success", 'green'), - "failure": colored("failure", 'red'), - "error": colored("error", 'red'), - "pending": colored("pending", 'yellow'), - "not run": colored("not run", 'white'), + "success": colored("success", "green"), + "failure": colored("failure", "red"), + "error": colored("error", "red"), + "pending": colored("pending", "yellow"), + "not run": colored("not run", "white"), } + def _filter_statuses(statuses): """ Squash statuses to latest state @@ -69,7 +70,7 @@ if __name__ == "__main__": date_since = datetime.datetime.strptime(args.since, "%Y-%m-%d %H:%M:%S") gh = Github(args.token) - repo = gh.get_repo('ClickHouse/ClickHouse') + repo = gh.get_repo("ClickHouse/ClickHouse") commits = get_commits(repo, date_since) longest_header = [] @@ -101,6 +102,6 @@ if __name__ == "__main__": result_data.append(current_result) if sys.stdout.isatty(): - longest_header = [colored(h, 'white', attrs=['bold']) for h in longest_header] + longest_header = [colored(h, "white", attrs=["bold"]) for h in longest_header] print(tabulate.tabulate(result_data, headers=longest_header, tablefmt="grid")) From 9a2a091c76e3c13c57b970eda072141b5413f27f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 23 Mar 2023 16:39:16 +0100 Subject: [PATCH 326/418] Get rid of temporary COMMON_BUILD_PREFIX --- docker/test/performance-comparison/entrypoint.sh | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/docker/test/performance-comparison/entrypoint.sh b/docker/test/performance-comparison/entrypoint.sh index 75b25412ac4..a4d93295eb9 100755 --- a/docker/test/performance-comparison/entrypoint.sh +++ b/docker/test/performance-comparison/entrypoint.sh @@ -7,11 +7,6 @@ export CHPC_CHECK_START_TIMESTAMP S3_URL=${S3_URL:="https://clickhouse-builds.s3.amazonaws.com"} BUILD_NAME=${BUILD_NAME:-package_release} -COMMON_BUILD_PREFIX="/clickhouse_build_check" -if [[ $S3_URL == *"s3.amazonaws.com"* ]]; then - COMMON_BUILD_PREFIX="" -fi - # Sometimes AWS responde with DNS error and it's impossible to retry it with # current curl version options. function curl_with_retry @@ -92,9 +87,9 @@ chmod 777 workspace output cd workspace # Download the package for the version we are going to test. -if curl_with_retry "$S3_URL/$PR_TO_TEST/$SHA_TO_TEST$COMMON_BUILD_PREFIX/$BUILD_NAME/performance.tar.zst" +if curl_with_retry "$S3_URL/$PR_TO_TEST/$SHA_TO_TEST/$BUILD_NAME/performance.tar.zst" then - right_path="$S3_URL/$PR_TO_TEST/$SHA_TO_TEST$COMMON_BUILD_PREFIX/$BUILD_NAME/performance.tar.zst" + right_path="$S3_URL/$PR_TO_TEST/$SHA_TO_TEST/$BUILD_NAME/performance.tar.zst" fi mkdir right From c2a367995965f7556e90fe411c8866a6599f5969 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 23 Mar 2023 15:53:22 +0000 Subject: [PATCH 327/418] Add buildSelectQueryInfo to Planner/Utils, use in unit tests --- .../InterpreterSelectQueryAnalyzer.h | 8 +-- src/Planner/Planner.cpp | 7 +- src/Planner/Planner.h | 5 ++ src/Planner/Utils.cpp | 10 +++ src/Planner/Utils.h | 4 ++ ..._transform_query_for_external_database.cpp | 66 ++++++++++--------- 6 files changed, 57 insertions(+), 43 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.h b/src/Interpreters/InterpreterSelectQueryAnalyzer.h index 8ca5cbb92d7..de97400e01b 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.h +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.h @@ -9,8 +9,6 @@ #include #include -struct TestExternalDatabaseQueryState; - namespace DB { @@ -73,11 +71,9 @@ public: /// Set number_of_current_replica and count_participating_replicas in client_info void setProperClientInfo(size_t replica_number, size_t count_participating_replicas); + const Planner & getPlanner() const { return planner; } + private: - - /// Access select query info from unit tests (see gtest_transform_query_for_external_database) - friend struct ::TestExternalDatabaseQueryState; - ASTPtr query; ContextMutablePtr context; SelectQueryOptions select_query_options; diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 9e7e0e62ba0..bb3e274e6ef 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1452,12 +1452,7 @@ void Planner::buildPlanForQueryNode() SelectQueryInfo Planner::buildSelectQueryInfo() const { - SelectQueryInfo select_query_info; - select_query_info.original_query = queryNodeToSelectQuery(query_tree); - select_query_info.query = select_query_info.original_query; - select_query_info.query_tree = query_tree; - select_query_info.planner_context = planner_context; - return select_query_info; + return ::DB::buildSelectQueryInfo(query_tree, planner_context); } void Planner::addStorageLimits(const StorageLimitsList & limits) diff --git a/src/Planner/Planner.h b/src/Planner/Planner.h index 6fdce80b73a..783a07f6e99 100644 --- a/src/Planner/Planner.h +++ b/src/Planner/Planner.h @@ -55,6 +55,11 @@ public: void addStorageLimits(const StorageLimitsList & limits); + PlannerContextPtr getPlannerContext() const + { + return planner_context; + } + private: void buildPlanForUnionNode(); diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 2018ddafcdd..4af8440befb 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -404,4 +404,14 @@ QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(const NamesAndTyp return query_node; } +SelectQueryInfo buildSelectQueryInfo(const QueryTreeNodePtr & query_tree, const PlannerContextPtr & planner_context) +{ + SelectQueryInfo select_query_info; + select_query_info.original_query = queryNodeToSelectQuery(query_tree); + select_query_info.query = select_query_info.original_query; + select_query_info.query_tree = query_tree; + select_query_info.planner_context = planner_context; + return select_query_info; +} + } diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index 0effb1d08ae..f6123e1a1d3 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -17,6 +17,8 @@ #include +#include + namespace DB { @@ -74,4 +76,6 @@ QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(const NamesAndTyp const QueryTreeNodePtr & table_expression, const ContextPtr & context); +SelectQueryInfo buildSelectQueryInfo(const QueryTreeNodePtr & query_tree, const PlannerContextPtr & planner_context); + } diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index ed0706f3492..3eff0a8ba70 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -18,29 +18,25 @@ #include #include #include +#include +#include using namespace DB; /// TODO: use gtest fixture -struct TestExternalDatabaseQueryState +struct State { - TestExternalDatabaseQueryState(const TestExternalDatabaseQueryState&) = delete; + State(const State&) = delete; ContextMutablePtr context; - static const TestExternalDatabaseQueryState & instance() + static const State & instance() { - static TestExternalDatabaseQueryState state; + static State state; return state; } - static SelectQueryInfo getSelectQueryInfo(InterpreterSelectQueryAnalyzer & interpreter) - { - interpreter.planner.buildQueryPlanIfNeeded(); - return interpreter.planner.buildSelectQueryInfo(); - } - const NamesAndTypesList & getColumns(size_t idx = 0) const { return tables[idx].columns; @@ -92,7 +88,7 @@ private: }), }; - explicit TestExternalDatabaseQueryState() + explicit State() : context(Context::createCopy(getContext().context)) { tryRegisterFunctions(); @@ -129,7 +125,7 @@ private: }; static void checkOld( - const TestExternalDatabaseQueryState & state, + const State & state, size_t table_num, const std::string & query, const std::string & expected) @@ -171,7 +167,7 @@ static QueryTreeNodePtr findTableExpression(const QueryTreeNodePtr & node, const /// `column_names` - Normally it's passed to query plan step. But in test we do it manually. static void checkNewAnalyzer( - const TestExternalDatabaseQueryState & state, + const State & state, const Names & column_names, const std::string & query, const std::string & expected) @@ -179,9 +175,17 @@ static void checkNewAnalyzer( ParserSelectQuery parser; ASTPtr ast = parseQuery(parser, query, 1000, 1000); - SelectQueryOptions select_options; - InterpreterSelectQueryAnalyzer interpreter(ast, state.context, select_options); - SelectQueryInfo query_info = state.getSelectQueryInfo(interpreter); + SelectQueryOptions select_query_options; + auto query_tree = buildQueryTree(ast, state.context); + QueryTreePassManager query_tree_pass_manager(state.context); + addQueryTreePasses(query_tree_pass_manager); + query_tree_pass_manager.run(query_tree); + + InterpreterSelectQueryAnalyzer interpreter(query_tree, state.context, select_query_options); + interpreter.getQueryPlan(); + + auto planner_context = interpreter.getPlanner().getPlannerContext(); + SelectQueryInfo query_info = buildSelectQueryInfo(query_tree, planner_context); const auto * query_node = query_info.query_tree->as(); if (!query_node) throw Exception(ErrorCodes::LOGICAL_ERROR, "QueryNode expected"); @@ -195,7 +199,7 @@ static void checkNewAnalyzer( } static void check( - const TestExternalDatabaseQueryState & state, + const State & state, size_t table_num, const Names & column_names, const std::string & query, @@ -214,7 +218,7 @@ static void check( TEST(TransformQueryForExternalDatabase, InWithSingleElement) { - const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); + const State & state = State::instance(); check(state, 1, {"column"}, "SELECT column FROM test.table WHERE 1 IN (1)", @@ -232,7 +236,7 @@ TEST(TransformQueryForExternalDatabase, InWithSingleElement) TEST(TransformQueryForExternalDatabase, InWithMultipleColumns) { - const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); + const State & state = State::instance(); check(state, 1, {"column"}, "SELECT column FROM test.table WHERE (1,1) IN ((1,1))", @@ -245,7 +249,7 @@ TEST(TransformQueryForExternalDatabase, InWithMultipleColumns) TEST(TransformQueryForExternalDatabase, InWithTable) { - const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); + const State & state = State::instance(); check(state, 1, {"column"}, "SELECT column FROM test.table WHERE 1 IN external_table", @@ -263,7 +267,7 @@ TEST(TransformQueryForExternalDatabase, InWithTable) TEST(TransformQueryForExternalDatabase, Like) { - const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); + const State & state = State::instance(); check(state, 1, {"field"}, "SELECT field FROM test.table WHERE field LIKE '%hello%'", @@ -275,7 +279,7 @@ TEST(TransformQueryForExternalDatabase, Like) TEST(TransformQueryForExternalDatabase, Substring) { - const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); + const State & state = State::instance(); check(state, 1, {"field"}, "SELECT field FROM test.table WHERE left(field, 10) = RIGHT(field, 10) AND SUBSTRING(field FROM 1 FOR 2) = 'Hello'", @@ -284,7 +288,7 @@ TEST(TransformQueryForExternalDatabase, Substring) TEST(TransformQueryForExternalDatabase, MultipleAndSubqueries) { - const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); + const State & state = State::instance(); check(state, 1, {"column"}, "SELECT column FROM test.table WHERE 1 = 1 AND toString(column) = '42' AND column = 42 AND left(toString(column), 10) = RIGHT(toString(column), 10) AND column IN (1, 42) AND SUBSTRING(toString(column) FROM 1 FOR 2) = 'Hello' AND column != 4", @@ -296,7 +300,7 @@ TEST(TransformQueryForExternalDatabase, MultipleAndSubqueries) TEST(TransformQueryForExternalDatabase, Issue7245) { - const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); + const State & state = State::instance(); check(state, 1, {"apply_id", "apply_type", "apply_status", "create_time"}, "SELECT apply_id FROM test.table WHERE apply_type = 2 AND create_time > addDays(toDateTime('2019-01-01 01:02:03'),-7) AND apply_status IN (3,4)", @@ -305,7 +309,7 @@ TEST(TransformQueryForExternalDatabase, Issue7245) TEST(TransformQueryForExternalDatabase, Aliases) { - const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); + const State & state = State::instance(); check(state, 1, {"field"}, "SELECT field AS value, field AS display FROM table WHERE field NOT IN ('') AND display LIKE '%test%'", @@ -314,7 +318,7 @@ TEST(TransformQueryForExternalDatabase, Aliases) TEST(TransformQueryForExternalDatabase, ForeignColumnInWhere) { - const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); + const State & state = State::instance(); check(state, 2, {"column", "apply_id"}, "SELECT column FROM test.table " @@ -325,7 +329,7 @@ TEST(TransformQueryForExternalDatabase, ForeignColumnInWhere) TEST(TransformQueryForExternalDatabase, NoStrict) { - const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); + const State & state = State::instance(); check(state, 1, {"field"}, "SELECT field FROM table WHERE field IN (SELECT attr FROM table2)", @@ -334,7 +338,7 @@ TEST(TransformQueryForExternalDatabase, NoStrict) TEST(TransformQueryForExternalDatabase, Strict) { - const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); + const State & state = State::instance(); state.context->setSetting("external_table_strict_query", true); check(state, 1, {"field"}, @@ -355,7 +359,7 @@ TEST(TransformQueryForExternalDatabase, Strict) TEST(TransformQueryForExternalDatabase, Null) { - const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); + const State & state = State::instance(); check(state, 1, {"field"}, "SELECT field FROM table WHERE field IS NULL", @@ -374,7 +378,7 @@ TEST(TransformQueryForExternalDatabase, Null) TEST(TransformQueryForExternalDatabase, ToDate) { - const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); + const State & state = State::instance(); check(state, 1, {"a", "b", "foo"}, "SELECT foo FROM table WHERE a=10 AND b=toDate('2019-10-05')", @@ -383,7 +387,7 @@ TEST(TransformQueryForExternalDatabase, ToDate) TEST(TransformQueryForExternalDatabase, Analyzer) { - const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance(); + const State & state = State::instance(); check(state, 1, {"field"}, "SELECT count() FROM table WHERE field LIKE '%name_%'", From 9bf4f0108e1f3bd4c9f3ae6e927a75f4b21d1daf Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 23 Mar 2023 18:02:31 +0100 Subject: [PATCH 328/418] set content-length for empty POST requests --- src/IO/ReadWriteBufferFromHTTP.h | 2 ++ src/Server/HTTP/HTTPServerRequest.cpp | 6 ++---- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index ed25c101d7e..784110f735e 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -159,6 +159,8 @@ namespace detail if (out_stream_callback) request.setChunkedTransferEncoding(true); + else if (method == Poco::Net::HTTPRequest::HTTP_POST) + request.setContentLength(0); /// No callback - no body for (auto & [header, value] : http_header_entries) request.set(header, value); diff --git a/src/Server/HTTP/HTTPServerRequest.cpp b/src/Server/HTTP/HTTPServerRequest.cpp index b66c6c6a62e..891ac39c931 100644 --- a/src/Server/HTTP/HTTPServerRequest.cpp +++ b/src/Server/HTTP/HTTPServerRequest.cpp @@ -63,11 +63,9 @@ HTTPServerRequest::HTTPServerRequest(HTTPContextPtr context, HTTPServerResponse } else if (getMethod() != HTTPRequest::HTTP_GET && getMethod() != HTTPRequest::HTTP_HEAD && getMethod() != HTTPRequest::HTTP_DELETE) { - /// That check for has_body may be false-negative in rare cases, but it's okay - bool has_body = in->hasPendingData(); stream = std::move(in); - if (!startsWith(getContentType(), "multipart/form-data") && has_body) - LOG_WARNING(&Poco::Logger::get("HTTPServerRequest"), "Got an HTTP request with no content length " + if (!startsWith(getContentType(), "multipart/form-data")) + LOG_WARNING(LogFrequencyLimiter(&Poco::Logger::get("HTTPServerRequest"), 10), "Got an HTTP request with no content length " "and no chunked/multipart encoding, it may be impossible to distinguish graceful EOF from abnormal connection loss"); } else From e9bb7efef6dda22d4a74ac5e0feecbabaf1d4266 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 23 Mar 2023 18:27:30 +0000 Subject: [PATCH 329/418] Fix fuzzed test. --- .../Optimizations/projectionsCommon.cpp | 19 ++++++++++++++++++- ...llegal_type_of_filter_projection.reference | 0 ...2680_illegal_type_of_filter_projection.sql | 3 +++ 3 files changed, 21 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02680_illegal_type_of_filter_projection.reference create mode 100644 tests/queries/0_stateless/02680_illegal_type_of_filter_projection.sql diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index 908d599bb06..237dcfa81e0 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -5,13 +5,22 @@ #include #include +#include #include #include #include #include -namespace DB::QueryPlanOptimizations +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER; +} + +namespace QueryPlanOptimizations { bool canUseProjectionForReadingStep(ReadFromMergeTree * reading) @@ -65,6 +74,13 @@ const ActionsDAG::Node * findInOutputs(ActionsDAG & dag, const std::string & nam if ((*it)->result_name == name) { const auto * node = *it; + + + if (!isUInt8(removeNullable(node->result_type))) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, + "Illegal type {} of column {} for filter. Must be UInt8 or Nullable(UInt8).", + node->result_type->getName(), name); + if (remove) { outputs.erase(it); @@ -240,3 +256,4 @@ bool analyzeProjectionCandidate( } } +} diff --git a/tests/queries/0_stateless/02680_illegal_type_of_filter_projection.reference b/tests/queries/0_stateless/02680_illegal_type_of_filter_projection.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02680_illegal_type_of_filter_projection.sql b/tests/queries/0_stateless/02680_illegal_type_of_filter_projection.sql new file mode 100644 index 00000000000..3ef3b8a4fe6 --- /dev/null +++ b/tests/queries/0_stateless/02680_illegal_type_of_filter_projection.sql @@ -0,0 +1,3 @@ +CREATE TABLE test_tuple (`p` DateTime, `i` int, `j` int) ENGINE = MergeTree PARTITION BY (toDate(p), i) ORDER BY j SETTINGS index_granularity = 1; +insert into test_tuple values (1, 1, 1); +SELECT count() FROM test_tuple PREWHERE sipHash64(sipHash64(p, toString(toDate(p))), toString(toDate(p))) % -0. WHERE i > NULL settings optimize_trivial_count_query=0; -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } From ed4a5b2be849b8608896493fc492e68fd6794431 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 23 Mar 2023 18:39:41 +0000 Subject: [PATCH 330/418] Fix test. Pre-build index for query dag in projection. --- .../optimizeUseAggregateProjection.cpp | 8 +++++--- .../QueryPlan/Optimizations/projectionsCommon.cpp | 2 +- src/Processors/Transforms/FilterTransform.cpp | 13 +++++++++++++ 3 files changed, 19 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index ad34e9d4a09..77b5547207c 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -231,10 +231,10 @@ bool areAggregatesMatch( ActionsDAGPtr analyzeAggregateProjection( const AggregateProjectionInfo & info, const QueryDAG & query, + const DAGIndex & query_index, 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); @@ -418,6 +418,8 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( if (!dag.build(*node.children.front())) return candidates; + auto query_index = buildDAGIndex(*dag.dag); + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", dag.dag->dumpDAG()); candidates.has_filter = dag.filter_node; @@ -428,7 +430,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, keys, aggregates)) + if (auto proj_dag = analyzeAggregateProjection(info, dag, query_index, 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)}; @@ -467,7 +469,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, keys, aggregates)) + if (auto proj_dag = analyzeAggregateProjection(info, dag, query_index, 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)}; diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index 237dcfa81e0..6367edafce8 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -76,7 +76,7 @@ const ActionsDAG::Node * findInOutputs(ActionsDAG & dag, const std::string & nam const auto * node = *it; - if (!isUInt8(removeNullable(node->result_type))) + if (!isUInt8(removeLowCardinality(removeNullable(node->result_type)))) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, "Illegal type {} of column {} for filter. Must be UInt8 or Nullable(UInt8).", node->result_type->getName(), name); diff --git a/src/Processors/Transforms/FilterTransform.cpp b/src/Processors/Transforms/FilterTransform.cpp index 2f5f3d69cff..7d164cdff3d 100644 --- a/src/Processors/Transforms/FilterTransform.cpp +++ b/src/Processors/Transforms/FilterTransform.cpp @@ -3,10 +3,17 @@ #include #include #include +#include +#include namespace DB { +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER; +} + static void replaceFilterToConstant(Block & block, const String & filter_column_name) { ConstantFilterDescription constant_filter_description; @@ -36,6 +43,12 @@ Block FilterTransform::transformHeader( if (expression) header = expression->updateHeader(std::move(header)); + auto filter_type = header.getByName(filter_column_name).type; + if (!isUInt8(removeNullable(removeLowCardinality(filter_type)))) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, + "Illegal type {} of column {} for filter. Must be UInt8 or Nullable(UInt8).", + filter_type->getName(), filter_column_name); + if (remove_filter_column) header.erase(filter_column_name); else From 9003b7d7893d9d8596dd6215d7cb78a9c2d92599 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 23 Mar 2023 20:14:32 +0100 Subject: [PATCH 331/418] Fix tsan error lock-order-inversion --- src/Interpreters/ProcessList.cpp | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 5db39ece2e5..c7916f3eed2 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -434,11 +434,16 @@ void QueryStatus::addPipelineExecutor(PipelineExecutor * e) void QueryStatus::removePipelineExecutor(PipelineExecutor * e) { - std::lock_guard lock(executors_mutex); - auto it = std::find_if(executors.begin(), executors.end(), [e](const ExecutorHolderPtr & x){ return x->executor == e; }); - assert(it != executors.end()); + ExecutorHolderPtr executor_holder; + { + std::lock_guard lock(executors_mutex); + auto it = std::find_if(executors.begin(), executors.end(), [e](const ExecutorHolderPtr & x){ return x->executor == e; }); + assert(it != executors.end()); + executor_holder = *it; + } /// Invalidate executor pointer inside holder, but don't remove holder from the executors (to avoid race with cancelQuery) - (*it)->remove(); + /// We should do it with released executors_mutex to avoid possible lock order inversion. + executor_holder->remove(); } bool QueryStatus::checkTimeLimit() From c41c2cfb1d28c06498a9485f872cacd2df921020 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 23 Mar 2023 20:47:29 +0100 Subject: [PATCH 332/418] make doc more concise --- .../sparse-primary-indexes.md | 25 +++++-------------- 1 file changed, 6 insertions(+), 19 deletions(-) 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 e409a5295f0..51167521018 100644 --- a/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md +++ b/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md @@ -335,28 +335,15 @@ UserID.bin,URL.bin,和EventTime.bin是UserID 第一个(根据磁盘上的物理顺序)8192行(它们的列值)在逻辑上属于颗粒0,然后下一个8192行(它们的列值)属于颗粒1,以此类推。 -:::note -- 为什么上面和接下来的图片中显示的橙色数据与结论(1.最后一个颗粒我们标记的是最大的值;2.索引条目(索引标记)不是基于表中的特定行,而是基于颗粒。)不符合? -- 因为图中的数据是禁用了自适应索引粒度后得到的(中文文档中直接引用了英文官方文档的图片内容),默认情况下索引粒度是自适应的。而基于自适应索引粒度的表中的数据就会像上述结论中一样,下文将会分析和讲解这些问题。 - -- 下面分别是英文官方文档和版本更新网页中的描述信息,您可以开启翻译找到对应的描述。 -- https://clickhouse.com/docs/en/optimize/sparse-primary-indexes -- 我们在本指南开头的“DDL 语句详细信息”中提到,我们禁用了自适应索引粒度(为了简化本指南中的讨论,并使图表和结果可重现)。 -- 对于具有自适应索引粒度的表(默认情况下索引粒度是自适应的),某些粒度的大小可以小于 8192 行,具体取决于行数据大小。 - -- https://clickhouse.com/docs/en/whats-new/changelog/2019#experimental-features-1 -- ClickHouse Release 19.6.3.18, 2019-06-13 -- Experimental Features:实验性特性 -- 为合并树家族的表设置index_granularity_bytes(自适应索引粒度)。 - -- ClickHouse Release 19.10.1.5, 2019-07-12 -- Performance Improvement:优化改进 -- 添加在合并树列末尾写入最终标记的可能性。它可以避免对超出表数据范围的键进行无用的读取。仅当使用自适应索引粒度时,才会启用它。 -::: - :::note - 最后一个颗粒(1082颗粒)是少于8192行的。 +- 我们在本指南开头的“DDL 语句详细信息”中提到,我们禁用了自适应索引粒度(为了简化本指南中的讨论,并使图表和结果可重现)。 + + 因此,示例表中所有颗粒(除了最后一个)都具有相同大小。 + +- 对于具有自适应索引粒度的表(默认情况下索引粒度是自适应的),某些粒度的大小可以小于 8192 行,具体取决于行数据大小。 + - 我们将主键列(UserID, URL)中的一些列值标记为橙色。 这些橙色标记的列值是每个颗粒中每个主键列的最小值。这里的例外是最后一个颗粒(上图中的颗粒1082),最后一个颗粒我们标记的是最大的值。 From 6b3d0c1a34b5b74869e540663e0c50866f4e0d32 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 Mar 2023 19:44:03 +0000 Subject: [PATCH 333/418] Bump vectorscan to 5.4.9 As a consequence, this enables to remove a temporary workaround from ClickHouse's llvm-fork --- contrib/llvm-project | 2 +- contrib/vectorscan | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/llvm-project b/contrib/llvm-project index d80af319f5f..4bfaeb31dd0 160000 --- a/contrib/llvm-project +++ b/contrib/llvm-project @@ -1 +1 @@ -Subproject commit d80af319f5f047067b956b2fe93a6c00038c1e0d +Subproject commit 4bfaeb31dd0ef13f025221f93c138974a3e0a22a diff --git a/contrib/vectorscan b/contrib/vectorscan index f6250ae3e5a..b4bba94b1a2 160000 --- a/contrib/vectorscan +++ b/contrib/vectorscan @@ -1 +1 @@ -Subproject commit f6250ae3e5a3085000239313ad0689cc1e00cdc2 +Subproject commit b4bba94b1a250603b0b198e0394946e32f6c3f30 From 42725964d45d23fa4963787e37b9b16874dd0d55 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Thu, 23 Mar 2023 14:20:26 -0600 Subject: [PATCH 334/418] Remove links from nav categories --- .../example-datasets/_category_.yml | 7 -- docs/en/interfaces/overview.md | 2 +- docs/en/operations/utilities/index.md | 2 +- docs/en/sql-reference/data-types/index.md | 30 ++++- .../nested-data-structures/index.md | 106 +++++++++++++++++- .../nested-data-structures/nested.md | 105 ----------------- docs/en/sql-reference/statements/index.md | 4 +- docs/zh/faq/general.md | 2 +- docs/zh/sql-reference/functions/geo/index.mdx | 10 -- .../sql-reference/statements/alter/index.md | 2 +- .../sql-reference/statements/create/index.md | 11 -- 11 files changed, 133 insertions(+), 148 deletions(-) delete mode 100644 docs/en/getting-started/example-datasets/_category_.yml delete mode 100644 docs/en/sql-reference/data-types/nested-data-structures/nested.md delete mode 100644 docs/zh/sql-reference/functions/geo/index.mdx delete mode 100644 docs/zh/sql-reference/statements/create/index.md diff --git a/docs/en/getting-started/example-datasets/_category_.yml b/docs/en/getting-started/example-datasets/_category_.yml deleted file mode 100644 index 2ee34c63e93..00000000000 --- a/docs/en/getting-started/example-datasets/_category_.yml +++ /dev/null @@ -1,7 +0,0 @@ -position: 1 -label: 'Example Datasets' -collapsible: true -collapsed: true -link: - type: doc - id: en/getting-started/example-datasets/ diff --git a/docs/en/interfaces/overview.md b/docs/en/interfaces/overview.md index e5fa503e8fc..ee47e010f9e 100644 --- a/docs/en/interfaces/overview.md +++ b/docs/en/interfaces/overview.md @@ -6,7 +6,7 @@ keywords: [clickhouse, network, interfaces, http, tcp, grpc, command-line, clien description: ClickHouse provides three network interfaces --- -# Interfaces +# Drivers and Interfaces ClickHouse provides three network interfaces (they can be optionally wrapped in TLS for additional security): diff --git a/docs/en/operations/utilities/index.md b/docs/en/operations/utilities/index.md index b2f66af1084..bf6b3a63d23 100644 --- a/docs/en/operations/utilities/index.md +++ b/docs/en/operations/utilities/index.md @@ -1,7 +1,7 @@ --- slug: /en/operations/utilities/ sidebar_position: 56 -sidebar_label: Utilities +sidebar_label: List of tools and utilities pagination_next: 'en/operations/utilities/clickhouse-copier' --- diff --git a/docs/en/sql-reference/data-types/index.md b/docs/en/sql-reference/data-types/index.md index ef6a0fb3ea5..c61a3069db6 100644 --- a/docs/en/sql-reference/data-types/index.md +++ b/docs/en/sql-reference/data-types/index.md @@ -1,13 +1,33 @@ --- slug: /en/sql-reference/data-types/ -sidebar_label: Data Types +sidebar_label: List of data types sidebar_position: 37 --- -# Data Types +# ClickHouse Data Types -ClickHouse can store various kinds of data in table cells. +ClickHouse can store various kinds of data in table cells. This section describes the supported data types and special considerations for using and/or implementing them if any. -This section describes the supported data types and special considerations for using and/or implementing them if any. +:::note +You can check whether a data type name is case-sensitive in the [system.data_type_families](../../operations/system-tables/data_type_families.md#system_tables-data_type_families) table. +::: -You can check whether data type name is case-sensitive in the [system.data_type_families](../../operations/system-tables/data_type_families.md#system_tables-data_type_families) table. +ClickHouse data types include: + +- **Integer types**: [signed and unsigned integers](./int-uint.md) (`UInt8`, `UInt16`, `UInt32`, `UInt64`, `UInt128`, `UInt256`, `Int8`, `Int16`, `Int32`, `Int64`, `Int128`, `Int256`) +- **Floating-point numbers**: [floats](./float.md)(`Float32` and `Float64`) and [`Decimal` values](./decimal.md) +- **Boolean**: ClickHouse has a [`Boolean` type](./boolean.md) +- **Strings**: [`String`](./string.md) and [`FixedString`](./fixedstring.md) +- **Dates**: use [`Date`](./date.md) and [`Date32`](./date32.md) for days, and [`DateTime`](./datetime.md) and [`DateTime64`](./datetime64.md) for instances in time +- **JSON**: the [`JSON` object](./json.md) stores a JSON document in a single column +- **UUID**: a performant option for storing [`UUID` values](./uuid.md) +- **Low cardinality types**: use an [`Enum`](./enum.md) when you have a handful of unique values, or use [`LowCardinality`](./lowcardinality.md) when you have up to 10,000 unique values of a column +- **Arrays**: any column can be defined as an [`Array` of values](./array.md) +- **Maps**: use [`Map`](./map.md) for storing key/value pairs +- **Aggregation function types**: use [`SimpleAggregateFunction`](./simpleaggregatefunction.md) and [`AggregateFunction`](./aggregatefunction.md) for storing the intermediate status of aggregate function results +- **Nested data structures**: A [`Nested` data structure](./nested-data-structures/index.md) is like a table inside a cell +- **Tuples**: A [`Tuple` of elements](./tuple.md), each having an individual type. +- **Nullable**: [`Nullbale`](./nullable.md) allows you to store a value as `NULL` when a value is "missing" (instead of the column gettings its default value for the data type) +- **IP addresses**: use [`IPv4`](./domains/ipv4.md) and [`IPv6`](./domains/ipv6.md) to efficiently store IP addresses +- **Geo types**: for[ geographical data](./geo.md), including `Point`, `Ring`, `Polygon` and `MultiPolygon` +- **Special data types**: including [`Expression`](./special-data-types/expression.md), [`Set`](./special-data-types/set.md), [`Nothing`](./special-data-types/nothing.md) and [`Interval`](./special-data-types/interval.md) \ No newline at end of file diff --git a/docs/en/sql-reference/data-types/nested-data-structures/index.md b/docs/en/sql-reference/data-types/nested-data-structures/index.md index 1d958c018d8..d118170cd39 100644 --- a/docs/en/sql-reference/data-types/nested-data-structures/index.md +++ b/docs/en/sql-reference/data-types/nested-data-structures/index.md @@ -1,7 +1,105 @@ --- -slug: /en/sql-reference/data-types/nested-data-structures/ -sidebar_label: Nested Data Structures -sidebar_position: 54 +slug: /en/sql-reference/data-types/nested-data-structures/nested +sidebar_position: 57 +sidebar_label: Nested(Name1 Type1, Name2 Type2, ...) --- -# Nested Data Structures +# Nested + +## Nested(name1 Type1, Name2 Type2, …) + +A nested data structure is like a table inside a cell. The parameters of a nested data structure – the column names and types – are specified the same way as in a [CREATE TABLE](../../../sql-reference/statements/create/table.md) query. Each table row can correspond to any number of rows in a nested data structure. + +Example: + +``` sql +CREATE TABLE test.visits +( + CounterID UInt32, + StartDate Date, + Sign Int8, + IsNew UInt8, + VisitID UInt64, + UserID UInt64, + ... + Goals Nested + ( + ID UInt32, + Serial UInt32, + EventTime DateTime, + Price Int64, + OrderID String, + CurrencyID UInt32 + ), + ... +) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192, Sign) +``` + +This example declares the `Goals` nested data structure, which contains data about conversions (goals reached). Each row in the ‘visits’ table can correspond to zero or any number of conversions. + +When [flatten_nested](../../../operations/settings/settings.md#flatten-nested) is set to `0` (which is not by default), arbitrary levels of nesting are supported. + +In most cases, when working with a nested data structure, its columns are specified with column names separated by a dot. These columns make up an array of matching types. All the column arrays of a single nested data structure have the same length. + +Example: + +``` sql +SELECT + Goals.ID, + Goals.EventTime +FROM test.visits +WHERE CounterID = 101500 AND length(Goals.ID) < 5 +LIMIT 10 +``` + +``` text +┌─Goals.ID───────────────────────┬─Goals.EventTime───────────────────────────────────────────────────────────────────────────┐ +│ [1073752,591325,591325] │ ['2014-03-17 16:38:10','2014-03-17 16:38:48','2014-03-17 16:42:27'] │ +│ [1073752] │ ['2014-03-17 00:28:25'] │ +│ [1073752] │ ['2014-03-17 10:46:20'] │ +│ [1073752,591325,591325,591325] │ ['2014-03-17 13:59:20','2014-03-17 22:17:55','2014-03-17 22:18:07','2014-03-17 22:18:51'] │ +│ [] │ [] │ +│ [1073752,591325,591325] │ ['2014-03-17 11:37:06','2014-03-17 14:07:47','2014-03-17 14:36:21'] │ +│ [] │ [] │ +│ [] │ [] │ +│ [591325,1073752] │ ['2014-03-17 00:46:05','2014-03-17 00:46:05'] │ +│ [1073752,591325,591325,591325] │ ['2014-03-17 13:28:33','2014-03-17 13:30:26','2014-03-17 18:51:21','2014-03-17 18:51:45'] │ +└────────────────────────────────┴───────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +It is easiest to think of a nested data structure as a set of multiple column arrays of the same length. + +The only place where a SELECT query can specify the name of an entire nested data structure instead of individual columns is the ARRAY JOIN clause. For more information, see “ARRAY JOIN clause”. Example: + +``` sql +SELECT + Goal.ID, + Goal.EventTime +FROM test.visits +ARRAY JOIN Goals AS Goal +WHERE CounterID = 101500 AND length(Goals.ID) < 5 +LIMIT 10 +``` + +``` text +┌─Goal.ID─┬──────Goal.EventTime─┐ +│ 1073752 │ 2014-03-17 16:38:10 │ +│ 591325 │ 2014-03-17 16:38:48 │ +│ 591325 │ 2014-03-17 16:42:27 │ +│ 1073752 │ 2014-03-17 00:28:25 │ +│ 1073752 │ 2014-03-17 10:46:20 │ +│ 1073752 │ 2014-03-17 13:59:20 │ +│ 591325 │ 2014-03-17 22:17:55 │ +│ 591325 │ 2014-03-17 22:18:07 │ +│ 591325 │ 2014-03-17 22:18:51 │ +│ 1073752 │ 2014-03-17 11:37:06 │ +└─────────┴─────────────────────┘ +``` + +You can’t perform SELECT for an entire nested data structure. You can only explicitly list individual columns that are part of it. + +For an INSERT query, you should pass all the component column arrays of a nested data structure separately (as if they were individual column arrays). During insertion, the system checks that they have the same length. + +For a DESCRIBE query, the columns in a nested data structure are listed separately in the same way. + +The ALTER query for elements in a nested data structure has limitations. diff --git a/docs/en/sql-reference/data-types/nested-data-structures/nested.md b/docs/en/sql-reference/data-types/nested-data-structures/nested.md deleted file mode 100644 index d118170cd39..00000000000 --- a/docs/en/sql-reference/data-types/nested-data-structures/nested.md +++ /dev/null @@ -1,105 +0,0 @@ ---- -slug: /en/sql-reference/data-types/nested-data-structures/nested -sidebar_position: 57 -sidebar_label: Nested(Name1 Type1, Name2 Type2, ...) ---- - -# Nested - -## Nested(name1 Type1, Name2 Type2, …) - -A nested data structure is like a table inside a cell. The parameters of a nested data structure – the column names and types – are specified the same way as in a [CREATE TABLE](../../../sql-reference/statements/create/table.md) query. Each table row can correspond to any number of rows in a nested data structure. - -Example: - -``` sql -CREATE TABLE test.visits -( - CounterID UInt32, - StartDate Date, - Sign Int8, - IsNew UInt8, - VisitID UInt64, - UserID UInt64, - ... - Goals Nested - ( - ID UInt32, - Serial UInt32, - EventTime DateTime, - Price Int64, - OrderID String, - CurrencyID UInt32 - ), - ... -) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192, Sign) -``` - -This example declares the `Goals` nested data structure, which contains data about conversions (goals reached). Each row in the ‘visits’ table can correspond to zero or any number of conversions. - -When [flatten_nested](../../../operations/settings/settings.md#flatten-nested) is set to `0` (which is not by default), arbitrary levels of nesting are supported. - -In most cases, when working with a nested data structure, its columns are specified with column names separated by a dot. These columns make up an array of matching types. All the column arrays of a single nested data structure have the same length. - -Example: - -``` sql -SELECT - Goals.ID, - Goals.EventTime -FROM test.visits -WHERE CounterID = 101500 AND length(Goals.ID) < 5 -LIMIT 10 -``` - -``` text -┌─Goals.ID───────────────────────┬─Goals.EventTime───────────────────────────────────────────────────────────────────────────┐ -│ [1073752,591325,591325] │ ['2014-03-17 16:38:10','2014-03-17 16:38:48','2014-03-17 16:42:27'] │ -│ [1073752] │ ['2014-03-17 00:28:25'] │ -│ [1073752] │ ['2014-03-17 10:46:20'] │ -│ [1073752,591325,591325,591325] │ ['2014-03-17 13:59:20','2014-03-17 22:17:55','2014-03-17 22:18:07','2014-03-17 22:18:51'] │ -│ [] │ [] │ -│ [1073752,591325,591325] │ ['2014-03-17 11:37:06','2014-03-17 14:07:47','2014-03-17 14:36:21'] │ -│ [] │ [] │ -│ [] │ [] │ -│ [591325,1073752] │ ['2014-03-17 00:46:05','2014-03-17 00:46:05'] │ -│ [1073752,591325,591325,591325] │ ['2014-03-17 13:28:33','2014-03-17 13:30:26','2014-03-17 18:51:21','2014-03-17 18:51:45'] │ -└────────────────────────────────┴───────────────────────────────────────────────────────────────────────────────────────────┘ -``` - -It is easiest to think of a nested data structure as a set of multiple column arrays of the same length. - -The only place where a SELECT query can specify the name of an entire nested data structure instead of individual columns is the ARRAY JOIN clause. For more information, see “ARRAY JOIN clause”. Example: - -``` sql -SELECT - Goal.ID, - Goal.EventTime -FROM test.visits -ARRAY JOIN Goals AS Goal -WHERE CounterID = 101500 AND length(Goals.ID) < 5 -LIMIT 10 -``` - -``` text -┌─Goal.ID─┬──────Goal.EventTime─┐ -│ 1073752 │ 2014-03-17 16:38:10 │ -│ 591325 │ 2014-03-17 16:38:48 │ -│ 591325 │ 2014-03-17 16:42:27 │ -│ 1073752 │ 2014-03-17 00:28:25 │ -│ 1073752 │ 2014-03-17 10:46:20 │ -│ 1073752 │ 2014-03-17 13:59:20 │ -│ 591325 │ 2014-03-17 22:17:55 │ -│ 591325 │ 2014-03-17 22:18:07 │ -│ 591325 │ 2014-03-17 22:18:51 │ -│ 1073752 │ 2014-03-17 11:37:06 │ -└─────────┴─────────────────────┘ -``` - -You can’t perform SELECT for an entire nested data structure. You can only explicitly list individual columns that are part of it. - -For an INSERT query, you should pass all the component column arrays of a nested data structure separately (as if they were individual column arrays). During insertion, the system checks that they have the same length. - -For a DESCRIBE query, the columns in a nested data structure are listed separately in the same way. - -The ALTER query for elements in a nested data structure has limitations. diff --git a/docs/en/sql-reference/statements/index.md b/docs/en/sql-reference/statements/index.md index b286d8c932d..100b8889aaa 100644 --- a/docs/en/sql-reference/statements/index.md +++ b/docs/en/sql-reference/statements/index.md @@ -1,10 +1,10 @@ --- slug: /en/sql-reference/statements/ sidebar_position: 1 -sidebar_label: Statements +sidebar_label: List of statements --- -# ClickHouse SQL Statements +# ClickHouse SQL Statements Statements represent various kinds of action you can perform using SQL queries. Each kind of statement has it’s own syntax and usage details that are described separately: diff --git a/docs/zh/faq/general.md b/docs/zh/faq/general.md index 5a95b9aad07..530be7f08d8 100644 --- a/docs/zh/faq/general.md +++ b/docs/zh/faq/general.md @@ -1,5 +1,5 @@ --- -slug: /zh/faq/general +slug: /zh/faq/general/overview --- # 常见问题 {#chang-jian-wen-ti} diff --git a/docs/zh/sql-reference/functions/geo/index.mdx b/docs/zh/sql-reference/functions/geo/index.mdx deleted file mode 100644 index fcfc4bd4717..00000000000 --- a/docs/zh/sql-reference/functions/geo/index.mdx +++ /dev/null @@ -1,10 +0,0 @@ ---- -slug: /zh/sql-reference/functions/geo/ -sidebar_label: Geo -sidebar_position: 62 -title: "Geo Functions" ---- - -import Content from '@site/docs/en/sql-reference/functions/geo/index.md'; - - diff --git a/docs/zh/sql-reference/statements/alter/index.md b/docs/zh/sql-reference/statements/alter/index.md index 8320b207725..e173837a16c 100644 --- a/docs/zh/sql-reference/statements/alter/index.md +++ b/docs/zh/sql-reference/statements/alter/index.md @@ -1,5 +1,5 @@ --- -slug: /zh/sql-reference/statements/alter/ +slug: /zh/sql-reference/statements/alter/overview sidebar_position: 35 sidebar_label: ALTER --- diff --git a/docs/zh/sql-reference/statements/create/index.md b/docs/zh/sql-reference/statements/create/index.md deleted file mode 100644 index f63ed0a7acd..00000000000 --- a/docs/zh/sql-reference/statements/create/index.md +++ /dev/null @@ -1,11 +0,0 @@ ---- -slug: /zh/sql-reference/statements/create/ -sidebar_label: CREATE -sidebar_position: 34 ---- - -# CREATE语法 {#create-queries} - -CREATE语法包含以下子集: - -- [DATABASE](../../../sql-reference/statements/create/database.md) From c8e7ca7c8c956a7f96871313fe64dfb75c5def39 Mon Sep 17 00:00:00 2001 From: Rich Raposa Date: Thu, 23 Mar 2023 14:34:29 -0600 Subject: [PATCH 335/418] Update backup.md --- docs/en/operations/backup.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index 69eb782868a..d58dd1376eb 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -331,7 +331,7 @@ It is also possible to `BACKUP`/`RESTORE` to S3 by configuring an S3 disk in the

      - s3 + s3_plain
      From 747eaf3f7222496d83cfa3840f19d20ba5036b75 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Thu, 23 Mar 2023 14:40:35 -0600 Subject: [PATCH 336/418] Fix broken links --- docs/en/interfaces/formats.md | 8 ++++---- docs/en/operations/settings/settings-formats.md | 8 ++++---- docs/en/operations/settings/settings.md | 2 +- docs/en/sql-reference/statements/describe-table.md | 4 ++-- docs/en/sql-reference/statements/select/array-join.md | 2 +- 5 files changed, 12 insertions(+), 12 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 3debea0087e..ae3756d5d41 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -154,7 +154,7 @@ Arrays are written as a list of comma-separated values in square brackets. Numbe In input data, ENUM values can be represented as names or as ids. First, we try to match the input value to the ENUM name. If we fail and the input value is a number, we try to match this number to ENUM id. If input data contains only ENUM ids, it's recommended to enable the setting [input_format_tsv_enum_as_number](/docs/en/operations/settings/settings-formats.md/#input_format_tsv_enum_as_number) to optimize ENUM parsing. -Each element of [Nested](/docs/en/sql-reference/data-types/nested-data-structures/nested.md) structures is represented as an array. +Each element of [Nested](/docs/en/sql-reference/data-types/nested-data-structures/index.md) structures is represented as an array. For example: @@ -1150,7 +1150,7 @@ Any set of bytes can be output in the strings. Use the `JSONEachRow` format if y ### Usage of Nested Structures {#jsoneachrow-nested} -If you have a table with [Nested](/docs/en/sql-reference/data-types/nested-data-structures/nested.md) data type columns, you can insert JSON data with the same structure. Enable this feature with the [input_format_import_nested_json](/docs/en/operations/settings/settings-formats.md/#input_format_import_nested_json) setting. +If you have a table with [Nested](/docs/en/sql-reference/data-types/nested-data-structures/index.md) data type columns, you can insert JSON data with the same structure. Enable this feature with the [input_format_import_nested_json](/docs/en/operations/settings/settings-formats.md/#input_format_import_nested_json) setting. For example, consider the following table: @@ -1776,7 +1776,7 @@ message MessageType { ``` ClickHouse tries to find a column named `x.y.z` (or `x_y_z` or `X.y_Z` and so on). -Nested messages are suitable to input or output a [nested data structures](/docs/en/sql-reference/data-types/nested-data-structures/nested.md). +Nested messages are suitable to input or output a [nested data structures](/docs/en/sql-reference/data-types/nested-data-structures/index.md). Default values defined in a protobuf schema like this @@ -1978,7 +1978,7 @@ To exchange data with Hadoop, you can use [HDFS table engine](/docs/en/engines/t - [output_format_parquet_row_group_size](/docs/en/operations/settings/settings-formats.md/#output_format_parquet_row_group_size) - row group size in rows while data output. Default value - `1000000`. - [output_format_parquet_string_as_string](/docs/en/operations/settings/settings-formats.md/#output_format_parquet_string_as_string) - use Parquet String type instead of Binary for String columns. Default value - `false`. -- [input_format_parquet_import_nested](/docs/en/operations/settings/settings-formats.md/#input_format_parquet_import_nested) - allow inserting array of structs into [Nested](/docs/en/sql-reference/data-types/nested-data-structures/nested.md) table in Parquet input format. Default value - `false`. +- [input_format_parquet_import_nested](/docs/en/operations/settings/settings-formats.md/#input_format_parquet_import_nested) - allow inserting array of structs into [Nested](/docs/en/sql-reference/data-types/nested-data-structures/index.md) table in Parquet input format. Default value - `false`. - [input_format_parquet_case_insensitive_column_matching](/docs/en/operations/settings/settings-formats.md/#input_format_parquet_case_insensitive_column_matching) - ignore case when matching Parquet columns with ClickHouse columns. Default value - `false`. - [input_format_parquet_allow_missing_columns](/docs/en/operations/settings/settings-formats.md/#input_format_parquet_allow_missing_columns) - allow missing columns while reading Parquet data. Default value - `false`. - [input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference](/docs/en/operations/settings/settings-formats.md/#input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference) - allow skipping columns with unsupported types while schema inference for Parquet format. Default value - `false`. diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 172627c7c3e..aa991cd9f15 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -964,7 +964,7 @@ Default value: 1. ### input_format_arrow_import_nested {#input_format_arrow_import_nested} -Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) columns as an array of structs in [Arrow](../../interfaces/formats.md/#data_types-matching-arrow) input format. +Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/index.md) columns as an array of structs in [Arrow](../../interfaces/formats.md/#data_types-matching-arrow) input format. Possible values: @@ -1024,7 +1024,7 @@ Default value: `none`. ### input_format_orc_import_nested {#input_format_orc_import_nested} -Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) columns as an array of structs in [ORC](../../interfaces/formats.md/#data-format-orc) input format. +Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/index.md) columns as an array of structs in [ORC](../../interfaces/formats.md/#data-format-orc) input format. Possible values: @@ -1073,7 +1073,7 @@ Default value: `none`. ### input_format_parquet_import_nested {#input_format_parquet_import_nested} -Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) columns as an array of structs in [Parquet](../../interfaces/formats.md/#data-format-parquet) input format. +Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/index.md) columns as an array of structs in [Parquet](../../interfaces/formats.md/#data-format-parquet) input format. Possible values: @@ -1538,6 +1538,6 @@ Default value: `1GiB`. ### input_format_native_allow_types_conversion {#input_format_native_allow_types_conversion} -Allow types conversion in Native input format between columns from input data and requested columns. +Allow types conversion in Native input format between columns from input data and requested columns. Enabled by default. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index ef630d3169d..f3c0f20f3a6 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3438,7 +3438,7 @@ Default value: `throw`. ## flatten_nested {#flatten-nested} -Sets the data format of a [nested](../../sql-reference/data-types/nested-data-structures/nested.md) columns. +Sets the data format of a [nested](../../sql-reference/data-types/nested-data-structures/index.md) columns. Possible values: diff --git a/docs/en/sql-reference/statements/describe-table.md b/docs/en/sql-reference/statements/describe-table.md index 4864743abbc..b9190107127 100644 --- a/docs/en/sql-reference/statements/describe-table.md +++ b/docs/en/sql-reference/statements/describe-table.md @@ -24,9 +24,9 @@ The `DESCRIBE` statement returns a row for each table column with the following - `ttl_expression` — A [TTL](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl) expression. - `is_subcolumn` — A flag that equals `1` for internal subcolumns. It is included into the result only if subcolumn description is enabled by the [describe_include_subcolumns](../../operations/settings/settings.md#describe_include_subcolumns) setting. -All columns in [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) data structures are described separately. The name of each column is prefixed with a parent column name and a dot. +All columns in [Nested](../../sql-reference/data-types/nested-data-structures/index.md) data structures are described separately. The name of each column is prefixed with a parent column name and a dot. -To show internal subcolumns of other data types, use the [describe_include_subcolumns](../../operations/settings/settings.md#describe_include_subcolumns) setting. +To show internal subcolumns of other data types, use the [describe_include_subcolumns](../../operations/settings/settings.md#describe_include_subcolumns) setting. **Example** diff --git a/docs/en/sql-reference/statements/select/array-join.md b/docs/en/sql-reference/statements/select/array-join.md index b8e6be24798..3d88a0f9b7a 100644 --- a/docs/en/sql-reference/statements/select/array-join.md +++ b/docs/en/sql-reference/statements/select/array-join.md @@ -185,7 +185,7 @@ SETTINGS enable_unaligned_array_join = 1; ## ARRAY JOIN with Nested Data Structure -`ARRAY JOIN` also works with [nested data structures](../../../sql-reference/data-types/nested-data-structures/nested.md): +`ARRAY JOIN` also works with [nested data structures](../../../sql-reference/data-types/nested-data-structures/index.md): ``` sql CREATE TABLE nested_test From a4771812cb64c8d8a515ba5ee6ae4121439a41f1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 23 Mar 2023 20:46:42 +0000 Subject: [PATCH 337/418] Fixing build. --- src/Processors/QueryPlan/ReadFromMergeTree.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 73c78e79509..5e4ba117967 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -152,7 +152,6 @@ public: 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 PrewhereInfoPtr & getPrewhereInfo() const { return prewhere_info; } /// Returns `false` if requested reading cannot be performed. @@ -174,7 +173,6 @@ public: 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; } From e27d47581e742844a9bf0f1093656a0baa746a80 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Thu, 23 Mar 2023 14:56:18 -0600 Subject: [PATCH 338/418] Link fixes --- docs/zh/engines/database-engines/materialized-mysql.md | 2 +- docs/zh/sql-reference/statements/index.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/zh/engines/database-engines/materialized-mysql.md b/docs/zh/engines/database-engines/materialized-mysql.md index 4cc4ae58840..5c735556c48 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.md), [RENAME](../../sql-reference/statements/rename.md)). 如果ClickHouse 无法解析某些语句DDL 操作,则会跳过。 +MySQL DDL 语句会被转换成对应的ClickHouse DDL 语句,比如: ([ALTER](../../sql-reference/statements/alter/index.md), [CREATE](../../sql-reference/statements/create.md), [DROP](../../sql-reference/statements/drop.md), [RENAME](../../sql-reference/statements/rename.md)). 如果ClickHouse 无法解析某些语句DDL 操作,则会跳过。 ### 数据复制 {#data-replication} diff --git a/docs/zh/sql-reference/statements/index.md b/docs/zh/sql-reference/statements/index.md index 989c368ebc4..2fdfeb1786f 100644 --- a/docs/zh/sql-reference/statements/index.md +++ b/docs/zh/sql-reference/statements/index.md @@ -10,7 +10,7 @@ sidebar_position: 31 - [SELECT](../../sql-reference/statements/select/index.md) - [INSERT INTO](../../sql-reference/statements/insert-into.md) -- [CREATE](../../sql-reference/statements/create/index.md) +- [CREATE](../../sql-reference/statements/create.md) - [ALTER](../../sql-reference/statements/alter/index.md) - [SYSTEM](../../sql-reference/statements/system.md) - [SHOW](../../sql-reference/statements/show.md) From 9571130b268d8783599d93938d856b6b5cae0dca Mon Sep 17 00:00:00 2001 From: rfraposa Date: Thu, 23 Mar 2023 15:08:31 -0600 Subject: [PATCH 339/418] Update materialize-mysql.md --- docs/zh/engines/database-engines/materialize-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/engines/database-engines/materialize-mysql.md b/docs/zh/engines/database-engines/materialize-mysql.md index b7ee3a038b8..5d1394f9456 100644 --- a/docs/zh/engines/database-engines/materialize-mysql.md +++ b/docs/zh/engines/database-engines/materialize-mysql.md @@ -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.md), [RENAME](../../sql-reference/statements/rename.md))。如果ClickHouse无法解析某个DDL查询,则该查询将被忽略。 +MySQL DDL查询转换为相应的ClickHouse DDL查询([ALTER](../../sql-reference/statements/alter/index.md), [CREATE](../../sql-reference/statements/create.md), [DROP](../../sql-reference/statements/drop.md), [RENAME](../../sql-reference/statements/rename.md))。如果ClickHouse无法解析某个DDL查询,则该查询将被忽略。 ### Data Replication {#data-replication} From 81bef1cb4b78d5f30f41a3a13b784f900e69bd81 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 23 Mar 2023 21:18:30 +0000 Subject: [PATCH 340/418] Fixing build. --- src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index 6367edafce8..f749ea7b2a2 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -106,7 +106,7 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs & 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()) { if (prewhere_info->row_level_filter) { From bd02627869297da311e8a2ca1183a03d366fbd0d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 20 Mar 2023 19:00:58 +0000 Subject: [PATCH 341/418] Update docs for parseDateTime() (follow-up to #46815) --- .../functions/date-time-functions.md | 7 +- .../functions/type-conversion-functions.md | 139 ++++++++++-------- src/Functions/formatDateTime.cpp | 4 +- 3 files changed, 82 insertions(+), 68 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index f216c1003b6..77ce47c1b6d 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1238,6 +1238,8 @@ Formats a Time according to the given Format string. Format is a constant expres formatDateTime uses MySQL datetime format style, refer to https://dev.mysql.com/doc/refman/8.0/en/date-and-time-functions.html#function_date-format. +The opposite operation of this function is [formatDateTime](/docs/en/sql-reference/functions/type-conversion-functions.md#formatdatetime). + Alias: `DATE_FORMAT`. **Syntax** @@ -1257,7 +1259,7 @@ Using replacement fields, you can define a pattern for the resulting string. “ |----------|---------------------------------------------------------|------------| | %a | abbreviated weekday name (Mon-Sun) | Mon | | %b | abbreviated month name (Jan-Dec) | Jan | -| %c | month as a decimal number (01-12) | 01 | +| %c | month as an integer number (01-12) | 01 | | %C | year divided by 100 and truncated to integer (00-99) | 20 | | %d | day of the month, zero-padded (01-31) | 02 | | %D | Short MM/DD/YY date, equivalent to %m/%d/%y | 01/02/18 | @@ -1273,7 +1275,7 @@ Using replacement fields, you can define a pattern for the resulting string. “ | %j | day of the year (001-366) | 002 | | %k | hour in 24h format (00-23) | 22 | | %l | hour in 12h format (01-12) | 09 | -| %m | month as a decimal number (01-12) | 01 | +| %m | month as an integer number (01-12) | 01 | | %M | minute (00-59) | 33 | | %n | new-line character (‘’) | | | %p | AM or PM designation | PM | @@ -1332,6 +1334,7 @@ Result: 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. +The opposite operation of this function is [formatDateTimeInJodaSyntax](/docs/en/sql-reference/functions/type-conversion-functions.md#formatdatetimeinjodasyntax). **Replacement fields** diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 5ece8bacdac..878e97e8712 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1148,6 +1148,81 @@ Result: └───────────────────────────┴──────────────────────────────┘ ``` +## parseDateTime + +Converts a [String](/docs/en/sql-reference/data-types/string.md) to [DateTime](/docs/en/sql-reference/data-types/datetime.md) according to a [MySQL format string](https://dev.mysql.com/doc/refman/8.0/en/date-and-time-functions.html#function_date-format). +This function is the opposite operation of function [formatDateTime](/docs/en/sql-reference/functions/date-time-functions.md#date_time_functions-formatDateTime). + +**Syntax** + +``` sql +parseDateTime(str, format[, timezone]) +``` + +**Arguments** + +- `str` — the String to be parsed +- `format` — the format string + +**Returned value(s)** + +Returns DateTime values parsed from input string according to a MySQL style format string. + +**Supported format specifiers** + +All format specifiers listed in [formatDateTime](/docs/en/sql-reference/functions/date-time-functions.md#date_time_functions-formatDateTime) 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 │ +└───────────────────────────────────────────────────────────┘ +``` + +Alias: `TO_TIMESTAMP`. + +## parseDateTimeInJodaSyntax + +Similar to [parseDateTime](#parsedatetime), except that the format string is in [Joda](https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html) instead of MySQL syntax. +This function is the opposite operation of function [formatDateTimeInJodaSyntax](/docs/en/sql-reference/functions/date-time-functions.md#date_time_functions-formatDateTimeInJodaSyntax). + +**Syntax** + +``` sql +parseDateTimeInJodaSyntax(str, format[, timezone]) +``` + +**Arguments** + +- `str` — the String to be parsed +- `format` — the format string + +**Returned value(s)** + +Returns DateTime values parsed from input string according to a Joda style format. + +**Supported format specifiers** + +All format specifiers listed in [formatDateTimeInJoda](/docs/en/sql-reference/functions/date-time-functions.md#date_time_functions-formatDateTime) are 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 │ +└─────────────────────────────────────────────────────────────────────────────────────────┘ +``` + ## parseDateTimeBestEffort ## parseDateTime32BestEffort @@ -1351,70 +1426,6 @@ 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_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 Converts input parameter to the [LowCardinality](/docs/en/sql-reference/data-types/lowcardinality.md) version of same data type. diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index dbcc88eb17c..bbb4c3ba5b0 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -926,7 +926,7 @@ public: out_template += "Jan"; break; - // Month as a decimal number (01-12) + // Month as a integer number (01-12) case 'c': instructions.emplace_back(&Instruction::mysqlMonth); out_template += "00"; @@ -989,7 +989,7 @@ public: out_template += "000"; break; - // Month as a decimal number (01-12) + // Month as a integer number (01-12) case 'm': instructions.emplace_back(&Instruction::mysqlMonth); out_template += "00"; From 4d281245fbafcc1f23d65a029044f0b55ac0bfaa Mon Sep 17 00:00:00 2001 From: rfraposa Date: Thu, 23 Mar 2023 15:43:40 -0600 Subject: [PATCH 342/418] Update index.md --- docs/zh/faq/general/index.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/zh/faq/general/index.md b/docs/zh/faq/general/index.md index 8b0b42cede2..d0e855285d7 100644 --- a/docs/zh/faq/general/index.md +++ b/docs/zh/faq/general/index.md @@ -23,6 +23,6 @@ sidebar_label: General !!! info "没找到您需要的内容?" - 请查阅 [其他 F.A.Q. 类别](../../faq/) 或者从左侧导航栏浏览其他文档 - + 请查阅 [其他 F.A.Q. 类别](../../faq/index.md) 或者从左侧导航栏浏览其他文档 + {## [原始文档](https://clickhouse.com/docs/en/faq/general/) ##} From 128753f7ab67b64d31365b0c0965dd85ec59bb94 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Thu, 23 Mar 2023 16:01:20 -0600 Subject: [PATCH 343/418] Update index.md --- docs/zh/faq/general/index.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/zh/faq/general/index.md b/docs/zh/faq/general/index.md index d0e855285d7..9693e7ffc82 100644 --- a/docs/zh/faq/general/index.md +++ b/docs/zh/faq/general/index.md @@ -21,7 +21,6 @@ sidebar_label: General - [我如何为 ClickHouse贡献代码?](../../faq/general/how-do-i-contribute-code-to-clickhouse.md) - !!! info "没找到您需要的内容?" 请查阅 [其他 F.A.Q. 类别](../../faq/index.md) 或者从左侧导航栏浏览其他文档 From 8cf031ee2e2285650cd67704cd4215ac42dc665e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 Mar 2023 22:36:38 +0000 Subject: [PATCH 344/418] Docs: Update secondary index example Fixes: #47923 --- .../table-engines/mergetree-family/mergetree.md | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 9fea158b100..f1b7a40094d 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -377,8 +377,9 @@ CREATE TABLE table_name i32 Int32, s String, ... - INDEX a (u64 * i32, s) TYPE minmax GRANULARITY 3, - INDEX b (u64 * length(s)) TYPE set(1000) GRANULARITY 4 + INDEX idx1 u64 TYPE bloom_filter GRANULARITY 3, + INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 3, + INDEX idx3 u64 * length(s) TYPE set(1000) GRANULARITY 4 ) ENGINE = MergeTree() ... ``` @@ -386,8 +387,9 @@ CREATE TABLE table_name Indices from the example can be used by ClickHouse to reduce the amount of data to read from disk in the following queries: ``` sql -SELECT count() FROM table WHERE s < 'z' -SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234 +SELECT count() FROM table WHERE u64 == 10; +SELECT count() FROM table WHERE u64 * i32 >= 1234 +SELECT count() FROM table WHERE u64 * length(s) == 1234 ``` Data skipping indexes can also be created on composite columns: From fd1d8d5c55fcd1afd48b4195aa68b31050affaaa Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 23 Mar 2023 19:00:20 -0400 Subject: [PATCH 345/418] Third attempt to fix crash. --- src/Functions/geometryConverters.h | 15 ++++++++++++++- ...25401_polygons_sym_difference_rollup.reference | 3 +++ .../25401_polygons_sym_difference_rollup.sql | 3 ++- 3 files changed, 19 insertions(+), 2 deletions(-) diff --git a/src/Functions/geometryConverters.h b/src/Functions/geometryConverters.h index 652c98d70d4..d6f769ae4a8 100644 --- a/src/Functions/geometryConverters.h +++ b/src/Functions/geometryConverters.h @@ -120,10 +120,23 @@ struct ColumnToPolygonsConverter std::vector> answer(offsets.size()); auto all_rings = ColumnToRingsConverter::convert(typeid_cast(*col).getDataPtr()); + // [1, 2, 3] + // [[1], + // [2, 3], + // []] + + // [1, 3, 3] + size_t prev_offset = 0; - for (size_t iter = 0; iter < offsets.size() && iter < all_rings.size(); ++iter) + for (size_t iter = 0; iter < offsets.size(); ++iter) { const auto current_array_size = offsets[iter] - prev_offset; + if (current_array_size == 0) + { + answer.emplace_back(); + continue; + } + answer[iter].outer() = std::move(all_rings[prev_offset]); answer[iter].inners().reserve(current_array_size); for (size_t inner_holes = prev_offset + 1; inner_holes < offsets[iter]; ++inner_holes) diff --git a/tests/queries/0_stateless/25401_polygons_sym_difference_rollup.reference b/tests/queries/0_stateless/25401_polygons_sym_difference_rollup.reference index 2323f050607..346025b277b 100644 --- a/tests/queries/0_stateless/25401_polygons_sym_difference_rollup.reference +++ b/tests/queries/0_stateless/25401_polygons_sym_difference_rollup.reference @@ -4,3 +4,6 @@ [[(2147483647,0),(10.0001,65535),(1,255),(1023,2147483646)]] [] [[[(100.0001,1000.0001),(1000.0001,1.1920928955078125e-7),(20,-20),(20,20),(10,10),(-20,20),(100.0001,1000.0001)]]] [[[(100.0001,1000.0001),(1000.0001,1.1920928955078125e-7),(20,-20),(20,20),(10,10),(-20,20),(100.0001,1000.0001)]]] +[(9223372036854775807,1.1754943508222875e-38)] [[(1,1.0001)]] \N [] + +[(9223372036854775807,1.1754943508222875e-38)] [] \N [] diff --git a/tests/queries/0_stateless/25401_polygons_sym_difference_rollup.sql b/tests/queries/0_stateless/25401_polygons_sym_difference_rollup.sql index df8625d5915..8b9b63f7996 100644 --- a/tests/queries/0_stateless/25401_polygons_sym_difference_rollup.sql +++ b/tests/queries/0_stateless/25401_polygons_sym_difference_rollup.sql @@ -1,4 +1,5 @@ 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; -SELECT polygonsSymDifferenceCartesian([[[(100.0001, 1000.0001), (-20., 20.), (10., 10.), (20., 20.), (20., -20.), (1000.0001, 1.1920928955078125e-7)]],[[(0.0001, 100000000000000000000.)]] AS x],[x]) GROUP BY x WITH ROLLUP; \ No newline at end of file +SELECT polygonsSymDifferenceCartesian([[[(100.0001, 1000.0001), (-20., 20.), (10., 10.), (20., 20.), (20., -20.), (1000.0001, 1.1920928955078125e-7)]],[[(0.0001, 100000000000000000000.)]] AS x],[x]) GROUP BY x WITH ROLLUP; +SELECT [(9223372036854775807, 1.1754943508222875e-38)], x, NULL, polygonsSymDifferenceCartesian([[[(1.1754943508222875e-38, 1.1920928955078125e-7), (0.5, 0.5)]], [[(1.1754943508222875e-38, 1.1920928955078125e-7), (1.1754943508222875e-38, 1.1920928955078125e-7)], [(0., 1.0001)]], [[(1., 1.0001)]] AS x], [[[(3.4028234663852886e38, 0.9999)]]]) GROUP BY GROUPING SETS ((x)) WITH TOTALS From 3b9213fb5ae7acd725cec66a2e8884f3a4c07be7 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 23 Mar 2023 16:44:10 +0100 Subject: [PATCH 346/418] Use variables in URLs, migrate to RPs prefix --- .../test/performance-comparison/download.sh | 7 +++++-- .../test/performance-comparison/entrypoint.sh | 19 +++++++++++++------ 2 files changed, 18 insertions(+), 8 deletions(-) diff --git a/docker/test/performance-comparison/download.sh b/docker/test/performance-comparison/download.sh index e3df98c7da1..aee11030068 100755 --- a/docker/test/performance-comparison/download.sh +++ b/docker/test/performance-comparison/download.sh @@ -3,7 +3,9 @@ set -ex set -o pipefail trap "exit" INT TERM trap 'kill $(jobs -pr) ||:' EXIT +S3_URL=${S3_URL:="https://clickhouse-builds.s3.amazonaws.com"} BUILD_NAME=${BUILD_NAME:-package_release} +export S3_URL BUILD_NAME mkdir db0 ||: mkdir left ||: @@ -28,8 +30,9 @@ function download # Historically there were various paths for the performance test package. # Test all of them. declare -a urls_to_try=( - "https://s3.amazonaws.com/clickhouse-builds/$left_pr/$left_sha/$BUILD_NAME/performance.tar.zst" - "https://s3.amazonaws.com/clickhouse-builds/$left_pr/$left_sha/$BUILD_NAME/performance.tgz" + "$S3_URL/PRs/$left_pr/$left_sha/$BUILD_NAME/performance.tar.zst" + "$S3_URL/$left_pr/$left_sha/$BUILD_NAME/performance.tar.zst" + "$S3_URL/$left_pr/$left_sha/$BUILD_NAME/performance.tgz" ) for path in "${urls_to_try[@]}" diff --git a/docker/test/performance-comparison/entrypoint.sh b/docker/test/performance-comparison/entrypoint.sh index a4d93295eb9..74571777be0 100755 --- a/docker/test/performance-comparison/entrypoint.sh +++ b/docker/test/performance-comparison/entrypoint.sh @@ -6,6 +6,7 @@ export CHPC_CHECK_START_TIMESTAMP S3_URL=${S3_URL:="https://clickhouse-builds.s3.amazonaws.com"} BUILD_NAME=${BUILD_NAME:-package_release} +export S3_URL BUILD_NAME # Sometimes AWS responde with DNS error and it's impossible to retry it with # current curl version options. @@ -61,8 +62,9 @@ function find_reference_sha # test all of them. unset found declare -a urls_to_try=( - "https://s3.amazonaws.com/clickhouse-builds/0/$REF_SHA/$BUILD_NAME/performance.tar.zst" - "https://s3.amazonaws.com/clickhouse-builds/0/$REF_SHA/$BUILD_NAME/performance.tgz" + "$S3_URL/PRs/0/$REF_SHA/$BUILD_NAME/performance.tar.zst" + "$S3_URL/0/$REF_SHA/$BUILD_NAME/performance.tar.zst" + "$S3_URL/0/$REF_SHA/$BUILD_NAME/performance.tgz" ) for path in "${urls_to_try[@]}" do @@ -87,10 +89,15 @@ chmod 777 workspace output cd workspace # Download the package for the version we are going to test. -if curl_with_retry "$S3_URL/$PR_TO_TEST/$SHA_TO_TEST/$BUILD_NAME/performance.tar.zst" -then - right_path="$S3_URL/$PR_TO_TEST/$SHA_TO_TEST/$BUILD_NAME/performance.tar.zst" -fi +# A temporary solution for migrating into PRs directory +for prefix in "$S3_URL/PRs" "$S3_URL"; +do + if curl_with_retry "$prefix/$PR_TO_TEST/$SHA_TO_TEST/$BUILD_NAME/performance.tar.zst" + then + right_path="$prefix/$PR_TO_TEST/$SHA_TO_TEST/$BUILD_NAME/performance.tar.zst" + break + fi +done mkdir right wget -nv -nd -c "$right_path" -O- | tar -C right --no-same-owner --strip-components=1 --zstd --extract --verbose From a9089da350c80e0721b5c9492de71a8f530db5ea Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 23 Mar 2023 20:36:33 -0400 Subject: [PATCH 347/418] Remove debug comment --- src/Functions/geometryConverters.h | 7 ------- 1 file changed, 7 deletions(-) diff --git a/src/Functions/geometryConverters.h b/src/Functions/geometryConverters.h index d6f769ae4a8..85e887dd498 100644 --- a/src/Functions/geometryConverters.h +++ b/src/Functions/geometryConverters.h @@ -120,13 +120,6 @@ struct ColumnToPolygonsConverter std::vector> answer(offsets.size()); auto all_rings = ColumnToRingsConverter::convert(typeid_cast(*col).getDataPtr()); - // [1, 2, 3] - // [[1], - // [2, 3], - // []] - - // [1, 3, 3] - size_t prev_offset = 0; for (size_t iter = 0; iter < offsets.size(); ++iter) { From cd7d1fb9905ed3e010c0600cf252a6510f4a339c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 24 Mar 2023 04:35:50 +0300 Subject: [PATCH 348/418] Revert "Revert "Revert "Backup_Restore_concurrency_check_node""" --- src/Backups/BackupCoordinationRemote.cpp | 9 ++-- .../test_disallow_concurrency.py | 45 +++++++------------ 2 files changed, 19 insertions(+), 35 deletions(-) diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index e6399cafb8f..e7ea9becd24 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -810,12 +810,9 @@ bool BackupCoordinationRemote::hasConcurrentBackups(const std::atomic &) if (existing_backup_uuid == toString(backup_uuid)) continue; - String status; - if (zk->tryGet(root_zookeeper_path + "/" + existing_backup_path + "/stage", status)) - { - if (status != Stage::COMPLETED) - return true; - } + const auto status = zk->get(root_zookeeper_path + "/" + existing_backup_path + "/stage"); + 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 9dd0a1c40ef..43c8adda65a 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,14 +9,13 @@ from helpers.test_tools import TSV, assert_eq_with_retry cluster = ClickHouseCluster(__file__) -num_nodes = 4 -ddl_task_timeout = 640 +num_nodes = 10 def generate_cluster_def(): path = os.path.join( os.path.dirname(os.path.realpath(__file__)), - "./_gen/cluster_for_disallow_concurrency_test.xml", + "./_gen/cluster_for_concurrency_test.xml", ) os.makedirs(os.path.dirname(path), exist_ok=True) with open(path, "w") as f: @@ -86,7 +85,7 @@ def drop_after_test(): node0.query( "DROP TABLE IF EXISTS tbl ON CLUSTER 'cluster' NO DELAY", settings={ - "distributed_ddl_task_timeout": ddl_task_timeout, + "distributed_ddl_task_timeout": 360, }, ) @@ -101,7 +100,6 @@ def new_backup_name(): def create_and_fill_table(): - node0.query("SET mutations_sync=2") node0.query( "CREATE TABLE tbl ON CLUSTER 'cluster' (" "x UInt64" @@ -109,10 +107,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(100000000)") - nodes[i].query( - f"INSERT INTO tbl SELECT number+100000000 FROM numbers(100000000)" - ) + nodes[i].query(f"INSERT INTO tbl SELECT number FROM numbers(40000000)") # All the tests have concurrent backup/restores with same backup names @@ -143,8 +138,6 @@ def test_concurrent_backups_on_same_node(): nodes[0], f"SELECT status FROM system.backups WHERE status == 'BACKUP_CREATED' AND id = '{id}'", "BACKUP_CREATED", - retry_count=100, - sleep_time=1, ) # This restore part is added to confirm creating an internal backup & restore work @@ -152,7 +145,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": ddl_task_timeout, + "distributed_ddl_task_timeout": 360, }, ) nodes[0].query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}") @@ -181,8 +174,6 @@ def test_concurrent_backups_on_different_nodes(): nodes[1], f"SELECT status FROM system.backups WHERE status == 'BACKUP_CREATED' AND id = '{id}'", "BACKUP_CREATED", - retry_count=100, - sleep_time=1, ) @@ -206,14 +197,12 @@ def test_concurrent_restores_on_same_node(): nodes[0], f"SELECT status FROM system.backups WHERE status == 'BACKUP_CREATED' AND id = '{id}'", "BACKUP_CREATED", - retry_count=100, - sleep_time=1, ) nodes[0].query( f"DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY", settings={ - "distributed_ddl_task_timeout": ddl_task_timeout, + "distributed_ddl_task_timeout": 360, }, ) restore_id = ( @@ -237,46 +226,44 @@ def test_concurrent_restores_on_different_node(): backup_name = new_backup_name() id = ( - nodes[1] + nodes[0] .query(f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name} ASYNC") .split("\t")[0] ) assert_eq_with_retry( - nodes[1], + nodes[0], f"SELECT status FROM system.backups WHERE status == 'CREATING_BACKUP' AND id = '{id}'", "CREATING_BACKUP", ) assert_eq_with_retry( - nodes[1], + nodes[0], f"SELECT status FROM system.backups WHERE status == 'BACKUP_CREATED' AND id = '{id}'", "BACKUP_CREATED", - retry_count=100, - sleep_time=1, ) - nodes[1].query( + nodes[0].query( f"DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY", settings={ - "distributed_ddl_task_timeout": ddl_task_timeout, + "distributed_ddl_task_timeout": 360, }, ) restore_id = ( - nodes[1] + nodes[0] .query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name} ASYNC") .split("\t")[0] ) assert_eq_with_retry( - nodes[1], - f"SELECT status FROM system.backups WHERE status == 'RESTORING' AND id == '{restore_id}'", + nodes[0], + f"SELECT status FROM system.backups WHERE status == 'RESTORING'", "RESTORING", ) - assert "Concurrent restores not supported" in nodes[0].query_and_get_error( + assert "Concurrent restores not supported" in nodes[1].query_and_get_error( f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" ) assert_eq_with_retry( - nodes[1], + nodes[0], f"SELECT status FROM system.backups WHERE status == 'RESTORED' AND id == '{restore_id}'", "RESTORED", ) From c4870ad7bc04732280afde192cf3e0337d240566 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 23 Mar 2023 16:56:20 +0100 Subject: [PATCH 349/418] Move PRs artifacts to the PRs prefix in S3 --- tests/ci/build_check.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index ecc36b1c4e3..4db601be803 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -196,19 +196,21 @@ def create_json_artifact( def get_release_or_pr(pr_info: PRInfo, version: ClickHouseVersion) -> Tuple[str, str]: + "Return prefixes for S3 artifacts paths" # FIXME performance # performance builds are havily relies on a fixed path for artifacts, that's why # we need to preserve 0 for anything but PR number # It should be fixed in performance-comparison image eventually - performance_pr = "0" + # For performance tests we always set PRs prefix + performance_pr = "PRs/0" if "release" in pr_info.labels or "release-lts" in pr_info.labels: # for release pull requests we use branch names prefixes, not pr numbers return pr_info.head_ref, performance_pr - elif pr_info.number == 0: + if pr_info.number == 0: # for pushes to master - major version return f"{version.major}.{version.minor}", performance_pr # PR number for anything else - pr_number = str(pr_info.number) + pr_number = f"PRs/{pr_info.number}" return pr_number, pr_number From 8917e1ad5714fe9fee4bfce13c7ea4bad2d7b880 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 24 Mar 2023 09:40:12 +0000 Subject: [PATCH 350/418] Fixing test --- src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index f749ea7b2a2..7008b164a3c 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -76,7 +76,7 @@ const ActionsDAG::Node * findInOutputs(ActionsDAG & dag, const std::string & nam const auto * node = *it; - if (!isUInt8(removeLowCardinality(removeNullable(node->result_type)))) + if (!isUInt8(removeNullable(removeLowCardinality(node->result_type)))) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, "Illegal type {} of column {} for filter. Must be UInt8 or Nullable(UInt8).", node->result_type->getName(), name); From 46e50b17e4962a39488cf4c63ddf0359936b9f3b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 24 Mar 2023 10:35:47 +0000 Subject: [PATCH 351/418] Fixing test --- src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp | 4 ++++ src/Processors/Transforms/FilterTransform.cpp | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index 7008b164a3c..a334450fb41 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -75,6 +75,10 @@ const ActionsDAG::Node * findInOutputs(ActionsDAG & dag, const std::string & nam { const auto * node = *it; + /// We allow to use Null as a filter. + /// In this case, result is empty. Ignore optimizations. + if (node->result_type->onlyNull()) + return nullptr; if (!isUInt8(removeNullable(removeLowCardinality(node->result_type)))) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, diff --git a/src/Processors/Transforms/FilterTransform.cpp b/src/Processors/Transforms/FilterTransform.cpp index 7d164cdff3d..089393a1430 100644 --- a/src/Processors/Transforms/FilterTransform.cpp +++ b/src/Processors/Transforms/FilterTransform.cpp @@ -44,7 +44,7 @@ Block FilterTransform::transformHeader( header = expression->updateHeader(std::move(header)); auto filter_type = header.getByName(filter_column_name).type; - if (!isUInt8(removeNullable(removeLowCardinality(filter_type)))) + if (!filter_type->onlyNull() && !isUInt8(removeNullable(removeLowCardinality(filter_type)))) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, "Illegal type {} of column {} for filter. Must be UInt8 or Nullable(UInt8).", filter_type->getName(), filter_column_name); From 4f2c336cfc52e743ef829b18679878a64586fb17 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 24 Mar 2023 11:54:38 +0100 Subject: [PATCH 352/418] Update black version, run it for the repository --- docker/test/performance-comparison/perf.py | 2 ++ docker/test/performance-comparison/report.py | 1 - docker/test/style/Dockerfile | 2 +- tests/ci/clickhouse_helper.py | 1 - tests/ci/docker_images_check.py | 1 - tests/ci/get_previous_release_tag.py | 1 - tests/ci/report.py | 2 +- tests/integration/helpers/cluster.py | 3 +-- tests/integration/helpers/network.py | 2 -- .../pytest_xdist_logging_to_separate_files.py | 1 + .../test_detach_part_wrong_partition_id.py | 1 - .../test_cluster_copier/test_three_nodes.py | 1 - .../test_cluster_copier/test_two_nodes.py | 1 - tests/integration/test_composable_protocols/test.py | 1 - .../test_create_query_constraints/test.py | 2 -- .../common.py | 1 - tests/integration/test_disks_app_func/test.py | 1 - .../test_distributed_ddl_parallel/test.py | 1 + tests/integration/test_fetch_memory_usage/test.py | 1 - .../scripts/stress_test.py | 1 - tests/integration/test_jbod_balancer/test.py | 1 - .../test_keeper_and_access_storage/test.py | 1 + tests/integration/test_keeper_back_to_back/test.py | 2 +- tests/integration/test_keeper_persistent_log/test.py | 1 - .../test_keeper_zookeeper_converter/test.py | 1 - tests/integration/test_merge_tree_load_parts/test.py | 6 +++--- .../s3_endpoint/endpoint.py | 1 - .../test_merge_tree_settings_constraints/test.py | 1 - .../test_old_parts_finally_removed/test.py | 1 - tests/integration/test_partition/test.py | 4 +++- tests/integration/test_password_constraints/test.py | 1 - tests/integration/test_read_only_table/test.py | 1 - .../test_reload_auxiliary_zookeepers/test.py | 1 - .../s3_endpoint/endpoint.py | 1 + tests/integration/test_s3_with_proxy/test.py | 1 + .../integration/test_ssl_cert_authentication/test.py | 1 - tests/integration/test_storage_kafka/kafka_pb2.py | 1 - .../test_storage_kafka/message_with_repeated_pb2.py | 1 - tests/integration/test_storage_kafka/social_pb2.py | 1 - tests/integration/test_storage_kafka/test.py | 12 ++---------- tests/integration/test_storage_nats/nats_pb2.py | 1 - .../test_storage_postgresql_replica/test.py | 1 - .../test_storage_rabbitmq/rabbitmq_pb2.py | 1 - tests/integration/test_storage_rabbitmq/test.py | 3 --- tests/integration/test_storage_s3/test.py | 1 + .../test_storage_s3/test_invalid_env_credentials.py | 1 + tests/integration/test_system_merges/test.py | 1 - tests/integration/test_ttl_move/test.py | 2 +- tests/integration/test_zero_copy_fetch/test.py | 1 - .../00411_long_accurate_number_comparison.python | 4 ++-- tests/queries/0_stateless/02010_lc_native.python | 1 - .../02458_insert_select_progress_tcp.python | 1 - .../0_stateless/02481_async_insert_dedup.python | 1 + utils/changelog-simple/format-changelog.py | 1 + utils/keeper-overload/keeper-overload.py | 2 +- utils/s3tools/s3uploader | 2 -- 56 files changed, 27 insertions(+), 64 deletions(-) diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 65bf49c2914..7a4e6386d0d 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -26,6 +26,7 @@ logging.basicConfig( total_start_seconds = time.perf_counter() stage_start_seconds = total_start_seconds + # Thread executor that does not hides exception that happens during function # execution, and rethrows it after join() class SafeThread(Thread): @@ -158,6 +159,7 @@ for e in subst_elems: available_parameters[name] = values + # Takes parallel lists of templates, substitutes them with all combos of # parameters. The set of parameters is determined based on the first list. # Note: keep the order of queries -- sometimes we have DROP IF EXISTS diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index 782cf29863c..214f2d550b4 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -670,7 +670,6 @@ if args.report == "main": ) elif args.report == "all-queries": - print((header_template.format())) add_tested_commits() diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index f7b2ef21c2a..746cc7bb2d5 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -18,7 +18,7 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ python3-pip \ shellcheck \ yamllint \ - && pip3 install black==22.8.0 boto3 codespell==2.2.1 dohq-artifactory mypy PyGithub unidiff pylint==2.6.2 \ + && pip3 install black==23.1.0 boto3 codespell==2.2.1 dohq-artifactory mypy PyGithub unidiff pylint==2.6.2 \ && apt-get clean \ && rm -rf /root/.cache/pip diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index d60a9e6afd1..64b64896f66 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -141,7 +141,6 @@ def prepare_tests_results_for_clickhouse( report_url: str, check_name: str, ) -> List[dict]: - pull_request_url = "https://github.com/ClickHouse/ClickHouse/commits/master" base_ref = "master" head_ref = "master" diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 192d216614e..f2b1105b3b0 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -96,7 +96,6 @@ def get_images_dict(repo_path: str, image_file_path: str) -> ImagesDict: def get_changed_docker_images( pr_info: PRInfo, images_dict: ImagesDict ) -> Set[DockerImage]: - if not images_dict: return set() diff --git a/tests/ci/get_previous_release_tag.py b/tests/ci/get_previous_release_tag.py index c6fe6cd5fb5..c2d279f7fec 100755 --- a/tests/ci/get_previous_release_tag.py +++ b/tests/ci/get_previous_release_tag.py @@ -51,7 +51,6 @@ def find_previous_release( for release in releases: if release.version < server_version: - # Check if the artifact exists on GitHub. # It can be not true for a short period of time # after creating a tag for a new release before uploading the packages. diff --git a/tests/ci/report.py b/tests/ci/report.py index 947fb33d905..ddee035d26f 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -473,7 +473,7 @@ def create_build_html_report( commit_url: str, ) -> str: rows = "" - for (build_result, build_log_url, artifact_urls) in zip( + for build_result, build_log_url, artifact_urls in zip( build_results, build_logs_urls, artifact_urls_list ): row = "" diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index dc5ada81995..a9a996e0a5f 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -63,6 +63,7 @@ DEFAULT_ENV_NAME = ".env" SANITIZER_SIGN = "==================" + # to create docker-compose env file def _create_env_file(path, variables): logging.debug(f"Env {variables} stored in {path}") @@ -1454,7 +1455,6 @@ class ClickHouseCluster: config_root_name="clickhouse", extra_configs=[], ) -> "ClickHouseInstance": - """Add an instance to the cluster. name - the name of the instance directory and the value of the 'instance' macro in ClickHouse. @@ -3089,7 +3089,6 @@ class ClickHouseInstance: config_root_name="clickhouse", extra_configs=[], ): - self.name = name self.base_cmd = cluster.base_cmd self.docker_id = cluster.get_instance_docker_id(self.name) diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index e408c9beec1..471aa2bdc2e 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -216,7 +216,6 @@ class _NetworkManager: container_exit_timeout=60, docker_api_version=os.environ.get("DOCKER_API_VERSION"), ): - self.container_expire_timeout = container_expire_timeout self.container_exit_timeout = container_exit_timeout @@ -232,7 +231,6 @@ class _NetworkManager: def _ensure_container(self): if self._container is None or self._container_expire_time <= time.time(): - for i in range(5): if self._container is not None: try: diff --git a/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py b/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py index d424ad58fa4..370aa23a014 100644 --- a/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py +++ b/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py @@ -1,6 +1,7 @@ import logging import os.path + # Makes the parallel workers of pytest-xdist to log to separate files. # Without this function all workers will log to the same log file # and mix everything together making it much more difficult for troubleshooting. diff --git a/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py b/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py index 02fccfae4e5..a6f7a8653da 100644 --- a/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py +++ b/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py @@ -24,7 +24,6 @@ def start_cluster(): def test_detach_part_wrong_partition_id(start_cluster): - # Here we create table with partition by UUID. node_21_6.query( "create table tab (id UUID, value UInt32) engine = MergeTree PARTITION BY (id) order by tuple()" diff --git a/tests/integration/test_cluster_copier/test_three_nodes.py b/tests/integration/test_cluster_copier/test_three_nodes.py index 31d6c0448f4..e7d07757adb 100644 --- a/tests/integration/test_cluster_copier/test_three_nodes.py +++ b/tests/integration/test_cluster_copier/test_three_nodes.py @@ -19,7 +19,6 @@ cluster = ClickHouseCluster(__file__) def started_cluster(): global cluster try: - for name in ["first", "second", "third"]: cluster.add_instance( name, diff --git a/tests/integration/test_cluster_copier/test_two_nodes.py b/tests/integration/test_cluster_copier/test_two_nodes.py index 10ab7d03b00..2b6fcf6cac2 100644 --- a/tests/integration/test_cluster_copier/test_two_nodes.py +++ b/tests/integration/test_cluster_copier/test_two_nodes.py @@ -19,7 +19,6 @@ cluster = ClickHouseCluster(__file__) def started_cluster(): global cluster try: - for name in ["first_of_two", "second_of_two"]: instance = cluster.add_instance( name, diff --git a/tests/integration/test_composable_protocols/test.py b/tests/integration/test_composable_protocols/test.py index bc87fea5296..df74cfffa54 100644 --- a/tests/integration/test_composable_protocols/test.py +++ b/tests/integration/test_composable_protocols/test.py @@ -63,7 +63,6 @@ def netcat(hostname, port, content): def test_connections(): - client = Client(server.ip_address, 9000, command=cluster.client_bin_path) assert client.query("SELECT 1") == "1\n" diff --git a/tests/integration/test_create_query_constraints/test.py b/tests/integration/test_create_query_constraints/test.py index 8df043fd24b..33c41b4f161 100644 --- a/tests/integration/test_create_query_constraints/test.py +++ b/tests/integration/test_create_query_constraints/test.py @@ -25,7 +25,6 @@ def start_cluster(): def test_create_query_const_constraints(): - instance.query("CREATE USER u_const SETTINGS max_threads = 1 CONST") instance.query("GRANT ALL ON *.* TO u_const") @@ -57,7 +56,6 @@ def test_create_query_const_constraints(): def test_create_query_minmax_constraints(): - instance.query("CREATE USER u_minmax SETTINGS max_threads = 4 MIN 2 MAX 6") instance.query("GRANT ALL ON *.* TO u_minmax") diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py index b38e81b0227..01addae2542 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py @@ -348,7 +348,6 @@ class RangedLayoutTester(BaseLayoutTester): self.layouts = LAYOUTS_RANGED def execute(self, layout_name, node): - if layout_name not in self.layout_to_dictionary: raise RuntimeError("Source doesn't support layout: {}".format(layout_name)) diff --git a/tests/integration/test_disks_app_func/test.py b/tests/integration/test_disks_app_func/test.py index 027ef8feed0..2428c53854e 100644 --- a/tests/integration/test_disks_app_func/test.py +++ b/tests/integration/test_disks_app_func/test.py @@ -7,7 +7,6 @@ import pytest def started_cluster(): global cluster try: - cluster = ClickHouseCluster(__file__) cluster.add_instance( "disks_app_test", main_configs=["config.xml"], with_minio=True diff --git a/tests/integration/test_distributed_ddl_parallel/test.py b/tests/integration/test_distributed_ddl_parallel/test.py index 6ebfe472e09..eb98dd3e230 100644 --- a/tests/integration/test_distributed_ddl_parallel/test.py +++ b/tests/integration/test_distributed_ddl_parallel/test.py @@ -10,6 +10,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) + # By default the exceptions that was throwed in threads will be ignored # (they will not mark the test as failed, only printed to stderr). # diff --git a/tests/integration/test_fetch_memory_usage/test.py b/tests/integration/test_fetch_memory_usage/test.py index a4371140150..7591cc0e8a9 100644 --- a/tests/integration/test_fetch_memory_usage/test.py +++ b/tests/integration/test_fetch_memory_usage/test.py @@ -18,7 +18,6 @@ def started_cluster(): def test_huge_column(started_cluster): - if ( node.is_built_with_thread_sanitizer() or node.is_built_with_memory_sanitizer() diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py index b8bafb3d0c1..fe69d72c1c7 100644 --- a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py +++ b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py @@ -13,7 +13,6 @@ number_of_iterations = 100 def perform_request(): - buffer = BytesIO() crl = pycurl.Curl() crl.setopt(pycurl.INTERFACE, client_ip) diff --git a/tests/integration/test_jbod_balancer/test.py b/tests/integration/test_jbod_balancer/test.py index e746698611a..df34a075d5a 100644 --- a/tests/integration/test_jbod_balancer/test.py +++ b/tests/integration/test_jbod_balancer/test.py @@ -45,7 +45,6 @@ def start_cluster(): def check_balance(node, table): - partitions = node.query( """ WITH diff --git a/tests/integration/test_keeper_and_access_storage/test.py b/tests/integration/test_keeper_and_access_storage/test.py index 6ec307f7082..0314825b6b7 100644 --- a/tests/integration/test_keeper_and_access_storage/test.py +++ b/tests/integration/test_keeper_and_access_storage/test.py @@ -10,6 +10,7 @@ node1 = cluster.add_instance( "node1", main_configs=["configs/keeper.xml"], stay_alive=True ) + # test that server is able to start @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_keeper_back_to_back/test.py b/tests/integration/test_keeper_back_to_back/test.py index 73fface02b4..b737ac284d2 100644 --- a/tests/integration/test_keeper_back_to_back/test.py +++ b/tests/integration/test_keeper_back_to_back/test.py @@ -546,7 +546,6 @@ def test_random_requests(started_cluster): def test_end_of_session(started_cluster): - fake_zk1 = None fake_zk2 = None genuine_zk1 = None @@ -685,6 +684,7 @@ def test_concurrent_watches(started_cluster): nonlocal watches_created nonlocal all_paths_created fake_zk.ensure_path(global_path + "/" + str(i)) + # new function each time def dumb_watch(event): nonlocal dumb_watch_triggered_counter diff --git a/tests/integration/test_keeper_persistent_log/test.py b/tests/integration/test_keeper_persistent_log/test.py index 70cc14fe26d..4164ffb33d3 100644 --- a/tests/integration/test_keeper_persistent_log/test.py +++ b/tests/integration/test_keeper_persistent_log/test.py @@ -163,7 +163,6 @@ def test_state_duplicate_restart(started_cluster): # http://zookeeper-user.578899.n2.nabble.com/Why-are-ephemeral-nodes-written-to-disk-tp7583403p7583418.html def test_ephemeral_after_restart(started_cluster): - try: node_zk = None node_zk2 = None diff --git a/tests/integration/test_keeper_zookeeper_converter/test.py b/tests/integration/test_keeper_zookeeper_converter/test.py index 063421bf922..de5a9416119 100644 --- a/tests/integration/test_keeper_zookeeper_converter/test.py +++ b/tests/integration/test_keeper_zookeeper_converter/test.py @@ -114,7 +114,6 @@ def start_clickhouse(): def copy_zookeeper_data(make_zk_snapshots): - if make_zk_snapshots: # force zookeeper to create snapshot generate_zk_snapshot() else: diff --git a/tests/integration/test_merge_tree_load_parts/test.py b/tests/integration/test_merge_tree_load_parts/test.py index 777b6f14fc6..dfbe00c8e28 100644 --- a/tests/integration/test_merge_tree_load_parts/test.py +++ b/tests/integration/test_merge_tree_load_parts/test.py @@ -148,17 +148,17 @@ def test_merge_tree_load_parts_corrupted(started_cluster): node1.query("SYSTEM WAIT LOADING PARTS mt_load_parts_2") def check_parts_loading(node, partition, loaded, failed, skipped): - for (min_block, max_block) in loaded: + for min_block, max_block in loaded: part_name = f"{partition}_{min_block}_{max_block}" assert node.contains_in_log(f"Loading Active part {part_name}") assert node.contains_in_log(f"Finished loading Active part {part_name}") - for (min_block, max_block) in failed: + for min_block, max_block in failed: part_name = f"{partition}_{min_block}_{max_block}" assert node.contains_in_log(f"Loading Active part {part_name}") assert not node.contains_in_log(f"Finished loading Active part {part_name}") - for (min_block, max_block) in skipped: + for min_block, max_block in skipped: part_name = f"{partition}_{min_block}_{max_block}" assert not node.contains_in_log(f"Loading Active part {part_name}") assert not node.contains_in_log(f"Finished loading Active part {part_name}") diff --git a/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py b/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py index b6567dfebc5..4613fdb850b 100644 --- a/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py +++ b/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py @@ -42,7 +42,6 @@ def delete(_bucket): @route("/<_bucket>/<_path:path>", ["GET", "POST", "PUT", "DELETE"]) def server(_bucket, _path): - # It's delete query for failed part if _path.endswith("delete"): response.set_header("Location", "http://minio1:9001/" + _bucket + "/" + _path) diff --git a/tests/integration/test_merge_tree_settings_constraints/test.py b/tests/integration/test_merge_tree_settings_constraints/test.py index 0bb0179108d..be6e2a31873 100644 --- a/tests/integration/test_merge_tree_settings_constraints/test.py +++ b/tests/integration/test_merge_tree_settings_constraints/test.py @@ -20,7 +20,6 @@ def start_cluster(): def test_merge_tree_settings_constraints(): - assert "Setting storage_policy should not be changed" in instance.query_and_get_error( f"CREATE TABLE wrong_table (number Int64) engine = MergeTree() ORDER BY number SETTINGS storage_policy = 'secret_policy'" ) diff --git a/tests/integration/test_old_parts_finally_removed/test.py b/tests/integration/test_old_parts_finally_removed/test.py index 108b72c5ccd..5347d433419 100644 --- a/tests/integration/test_old_parts_finally_removed/test.py +++ b/tests/integration/test_old_parts_finally_removed/test.py @@ -63,7 +63,6 @@ def test_part_finally_removed(started_cluster): ) for i in range(60): - if ( node1.query( "SELECT count() from system.parts WHERE table = 'drop_outdated_part'" diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index ae4393fc6f6..a34141c6189 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -528,7 +528,9 @@ def test_make_clone_in_detached(started_cluster): ["cp", "-r", path + "all_0_0_0", path + "detached/broken_all_0_0_0"] ) assert_eq_with_retry(instance, "select * from clone_in_detached", "\n") - assert ["broken_all_0_0_0",] == sorted( + assert [ + "broken_all_0_0_0", + ] == sorted( instance.exec_in_container(["ls", path + "detached/"]).strip().split("\n") ) diff --git a/tests/integration/test_password_constraints/test.py b/tests/integration/test_password_constraints/test.py index e3628861b28..9cdff51caa1 100644 --- a/tests/integration/test_password_constraints/test.py +++ b/tests/integration/test_password_constraints/test.py @@ -17,7 +17,6 @@ def start_cluster(): def test_complexity_rules(start_cluster): - error_message = "DB::Exception: Invalid password. The password should: be at least 12 characters long, contain at least 1 numeric character, contain at least 1 lowercase character, contain at least 1 uppercase character, contain at least 1 special character" assert error_message in node.query_and_get_error( "CREATE USER u_1 IDENTIFIED WITH plaintext_password BY ''" diff --git a/tests/integration/test_read_only_table/test.py b/tests/integration/test_read_only_table/test.py index 914c6a99508..df084f9dbbd 100644 --- a/tests/integration/test_read_only_table/test.py +++ b/tests/integration/test_read_only_table/test.py @@ -49,7 +49,6 @@ def start_cluster(): def test_restart_zookeeper(start_cluster): - for table_id in range(NUM_TABLES): node1.query( f"INSERT INTO test_table_{table_id} VALUES (1), (2), (3), (4), (5);" diff --git a/tests/integration/test_reload_auxiliary_zookeepers/test.py b/tests/integration/test_reload_auxiliary_zookeepers/test.py index bb1455333fc..476c5dee99e 100644 --- a/tests/integration/test_reload_auxiliary_zookeepers/test.py +++ b/tests/integration/test_reload_auxiliary_zookeepers/test.py @@ -20,7 +20,6 @@ def start_cluster(): def test_reload_auxiliary_zookeepers(start_cluster): - node.query( "CREATE TABLE simple (date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', 'node') ORDER BY tuple() PARTITION BY date;" ) diff --git a/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py b/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py index d6a732cc681..1d33ca02f86 100644 --- a/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py +++ b/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py @@ -1,6 +1,7 @@ #!/usr/bin/env python3 from bottle import request, route, run, response + # Handle for MultipleObjectsDelete. @route("/<_bucket>", ["POST"]) def delete(_bucket): diff --git a/tests/integration/test_s3_with_proxy/test.py b/tests/integration/test_s3_with_proxy/test.py index 1102d190a87..1af040c3c30 100644 --- a/tests/integration/test_s3_with_proxy/test.py +++ b/tests/integration/test_s3_with_proxy/test.py @@ -5,6 +5,7 @@ import time import pytest from helpers.cluster import ClickHouseCluster + # Runs simple proxy resolver in python env container. def run_resolver(cluster): container_id = cluster.get_container_id("resolver") diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index 7c62ca0d8b6..b3570b6e281 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -87,7 +87,6 @@ config = """ def execute_query_native(node, query, user, cert_name): - config_path = f"{SCRIPT_DIR}/configs/client.xml" formatted = config.format( diff --git a/tests/integration/test_storage_kafka/kafka_pb2.py b/tests/integration/test_storage_kafka/kafka_pb2.py index 7de1363bbf1..3e47af6c1e0 100644 --- a/tests/integration/test_storage_kafka/kafka_pb2.py +++ b/tests/integration/test_storage_kafka/kafka_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.kafka_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _KEYVALUEPAIR._serialized_start = 46 _KEYVALUEPAIR._serialized_end = 88 diff --git a/tests/integration/test_storage_kafka/message_with_repeated_pb2.py b/tests/integration/test_storage_kafka/message_with_repeated_pb2.py index 4d1a23c0b43..3715a9bea04 100644 --- a/tests/integration/test_storage_kafka/message_with_repeated_pb2.py +++ b/tests/integration/test_storage_kafka/message_with_repeated_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.message_with_repeated_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None DESCRIPTOR._serialized_options = b"H\001" _MESSAGE._serialized_start = 62 diff --git a/tests/integration/test_storage_kafka/social_pb2.py b/tests/integration/test_storage_kafka/social_pb2.py index 830ade81d33..f91a7bd0539 100644 --- a/tests/integration/test_storage_kafka/social_pb2.py +++ b/tests/integration/test_storage_kafka/social_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.social_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _USER._serialized_start = 47 _USER._serialized_end = 90 diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 51952ac1eb7..3a4fa6c6bfe 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -121,7 +121,7 @@ def kafka_create_topic( def kafka_delete_topic(admin_client, topic, max_retries=50): result = admin_client.delete_topics([topic]) - for (topic, e) in result.topic_error_codes: + for topic, e in result.topic_error_codes: if e == 0: logging.debug(f"Topic {topic} deleted") else: @@ -917,9 +917,7 @@ def describe_consumer_group(kafka_cluster, name): member_info["client_id"] = client_id member_info["client_host"] = client_host member_topics_assignment = [] - for (topic, partitions) in MemberAssignment.decode( - member_assignment - ).assignment: + for topic, partitions in MemberAssignment.decode(member_assignment).assignment: member_topics_assignment.append({"topic": topic, "partitions": partitions}) member_info["assignment"] = member_topics_assignment res.append(member_info) @@ -1537,7 +1535,6 @@ def test_kafka_protobuf_no_delimiter(kafka_cluster): def test_kafka_materialized_view(kafka_cluster): - instance.query( """ DROP TABLE IF EXISTS test.view; @@ -2315,7 +2312,6 @@ def test_kafka_virtual_columns2(kafka_cluster): def test_kafka_produce_key_timestamp(kafka_cluster): - admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) @@ -2444,7 +2440,6 @@ def test_kafka_insert_avro(kafka_cluster): def test_kafka_produce_consume_avro(kafka_cluster): - admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) @@ -4031,7 +4026,6 @@ def test_kafka_predefined_configuration(kafka_cluster): # https://github.com/ClickHouse/ClickHouse/issues/26643 def test_issue26643(kafka_cluster): - # for backporting: # admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") admin_client = KafkaAdminClient( @@ -4313,7 +4307,6 @@ def test_row_based_formats(kafka_cluster): "RowBinaryWithNamesAndTypes", "MsgPack", ]: - print(format_name) kafka_create_topic(admin_client, format_name) @@ -4438,7 +4431,6 @@ def test_block_based_formats_2(kafka_cluster): "ORC", "JSONCompactColumns", ]: - kafka_create_topic(admin_client, format_name) instance.query( diff --git a/tests/integration/test_storage_nats/nats_pb2.py b/tests/integration/test_storage_nats/nats_pb2.py index 4330ff57950..e9e5cb72363 100644 --- a/tests/integration/test_storage_nats/nats_pb2.py +++ b/tests/integration/test_storage_nats/nats_pb2.py @@ -31,7 +31,6 @@ ProtoKeyValue = _reflection.GeneratedProtocolMessageType( _sym_db.RegisterMessage(ProtoKeyValue) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _PROTOKEYVALUE._serialized_start = 45 _PROTOKEYVALUE._serialized_end = 88 diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 5df8b9029e6..8666d7ae58c 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -706,7 +706,6 @@ def test_abrupt_connection_loss_while_heavy_replication(started_cluster): def test_abrupt_server_restart_while_heavy_replication(started_cluster): - # FIXME (kssenii) temporary disabled if instance.is_built_with_sanitizer(): pytest.skip("Temporary disabled (FIXME)") diff --git a/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py b/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py index e017b4e66c2..a5845652eef 100644 --- a/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py +++ b/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.rabbitmq_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _KEYVALUEPROTO._serialized_start = 49 _KEYVALUEPROTO._serialized_end = 92 diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 2e54f21787a..53b6c4109ef 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -2864,7 +2864,6 @@ def test_rabbitmq_predefined_configuration(rabbitmq_cluster): def test_rabbitmq_msgpack(rabbitmq_cluster): - instance.query( """ drop table if exists rabbit_in; @@ -2908,7 +2907,6 @@ def test_rabbitmq_msgpack(rabbitmq_cluster): def test_rabbitmq_address(rabbitmq_cluster): - instance2.query( """ drop table if exists rabbit_in; @@ -3243,7 +3241,6 @@ def test_block_based_formats_2(rabbitmq_cluster): "ORC", "JSONCompactColumns", ]: - print(format_name) instance.query( diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 8b20727a7b5..4d493d9526b 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -18,6 +18,7 @@ MINIO_INTERNAL_PORT = 9001 SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + # Creates S3 bucket for tests and allows anonymous read-write access to it. def prepare_s3_bucket(started_cluster): # Allows read-write access for bucket without authorization. diff --git a/tests/integration/test_storage_s3/test_invalid_env_credentials.py b/tests/integration/test_storage_s3/test_invalid_env_credentials.py index 2f5d9349904..aa6479a2ed3 100644 --- a/tests/integration/test_storage_s3/test_invalid_env_credentials.py +++ b/tests/integration/test_storage_s3/test_invalid_env_credentials.py @@ -11,6 +11,7 @@ MINIO_INTERNAL_PORT = 9001 SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + # Creates S3 bucket for tests and allows anonymous read-write access to it. def prepare_s3_bucket(started_cluster): # Allows read-write access for bucket without authorization. diff --git a/tests/integration/test_system_merges/test.py b/tests/integration/test_system_merges/test.py index 0a469bd7bbd..ff303afe19e 100644 --- a/tests/integration/test_system_merges/test.py +++ b/tests/integration/test_system_merges/test.py @@ -171,7 +171,6 @@ def test_mutation_simple(started_cluster, replicated): starting_block = 0 if replicated else 1 try: - for node in nodes: node.query( f"create table {name} (a Int64) engine={engine} order by tuple()" diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 99978cbf6dc..89824293320 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1863,7 +1863,7 @@ def test_ttl_move_if_exists(started_cluster, name, dest_type): ) ) - for (node, policy) in zip( + for node, policy in zip( [node1, node2], ["only_jbod_1", "small_jbod_with_external"] ): node.query( diff --git a/tests/integration/test_zero_copy_fetch/test.py b/tests/integration/test_zero_copy_fetch/test.py index b71752528d3..9b9aa5e0da7 100644 --- a/tests/integration/test_zero_copy_fetch/test.py +++ b/tests/integration/test_zero_copy_fetch/test.py @@ -16,7 +16,6 @@ cluster = ClickHouseCluster(__file__) @pytest.fixture(scope="module") def started_cluster(): try: - cluster.add_instance( "node1", main_configs=["configs/storage_conf.xml"], diff --git a/tests/queries/0_stateless/00411_long_accurate_number_comparison.python b/tests/queries/0_stateless/00411_long_accurate_number_comparison.python index e713e8814bc..183a2637d36 100644 --- a/tests/queries/0_stateless/00411_long_accurate_number_comparison.python +++ b/tests/queries/0_stateless/00411_long_accurate_number_comparison.python @@ -183,7 +183,7 @@ def main(): num_parts = 4 for part in range(0, num_parts): if "int" + str(part + 1) in sys.argv[1:]: - for (v1, v2) in itertools.islice( + for v1, v2 in itertools.islice( itertools.combinations(VALUES, 2), part * num_int_tests // num_parts, (part + 1) * num_int_tests // num_parts, @@ -194,7 +194,7 @@ def main(): ref_file.write(a + "\n") if "float" in sys.argv[1:]: - for (i, f) in itertools.product(VALUES_INT, VALUES_FLOAT): + for i, f in itertools.product(VALUES_INT, VALUES_FLOAT): q, a = test_float_pair(i, f) if GENERATE_TEST_FILES: sql_file.write(q + ";\n") diff --git a/tests/queries/0_stateless/02010_lc_native.python b/tests/queries/0_stateless/02010_lc_native.python index 09ac60405e7..a197d32a3b9 100755 --- a/tests/queries/0_stateless/02010_lc_native.python +++ b/tests/queries/0_stateless/02010_lc_native.python @@ -12,7 +12,6 @@ CLICKHOUSE_DATABASE = os.environ.get("CLICKHOUSE_DATABASE", "default") def writeVarUInt(x, ba): for _ in range(0, 9): - byte = x & 0x7F if x > 0x7F: byte |= 0x80 diff --git a/tests/queries/0_stateless/02458_insert_select_progress_tcp.python b/tests/queries/0_stateless/02458_insert_select_progress_tcp.python index 9d64201afd9..696eb01ff7e 100644 --- a/tests/queries/0_stateless/02458_insert_select_progress_tcp.python +++ b/tests/queries/0_stateless/02458_insert_select_progress_tcp.python @@ -12,7 +12,6 @@ CLICKHOUSE_DATABASE = os.environ.get("CLICKHOUSE_DATABASE", "default") def writeVarUInt(x, ba): for _ in range(0, 9): - byte = x & 0x7F if x > 0x7F: byte |= 0x80 diff --git a/tests/queries/0_stateless/02481_async_insert_dedup.python b/tests/queries/0_stateless/02481_async_insert_dedup.python index 1bf0edcbd05..1be2b673b73 100644 --- a/tests/queries/0_stateless/02481_async_insert_dedup.python +++ b/tests/queries/0_stateless/02481_async_insert_dedup.python @@ -37,6 +37,7 @@ result = client.query("select count(*) from t_async_insert_dedup_no_part") print(result, flush=True) client.query("DROP TABLE IF EXISTS t_async_insert_dedup_no_part NO DELAY") + # generate data and push to queue def generate_data(q, total_number): old_data = [] diff --git a/utils/changelog-simple/format-changelog.py b/utils/changelog-simple/format-changelog.py index d5e1518270e..01f2694dd0f 100755 --- a/utils/changelog-simple/format-changelog.py +++ b/utils/changelog-simple/format-changelog.py @@ -20,6 +20,7 @@ parser.add_argument( ) args = parser.parse_args() + # This function mirrors the PR description checks in ClickhousePullRequestTrigger. # Returns False if the PR should not be mentioned changelog. def parse_one_pull_request(item): diff --git a/utils/keeper-overload/keeper-overload.py b/utils/keeper-overload/keeper-overload.py index bdb4563c713..0a059b10588 100755 --- a/utils/keeper-overload/keeper-overload.py +++ b/utils/keeper-overload/keeper-overload.py @@ -166,7 +166,7 @@ def main(args): keeper_bench_path = args.keeper_bench_path keepers = [] - for (port, server_id) in zip(PORTS, SERVER_IDS): + for port, server_id in zip(PORTS, SERVER_IDS): keepers.append( Keeper( keeper_binary_path, server_id, port, workdir, args.with_thread_fuzzer diff --git a/utils/s3tools/s3uploader b/utils/s3tools/s3uploader index 4855bdb3f96..d53661614c0 100755 --- a/utils/s3tools/s3uploader +++ b/utils/s3tools/s3uploader @@ -107,7 +107,6 @@ class S3API(object): def make_tar_file_for_table(clickhouse_data_path, db_name, table_name, tmp_prefix): - relative_data_path = os.path.join("data", db_name, table_name) relative_meta_path = os.path.join("metadata", db_name, table_name + ".sql") path_to_data = os.path.join(clickhouse_data_path, relative_data_path) @@ -193,7 +192,6 @@ if __name__ == "__main__": ) logging.info("Data uploaded: %s", url) else: - if args.table_name is not None: if "." not in args.table_name: db_name = "default" From 3628d114059a0f189f12b1bb9b6202b89b833220 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 24 Mar 2023 10:56:02 +0000 Subject: [PATCH 353/418] Fix "Field value too long" in catboostEvaluate() Fixes #47616 --- src/Bridge/IBridge.cpp | 27 ++++++++++++------- src/Bridge/IBridge.h | 1 + src/BridgeHelper/IBridgeHelper.cpp | 2 ++ .../test_catboost_evaluate/test.py | 2 +- 4 files changed, 22 insertions(+), 10 deletions(-) diff --git a/src/Bridge/IBridge.cpp b/src/Bridge/IBridge.cpp index daae05de892..62365f4c93c 100644 --- a/src/Bridge/IBridge.cpp +++ b/src/Bridge/IBridge.cpp @@ -4,21 +4,22 @@ #include #include -#include - -#include #include -#include "config.h" +#include #include -#include -#include #include -#include +#include #include #include +#include +#include +#include + #include #include +#include "config.h" + #if USE_ODBC # include #endif @@ -89,7 +90,7 @@ void IBridge::defineOptions(Poco::Util::OptionSet & options) Poco::Util::Option("listen-host", "", "hostname or address to listen, default 127.0.0.1").argument("listen-host").binding("listen-host")); options.addOption( - Poco::Util::Option("http-timeout", "", "http timeout for socket, default 1800").argument("http-timeout").binding("http-timeout")); + Poco::Util::Option("http-timeout", "", "http timeout for socket, default 180").argument("http-timeout").binding("http-timeout")); options.addOption( Poco::Util::Option("max-server-connections", "", "max connections to server, default 1024").argument("max-server-connections").binding("max-server-connections")); @@ -97,6 +98,9 @@ void IBridge::defineOptions(Poco::Util::OptionSet & options) options.addOption( Poco::Util::Option("keep-alive-timeout", "", "keepalive timeout, default 10").argument("keep-alive-timeout").binding("keep-alive-timeout")); + options.addOption( + Poco::Util::Option("http-max-field-value-size", "", "max http field value size, default 1048576").argument("http-max-field-value-size").binding("http-max-field-value-size")); + options.addOption( Poco::Util::Option("log-level", "", "sets log level, default info") .argument("log-level").binding("logger.level")); @@ -117,7 +121,7 @@ void IBridge::defineOptions(Poco::Util::OptionSet & options) options.addOption( Poco::Util::Option("help", "", "produce this help message").binding("help").callback(Poco::Util::OptionCallback(this, &Me::handleHelp))); - ServerApplication::defineOptions(options); // NOLINT Don't need complex BaseDaemon's .xml config + ServerApplication::defineOptions(options); // Don't need complex BaseDaemon's .xml config } @@ -165,6 +169,7 @@ void IBridge::initialize(Application & self) http_timeout = config().getUInt64("http-timeout", DEFAULT_HTTP_READ_BUFFER_TIMEOUT); max_server_connections = config().getUInt("max-server-connections", 1024); keep_alive_timeout = config().getUInt64("keep-alive-timeout", 10); + http_max_field_value_size = config().getUInt64("http-max-field-value-size", 1048576); struct rlimit limit; const UInt64 gb = 1024 * 1024 * 1024; @@ -226,6 +231,10 @@ int IBridge::main(const std::vector & /*args*/) auto context = Context::createGlobal(shared_context.get()); context->makeGlobalContext(); + auto settings = context->getSettings(); + settings.set("http_max_field_value_size", http_max_field_value_size); + context->setSettings(settings); + if (config().has("query_masking_rules")) SensitiveDataMasker::setInstance(std::make_unique(config(), "query_masking_rules")); diff --git a/src/Bridge/IBridge.h b/src/Bridge/IBridge.h index 68af8860cb3..e6855b69ddd 100644 --- a/src/Bridge/IBridge.h +++ b/src/Bridge/IBridge.h @@ -45,6 +45,7 @@ private: std::string log_level; unsigned max_server_connections; size_t http_timeout; + size_t http_max_field_value_size; Poco::Logger * log; }; diff --git a/src/BridgeHelper/IBridgeHelper.cpp b/src/BridgeHelper/IBridgeHelper.cpp index 3a8a8b8fdd2..844e2505e9c 100644 --- a/src/BridgeHelper/IBridgeHelper.cpp +++ b/src/BridgeHelper/IBridgeHelper.cpp @@ -67,6 +67,8 @@ std::unique_ptr IBridgeHelper::startBridgeCommand() cmd_args.push_back(config.getString(configPrefix() + ".listen_host", DEFAULT_HOST)); cmd_args.push_back("--http-timeout"); cmd_args.push_back(std::to_string(getHTTPTimeout().totalMicroseconds())); + cmd_args.push_back("--http-max-field-value-size"); + cmd_args.push_back("99999999999999999"); // something "big" to accept large datasets (issue 47616) if (config.has("logger." + configPrefix() + "_log")) { cmd_args.push_back("--log-path"); diff --git a/tests/integration/test_catboost_evaluate/test.py b/tests/integration/test_catboost_evaluate/test.py index a0915977ab6..0aeffdb3e02 100644 --- a/tests/integration/test_catboost_evaluate/test.py +++ b/tests/integration/test_catboost_evaluate/test.py @@ -279,7 +279,7 @@ def testAmazonModelManyRows(ch_cluster): ) result = instance.query( - "insert into amazon select number % 256, number, number, number, number, number, number, number, number, number from numbers(7500)" + "insert into amazon select number % 256, number, number, number, number, number, number, number, number, number from numbers(750000)" ) # First compute prediction, then as a very crude way to fingerprint and compare the result: sum and floor From 0712627c058cb924676ea741cbe9a2f29a405730 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 16 Mar 2023 15:47:01 +0000 Subject: [PATCH 354/418] Query cache: Fix config naming ambiguity Add 'max_' prefix to 'size' setting to make it more clear that the cache consumes no space if not used. --- docs/en/operations/query-cache.md | 4 ++-- .../en/operations/server-configuration-parameters/settings.md | 4 ++-- programs/server/config.xml | 2 +- src/Interpreters/Cache/QueryCache.cpp | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index 1a486de7904..a5afee63e6e 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -85,8 +85,8 @@ make the matching more natural, all query-level settings related to the query ca If the query was aborted due to an exception or user cancellation, no entry is written into the query cache. -The size of the query cache, the maximum number of cache entries and the maximum size of cache entries (in bytes and in records) can -be configured using different [server configuration options](server-configuration-parameters/settings.md#server_configuration_parameters_query-cache). +The size of the query cache in bytes, the maximum number of cache entries and the maximum size of individual cache entries (in bytes and in +records) can be configured using different [server configuration options](server-configuration-parameters/settings.md#server_configuration_parameters_query-cache). To define how long a query must run at least such that its result can be cached, you can use setting [query_cache_min_query_duration](settings/settings.md#query-cache-min-query-duration). For example, the result of query diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 08be318f334..f7fe0072e33 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1361,7 +1361,7 @@ If the table does not exist, ClickHouse will create it. If the structure of the The following settings are available: -- `size`: The maximum cache size in bytes. 0 means the query cache is disabled. Default value: `1073741824` (1 GiB). +- `max_size`: The maximum cache size in bytes. 0 means the query cache is disabled. Default value: `1073741824` (1 GiB). - `max_entries`: The maximum number of `SELECT` query results stored in the cache. Default value: `1024`. - `max_entry_size`: The maximum size in bytes `SELECT` query results may have to be saved in the cache. Default value: `1048576` (1 MiB). - `max_entry_rows`: The maximum number of rows `SELECT` query results may have to be saved in the cache. Default value: `30000000` (30 mil). @@ -1369,7 +1369,7 @@ The following settings are available: Changed settings take effect immediately. :::warning -Data for the query cache is allocated in DRAM. If memory is scarce, make sure to set a small value for `size` or disable the query cache altogether. +Data for the query cache is allocated in DRAM. If memory is scarce, make sure to set a small value for `max_size` or disable the query cache altogether. ::: **Example** diff --git a/programs/server/config.xml b/programs/server/config.xml index 212c6401503..cfcd2ff93e0 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1517,7 +1517,7 @@ - + diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index ce2373a8af9..4f50b689a31 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -316,7 +316,7 @@ void QueryCache::updateConfiguration(const Poco::Util::AbstractConfiguration & c { std::lock_guard lock(mutex); - size_t max_size_in_bytes = config.getUInt64("query_cache.size", 1_GiB); + size_t max_size_in_bytes = config.getUInt64("query_cache.max_size", 1_GiB); cache.setMaxSize(max_size_in_bytes); size_t max_entries = config.getUInt64("query_cache.max_entries", 1024); From 3d21b67895261eecd939371779bc857fc014a986 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 24 Mar 2023 13:02:41 +0000 Subject: [PATCH 355/418] Automatic style fix --- docker/test/performance-comparison/perf.py | 2 ++ docker/test/performance-comparison/report.py | 1 - tests/ci/clickhouse_helper.py | 1 - tests/ci/docker_images_check.py | 1 - tests/ci/get_previous_release_tag.py | 1 - tests/ci/report.py | 2 +- tests/integration/helpers/cluster.py | 3 +-- tests/integration/helpers/network.py | 2 -- .../pytest_xdist_logging_to_separate_files.py | 1 + .../test_detach_part_wrong_partition_id.py | 1 - .../test_cluster_copier/test_three_nodes.py | 1 - .../test_cluster_copier/test_two_nodes.py | 1 - tests/integration/test_composable_protocols/test.py | 1 - .../test_create_query_constraints/test.py | 2 -- .../common.py | 1 - tests/integration/test_disks_app_func/test.py | 1 - .../test_distributed_ddl_parallel/test.py | 1 + tests/integration/test_fetch_memory_usage/test.py | 1 - .../scripts/stress_test.py | 1 - tests/integration/test_jbod_balancer/test.py | 1 - .../test_keeper_and_access_storage/test.py | 1 + tests/integration/test_keeper_back_to_back/test.py | 2 +- tests/integration/test_keeper_persistent_log/test.py | 1 - .../test_keeper_zookeeper_converter/test.py | 1 - tests/integration/test_merge_tree_load_parts/test.py | 6 +++--- .../s3_endpoint/endpoint.py | 1 - .../test_merge_tree_settings_constraints/test.py | 1 - .../test_old_parts_finally_removed/test.py | 1 - tests/integration/test_partition/test.py | 4 +++- tests/integration/test_password_constraints/test.py | 1 - tests/integration/test_read_only_table/test.py | 1 - .../test_reload_auxiliary_zookeepers/test.py | 1 - .../s3_endpoint/endpoint.py | 1 + tests/integration/test_s3_with_proxy/test.py | 1 + .../integration/test_ssl_cert_authentication/test.py | 1 - tests/integration/test_storage_kafka/kafka_pb2.py | 1 - .../test_storage_kafka/message_with_repeated_pb2.py | 1 - tests/integration/test_storage_kafka/social_pb2.py | 1 - tests/integration/test_storage_kafka/test.py | 12 ++---------- tests/integration/test_storage_nats/nats_pb2.py | 1 - .../test_storage_postgresql_replica/test.py | 1 - .../test_storage_rabbitmq/rabbitmq_pb2.py | 1 - tests/integration/test_storage_rabbitmq/test.py | 3 --- tests/integration/test_storage_s3/test.py | 1 + .../test_storage_s3/test_invalid_env_credentials.py | 1 + tests/integration/test_system_merges/test.py | 1 - tests/integration/test_ttl_move/test.py | 2 +- tests/integration/test_zero_copy_fetch/test.py | 1 - utils/changelog-simple/format-changelog.py | 1 + utils/keeper-overload/keeper-overload.py | 2 +- 50 files changed, 23 insertions(+), 57 deletions(-) diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 65bf49c2914..7a4e6386d0d 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -26,6 +26,7 @@ logging.basicConfig( total_start_seconds = time.perf_counter() stage_start_seconds = total_start_seconds + # Thread executor that does not hides exception that happens during function # execution, and rethrows it after join() class SafeThread(Thread): @@ -158,6 +159,7 @@ for e in subst_elems: available_parameters[name] = values + # Takes parallel lists of templates, substitutes them with all combos of # parameters. The set of parameters is determined based on the first list. # Note: keep the order of queries -- sometimes we have DROP IF EXISTS diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index 782cf29863c..214f2d550b4 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -670,7 +670,6 @@ if args.report == "main": ) elif args.report == "all-queries": - print((header_template.format())) add_tested_commits() diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index d60a9e6afd1..64b64896f66 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -141,7 +141,6 @@ def prepare_tests_results_for_clickhouse( report_url: str, check_name: str, ) -> List[dict]: - pull_request_url = "https://github.com/ClickHouse/ClickHouse/commits/master" base_ref = "master" head_ref = "master" diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 192d216614e..f2b1105b3b0 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -96,7 +96,6 @@ def get_images_dict(repo_path: str, image_file_path: str) -> ImagesDict: def get_changed_docker_images( pr_info: PRInfo, images_dict: ImagesDict ) -> Set[DockerImage]: - if not images_dict: return set() diff --git a/tests/ci/get_previous_release_tag.py b/tests/ci/get_previous_release_tag.py index c6fe6cd5fb5..c2d279f7fec 100755 --- a/tests/ci/get_previous_release_tag.py +++ b/tests/ci/get_previous_release_tag.py @@ -51,7 +51,6 @@ def find_previous_release( for release in releases: if release.version < server_version: - # Check if the artifact exists on GitHub. # It can be not true for a short period of time # after creating a tag for a new release before uploading the packages. diff --git a/tests/ci/report.py b/tests/ci/report.py index 947fb33d905..ddee035d26f 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -473,7 +473,7 @@ def create_build_html_report( commit_url: str, ) -> str: rows = "" - for (build_result, build_log_url, artifact_urls) in zip( + for build_result, build_log_url, artifact_urls in zip( build_results, build_logs_urls, artifact_urls_list ): row = "" diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index dc5ada81995..a9a996e0a5f 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -63,6 +63,7 @@ DEFAULT_ENV_NAME = ".env" SANITIZER_SIGN = "==================" + # to create docker-compose env file def _create_env_file(path, variables): logging.debug(f"Env {variables} stored in {path}") @@ -1454,7 +1455,6 @@ class ClickHouseCluster: config_root_name="clickhouse", extra_configs=[], ) -> "ClickHouseInstance": - """Add an instance to the cluster. name - the name of the instance directory and the value of the 'instance' macro in ClickHouse. @@ -3089,7 +3089,6 @@ class ClickHouseInstance: config_root_name="clickhouse", extra_configs=[], ): - self.name = name self.base_cmd = cluster.base_cmd self.docker_id = cluster.get_instance_docker_id(self.name) diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index e408c9beec1..471aa2bdc2e 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -216,7 +216,6 @@ class _NetworkManager: container_exit_timeout=60, docker_api_version=os.environ.get("DOCKER_API_VERSION"), ): - self.container_expire_timeout = container_expire_timeout self.container_exit_timeout = container_exit_timeout @@ -232,7 +231,6 @@ class _NetworkManager: def _ensure_container(self): if self._container is None or self._container_expire_time <= time.time(): - for i in range(5): if self._container is not None: try: diff --git a/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py b/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py index d424ad58fa4..370aa23a014 100644 --- a/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py +++ b/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py @@ -1,6 +1,7 @@ import logging import os.path + # Makes the parallel workers of pytest-xdist to log to separate files. # Without this function all workers will log to the same log file # and mix everything together making it much more difficult for troubleshooting. diff --git a/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py b/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py index 02fccfae4e5..a6f7a8653da 100644 --- a/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py +++ b/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py @@ -24,7 +24,6 @@ def start_cluster(): def test_detach_part_wrong_partition_id(start_cluster): - # Here we create table with partition by UUID. node_21_6.query( "create table tab (id UUID, value UInt32) engine = MergeTree PARTITION BY (id) order by tuple()" diff --git a/tests/integration/test_cluster_copier/test_three_nodes.py b/tests/integration/test_cluster_copier/test_three_nodes.py index 31d6c0448f4..e7d07757adb 100644 --- a/tests/integration/test_cluster_copier/test_three_nodes.py +++ b/tests/integration/test_cluster_copier/test_three_nodes.py @@ -19,7 +19,6 @@ cluster = ClickHouseCluster(__file__) def started_cluster(): global cluster try: - for name in ["first", "second", "third"]: cluster.add_instance( name, diff --git a/tests/integration/test_cluster_copier/test_two_nodes.py b/tests/integration/test_cluster_copier/test_two_nodes.py index 10ab7d03b00..2b6fcf6cac2 100644 --- a/tests/integration/test_cluster_copier/test_two_nodes.py +++ b/tests/integration/test_cluster_copier/test_two_nodes.py @@ -19,7 +19,6 @@ cluster = ClickHouseCluster(__file__) def started_cluster(): global cluster try: - for name in ["first_of_two", "second_of_two"]: instance = cluster.add_instance( name, diff --git a/tests/integration/test_composable_protocols/test.py b/tests/integration/test_composable_protocols/test.py index bc87fea5296..df74cfffa54 100644 --- a/tests/integration/test_composable_protocols/test.py +++ b/tests/integration/test_composable_protocols/test.py @@ -63,7 +63,6 @@ def netcat(hostname, port, content): def test_connections(): - client = Client(server.ip_address, 9000, command=cluster.client_bin_path) assert client.query("SELECT 1") == "1\n" diff --git a/tests/integration/test_create_query_constraints/test.py b/tests/integration/test_create_query_constraints/test.py index 8df043fd24b..33c41b4f161 100644 --- a/tests/integration/test_create_query_constraints/test.py +++ b/tests/integration/test_create_query_constraints/test.py @@ -25,7 +25,6 @@ def start_cluster(): def test_create_query_const_constraints(): - instance.query("CREATE USER u_const SETTINGS max_threads = 1 CONST") instance.query("GRANT ALL ON *.* TO u_const") @@ -57,7 +56,6 @@ def test_create_query_const_constraints(): def test_create_query_minmax_constraints(): - instance.query("CREATE USER u_minmax SETTINGS max_threads = 4 MIN 2 MAX 6") instance.query("GRANT ALL ON *.* TO u_minmax") diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py index b38e81b0227..01addae2542 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py @@ -348,7 +348,6 @@ class RangedLayoutTester(BaseLayoutTester): self.layouts = LAYOUTS_RANGED def execute(self, layout_name, node): - if layout_name not in self.layout_to_dictionary: raise RuntimeError("Source doesn't support layout: {}".format(layout_name)) diff --git a/tests/integration/test_disks_app_func/test.py b/tests/integration/test_disks_app_func/test.py index 027ef8feed0..2428c53854e 100644 --- a/tests/integration/test_disks_app_func/test.py +++ b/tests/integration/test_disks_app_func/test.py @@ -7,7 +7,6 @@ import pytest def started_cluster(): global cluster try: - cluster = ClickHouseCluster(__file__) cluster.add_instance( "disks_app_test", main_configs=["config.xml"], with_minio=True diff --git a/tests/integration/test_distributed_ddl_parallel/test.py b/tests/integration/test_distributed_ddl_parallel/test.py index 6ebfe472e09..eb98dd3e230 100644 --- a/tests/integration/test_distributed_ddl_parallel/test.py +++ b/tests/integration/test_distributed_ddl_parallel/test.py @@ -10,6 +10,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) + # By default the exceptions that was throwed in threads will be ignored # (they will not mark the test as failed, only printed to stderr). # diff --git a/tests/integration/test_fetch_memory_usage/test.py b/tests/integration/test_fetch_memory_usage/test.py index a4371140150..7591cc0e8a9 100644 --- a/tests/integration/test_fetch_memory_usage/test.py +++ b/tests/integration/test_fetch_memory_usage/test.py @@ -18,7 +18,6 @@ def started_cluster(): def test_huge_column(started_cluster): - if ( node.is_built_with_thread_sanitizer() or node.is_built_with_memory_sanitizer() diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py index b8bafb3d0c1..fe69d72c1c7 100644 --- a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py +++ b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py @@ -13,7 +13,6 @@ number_of_iterations = 100 def perform_request(): - buffer = BytesIO() crl = pycurl.Curl() crl.setopt(pycurl.INTERFACE, client_ip) diff --git a/tests/integration/test_jbod_balancer/test.py b/tests/integration/test_jbod_balancer/test.py index e746698611a..df34a075d5a 100644 --- a/tests/integration/test_jbod_balancer/test.py +++ b/tests/integration/test_jbod_balancer/test.py @@ -45,7 +45,6 @@ def start_cluster(): def check_balance(node, table): - partitions = node.query( """ WITH diff --git a/tests/integration/test_keeper_and_access_storage/test.py b/tests/integration/test_keeper_and_access_storage/test.py index 6ec307f7082..0314825b6b7 100644 --- a/tests/integration/test_keeper_and_access_storage/test.py +++ b/tests/integration/test_keeper_and_access_storage/test.py @@ -10,6 +10,7 @@ node1 = cluster.add_instance( "node1", main_configs=["configs/keeper.xml"], stay_alive=True ) + # test that server is able to start @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_keeper_back_to_back/test.py b/tests/integration/test_keeper_back_to_back/test.py index 73fface02b4..b737ac284d2 100644 --- a/tests/integration/test_keeper_back_to_back/test.py +++ b/tests/integration/test_keeper_back_to_back/test.py @@ -546,7 +546,6 @@ def test_random_requests(started_cluster): def test_end_of_session(started_cluster): - fake_zk1 = None fake_zk2 = None genuine_zk1 = None @@ -685,6 +684,7 @@ def test_concurrent_watches(started_cluster): nonlocal watches_created nonlocal all_paths_created fake_zk.ensure_path(global_path + "/" + str(i)) + # new function each time def dumb_watch(event): nonlocal dumb_watch_triggered_counter diff --git a/tests/integration/test_keeper_persistent_log/test.py b/tests/integration/test_keeper_persistent_log/test.py index 70cc14fe26d..4164ffb33d3 100644 --- a/tests/integration/test_keeper_persistent_log/test.py +++ b/tests/integration/test_keeper_persistent_log/test.py @@ -163,7 +163,6 @@ def test_state_duplicate_restart(started_cluster): # http://zookeeper-user.578899.n2.nabble.com/Why-are-ephemeral-nodes-written-to-disk-tp7583403p7583418.html def test_ephemeral_after_restart(started_cluster): - try: node_zk = None node_zk2 = None diff --git a/tests/integration/test_keeper_zookeeper_converter/test.py b/tests/integration/test_keeper_zookeeper_converter/test.py index 063421bf922..de5a9416119 100644 --- a/tests/integration/test_keeper_zookeeper_converter/test.py +++ b/tests/integration/test_keeper_zookeeper_converter/test.py @@ -114,7 +114,6 @@ def start_clickhouse(): def copy_zookeeper_data(make_zk_snapshots): - if make_zk_snapshots: # force zookeeper to create snapshot generate_zk_snapshot() else: diff --git a/tests/integration/test_merge_tree_load_parts/test.py b/tests/integration/test_merge_tree_load_parts/test.py index 777b6f14fc6..dfbe00c8e28 100644 --- a/tests/integration/test_merge_tree_load_parts/test.py +++ b/tests/integration/test_merge_tree_load_parts/test.py @@ -148,17 +148,17 @@ def test_merge_tree_load_parts_corrupted(started_cluster): node1.query("SYSTEM WAIT LOADING PARTS mt_load_parts_2") def check_parts_loading(node, partition, loaded, failed, skipped): - for (min_block, max_block) in loaded: + for min_block, max_block in loaded: part_name = f"{partition}_{min_block}_{max_block}" assert node.contains_in_log(f"Loading Active part {part_name}") assert node.contains_in_log(f"Finished loading Active part {part_name}") - for (min_block, max_block) in failed: + for min_block, max_block in failed: part_name = f"{partition}_{min_block}_{max_block}" assert node.contains_in_log(f"Loading Active part {part_name}") assert not node.contains_in_log(f"Finished loading Active part {part_name}") - for (min_block, max_block) in skipped: + for min_block, max_block in skipped: part_name = f"{partition}_{min_block}_{max_block}" assert not node.contains_in_log(f"Loading Active part {part_name}") assert not node.contains_in_log(f"Finished loading Active part {part_name}") diff --git a/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py b/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py index b6567dfebc5..4613fdb850b 100644 --- a/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py +++ b/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py @@ -42,7 +42,6 @@ def delete(_bucket): @route("/<_bucket>/<_path:path>", ["GET", "POST", "PUT", "DELETE"]) def server(_bucket, _path): - # It's delete query for failed part if _path.endswith("delete"): response.set_header("Location", "http://minio1:9001/" + _bucket + "/" + _path) diff --git a/tests/integration/test_merge_tree_settings_constraints/test.py b/tests/integration/test_merge_tree_settings_constraints/test.py index 0bb0179108d..be6e2a31873 100644 --- a/tests/integration/test_merge_tree_settings_constraints/test.py +++ b/tests/integration/test_merge_tree_settings_constraints/test.py @@ -20,7 +20,6 @@ def start_cluster(): def test_merge_tree_settings_constraints(): - assert "Setting storage_policy should not be changed" in instance.query_and_get_error( f"CREATE TABLE wrong_table (number Int64) engine = MergeTree() ORDER BY number SETTINGS storage_policy = 'secret_policy'" ) diff --git a/tests/integration/test_old_parts_finally_removed/test.py b/tests/integration/test_old_parts_finally_removed/test.py index 108b72c5ccd..5347d433419 100644 --- a/tests/integration/test_old_parts_finally_removed/test.py +++ b/tests/integration/test_old_parts_finally_removed/test.py @@ -63,7 +63,6 @@ def test_part_finally_removed(started_cluster): ) for i in range(60): - if ( node1.query( "SELECT count() from system.parts WHERE table = 'drop_outdated_part'" diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index ae4393fc6f6..a34141c6189 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -528,7 +528,9 @@ def test_make_clone_in_detached(started_cluster): ["cp", "-r", path + "all_0_0_0", path + "detached/broken_all_0_0_0"] ) assert_eq_with_retry(instance, "select * from clone_in_detached", "\n") - assert ["broken_all_0_0_0",] == sorted( + assert [ + "broken_all_0_0_0", + ] == sorted( instance.exec_in_container(["ls", path + "detached/"]).strip().split("\n") ) diff --git a/tests/integration/test_password_constraints/test.py b/tests/integration/test_password_constraints/test.py index e3628861b28..9cdff51caa1 100644 --- a/tests/integration/test_password_constraints/test.py +++ b/tests/integration/test_password_constraints/test.py @@ -17,7 +17,6 @@ def start_cluster(): def test_complexity_rules(start_cluster): - error_message = "DB::Exception: Invalid password. The password should: be at least 12 characters long, contain at least 1 numeric character, contain at least 1 lowercase character, contain at least 1 uppercase character, contain at least 1 special character" assert error_message in node.query_and_get_error( "CREATE USER u_1 IDENTIFIED WITH plaintext_password BY ''" diff --git a/tests/integration/test_read_only_table/test.py b/tests/integration/test_read_only_table/test.py index 914c6a99508..df084f9dbbd 100644 --- a/tests/integration/test_read_only_table/test.py +++ b/tests/integration/test_read_only_table/test.py @@ -49,7 +49,6 @@ def start_cluster(): def test_restart_zookeeper(start_cluster): - for table_id in range(NUM_TABLES): node1.query( f"INSERT INTO test_table_{table_id} VALUES (1), (2), (3), (4), (5);" diff --git a/tests/integration/test_reload_auxiliary_zookeepers/test.py b/tests/integration/test_reload_auxiliary_zookeepers/test.py index bb1455333fc..476c5dee99e 100644 --- a/tests/integration/test_reload_auxiliary_zookeepers/test.py +++ b/tests/integration/test_reload_auxiliary_zookeepers/test.py @@ -20,7 +20,6 @@ def start_cluster(): def test_reload_auxiliary_zookeepers(start_cluster): - node.query( "CREATE TABLE simple (date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', 'node') ORDER BY tuple() PARTITION BY date;" ) diff --git a/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py b/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py index d6a732cc681..1d33ca02f86 100644 --- a/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py +++ b/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py @@ -1,6 +1,7 @@ #!/usr/bin/env python3 from bottle import request, route, run, response + # Handle for MultipleObjectsDelete. @route("/<_bucket>", ["POST"]) def delete(_bucket): diff --git a/tests/integration/test_s3_with_proxy/test.py b/tests/integration/test_s3_with_proxy/test.py index 1102d190a87..1af040c3c30 100644 --- a/tests/integration/test_s3_with_proxy/test.py +++ b/tests/integration/test_s3_with_proxy/test.py @@ -5,6 +5,7 @@ import time import pytest from helpers.cluster import ClickHouseCluster + # Runs simple proxy resolver in python env container. def run_resolver(cluster): container_id = cluster.get_container_id("resolver") diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index 7c62ca0d8b6..b3570b6e281 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -87,7 +87,6 @@ config = """ def execute_query_native(node, query, user, cert_name): - config_path = f"{SCRIPT_DIR}/configs/client.xml" formatted = config.format( diff --git a/tests/integration/test_storage_kafka/kafka_pb2.py b/tests/integration/test_storage_kafka/kafka_pb2.py index 7de1363bbf1..3e47af6c1e0 100644 --- a/tests/integration/test_storage_kafka/kafka_pb2.py +++ b/tests/integration/test_storage_kafka/kafka_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.kafka_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _KEYVALUEPAIR._serialized_start = 46 _KEYVALUEPAIR._serialized_end = 88 diff --git a/tests/integration/test_storage_kafka/message_with_repeated_pb2.py b/tests/integration/test_storage_kafka/message_with_repeated_pb2.py index 4d1a23c0b43..3715a9bea04 100644 --- a/tests/integration/test_storage_kafka/message_with_repeated_pb2.py +++ b/tests/integration/test_storage_kafka/message_with_repeated_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.message_with_repeated_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None DESCRIPTOR._serialized_options = b"H\001" _MESSAGE._serialized_start = 62 diff --git a/tests/integration/test_storage_kafka/social_pb2.py b/tests/integration/test_storage_kafka/social_pb2.py index 830ade81d33..f91a7bd0539 100644 --- a/tests/integration/test_storage_kafka/social_pb2.py +++ b/tests/integration/test_storage_kafka/social_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.social_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _USER._serialized_start = 47 _USER._serialized_end = 90 diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 51952ac1eb7..3a4fa6c6bfe 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -121,7 +121,7 @@ def kafka_create_topic( def kafka_delete_topic(admin_client, topic, max_retries=50): result = admin_client.delete_topics([topic]) - for (topic, e) in result.topic_error_codes: + for topic, e in result.topic_error_codes: if e == 0: logging.debug(f"Topic {topic} deleted") else: @@ -917,9 +917,7 @@ def describe_consumer_group(kafka_cluster, name): member_info["client_id"] = client_id member_info["client_host"] = client_host member_topics_assignment = [] - for (topic, partitions) in MemberAssignment.decode( - member_assignment - ).assignment: + for topic, partitions in MemberAssignment.decode(member_assignment).assignment: member_topics_assignment.append({"topic": topic, "partitions": partitions}) member_info["assignment"] = member_topics_assignment res.append(member_info) @@ -1537,7 +1535,6 @@ def test_kafka_protobuf_no_delimiter(kafka_cluster): def test_kafka_materialized_view(kafka_cluster): - instance.query( """ DROP TABLE IF EXISTS test.view; @@ -2315,7 +2312,6 @@ def test_kafka_virtual_columns2(kafka_cluster): def test_kafka_produce_key_timestamp(kafka_cluster): - admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) @@ -2444,7 +2440,6 @@ def test_kafka_insert_avro(kafka_cluster): def test_kafka_produce_consume_avro(kafka_cluster): - admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) @@ -4031,7 +4026,6 @@ def test_kafka_predefined_configuration(kafka_cluster): # https://github.com/ClickHouse/ClickHouse/issues/26643 def test_issue26643(kafka_cluster): - # for backporting: # admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") admin_client = KafkaAdminClient( @@ -4313,7 +4307,6 @@ def test_row_based_formats(kafka_cluster): "RowBinaryWithNamesAndTypes", "MsgPack", ]: - print(format_name) kafka_create_topic(admin_client, format_name) @@ -4438,7 +4431,6 @@ def test_block_based_formats_2(kafka_cluster): "ORC", "JSONCompactColumns", ]: - kafka_create_topic(admin_client, format_name) instance.query( diff --git a/tests/integration/test_storage_nats/nats_pb2.py b/tests/integration/test_storage_nats/nats_pb2.py index 4330ff57950..e9e5cb72363 100644 --- a/tests/integration/test_storage_nats/nats_pb2.py +++ b/tests/integration/test_storage_nats/nats_pb2.py @@ -31,7 +31,6 @@ ProtoKeyValue = _reflection.GeneratedProtocolMessageType( _sym_db.RegisterMessage(ProtoKeyValue) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _PROTOKEYVALUE._serialized_start = 45 _PROTOKEYVALUE._serialized_end = 88 diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 5df8b9029e6..8666d7ae58c 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -706,7 +706,6 @@ def test_abrupt_connection_loss_while_heavy_replication(started_cluster): def test_abrupt_server_restart_while_heavy_replication(started_cluster): - # FIXME (kssenii) temporary disabled if instance.is_built_with_sanitizer(): pytest.skip("Temporary disabled (FIXME)") diff --git a/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py b/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py index e017b4e66c2..a5845652eef 100644 --- a/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py +++ b/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.rabbitmq_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _KEYVALUEPROTO._serialized_start = 49 _KEYVALUEPROTO._serialized_end = 92 diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 2e54f21787a..53b6c4109ef 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -2864,7 +2864,6 @@ def test_rabbitmq_predefined_configuration(rabbitmq_cluster): def test_rabbitmq_msgpack(rabbitmq_cluster): - instance.query( """ drop table if exists rabbit_in; @@ -2908,7 +2907,6 @@ def test_rabbitmq_msgpack(rabbitmq_cluster): def test_rabbitmq_address(rabbitmq_cluster): - instance2.query( """ drop table if exists rabbit_in; @@ -3243,7 +3241,6 @@ def test_block_based_formats_2(rabbitmq_cluster): "ORC", "JSONCompactColumns", ]: - print(format_name) instance.query( diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 8b20727a7b5..4d493d9526b 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -18,6 +18,7 @@ MINIO_INTERNAL_PORT = 9001 SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + # Creates S3 bucket for tests and allows anonymous read-write access to it. def prepare_s3_bucket(started_cluster): # Allows read-write access for bucket without authorization. diff --git a/tests/integration/test_storage_s3/test_invalid_env_credentials.py b/tests/integration/test_storage_s3/test_invalid_env_credentials.py index 2f5d9349904..aa6479a2ed3 100644 --- a/tests/integration/test_storage_s3/test_invalid_env_credentials.py +++ b/tests/integration/test_storage_s3/test_invalid_env_credentials.py @@ -11,6 +11,7 @@ MINIO_INTERNAL_PORT = 9001 SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + # Creates S3 bucket for tests and allows anonymous read-write access to it. def prepare_s3_bucket(started_cluster): # Allows read-write access for bucket without authorization. diff --git a/tests/integration/test_system_merges/test.py b/tests/integration/test_system_merges/test.py index 0a469bd7bbd..ff303afe19e 100644 --- a/tests/integration/test_system_merges/test.py +++ b/tests/integration/test_system_merges/test.py @@ -171,7 +171,6 @@ def test_mutation_simple(started_cluster, replicated): starting_block = 0 if replicated else 1 try: - for node in nodes: node.query( f"create table {name} (a Int64) engine={engine} order by tuple()" diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 99978cbf6dc..89824293320 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1863,7 +1863,7 @@ def test_ttl_move_if_exists(started_cluster, name, dest_type): ) ) - for (node, policy) in zip( + for node, policy in zip( [node1, node2], ["only_jbod_1", "small_jbod_with_external"] ): node.query( diff --git a/tests/integration/test_zero_copy_fetch/test.py b/tests/integration/test_zero_copy_fetch/test.py index b71752528d3..9b9aa5e0da7 100644 --- a/tests/integration/test_zero_copy_fetch/test.py +++ b/tests/integration/test_zero_copy_fetch/test.py @@ -16,7 +16,6 @@ cluster = ClickHouseCluster(__file__) @pytest.fixture(scope="module") def started_cluster(): try: - cluster.add_instance( "node1", main_configs=["configs/storage_conf.xml"], diff --git a/utils/changelog-simple/format-changelog.py b/utils/changelog-simple/format-changelog.py index d5e1518270e..01f2694dd0f 100755 --- a/utils/changelog-simple/format-changelog.py +++ b/utils/changelog-simple/format-changelog.py @@ -20,6 +20,7 @@ parser.add_argument( ) args = parser.parse_args() + # This function mirrors the PR description checks in ClickhousePullRequestTrigger. # Returns False if the PR should not be mentioned changelog. def parse_one_pull_request(item): diff --git a/utils/keeper-overload/keeper-overload.py b/utils/keeper-overload/keeper-overload.py index bdb4563c713..0a059b10588 100755 --- a/utils/keeper-overload/keeper-overload.py +++ b/utils/keeper-overload/keeper-overload.py @@ -166,7 +166,7 @@ def main(args): keeper_bench_path = args.keeper_bench_path keepers = [] - for (port, server_id) in zip(PORTS, SERVER_IDS): + for port, server_id in zip(PORTS, SERVER_IDS): keepers.append( Keeper( keeper_binary_path, server_id, port, workdir, args.with_thread_fuzzer From 6df169dce346ed41812291f247310acfd0196aa4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=BE=D1=80=D0=B5=D0=BD=D0=B1=D0=B5=D1=80=D0=B3=20?= =?UTF-8?q?=D0=9C=D0=B0=D1=80=D0=BA?= Date: Fri, 24 Mar 2023 16:04:09 +0300 Subject: [PATCH 356/418] Fix #36971: Watchdog: exit with non-zero code if child process exits --- src/Daemon/BaseDaemon.cpp | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index e050124e497..345d6e765fe 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -1125,16 +1125,21 @@ void BaseDaemon::setupWatchdog() logger().information("Child process no longer exists."); _exit(WEXITSTATUS(status)); } - else if (WIFEXITED(status)) + + if (WIFEXITED(status)) { logger().information(fmt::format("Child process exited normally with code {}.", WEXITSTATUS(status))); _exit(WEXITSTATUS(status)); } + int exit_code; + if (WIFSIGNALED(status)) { int sig = WTERMSIG(status); + exit_code = 128 + sig; + if (sig == SIGKILL) { logger().fatal(fmt::format("Child process was terminated by signal {} (KILL)." @@ -1146,12 +1151,14 @@ void BaseDaemon::setupWatchdog() logger().fatal(fmt::format("Child process was terminated by signal {}.", sig)); if (sig == SIGINT || sig == SIGTERM || sig == SIGQUIT) - _exit(128 + sig); + _exit(exit_code); } } else { + // According to POSIX, this should never happen. logger().fatal("Child process was not exited normally by unknown reason."); + exit_code = 42; } if (restart) @@ -1161,7 +1168,7 @@ void BaseDaemon::setupWatchdog() memcpy(argv0, original_process_name.c_str(), original_process_name.size()); } else - _exit(WEXITSTATUS(status)); + _exit(exit_code); } } From 48a99e5491b9f4021d9303353d0d2ec56d08f0b1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 24 Mar 2023 13:28:32 +0000 Subject: [PATCH 357/418] Fix flaky tests due to time zone randomization --- .../02668_parse_datetime_in_joda_syntax.reference | 14 +++++++------- .../02668_parse_datetime_in_joda_syntax.sql | 14 +++++++------- 2 files changed, 14 insertions(+), 14 deletions(-) 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 e8cc31944c3..90515055436 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 @@ -3,19 +3,19 @@ select parseDateTimeInJodaSyntax(' ', ' ', 'UTC') = toDateTime('1970-01-01', 'UTC'); 1 -- era -select parseDateTimeInJodaSyntax('AD 1999', 'G YYYY') = toDateTime('1999-01-01'); +select parseDateTimeInJodaSyntax('AD 1999', 'G YYYY') = toDateTime('1999-01-01', 'UTC'); 1 -select parseDateTimeInJodaSyntax('ad 1999', 'G YYYY') = toDateTime('1999-01-01'); +select parseDateTimeInJodaSyntax('ad 1999', 'G YYYY') = toDateTime('1999-01-01', 'UTC'); 1 -select parseDateTimeInJodaSyntax('Ad 1999', 'G YYYY') = toDateTime('1999-01-01'); +select parseDateTimeInJodaSyntax('Ad 1999', 'G YYYY') = toDateTime('1999-01-01', 'UTC'); 1 -select parseDateTimeInJodaSyntax('AD 1999', 'G YYYY') = toDateTime('1999-01-01'); +select parseDateTimeInJodaSyntax('AD 1999', 'G YYYY') = toDateTime('1999-01-01', 'UTC'); 1 -select parseDateTimeInJodaSyntax('AD 1999', 'G yyyy') = toDateTime('1999-01-01'); +select parseDateTimeInJodaSyntax('AD 1999', 'G yyyy') = toDateTime('1999-01-01', 'UTC'); 1 -select parseDateTimeInJodaSyntax('AD 1999 2000', 'G YYYY yyyy') = toDateTime('2000-01-01'); +select parseDateTimeInJodaSyntax('AD 1999 2000', 'G YYYY yyyy') = toDateTime('2000-01-01', 'UTC'); 1 -select parseDateTimeInJodaSyntax('AD 1999 2000', 'G yyyy YYYY') = toDateTime('2000-01-01'); +select parseDateTimeInJodaSyntax('AD 1999 2000', 'G yyyy YYYY') = toDateTime('2000-01-01', 'UTC'); 1 select parseDateTimeInJodaSyntax('AD 1999', 'G Y'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTimeInJodaSyntax('AD 1999', 'G YY'); -- { serverError CANNOT_PARSE_DATETIME } 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 99426a8c8d8..ae5aff64416 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 @@ -3,13 +3,13 @@ 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 YYYY') = toDateTime('1999-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('ad 1999', 'G YYYY') = toDateTime('1999-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('Ad 1999', 'G YYYY') = toDateTime('1999-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('AD 1999', 'G YYYY') = toDateTime('1999-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('AD 1999', 'G yyyy') = toDateTime('1999-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('AD 1999 2000', 'G YYYY yyyy') = toDateTime('2000-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('AD 1999 2000', 'G yyyy YYYY') = toDateTime('2000-01-01', 'UTC'); 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 } From afe3af230ff9b99e96d9f3283a7245d6108e83f9 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 23 Mar 2023 00:32:58 +0100 Subject: [PATCH 358/418] Limit keeper request batching by size in bytes --- src/Coordination/CoordinationSettings.cpp | 2 ++ src/Coordination/CoordinationSettings.h | 3 ++- src/Coordination/KeeperDispatcher.cpp | 18 +++++++++++++++--- .../test_keeper_four_word_command/test.py | 3 ++- 4 files changed, 21 insertions(+), 5 deletions(-) diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index 5e1ac1e2d7f..7a66134f43f 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -140,6 +140,8 @@ void KeeperConfigurationAndSettings::dump(WriteBufferFromOwnString & buf) const writeText("max_requests_batch_size=", buf); write_int(coordination_settings->max_requests_batch_size); + writeText("max_requests_batch_bytes_size=", buf); + write_int(coordination_settings->max_requests_batch_bytes_size); writeText("max_request_queue_size=", buf); write_int(coordination_settings->max_request_queue_size); writeText("max_requests_quick_batch_size=", buf); diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index a16f5be1b01..90f66ccc09b 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -39,7 +39,8 @@ struct Settings; M(UInt64, fresh_log_gap, 200, "When node became fresh", 0) \ M(UInt64, max_request_queue_size, 100000, "Maximum number of request that can be in queue for processing", 0) \ M(UInt64, max_requests_batch_size, 100, "Max size of batch of requests that can be sent to RAFT", 0) \ - M(UInt64, max_requests_quick_batch_size, 10, "Max size of batch of requests to try to get before proceeding with RAFT. Keeper will not wait for requests but take only requests that are already in queue" , 0) \ + M(UInt64, max_requests_batch_bytes_size, 100*1024, "Max size in bytes of batch of requests that can be sent to RAFT", 0) \ + M(UInt64, max_requests_quick_batch_size, 100, "Max size of batch of requests to try to get before proceeding with RAFT. Keeper will not wait for requests but take only requests that are already in queue" , 0) \ M(Bool, quorum_reads, false, "Execute read requests as writes through whole RAFT consesus with similar speed", 0) \ M(Bool, force_sync, true, "Call fsync on each change in RAFT changelog", 0) \ M(Bool, compress_logs, true, "Write compressed coordination logs in ZSTD format", 0) \ diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 2aa11dd9eed..16d9fdcf5bd 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -73,6 +73,7 @@ void KeeperDispatcher::requestThread() auto coordination_settings = configuration_and_settings->coordination_settings; uint64_t max_wait = coordination_settings->operation_timeout_ms.totalMilliseconds(); uint64_t max_batch_size = coordination_settings->max_requests_batch_size; + uint64_t max_batch_bytes_size = coordination_settings->max_requests_batch_bytes_size; /// The code below do a very simple thing: batch all write (quorum) requests into vector until /// previous write batch is not finished or max_batch size achieved. The main complexity goes from @@ -89,6 +90,7 @@ void KeeperDispatcher::requestThread() break; KeeperStorage::RequestsForSessions current_batch; + size_t current_batch_bytes_size = 0; bool has_read_request = false; @@ -96,6 +98,7 @@ void KeeperDispatcher::requestThread() /// Otherwise we will process it locally. if (coordination_settings->quorum_reads || !request.request->isReadRequest()) { + current_batch_bytes_size += request.request->bytesSize(); current_batch.emplace_back(request); const auto try_get_request = [&] @@ -108,7 +111,10 @@ void KeeperDispatcher::requestThread() if (!coordination_settings->quorum_reads && request.request->isReadRequest()) has_read_request = true; else + { + current_batch_bytes_size += request.request->bytesSize(); current_batch.emplace_back(request); + } return true; } @@ -116,9 +122,11 @@ void KeeperDispatcher::requestThread() return false; }; - /// If we have enough requests in queue, we will try to batch at least max_quick_batch_size of them. + /// TODO: Deprecate max_requests_quick_batch_size and use only max_requests_batch_size and max_requests_batch_bytes_size size_t max_quick_batch_size = coordination_settings->max_requests_quick_batch_size; - while (!shutdown_called && !has_read_request && current_batch.size() < max_quick_batch_size && try_get_request()) + while (!shutdown_called && !has_read_request && + current_batch.size() < max_quick_batch_size && current_batch_bytes_size < max_batch_bytes_size && + try_get_request()) ; const auto prev_result_done = [&] @@ -129,7 +137,8 @@ void KeeperDispatcher::requestThread() }; /// Waiting until previous append will be successful, or batch is big enough - while (!shutdown_called && !has_read_request && !prev_result_done() && current_batch.size() <= max_batch_size) + while (!shutdown_called && !has_read_request && !prev_result_done() && + current_batch.size() <= max_batch_size && current_batch_bytes_size < max_batch_bytes_size) { try_get_request(); } @@ -147,6 +156,8 @@ void KeeperDispatcher::requestThread() /// Process collected write requests batch if (!current_batch.empty()) { + LOG_TRACE(log, "Processing requests batch, size: {}, bytes: {}", current_batch.size(), current_batch_bytes_size); + auto result = server->putRequestBatch(current_batch); if (result) @@ -158,6 +169,7 @@ void KeeperDispatcher::requestThread() { addErrorResponses(current_batch, Coordination::Error::ZCONNECTIONLOSS); current_batch.clear(); + current_batch_bytes_size = 0; } prev_batch = std::move(current_batch); diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 412780c8f0f..1a214b87445 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -285,8 +285,9 @@ def test_cmd_conf(started_cluster): assert result["fresh_log_gap"] == "200" assert result["max_requests_batch_size"] == "100" + assert result["max_requests_batch_bytes_size"] == "102400" assert result["max_request_queue_size"] == "100000" - assert result["max_requests_quick_batch_size"] == "10" + assert result["max_requests_quick_batch_size"] == "100" assert result["quorum_reads"] == "false" assert result["force_sync"] == "true" From 129245da2e6625b5f39d10ce6c6798117450c971 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 24 Mar 2023 13:52:56 +0000 Subject: [PATCH 359/418] Docs: Fix links --- docs/en/sql-reference/functions/date-time-functions.md | 4 ++-- .../sql-reference/functions/type-conversion-functions.md | 8 ++++++-- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 77ce47c1b6d..d06ab253cf7 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1238,7 +1238,7 @@ Formats a Time according to the given Format string. Format is a constant expres formatDateTime uses MySQL datetime format style, refer to https://dev.mysql.com/doc/refman/8.0/en/date-and-time-functions.html#function_date-format. -The opposite operation of this function is [formatDateTime](/docs/en/sql-reference/functions/type-conversion-functions.md#formatdatetime). +The opposite operation of this function is [parseDateTime](/docs/en/sql-reference/functions/type-conversion-functions.md#type_conversion_functions-parseDateTime). Alias: `DATE_FORMAT`. @@ -1334,7 +1334,7 @@ Result: 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. -The opposite operation of this function is [formatDateTimeInJodaSyntax](/docs/en/sql-reference/functions/type-conversion-functions.md#formatdatetimeinjodasyntax). +The opposite operation of this function is [parseDateTimeInJodaSyntax](/docs/en/sql-reference/functions/type-conversion-functions.md#type_conversion_functions-parseDateTimeInJodaSyntax). **Replacement fields** diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 878e97e8712..90f6cf0aa7d 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1148,9 +1148,10 @@ Result: └───────────────────────────┴──────────────────────────────┘ ``` -## parseDateTime +## parseDateTime {#type_conversion_functions-parseDateTime} Converts a [String](/docs/en/sql-reference/data-types/string.md) to [DateTime](/docs/en/sql-reference/data-types/datetime.md) according to a [MySQL format string](https://dev.mysql.com/doc/refman/8.0/en/date-and-time-functions.html#function_date-format). + This function is the opposite operation of function [formatDateTime](/docs/en/sql-reference/functions/date-time-functions.md#date_time_functions-formatDateTime). **Syntax** @@ -1163,6 +1164,7 @@ parseDateTime(str, format[, timezone]) - `str` — the String to be parsed - `format` — the format string +- `timezone` — [Timezone](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-timezone). Optional. **Returned value(s)** @@ -1186,9 +1188,10 @@ SELECT parseDateTime('2021-01-04+23:00:00', '%Y-%m-%d+%H:%i:%s') Alias: `TO_TIMESTAMP`. -## parseDateTimeInJodaSyntax +## parseDateTimeInJodaSyntax {#type_conversion_functions-parseDateTimeInJodaSyntax} Similar to [parseDateTime](#parsedatetime), except that the format string is in [Joda](https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html) instead of MySQL syntax. + This function is the opposite operation of function [formatDateTimeInJodaSyntax](/docs/en/sql-reference/functions/date-time-functions.md#date_time_functions-formatDateTimeInJodaSyntax). **Syntax** @@ -1201,6 +1204,7 @@ parseDateTimeInJodaSyntax(str, format[, timezone]) - `str` — the String to be parsed - `format` — the format string +- `timezone` — [Timezone](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-timezone). Optional. **Returned value(s)** From 17cff78393ba4efd45afce9cbf31254584b5f48b Mon Sep 17 00:00:00 2001 From: SmitaRKulkarni <64093672+SmitaRKulkarni@users.noreply.github.com> Date: Fri, 24 Mar 2023 14:56:48 +0100 Subject: [PATCH 360/418] Removed SYSTEM SYNC from test test_concurrent_backups_on_same_node. (#47944) --- .../test_backup_restore_on_cluster/test_disallow_concurrency.py | 1 - 1 file changed, 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 43c8adda65a..9dcb036cdf9 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 @@ -149,7 +149,6 @@ def test_concurrent_backups_on_same_node(): }, ) nodes[0].query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}") - nodes[0].query("SYSTEM SYNC REPLICA ON CLUSTER 'cluster' tbl") def test_concurrent_backups_on_different_nodes(): From f353561204c5707fd229fcaeefc880096006cff2 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 24 Mar 2023 14:37:40 +0000 Subject: [PATCH 361/418] Avoid breaking batches with read requests --- src/Coordination/KeeperDispatcher.cpp | 32 +++++++++++++++++++++---- src/Coordination/KeeperDispatcher.h | 16 +++++++++++++ src/Coordination/KeeperServer.cpp | 4 +++- src/Coordination/KeeperServer.h | 3 ++- src/Coordination/KeeperStateMachine.cpp | 5 +++- src/Coordination/KeeperStateMachine.h | 4 ++++ 6 files changed, 56 insertions(+), 8 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 2aa11dd9eed..fc97bc5e0e1 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -90,7 +90,7 @@ void KeeperDispatcher::requestThread() KeeperStorage::RequestsForSessions current_batch; - bool has_read_request = false; + bool has_read_request{false}; /// If new request is not read request or we must to process it through quorum. /// Otherwise we will process it locally. @@ -98,6 +98,8 @@ void KeeperDispatcher::requestThread() { current_batch.emplace_back(request); + size_t read_requests = 0; + const auto try_get_request = [&] { /// Trying to get batch requests as fast as possible @@ -106,7 +108,12 @@ void KeeperDispatcher::requestThread() CurrentMetrics::sub(CurrentMetrics::KeeperOutstandingRequets); /// Don't append read request into batch, we have to process them separately if (!coordination_settings->quorum_reads && request.request->isReadRequest()) - has_read_request = true; + { + ++read_requests; + std::pair key{current_batch.back().session_id, current_batch.back().request->xid}; + std::lock_guard lock(read_mutex); + related_read_requests[key].push_back(request); + } else current_batch.emplace_back(request); @@ -118,7 +125,7 @@ void KeeperDispatcher::requestThread() /// If we have enough requests in queue, we will try to batch at least max_quick_batch_size of them. size_t max_quick_batch_size = coordination_settings->max_requests_quick_batch_size; - while (!shutdown_called && !has_read_request && current_batch.size() < max_quick_batch_size && try_get_request()) + while (!shutdown_called && current_batch.size() + read_requests < max_quick_batch_size && try_get_request()) ; const auto prev_result_done = [&] @@ -129,7 +136,7 @@ void KeeperDispatcher::requestThread() }; /// Waiting until previous append will be successful, or batch is big enough - while (!shutdown_called && !has_read_request && !prev_result_done() && current_batch.size() <= max_batch_size) + while (!shutdown_called && !prev_result_done() && current_batch.size() <= max_batch_size) { try_get_request(); } @@ -319,7 +326,22 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf snapshot_s3.startup(config, macros); - server = std::make_unique(configuration_and_settings, config, responses_queue, snapshots_queue, snapshot_s3); + server = std::make_unique(configuration_and_settings, config, responses_queue, snapshots_queue, snapshot_s3, [this](const KeeperStorage::RequestForSession & request_for_session) + { + std::lock_guard lock(read_mutex); + if (auto it = related_read_requests.find(std::pair{request_for_session.session_id, request_for_session.request->xid}); it != related_read_requests.end()) + { + for (const auto & read_request : it->second) + { + if (server->isLeaderAlive()) + server->putLocalReadRequest(read_request); + else + addErrorResponses({read_request}, Coordination::Error::ZCONNECTIONLOSS); + } + + related_read_requests.erase(it); + } + }); try { diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 9371d2fbbac..e7570727b9a 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -1,5 +1,6 @@ #pragma once +#include "Common/ZooKeeper/ZooKeeperCommon.h" #include "config.h" #if USE_NURAFT @@ -103,6 +104,21 @@ private: void forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorage::RequestsForSessions & requests_for_sessions); public: + std::mutex read_mutex; + + struct PairHash + { + auto operator()(std::pair pair) const + { + SipHash hash; + hash.update(pair.first); + hash.update(pair.second); + return hash.get64(); + } + }; + + std::unordered_map, KeeperStorage::RequestsForSessions, PairHash> related_read_requests; + /// Just allocate some objects, real initialization is done by `intialize method` KeeperDispatcher(); diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 78a095f8c8d..56ed8f4eafe 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -107,7 +107,8 @@ KeeperServer::KeeperServer( const Poco::Util::AbstractConfiguration & config, ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, - KeeperSnapshotManagerS3 & snapshot_manager_s3) + KeeperSnapshotManagerS3 & snapshot_manager_s3, + KeeperStateMachine::CommitCallback commit_callback) : server_id(configuration_and_settings_->server_id) , coordination_settings(configuration_and_settings_->coordination_settings) , log(&Poco::Logger::get("KeeperServer")) @@ -128,6 +129,7 @@ KeeperServer::KeeperServer( coordination_settings, keeper_context, config.getBool("keeper_server.upload_snapshot_on_exit", true) ? &snapshot_manager_s3 : nullptr, + commit_callback, checkAndGetSuperdigest(configuration_and_settings_->super_digest)); state_manager = nuraft::cs_new( diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index bcff81f66f2..db4e9c1962e 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -72,7 +72,8 @@ public: const Poco::Util::AbstractConfiguration & config_, ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, - KeeperSnapshotManagerS3 & snapshot_manager_s3); + KeeperSnapshotManagerS3 & snapshot_manager_s3, + KeeperStateMachine::CommitCallback commit_callback); /// Load state machine from the latest snapshot and load log storage. Start NuRaft with required settings. void startup(const Poco::Util::AbstractConfiguration & config, bool enable_ipv6 = true); diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index d0bd18b63e2..0b69b00bf0e 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -46,8 +46,10 @@ KeeperStateMachine::KeeperStateMachine( const CoordinationSettingsPtr & coordination_settings_, const KeeperContextPtr & keeper_context_, KeeperSnapshotManagerS3 * snapshot_manager_s3_, + CommitCallback commit_callback_, const std::string & superdigest_) - : coordination_settings(coordination_settings_) + : commit_callback(commit_callback_) + , coordination_settings(coordination_settings_) , snapshot_manager( snapshots_path_, coordination_settings->snapshots_to_keep, @@ -274,6 +276,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n ProfileEvents::increment(ProfileEvents::KeeperCommits); last_committed_idx = log_idx; + commit_callback(request_for_session); return nullptr; } diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index d8181532f09..6babf741dbd 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -22,6 +22,8 @@ using SnapshotsQueue = ConcurrentBoundedQueue; class KeeperStateMachine : public nuraft::state_machine { public: + using CommitCallback = std::function; + KeeperStateMachine( ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, @@ -29,6 +31,7 @@ public: const CoordinationSettingsPtr & coordination_settings_, const KeeperContextPtr & keeper_context_, KeeperSnapshotManagerS3 * snapshot_manager_s3_, + CommitCallback commit_callback_, const std::string & superdigest_ = ""); /// Read state from the latest snapshot @@ -105,6 +108,7 @@ public: void recalculateStorageStats(); private: + CommitCallback commit_callback; /// In our state machine we always have a single snapshot which is stored /// in memory in compressed (serialized) format. SnapshotMetadataPtr latest_snapshot_meta = nullptr; From f7c0cca297c0e093fe24eaa225f125c2f0cfd31e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 24 Mar 2023 14:37:51 +0000 Subject: [PATCH 362/418] Maybe better keeper-bench --- utils/keeper-bench/Generator.cpp | 21 ++++++++++++++++++++- utils/keeper-bench/Generator.h | 14 ++++++++++++++ utils/keeper-bench/Runner.cpp | 2 +- 3 files changed, 35 insertions(+), 2 deletions(-) diff --git a/utils/keeper-bench/Generator.cpp b/utils/keeper-bench/Generator.cpp index 5d1d0f8a491..b6d8223862c 100644 --- a/utils/keeper-bench/Generator.cpp +++ b/utils/keeper-bench/Generator.cpp @@ -162,6 +162,19 @@ ZooKeeperRequestPtr SetRequestGenerator::generate() return request; } +void MixedRequestGenerator::startup(Coordination::ZooKeeper & zookeeper) +{ + for (auto & generator : generators) + generator->startup(zookeeper); +} + +ZooKeeperRequestPtr MixedRequestGenerator::generate() +{ + pcg64 rng(randomSeed()); + std::uniform_int_distribution distribution(0, generators.size() - 1); + + return generators[distribution(rng)]->generate(); +} void GetRequestGenerator::startup(Coordination::ZooKeeper & zookeeper) { @@ -315,7 +328,13 @@ std::unique_ptr getGenerator(const std::string & name) { return std::make_unique("/set_generator", 5); } - + else if (name == "mixed_small_data") + { + std::vector> generators; + generators.push_back(std::make_unique("/set_generator", 5)); + generators.push_back(std::make_unique("/get_generator", 10, 32)); + return std::make_unique(std::move(generators)); + } throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unknown generator {}", name); } diff --git a/utils/keeper-bench/Generator.h b/utils/keeper-bench/Generator.h index 1ff01b25ed4..e2c546e4bce 100644 --- a/utils/keeper-bench/Generator.h +++ b/utils/keeper-bench/Generator.h @@ -122,5 +122,19 @@ private: uint64_t data_size; }; +class MixedRequestGenerator final : public IGenerator +{ +public: + explicit MixedRequestGenerator(std::vector> generators_) + : generators(std::move(generators_)) + {} + + void startup(Coordination::ZooKeeper & zookeeper) override; + Coordination::ZooKeeperRequestPtr generate() override; + +private: + std::vector> generators; +}; + std::unique_ptr getGenerator(const std::string & name); diff --git a/utils/keeper-bench/Runner.cpp b/utils/keeper-bench/Runner.cpp index 2f3cf4b0620..c858b476483 100644 --- a/utils/keeper-bench/Runner.cpp +++ b/utils/keeper-bench/Runner.cpp @@ -159,9 +159,9 @@ void Runner::runBenchmark() std::cerr << "Prepared\n"; try { + auto connections = getConnections(); for (size_t i = 0; i < concurrency; ++i) { - auto connections = getConnections(); pool.scheduleOrThrowOnError([this, connections]() mutable { thread(connections); }); } } From 912fe0115e45f00972f352fb7e9189e7f309f446 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 24 Mar 2023 14:57:49 +0000 Subject: [PATCH 363/418] UUUUUUU TTTTTTT CCCCCCC --- .../02668_parse_datetime_in_joda_syntax.reference | 12 +++++------- .../02668_parse_datetime_in_joda_syntax.sql | 11 +++++------ 2 files changed, 10 insertions(+), 13 deletions(-) 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 90515055436..21c2754925d 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 @@ -3,19 +3,17 @@ select parseDateTimeInJodaSyntax(' ', ' ', 'UTC') = toDateTime('1970-01-01', 'UTC'); 1 -- era -select parseDateTimeInJodaSyntax('AD 1999', 'G YYYY') = toDateTime('1999-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('AD 1999', 'G YYYY', 'UTC') = toDateTime('1999-01-01', 'UTC'); 1 -select parseDateTimeInJodaSyntax('ad 1999', 'G YYYY') = toDateTime('1999-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('ad 1999', 'G YYYY', 'UTC') = toDateTime('1999-01-01', 'UTC'); 1 -select parseDateTimeInJodaSyntax('Ad 1999', 'G YYYY') = toDateTime('1999-01-01', 'UTC'); -1 -select parseDateTimeInJodaSyntax('AD 1999', 'G YYYY') = toDateTime('1999-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('Ad 1999', 'G YYYY', 'UTC') = toDateTime('1999-01-01', 'UTC'); 1 select parseDateTimeInJodaSyntax('AD 1999', 'G yyyy') = toDateTime('1999-01-01', 'UTC'); 1 -select parseDateTimeInJodaSyntax('AD 1999 2000', 'G YYYY yyyy') = toDateTime('2000-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('AD 1999 2000', 'G YYYY yyyy', 'UTC') = toDateTime('2000-01-01', 'UTC'); 1 -select parseDateTimeInJodaSyntax('AD 1999 2000', 'G yyyy YYYY') = toDateTime('2000-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('AD 1999 2000', 'G yyyy YYYY', 'UTC') = toDateTime('2000-01-01', 'UTC'); 1 select parseDateTimeInJodaSyntax('AD 1999', 'G Y'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTimeInJodaSyntax('AD 1999', 'G YY'); -- { serverError CANNOT_PARSE_DATETIME } 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 ae5aff64416..6bfd3a986cd 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 @@ -3,13 +3,12 @@ select parseDateTimeInJodaSyntax(' ', ' ', 'UTC') = toDateTime('1970-01-01', 'UTC'); -- era -select parseDateTimeInJodaSyntax('AD 1999', 'G YYYY') = toDateTime('1999-01-01', 'UTC'); -select parseDateTimeInJodaSyntax('ad 1999', 'G YYYY') = toDateTime('1999-01-01', 'UTC'); -select parseDateTimeInJodaSyntax('Ad 1999', 'G YYYY') = toDateTime('1999-01-01', 'UTC'); -select parseDateTimeInJodaSyntax('AD 1999', 'G YYYY') = toDateTime('1999-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('AD 1999', 'G YYYY', 'UTC') = toDateTime('1999-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('ad 1999', 'G YYYY', 'UTC') = toDateTime('1999-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('Ad 1999', 'G YYYY', 'UTC') = toDateTime('1999-01-01', 'UTC'); select parseDateTimeInJodaSyntax('AD 1999', 'G yyyy') = toDateTime('1999-01-01', 'UTC'); -select parseDateTimeInJodaSyntax('AD 1999 2000', 'G YYYY yyyy') = toDateTime('2000-01-01', 'UTC'); -select parseDateTimeInJodaSyntax('AD 1999 2000', 'G yyyy YYYY') = toDateTime('2000-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('AD 1999 2000', 'G YYYY yyyy', 'UTC') = toDateTime('2000-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('AD 1999 2000', 'G yyyy YYYY', 'UTC') = toDateTime('2000-01-01', 'UTC'); 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 } From 7a82830f1086548f348b398b6e2c33adbb12ba9c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 24 Mar 2023 15:04:05 +0000 Subject: [PATCH 364/418] Automatic style fix --- docker/test/performance-comparison/perf.py | 2 ++ docker/test/performance-comparison/report.py | 1 - tests/ci/clickhouse_helper.py | 1 - tests/ci/docker_images_check.py | 1 - tests/ci/get_previous_release_tag.py | 1 - tests/ci/report.py | 2 +- tests/integration/helpers/cluster.py | 3 +-- tests/integration/helpers/network.py | 2 -- .../pytest_xdist_logging_to_separate_files.py | 1 + .../test_detach_part_wrong_partition_id.py | 1 - .../test_cluster_copier/test_three_nodes.py | 1 - .../test_cluster_copier/test_two_nodes.py | 1 - tests/integration/test_composable_protocols/test.py | 1 - .../test_create_query_constraints/test.py | 2 -- .../common.py | 1 - tests/integration/test_disks_app_func/test.py | 1 - .../test_distributed_ddl_parallel/test.py | 1 + tests/integration/test_fetch_memory_usage/test.py | 1 - .../scripts/stress_test.py | 1 - tests/integration/test_jbod_balancer/test.py | 1 - .../test_keeper_and_access_storage/test.py | 1 + tests/integration/test_keeper_back_to_back/test.py | 2 +- tests/integration/test_keeper_persistent_log/test.py | 1 - .../test_keeper_zookeeper_converter/test.py | 1 - tests/integration/test_merge_tree_load_parts/test.py | 6 +++--- .../s3_endpoint/endpoint.py | 1 - .../test_merge_tree_settings_constraints/test.py | 1 - .../test_old_parts_finally_removed/test.py | 1 - tests/integration/test_partition/test.py | 4 +++- tests/integration/test_password_constraints/test.py | 1 - tests/integration/test_read_only_table/test.py | 1 - .../test_reload_auxiliary_zookeepers/test.py | 1 - .../s3_endpoint/endpoint.py | 1 + tests/integration/test_s3_with_proxy/test.py | 1 + .../integration/test_ssl_cert_authentication/test.py | 1 - tests/integration/test_storage_kafka/kafka_pb2.py | 1 - .../test_storage_kafka/message_with_repeated_pb2.py | 1 - tests/integration/test_storage_kafka/social_pb2.py | 1 - tests/integration/test_storage_kafka/test.py | 12 ++---------- tests/integration/test_storage_nats/nats_pb2.py | 1 - .../test_storage_postgresql_replica/test.py | 1 - .../test_storage_rabbitmq/rabbitmq_pb2.py | 1 - tests/integration/test_storage_rabbitmq/test.py | 3 --- tests/integration/test_storage_s3/test.py | 1 + .../test_storage_s3/test_invalid_env_credentials.py | 1 + tests/integration/test_system_merges/test.py | 1 - tests/integration/test_ttl_move/test.py | 2 +- tests/integration/test_zero_copy_fetch/test.py | 1 - utils/changelog-simple/format-changelog.py | 1 + utils/keeper-overload/keeper-overload.py | 2 +- 50 files changed, 23 insertions(+), 57 deletions(-) diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 65bf49c2914..7a4e6386d0d 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -26,6 +26,7 @@ logging.basicConfig( total_start_seconds = time.perf_counter() stage_start_seconds = total_start_seconds + # Thread executor that does not hides exception that happens during function # execution, and rethrows it after join() class SafeThread(Thread): @@ -158,6 +159,7 @@ for e in subst_elems: available_parameters[name] = values + # Takes parallel lists of templates, substitutes them with all combos of # parameters. The set of parameters is determined based on the first list. # Note: keep the order of queries -- sometimes we have DROP IF EXISTS diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index 782cf29863c..214f2d550b4 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -670,7 +670,6 @@ if args.report == "main": ) elif args.report == "all-queries": - print((header_template.format())) add_tested_commits() diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index d60a9e6afd1..64b64896f66 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -141,7 +141,6 @@ def prepare_tests_results_for_clickhouse( report_url: str, check_name: str, ) -> List[dict]: - pull_request_url = "https://github.com/ClickHouse/ClickHouse/commits/master" base_ref = "master" head_ref = "master" diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 192d216614e..f2b1105b3b0 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -96,7 +96,6 @@ def get_images_dict(repo_path: str, image_file_path: str) -> ImagesDict: def get_changed_docker_images( pr_info: PRInfo, images_dict: ImagesDict ) -> Set[DockerImage]: - if not images_dict: return set() diff --git a/tests/ci/get_previous_release_tag.py b/tests/ci/get_previous_release_tag.py index c6fe6cd5fb5..c2d279f7fec 100755 --- a/tests/ci/get_previous_release_tag.py +++ b/tests/ci/get_previous_release_tag.py @@ -51,7 +51,6 @@ def find_previous_release( for release in releases: if release.version < server_version: - # Check if the artifact exists on GitHub. # It can be not true for a short period of time # after creating a tag for a new release before uploading the packages. diff --git a/tests/ci/report.py b/tests/ci/report.py index 947fb33d905..ddee035d26f 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -473,7 +473,7 @@ def create_build_html_report( commit_url: str, ) -> str: rows = "" - for (build_result, build_log_url, artifact_urls) in zip( + for build_result, build_log_url, artifact_urls in zip( build_results, build_logs_urls, artifact_urls_list ): row = "" diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index dc5ada81995..a9a996e0a5f 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -63,6 +63,7 @@ DEFAULT_ENV_NAME = ".env" SANITIZER_SIGN = "==================" + # to create docker-compose env file def _create_env_file(path, variables): logging.debug(f"Env {variables} stored in {path}") @@ -1454,7 +1455,6 @@ class ClickHouseCluster: config_root_name="clickhouse", extra_configs=[], ) -> "ClickHouseInstance": - """Add an instance to the cluster. name - the name of the instance directory and the value of the 'instance' macro in ClickHouse. @@ -3089,7 +3089,6 @@ class ClickHouseInstance: config_root_name="clickhouse", extra_configs=[], ): - self.name = name self.base_cmd = cluster.base_cmd self.docker_id = cluster.get_instance_docker_id(self.name) diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index e408c9beec1..471aa2bdc2e 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -216,7 +216,6 @@ class _NetworkManager: container_exit_timeout=60, docker_api_version=os.environ.get("DOCKER_API_VERSION"), ): - self.container_expire_timeout = container_expire_timeout self.container_exit_timeout = container_exit_timeout @@ -232,7 +231,6 @@ class _NetworkManager: def _ensure_container(self): if self._container is None or self._container_expire_time <= time.time(): - for i in range(5): if self._container is not None: try: diff --git a/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py b/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py index d424ad58fa4..370aa23a014 100644 --- a/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py +++ b/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py @@ -1,6 +1,7 @@ import logging import os.path + # Makes the parallel workers of pytest-xdist to log to separate files. # Without this function all workers will log to the same log file # and mix everything together making it much more difficult for troubleshooting. diff --git a/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py b/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py index 02fccfae4e5..a6f7a8653da 100644 --- a/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py +++ b/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py @@ -24,7 +24,6 @@ def start_cluster(): def test_detach_part_wrong_partition_id(start_cluster): - # Here we create table with partition by UUID. node_21_6.query( "create table tab (id UUID, value UInt32) engine = MergeTree PARTITION BY (id) order by tuple()" diff --git a/tests/integration/test_cluster_copier/test_three_nodes.py b/tests/integration/test_cluster_copier/test_three_nodes.py index 31d6c0448f4..e7d07757adb 100644 --- a/tests/integration/test_cluster_copier/test_three_nodes.py +++ b/tests/integration/test_cluster_copier/test_three_nodes.py @@ -19,7 +19,6 @@ cluster = ClickHouseCluster(__file__) def started_cluster(): global cluster try: - for name in ["first", "second", "third"]: cluster.add_instance( name, diff --git a/tests/integration/test_cluster_copier/test_two_nodes.py b/tests/integration/test_cluster_copier/test_two_nodes.py index 10ab7d03b00..2b6fcf6cac2 100644 --- a/tests/integration/test_cluster_copier/test_two_nodes.py +++ b/tests/integration/test_cluster_copier/test_two_nodes.py @@ -19,7 +19,6 @@ cluster = ClickHouseCluster(__file__) def started_cluster(): global cluster try: - for name in ["first_of_two", "second_of_two"]: instance = cluster.add_instance( name, diff --git a/tests/integration/test_composable_protocols/test.py b/tests/integration/test_composable_protocols/test.py index bc87fea5296..df74cfffa54 100644 --- a/tests/integration/test_composable_protocols/test.py +++ b/tests/integration/test_composable_protocols/test.py @@ -63,7 +63,6 @@ def netcat(hostname, port, content): def test_connections(): - client = Client(server.ip_address, 9000, command=cluster.client_bin_path) assert client.query("SELECT 1") == "1\n" diff --git a/tests/integration/test_create_query_constraints/test.py b/tests/integration/test_create_query_constraints/test.py index 8df043fd24b..33c41b4f161 100644 --- a/tests/integration/test_create_query_constraints/test.py +++ b/tests/integration/test_create_query_constraints/test.py @@ -25,7 +25,6 @@ def start_cluster(): def test_create_query_const_constraints(): - instance.query("CREATE USER u_const SETTINGS max_threads = 1 CONST") instance.query("GRANT ALL ON *.* TO u_const") @@ -57,7 +56,6 @@ def test_create_query_const_constraints(): def test_create_query_minmax_constraints(): - instance.query("CREATE USER u_minmax SETTINGS max_threads = 4 MIN 2 MAX 6") instance.query("GRANT ALL ON *.* TO u_minmax") diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py index b38e81b0227..01addae2542 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py @@ -348,7 +348,6 @@ class RangedLayoutTester(BaseLayoutTester): self.layouts = LAYOUTS_RANGED def execute(self, layout_name, node): - if layout_name not in self.layout_to_dictionary: raise RuntimeError("Source doesn't support layout: {}".format(layout_name)) diff --git a/tests/integration/test_disks_app_func/test.py b/tests/integration/test_disks_app_func/test.py index 027ef8feed0..2428c53854e 100644 --- a/tests/integration/test_disks_app_func/test.py +++ b/tests/integration/test_disks_app_func/test.py @@ -7,7 +7,6 @@ import pytest def started_cluster(): global cluster try: - cluster = ClickHouseCluster(__file__) cluster.add_instance( "disks_app_test", main_configs=["config.xml"], with_minio=True diff --git a/tests/integration/test_distributed_ddl_parallel/test.py b/tests/integration/test_distributed_ddl_parallel/test.py index 6ebfe472e09..eb98dd3e230 100644 --- a/tests/integration/test_distributed_ddl_parallel/test.py +++ b/tests/integration/test_distributed_ddl_parallel/test.py @@ -10,6 +10,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) + # By default the exceptions that was throwed in threads will be ignored # (they will not mark the test as failed, only printed to stderr). # diff --git a/tests/integration/test_fetch_memory_usage/test.py b/tests/integration/test_fetch_memory_usage/test.py index a4371140150..7591cc0e8a9 100644 --- a/tests/integration/test_fetch_memory_usage/test.py +++ b/tests/integration/test_fetch_memory_usage/test.py @@ -18,7 +18,6 @@ def started_cluster(): def test_huge_column(started_cluster): - if ( node.is_built_with_thread_sanitizer() or node.is_built_with_memory_sanitizer() diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py index b8bafb3d0c1..fe69d72c1c7 100644 --- a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py +++ b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py @@ -13,7 +13,6 @@ number_of_iterations = 100 def perform_request(): - buffer = BytesIO() crl = pycurl.Curl() crl.setopt(pycurl.INTERFACE, client_ip) diff --git a/tests/integration/test_jbod_balancer/test.py b/tests/integration/test_jbod_balancer/test.py index e746698611a..df34a075d5a 100644 --- a/tests/integration/test_jbod_balancer/test.py +++ b/tests/integration/test_jbod_balancer/test.py @@ -45,7 +45,6 @@ def start_cluster(): def check_balance(node, table): - partitions = node.query( """ WITH diff --git a/tests/integration/test_keeper_and_access_storage/test.py b/tests/integration/test_keeper_and_access_storage/test.py index 6ec307f7082..0314825b6b7 100644 --- a/tests/integration/test_keeper_and_access_storage/test.py +++ b/tests/integration/test_keeper_and_access_storage/test.py @@ -10,6 +10,7 @@ node1 = cluster.add_instance( "node1", main_configs=["configs/keeper.xml"], stay_alive=True ) + # test that server is able to start @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_keeper_back_to_back/test.py b/tests/integration/test_keeper_back_to_back/test.py index 73fface02b4..b737ac284d2 100644 --- a/tests/integration/test_keeper_back_to_back/test.py +++ b/tests/integration/test_keeper_back_to_back/test.py @@ -546,7 +546,6 @@ def test_random_requests(started_cluster): def test_end_of_session(started_cluster): - fake_zk1 = None fake_zk2 = None genuine_zk1 = None @@ -685,6 +684,7 @@ def test_concurrent_watches(started_cluster): nonlocal watches_created nonlocal all_paths_created fake_zk.ensure_path(global_path + "/" + str(i)) + # new function each time def dumb_watch(event): nonlocal dumb_watch_triggered_counter diff --git a/tests/integration/test_keeper_persistent_log/test.py b/tests/integration/test_keeper_persistent_log/test.py index 70cc14fe26d..4164ffb33d3 100644 --- a/tests/integration/test_keeper_persistent_log/test.py +++ b/tests/integration/test_keeper_persistent_log/test.py @@ -163,7 +163,6 @@ def test_state_duplicate_restart(started_cluster): # http://zookeeper-user.578899.n2.nabble.com/Why-are-ephemeral-nodes-written-to-disk-tp7583403p7583418.html def test_ephemeral_after_restart(started_cluster): - try: node_zk = None node_zk2 = None diff --git a/tests/integration/test_keeper_zookeeper_converter/test.py b/tests/integration/test_keeper_zookeeper_converter/test.py index 063421bf922..de5a9416119 100644 --- a/tests/integration/test_keeper_zookeeper_converter/test.py +++ b/tests/integration/test_keeper_zookeeper_converter/test.py @@ -114,7 +114,6 @@ def start_clickhouse(): def copy_zookeeper_data(make_zk_snapshots): - if make_zk_snapshots: # force zookeeper to create snapshot generate_zk_snapshot() else: diff --git a/tests/integration/test_merge_tree_load_parts/test.py b/tests/integration/test_merge_tree_load_parts/test.py index 777b6f14fc6..dfbe00c8e28 100644 --- a/tests/integration/test_merge_tree_load_parts/test.py +++ b/tests/integration/test_merge_tree_load_parts/test.py @@ -148,17 +148,17 @@ def test_merge_tree_load_parts_corrupted(started_cluster): node1.query("SYSTEM WAIT LOADING PARTS mt_load_parts_2") def check_parts_loading(node, partition, loaded, failed, skipped): - for (min_block, max_block) in loaded: + for min_block, max_block in loaded: part_name = f"{partition}_{min_block}_{max_block}" assert node.contains_in_log(f"Loading Active part {part_name}") assert node.contains_in_log(f"Finished loading Active part {part_name}") - for (min_block, max_block) in failed: + for min_block, max_block in failed: part_name = f"{partition}_{min_block}_{max_block}" assert node.contains_in_log(f"Loading Active part {part_name}") assert not node.contains_in_log(f"Finished loading Active part {part_name}") - for (min_block, max_block) in skipped: + for min_block, max_block in skipped: part_name = f"{partition}_{min_block}_{max_block}" assert not node.contains_in_log(f"Loading Active part {part_name}") assert not node.contains_in_log(f"Finished loading Active part {part_name}") diff --git a/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py b/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py index b6567dfebc5..4613fdb850b 100644 --- a/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py +++ b/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py @@ -42,7 +42,6 @@ def delete(_bucket): @route("/<_bucket>/<_path:path>", ["GET", "POST", "PUT", "DELETE"]) def server(_bucket, _path): - # It's delete query for failed part if _path.endswith("delete"): response.set_header("Location", "http://minio1:9001/" + _bucket + "/" + _path) diff --git a/tests/integration/test_merge_tree_settings_constraints/test.py b/tests/integration/test_merge_tree_settings_constraints/test.py index 0bb0179108d..be6e2a31873 100644 --- a/tests/integration/test_merge_tree_settings_constraints/test.py +++ b/tests/integration/test_merge_tree_settings_constraints/test.py @@ -20,7 +20,6 @@ def start_cluster(): def test_merge_tree_settings_constraints(): - assert "Setting storage_policy should not be changed" in instance.query_and_get_error( f"CREATE TABLE wrong_table (number Int64) engine = MergeTree() ORDER BY number SETTINGS storage_policy = 'secret_policy'" ) diff --git a/tests/integration/test_old_parts_finally_removed/test.py b/tests/integration/test_old_parts_finally_removed/test.py index 108b72c5ccd..5347d433419 100644 --- a/tests/integration/test_old_parts_finally_removed/test.py +++ b/tests/integration/test_old_parts_finally_removed/test.py @@ -63,7 +63,6 @@ def test_part_finally_removed(started_cluster): ) for i in range(60): - if ( node1.query( "SELECT count() from system.parts WHERE table = 'drop_outdated_part'" diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index ae4393fc6f6..a34141c6189 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -528,7 +528,9 @@ def test_make_clone_in_detached(started_cluster): ["cp", "-r", path + "all_0_0_0", path + "detached/broken_all_0_0_0"] ) assert_eq_with_retry(instance, "select * from clone_in_detached", "\n") - assert ["broken_all_0_0_0",] == sorted( + assert [ + "broken_all_0_0_0", + ] == sorted( instance.exec_in_container(["ls", path + "detached/"]).strip().split("\n") ) diff --git a/tests/integration/test_password_constraints/test.py b/tests/integration/test_password_constraints/test.py index e3628861b28..9cdff51caa1 100644 --- a/tests/integration/test_password_constraints/test.py +++ b/tests/integration/test_password_constraints/test.py @@ -17,7 +17,6 @@ def start_cluster(): def test_complexity_rules(start_cluster): - error_message = "DB::Exception: Invalid password. The password should: be at least 12 characters long, contain at least 1 numeric character, contain at least 1 lowercase character, contain at least 1 uppercase character, contain at least 1 special character" assert error_message in node.query_and_get_error( "CREATE USER u_1 IDENTIFIED WITH plaintext_password BY ''" diff --git a/tests/integration/test_read_only_table/test.py b/tests/integration/test_read_only_table/test.py index 914c6a99508..df084f9dbbd 100644 --- a/tests/integration/test_read_only_table/test.py +++ b/tests/integration/test_read_only_table/test.py @@ -49,7 +49,6 @@ def start_cluster(): def test_restart_zookeeper(start_cluster): - for table_id in range(NUM_TABLES): node1.query( f"INSERT INTO test_table_{table_id} VALUES (1), (2), (3), (4), (5);" diff --git a/tests/integration/test_reload_auxiliary_zookeepers/test.py b/tests/integration/test_reload_auxiliary_zookeepers/test.py index bb1455333fc..476c5dee99e 100644 --- a/tests/integration/test_reload_auxiliary_zookeepers/test.py +++ b/tests/integration/test_reload_auxiliary_zookeepers/test.py @@ -20,7 +20,6 @@ def start_cluster(): def test_reload_auxiliary_zookeepers(start_cluster): - node.query( "CREATE TABLE simple (date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', 'node') ORDER BY tuple() PARTITION BY date;" ) diff --git a/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py b/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py index d6a732cc681..1d33ca02f86 100644 --- a/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py +++ b/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py @@ -1,6 +1,7 @@ #!/usr/bin/env python3 from bottle import request, route, run, response + # Handle for MultipleObjectsDelete. @route("/<_bucket>", ["POST"]) def delete(_bucket): diff --git a/tests/integration/test_s3_with_proxy/test.py b/tests/integration/test_s3_with_proxy/test.py index 1102d190a87..1af040c3c30 100644 --- a/tests/integration/test_s3_with_proxy/test.py +++ b/tests/integration/test_s3_with_proxy/test.py @@ -5,6 +5,7 @@ import time import pytest from helpers.cluster import ClickHouseCluster + # Runs simple proxy resolver in python env container. def run_resolver(cluster): container_id = cluster.get_container_id("resolver") diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index 7c62ca0d8b6..b3570b6e281 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -87,7 +87,6 @@ config = """ def execute_query_native(node, query, user, cert_name): - config_path = f"{SCRIPT_DIR}/configs/client.xml" formatted = config.format( diff --git a/tests/integration/test_storage_kafka/kafka_pb2.py b/tests/integration/test_storage_kafka/kafka_pb2.py index 7de1363bbf1..3e47af6c1e0 100644 --- a/tests/integration/test_storage_kafka/kafka_pb2.py +++ b/tests/integration/test_storage_kafka/kafka_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.kafka_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _KEYVALUEPAIR._serialized_start = 46 _KEYVALUEPAIR._serialized_end = 88 diff --git a/tests/integration/test_storage_kafka/message_with_repeated_pb2.py b/tests/integration/test_storage_kafka/message_with_repeated_pb2.py index 4d1a23c0b43..3715a9bea04 100644 --- a/tests/integration/test_storage_kafka/message_with_repeated_pb2.py +++ b/tests/integration/test_storage_kafka/message_with_repeated_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.message_with_repeated_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None DESCRIPTOR._serialized_options = b"H\001" _MESSAGE._serialized_start = 62 diff --git a/tests/integration/test_storage_kafka/social_pb2.py b/tests/integration/test_storage_kafka/social_pb2.py index 830ade81d33..f91a7bd0539 100644 --- a/tests/integration/test_storage_kafka/social_pb2.py +++ b/tests/integration/test_storage_kafka/social_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.social_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _USER._serialized_start = 47 _USER._serialized_end = 90 diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 51952ac1eb7..3a4fa6c6bfe 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -121,7 +121,7 @@ def kafka_create_topic( def kafka_delete_topic(admin_client, topic, max_retries=50): result = admin_client.delete_topics([topic]) - for (topic, e) in result.topic_error_codes: + for topic, e in result.topic_error_codes: if e == 0: logging.debug(f"Topic {topic} deleted") else: @@ -917,9 +917,7 @@ def describe_consumer_group(kafka_cluster, name): member_info["client_id"] = client_id member_info["client_host"] = client_host member_topics_assignment = [] - for (topic, partitions) in MemberAssignment.decode( - member_assignment - ).assignment: + for topic, partitions in MemberAssignment.decode(member_assignment).assignment: member_topics_assignment.append({"topic": topic, "partitions": partitions}) member_info["assignment"] = member_topics_assignment res.append(member_info) @@ -1537,7 +1535,6 @@ def test_kafka_protobuf_no_delimiter(kafka_cluster): def test_kafka_materialized_view(kafka_cluster): - instance.query( """ DROP TABLE IF EXISTS test.view; @@ -2315,7 +2312,6 @@ def test_kafka_virtual_columns2(kafka_cluster): def test_kafka_produce_key_timestamp(kafka_cluster): - admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) @@ -2444,7 +2440,6 @@ def test_kafka_insert_avro(kafka_cluster): def test_kafka_produce_consume_avro(kafka_cluster): - admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) @@ -4031,7 +4026,6 @@ def test_kafka_predefined_configuration(kafka_cluster): # https://github.com/ClickHouse/ClickHouse/issues/26643 def test_issue26643(kafka_cluster): - # for backporting: # admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") admin_client = KafkaAdminClient( @@ -4313,7 +4307,6 @@ def test_row_based_formats(kafka_cluster): "RowBinaryWithNamesAndTypes", "MsgPack", ]: - print(format_name) kafka_create_topic(admin_client, format_name) @@ -4438,7 +4431,6 @@ def test_block_based_formats_2(kafka_cluster): "ORC", "JSONCompactColumns", ]: - kafka_create_topic(admin_client, format_name) instance.query( diff --git a/tests/integration/test_storage_nats/nats_pb2.py b/tests/integration/test_storage_nats/nats_pb2.py index 4330ff57950..e9e5cb72363 100644 --- a/tests/integration/test_storage_nats/nats_pb2.py +++ b/tests/integration/test_storage_nats/nats_pb2.py @@ -31,7 +31,6 @@ ProtoKeyValue = _reflection.GeneratedProtocolMessageType( _sym_db.RegisterMessage(ProtoKeyValue) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _PROTOKEYVALUE._serialized_start = 45 _PROTOKEYVALUE._serialized_end = 88 diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 5df8b9029e6..8666d7ae58c 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -706,7 +706,6 @@ def test_abrupt_connection_loss_while_heavy_replication(started_cluster): def test_abrupt_server_restart_while_heavy_replication(started_cluster): - # FIXME (kssenii) temporary disabled if instance.is_built_with_sanitizer(): pytest.skip("Temporary disabled (FIXME)") diff --git a/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py b/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py index e017b4e66c2..a5845652eef 100644 --- a/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py +++ b/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.rabbitmq_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _KEYVALUEPROTO._serialized_start = 49 _KEYVALUEPROTO._serialized_end = 92 diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 2e54f21787a..53b6c4109ef 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -2864,7 +2864,6 @@ def test_rabbitmq_predefined_configuration(rabbitmq_cluster): def test_rabbitmq_msgpack(rabbitmq_cluster): - instance.query( """ drop table if exists rabbit_in; @@ -2908,7 +2907,6 @@ def test_rabbitmq_msgpack(rabbitmq_cluster): def test_rabbitmq_address(rabbitmq_cluster): - instance2.query( """ drop table if exists rabbit_in; @@ -3243,7 +3241,6 @@ def test_block_based_formats_2(rabbitmq_cluster): "ORC", "JSONCompactColumns", ]: - print(format_name) instance.query( diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 8b20727a7b5..4d493d9526b 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -18,6 +18,7 @@ MINIO_INTERNAL_PORT = 9001 SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + # Creates S3 bucket for tests and allows anonymous read-write access to it. def prepare_s3_bucket(started_cluster): # Allows read-write access for bucket without authorization. diff --git a/tests/integration/test_storage_s3/test_invalid_env_credentials.py b/tests/integration/test_storage_s3/test_invalid_env_credentials.py index 2f5d9349904..aa6479a2ed3 100644 --- a/tests/integration/test_storage_s3/test_invalid_env_credentials.py +++ b/tests/integration/test_storage_s3/test_invalid_env_credentials.py @@ -11,6 +11,7 @@ MINIO_INTERNAL_PORT = 9001 SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + # Creates S3 bucket for tests and allows anonymous read-write access to it. def prepare_s3_bucket(started_cluster): # Allows read-write access for bucket without authorization. diff --git a/tests/integration/test_system_merges/test.py b/tests/integration/test_system_merges/test.py index 0a469bd7bbd..ff303afe19e 100644 --- a/tests/integration/test_system_merges/test.py +++ b/tests/integration/test_system_merges/test.py @@ -171,7 +171,6 @@ def test_mutation_simple(started_cluster, replicated): starting_block = 0 if replicated else 1 try: - for node in nodes: node.query( f"create table {name} (a Int64) engine={engine} order by tuple()" diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 99978cbf6dc..89824293320 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1863,7 +1863,7 @@ def test_ttl_move_if_exists(started_cluster, name, dest_type): ) ) - for (node, policy) in zip( + for node, policy in zip( [node1, node2], ["only_jbod_1", "small_jbod_with_external"] ): node.query( diff --git a/tests/integration/test_zero_copy_fetch/test.py b/tests/integration/test_zero_copy_fetch/test.py index b71752528d3..9b9aa5e0da7 100644 --- a/tests/integration/test_zero_copy_fetch/test.py +++ b/tests/integration/test_zero_copy_fetch/test.py @@ -16,7 +16,6 @@ cluster = ClickHouseCluster(__file__) @pytest.fixture(scope="module") def started_cluster(): try: - cluster.add_instance( "node1", main_configs=["configs/storage_conf.xml"], diff --git a/utils/changelog-simple/format-changelog.py b/utils/changelog-simple/format-changelog.py index d5e1518270e..01f2694dd0f 100755 --- a/utils/changelog-simple/format-changelog.py +++ b/utils/changelog-simple/format-changelog.py @@ -20,6 +20,7 @@ parser.add_argument( ) args = parser.parse_args() + # This function mirrors the PR description checks in ClickhousePullRequestTrigger. # Returns False if the PR should not be mentioned changelog. def parse_one_pull_request(item): diff --git a/utils/keeper-overload/keeper-overload.py b/utils/keeper-overload/keeper-overload.py index bdb4563c713..0a059b10588 100755 --- a/utils/keeper-overload/keeper-overload.py +++ b/utils/keeper-overload/keeper-overload.py @@ -166,7 +166,7 @@ def main(args): keeper_bench_path = args.keeper_bench_path keepers = [] - for (port, server_id) in zip(PORTS, SERVER_IDS): + for port, server_id in zip(PORTS, SERVER_IDS): keepers.append( Keeper( keeper_binary_path, server_id, port, workdir, args.with_thread_fuzzer From 42cafb487254609c52c56d691450c2193beae16d Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 24 Mar 2023 16:02:03 +0100 Subject: [PATCH 365/418] clearer --- src/Processors/QueryPlan/PartsSplitter.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/PartsSplitter.cpp b/src/Processors/QueryPlan/PartsSplitter.cpp index f86879e474d..3024a79bcbb 100644 --- a/src/Processors/QueryPlan/PartsSplitter.cpp +++ b/src/Processors/QueryPlan/PartsSplitter.cpp @@ -94,8 +94,8 @@ std::pair, std::vector> split(RangesInDat parts_ranges_queue.push( {index_access->getValue(part_idx, range.begin), {range, part_idx}, PartsRangesIterator::EventType::RangeStart}); const auto & index_granularity = parts[part_idx].data_part->index_granularity; - if (range.end + 1 < index_granularity.getMarksCount() - || (index_granularity.hasFinalMark() && range.end + 1 == index_granularity.getMarksCount())) + const bool value_is_defined_at_end_mark = range.end < index_granularity.getMarksCount(); + if (value_is_defined_at_end_mark) parts_ranges_queue.push( {index_access->getValue(part_idx, range.end), {range, part_idx}, PartsRangesIterator::EventType::RangeEnd}); } From ee0fd39f2a1959a792977a114523bdd477c7ed16 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 24 Mar 2023 15:20:19 +0000 Subject: [PATCH 366/418] Fix data-race --- src/Interpreters/ProcessList.cpp | 10 ++++++++-- src/Interpreters/ProcessList.h | 2 ++ 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index c7916f3eed2..d7ba0b8330f 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -388,6 +388,12 @@ void QueryStatus::ExecutorHolder::remove() executor = nullptr; } +bool QueryStatus::ExecutorHolder::equals(const PipelineExecutor * e) +{ + std::lock_guard lock(mutex); + return executor == e; +} + CancellationCode QueryStatus::cancelQuery(bool) { if (is_killed.load()) @@ -428,7 +434,7 @@ void QueryStatus::addPipelineExecutor(PipelineExecutor * e) throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled"); std::lock_guard lock(executors_mutex); - assert(std::find_if(executors.begin(), executors.end(), [e](const ExecutorHolderPtr & x){ return x->executor == e; }) == executors.end()); + assert(std::find_if(executors.begin(), executors.end(), [e](ExecutorHolderPtr & x){ return x->equals(e); }) == executors.end()); executors.push_back(std::make_shared(e)); } @@ -437,7 +443,7 @@ void QueryStatus::removePipelineExecutor(PipelineExecutor * e) ExecutorHolderPtr executor_holder; { std::lock_guard lock(executors_mutex); - auto it = std::find_if(executors.begin(), executors.end(), [e](const ExecutorHolderPtr & x){ return x->executor == e; }); + auto it = std::find_if(executors.begin(), executors.end(), [e](ExecutorHolderPtr & x){ return x->equals(e); }); assert(it != executors.end()); executor_holder = *it; } diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index 30bfde4e218..98d7daa0949 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -127,6 +127,8 @@ protected: void remove(); + bool equals(const PipelineExecutor * e); + PipelineExecutor * executor; std::mutex mutex; }; From 1badc3cba05015972ea2690ba2caca4b4b3a7cef Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 24 Mar 2023 17:38:19 +0100 Subject: [PATCH 367/418] Move information about current hosts and list of all hosts to BackupCoordination (#47971) to simplify the code and help implementing other features. Co-authored-by: Nikita Mikhaylov --- src/Access/ReplicatedAccessStorage.cpp | 6 +- src/Backups/BackupCoordinationLocal.cpp | 24 ++-- src/Backups/BackupCoordinationLocal.h | 16 +-- src/Backups/BackupCoordinationRemote.cpp | 48 +++++--- src/Backups/BackupCoordinationRemote.h | 34 +++--- src/Backups/BackupEntriesCollector.cpp | 8 +- src/Backups/BackupsWorker.cpp | 104 +++++++++--------- src/Backups/IBackupCoordination.h | 16 +-- src/Backups/IRestoreCoordination.h | 8 +- src/Backups/RestoreCoordinationLocal.cpp | 8 +- src/Backups/RestoreCoordinationLocal.h | 8 +- src/Backups/RestoreCoordinationRemote.cpp | 24 ++-- src/Backups/RestoreCoordinationRemote.h | 26 +++-- src/Backups/RestorerFromBackup.cpp | 6 +- .../UserDefinedSQLObjectsBackup.cpp | 6 +- 15 files changed, 187 insertions(+), 155 deletions(-) diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp index a7cb2b6e08e..ddc5e8bfed1 100644 --- a/src/Access/ReplicatedAccessStorage.cpp +++ b/src/Access/ReplicatedAccessStorage.cpp @@ -674,18 +674,16 @@ void ReplicatedAccessStorage::backup(BackupEntriesCollector & backup_entries_col backup_entries_collector.getContext()->getAccessControl()); auto backup_coordination = backup_entries_collector.getBackupCoordination(); - String current_host_id = backup_entries_collector.getBackupSettings().host_id; - backup_coordination->addReplicatedAccessFilePath(zookeeper_path, type, current_host_id, backup_entry_with_path.first); + backup_coordination->addReplicatedAccessFilePath(zookeeper_path, type, backup_entry_with_path.first); backup_entries_collector.addPostTask( [backup_entry = backup_entry_with_path.second, zookeeper_path = zookeeper_path, type, - current_host_id, &backup_entries_collector, backup_coordination] { - for (const String & path : backup_coordination->getReplicatedAccessFilePaths(zookeeper_path, type, current_host_id)) + for (const String & path : backup_coordination->getReplicatedAccessFilePaths(zookeeper_path, type)) backup_entries_collector.addBackupEntry(path, backup_entry); }); } diff --git a/src/Backups/BackupCoordinationLocal.cpp b/src/Backups/BackupCoordinationLocal.cpp index 2d0bf978b83..90f64f15d97 100644 --- a/src/Backups/BackupCoordinationLocal.cpp +++ b/src/Backups/BackupCoordinationLocal.cpp @@ -13,20 +13,20 @@ using FileInfo = IBackupCoordination::FileInfo; BackupCoordinationLocal::BackupCoordinationLocal() = default; BackupCoordinationLocal::~BackupCoordinationLocal() = default; -void BackupCoordinationLocal::setStage(const String &, const String &, const String &) +void BackupCoordinationLocal::setStage(const String &, const String &) { } -void BackupCoordinationLocal::setError(const String &, const Exception &) +void BackupCoordinationLocal::setError(const Exception &) { } -Strings BackupCoordinationLocal::waitForStage(const Strings &, const String &) +Strings BackupCoordinationLocal::waitForStage(const String &) { return {}; } -Strings BackupCoordinationLocal::waitForStage(const Strings &, const String &, std::chrono::milliseconds) +Strings BackupCoordinationLocal::waitForStage(const String &, std::chrono::milliseconds) { return {}; } @@ -70,29 +70,29 @@ Strings BackupCoordinationLocal::getReplicatedDataPaths(const String & table_sha } -void BackupCoordinationLocal::addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id, const String & file_path) +void BackupCoordinationLocal::addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & file_path) { std::lock_guard lock{mutex}; - replicated_access.addFilePath(access_zk_path, access_entity_type, host_id, file_path); + replicated_access.addFilePath(access_zk_path, access_entity_type, "", file_path); } -Strings BackupCoordinationLocal::getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const +Strings BackupCoordinationLocal::getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type) const { std::lock_guard lock{mutex}; - return replicated_access.getFilePaths(access_zk_path, access_entity_type, host_id); + return replicated_access.getFilePaths(access_zk_path, access_entity_type, ""); } -void BackupCoordinationLocal::addReplicatedSQLObjectsDir(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & host_id, const String & dir_path) +void BackupCoordinationLocal::addReplicatedSQLObjectsDir(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & dir_path) { std::lock_guard lock{mutex}; - replicated_sql_objects.addDirectory(loader_zk_path, object_type, host_id, dir_path); + replicated_sql_objects.addDirectory(loader_zk_path, object_type, "", dir_path); } -Strings BackupCoordinationLocal::getReplicatedSQLObjectsDirs(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & host_id) const +Strings BackupCoordinationLocal::getReplicatedSQLObjectsDirs(const String & loader_zk_path, UserDefinedSQLObjectType object_type) const { std::lock_guard lock{mutex}; - return replicated_sql_objects.getDirectories(loader_zk_path, object_type, host_id); + return replicated_sql_objects.getDirectories(loader_zk_path, object_type, ""); } diff --git a/src/Backups/BackupCoordinationLocal.h b/src/Backups/BackupCoordinationLocal.h index edfa6c9973b..21db165be67 100644 --- a/src/Backups/BackupCoordinationLocal.h +++ b/src/Backups/BackupCoordinationLocal.h @@ -21,10 +21,10 @@ public: BackupCoordinationLocal(); ~BackupCoordinationLocal() override; - void setStage(const String & current_host, const String & new_stage, const String & message) override; - void setError(const String & current_host, const Exception & exception) override; - Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait) override; - Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait, std::chrono::milliseconds timeout) override; + void setStage(const String & new_stage, const String & message) override; + void setError(const Exception & exception) override; + Strings waitForStage(const String & stage_to_wait) override; + Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; void addReplicatedPartNames(const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, const std::vector & part_names_and_checksums) override; @@ -37,11 +37,11 @@ public: void addReplicatedDataPath(const String & table_shared_id, const String & data_path) override; Strings getReplicatedDataPaths(const String & table_shared_id) const override; - 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 addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & file_path) override; + Strings getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type) 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 addReplicatedSQLObjectsDir(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & dir_path) override; + Strings getReplicatedSQLObjectsDirs(const String & loader_zk_path, UserDefinedSQLObjectType object_type) const override; void addFileInfo(const FileInfo & file_info, bool & is_data_file_required) override; void updateFileInfo(const FileInfo & file_info) override; diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index e7ea9becd24..5ad95490c95 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -166,17 +166,30 @@ namespace } } +size_t BackupCoordinationRemote::findCurrentHostIndex(const Strings & all_hosts, const String & current_host) +{ + auto it = std::find(all_hosts.begin(), all_hosts.end(), current_host); + if (it == all_hosts.end()) + return 0; + return it - all_hosts.begin(); +} + BackupCoordinationRemote::BackupCoordinationRemote( - const BackupKeeperSettings & keeper_settings_, - const String & root_zookeeper_path_, - const String & backup_uuid_, zkutil::GetZooKeeper get_zookeeper_, + const String & root_zookeeper_path_, + const BackupKeeperSettings & keeper_settings_, + const String & backup_uuid_, + const Strings & all_hosts_, + const String & current_host_, bool is_internal_) - : keeper_settings(keeper_settings_) + : get_zookeeper(get_zookeeper_) , root_zookeeper_path(root_zookeeper_path_) , zookeeper_path(root_zookeeper_path_ + "/backup-" + backup_uuid_) + , keeper_settings(keeper_settings_) , backup_uuid(backup_uuid_) - , get_zookeeper(get_zookeeper_) + , all_hosts(all_hosts_) + , current_host(current_host_) + , current_host_index(findCurrentHostIndex(all_hosts, current_host)) , is_internal(is_internal_) { zookeeper_retries_info = ZooKeeperRetriesInfo( @@ -251,22 +264,22 @@ void BackupCoordinationRemote::removeAllNodes() } -void BackupCoordinationRemote::setStage(const String & current_host, const String & new_stage, const String & message) +void BackupCoordinationRemote::setStage(const String & new_stage, const String & message) { stage_sync->set(current_host, new_stage, message); } -void BackupCoordinationRemote::setError(const String & current_host, const Exception & exception) +void BackupCoordinationRemote::setError(const Exception & exception) { stage_sync->setError(current_host, exception); } -Strings BackupCoordinationRemote::waitForStage(const Strings & all_hosts, const String & stage_to_wait) +Strings BackupCoordinationRemote::waitForStage(const String & stage_to_wait) { return stage_sync->wait(all_hosts, stage_to_wait); } -Strings BackupCoordinationRemote::waitForStage(const Strings & all_hosts, const String & stage_to_wait, std::chrono::milliseconds timeout) +Strings BackupCoordinationRemote::waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) { return stage_sync->waitFor(all_hosts, stage_to_wait, timeout); } @@ -403,7 +416,7 @@ void BackupCoordinationRemote::prepareReplicatedTables() const } -void BackupCoordinationRemote::addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id, const String & file_path) +void BackupCoordinationRemote::addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & file_path) { { std::lock_guard lock{mutex}; @@ -416,15 +429,15 @@ void BackupCoordinationRemote::addReplicatedAccessFilePath(const String & access zk->createIfNotExists(path, ""); path += "/" + AccessEntityTypeInfo::get(access_entity_type).name; zk->createIfNotExists(path, ""); - path += "/" + host_id; + path += "/" + current_host; zk->createIfNotExists(path, file_path); } -Strings BackupCoordinationRemote::getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const +Strings BackupCoordinationRemote::getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type) const { std::lock_guard lock{mutex}; prepareReplicatedAccess(); - return replicated_access->getFilePaths(access_zk_path, access_entity_type, host_id); + return replicated_access->getFilePaths(access_zk_path, access_entity_type, current_host); } void BackupCoordinationRemote::prepareReplicatedAccess() const @@ -453,7 +466,7 @@ void BackupCoordinationRemote::prepareReplicatedAccess() const } } -void BackupCoordinationRemote::addReplicatedSQLObjectsDir(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & host_id, const String & dir_path) +void BackupCoordinationRemote::addReplicatedSQLObjectsDir(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & dir_path) { { std::lock_guard lock{mutex}; @@ -474,15 +487,15 @@ void BackupCoordinationRemote::addReplicatedSQLObjectsDir(const String & loader_ } zk->createIfNotExists(path, ""); - path += "/" + host_id; + path += "/" + current_host; zk->createIfNotExists(path, dir_path); } -Strings BackupCoordinationRemote::getReplicatedSQLObjectsDirs(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & host_id) const +Strings BackupCoordinationRemote::getReplicatedSQLObjectsDirs(const String & loader_zk_path, UserDefinedSQLObjectType object_type) const { std::lock_guard lock{mutex}; prepareReplicatedSQLObjects(); - return replicated_sql_objects->getDirectories(loader_zk_path, object_type, host_id); + return replicated_sql_objects->getDirectories(loader_zk_path, object_type, current_host); } void BackupCoordinationRemote::prepareReplicatedSQLObjects() const @@ -827,5 +840,4 @@ bool BackupCoordinationRemote::hasConcurrentBackups(const std::atomic &) return false; } - } diff --git a/src/Backups/BackupCoordinationRemote.h b/src/Backups/BackupCoordinationRemote.h index 74c6b58341a..268f20b9e39 100644 --- a/src/Backups/BackupCoordinationRemote.h +++ b/src/Backups/BackupCoordinationRemote.h @@ -27,17 +27,20 @@ public: }; BackupCoordinationRemote( - const BackupKeeperSettings & keeper_settings_, - const String & root_zookeeper_path_, - const String & backup_uuid_, zkutil::GetZooKeeper get_zookeeper_, + const String & root_zookeeper_path_, + const BackupKeeperSettings & keeper_settings_, + const String & backup_uuid_, + const Strings & all_hosts_, + const String & current_host_, bool is_internal_); + ~BackupCoordinationRemote() override; - void setStage(const String & current_host, const String & new_stage, const String & message) override; - void setError(const String & current_host, const Exception & exception) override; - Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait) override; - Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait, std::chrono::milliseconds timeout) override; + void setStage(const String & new_stage, const String & message) override; + void setError(const Exception & exception) override; + Strings waitForStage(const String & stage_to_wait) override; + Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; void addReplicatedPartNames( const String & table_shared_id, @@ -58,11 +61,11 @@ public: void addReplicatedDataPath(const String & table_shared_id, const String & data_path) override; Strings getReplicatedDataPaths(const String & table_shared_id) const override; - 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 addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & file_path) override; + Strings getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type) 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 addReplicatedSQLObjectsDir(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & dir_path) override; + Strings getReplicatedSQLObjectsDirs(const String & loader_zk_path, UserDefinedSQLObjectType object_type) const override; void addFileInfo(const FileInfo & file_info, bool & is_data_file_required) override; void updateFileInfo(const FileInfo & file_info) override; @@ -78,6 +81,8 @@ public: bool hasConcurrentBackups(const std::atomic & num_active_backups) const override; + static size_t findCurrentHostIndex(const Strings & all_hosts, const String & current_host); + private: zkutil::ZooKeeperPtr getZooKeeper() const; zkutil::ZooKeeperPtr getZooKeeperNoLock() const; @@ -91,11 +96,14 @@ private: void prepareReplicatedAccess() const; void prepareReplicatedSQLObjects() const; - const BackupKeeperSettings keeper_settings; + const zkutil::GetZooKeeper get_zookeeper; const String root_zookeeper_path; const String zookeeper_path; + const BackupKeeperSettings keeper_settings; const String backup_uuid; - const zkutil::GetZooKeeper get_zookeeper; + const Strings all_hosts; + const String current_host; + const size_t current_host_index; const bool is_internal; mutable ZooKeeperRetriesInfo zookeeper_retries_info; diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index 92526f0094e..fc3eab5b8f0 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -133,22 +133,22 @@ Strings BackupEntriesCollector::setStage(const String & new_stage, const String LOG_TRACE(log, fmt::runtime(toUpperFirst(new_stage))); current_stage = new_stage; - backup_coordination->setStage(backup_settings.host_id, new_stage, message); + backup_coordination->setStage(new_stage, message); if (new_stage == Stage::formatGatheringMetadata(1)) { - return backup_coordination->waitForStage(all_hosts, new_stage, on_cluster_first_sync_timeout); + return backup_coordination->waitForStage(new_stage, on_cluster_first_sync_timeout); } else if (new_stage.starts_with(Stage::GATHERING_METADATA)) { auto current_time = std::chrono::steady_clock::now(); auto end_of_timeout = std::max(current_time, consistent_metadata_snapshot_end_time); return backup_coordination->waitForStage( - all_hosts, new_stage, std::chrono::duration_cast(end_of_timeout - current_time)); + new_stage, std::chrono::duration_cast(end_of_timeout - current_time)); } else { - return backup_coordination->waitForStage(all_hosts, new_stage); + return backup_coordination->waitForStage(new_stage); } } diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 7699641a974..123a5ca93f1 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -38,14 +38,33 @@ namespace Stage = BackupCoordinationStage; namespace { - std::shared_ptr makeBackupCoordination(std::optional keeper_settings, String & root_zk_path, const String & backup_uuid, const ContextPtr & context, bool is_internal_backup) + std::shared_ptr makeBackupCoordination(const ContextPtr & context, const BackupSettings & backup_settings, bool remote) { - if (!root_zk_path.empty()) + if (remote) { - if (!keeper_settings.has_value()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Parameter keeper_settings is empty while root_zk_path is not. This is bug"); + String root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); + auto get_zookeeper = [global_context = context->getGlobalContext()] { return global_context->getZooKeeper(); }; - return std::make_shared(*keeper_settings, root_zk_path, backup_uuid, get_zookeeper, is_internal_backup); + + BackupCoordinationRemote::BackupKeeperSettings keeper_settings + { + .keeper_max_retries = context->getSettingsRef().backup_keeper_max_retries, + .keeper_retry_initial_backoff_ms = context->getSettingsRef().backup_keeper_retry_initial_backoff_ms, + .keeper_retry_max_backoff_ms = context->getSettingsRef().backup_keeper_retry_max_backoff_ms, + .batch_size_for_keeper_multiread = context->getSettingsRef().backup_batch_size_for_keeper_multiread, + }; + + auto all_hosts = BackupSettings::Util::filterHostIDs( + backup_settings.cluster_host_ids, backup_settings.shard_num, backup_settings.replica_num); + + return std::make_shared( + get_zookeeper, + root_zk_path, + keeper_settings, + toString(*backup_settings.backup_uuid), + all_hosts, + backup_settings.host_id, + backup_settings.internal); } else { @@ -53,12 +72,19 @@ namespace } } - std::shared_ptr makeRestoreCoordination(const String & root_zk_path, const String & restore_uuid, const ContextPtr & context, bool is_internal_backup) + std::shared_ptr + makeRestoreCoordination(const ContextPtr & context, const RestoreSettings & restore_settings, bool remote) { - if (!root_zk_path.empty()) + if (remote) { + String root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); + auto get_zookeeper = [global_context = context->getGlobalContext()] { return global_context->getZooKeeper(); }; - return std::make_shared(root_zk_path, restore_uuid, get_zookeeper, is_internal_backup); + + auto all_hosts = BackupSettings::Util::filterHostIDs( + restore_settings.cluster_host_ids, restore_settings.shard_num, restore_settings.replica_num); + + return std::make_shared(get_zookeeper, root_zk_path, toString(*restore_settings.restore_uuid), all_hosts, restore_settings.host_id, restore_settings.internal); } else { @@ -68,12 +94,12 @@ namespace /// Sends information about an exception to IBackupCoordination or IRestoreCoordination. template - void sendExceptionToCoordination(std::shared_ptr coordination, const String & current_host, const Exception & exception) + void sendExceptionToCoordination(std::shared_ptr coordination, const Exception & exception) { try { if (coordination) - coordination->setError(current_host, exception); + coordination->setError(exception); } catch (...) { @@ -82,7 +108,7 @@ namespace /// Sends information about the current exception to IBackupCoordination or IRestoreCoordination. template - void sendCurrentExceptionToCoordination(std::shared_ptr coordination, const String & current_host) + void sendCurrentExceptionToCoordination(std::shared_ptr coordination) { try { @@ -90,12 +116,12 @@ namespace } catch (const Exception & e) { - sendExceptionToCoordination(coordination, current_host, e); + sendExceptionToCoordination(coordination, e); } catch (...) { if (coordination) - coordination->setError(current_host, Exception(getCurrentExceptionMessageAndPattern(true, true), getCurrentExceptionCode())); + coordination->setError(Exception(getCurrentExceptionMessageAndPattern(true, true), getCurrentExceptionCode())); } } @@ -162,24 +188,13 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context else backup_id = toString(*backup_settings.backup_uuid); - String root_zk_path; - std::shared_ptr backup_coordination; if (backup_settings.internal) { /// The following call of makeBackupCoordination() is not essential because doBackup() will later create a backup coordination /// if it's not created here. However to handle errors better it's better to make a coordination here because this way /// if an exception will be thrown in startMakingBackup() other hosts will know about that. - root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); - - BackupCoordinationRemote::BackupKeeperSettings keeper_settings - { - .keeper_max_retries = context->getSettingsRef().backup_keeper_max_retries, - .keeper_retry_initial_backoff_ms = context->getSettingsRef().backup_keeper_retry_initial_backoff_ms, - .keeper_retry_max_backoff_ms = context->getSettingsRef().backup_keeper_retry_max_backoff_ms, - .batch_size_for_keeper_multiread = context->getSettingsRef().backup_batch_size_for_keeper_multiread, - }; - backup_coordination = makeBackupCoordination(keeper_settings, root_zk_path, toString(*backup_settings.backup_uuid), context, backup_settings.internal); + backup_coordination = makeBackupCoordination(context, backup_settings, /* remote= */ true); } auto backup_info = BackupInfo::fromAST(*backup_query->backup_name); @@ -238,7 +253,7 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context tryLogCurrentException(log, fmt::format("Failed to start {} {}", (backup_settings.internal ? "internal backup" : "backup"), backup_name_for_logging)); /// Something bad happened, the backup has not built. setStatusSafe(backup_id, BackupStatus::BACKUP_FAILED); - sendCurrentExceptionToCoordination(backup_coordination, backup_settings.host_id); + sendCurrentExceptionToCoordination(backup_coordination); throw; } } @@ -274,19 +289,9 @@ void BackupsWorker::doBackup( if (!on_cluster) context->checkAccess(required_access); - String root_zk_path; - std::optional keeper_settings; ClusterPtr cluster; if (on_cluster) { - keeper_settings = BackupCoordinationRemote::BackupKeeperSettings - { - .keeper_max_retries = context->getSettingsRef().backup_keeper_max_retries, - .keeper_retry_initial_backoff_ms = context->getSettingsRef().backup_keeper_retry_initial_backoff_ms, - .keeper_retry_max_backoff_ms = context->getSettingsRef().backup_keeper_retry_max_backoff_ms, - .batch_size_for_keeper_multiread = context->getSettingsRef().backup_batch_size_for_keeper_multiread, - }; - root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); backup_query->cluster = context->getMacros()->expand(backup_query->cluster); cluster = context->getCluster(backup_query->cluster); backup_settings.cluster_host_ids = cluster->getHostIDs(); @@ -294,7 +299,7 @@ void BackupsWorker::doBackup( /// Make a backup coordination. if (!backup_coordination) - backup_coordination = makeBackupCoordination(keeper_settings, root_zk_path, toString(*backup_settings.backup_uuid), context, backup_settings.internal); + backup_coordination = makeBackupCoordination(context, backup_settings, /* remote= */ on_cluster); if (!allow_concurrent_backups && backup_coordination->hasConcurrentBackups(std::ref(num_active_backups))) throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Concurrent backups not supported, turn on setting 'allow_concurrent_backups'"); @@ -330,9 +335,7 @@ void BackupsWorker::doBackup( executeDDLQueryOnCluster(backup_query, mutable_context, params); /// Wait until all the hosts have written their backup entries. - auto all_hosts = BackupSettings::Util::filterHostIDs( - backup_settings.cluster_host_ids, backup_settings.shard_num, backup_settings.replica_num); - backup_coordination->waitForStage(all_hosts, Stage::COMPLETED); + backup_coordination->waitForStage(Stage::COMPLETED); } else { @@ -349,7 +352,7 @@ void BackupsWorker::doBackup( writeBackupEntries(backup, std::move(backup_entries), backups_thread_pool); /// We have written our backup entries, we need to tell other hosts (they could be waiting for it). - backup_coordination->setStage(backup_settings.host_id, Stage::COMPLETED, ""); + backup_coordination->setStage(Stage::COMPLETED, ""); } size_t num_files = 0; @@ -383,7 +386,7 @@ void BackupsWorker::doBackup( { tryLogCurrentException(log, fmt::format("Failed to make {} {}", (backup_settings.internal ? "internal backup" : "backup"), backup_name_for_logging)); setStatusSafe(backup_id, BackupStatus::BACKUP_FAILED); - sendCurrentExceptionToCoordination(backup_coordination, backup_settings.host_id); + sendCurrentExceptionToCoordination(backup_coordination); } else { @@ -417,8 +420,7 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt /// The following call of makeRestoreCoordination() is not essential because doRestore() will later create a restore coordination /// if it's not created here. However to handle errors better it's better to make a coordination here because this way /// if an exception will be thrown in startRestoring() other hosts will know about that. - auto root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); - restore_coordination = makeRestoreCoordination(root_zk_path, toString(*restore_settings.restore_uuid), context, restore_settings.internal); + restore_coordination = makeRestoreCoordination(context, restore_settings, /* remote= */ true); } try @@ -474,7 +476,7 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt { /// Something bad happened, the backup has not built. setStatusSafe(restore_id, BackupStatus::RESTORE_FAILED); - sendCurrentExceptionToCoordination(restore_coordination, restore_settings.host_id); + sendCurrentExceptionToCoordination(restore_coordination); throw; } } @@ -509,14 +511,12 @@ void BackupsWorker::doRestore( BackupPtr backup = BackupFactory::instance().createBackup(backup_open_params); String current_database = context->getCurrentDatabase(); - String root_zk_path; /// Checks access rights if this is ON CLUSTER query. /// (If this isn't ON CLUSTER query RestorerFromBackup will check access rights later.) ClusterPtr cluster; bool on_cluster = !restore_query->cluster.empty(); if (on_cluster) { - root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); restore_query->cluster = context->getMacros()->expand(restore_query->cluster); cluster = context->getCluster(restore_query->cluster); restore_settings.cluster_host_ids = cluster->getHostIDs(); @@ -539,7 +539,7 @@ void BackupsWorker::doRestore( /// Make a restore coordination. if (!restore_coordination) - restore_coordination = makeRestoreCoordination(root_zk_path, toString(*restore_settings.restore_uuid), context, restore_settings.internal); + restore_coordination = makeRestoreCoordination(context, restore_settings, /* remote= */ on_cluster); if (!allow_concurrent_restores && restore_coordination->hasConcurrentRestores(std::ref(num_active_restores))) throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Concurrent restores not supported, turn on setting 'allow_concurrent_restores'"); @@ -561,9 +561,7 @@ void BackupsWorker::doRestore( executeDDLQueryOnCluster(restore_query, context, params); /// Wait until all the hosts have written their backup entries. - auto all_hosts = BackupSettings::Util::filterHostIDs( - restore_settings.cluster_host_ids, restore_settings.shard_num, restore_settings.replica_num); - restore_coordination->waitForStage(all_hosts, Stage::COMPLETED); + restore_coordination->waitForStage(Stage::COMPLETED); } else { @@ -581,7 +579,7 @@ void BackupsWorker::doRestore( restoreTablesData(std::move(data_restore_tasks), restores_thread_pool); /// We have restored everything, we need to tell other hosts (they could be waiting for it). - restore_coordination->setStage(restore_settings.host_id, Stage::COMPLETED, ""); + restore_coordination->setStage(Stage::COMPLETED, ""); } LOG_INFO(log, "Restored from {} {} successfully", (restore_settings.internal ? "internal backup" : "backup"), backup_name_for_logging); @@ -603,7 +601,7 @@ void BackupsWorker::doRestore( { tryLogCurrentException(log, fmt::format("Failed to restore from {} {}", (restore_settings.internal ? "internal backup" : "backup"), backup_name_for_logging)); setStatusSafe(restore_id, BackupStatus::RESTORE_FAILED); - sendCurrentExceptionToCoordination(restore_coordination, restore_settings.host_id); + sendCurrentExceptionToCoordination(restore_coordination); } else { diff --git a/src/Backups/IBackupCoordination.h b/src/Backups/IBackupCoordination.h index f87aa8b8f41..26f101f29e5 100644 --- a/src/Backups/IBackupCoordination.h +++ b/src/Backups/IBackupCoordination.h @@ -22,10 +22,10 @@ public: virtual ~IBackupCoordination() = default; /// Sets the current stage and waits for other hosts to come to this stage too. - virtual void setStage(const String & current_host, const String & new_stage, const String & message) = 0; - virtual void setError(const String & current_host, const Exception & exception) = 0; - virtual Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait) = 0; - virtual Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait, std::chrono::milliseconds timeout) = 0; + virtual void setStage(const String & new_stage, const String & message) = 0; + virtual void setError(const Exception & exception) = 0; + virtual Strings waitForStage(const String & stage_to_wait) = 0; + virtual Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) = 0; struct PartNameAndChecksum { @@ -66,12 +66,12 @@ public: virtual Strings getReplicatedDataPaths(const String & table_shared_id) const = 0; /// Adds a path to access.txt file keeping access entities of a ReplicatedAccessStorage. - 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; + virtual void addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & file_path) = 0; + virtual Strings getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type) 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; + virtual void addReplicatedSQLObjectsDir(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & dir_path) = 0; + virtual Strings getReplicatedSQLObjectsDirs(const String & loader_zk_path, UserDefinedSQLObjectType object_type) const = 0; struct FileInfo { diff --git a/src/Backups/IRestoreCoordination.h b/src/Backups/IRestoreCoordination.h index 61aee533dd2..2f9e8d171f6 100644 --- a/src/Backups/IRestoreCoordination.h +++ b/src/Backups/IRestoreCoordination.h @@ -18,10 +18,10 @@ public: virtual ~IRestoreCoordination() = default; /// Sets the current stage and waits for other hosts to come to this stage too. - virtual void setStage(const String & current_host, const String & new_stage, const String & message) = 0; - virtual void setError(const String & current_host, const Exception & exception) = 0; - virtual Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait) = 0; - virtual Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait, std::chrono::milliseconds timeout) = 0; + virtual void setStage(const String & new_stage, const String & message) = 0; + virtual void setError(const Exception & exception) = 0; + virtual Strings waitForStage(const String & stage_to_wait) = 0; + virtual Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) = 0; static constexpr const char * kErrorStatus = "error"; diff --git a/src/Backups/RestoreCoordinationLocal.cpp b/src/Backups/RestoreCoordinationLocal.cpp index d5334e2c6aa..191cde40aa1 100644 --- a/src/Backups/RestoreCoordinationLocal.cpp +++ b/src/Backups/RestoreCoordinationLocal.cpp @@ -7,20 +7,20 @@ namespace DB RestoreCoordinationLocal::RestoreCoordinationLocal() = default; RestoreCoordinationLocal::~RestoreCoordinationLocal() = default; -void RestoreCoordinationLocal::setStage(const String &, const String &, const String &) +void RestoreCoordinationLocal::setStage(const String &, const String &) { } -void RestoreCoordinationLocal::setError(const String &, const Exception &) +void RestoreCoordinationLocal::setError(const Exception &) { } -Strings RestoreCoordinationLocal::waitForStage(const Strings &, const String &) +Strings RestoreCoordinationLocal::waitForStage(const String &) { return {}; } -Strings RestoreCoordinationLocal::waitForStage(const Strings &, const String &, std::chrono::milliseconds) +Strings RestoreCoordinationLocal::waitForStage(const String &, std::chrono::milliseconds) { return {}; } diff --git a/src/Backups/RestoreCoordinationLocal.h b/src/Backups/RestoreCoordinationLocal.h index cb3a8c55d22..bbe76cdf5fd 100644 --- a/src/Backups/RestoreCoordinationLocal.h +++ b/src/Backups/RestoreCoordinationLocal.h @@ -19,10 +19,10 @@ public: ~RestoreCoordinationLocal() override; /// Sets the current stage and waits for other hosts to come to this stage too. - void setStage(const String & current_host, const String & new_stage, const String & message) override; - void setError(const String & current_host, const Exception & exception) override; - Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait) override; - Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait, std::chrono::milliseconds timeout) override; + void setStage(const String & new_stage, const String & message) override; + void setError(const Exception & exception) override; + Strings waitForStage(const String & stage_to_wait) override; + Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; /// Starts creating a table in a replicated database. Returns false if there is another host which is already creating this table. bool acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) override; diff --git a/src/Backups/RestoreCoordinationRemote.cpp b/src/Backups/RestoreCoordinationRemote.cpp index 646c2c68d3b..10d085a696a 100644 --- a/src/Backups/RestoreCoordinationRemote.cpp +++ b/src/Backups/RestoreCoordinationRemote.cpp @@ -11,11 +11,19 @@ namespace DB namespace Stage = BackupCoordinationStage; RestoreCoordinationRemote::RestoreCoordinationRemote( - const String & root_zookeeper_path_, const String & restore_uuid_, zkutil::GetZooKeeper get_zookeeper_, bool is_internal_) - : root_zookeeper_path(root_zookeeper_path_) - , zookeeper_path(root_zookeeper_path_ + "/restore-" + restore_uuid_) + zkutil::GetZooKeeper get_zookeeper_, + const String & root_zookeeper_path_, + const String & restore_uuid_, + const Strings & all_hosts_, + const String & current_host_, + bool is_internal_) + : get_zookeeper(get_zookeeper_) + , root_zookeeper_path(root_zookeeper_path_) , restore_uuid(restore_uuid_) - , get_zookeeper(get_zookeeper_) + , zookeeper_path(root_zookeeper_path_ + "/restore-" + restore_uuid_) + , all_hosts(all_hosts_) + , current_host(current_host_) + , current_host_index(BackupCoordinationRemote::findCurrentHostIndex(all_hosts, current_host)) , is_internal(is_internal_) { createRootNodes(); @@ -63,22 +71,22 @@ void RestoreCoordinationRemote::createRootNodes() } -void RestoreCoordinationRemote::setStage(const String & current_host, const String & new_stage, const String & message) +void RestoreCoordinationRemote::setStage(const String & new_stage, const String & message) { stage_sync->set(current_host, new_stage, message); } -void RestoreCoordinationRemote::setError(const String & current_host, const Exception & exception) +void RestoreCoordinationRemote::setError(const Exception & exception) { stage_sync->setError(current_host, exception); } -Strings RestoreCoordinationRemote::waitForStage(const Strings & all_hosts, const String & stage_to_wait) +Strings RestoreCoordinationRemote::waitForStage(const String & stage_to_wait) { return stage_sync->wait(all_hosts, stage_to_wait); } -Strings RestoreCoordinationRemote::waitForStage(const Strings & all_hosts, const String & stage_to_wait, std::chrono::milliseconds timeout) +Strings RestoreCoordinationRemote::waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) { return stage_sync->waitFor(all_hosts, stage_to_wait, timeout); } diff --git a/src/Backups/RestoreCoordinationRemote.h b/src/Backups/RestoreCoordinationRemote.h index 4ee87796e8e..b78c2e96f9e 100644 --- a/src/Backups/RestoreCoordinationRemote.h +++ b/src/Backups/RestoreCoordinationRemote.h @@ -11,14 +11,21 @@ namespace DB class RestoreCoordinationRemote : public IRestoreCoordination { public: - RestoreCoordinationRemote(const String & root_zookeeper_path_, const String & restore_uuid_, zkutil::GetZooKeeper get_zookeeper_, bool is_internal_); + RestoreCoordinationRemote( + zkutil::GetZooKeeper get_zookeeper_, + const String & root_zookeeper_path_, + const String & restore_uuid_, + const Strings & all_hosts_, + const String & current_host_, + bool is_internal_); + ~RestoreCoordinationRemote() override; /// Sets the current stage and waits for other hosts to come to this stage too. - void setStage(const String & current_host, const String & new_stage, const String & message) override; - void setError(const String & current_host, const Exception & exception) override; - Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait) override; - Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait, std::chrono::milliseconds timeout) override; + void setStage(const String & new_stage, const String & message) override; + void setError(const Exception & exception) override; + Strings waitForStage(const String & stage_to_wait) override; + Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; /// Starts creating a table in a replicated database. Returns false if there is another host which is already creating this table. bool acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) override; @@ -44,10 +51,13 @@ private: class ReplicatedDatabasesMetadataSync; - const String root_zookeeper_path; - const String zookeeper_path; - const String restore_uuid; const zkutil::GetZooKeeper get_zookeeper; + const String root_zookeeper_path; + const String restore_uuid; + const String zookeeper_path; + const Strings all_hosts; + const String current_host; + const size_t current_host_index; const bool is_internal; std::optional stage_sync; diff --git a/src/Backups/RestorerFromBackup.cpp b/src/Backups/RestorerFromBackup.cpp index 68a68379f79..77f7512e0d1 100644 --- a/src/Backups/RestorerFromBackup.cpp +++ b/src/Backups/RestorerFromBackup.cpp @@ -150,11 +150,11 @@ void RestorerFromBackup::setStage(const String & new_stage, const String & messa if (restore_coordination) { - restore_coordination->setStage(restore_settings.host_id, new_stage, message); + restore_coordination->setStage(new_stage, message); if (new_stage == Stage::FINDING_TABLES_IN_BACKUP) - restore_coordination->waitForStage(all_hosts, new_stage, on_cluster_first_sync_timeout); + restore_coordination->waitForStage(new_stage, on_cluster_first_sync_timeout); else - restore_coordination->waitForStage(all_hosts, new_stage); + restore_coordination->waitForStage(new_stage); } } diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsBackup.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsBackup.cpp index 0fc86bf39ba..53d8ebc7b61 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsBackup.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsBackup.cpp @@ -48,10 +48,9 @@ void backupUserDefinedSQLObjects( } 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); + backup_coordination->addReplicatedSQLObjectsDir(replication_id, object_type, 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. @@ -60,11 +59,10 @@ void backupUserDefinedSQLObjects( [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); + auto dirs = backup_coordination->getReplicatedSQLObjectsDirs(replication_id, object_type); for (const auto & dir : dirs) { From 601aaf4d2f4ce862c96d6472e8fde9d0bdb10c7b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 24 Mar 2023 21:33:24 +0300 Subject: [PATCH 368/418] Update tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql --- .../queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 6bfd3a986cd..27b4a8bf83d 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 @@ -6,7 +6,7 @@ select parseDateTimeInJodaSyntax(' ', ' ', 'UTC') = toDateTime('1970-01-01', 'UT select parseDateTimeInJodaSyntax('AD 1999', 'G YYYY', 'UTC') = toDateTime('1999-01-01', 'UTC'); select parseDateTimeInJodaSyntax('ad 1999', 'G YYYY', 'UTC') = toDateTime('1999-01-01', 'UTC'); select parseDateTimeInJodaSyntax('Ad 1999', 'G YYYY', 'UTC') = toDateTime('1999-01-01', 'UTC'); -select parseDateTimeInJodaSyntax('AD 1999', 'G yyyy') = toDateTime('1999-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('AD 1999', 'G yyyy', 'UTC') = toDateTime('1999-01-01', 'UTC'); select parseDateTimeInJodaSyntax('AD 1999 2000', 'G YYYY yyyy', 'UTC') = toDateTime('2000-01-01', 'UTC'); select parseDateTimeInJodaSyntax('AD 1999 2000', 'G yyyy YYYY', 'UTC') = toDateTime('2000-01-01', 'UTC'); select parseDateTimeInJodaSyntax('AD 1999', 'G Y'); -- { serverError CANNOT_PARSE_DATETIME } From 6accefe114e0edce8a29cfd4b8256c4f2026718a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 24 Mar 2023 21:33:31 +0300 Subject: [PATCH 369/418] Update tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference --- .../0_stateless/02668_parse_datetime_in_joda_syntax.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 21c2754925d..9391af957a1 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 @@ -9,7 +9,7 @@ select parseDateTimeInJodaSyntax('ad 1999', 'G YYYY', 'UTC') = toDateTime('1999- 1 select parseDateTimeInJodaSyntax('Ad 1999', 'G YYYY', 'UTC') = toDateTime('1999-01-01', 'UTC'); 1 -select parseDateTimeInJodaSyntax('AD 1999', 'G yyyy') = toDateTime('1999-01-01', 'UTC'); +select parseDateTimeInJodaSyntax('AD 1999', 'G yyyy', 'UTC') = toDateTime('1999-01-01', 'UTC'); 1 select parseDateTimeInJodaSyntax('AD 1999 2000', 'G YYYY yyyy', 'UTC') = toDateTime('2000-01-01', 'UTC'); 1 From d00587202dd2b1aad5188b09acdb4fe40c950cde Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 24 Mar 2023 19:25:07 +0000 Subject: [PATCH 370/418] Make better --- src/Interpreters/ProcessList.cpp | 17 +++++++---------- src/Interpreters/ProcessList.h | 3 +-- 2 files changed, 8 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index d7ba0b8330f..a86377a37d5 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -388,12 +388,6 @@ void QueryStatus::ExecutorHolder::remove() executor = nullptr; } -bool QueryStatus::ExecutorHolder::equals(const PipelineExecutor * e) -{ - std::lock_guard lock(mutex); - return executor == e; -} - CancellationCode QueryStatus::cancelQuery(bool) { if (is_killed.load()) @@ -434,19 +428,22 @@ void QueryStatus::addPipelineExecutor(PipelineExecutor * e) throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled"); std::lock_guard lock(executors_mutex); - assert(std::find_if(executors.begin(), executors.end(), [e](ExecutorHolderPtr & x){ return x->equals(e); }) == executors.end()); + assert(!executor_indexes.contains(e)); executors.push_back(std::make_shared(e)); + executor_indexes[e] = executors.size() - 1; } void QueryStatus::removePipelineExecutor(PipelineExecutor * e) { ExecutorHolderPtr executor_holder; + { std::lock_guard lock(executors_mutex); - auto it = std::find_if(executors.begin(), executors.end(), [e](ExecutorHolderPtr & x){ return x->equals(e); }); - assert(it != executors.end()); - executor_holder = *it; + assert(executor_indexes.contains(e)); + executor_holder = executors[executor_indexes[e]]; + executor_indexes.erase(e); } + /// Invalidate executor pointer inside holder, but don't remove holder from the executors (to avoid race with cancelQuery) /// We should do it with released executors_mutex to avoid possible lock order inversion. executor_holder->remove(); diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index 98d7daa0949..9e5c91bf2ed 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -127,8 +127,6 @@ protected: void remove(); - bool equals(const PipelineExecutor * e); - PipelineExecutor * executor; std::mutex mutex; }; @@ -137,6 +135,7 @@ protected: /// Array of PipelineExecutors to be cancelled when a cancelQuery is received std::vector executors; + std::unordered_map executor_indexes; enum QueryStreamsStatus { From 6c3765c8b663554efde86816395385b97dcabb9c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 24 Mar 2023 20:40:45 +0000 Subject: [PATCH 371/418] Small fix --- src/Coordination/KeeperDispatcher.cpp | 48 +++++++++++++++---------- src/Coordination/KeeperDispatcher.h | 16 ++------- src/Coordination/KeeperStateMachine.cpp | 4 ++- src/Coordination/KeeperStateMachine.h | 2 +- 4 files changed, 37 insertions(+), 33 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index fc97bc5e0e1..69abf85ae4e 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -110,9 +110,9 @@ void KeeperDispatcher::requestThread() if (!coordination_settings->quorum_reads && request.request->isReadRequest()) { ++read_requests; - std::pair key{current_batch.back().session_id, current_batch.back().request->xid}; - std::lock_guard lock(read_mutex); - related_read_requests[key].push_back(request); + const auto & last_request = current_batch.back(); + std::lock_guard lock(read_request_queue_mutex); + read_request_queue[last_request.session_id][last_request.request->xid].push_back(request); } else current_batch.emplace_back(request); @@ -328,18 +328,24 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf server = std::make_unique(configuration_and_settings, config, responses_queue, snapshots_queue, snapshot_s3, [this](const KeeperStorage::RequestForSession & request_for_session) { - std::lock_guard lock(read_mutex); - if (auto it = related_read_requests.find(std::pair{request_for_session.session_id, request_for_session.request->xid}); it != related_read_requests.end()) + /// check if we have queue of read requests depending on this request to be committed + std::lock_guard lock(read_request_queue_mutex); + if (auto it = read_request_queue.find(request_for_session.session_id); it != read_request_queue.end()) { - for (const auto & read_request : it->second) - { - if (server->isLeaderAlive()) - server->putLocalReadRequest(read_request); - else - addErrorResponses({read_request}, Coordination::Error::ZCONNECTIONLOSS); - } + auto & xid_to_request_queue = it->second; - related_read_requests.erase(it); + if (auto request_queue_it = xid_to_request_queue.find(request_for_session.request->xid); request_queue_it != xid_to_request_queue.end()) + { + for (const auto & read_request : request_queue_it->second) + { + if (server->isLeaderAlive()) + server->putLocalReadRequest(read_request); + else + addErrorResponses({read_request}, Coordination::Error::ZCONNECTIONLOSS); + } + + xid_to_request_queue.erase(request_queue_it); + } } }); @@ -554,12 +560,18 @@ void KeeperDispatcher::sessionCleanerTask() void KeeperDispatcher::finishSession(int64_t session_id) { - std::lock_guard lock(session_to_response_callback_mutex); - auto session_it = session_to_response_callback.find(session_id); - if (session_it != session_to_response_callback.end()) { - session_to_response_callback.erase(session_it); - CurrentMetrics::sub(CurrentMetrics::KeeperAliveConnections); + std::lock_guard lock(session_to_response_callback_mutex); + auto session_it = session_to_response_callback.find(session_id); + if (session_it != session_to_response_callback.end()) + { + session_to_response_callback.erase(session_it); + CurrentMetrics::sub(CurrentMetrics::KeeperAliveConnections); + } + } + { + std::lock_guard lock(read_request_queue_mutex); + read_request_queue.erase(session_id); } } diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index e7570727b9a..77b5510cbb3 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -104,20 +104,10 @@ private: void forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorage::RequestsForSessions & requests_for_sessions); public: - std::mutex read_mutex; + std::mutex read_request_queue_mutex; - struct PairHash - { - auto operator()(std::pair pair) const - { - SipHash hash; - hash.update(pair.first); - hash.update(pair.second); - return hash.get64(); - } - }; - - std::unordered_map, KeeperStorage::RequestsForSessions, PairHash> related_read_requests; + /// queue of read requests that can be processed after a request with specific session ID and XID is committed + std::unordered_map> read_request_queue; /// Just allocate some objects, real initialization is done by `intialize method` KeeperDispatcher(); diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 0b69b00bf0e..632aaec6b54 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -276,7 +276,9 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n ProfileEvents::increment(ProfileEvents::KeeperCommits); last_committed_idx = log_idx; - commit_callback(request_for_session); + + if (commit_callback) + commit_callback(request_for_session); return nullptr; } diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 6babf741dbd..e4f0295db99 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -31,7 +31,7 @@ public: const CoordinationSettingsPtr & coordination_settings_, const KeeperContextPtr & keeper_context_, KeeperSnapshotManagerS3 * snapshot_manager_s3_, - CommitCallback commit_callback_, + CommitCallback commit_callback_ = {}, const std::string & superdigest_ = ""); /// Read state from the latest snapshot From 585b7b934508cf9a08b6218710dc90aa510df002 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Fri, 24 Mar 2023 23:36:45 +0100 Subject: [PATCH 372/418] Better type check in arrayElement function --- src/Functions/array/arrayElement.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/array/arrayElement.cpp b/src/Functions/array/arrayElement.cpp index 789b2b55ca5..e972a6846da 100644 --- a/src/Functions/array/arrayElement.cpp +++ b/src/Functions/array/arrayElement.cpp @@ -1065,7 +1065,7 @@ DataTypePtr FunctionArrayElement::getReturnTypeImpl(const DataTypes & arguments) getName(), arguments[0]->getName()); } - if (!isInteger(arguments[1])) + if (!isNativeInteger(arguments[1])) { throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument for function '{}' must be integer, got '{}' instead", From 24b5b32dc341f2b87598819e436741e41c809fa5 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Fri, 24 Mar 2023 22:28:14 -0600 Subject: [PATCH 373/418] Update user.md --- .../sql-reference/statements/create/user.md | 56 ++++++++++++++++++- 1 file changed, 55 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/create/user.md b/docs/en/sql-reference/statements/create/user.md index 454195db3fa..a86423b7a3d 100644 --- a/docs/en/sql-reference/statements/create/user.md +++ b/docs/en/sql-reference/statements/create/user.md @@ -36,7 +36,61 @@ There are multiple ways of user identification: - `IDENTIFIED WITH kerberos` or `IDENTIFIED WITH kerberos REALM 'realm'` - `IDENTIFIED WITH ssl_certificate CN 'mysite.com:user'` -For identification with sha256_hash using `SALT` - hash must be calculated from concatination of 'password' and 'salt'. +## Examples + +1. The following username is `name1` and does not require a password - which obviously doesn't provide much security: + + ```sql + CREATE USER name1 NOT IDENTIFIED + ``` + +2. To specify a plaintext password: + + ```sql + CREATE USER name2 IDENTIFIED WITH plaintext_password BY 'my_password' + ``` + + :::warning + The password is stored in a SQL text file in `/var/lib/clickhouse/access`, so it's not a good idea to use `plaintext_password`. Try `sha256_password` instead, as demonstrated next... + ::: + +3. The best option is o use a password that is hashed using SHA-256. ClickHouse will hash the password for you when you specify `IDENTIFIED WITH sha256_password`. For example: + + ```sql + CREATE USER name3 IDENTIFIED WITH sha256_password BY 'my_password' + ``` + + Notice ClickHouse generates and runs the following command for you: + + ```response + CREATE USER name3 + IDENTIFIED WITH sha256_hash + BY '8B3404953FCAA509540617F082DB13B3E0734F90FF6365C19300CC6A6EA818D6' + SALT 'D6489D8B5692D82FF944EA6415785A8A8A1AF33825456AFC554487725A74A609' + ``` + + The `name3` user can now login using `my_password`, but the passowrd is stored as the hashed value above. THe following SQL file was created in `/var/lib/clickhouse/access` and gets executed at server startup: + + ```bash + /var/lib/clickhouse/access $ cat 3843f510-6ebd-a52d-72ac-e021686d8a93.sql + ATTACH USER name3 IDENTIFIED WITH sha256_hash BY '0C268556C1680BEF0640AAC1E7187566704208398DA31F03D18C74F5C5BE5053' SALT '4FB16307F5E10048196966DD7E6876AE53DE6A1D1F625488482C75F14A5097C7'; + ``` + + :::note + If you have already created a hash value and corresponding salt value for a username, then you can use `IDENTIFIED WITH sha256_hash BY 'hash'` or `IDENTIFIED WITH sha256_hash BY 'hash' SALT 'salt'`. For identification with `sha256_hash` using `SALT` - hash must be calculated from concatenation of 'password' and 'salt'. + ::: + +4. The `double_sha1_password` is not typically needed, but comes in handy when working with clients that require it (like the MySQL interface): + + ```sql + CREATE USER name4 IDENTIFIED WITH double_sha1_password BY 'my_password' + ``` + + ClickHouse generates and runs the following query: + + ```response + CREATE USER name4 IDENTIFIED WITH double_sha1_hash BY 'CCD3A959D6A004B9C3807B728BC2E55B67E10518' + ``` ## User Host From 9f548d658ec888cead96f8ffd3a96380f9a24b16 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Fri, 24 Mar 2023 22:45:24 -0600 Subject: [PATCH 374/418] Update user.md --- docs/en/sql-reference/statements/create/user.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/create/user.md b/docs/en/sql-reference/statements/create/user.md index a86423b7a3d..fc1fd10571c 100644 --- a/docs/en/sql-reference/statements/create/user.md +++ b/docs/en/sql-reference/statements/create/user.md @@ -54,7 +54,7 @@ There are multiple ways of user identification: The password is stored in a SQL text file in `/var/lib/clickhouse/access`, so it's not a good idea to use `plaintext_password`. Try `sha256_password` instead, as demonstrated next... ::: -3. The best option is o use a password that is hashed using SHA-256. ClickHouse will hash the password for you when you specify `IDENTIFIED WITH sha256_password`. For example: +3. The best option is to use a password that is hashed using SHA-256. ClickHouse will hash the password for you when you specify `IDENTIFIED WITH sha256_password`. For example: ```sql CREATE USER name3 IDENTIFIED WITH sha256_password BY 'my_password' From ac72ed035cbc3430e5d99a860c6f55d661033776 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Fri, 24 Mar 2023 22:50:29 -0600 Subject: [PATCH 375/418] Update user.md --- docs/en/sql-reference/statements/create/user.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/create/user.md b/docs/en/sql-reference/statements/create/user.md index fc1fd10571c..a38523ee37b 100644 --- a/docs/en/sql-reference/statements/create/user.md +++ b/docs/en/sql-reference/statements/create/user.md @@ -69,7 +69,7 @@ There are multiple ways of user identification: SALT 'D6489D8B5692D82FF944EA6415785A8A8A1AF33825456AFC554487725A74A609' ``` - The `name3` user can now login using `my_password`, but the passowrd is stored as the hashed value above. THe following SQL file was created in `/var/lib/clickhouse/access` and gets executed at server startup: + The `name3` user can now login using `my_password`, but the password is stored as the hashed value above. THe following SQL file was created in `/var/lib/clickhouse/access` and gets executed at server startup: ```bash /var/lib/clickhouse/access $ cat 3843f510-6ebd-a52d-72ac-e021686d8a93.sql From 2c8916a00573dd40e38db57b15ca656f83d4d793 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 25 Mar 2023 18:05:09 +0000 Subject: [PATCH 376/418] Fix clang-tidy --- src/Bridge/IBridge.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Bridge/IBridge.cpp b/src/Bridge/IBridge.cpp index 62365f4c93c..1ea77573e5f 100644 --- a/src/Bridge/IBridge.cpp +++ b/src/Bridge/IBridge.cpp @@ -121,7 +121,7 @@ void IBridge::defineOptions(Poco::Util::OptionSet & options) options.addOption( Poco::Util::Option("help", "", "produce this help message").binding("help").callback(Poco::Util::OptionCallback(this, &Me::handleHelp))); - ServerApplication::defineOptions(options); // Don't need complex BaseDaemon's .xml config + ServerApplication::defineOptions(options); // NOLINT Don't need complex BaseDaemon's .xml config } From b46ea5572d30778975b9b97d466337dd1efc0451 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 25 Mar 2023 18:06:03 +0000 Subject: [PATCH 377/418] Fix expected results --- tests/integration/test_catboost_evaluate/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_catboost_evaluate/test.py b/tests/integration/test_catboost_evaluate/test.py index 0aeffdb3e02..7412d34dd40 100644 --- a/tests/integration/test_catboost_evaluate/test.py +++ b/tests/integration/test_catboost_evaluate/test.py @@ -288,7 +288,7 @@ def testAmazonModelManyRows(ch_cluster): "SELECT floor(sum(catboostEvaluate('/etc/clickhouse-server/model/amazon_model.bin', RESOURCE, MGR_ID, ROLE_ROLLUP_1, ROLE_ROLLUP_2, ROLE_DEPTNAME, ROLE_TITLE, ROLE_FAMILY_DESC, ROLE_FAMILY, ROLE_CODE))) FROM amazon" ) - expected = "5834\n" + expected = "583092\n" assert result == expected result = instance.query("drop table if exists amazon") From e46ec88ccb729fb24a97869a926d3cbd80d97c8c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sat, 25 Mar 2023 18:26:36 +0000 Subject: [PATCH 378/418] Automatic style fix --- docker/test/performance-comparison/perf.py | 2 ++ docker/test/performance-comparison/report.py | 1 - tests/ci/clickhouse_helper.py | 1 - tests/ci/docker_images_check.py | 1 - tests/ci/get_previous_release_tag.py | 1 - tests/ci/report.py | 2 +- tests/integration/helpers/cluster.py | 3 +-- tests/integration/helpers/network.py | 2 -- .../pytest_xdist_logging_to_separate_files.py | 1 + .../test_detach_part_wrong_partition_id.py | 1 - .../test_cluster_copier/test_three_nodes.py | 1 - .../test_cluster_copier/test_two_nodes.py | 1 - tests/integration/test_composable_protocols/test.py | 1 - .../test_create_query_constraints/test.py | 2 -- .../common.py | 1 - tests/integration/test_disks_app_func/test.py | 1 - .../test_distributed_ddl_parallel/test.py | 1 + tests/integration/test_fetch_memory_usage/test.py | 1 - .../scripts/stress_test.py | 1 - tests/integration/test_jbod_balancer/test.py | 1 - .../test_keeper_and_access_storage/test.py | 1 + tests/integration/test_keeper_back_to_back/test.py | 2 +- tests/integration/test_keeper_persistent_log/test.py | 1 - .../test_keeper_zookeeper_converter/test.py | 1 - tests/integration/test_merge_tree_load_parts/test.py | 6 +++--- .../s3_endpoint/endpoint.py | 1 - .../test_merge_tree_settings_constraints/test.py | 1 - .../test_old_parts_finally_removed/test.py | 1 - tests/integration/test_partition/test.py | 4 +++- tests/integration/test_password_constraints/test.py | 1 - tests/integration/test_read_only_table/test.py | 1 - .../test_reload_auxiliary_zookeepers/test.py | 1 - .../s3_endpoint/endpoint.py | 1 + tests/integration/test_s3_with_proxy/test.py | 1 + .../integration/test_ssl_cert_authentication/test.py | 1 - tests/integration/test_storage_kafka/kafka_pb2.py | 1 - .../test_storage_kafka/message_with_repeated_pb2.py | 1 - tests/integration/test_storage_kafka/social_pb2.py | 1 - tests/integration/test_storage_kafka/test.py | 12 ++---------- tests/integration/test_storage_nats/nats_pb2.py | 1 - .../test_storage_postgresql_replica/test.py | 1 - .../test_storage_rabbitmq/rabbitmq_pb2.py | 1 - tests/integration/test_storage_rabbitmq/test.py | 3 --- tests/integration/test_storage_s3/test.py | 1 + .../test_storage_s3/test_invalid_env_credentials.py | 1 + tests/integration/test_system_merges/test.py | 1 - tests/integration/test_ttl_move/test.py | 2 +- tests/integration/test_zero_copy_fetch/test.py | 1 - utils/changelog-simple/format-changelog.py | 1 + utils/keeper-overload/keeper-overload.py | 2 +- 50 files changed, 23 insertions(+), 57 deletions(-) diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 65bf49c2914..7a4e6386d0d 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -26,6 +26,7 @@ logging.basicConfig( total_start_seconds = time.perf_counter() stage_start_seconds = total_start_seconds + # Thread executor that does not hides exception that happens during function # execution, and rethrows it after join() class SafeThread(Thread): @@ -158,6 +159,7 @@ for e in subst_elems: available_parameters[name] = values + # Takes parallel lists of templates, substitutes them with all combos of # parameters. The set of parameters is determined based on the first list. # Note: keep the order of queries -- sometimes we have DROP IF EXISTS diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index 782cf29863c..214f2d550b4 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -670,7 +670,6 @@ if args.report == "main": ) elif args.report == "all-queries": - print((header_template.format())) add_tested_commits() diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index d60a9e6afd1..64b64896f66 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -141,7 +141,6 @@ def prepare_tests_results_for_clickhouse( report_url: str, check_name: str, ) -> List[dict]: - pull_request_url = "https://github.com/ClickHouse/ClickHouse/commits/master" base_ref = "master" head_ref = "master" diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 192d216614e..f2b1105b3b0 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -96,7 +96,6 @@ def get_images_dict(repo_path: str, image_file_path: str) -> ImagesDict: def get_changed_docker_images( pr_info: PRInfo, images_dict: ImagesDict ) -> Set[DockerImage]: - if not images_dict: return set() diff --git a/tests/ci/get_previous_release_tag.py b/tests/ci/get_previous_release_tag.py index c6fe6cd5fb5..c2d279f7fec 100755 --- a/tests/ci/get_previous_release_tag.py +++ b/tests/ci/get_previous_release_tag.py @@ -51,7 +51,6 @@ def find_previous_release( for release in releases: if release.version < server_version: - # Check if the artifact exists on GitHub. # It can be not true for a short period of time # after creating a tag for a new release before uploading the packages. diff --git a/tests/ci/report.py b/tests/ci/report.py index 947fb33d905..ddee035d26f 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -473,7 +473,7 @@ def create_build_html_report( commit_url: str, ) -> str: rows = "" - for (build_result, build_log_url, artifact_urls) in zip( + for build_result, build_log_url, artifact_urls in zip( build_results, build_logs_urls, artifact_urls_list ): row = "" diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index dc5ada81995..a9a996e0a5f 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -63,6 +63,7 @@ DEFAULT_ENV_NAME = ".env" SANITIZER_SIGN = "==================" + # to create docker-compose env file def _create_env_file(path, variables): logging.debug(f"Env {variables} stored in {path}") @@ -1454,7 +1455,6 @@ class ClickHouseCluster: config_root_name="clickhouse", extra_configs=[], ) -> "ClickHouseInstance": - """Add an instance to the cluster. name - the name of the instance directory and the value of the 'instance' macro in ClickHouse. @@ -3089,7 +3089,6 @@ class ClickHouseInstance: config_root_name="clickhouse", extra_configs=[], ): - self.name = name self.base_cmd = cluster.base_cmd self.docker_id = cluster.get_instance_docker_id(self.name) diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index e408c9beec1..471aa2bdc2e 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -216,7 +216,6 @@ class _NetworkManager: container_exit_timeout=60, docker_api_version=os.environ.get("DOCKER_API_VERSION"), ): - self.container_expire_timeout = container_expire_timeout self.container_exit_timeout = container_exit_timeout @@ -232,7 +231,6 @@ class _NetworkManager: def _ensure_container(self): if self._container is None or self._container_expire_time <= time.time(): - for i in range(5): if self._container is not None: try: diff --git a/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py b/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py index d424ad58fa4..370aa23a014 100644 --- a/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py +++ b/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py @@ -1,6 +1,7 @@ import logging import os.path + # Makes the parallel workers of pytest-xdist to log to separate files. # Without this function all workers will log to the same log file # and mix everything together making it much more difficult for troubleshooting. diff --git a/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py b/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py index 02fccfae4e5..a6f7a8653da 100644 --- a/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py +++ b/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py @@ -24,7 +24,6 @@ def start_cluster(): def test_detach_part_wrong_partition_id(start_cluster): - # Here we create table with partition by UUID. node_21_6.query( "create table tab (id UUID, value UInt32) engine = MergeTree PARTITION BY (id) order by tuple()" diff --git a/tests/integration/test_cluster_copier/test_three_nodes.py b/tests/integration/test_cluster_copier/test_three_nodes.py index 31d6c0448f4..e7d07757adb 100644 --- a/tests/integration/test_cluster_copier/test_three_nodes.py +++ b/tests/integration/test_cluster_copier/test_three_nodes.py @@ -19,7 +19,6 @@ cluster = ClickHouseCluster(__file__) def started_cluster(): global cluster try: - for name in ["first", "second", "third"]: cluster.add_instance( name, diff --git a/tests/integration/test_cluster_copier/test_two_nodes.py b/tests/integration/test_cluster_copier/test_two_nodes.py index 10ab7d03b00..2b6fcf6cac2 100644 --- a/tests/integration/test_cluster_copier/test_two_nodes.py +++ b/tests/integration/test_cluster_copier/test_two_nodes.py @@ -19,7 +19,6 @@ cluster = ClickHouseCluster(__file__) def started_cluster(): global cluster try: - for name in ["first_of_two", "second_of_two"]: instance = cluster.add_instance( name, diff --git a/tests/integration/test_composable_protocols/test.py b/tests/integration/test_composable_protocols/test.py index bc87fea5296..df74cfffa54 100644 --- a/tests/integration/test_composable_protocols/test.py +++ b/tests/integration/test_composable_protocols/test.py @@ -63,7 +63,6 @@ def netcat(hostname, port, content): def test_connections(): - client = Client(server.ip_address, 9000, command=cluster.client_bin_path) assert client.query("SELECT 1") == "1\n" diff --git a/tests/integration/test_create_query_constraints/test.py b/tests/integration/test_create_query_constraints/test.py index 8df043fd24b..33c41b4f161 100644 --- a/tests/integration/test_create_query_constraints/test.py +++ b/tests/integration/test_create_query_constraints/test.py @@ -25,7 +25,6 @@ def start_cluster(): def test_create_query_const_constraints(): - instance.query("CREATE USER u_const SETTINGS max_threads = 1 CONST") instance.query("GRANT ALL ON *.* TO u_const") @@ -57,7 +56,6 @@ def test_create_query_const_constraints(): def test_create_query_minmax_constraints(): - instance.query("CREATE USER u_minmax SETTINGS max_threads = 4 MIN 2 MAX 6") instance.query("GRANT ALL ON *.* TO u_minmax") diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py index b38e81b0227..01addae2542 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py @@ -348,7 +348,6 @@ class RangedLayoutTester(BaseLayoutTester): self.layouts = LAYOUTS_RANGED def execute(self, layout_name, node): - if layout_name not in self.layout_to_dictionary: raise RuntimeError("Source doesn't support layout: {}".format(layout_name)) diff --git a/tests/integration/test_disks_app_func/test.py b/tests/integration/test_disks_app_func/test.py index 027ef8feed0..2428c53854e 100644 --- a/tests/integration/test_disks_app_func/test.py +++ b/tests/integration/test_disks_app_func/test.py @@ -7,7 +7,6 @@ import pytest def started_cluster(): global cluster try: - cluster = ClickHouseCluster(__file__) cluster.add_instance( "disks_app_test", main_configs=["config.xml"], with_minio=True diff --git a/tests/integration/test_distributed_ddl_parallel/test.py b/tests/integration/test_distributed_ddl_parallel/test.py index 6ebfe472e09..eb98dd3e230 100644 --- a/tests/integration/test_distributed_ddl_parallel/test.py +++ b/tests/integration/test_distributed_ddl_parallel/test.py @@ -10,6 +10,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) + # By default the exceptions that was throwed in threads will be ignored # (they will not mark the test as failed, only printed to stderr). # diff --git a/tests/integration/test_fetch_memory_usage/test.py b/tests/integration/test_fetch_memory_usage/test.py index a4371140150..7591cc0e8a9 100644 --- a/tests/integration/test_fetch_memory_usage/test.py +++ b/tests/integration/test_fetch_memory_usage/test.py @@ -18,7 +18,6 @@ def started_cluster(): def test_huge_column(started_cluster): - if ( node.is_built_with_thread_sanitizer() or node.is_built_with_memory_sanitizer() diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py index b8bafb3d0c1..fe69d72c1c7 100644 --- a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py +++ b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py @@ -13,7 +13,6 @@ number_of_iterations = 100 def perform_request(): - buffer = BytesIO() crl = pycurl.Curl() crl.setopt(pycurl.INTERFACE, client_ip) diff --git a/tests/integration/test_jbod_balancer/test.py b/tests/integration/test_jbod_balancer/test.py index e746698611a..df34a075d5a 100644 --- a/tests/integration/test_jbod_balancer/test.py +++ b/tests/integration/test_jbod_balancer/test.py @@ -45,7 +45,6 @@ def start_cluster(): def check_balance(node, table): - partitions = node.query( """ WITH diff --git a/tests/integration/test_keeper_and_access_storage/test.py b/tests/integration/test_keeper_and_access_storage/test.py index 6ec307f7082..0314825b6b7 100644 --- a/tests/integration/test_keeper_and_access_storage/test.py +++ b/tests/integration/test_keeper_and_access_storage/test.py @@ -10,6 +10,7 @@ node1 = cluster.add_instance( "node1", main_configs=["configs/keeper.xml"], stay_alive=True ) + # test that server is able to start @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_keeper_back_to_back/test.py b/tests/integration/test_keeper_back_to_back/test.py index 73fface02b4..b737ac284d2 100644 --- a/tests/integration/test_keeper_back_to_back/test.py +++ b/tests/integration/test_keeper_back_to_back/test.py @@ -546,7 +546,6 @@ def test_random_requests(started_cluster): def test_end_of_session(started_cluster): - fake_zk1 = None fake_zk2 = None genuine_zk1 = None @@ -685,6 +684,7 @@ def test_concurrent_watches(started_cluster): nonlocal watches_created nonlocal all_paths_created fake_zk.ensure_path(global_path + "/" + str(i)) + # new function each time def dumb_watch(event): nonlocal dumb_watch_triggered_counter diff --git a/tests/integration/test_keeper_persistent_log/test.py b/tests/integration/test_keeper_persistent_log/test.py index 70cc14fe26d..4164ffb33d3 100644 --- a/tests/integration/test_keeper_persistent_log/test.py +++ b/tests/integration/test_keeper_persistent_log/test.py @@ -163,7 +163,6 @@ def test_state_duplicate_restart(started_cluster): # http://zookeeper-user.578899.n2.nabble.com/Why-are-ephemeral-nodes-written-to-disk-tp7583403p7583418.html def test_ephemeral_after_restart(started_cluster): - try: node_zk = None node_zk2 = None diff --git a/tests/integration/test_keeper_zookeeper_converter/test.py b/tests/integration/test_keeper_zookeeper_converter/test.py index 063421bf922..de5a9416119 100644 --- a/tests/integration/test_keeper_zookeeper_converter/test.py +++ b/tests/integration/test_keeper_zookeeper_converter/test.py @@ -114,7 +114,6 @@ def start_clickhouse(): def copy_zookeeper_data(make_zk_snapshots): - if make_zk_snapshots: # force zookeeper to create snapshot generate_zk_snapshot() else: diff --git a/tests/integration/test_merge_tree_load_parts/test.py b/tests/integration/test_merge_tree_load_parts/test.py index 777b6f14fc6..dfbe00c8e28 100644 --- a/tests/integration/test_merge_tree_load_parts/test.py +++ b/tests/integration/test_merge_tree_load_parts/test.py @@ -148,17 +148,17 @@ def test_merge_tree_load_parts_corrupted(started_cluster): node1.query("SYSTEM WAIT LOADING PARTS mt_load_parts_2") def check_parts_loading(node, partition, loaded, failed, skipped): - for (min_block, max_block) in loaded: + for min_block, max_block in loaded: part_name = f"{partition}_{min_block}_{max_block}" assert node.contains_in_log(f"Loading Active part {part_name}") assert node.contains_in_log(f"Finished loading Active part {part_name}") - for (min_block, max_block) in failed: + for min_block, max_block in failed: part_name = f"{partition}_{min_block}_{max_block}" assert node.contains_in_log(f"Loading Active part {part_name}") assert not node.contains_in_log(f"Finished loading Active part {part_name}") - for (min_block, max_block) in skipped: + for min_block, max_block in skipped: part_name = f"{partition}_{min_block}_{max_block}" assert not node.contains_in_log(f"Loading Active part {part_name}") assert not node.contains_in_log(f"Finished loading Active part {part_name}") diff --git a/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py b/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py index b6567dfebc5..4613fdb850b 100644 --- a/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py +++ b/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py @@ -42,7 +42,6 @@ def delete(_bucket): @route("/<_bucket>/<_path:path>", ["GET", "POST", "PUT", "DELETE"]) def server(_bucket, _path): - # It's delete query for failed part if _path.endswith("delete"): response.set_header("Location", "http://minio1:9001/" + _bucket + "/" + _path) diff --git a/tests/integration/test_merge_tree_settings_constraints/test.py b/tests/integration/test_merge_tree_settings_constraints/test.py index 0bb0179108d..be6e2a31873 100644 --- a/tests/integration/test_merge_tree_settings_constraints/test.py +++ b/tests/integration/test_merge_tree_settings_constraints/test.py @@ -20,7 +20,6 @@ def start_cluster(): def test_merge_tree_settings_constraints(): - assert "Setting storage_policy should not be changed" in instance.query_and_get_error( f"CREATE TABLE wrong_table (number Int64) engine = MergeTree() ORDER BY number SETTINGS storage_policy = 'secret_policy'" ) diff --git a/tests/integration/test_old_parts_finally_removed/test.py b/tests/integration/test_old_parts_finally_removed/test.py index 108b72c5ccd..5347d433419 100644 --- a/tests/integration/test_old_parts_finally_removed/test.py +++ b/tests/integration/test_old_parts_finally_removed/test.py @@ -63,7 +63,6 @@ def test_part_finally_removed(started_cluster): ) for i in range(60): - if ( node1.query( "SELECT count() from system.parts WHERE table = 'drop_outdated_part'" diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index ae4393fc6f6..a34141c6189 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -528,7 +528,9 @@ def test_make_clone_in_detached(started_cluster): ["cp", "-r", path + "all_0_0_0", path + "detached/broken_all_0_0_0"] ) assert_eq_with_retry(instance, "select * from clone_in_detached", "\n") - assert ["broken_all_0_0_0",] == sorted( + assert [ + "broken_all_0_0_0", + ] == sorted( instance.exec_in_container(["ls", path + "detached/"]).strip().split("\n") ) diff --git a/tests/integration/test_password_constraints/test.py b/tests/integration/test_password_constraints/test.py index e3628861b28..9cdff51caa1 100644 --- a/tests/integration/test_password_constraints/test.py +++ b/tests/integration/test_password_constraints/test.py @@ -17,7 +17,6 @@ def start_cluster(): def test_complexity_rules(start_cluster): - error_message = "DB::Exception: Invalid password. The password should: be at least 12 characters long, contain at least 1 numeric character, contain at least 1 lowercase character, contain at least 1 uppercase character, contain at least 1 special character" assert error_message in node.query_and_get_error( "CREATE USER u_1 IDENTIFIED WITH plaintext_password BY ''" diff --git a/tests/integration/test_read_only_table/test.py b/tests/integration/test_read_only_table/test.py index 914c6a99508..df084f9dbbd 100644 --- a/tests/integration/test_read_only_table/test.py +++ b/tests/integration/test_read_only_table/test.py @@ -49,7 +49,6 @@ def start_cluster(): def test_restart_zookeeper(start_cluster): - for table_id in range(NUM_TABLES): node1.query( f"INSERT INTO test_table_{table_id} VALUES (1), (2), (3), (4), (5);" diff --git a/tests/integration/test_reload_auxiliary_zookeepers/test.py b/tests/integration/test_reload_auxiliary_zookeepers/test.py index bb1455333fc..476c5dee99e 100644 --- a/tests/integration/test_reload_auxiliary_zookeepers/test.py +++ b/tests/integration/test_reload_auxiliary_zookeepers/test.py @@ -20,7 +20,6 @@ def start_cluster(): def test_reload_auxiliary_zookeepers(start_cluster): - node.query( "CREATE TABLE simple (date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', 'node') ORDER BY tuple() PARTITION BY date;" ) diff --git a/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py b/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py index d6a732cc681..1d33ca02f86 100644 --- a/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py +++ b/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py @@ -1,6 +1,7 @@ #!/usr/bin/env python3 from bottle import request, route, run, response + # Handle for MultipleObjectsDelete. @route("/<_bucket>", ["POST"]) def delete(_bucket): diff --git a/tests/integration/test_s3_with_proxy/test.py b/tests/integration/test_s3_with_proxy/test.py index 1102d190a87..1af040c3c30 100644 --- a/tests/integration/test_s3_with_proxy/test.py +++ b/tests/integration/test_s3_with_proxy/test.py @@ -5,6 +5,7 @@ import time import pytest from helpers.cluster import ClickHouseCluster + # Runs simple proxy resolver in python env container. def run_resolver(cluster): container_id = cluster.get_container_id("resolver") diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index 7c62ca0d8b6..b3570b6e281 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -87,7 +87,6 @@ config = """ def execute_query_native(node, query, user, cert_name): - config_path = f"{SCRIPT_DIR}/configs/client.xml" formatted = config.format( diff --git a/tests/integration/test_storage_kafka/kafka_pb2.py b/tests/integration/test_storage_kafka/kafka_pb2.py index 7de1363bbf1..3e47af6c1e0 100644 --- a/tests/integration/test_storage_kafka/kafka_pb2.py +++ b/tests/integration/test_storage_kafka/kafka_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.kafka_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _KEYVALUEPAIR._serialized_start = 46 _KEYVALUEPAIR._serialized_end = 88 diff --git a/tests/integration/test_storage_kafka/message_with_repeated_pb2.py b/tests/integration/test_storage_kafka/message_with_repeated_pb2.py index 4d1a23c0b43..3715a9bea04 100644 --- a/tests/integration/test_storage_kafka/message_with_repeated_pb2.py +++ b/tests/integration/test_storage_kafka/message_with_repeated_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.message_with_repeated_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None DESCRIPTOR._serialized_options = b"H\001" _MESSAGE._serialized_start = 62 diff --git a/tests/integration/test_storage_kafka/social_pb2.py b/tests/integration/test_storage_kafka/social_pb2.py index 830ade81d33..f91a7bd0539 100644 --- a/tests/integration/test_storage_kafka/social_pb2.py +++ b/tests/integration/test_storage_kafka/social_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.social_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _USER._serialized_start = 47 _USER._serialized_end = 90 diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 51952ac1eb7..3a4fa6c6bfe 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -121,7 +121,7 @@ def kafka_create_topic( def kafka_delete_topic(admin_client, topic, max_retries=50): result = admin_client.delete_topics([topic]) - for (topic, e) in result.topic_error_codes: + for topic, e in result.topic_error_codes: if e == 0: logging.debug(f"Topic {topic} deleted") else: @@ -917,9 +917,7 @@ def describe_consumer_group(kafka_cluster, name): member_info["client_id"] = client_id member_info["client_host"] = client_host member_topics_assignment = [] - for (topic, partitions) in MemberAssignment.decode( - member_assignment - ).assignment: + for topic, partitions in MemberAssignment.decode(member_assignment).assignment: member_topics_assignment.append({"topic": topic, "partitions": partitions}) member_info["assignment"] = member_topics_assignment res.append(member_info) @@ -1537,7 +1535,6 @@ def test_kafka_protobuf_no_delimiter(kafka_cluster): def test_kafka_materialized_view(kafka_cluster): - instance.query( """ DROP TABLE IF EXISTS test.view; @@ -2315,7 +2312,6 @@ def test_kafka_virtual_columns2(kafka_cluster): def test_kafka_produce_key_timestamp(kafka_cluster): - admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) @@ -2444,7 +2440,6 @@ def test_kafka_insert_avro(kafka_cluster): def test_kafka_produce_consume_avro(kafka_cluster): - admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) @@ -4031,7 +4026,6 @@ def test_kafka_predefined_configuration(kafka_cluster): # https://github.com/ClickHouse/ClickHouse/issues/26643 def test_issue26643(kafka_cluster): - # for backporting: # admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") admin_client = KafkaAdminClient( @@ -4313,7 +4307,6 @@ def test_row_based_formats(kafka_cluster): "RowBinaryWithNamesAndTypes", "MsgPack", ]: - print(format_name) kafka_create_topic(admin_client, format_name) @@ -4438,7 +4431,6 @@ def test_block_based_formats_2(kafka_cluster): "ORC", "JSONCompactColumns", ]: - kafka_create_topic(admin_client, format_name) instance.query( diff --git a/tests/integration/test_storage_nats/nats_pb2.py b/tests/integration/test_storage_nats/nats_pb2.py index 4330ff57950..e9e5cb72363 100644 --- a/tests/integration/test_storage_nats/nats_pb2.py +++ b/tests/integration/test_storage_nats/nats_pb2.py @@ -31,7 +31,6 @@ ProtoKeyValue = _reflection.GeneratedProtocolMessageType( _sym_db.RegisterMessage(ProtoKeyValue) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _PROTOKEYVALUE._serialized_start = 45 _PROTOKEYVALUE._serialized_end = 88 diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 5df8b9029e6..8666d7ae58c 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -706,7 +706,6 @@ def test_abrupt_connection_loss_while_heavy_replication(started_cluster): def test_abrupt_server_restart_while_heavy_replication(started_cluster): - # FIXME (kssenii) temporary disabled if instance.is_built_with_sanitizer(): pytest.skip("Temporary disabled (FIXME)") diff --git a/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py b/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py index e017b4e66c2..a5845652eef 100644 --- a/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py +++ b/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.rabbitmq_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _KEYVALUEPROTO._serialized_start = 49 _KEYVALUEPROTO._serialized_end = 92 diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 2e54f21787a..53b6c4109ef 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -2864,7 +2864,6 @@ def test_rabbitmq_predefined_configuration(rabbitmq_cluster): def test_rabbitmq_msgpack(rabbitmq_cluster): - instance.query( """ drop table if exists rabbit_in; @@ -2908,7 +2907,6 @@ def test_rabbitmq_msgpack(rabbitmq_cluster): def test_rabbitmq_address(rabbitmq_cluster): - instance2.query( """ drop table if exists rabbit_in; @@ -3243,7 +3241,6 @@ def test_block_based_formats_2(rabbitmq_cluster): "ORC", "JSONCompactColumns", ]: - print(format_name) instance.query( diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 8b20727a7b5..4d493d9526b 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -18,6 +18,7 @@ MINIO_INTERNAL_PORT = 9001 SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + # Creates S3 bucket for tests and allows anonymous read-write access to it. def prepare_s3_bucket(started_cluster): # Allows read-write access for bucket without authorization. diff --git a/tests/integration/test_storage_s3/test_invalid_env_credentials.py b/tests/integration/test_storage_s3/test_invalid_env_credentials.py index 2f5d9349904..aa6479a2ed3 100644 --- a/tests/integration/test_storage_s3/test_invalid_env_credentials.py +++ b/tests/integration/test_storage_s3/test_invalid_env_credentials.py @@ -11,6 +11,7 @@ MINIO_INTERNAL_PORT = 9001 SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + # Creates S3 bucket for tests and allows anonymous read-write access to it. def prepare_s3_bucket(started_cluster): # Allows read-write access for bucket without authorization. diff --git a/tests/integration/test_system_merges/test.py b/tests/integration/test_system_merges/test.py index 0a469bd7bbd..ff303afe19e 100644 --- a/tests/integration/test_system_merges/test.py +++ b/tests/integration/test_system_merges/test.py @@ -171,7 +171,6 @@ def test_mutation_simple(started_cluster, replicated): starting_block = 0 if replicated else 1 try: - for node in nodes: node.query( f"create table {name} (a Int64) engine={engine} order by tuple()" diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 99978cbf6dc..89824293320 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1863,7 +1863,7 @@ def test_ttl_move_if_exists(started_cluster, name, dest_type): ) ) - for (node, policy) in zip( + for node, policy in zip( [node1, node2], ["only_jbod_1", "small_jbod_with_external"] ): node.query( diff --git a/tests/integration/test_zero_copy_fetch/test.py b/tests/integration/test_zero_copy_fetch/test.py index b71752528d3..9b9aa5e0da7 100644 --- a/tests/integration/test_zero_copy_fetch/test.py +++ b/tests/integration/test_zero_copy_fetch/test.py @@ -16,7 +16,6 @@ cluster = ClickHouseCluster(__file__) @pytest.fixture(scope="module") def started_cluster(): try: - cluster.add_instance( "node1", main_configs=["configs/storage_conf.xml"], diff --git a/utils/changelog-simple/format-changelog.py b/utils/changelog-simple/format-changelog.py index d5e1518270e..01f2694dd0f 100755 --- a/utils/changelog-simple/format-changelog.py +++ b/utils/changelog-simple/format-changelog.py @@ -20,6 +20,7 @@ parser.add_argument( ) args = parser.parse_args() + # This function mirrors the PR description checks in ClickhousePullRequestTrigger. # Returns False if the PR should not be mentioned changelog. def parse_one_pull_request(item): diff --git a/utils/keeper-overload/keeper-overload.py b/utils/keeper-overload/keeper-overload.py index bdb4563c713..0a059b10588 100755 --- a/utils/keeper-overload/keeper-overload.py +++ b/utils/keeper-overload/keeper-overload.py @@ -166,7 +166,7 @@ def main(args): keeper_bench_path = args.keeper_bench_path keepers = [] - for (port, server_id) in zip(PORTS, SERVER_IDS): + for port, server_id in zip(PORTS, SERVER_IDS): keepers.append( Keeper( keeper_binary_path, server_id, port, workdir, args.with_thread_fuzzer From 70912b15bca9585313db27d951d21905a3ff6950 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Sat, 25 Mar 2023 15:56:04 -0600 Subject: [PATCH 379/418] Update settings.md --- .../server-configuration-parameters/settings.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 08be318f334..51fdc902189 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1882,6 +1882,16 @@ The update is performed asynchronously, in a separate system thread. Manage executing [distributed ddl queries](../../sql-reference/distributed-ddl.md) (CREATE, DROP, ALTER, RENAME) on cluster. Works only if [ZooKeeper](#server-settings_zookeeper) is enabled. +The configurable settings within `` include: + +- **path**: the path in Keeper for the `task_queue` for DDL queries +- **profile**: the profile used to execute the DDL queries +- **pool_size**: how many `ON CLUSTER` queries can be run simultaneously +- **max_tasks_in_queue**: the maximum number of tasks that can be in the queue. Default is 1,000 +- **task_max_lifetime**: delete node if its age is greater than this value. Default is `7 * 24 * 60 * 60` (a week in seconds) +- **cleanup_delay_period**: cleaning starts after new node event is received if the last cleaning wasn't made sooner than `cleanup_delay_period` seconds ago. Default is 60 seconds + + **Example** ```xml From ca732c4013fe2ee1a766bf0b81380015f9dbc6e3 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Sat, 25 Mar 2023 21:21:06 -0600 Subject: [PATCH 380/418] Create youtube-dislikes.md --- .../example-datasets/youtube-dislikes.md | 116 ++++++++++++++++++ 1 file changed, 116 insertions(+) create mode 100644 docs/en/getting-started/example-datasets/youtube-dislikes.md diff --git a/docs/en/getting-started/example-datasets/youtube-dislikes.md b/docs/en/getting-started/example-datasets/youtube-dislikes.md new file mode 100644 index 00000000000..a78b4306685 --- /dev/null +++ b/docs/en/getting-started/example-datasets/youtube-dislikes.md @@ -0,0 +1,116 @@ +--- +slug: /en/getting-started/example-datasets/youtube-dislikes +sidebar_label: YouTube Dislikes +description: A collection is dislikes of YouTube videos. +--- + +# YouTube dataset of dislikes + +In November of 2021, YouTube removed the public ***dislike*** count from all of its videos. While creators can still see the number of dislikes, viewers can only see how many ***likes*** a video has received. + +:::important +The dataset has over 4 billion records, so be careful just copying-and-pasting the commands below unless your resources can handle that type of volume. The commands below were tested on a Development instance of [ClickHouse Cloud](https://clickhouse.cloud). +::: + +The data is in a JSON format and can be downloaded from [archive.org](https://archive.org/download/dislikes_youtube_2021_12_video_json_files). We have made this same data available in S3 so that it can be downloaded much more efficiently into a ClickHouse Cloud instance. + +Here are the steps to create a table in ClickHouse Cloud and insert the data. + +:::note +The steps below will easily work on a local install of ClickHouse too. The only change would be to use the `s3` function instead of `s3cluster` (unless you have a cluster configured - in which case change `default` to the name of your cluster). +::: + +1. Let's see what the data looks like. The `s3cluster` table function returns a table, so we can `DESCRIBE` the reult: + +```sql +DESCRIBE s3Cluster('default', 'https://clickhouse-public-datasets.s3.amazonaws.com/youtube/original/files/*.zst', 'JSONLines'); +``` + +ClickHouse infers the following schema from the JSON file: + +```response +┌─name────────────────┬─type─────────────────────────────────┐ +│ id │ Nullable(String) │ +│ fetch_date │ Nullable(Int64) │ +│ upload_date │ Nullable(String) │ +│ title │ Nullable(String) │ +│ uploader_id │ Nullable(String) │ +│ uploader │ Nullable(String) │ +│ uploader_sub_count │ Nullable(Int64) │ +│ is_age_limit │ Nullable(Bool) │ +│ view_count │ Nullable(Int64) │ +│ like_count │ Nullable(Int64) │ +│ dislike_count │ Nullable(Int64) │ +│ is_crawlable │ Nullable(Bool) │ +│ is_live_content │ Nullable(Bool) │ +│ has_subtitles │ Nullable(Bool) │ +│ is_ads_enabled │ Nullable(Bool) │ +│ is_comments_enabled │ Nullable(Bool) │ +│ description │ Nullable(String) │ +│ rich_metadata │ Array(Map(String, Nullable(String))) │ +│ super_titles │ Array(Map(String, Nullable(String))) │ +│ uploader_badges │ Nullable(String) │ +│ video_badges │ Nullable(String) │ +└─────────────────────┴──────────────────────────────────────┘ +``` + +2. Based on the inferred schema, we cleaned up the data types and added a primary key. Define the following table: + +```sql +CREATE TABLE youtube +( + `id` String, + `fetch_date` DateTime, + `upload_date` String, + `title` String, + `uploader_id` String, + `uploader` String, + `uploader_sub_count` Int64, + `is_age_limit` Bool, + `view_count` Int64, + `like_count` Int64, + `dislike_count` Int64, + `is_crawlable` Bool, + `has_subtitles` Bool, + `is_ads_enabled` Bool, + `is_comments_enabled` Bool, + `description` String, + `rich_metadata` Array(Map(String, String)), + `super_titles` Array(Map(String, String)), + `uploader_badges` String, + `video_badges` String +) +ENGINE = MergeTree +ORDER BY (upload_date, uploader); +``` + +3. The following command streams the records from the S3 files into the `youtube` table. **(NOTE: It's a lot of data!)** If you do not want the entire dataset, add a `LIMIT` clause: + +```sql +INSERT INTO youtube +SETTINGS input_format_null_as_default = 1 +SELECT + id, + parseDateTimeBestEffortUS(toString(fetch_date)) AS fetch_date, + upload_date, + ifNull(title, '') AS title, + uploader_id, + ifNull(uploader, '') AS uploader, + uploader_sub_count, + is_age_limit, + view_count, + like_count, + dislike_count, + is_crawlable, + has_subtitles, + is_ads_enabled, + is_comments_enabled, + ifNull(description, '') AS description, + rich_metadata, + super_titles, + ifNull(uploader_badges, '') AS uploader_badges, + ifNull(video_badges, '') AS video_badges +FROM s3Cluster('default','https://clickhouse-public-datasets.s3.amazonaws.com/youtube/original/files/*.zst', 'JSONLines'); +``` + +4. \ No newline at end of file From bc91bd518b5fec99049df9bd8b1e11a0fcb93a7e Mon Sep 17 00:00:00 2001 From: rfraposa Date: Sat, 25 Mar 2023 21:47:55 -0600 Subject: [PATCH 381/418] Update youtube-dislikes.md --- .../example-datasets/youtube-dislikes.md | 33 +++++++++++++++++-- 1 file changed, 31 insertions(+), 2 deletions(-) diff --git a/docs/en/getting-started/example-datasets/youtube-dislikes.md b/docs/en/getting-started/example-datasets/youtube-dislikes.md index a78b4306685..ee94fd3cacd 100644 --- a/docs/en/getting-started/example-datasets/youtube-dislikes.md +++ b/docs/en/getting-started/example-datasets/youtube-dislikes.md @@ -9,7 +9,7 @@ description: A collection is dislikes of YouTube videos. In November of 2021, YouTube removed the public ***dislike*** count from all of its videos. While creators can still see the number of dislikes, viewers can only see how many ***likes*** a video has received. :::important -The dataset has over 4 billion records, so be careful just copying-and-pasting the commands below unless your resources can handle that type of volume. The commands below were tested on a Development instance of [ClickHouse Cloud](https://clickhouse.cloud). +The dataset has over 4.5 billion records, so be careful just copying-and-pasting the commands below unless your resources can handle that type of volume. The commands below were ran on a **Development** instance of [ClickHouse Cloud](https://clickhouse.cloud). ::: The data is in a JSON format and can be downloaded from [archive.org](https://archive.org/download/dislikes_youtube_2021_12_video_json_files). We have made this same data available in S3 so that it can be downloaded much more efficiently into a ClickHouse Cloud instance. @@ -113,4 +113,33 @@ SELECT FROM s3Cluster('default','https://clickhouse-public-datasets.s3.amazonaws.com/youtube/original/files/*.zst', 'JSONLines'); ``` -4. \ No newline at end of file +4. Open a new tab in the SQL Console of ClickHouse Cloud (or a new `clickhouse-client` window) and watch the count increase: + +```sql +select formatReadableQuantity(count()) from youtube; +``` + +5. It will take a while to insert 4.56B rows, depending on your server resources. Once the data is inserted, go ahead and count the number of dislikes of your favorite videos or channels. Let's see how many videos were uploaded by ClickHouse: + +```sql +SELECT * +FROM youtube +WHERE uploader ILIKE '%ClickHouse%'; +``` + +6. Here is a search for videos with **ClickHouse** in the `title` or `description` fields: + +```sql +SELECT + view_count, + like_count, + dislike_count, + concat('https://youtu.be/', id) AS url, + title +FROM youtube +WHERE (title ILIKE '%ClickHouse%') OR (description ILIKE '%ClickHouse%') +ORDER BY + like_count DESC, + view_count DESC; +``` + From 5de24798b153ddd66c9735abf891a869a42ec584 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Sat, 25 Mar 2023 22:06:45 -0600 Subject: [PATCH 382/418] Update youtube-dislikes.md --- .../example-datasets/youtube-dislikes.md | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/docs/en/getting-started/example-datasets/youtube-dislikes.md b/docs/en/getting-started/example-datasets/youtube-dislikes.md index ee94fd3cacd..7a8d1681a4f 100644 --- a/docs/en/getting-started/example-datasets/youtube-dislikes.md +++ b/docs/en/getting-started/example-datasets/youtube-dislikes.md @@ -9,7 +9,7 @@ description: A collection is dislikes of YouTube videos. In November of 2021, YouTube removed the public ***dislike*** count from all of its videos. While creators can still see the number of dislikes, viewers can only see how many ***likes*** a video has received. :::important -The dataset has over 4.5 billion records, so be careful just copying-and-pasting the commands below unless your resources can handle that type of volume. The commands below were ran on a **Development** instance of [ClickHouse Cloud](https://clickhouse.cloud). +The dataset has over 4.5 billion records, so be careful just copying-and-pasting the commands below unless your resources can handle that type of volume. The commands below were executed on a **Development** instance of [ClickHouse Cloud](https://clickhouse.cloud). ::: The data is in a JSON format and can be downloaded from [archive.org](https://archive.org/download/dislikes_youtube_2021_12_video_json_files). We have made this same data available in S3 so that it can be downloaded much more efficiently into a ClickHouse Cloud instance. @@ -20,6 +20,8 @@ Here are the steps to create a table in ClickHouse Cloud and insert the data. The steps below will easily work on a local install of ClickHouse too. The only change would be to use the `s3` function instead of `s3cluster` (unless you have a cluster configured - in which case change `default` to the name of your cluster). ::: +## Step-by-step instructions + 1. Let's see what the data looks like. The `s3cluster` table function returns a table, so we can `DESCRIBE` the reult: ```sql @@ -84,7 +86,11 @@ ENGINE = MergeTree ORDER BY (upload_date, uploader); ``` -3. The following command streams the records from the S3 files into the `youtube` table. **(NOTE: It's a lot of data!)** If you do not want the entire dataset, add a `LIMIT` clause: +3. The following command streams the records from the S3 files into the `youtube` table. + +:::important +This inserts a lot of data - 4.65 billion rows. If you do not want the entire dataset, simply add a `LIMIT` clause with the desired number of rows. +::: ```sql INSERT INTO youtube From decdf0c507888e5c0d2648ef66259c4bf4e8157f Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 26 Mar 2023 10:35:44 +0200 Subject: [PATCH 383/418] Updated to delete added keys if there is an exception during update of metadata cache --- .../PartMetadataManagerWithCache.cpp | 52 ++++++++++++------- 1 file changed, 34 insertions(+), 18 deletions(-) diff --git a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp index 3a53cf25745..05b560a0dd1 100644 --- a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp +++ b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp @@ -117,30 +117,46 @@ void PartMetadataManagerWithCache::updateAll(bool include_projection) String value; String read_value; - for (const auto & file_name : file_names) + + /// This is used to remove the keys in case of any exception while caching other keys + Strings keys_added_to_cache(file_names.size()); + + try { - String file_path = fs::path(part->getDataPartStorage().getRelativePath()) / file_name; - if (!part->getDataPartStorage().exists(file_name)) - continue; - auto in = part->getDataPartStorage().readFile(file_name, {}, std::nullopt, std::nullopt); - readStringUntilEOF(value, *in); - - String key = getKeyFromFilePath(file_path); - auto status = cache->put(key, value); - if (!status.ok()) + for (const auto & file_name : file_names) { - status = cache->get(key, read_value); - if (status.IsNotFound() || read_value == value) + String file_path = fs::path(part->getDataPartStorage().getRelativePath()) / file_name; + if (!part->getDataPartStorage().exists(file_name)) continue; + auto in = part->getDataPartStorage().readFile(file_name, {}, std::nullopt, std::nullopt); + readStringUntilEOF(value, *in); - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "updateAll failed include_projection:{} status:{}, file_path:{}", - include_projection, - status.ToString(), - file_path); + String key = getKeyFromFilePath(file_path); + auto status = cache->put(key, value); + if (!status.ok()) + { + status = cache->get(key, read_value); + if (status.IsNotFound() || read_value == value) + continue; + + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "updateAll failed include_projection:{} status:{}, file_path:{}", + include_projection, + status.ToString(), + file_path); + } + keys_added_to_cache.emplace_back(key); } } + catch (...) + { + for (const auto & key : keys_added_to_cache) + { + cache->del(key); + } + throw; + } } void PartMetadataManagerWithCache::assertAllDeleted(bool include_projection) const From e665ad9f2665a29bd484a05adc55b7153f6021e6 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 26 Mar 2023 08:47:07 +0000 Subject: [PATCH 384/418] Automatic style fix --- docker/test/performance-comparison/perf.py | 2 ++ docker/test/performance-comparison/report.py | 1 - tests/ci/clickhouse_helper.py | 1 - tests/ci/docker_images_check.py | 1 - tests/ci/get_previous_release_tag.py | 1 - tests/ci/report.py | 2 +- tests/integration/helpers/cluster.py | 3 +-- tests/integration/helpers/network.py | 2 -- .../pytest_xdist_logging_to_separate_files.py | 1 + .../test_detach_part_wrong_partition_id.py | 1 - .../test_cluster_copier/test_three_nodes.py | 1 - .../test_cluster_copier/test_two_nodes.py | 1 - tests/integration/test_composable_protocols/test.py | 1 - .../test_create_query_constraints/test.py | 2 -- .../common.py | 1 - tests/integration/test_disks_app_func/test.py | 1 - .../test_distributed_ddl_parallel/test.py | 1 + tests/integration/test_fetch_memory_usage/test.py | 1 - .../scripts/stress_test.py | 1 - tests/integration/test_jbod_balancer/test.py | 1 - .../test_keeper_and_access_storage/test.py | 1 + tests/integration/test_keeper_back_to_back/test.py | 2 +- tests/integration/test_keeper_persistent_log/test.py | 1 - .../test_keeper_zookeeper_converter/test.py | 1 - tests/integration/test_merge_tree_load_parts/test.py | 6 +++--- .../s3_endpoint/endpoint.py | 1 - .../test_merge_tree_settings_constraints/test.py | 1 - .../test_old_parts_finally_removed/test.py | 1 - tests/integration/test_partition/test.py | 4 +++- tests/integration/test_password_constraints/test.py | 1 - tests/integration/test_read_only_table/test.py | 1 - .../test_reload_auxiliary_zookeepers/test.py | 1 - .../s3_endpoint/endpoint.py | 1 + tests/integration/test_s3_with_proxy/test.py | 1 + .../integration/test_ssl_cert_authentication/test.py | 1 - tests/integration/test_storage_kafka/kafka_pb2.py | 1 - .../test_storage_kafka/message_with_repeated_pb2.py | 1 - tests/integration/test_storage_kafka/social_pb2.py | 1 - tests/integration/test_storage_kafka/test.py | 12 ++---------- tests/integration/test_storage_nats/nats_pb2.py | 1 - .../test_storage_postgresql_replica/test.py | 1 - .../test_storage_rabbitmq/rabbitmq_pb2.py | 1 - tests/integration/test_storage_rabbitmq/test.py | 3 --- tests/integration/test_storage_s3/test.py | 1 + .../test_storage_s3/test_invalid_env_credentials.py | 1 + tests/integration/test_system_merges/test.py | 1 - tests/integration/test_ttl_move/test.py | 2 +- tests/integration/test_zero_copy_fetch/test.py | 1 - utils/changelog-simple/format-changelog.py | 1 + utils/keeper-overload/keeper-overload.py | 2 +- 50 files changed, 23 insertions(+), 57 deletions(-) diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 65bf49c2914..7a4e6386d0d 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -26,6 +26,7 @@ logging.basicConfig( total_start_seconds = time.perf_counter() stage_start_seconds = total_start_seconds + # Thread executor that does not hides exception that happens during function # execution, and rethrows it after join() class SafeThread(Thread): @@ -158,6 +159,7 @@ for e in subst_elems: available_parameters[name] = values + # Takes parallel lists of templates, substitutes them with all combos of # parameters. The set of parameters is determined based on the first list. # Note: keep the order of queries -- sometimes we have DROP IF EXISTS diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index 782cf29863c..214f2d550b4 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -670,7 +670,6 @@ if args.report == "main": ) elif args.report == "all-queries": - print((header_template.format())) add_tested_commits() diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index d60a9e6afd1..64b64896f66 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -141,7 +141,6 @@ def prepare_tests_results_for_clickhouse( report_url: str, check_name: str, ) -> List[dict]: - pull_request_url = "https://github.com/ClickHouse/ClickHouse/commits/master" base_ref = "master" head_ref = "master" diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 192d216614e..f2b1105b3b0 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -96,7 +96,6 @@ def get_images_dict(repo_path: str, image_file_path: str) -> ImagesDict: def get_changed_docker_images( pr_info: PRInfo, images_dict: ImagesDict ) -> Set[DockerImage]: - if not images_dict: return set() diff --git a/tests/ci/get_previous_release_tag.py b/tests/ci/get_previous_release_tag.py index c6fe6cd5fb5..c2d279f7fec 100755 --- a/tests/ci/get_previous_release_tag.py +++ b/tests/ci/get_previous_release_tag.py @@ -51,7 +51,6 @@ def find_previous_release( for release in releases: if release.version < server_version: - # Check if the artifact exists on GitHub. # It can be not true for a short period of time # after creating a tag for a new release before uploading the packages. diff --git a/tests/ci/report.py b/tests/ci/report.py index 947fb33d905..ddee035d26f 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -473,7 +473,7 @@ def create_build_html_report( commit_url: str, ) -> str: rows = "" - for (build_result, build_log_url, artifact_urls) in zip( + for build_result, build_log_url, artifact_urls in zip( build_results, build_logs_urls, artifact_urls_list ): row = "" diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index dc5ada81995..a9a996e0a5f 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -63,6 +63,7 @@ DEFAULT_ENV_NAME = ".env" SANITIZER_SIGN = "==================" + # to create docker-compose env file def _create_env_file(path, variables): logging.debug(f"Env {variables} stored in {path}") @@ -1454,7 +1455,6 @@ class ClickHouseCluster: config_root_name="clickhouse", extra_configs=[], ) -> "ClickHouseInstance": - """Add an instance to the cluster. name - the name of the instance directory and the value of the 'instance' macro in ClickHouse. @@ -3089,7 +3089,6 @@ class ClickHouseInstance: config_root_name="clickhouse", extra_configs=[], ): - self.name = name self.base_cmd = cluster.base_cmd self.docker_id = cluster.get_instance_docker_id(self.name) diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index e408c9beec1..471aa2bdc2e 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -216,7 +216,6 @@ class _NetworkManager: container_exit_timeout=60, docker_api_version=os.environ.get("DOCKER_API_VERSION"), ): - self.container_expire_timeout = container_expire_timeout self.container_exit_timeout = container_exit_timeout @@ -232,7 +231,6 @@ class _NetworkManager: def _ensure_container(self): if self._container is None or self._container_expire_time <= time.time(): - for i in range(5): if self._container is not None: try: diff --git a/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py b/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py index d424ad58fa4..370aa23a014 100644 --- a/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py +++ b/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py @@ -1,6 +1,7 @@ import logging import os.path + # Makes the parallel workers of pytest-xdist to log to separate files. # Without this function all workers will log to the same log file # and mix everything together making it much more difficult for troubleshooting. diff --git a/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py b/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py index 02fccfae4e5..a6f7a8653da 100644 --- a/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py +++ b/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py @@ -24,7 +24,6 @@ def start_cluster(): def test_detach_part_wrong_partition_id(start_cluster): - # Here we create table with partition by UUID. node_21_6.query( "create table tab (id UUID, value UInt32) engine = MergeTree PARTITION BY (id) order by tuple()" diff --git a/tests/integration/test_cluster_copier/test_three_nodes.py b/tests/integration/test_cluster_copier/test_three_nodes.py index 31d6c0448f4..e7d07757adb 100644 --- a/tests/integration/test_cluster_copier/test_three_nodes.py +++ b/tests/integration/test_cluster_copier/test_three_nodes.py @@ -19,7 +19,6 @@ cluster = ClickHouseCluster(__file__) def started_cluster(): global cluster try: - for name in ["first", "second", "third"]: cluster.add_instance( name, diff --git a/tests/integration/test_cluster_copier/test_two_nodes.py b/tests/integration/test_cluster_copier/test_two_nodes.py index 10ab7d03b00..2b6fcf6cac2 100644 --- a/tests/integration/test_cluster_copier/test_two_nodes.py +++ b/tests/integration/test_cluster_copier/test_two_nodes.py @@ -19,7 +19,6 @@ cluster = ClickHouseCluster(__file__) def started_cluster(): global cluster try: - for name in ["first_of_two", "second_of_two"]: instance = cluster.add_instance( name, diff --git a/tests/integration/test_composable_protocols/test.py b/tests/integration/test_composable_protocols/test.py index bc87fea5296..df74cfffa54 100644 --- a/tests/integration/test_composable_protocols/test.py +++ b/tests/integration/test_composable_protocols/test.py @@ -63,7 +63,6 @@ def netcat(hostname, port, content): def test_connections(): - client = Client(server.ip_address, 9000, command=cluster.client_bin_path) assert client.query("SELECT 1") == "1\n" diff --git a/tests/integration/test_create_query_constraints/test.py b/tests/integration/test_create_query_constraints/test.py index 8df043fd24b..33c41b4f161 100644 --- a/tests/integration/test_create_query_constraints/test.py +++ b/tests/integration/test_create_query_constraints/test.py @@ -25,7 +25,6 @@ def start_cluster(): def test_create_query_const_constraints(): - instance.query("CREATE USER u_const SETTINGS max_threads = 1 CONST") instance.query("GRANT ALL ON *.* TO u_const") @@ -57,7 +56,6 @@ def test_create_query_const_constraints(): def test_create_query_minmax_constraints(): - instance.query("CREATE USER u_minmax SETTINGS max_threads = 4 MIN 2 MAX 6") instance.query("GRANT ALL ON *.* TO u_minmax") diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py index b38e81b0227..01addae2542 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py @@ -348,7 +348,6 @@ class RangedLayoutTester(BaseLayoutTester): self.layouts = LAYOUTS_RANGED def execute(self, layout_name, node): - if layout_name not in self.layout_to_dictionary: raise RuntimeError("Source doesn't support layout: {}".format(layout_name)) diff --git a/tests/integration/test_disks_app_func/test.py b/tests/integration/test_disks_app_func/test.py index 027ef8feed0..2428c53854e 100644 --- a/tests/integration/test_disks_app_func/test.py +++ b/tests/integration/test_disks_app_func/test.py @@ -7,7 +7,6 @@ import pytest def started_cluster(): global cluster try: - cluster = ClickHouseCluster(__file__) cluster.add_instance( "disks_app_test", main_configs=["config.xml"], with_minio=True diff --git a/tests/integration/test_distributed_ddl_parallel/test.py b/tests/integration/test_distributed_ddl_parallel/test.py index 6ebfe472e09..eb98dd3e230 100644 --- a/tests/integration/test_distributed_ddl_parallel/test.py +++ b/tests/integration/test_distributed_ddl_parallel/test.py @@ -10,6 +10,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) + # By default the exceptions that was throwed in threads will be ignored # (they will not mark the test as failed, only printed to stderr). # diff --git a/tests/integration/test_fetch_memory_usage/test.py b/tests/integration/test_fetch_memory_usage/test.py index a4371140150..7591cc0e8a9 100644 --- a/tests/integration/test_fetch_memory_usage/test.py +++ b/tests/integration/test_fetch_memory_usage/test.py @@ -18,7 +18,6 @@ def started_cluster(): def test_huge_column(started_cluster): - if ( node.is_built_with_thread_sanitizer() or node.is_built_with_memory_sanitizer() diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py index b8bafb3d0c1..fe69d72c1c7 100644 --- a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py +++ b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py @@ -13,7 +13,6 @@ number_of_iterations = 100 def perform_request(): - buffer = BytesIO() crl = pycurl.Curl() crl.setopt(pycurl.INTERFACE, client_ip) diff --git a/tests/integration/test_jbod_balancer/test.py b/tests/integration/test_jbod_balancer/test.py index e746698611a..df34a075d5a 100644 --- a/tests/integration/test_jbod_balancer/test.py +++ b/tests/integration/test_jbod_balancer/test.py @@ -45,7 +45,6 @@ def start_cluster(): def check_balance(node, table): - partitions = node.query( """ WITH diff --git a/tests/integration/test_keeper_and_access_storage/test.py b/tests/integration/test_keeper_and_access_storage/test.py index 6ec307f7082..0314825b6b7 100644 --- a/tests/integration/test_keeper_and_access_storage/test.py +++ b/tests/integration/test_keeper_and_access_storage/test.py @@ -10,6 +10,7 @@ node1 = cluster.add_instance( "node1", main_configs=["configs/keeper.xml"], stay_alive=True ) + # test that server is able to start @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_keeper_back_to_back/test.py b/tests/integration/test_keeper_back_to_back/test.py index 73fface02b4..b737ac284d2 100644 --- a/tests/integration/test_keeper_back_to_back/test.py +++ b/tests/integration/test_keeper_back_to_back/test.py @@ -546,7 +546,6 @@ def test_random_requests(started_cluster): def test_end_of_session(started_cluster): - fake_zk1 = None fake_zk2 = None genuine_zk1 = None @@ -685,6 +684,7 @@ def test_concurrent_watches(started_cluster): nonlocal watches_created nonlocal all_paths_created fake_zk.ensure_path(global_path + "/" + str(i)) + # new function each time def dumb_watch(event): nonlocal dumb_watch_triggered_counter diff --git a/tests/integration/test_keeper_persistent_log/test.py b/tests/integration/test_keeper_persistent_log/test.py index 70cc14fe26d..4164ffb33d3 100644 --- a/tests/integration/test_keeper_persistent_log/test.py +++ b/tests/integration/test_keeper_persistent_log/test.py @@ -163,7 +163,6 @@ def test_state_duplicate_restart(started_cluster): # http://zookeeper-user.578899.n2.nabble.com/Why-are-ephemeral-nodes-written-to-disk-tp7583403p7583418.html def test_ephemeral_after_restart(started_cluster): - try: node_zk = None node_zk2 = None diff --git a/tests/integration/test_keeper_zookeeper_converter/test.py b/tests/integration/test_keeper_zookeeper_converter/test.py index 063421bf922..de5a9416119 100644 --- a/tests/integration/test_keeper_zookeeper_converter/test.py +++ b/tests/integration/test_keeper_zookeeper_converter/test.py @@ -114,7 +114,6 @@ def start_clickhouse(): def copy_zookeeper_data(make_zk_snapshots): - if make_zk_snapshots: # force zookeeper to create snapshot generate_zk_snapshot() else: diff --git a/tests/integration/test_merge_tree_load_parts/test.py b/tests/integration/test_merge_tree_load_parts/test.py index 777b6f14fc6..dfbe00c8e28 100644 --- a/tests/integration/test_merge_tree_load_parts/test.py +++ b/tests/integration/test_merge_tree_load_parts/test.py @@ -148,17 +148,17 @@ def test_merge_tree_load_parts_corrupted(started_cluster): node1.query("SYSTEM WAIT LOADING PARTS mt_load_parts_2") def check_parts_loading(node, partition, loaded, failed, skipped): - for (min_block, max_block) in loaded: + for min_block, max_block in loaded: part_name = f"{partition}_{min_block}_{max_block}" assert node.contains_in_log(f"Loading Active part {part_name}") assert node.contains_in_log(f"Finished loading Active part {part_name}") - for (min_block, max_block) in failed: + for min_block, max_block in failed: part_name = f"{partition}_{min_block}_{max_block}" assert node.contains_in_log(f"Loading Active part {part_name}") assert not node.contains_in_log(f"Finished loading Active part {part_name}") - for (min_block, max_block) in skipped: + for min_block, max_block in skipped: part_name = f"{partition}_{min_block}_{max_block}" assert not node.contains_in_log(f"Loading Active part {part_name}") assert not node.contains_in_log(f"Finished loading Active part {part_name}") diff --git a/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py b/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py index b6567dfebc5..4613fdb850b 100644 --- a/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py +++ b/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py @@ -42,7 +42,6 @@ def delete(_bucket): @route("/<_bucket>/<_path:path>", ["GET", "POST", "PUT", "DELETE"]) def server(_bucket, _path): - # It's delete query for failed part if _path.endswith("delete"): response.set_header("Location", "http://minio1:9001/" + _bucket + "/" + _path) diff --git a/tests/integration/test_merge_tree_settings_constraints/test.py b/tests/integration/test_merge_tree_settings_constraints/test.py index 0bb0179108d..be6e2a31873 100644 --- a/tests/integration/test_merge_tree_settings_constraints/test.py +++ b/tests/integration/test_merge_tree_settings_constraints/test.py @@ -20,7 +20,6 @@ def start_cluster(): def test_merge_tree_settings_constraints(): - assert "Setting storage_policy should not be changed" in instance.query_and_get_error( f"CREATE TABLE wrong_table (number Int64) engine = MergeTree() ORDER BY number SETTINGS storage_policy = 'secret_policy'" ) diff --git a/tests/integration/test_old_parts_finally_removed/test.py b/tests/integration/test_old_parts_finally_removed/test.py index 108b72c5ccd..5347d433419 100644 --- a/tests/integration/test_old_parts_finally_removed/test.py +++ b/tests/integration/test_old_parts_finally_removed/test.py @@ -63,7 +63,6 @@ def test_part_finally_removed(started_cluster): ) for i in range(60): - if ( node1.query( "SELECT count() from system.parts WHERE table = 'drop_outdated_part'" diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index ae4393fc6f6..a34141c6189 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -528,7 +528,9 @@ def test_make_clone_in_detached(started_cluster): ["cp", "-r", path + "all_0_0_0", path + "detached/broken_all_0_0_0"] ) assert_eq_with_retry(instance, "select * from clone_in_detached", "\n") - assert ["broken_all_0_0_0",] == sorted( + assert [ + "broken_all_0_0_0", + ] == sorted( instance.exec_in_container(["ls", path + "detached/"]).strip().split("\n") ) diff --git a/tests/integration/test_password_constraints/test.py b/tests/integration/test_password_constraints/test.py index e3628861b28..9cdff51caa1 100644 --- a/tests/integration/test_password_constraints/test.py +++ b/tests/integration/test_password_constraints/test.py @@ -17,7 +17,6 @@ def start_cluster(): def test_complexity_rules(start_cluster): - error_message = "DB::Exception: Invalid password. The password should: be at least 12 characters long, contain at least 1 numeric character, contain at least 1 lowercase character, contain at least 1 uppercase character, contain at least 1 special character" assert error_message in node.query_and_get_error( "CREATE USER u_1 IDENTIFIED WITH plaintext_password BY ''" diff --git a/tests/integration/test_read_only_table/test.py b/tests/integration/test_read_only_table/test.py index 914c6a99508..df084f9dbbd 100644 --- a/tests/integration/test_read_only_table/test.py +++ b/tests/integration/test_read_only_table/test.py @@ -49,7 +49,6 @@ def start_cluster(): def test_restart_zookeeper(start_cluster): - for table_id in range(NUM_TABLES): node1.query( f"INSERT INTO test_table_{table_id} VALUES (1), (2), (3), (4), (5);" diff --git a/tests/integration/test_reload_auxiliary_zookeepers/test.py b/tests/integration/test_reload_auxiliary_zookeepers/test.py index bb1455333fc..476c5dee99e 100644 --- a/tests/integration/test_reload_auxiliary_zookeepers/test.py +++ b/tests/integration/test_reload_auxiliary_zookeepers/test.py @@ -20,7 +20,6 @@ def start_cluster(): def test_reload_auxiliary_zookeepers(start_cluster): - node.query( "CREATE TABLE simple (date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', 'node') ORDER BY tuple() PARTITION BY date;" ) diff --git a/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py b/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py index d6a732cc681..1d33ca02f86 100644 --- a/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py +++ b/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py @@ -1,6 +1,7 @@ #!/usr/bin/env python3 from bottle import request, route, run, response + # Handle for MultipleObjectsDelete. @route("/<_bucket>", ["POST"]) def delete(_bucket): diff --git a/tests/integration/test_s3_with_proxy/test.py b/tests/integration/test_s3_with_proxy/test.py index 1102d190a87..1af040c3c30 100644 --- a/tests/integration/test_s3_with_proxy/test.py +++ b/tests/integration/test_s3_with_proxy/test.py @@ -5,6 +5,7 @@ import time import pytest from helpers.cluster import ClickHouseCluster + # Runs simple proxy resolver in python env container. def run_resolver(cluster): container_id = cluster.get_container_id("resolver") diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index 7c62ca0d8b6..b3570b6e281 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -87,7 +87,6 @@ config = """ def execute_query_native(node, query, user, cert_name): - config_path = f"{SCRIPT_DIR}/configs/client.xml" formatted = config.format( diff --git a/tests/integration/test_storage_kafka/kafka_pb2.py b/tests/integration/test_storage_kafka/kafka_pb2.py index 7de1363bbf1..3e47af6c1e0 100644 --- a/tests/integration/test_storage_kafka/kafka_pb2.py +++ b/tests/integration/test_storage_kafka/kafka_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.kafka_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _KEYVALUEPAIR._serialized_start = 46 _KEYVALUEPAIR._serialized_end = 88 diff --git a/tests/integration/test_storage_kafka/message_with_repeated_pb2.py b/tests/integration/test_storage_kafka/message_with_repeated_pb2.py index 4d1a23c0b43..3715a9bea04 100644 --- a/tests/integration/test_storage_kafka/message_with_repeated_pb2.py +++ b/tests/integration/test_storage_kafka/message_with_repeated_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.message_with_repeated_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None DESCRIPTOR._serialized_options = b"H\001" _MESSAGE._serialized_start = 62 diff --git a/tests/integration/test_storage_kafka/social_pb2.py b/tests/integration/test_storage_kafka/social_pb2.py index 830ade81d33..f91a7bd0539 100644 --- a/tests/integration/test_storage_kafka/social_pb2.py +++ b/tests/integration/test_storage_kafka/social_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.social_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _USER._serialized_start = 47 _USER._serialized_end = 90 diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 51952ac1eb7..3a4fa6c6bfe 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -121,7 +121,7 @@ def kafka_create_topic( def kafka_delete_topic(admin_client, topic, max_retries=50): result = admin_client.delete_topics([topic]) - for (topic, e) in result.topic_error_codes: + for topic, e in result.topic_error_codes: if e == 0: logging.debug(f"Topic {topic} deleted") else: @@ -917,9 +917,7 @@ def describe_consumer_group(kafka_cluster, name): member_info["client_id"] = client_id member_info["client_host"] = client_host member_topics_assignment = [] - for (topic, partitions) in MemberAssignment.decode( - member_assignment - ).assignment: + for topic, partitions in MemberAssignment.decode(member_assignment).assignment: member_topics_assignment.append({"topic": topic, "partitions": partitions}) member_info["assignment"] = member_topics_assignment res.append(member_info) @@ -1537,7 +1535,6 @@ def test_kafka_protobuf_no_delimiter(kafka_cluster): def test_kafka_materialized_view(kafka_cluster): - instance.query( """ DROP TABLE IF EXISTS test.view; @@ -2315,7 +2312,6 @@ def test_kafka_virtual_columns2(kafka_cluster): def test_kafka_produce_key_timestamp(kafka_cluster): - admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) @@ -2444,7 +2440,6 @@ def test_kafka_insert_avro(kafka_cluster): def test_kafka_produce_consume_avro(kafka_cluster): - admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) @@ -4031,7 +4026,6 @@ def test_kafka_predefined_configuration(kafka_cluster): # https://github.com/ClickHouse/ClickHouse/issues/26643 def test_issue26643(kafka_cluster): - # for backporting: # admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") admin_client = KafkaAdminClient( @@ -4313,7 +4307,6 @@ def test_row_based_formats(kafka_cluster): "RowBinaryWithNamesAndTypes", "MsgPack", ]: - print(format_name) kafka_create_topic(admin_client, format_name) @@ -4438,7 +4431,6 @@ def test_block_based_formats_2(kafka_cluster): "ORC", "JSONCompactColumns", ]: - kafka_create_topic(admin_client, format_name) instance.query( diff --git a/tests/integration/test_storage_nats/nats_pb2.py b/tests/integration/test_storage_nats/nats_pb2.py index 4330ff57950..e9e5cb72363 100644 --- a/tests/integration/test_storage_nats/nats_pb2.py +++ b/tests/integration/test_storage_nats/nats_pb2.py @@ -31,7 +31,6 @@ ProtoKeyValue = _reflection.GeneratedProtocolMessageType( _sym_db.RegisterMessage(ProtoKeyValue) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _PROTOKEYVALUE._serialized_start = 45 _PROTOKEYVALUE._serialized_end = 88 diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 5df8b9029e6..8666d7ae58c 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -706,7 +706,6 @@ def test_abrupt_connection_loss_while_heavy_replication(started_cluster): def test_abrupt_server_restart_while_heavy_replication(started_cluster): - # FIXME (kssenii) temporary disabled if instance.is_built_with_sanitizer(): pytest.skip("Temporary disabled (FIXME)") diff --git a/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py b/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py index e017b4e66c2..a5845652eef 100644 --- a/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py +++ b/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.rabbitmq_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _KEYVALUEPROTO._serialized_start = 49 _KEYVALUEPROTO._serialized_end = 92 diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 2e54f21787a..53b6c4109ef 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -2864,7 +2864,6 @@ def test_rabbitmq_predefined_configuration(rabbitmq_cluster): def test_rabbitmq_msgpack(rabbitmq_cluster): - instance.query( """ drop table if exists rabbit_in; @@ -2908,7 +2907,6 @@ def test_rabbitmq_msgpack(rabbitmq_cluster): def test_rabbitmq_address(rabbitmq_cluster): - instance2.query( """ drop table if exists rabbit_in; @@ -3243,7 +3241,6 @@ def test_block_based_formats_2(rabbitmq_cluster): "ORC", "JSONCompactColumns", ]: - print(format_name) instance.query( diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 8b20727a7b5..4d493d9526b 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -18,6 +18,7 @@ MINIO_INTERNAL_PORT = 9001 SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + # Creates S3 bucket for tests and allows anonymous read-write access to it. def prepare_s3_bucket(started_cluster): # Allows read-write access for bucket without authorization. diff --git a/tests/integration/test_storage_s3/test_invalid_env_credentials.py b/tests/integration/test_storage_s3/test_invalid_env_credentials.py index 2f5d9349904..aa6479a2ed3 100644 --- a/tests/integration/test_storage_s3/test_invalid_env_credentials.py +++ b/tests/integration/test_storage_s3/test_invalid_env_credentials.py @@ -11,6 +11,7 @@ MINIO_INTERNAL_PORT = 9001 SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + # Creates S3 bucket for tests and allows anonymous read-write access to it. def prepare_s3_bucket(started_cluster): # Allows read-write access for bucket without authorization. diff --git a/tests/integration/test_system_merges/test.py b/tests/integration/test_system_merges/test.py index 0a469bd7bbd..ff303afe19e 100644 --- a/tests/integration/test_system_merges/test.py +++ b/tests/integration/test_system_merges/test.py @@ -171,7 +171,6 @@ def test_mutation_simple(started_cluster, replicated): starting_block = 0 if replicated else 1 try: - for node in nodes: node.query( f"create table {name} (a Int64) engine={engine} order by tuple()" diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 99978cbf6dc..89824293320 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1863,7 +1863,7 @@ def test_ttl_move_if_exists(started_cluster, name, dest_type): ) ) - for (node, policy) in zip( + for node, policy in zip( [node1, node2], ["only_jbod_1", "small_jbod_with_external"] ): node.query( diff --git a/tests/integration/test_zero_copy_fetch/test.py b/tests/integration/test_zero_copy_fetch/test.py index b71752528d3..9b9aa5e0da7 100644 --- a/tests/integration/test_zero_copy_fetch/test.py +++ b/tests/integration/test_zero_copy_fetch/test.py @@ -16,7 +16,6 @@ cluster = ClickHouseCluster(__file__) @pytest.fixture(scope="module") def started_cluster(): try: - cluster.add_instance( "node1", main_configs=["configs/storage_conf.xml"], diff --git a/utils/changelog-simple/format-changelog.py b/utils/changelog-simple/format-changelog.py index d5e1518270e..01f2694dd0f 100755 --- a/utils/changelog-simple/format-changelog.py +++ b/utils/changelog-simple/format-changelog.py @@ -20,6 +20,7 @@ parser.add_argument( ) args = parser.parse_args() + # This function mirrors the PR description checks in ClickhousePullRequestTrigger. # Returns False if the PR should not be mentioned changelog. def parse_one_pull_request(item): diff --git a/utils/keeper-overload/keeper-overload.py b/utils/keeper-overload/keeper-overload.py index bdb4563c713..0a059b10588 100755 --- a/utils/keeper-overload/keeper-overload.py +++ b/utils/keeper-overload/keeper-overload.py @@ -166,7 +166,7 @@ def main(args): keeper_bench_path = args.keeper_bench_path keepers = [] - for (port, server_id) in zip(PORTS, SERVER_IDS): + for port, server_id in zip(PORTS, SERVER_IDS): keepers.append( Keeper( keeper_binary_path, server_id, port, workdir, args.with_thread_fuzzer From 680ab4fae3c73d5e9146c62d97210dcbed2ae51b Mon Sep 17 00:00:00 2001 From: exmy Date: Sun, 26 Mar 2023 17:14:01 +0800 Subject: [PATCH 385/418] Fix incorrect code indentation --- src/Functions/randConstant.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/randConstant.cpp b/src/Functions/randConstant.cpp index dd629538e85..22ce6d88ea6 100644 --- a/src/Functions/randConstant.cpp +++ b/src/Functions/randConstant.cpp @@ -20,7 +20,7 @@ public: String getName() const override { return Name::name; } -bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForNulls() const override { return false; } ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override { From b143a1a3ebabb9333192fcbbf98bbcff791d1819 Mon Sep 17 00:00:00 2001 From: Sergei Solomatov Date: Sun, 26 Mar 2023 11:20:53 +0000 Subject: [PATCH 386/418] fix query --- src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 4d8de325902..2c97c92ba99 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -302,11 +302,12 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( "and a.attnum = ANY(ix.indkey) " "and t.relkind in ('r', 'p') " /// simple tables "and t.relname = {} " /// Connection is already done to a needed database, only table name is needed. - "{}" + "and t.relnamespace = (select oid from pg_namespace where nspname = {}) " "and ix.indisreplident = 't' " /// index is is replica identity index "ORDER BY a.attname", /// column name - (postgres_schema.empty() ? "" : "and t.relnamespace = " + quoteString(postgres_schema)) + " ", - quoteString(postgres_table)); + quoteString(postgres_table), + (postgres_schema.empty() ? quoteString("public") : quoteString(postgres_schema)) + ); table.replica_identity_columns = readNamesAndTypesList(tx, postgres_table_with_schema, query, use_nulls, true); } From eabcefe452ebc44c9a1081349dad2722415923cc Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 26 Mar 2023 14:52:08 +0200 Subject: [PATCH 387/418] Fixed issue with reserving vector size --- src/Storages/MergeTree/PartMetadataManagerWithCache.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp index 05b560a0dd1..7c0aedf699b 100644 --- a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp +++ b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp @@ -119,7 +119,8 @@ void PartMetadataManagerWithCache::updateAll(bool include_projection) String read_value; /// This is used to remove the keys in case of any exception while caching other keys - Strings keys_added_to_cache(file_names.size()); + Strings keys_added_to_cache; + keys_added_to_cache.reserve(file_names.size()); try { From 355818e464f9c6d8f9d0c9a78663a59b53836d4f Mon Sep 17 00:00:00 2001 From: Joey Date: Sun, 26 Mar 2023 22:08:19 +0800 Subject: [PATCH 388/418] fix img path --- docs/zh/faq/general/columnar-database.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/zh/faq/general/columnar-database.md b/docs/zh/faq/general/columnar-database.md index 57541aec69b..afff4feaf8f 100644 --- a/docs/zh/faq/general/columnar-database.md +++ b/docs/zh/faq/general/columnar-database.md @@ -7,20 +7,20 @@ sidebar_position: 101 # 什么是列存储数据库? {#what-is-a-columnar-database} -列存储数据库独立存储每个列的数据。这只允许从磁盘读取任何给定查询中使用的列的数据。其代价是,影响整行的操作会按比例变得更昂贵。列存储数据库的同义词是面向列的数据库管理系统。ClickHouse就是这样一个典型的例子。 +列存储数据库独立存储每个列的数据。这只允许从磁盘读取任何给定查询中使用的列的数据。其代价是,影响整行的操作会按比例变得更昂贵。列存储数据库的同义词是面向列的数据库管理系统。ClickHouse 就是这样一个典型的例子。 列存储数据库的主要优点是: - 查询只使用许多列其中的少数列。 -— 聚合对大量数据的查询。 -— 按列压缩。 + — 聚合对大量数据的查询。 + — 按列压缩。 下面是构建报表时传统的面向行系统和柱状数据库之间的区别: **传统行存储** -!(传统行存储)(https://clickhouse.com/docs/en/images/row-oriented.gif) +!(传统行存储)(https://clickhouse.com/docs/assets/images/row-oriented-3e6fd5aa48e3075202d242b4799da8fa.gif) **列存储** -!(列存储)(https://clickhouse.com/docs/en/images/column-oriented.gif) +!(列存储)(https://clickhouse.com/docs/assets/images/column-oriented-d082e49b7743d4ded32c7952bfdb028f.gif) -列存储数据库是分析应用程序的首选,因为它允许在一个表中有许多列以防万一,但不会在读取查询执行时为未使用的列付出代价。面向列的数据库是为大数据处理而设计的,因为和数据仓库一样,它们通常使用分布式的低成本硬件集群来提高吞吐量。ClickHouse结合了[分布式](../../engines/table-engines/special/distributed.md)和[复制式](../../engines/table-engines/mergetree-family/replication.md)两类表。 \ No newline at end of file +列存储数据库是分析应用程序的首选,因为它允许在一个表中有许多列以防万一,但不会在读取查询执行时为未使用的列付出代价。面向列的数据库是为大数据处理而设计的,因为和数据仓库一样,它们通常使用分布式的低成本硬件集群来提高吞吐量。ClickHouse 结合了[分布式](../../engines/table-engines/special/distributed.md)和[复制式](../../engines/table-engines/mergetree-family/replication.md)两类表。 From 2af34bd9db5282f57059dda2873b5ffc2b9bb81b Mon Sep 17 00:00:00 2001 From: Joey Date: Sun, 26 Mar 2023 22:10:52 +0800 Subject: [PATCH 389/418] fix img path --- docs/zh/faq/general/columnar-database.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/zh/faq/general/columnar-database.md b/docs/zh/faq/general/columnar-database.md index afff4feaf8f..27731421a09 100644 --- a/docs/zh/faq/general/columnar-database.md +++ b/docs/zh/faq/general/columnar-database.md @@ -12,8 +12,8 @@ sidebar_position: 101 列存储数据库的主要优点是: - 查询只使用许多列其中的少数列。 - — 聚合对大量数据的查询。 - — 按列压缩。 +- 聚合对大量数据的查询。 +- 按列压缩。 下面是构建报表时传统的面向行系统和柱状数据库之间的区别: From d78b9ee006c66eafa2dc81bbdcfc36a2b8224362 Mon Sep 17 00:00:00 2001 From: Joey Date: Sun, 26 Mar 2023 22:15:23 +0800 Subject: [PATCH 390/418] fix md --- docs/zh/faq/general/columnar-database.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/zh/faq/general/columnar-database.md b/docs/zh/faq/general/columnar-database.md index 27731421a09..004da067900 100644 --- a/docs/zh/faq/general/columnar-database.md +++ b/docs/zh/faq/general/columnar-database.md @@ -18,9 +18,9 @@ sidebar_position: 101 下面是构建报表时传统的面向行系统和柱状数据库之间的区别: **传统行存储** -!(传统行存储)(https://clickhouse.com/docs/assets/images/row-oriented-3e6fd5aa48e3075202d242b4799da8fa.gif) +![传统行存储](https://clickhouse.com/docs/assets/images/row-oriented-3e6fd5aa48e3075202d242b4799da8fa.gif) **列存储** -!(列存储)(https://clickhouse.com/docs/assets/images/column-oriented-d082e49b7743d4ded32c7952bfdb028f.gif) +![列存储](https://clickhouse.com/docs/assets/images/column-oriented-d082e49b7743d4ded32c7952bfdb028f.gif) 列存储数据库是分析应用程序的首选,因为它允许在一个表中有许多列以防万一,但不会在读取查询执行时为未使用的列付出代价。面向列的数据库是为大数据处理而设计的,因为和数据仓库一样,它们通常使用分布式的低成本硬件集群来提高吞吐量。ClickHouse 结合了[分布式](../../engines/table-engines/special/distributed.md)和[复制式](../../engines/table-engines/mergetree-family/replication.md)两类表。 From 5d5308e060bcfe1d7a07cbebd2cc3eae833afe2d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 26 Mar 2023 17:59:39 +0000 Subject: [PATCH 391/418] CMake cleanup: Remove configuration of CMAKE_SHARED_LINKER_FLAGS Follow-up to #44828 --- CMakeLists.txt | 1 - PreLoad.cmake | 5 ++--- cmake/linux/toolchain-riscv64.cmake | 1 - cmake/linux/toolchain-x86_64.cmake | 1 - cmake/tools.cmake | 2 -- 5 files changed, 2 insertions(+), 8 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 49367705a08..5550a19b699 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -180,7 +180,6 @@ if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE") # Can be lld or ld-lld or lld-13 or /path/to/lld. if (LINKER_NAME MATCHES "lld" AND OS_LINUX) set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--gdb-index") - set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--gdb-index") message (STATUS "Adding .gdb-index via --gdb-index linker option.") endif () endif() diff --git a/PreLoad.cmake b/PreLoad.cmake index 0e1ee70fc8f..b456c724cc6 100644 --- a/PreLoad.cmake +++ b/PreLoad.cmake @@ -19,8 +19,8 @@ endif() if (NOT "$ENV{CFLAGS}" STREQUAL "" OR NOT "$ENV{CXXFLAGS}" STREQUAL "" OR NOT "$ENV{LDFLAGS}" STREQUAL "" - OR CMAKE_C_FLAGS OR CMAKE_CXX_FLAGS OR CMAKE_EXE_LINKER_FLAGS OR CMAKE_SHARED_LINKER_FLAGS OR CMAKE_MODULE_LINKER_FLAGS - OR CMAKE_C_FLAGS_INIT OR CMAKE_CXX_FLAGS_INIT OR CMAKE_EXE_LINKER_FLAGS_INIT OR CMAKE_SHARED_LINKER_FLAGS_INIT OR CMAKE_MODULE_LINKER_FLAGS_INIT) + OR CMAKE_C_FLAGS OR CMAKE_CXX_FLAGS OR CMAKE_EXE_LINKER_FLAGS OR CMAKE_MODULE_LINKER_FLAGS + OR CMAKE_C_FLAGS_INIT OR CMAKE_CXX_FLAGS_INIT OR CMAKE_EXE_LINKER_FLAGS_INIT OR CMAKE_MODULE_LINKER_FLAGS_INIT) # if $ENV message("CFLAGS: $ENV{CFLAGS}") @@ -36,7 +36,6 @@ if (NOT "$ENV{CFLAGS}" STREQUAL "" message("CMAKE_C_FLAGS_INIT: ${CMAKE_C_FLAGS_INIT}") message("CMAKE_CXX_FLAGS_INIT: ${CMAKE_CXX_FLAGS_INIT}") message("CMAKE_EXE_LINKER_FLAGS_INIT: ${CMAKE_EXE_LINKER_FLAGS_INIT}") - message("CMAKE_SHARED_LINKER_FLAGS_INIT: ${CMAKE_SHARED_LINKER_FLAGS_INIT}") message("CMAKE_MODULE_LINKER_FLAGS_INIT: ${CMAKE_MODULE_LINKER_FLAGS_INIT}") message(FATAL_ERROR " diff --git a/cmake/linux/toolchain-riscv64.cmake b/cmake/linux/toolchain-riscv64.cmake index 49a036c2972..ea57c3b2c42 100644 --- a/cmake/linux/toolchain-riscv64.cmake +++ b/cmake/linux/toolchain-riscv64.cmake @@ -22,7 +22,6 @@ set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_EXE_LINKER_FLAGS_INIT "-fuse-ld=bfd") -set (CMAKE_SHARED_LINKER_FLAGS_INIT "-fuse-ld=bfd") # Currently, lld does not work with the error: # ld.lld: error: section size decrease is too large diff --git a/cmake/linux/toolchain-x86_64.cmake b/cmake/linux/toolchain-x86_64.cmake index e73d779284a..55b9df79f70 100644 --- a/cmake/linux/toolchain-x86_64.cmake +++ b/cmake/linux/toolchain-x86_64.cmake @@ -30,7 +30,6 @@ set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/x86_64-linux-gnu/libc") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") -set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") diff --git a/cmake/tools.cmake b/cmake/tools.cmake index f17ab4da5cd..974b0bd1d3d 100644 --- a/cmake/tools.cmake +++ b/cmake/tools.cmake @@ -95,10 +95,8 @@ if (LINKER_NAME) configure_file ("${CMAKE_CURRENT_SOURCE_DIR}/cmake/ld.lld.in" "${LLD_WRAPPER}" @ONLY) set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} --ld-path=${LLD_WRAPPER}") - set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} --ld-path=${LLD_WRAPPER}") else () set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=${LINKER_NAME}") - set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=${LINKER_NAME}") endif () endif () From f9d473a94d025a831d51ae579bbd31d0a4678187 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 26 Mar 2023 20:00:04 +0000 Subject: [PATCH 392/418] parseDateTime(): Fix UB (signed integer overflow) --- src/Functions/parseDateTime.cpp | 81 +++++++++++-------- .../02668_parse_datetime_in_joda_syntax.sql | 3 + 2 files changed, 49 insertions(+), 35 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 7799520b8e5..abee7e0d8f8 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -1032,11 +1032,12 @@ namespace bool allow_negative, bool allow_plus_sign, bool is_year, - int repetitions, - int max_digits_to_read, + size_t repetitions, + size_t max_digits_to_read, const String & fragment, Int32 & result) { + bool negative = false; if (allow_negative && cur < end && *cur == '-') { @@ -1051,6 +1052,15 @@ namespace Int64 number = 0; const Pos start = cur; + + /// Avoid integer overflow in (*) + if (max_digits_to_read >= std::numeric_limits::digits10) [[unlikely]] + throw Exception( + ErrorCodes::CANNOT_PARSE_DATETIME, + "Unable to parse fragment {} from {} because max_digits_to_read is too big", + fragment, + std::string_view(start, cur - start)); + if (is_year && repetitions == 2) { // If abbreviated two year digit is provided in format string, try to read @@ -1059,10 +1069,10 @@ namespace // [70, 99] -> [1970, 1999] // If more than two digits are provided, then simply read in full year // normally without conversion - int count = 0; + size_t count = 0; while (cur < end && cur < start + max_digits_to_read && *cur >= '0' && *cur <= '9') { - number = number * 10 + (*cur - '0'); + number = number * 10 + (*cur - '0'); /// (*) ++cur; ++count; } @@ -1077,7 +1087,7 @@ namespace { while (cur < end && cur < start + max_digits_to_read && *cur >= '0' && *cur <= '9') { - number = number * 10 + (*cur - '0'); + number = number * 10 + (*cur - '0'); /// (*) ++cur; } } @@ -1091,24 +1101,25 @@ namespace } } + if (negative) + number *= -1; + /// Need to have read at least one digit. - if (cur == start) + if (cur == start) [[unlikely]] throw Exception( ErrorCodes::CANNOT_PARSE_DATETIME, "Unable to parse fragment {} from {} because read number failed", fragment, std::string_view(cur, end - cur)); - if (negative) - number *= -1; - /// Check if number exceeds the range of Int32 - if (number < std::numeric_limits::lowest() || number > std::numeric_limits::max()) + if (number < std::numeric_limits::min() || number > std::numeric_limits::max()) [[unlikely]] throw Exception( ErrorCodes::CANNOT_PARSE_DATETIME, "Unable to parse fragment {} from {} because number is out of range of Int32", fragment, std::string_view(start, cur - start)); + result = static_cast(number); return cur; @@ -1125,7 +1136,7 @@ namespace return cur; } - static Pos jodaCenturyOfEra(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) + static Pos jodaCenturyOfEra(size_t repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 century; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, repetitions, fragment, century); @@ -1133,7 +1144,7 @@ namespace return cur; } - static Pos jodaYearOfEra(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) + static Pos jodaYearOfEra(size_t repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 year_of_era; cur = readNumberWithVariableLength(cur, end, false, false, true, repetitions, repetitions, fragment, year_of_era); @@ -1141,7 +1152,7 @@ namespace return cur; } - static Pos jodaWeekYear(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) + static Pos jodaWeekYear(size_t repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 week_year; cur = readNumberWithVariableLength(cur, end, true, true, true, repetitions, repetitions, fragment, week_year); @@ -1149,15 +1160,15 @@ namespace return cur; } - static Pos jodaWeekOfWeekYear(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) + static Pos jodaWeekOfWeekYear(size_t repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 week; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), fragment, week); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2uz), fragment, week); date.setWeek(week); return cur; } - static Pos jodaDayOfWeek1Based(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) + static Pos jodaDayOfWeek1Based(size_t repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 day_of_week; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, repetitions, fragment, day_of_week); @@ -1197,7 +1208,7 @@ namespace return cur; } - static Pos jodaYear(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) + static Pos jodaYear(size_t repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 year; cur = readNumberWithVariableLength(cur, end, true, true, true, repetitions, repetitions, fragment, year); @@ -1205,15 +1216,15 @@ namespace return cur; } - static Pos jodaDayOfYear(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) + static Pos jodaDayOfYear(size_t repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 day_of_year; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 3), fragment, day_of_year); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 3uz), fragment, day_of_year); date.setDayOfYear(day_of_year); return cur; } - static Pos jodaMonthOfYear(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) + static Pos jodaMonthOfYear(size_t repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 month; cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, 2, fragment, month); @@ -1251,11 +1262,11 @@ namespace return cur; } - static Pos jodaDayOfMonth(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) + static Pos jodaDayOfMonth(size_t repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 day_of_month; cur = readNumberWithVariableLength( - cur, end, false, false, false, repetitions, std::max(repetitions, 2), fragment, day_of_month); + cur, end, false, false, false, repetitions, std::max(repetitions, 2uz), fragment, day_of_month); date.setDayOfMonth(day_of_month); return cur; } @@ -1271,50 +1282,50 @@ namespace return cur; } - static Pos jodaHourOfHalfDay(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) + static Pos jodaHourOfHalfDay(size_t repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 hour; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), fragment, hour); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2uz), fragment, hour); date.setHour(hour, true, false); return cur; } - static Pos jodaClockHourOfHalfDay(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) + static Pos jodaClockHourOfHalfDay(size_t repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 hour; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), fragment, hour); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2uz), fragment, hour); date.setHour(hour, true, true); return cur; } - static Pos jodaHourOfDay(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) + static Pos jodaHourOfDay(size_t repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 hour; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), fragment, hour); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2uz), fragment, hour); date.setHour(hour, false, false); return cur; } - static Pos jodaClockHourOfDay(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) + static Pos jodaClockHourOfDay(size_t repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 hour; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), fragment, hour); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2uz), fragment, hour); date.setHour(hour, false, true); return cur; } - static Pos jodaMinuteOfHour(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) + static Pos jodaMinuteOfHour(size_t repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 minute; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), fragment, minute); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2uz), fragment, minute); date.setMinute(minute); return cur; } - static Pos jodaSecondOfMinute(int repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) + static Pos jodaSecondOfMinute(size_t repetitions, Pos cur, Pos end, const String & fragment, DateTime & date) { Int32 second; - cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2), fragment, second); + cur = readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2uz), fragment, second); date.setSecond(second); return cur; } @@ -1612,7 +1623,7 @@ namespace } else { - int repetitions = 1; + size_t repetitions = 1; ++pos; while (pos < end && *cur_token == *pos) { 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 27b4a8bf83d..7ce5c1a4fdd 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 @@ -229,4 +229,7 @@ select parseDateTimeInJodaSyntax('60', 's', 'UTC'); -- { serverError CANNOT_PARS select parseDateTimeInJodaSyntax('-1', 's', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTimeInJodaSyntax('123456789', 's', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +-- integer overflow in AST Fuzzer +select parseDateTimeInJodaSyntax('19191919191919191919191919191919', 'CCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCC', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } + -- { echoOff } From f0b3007deea110c1f7adae646920cad455f34794 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Sun, 26 Mar 2023 16:11:05 -0600 Subject: [PATCH 393/418] Update youtube-dislikes.md --- .../example-datasets/youtube-dislikes.md | 69 +++++++++++++++---- 1 file changed, 55 insertions(+), 14 deletions(-) diff --git a/docs/en/getting-started/example-datasets/youtube-dislikes.md b/docs/en/getting-started/example-datasets/youtube-dislikes.md index 7a8d1681a4f..5353aee42db 100644 --- a/docs/en/getting-started/example-datasets/youtube-dislikes.md +++ b/docs/en/getting-started/example-datasets/youtube-dislikes.md @@ -9,10 +9,10 @@ description: A collection is dislikes of YouTube videos. In November of 2021, YouTube removed the public ***dislike*** count from all of its videos. While creators can still see the number of dislikes, viewers can only see how many ***likes*** a video has received. :::important -The dataset has over 4.5 billion records, so be careful just copying-and-pasting the commands below unless your resources can handle that type of volume. The commands below were executed on a **Development** instance of [ClickHouse Cloud](https://clickhouse.cloud). +The dataset has over 4.55 billion records, so be careful just copying-and-pasting the commands below unless your resources can handle that type of volume. The commands below were executed on a **Production** instance of [ClickHouse Cloud](https://clickhouse.cloud). ::: -The data is in a JSON format and can be downloaded from [archive.org](https://archive.org/download/dislikes_youtube_2021_12_video_json_files). We have made this same data available in S3 so that it can be downloaded much more efficiently into a ClickHouse Cloud instance. +The data is in a JSON format and can be downloaded from [archive.org](https://archive.org/download/dislikes_youtube_2021_12_video_json_files). We have made this same data available in S3 so that it can be downloaded more efficiently into a ClickHouse Cloud instance. Here are the steps to create a table in ClickHouse Cloud and insert the data. @@ -25,7 +25,11 @@ The steps below will easily work on a local install of ClickHouse too. The only 1. Let's see what the data looks like. The `s3cluster` table function returns a table, so we can `DESCRIBE` the reult: ```sql -DESCRIBE s3Cluster('default', 'https://clickhouse-public-datasets.s3.amazonaws.com/youtube/original/files/*.zst', 'JSONLines'); +DESCRIBE s3Cluster( + 'default', + 'https://clickhouse-public-datasets.s3.amazonaws.com/youtube/original/files/*.zst', + 'JSONLines' +); ``` ClickHouse infers the following schema from the JSON file: @@ -116,23 +120,46 @@ SELECT super_titles, ifNull(uploader_badges, '') AS uploader_badges, ifNull(video_badges, '') AS video_badges -FROM s3Cluster('default','https://clickhouse-public-datasets.s3.amazonaws.com/youtube/original/files/*.zst', 'JSONLines'); +FROM s3Cluster( + 'default', + 'https://clickhouse-public-datasets.s3.amazonaws.com/youtube/original/files/*.zst', + 'JSONLines' + ); ``` -4. Open a new tab in the SQL Console of ClickHouse Cloud (or a new `clickhouse-client` window) and watch the count increase: +4. Open a new tab in the SQL Console of ClickHouse Cloud (or a new `clickhouse-client` window) and watch the count increase. It will take a while to insert 4.56B rows, depending on your server resources. (Withtout any tweaking of settings, it takes about 4.5 hours.) ```sql -select formatReadableQuantity(count()) from youtube; -``` - -5. It will take a while to insert 4.56B rows, depending on your server resources. Once the data is inserted, go ahead and count the number of dislikes of your favorite videos or channels. Let's see how many videos were uploaded by ClickHouse: - -```sql -SELECT * +SELECT formatReadableQuantity(count()) FROM youtube -WHERE uploader ILIKE '%ClickHouse%'; ``` +```response +┌─formatReadableQuantity(count())─┐ +│ 4.56 billion │ +└─────────────────────────────────┘ +``` + +5. Once the data is inserted, go ahead and count the number of dislikes of your favorite videos or channels. Let's see how many videos were uploaded by ClickHouse: + +```sql +SELECT count() +FROM youtube +WHERE uploader = 'ClickHouse'; +``` + +```response +┌─count()─┐ +│ 84 │ +└─────────┘ + +1 row in set. Elapsed: 0.570 sec. Processed 237.57 thousand rows, 5.77 MB (416.54 thousand rows/s., 10.12 MB/s.) +``` + +:::note +The query above runs so quickly because we chose `uploader` as the first column of the primary key - so it only had to process 237k rows. +::: + 6. Here is a search for videos with **ClickHouse** in the `title` or `description` fields: ```sql @@ -146,6 +173,20 @@ FROM youtube WHERE (title ILIKE '%ClickHouse%') OR (description ILIKE '%ClickHouse%') ORDER BY like_count DESC, - view_count DESC; + view_count DESC ``` +This query has to process every row, and also parse through two columns of strings. Even then, we get decent performance at 4.15M rows/second: + +```response +1174 rows in set. Elapsed: 1099.368 sec. Processed 4.56 billion rows, 1.98 TB (4.15 million rows/s., 1.80 GB/s.) +``` + +The results look like: + +```response +┌─view_count─┬─like_count─┬─dislike_count─┬─url──────────────────────────┬─title──────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ 1919 │ 63 │ 1 │ https://youtu.be/b9MeoOtAivQ │ ClickHouse v21.10 Release Webinar │ +│ 8710 │ 62 │ 4 │ https://youtu.be/PeV1mC2z--M │ What is JDBC DriverManager? | JDBC │ +│ 3534 │ 62 │ 1 │ https://youtu.be/8nWRhK9gw10 │ CLICKHOUSE - Arquitetura Modular │ +``` \ No newline at end of file From 997ab8e0b2396666d1be992f41d4d9df18f4af20 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Sun, 26 Mar 2023 16:17:07 -0600 Subject: [PATCH 394/418] Update youtube-dislikes.md --- .../example-datasets/youtube-dislikes.md | 29 ++++++++++++++++++- 1 file changed, 28 insertions(+), 1 deletion(-) diff --git a/docs/en/getting-started/example-datasets/youtube-dislikes.md b/docs/en/getting-started/example-datasets/youtube-dislikes.md index 5353aee42db..2eb2071d5f2 100644 --- a/docs/en/getting-started/example-datasets/youtube-dislikes.md +++ b/docs/en/getting-started/example-datasets/youtube-dislikes.md @@ -160,7 +160,34 @@ WHERE uploader = 'ClickHouse'; The query above runs so quickly because we chose `uploader` as the first column of the primary key - so it only had to process 237k rows. ::: -6. Here is a search for videos with **ClickHouse** in the `title` or `description` fields: +6. Let's look and likes and dislikes of ClickHouse videos: + +```sql +SELECT + title, + like_count, + dislike_count +FROM youtube +WHERE uploader = 'ClickHouse' +ORDER BY dislike_count DESC; +``` + +The response looks like: + +```response +┌─title────────────────────────────────────────────────────────────────────────────────────────────────┬─like_count─┬─dislike_count─┐ +│ ClickHouse v21.11 Release Webinar │ 52 │ 3 │ +│ ClickHouse Introduction │ 97 │ 3 │ +│ Casa Modelo Algarve │ 180 │ 3 │ +│ Профайлер запросов: трудный путь │ 33 │ 3 │ +│ ClickHouse в Курсометре │ 4 │ 2 │ +│ 10 Good Reasons to Use ClickHouse │ 27 │ 2 │ +... + +84 rows in set. Elapsed: 0.013 sec. Processed 155.65 thousand rows, 16.94 MB (11.96 million rows/s., 1.30 GB/s.) +``` + +7. Here is a search for videos with **ClickHouse** in the `title` or `description` fields: ```sql SELECT From 6cff504f637b5e1691de930b6733b1feaf04afa5 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Sun, 26 Mar 2023 23:14:06 -0600 Subject: [PATCH 395/418] Create covid19.md --- .../example-datasets/covid19.md | 265 ++++++++++++++++++ 1 file changed, 265 insertions(+) create mode 100644 docs/en/getting-started/example-datasets/covid19.md diff --git a/docs/en/getting-started/example-datasets/covid19.md b/docs/en/getting-started/example-datasets/covid19.md new file mode 100644 index 00000000000..1e4977b1ec3 --- /dev/null +++ b/docs/en/getting-started/example-datasets/covid19.md @@ -0,0 +1,265 @@ +--- +slug: /en/getting-started/example-datasets/covid19 +sidebar_label: COVID-19 Open-Data +--- + +# COVID-19 Open-Data + +COVID-19 Open-Data attempts to assemble the largest Covid-19 epidemiological database, in addition to a powerful set of expansive covariates. It includes open, publicly sourced, licensed data relating to demographics, economy, epidemiology, geography, health, hospitalizations, mobility, government response, weather, and more. + +The details are in GitHub [here](https://github.com/GoogleCloudPlatform/covid-19-open-data). + +It's easy to insert this data into ClickHouse... + +:::note +The following commands were executed on a **Production** instance of [ClickHouse Cloud](https://clickhouse.cloud). You can easily run them on a local install as well. +::: + +1. Let's see what the data looks like: + +```sql +DESCRIBE url( + 'https://storage.googleapis.com/covid19-open-data/v3/epidemiology.csv', + 'CSVWithNames' +); +``` + +The CSV file has 10 columns: + +```response +┌─name─────────────────┬─type─────────────┐ +│ date │ Nullable(String) │ +│ location_key │ Nullable(String) │ +│ new_confirmed │ Nullable(Int64) │ +│ new_deceased │ Nullable(Int64) │ +│ new_recovered │ Nullable(Int64) │ +│ new_tested │ Nullable(Int64) │ +│ cumulative_confirmed │ Nullable(Int64) │ +│ cumulative_deceased │ Nullable(Int64) │ +│ cumulative_recovered │ Nullable(Int64) │ +│ cumulative_tested │ Nullable(Int64) │ +└──────────────────────┴──────────────────┘ + +10 rows in set. Elapsed: 0.745 sec. +``` + +2. Now let's view some of the rows: + +```sql +SELECT * +FROM url('https://storage.googleapis.com/covid19-open-data/v3/epidemiology.csv') +LIMIT 100; +``` + +Notice the `url` function easily reads data from a CSV file: + +```response +┌─c1─────────┬─c2───────────┬─c3────────────┬─c4───────────┬─c5────────────┬─c6─────────┬─c7───────────────────┬─c8──────────────────┬─c9───────────────────┬─c10───────────────┐ +│ date │ location_key │ new_confirmed │ new_deceased │ new_recovered │ new_tested │ cumulative_confirmed │ cumulative_deceased │ cumulative_recovered │ cumulative_tested │ +│ 2020-04-03 │ AD │ 24 │ 1 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 466 │ 17 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ +│ 2020-04-04 │ AD │ 57 │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 523 │ 17 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ +│ 2020-04-05 │ AD │ 17 │ 4 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 540 │ 21 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ +│ 2020-04-06 │ AD │ 11 │ 1 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 551 │ 22 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ +│ 2020-04-07 │ AD │ 15 │ 2 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 566 │ 24 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ +│ 2020-04-08 │ AD │ 23 │ 2 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 589 │ 26 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ +└────────────┴──────────────┴───────────────┴──────────────┴───────────────┴────────────┴──────────────────────┴─────────────────────┴──────────────────────┴───────────────────┘ +``` + +3. We will create a table now that we know what the data looks like: + +```sql +CREATE TABLE covid19 ( + date Date, + location_key LowCardinality(String), + new_confirmed Int32, + new_deceased Int32, + new_recovered Int32, + new_tested Int32, + cumulative_confirmed Int32, + cumulative_deceased Int32, + cumulative_recovered Int32, + cumulative_tested Int32 +) +ENGINE = MergeTree +ORDER BY (location_key, date); +``` + +4. The following command inserts the entire dataset into the `covid19` table: + +```sql +INSERT INTO covid19 + SELECT * + FROM + url( + 'https://storage.googleapis.com/covid19-open-data/v3/epidemiology.csv', + CSVWithNames, + 'date Date, + location_key LowCardinality(String), + new_confirmed Int32, + new_deceased Int32, + new_recovered Int32, + new_tested Int32, + cumulative_confirmed Int32, + cumulative_deceased Int32, + cumulative_recovered Int32, + cumulative_tested Int32' + ); +``` + +5. It goes pretty quick - let's see how many rows were inserted: + +```sql +SELECT formatReadableQuantity(count()) +FROM covid19; +``` + +```response +┌─formatReadableQuantity(count())─┐ +│ 12.53 million │ +└─────────────────────────────────┘ +``` + +6. Let's see how many total cases of Covid-19 were recorded: + +```sql +SELECT formatReadableQuantity(sum(new_confirmed)) +FROM covid19; +``` + +```response +┌─formatReadableQuantity(sum(new_confirmed))─┐ +│ 1.39 billion │ +└────────────────────────────────────────────┘ +``` + +7. You will notice the data has a lot of 0's for dates - either weekends or days where numbers were not reported each day. We can use a window function to smooth out the daily averages of new cases: + +```sql +SELECT + AVG(new_confirmed) OVER (PARTITION BY location_key ORDER BY date ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING) AS cases_smoothed, + new_confirmed, + location_key, + date +FROM covid19; +``` + +8. This query determines the latest values for each location. We can't use `max(date)` because not all countries reported every day, so we grab the last row use the row number: + +```sql +WITH latest_deaths_data AS + ( SELECT location_key, + date, + new_deceased, + new_confirmed, + ROW_NUMBER() OVER (PARTITION BY location_key ORDER BY date DESC) as rn + FROM covid19) +SELECT location_key, + date, + new_deceased, + new_confirmed, + rn +FROM latest_deaths_data +WHERE rn=1; +``` + +9. We can use `lagInFrame` to determine the `LAG` of new cases each day. In this query we filter by the `US_DC` location: + +```sql +SELECT + new_confirmed - lagInFrame(new_confirmed,1) OVER (PARTITION BY location_key ORDER BY date) AS confirmed_cases_delta, + new_confirmed, + location_key, + date +FROM covid19 +WHERE location_key = 'US_DC'; +``` + +The response look like: + +```response +┌─confirmed_cases_delta─┬─new_confirmed─┬─location_key─┬───────date─┐ +│ 0 │ 0 │ US_DC │ 2020-03-08 │ +│ 2 │ 2 │ US_DC │ 2020-03-09 │ +│ -2 │ 0 │ US_DC │ 2020-03-10 │ +│ 6 │ 6 │ US_DC │ 2020-03-11 │ +│ -6 │ 0 │ US_DC │ 2020-03-12 │ +│ 0 │ 0 │ US_DC │ 2020-03-13 │ +│ 6 │ 6 │ US_DC │ 2020-03-14 │ +│ -5 │ 1 │ US_DC │ 2020-03-15 │ +│ 4 │ 5 │ US_DC │ 2020-03-16 │ +│ 4 │ 9 │ US_DC │ 2020-03-17 │ +│ -1 │ 8 │ US_DC │ 2020-03-18 │ +│ 24 │ 32 │ US_DC │ 2020-03-19 │ +│ -26 │ 6 │ US_DC │ 2020-03-20 │ +│ 15 │ 21 │ US_DC │ 2020-03-21 │ +│ -3 │ 18 │ US_DC │ 2020-03-22 │ +│ 3 │ 21 │ US_DC │ 2020-03-23 │ +``` + +10. This query calculates the percentage of change in new cases each day, and includes a simple `increase` or `decrease` column in the result set: + +```sql +WITH confirmed_lag AS ( + SELECT + *, + lagInFrame(new_confirmed) OVER( + PARTITION BY location_key + ORDER BY date + ) AS confirmed_previous_day + FROM covid19 +), +confirmed_percent_change AS ( + SELECT + *, + COALESCE(ROUND((new_confirmed - confirmed_previous_day) / confirmed_previous_day * 100), 0) AS percent_change + FROM confirmed_lag +) +SELECT + date, + new_confirmed, + percent_change, + CASE + WHEN percent_change > 0 THEN 'increase' + WHEN percent_change = 0 THEN 'no change' + ELSE 'decrease' + END AS trend +FROM confirmed_percent_change +WHERE location_key = 'US_DC'; +``` + +The results look like + +```response +┌───────date─┬─new_confirmed─┬─percent_change─┬─trend─────┐ +│ 2020-03-08 │ 0 │ nan │ decrease │ +│ 2020-03-09 │ 2 │ inf │ increase │ +│ 2020-03-10 │ 0 │ -100 │ decrease │ +│ 2020-03-11 │ 6 │ inf │ increase │ +│ 2020-03-12 │ 0 │ -100 │ decrease │ +│ 2020-03-13 │ 0 │ nan │ decrease │ +│ 2020-03-14 │ 6 │ inf │ increase │ +│ 2020-03-15 │ 1 │ -83 │ decrease │ +│ 2020-03-16 │ 5 │ 400 │ increase │ +│ 2020-03-17 │ 9 │ 80 │ increase │ +│ 2020-03-18 │ 8 │ -11 │ decrease │ +│ 2020-03-19 │ 32 │ 300 │ increase │ +│ 2020-03-20 │ 6 │ -81 │ decrease │ +│ 2020-03-21 │ 21 │ 250 │ increase │ +│ 2020-03-22 │ 18 │ -14 │ decrease │ +│ 2020-03-23 │ 21 │ 17 │ increase │ +│ 2020-03-24 │ 46 │ 119 │ increase │ +│ 2020-03-25 │ 48 │ 4 │ increase │ +│ 2020-03-26 │ 36 │ -25 │ decrease │ +│ 2020-03-27 │ 37 │ 3 │ increase │ +│ 2020-03-28 │ 38 │ 3 │ increase │ +│ 2020-03-29 │ 59 │ 55 │ increase │ +│ 2020-03-30 │ 94 │ 59 │ increase │ +│ 2020-03-31 │ 91 │ -3 │ decrease │ +│ 2020-04-01 │ 67 │ -26 │ decrease │ +│ 2020-04-02 │ 104 │ 55 │ increase │ +│ 2020-04-03 │ 145 │ 39 │ increase │ +``` + +:::note +As mentioned in the [GitHub repo](https://github.com/GoogleCloudPlatform/covid-19-open-data), the datset is no longer updated as of September 15, 2022. +::: \ No newline at end of file From de177de28ade12dd5170271ab26289c66b041e37 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Sun, 26 Mar 2023 23:24:33 -0600 Subject: [PATCH 396/418] Update covid19.md --- docs/en/getting-started/example-datasets/covid19.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/example-datasets/covid19.md b/docs/en/getting-started/example-datasets/covid19.md index 1e4977b1ec3..9482e8870d2 100644 --- a/docs/en/getting-started/example-datasets/covid19.md +++ b/docs/en/getting-started/example-datasets/covid19.md @@ -143,7 +143,7 @@ SELECT FROM covid19; ``` -8. This query determines the latest values for each location. We can't use `max(date)` because not all countries reported every day, so we grab the last row use the row number: +8. This query determines the latest values for each location. We can't use `max(date)` because not all countries reported every day, so we grab the last row using `ROW_NUMBER`: ```sql WITH latest_deaths_data AS From 79e22d13fbaf495b83e10b4ca87363eb81f2635c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 27 Mar 2023 07:36:39 +0000 Subject: [PATCH 397/418] small fix --- src/Coordination/KeeperDispatcher.cpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 12248950c32..1828182751d 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -92,7 +92,7 @@ void KeeperDispatcher::requestThread() KeeperStorage::RequestsForSessions current_batch; size_t current_batch_bytes_size = 0; - bool has_read_request{false}; + bool has_read_request = false; /// If new request is not read request or we must to process it through quorum. /// Otherwise we will process it locally. @@ -101,8 +101,6 @@ void KeeperDispatcher::requestThread() current_batch_bytes_size += request.request->bytesSize(); current_batch.emplace_back(request); - size_t read_requests = 0; - const auto try_get_request = [&] { /// Trying to get batch requests as fast as possible @@ -112,7 +110,6 @@ void KeeperDispatcher::requestThread() /// Don't append read request into batch, we have to process them separately if (!coordination_settings->quorum_reads && request.request->isReadRequest()) { - ++read_requests; const auto & last_request = current_batch.back(); std::lock_guard lock(read_request_queue_mutex); read_request_queue[last_request.session_id][last_request.request->xid].push_back(request); From 4fdce459ebdda82c65f9eefecbad76cc25de4004 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 27 Mar 2023 08:34:18 +0000 Subject: [PATCH 398/418] Fix expected result --- .../0_stateless/02668_parse_datetime_in_joda_syntax.reference | 2 ++ 1 file changed, 2 insertions(+) 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 9391af957a1..7402f104ae4 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 @@ -342,3 +342,5 @@ select parseDateTimeInJodaSyntax('0/', 's/', 'UTC') = toDateTime('1970-01-01 00: 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 } +-- integer overflow in AST Fuzzer +select parseDateTimeInJodaSyntax('19191919191919191919191919191919', 'CCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCC', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } From 2981890ab4d583216f2a28722a0faddaef9a0329 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Mon, 27 Mar 2023 12:03:47 +0200 Subject: [PATCH 399/418] Fix Sample... --- docs/en/sql-reference/statements/alter/projection.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/projection.md b/docs/en/sql-reference/statements/alter/projection.md index 626d71709ac..030e9352a00 100644 --- a/docs/en/sql-reference/statements/alter/projection.md +++ b/docs/en/sql-reference/statements/alter/projection.md @@ -128,7 +128,7 @@ SELECT user_agent, sum(pages_visited) FROM visits -GROUP BY user_id +GROUP BY user_agent ``` As mentioned before, we could review the `system.query_log` table. On the `projections` field we have the name of the projection used or empty if none has been used: From 78eb71a49b58a89e5fa3268c216bf0217fdd0884 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Mar 2023 12:11:32 +0200 Subject: [PATCH 400/418] Remove the old changelog script --- utils/changelog-simple/.gitignore | 2 - utils/changelog-simple/README.md | 21 --- utils/changelog-simple/changelog.sh | 96 ------------ utils/changelog-simple/format-changelog.py | 165 --------------------- 4 files changed, 284 deletions(-) delete mode 100644 utils/changelog-simple/.gitignore delete mode 100644 utils/changelog-simple/README.md delete mode 100755 utils/changelog-simple/changelog.sh delete mode 100755 utils/changelog-simple/format-changelog.py diff --git a/utils/changelog-simple/.gitignore b/utils/changelog-simple/.gitignore deleted file mode 100644 index 78caa68e38e..00000000000 --- a/utils/changelog-simple/.gitignore +++ /dev/null @@ -1,2 +0,0 @@ -*.txt -*.json diff --git a/utils/changelog-simple/README.md b/utils/changelog-simple/README.md deleted file mode 100644 index cd8f8da9b61..00000000000 --- a/utils/changelog-simple/README.md +++ /dev/null @@ -1,21 +0,0 @@ -## How To Generate Changelog - -Generate github token: -* https://github.com/settings/tokens - keep all checkboxes unchecked, no scopes need to be enabled. - -Dependencies: -``` -sudo apt-get install git curl jq python3 python3-fuzzywuzzy -``` - -Update information about tags: -``` -git fetch --tags -``` - -Usage example: - -``` -export GITHUB_USER=... GITHUB_TOKEN=ghp_... -./changelog.sh v21.5.6.6-stable v21.6.2.7-prestable -``` diff --git a/utils/changelog-simple/changelog.sh b/utils/changelog-simple/changelog.sh deleted file mode 100755 index 52817acfae4..00000000000 --- a/utils/changelog-simple/changelog.sh +++ /dev/null @@ -1,96 +0,0 @@ -#!/bin/bash -set -e - -script_dir="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" - -from="$1" -to="$2" -log_command=(git log "$from..$to" --first-parent) - -"${log_command[@]}" > "changelog-log.txt" - -# Check for diamond merges. -if "${log_command[@]}" --oneline --grep "Merge branch '" | grep '' -then - # DO NOT ADD automated handling of diamond merges to this script. - # It is an unsustainable way to work with git, and it MUST be visible. - echo Warning: suspected diamond merges above. - echo Some commits will be missed, review these manually. -fi - -# Search for PR numbers in commit messages. First variant is normal merge, and second -# variant is squashed. Next are some backport message variants. -find_prs=(sed -n "s/^.*merg[eding]*.*#\([[:digit:]]\+\).*$/\1/Ip; - s/^.*(#\([[:digit:]]\+\))$/\1/p; - s/^.*back[- ]*port[ed of]*.*#\([[:digit:]]\+\).*$/\1/Ip; - s/^.*cherry[- ]*pick[ed of]*.*#\([[:digit:]]\+\).*$/\1/Ip") - -# awk is to filter out small task numbers from different task tracker, which are -# referenced by documentation commits like '* DOCSUP-824: query log (#115)'. -"${find_prs[@]}" "changelog-log.txt" | sort -rn | uniq | awk '$0 > 1000 { print $0 }' > "changelog-prs.txt" - -echo "$(wc -l < "changelog-prs.txt") PRs added between $from and $to." -if [ $(wc -l < "changelog-prs.txt") -eq 0 ] ; then exit 0 ; fi - -function github_download() -{ - local url=${1} - local file=${2} - if ! [ -f "$file" ] - then - echo "curl -u \"$GITHUB_USER:***\" -sSf \"$url\" > \"$file\"" - - if ! curl -u "$GITHUB_USER:$GITHUB_TOKEN" \ - -sSf "$url" \ - > "$file" - then - >&2 echo "Failed to download '$url' to '$file'. Contents: '$(cat "$file")'." - rm "$file" - return 1 - fi - sleep 0.1 - fi -} - -rm changelog-prs-filtered.txt &> /dev/null ||: -for pr in $(cat "changelog-prs.txt") -do - # Download PR info from github. - file="pr$pr.json" - github_download "https://api.github.com/repos/ClickHouse/ClickHouse/pulls/$pr" "$file" || continue - - if ! [ "$pr" == "$(jq -r .number "$file")" ] - then - >&2 echo "Got wrong data for PR #$pr (please check and remove '$file')." - continue - fi - - # Filter out PRs by bots. - user_login=$(jq -r .user.login "$file") - - filter_bot=$(echo "$user_login" | grep -q "\[bot\]$" && echo "Skip." || echo "Ok." ||:) - filter_robot=$(echo "$user_login" | grep -q "robot-clickhouse" && echo "Skip." || echo "Ok." ||:) - - if [ "Skip." == "$filter_robot" ] || [ "Skip." == "$filter_bot" ] - then - continue - fi - - # Download author info from github. - user_id=$(jq -r .user.id "$file") - user_file="user$user_id.json" - github_download "$(jq -r .user.url "$file")" "$user_file" || continue - - if ! [ "$user_id" == "$(jq -r .id "$user_file")" ] - then - >&2 echo "Got wrong data for user #$user_id (please check and remove '$user_file')." - continue - fi - - echo "$pr" >> changelog-prs-filtered.txt -done - -echo "### ClickHouse release $to FIXME as compared to $from -" > changelog.md -"$script_dir/format-changelog.py" changelog-prs-filtered.txt >> changelog.md -cat changelog.md diff --git a/utils/changelog-simple/format-changelog.py b/utils/changelog-simple/format-changelog.py deleted file mode 100755 index 01f2694dd0f..00000000000 --- a/utils/changelog-simple/format-changelog.py +++ /dev/null @@ -1,165 +0,0 @@ -#!/usr/bin/python3 - -import argparse -import collections -import fuzzywuzzy.fuzz -import itertools -import json -import os -import re -import sys - -parser = argparse.ArgumentParser(description="Format changelog for given PRs.") -parser.add_argument( - "file", - metavar="FILE", - type=argparse.FileType("r", encoding="utf-8"), - nargs="?", - default=sys.stdin, - help="File with PR numbers, one per line.", -) -args = parser.parse_args() - - -# This function mirrors the PR description checks in ClickhousePullRequestTrigger. -# Returns False if the PR should not be mentioned changelog. -def parse_one_pull_request(item): - description = item["body"] - # Don't skip empty lines because they delimit parts of description - lines = [ - line - for line in [ - x.strip() for x in (description.split("\n") if description else []) - ] - ] - lines = [re.sub(r"\s+", " ", l) for l in lines] - - category = "" - entry = "" - - if lines: - i = 0 - while i < len(lines): - if re.match(r"(?i).*change\s*log\s*category", lines[i]): - i += 1 - if i >= len(lines): - break - # Can have one empty line between header and the category itself. Filter it out. - if not lines[i]: - i += 1 - if i >= len(lines): - break - category = re.sub(r"^[-*\s]*", "", lines[i]) - i += 1 - - elif re.match(r"(?i).*change\s*log\s*entry", lines[i]): - i += 1 - # Can have one empty line between header and the entry itself. Filter it out. - if i < len(lines) and not lines[i]: - i += 1 - # All following lines until empty one are the changelog entry. - entry_lines = [] - while i < len(lines) and lines[i]: - entry_lines.append(lines[i]) - i += 1 - entry = " ".join(entry_lines) - else: - i += 1 - - if not category: - # Shouldn't happen, because description check in CI should catch such PRs. - # Fall through, so that it shows up in output and the user can fix it. - category = "NO CL CATEGORY" - - # Filter out the PR categories that are not for changelog. - if re.match( - r"(?i)doc|((non|in|not|un)[-\s]*significant)|(not[ ]*for[ ]*changelog)", - category, - ): - return False - - if not entry: - # Shouldn't happen, because description check in CI should catch such PRs. - category = "NO CL ENTRY" - entry = "NO CL ENTRY: '" + item["title"] + "'" - - entry = entry.strip() - if entry[-1] != ".": - entry += "." - - item["entry"] = entry - item["category"] = category - - return True - - -# This array gives the preferred category order, and is also used to -# normalize category names. -categories_preferred_order = [ - "Backward Incompatible Change", - "New Feature", - "Performance Improvement", - "Improvement", - "Bug Fix", - "Build/Testing/Packaging Improvement", - "Other", -] - -category_to_pr = collections.defaultdict(lambda: []) -users = {} -for line in args.file: - pr = json.loads(open(f"pr{line.strip()}.json").read()) - assert pr["number"] - if not parse_one_pull_request(pr): - continue - - assert pr["category"] - - # Normalize category name - for c in categories_preferred_order: - if fuzzywuzzy.fuzz.ratio(pr["category"].lower(), c.lower()) >= 90: - pr["category"] = c - break - - category_to_pr[pr["category"]].append(pr) - user_id = pr["user"]["id"] - users[user_id] = json.loads(open(f"user{user_id}.json").read()) - - -def print_category(category): - print(("#### " + category)) - print() - for pr in category_to_pr[category]: - user = users[pr["user"]["id"]] - user_name = user["name"] if user["name"] else user["login"] - - # Substitute issue links. - # 1) issue number w/o markdown link - pr["entry"] = re.sub( - r"([^[])#([0-9]{4,})", - r"\1[#\2](https://github.com/ClickHouse/ClickHouse/issues/\2)", - pr["entry"], - ) - # 2) issue URL w/o markdown link - pr["entry"] = re.sub( - r"([^(])https://github.com/ClickHouse/ClickHouse/issues/([0-9]{4,})", - r"\1[#\2](https://github.com/ClickHouse/ClickHouse/issues/\2)", - pr["entry"], - ) - - print( - f'* {pr["entry"]} [#{pr["number"]}]({pr["html_url"]}) ([{user_name}]({user["html_url"]})).' - ) - - print() - - -# Print categories in preferred order -for category in categories_preferred_order: - if category in category_to_pr: - print_category(category) - category_to_pr.pop(category) - -# Print the rest of the categories -for category in category_to_pr: - print_category(category) From 1963268fecad6fa100b4945172dff1604645d22e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Mar 2023 13:17:00 +0300 Subject: [PATCH 401/418] Update PULL_REQUEST_TEMPLATE.md There is no "prestable" version. --- .github/PULL_REQUEST_TEMPLATE.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 5d09d3a9ef3..449abc9484d 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -7,7 +7,7 @@ tests/ci/run_check.py ### Changelog category (leave one): - New Feature - Improvement -- Bug Fix (user-visible misbehavior in official stable or prestable release) +- Bug Fix (user-visible misbehavior in an official stable release) - Performance Improvement - Backward Incompatible Change - Build/Testing/Packaging Improvement From e0252db8d457297ef64d849abd62be7b7efc8307 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Mar 2023 12:19:32 +0200 Subject: [PATCH 402/418] No prestable --- CHANGELOG.md | 2 +- docs/changelogs/v21.10.1.8013-prestable.md | 2 +- docs/changelogs/v21.10.2.15-stable.md | 2 +- docs/changelogs/v21.10.3.9-stable.md | 4 ++-- docs/changelogs/v21.10.4.26-stable.md | 2 +- docs/changelogs/v21.10.5.3-stable.md | 2 +- docs/changelogs/v21.10.6.2-stable.md | 4 ++-- docs/changelogs/v21.11.1.8636-prestable.md | 4 ++-- docs/changelogs/v21.11.11.1-stable.md | 2 +- docs/changelogs/v21.11.2.2-stable.md | 2 +- docs/changelogs/v21.11.3.6-stable.md | 2 +- docs/changelogs/v21.11.4.14-stable.md | 2 +- docs/changelogs/v21.11.5.33-stable.md | 2 +- docs/changelogs/v21.11.6.7-stable.md | 2 +- docs/changelogs/v21.11.7.9-stable.md | 4 ++-- docs/changelogs/v21.11.8.4-stable.md | 2 +- docs/changelogs/v21.11.9.1-stable.md | 2 +- docs/changelogs/v21.12.1.9017-prestable.md | 2 +- docs/changelogs/v21.12.2.17-stable.md | 4 ++-- docs/changelogs/v21.12.3.32-stable.md | 2 +- docs/changelogs/v21.12.4.1-stable.md | 2 +- docs/changelogs/v21.3.16.5-lts.md | 2 +- docs/changelogs/v21.3.17.2-lts.md | 2 +- docs/changelogs/v21.3.18.4-lts.md | 2 +- docs/changelogs/v21.3.19.1-lts.md | 4 ++-- docs/changelogs/v21.3.20.1-lts.md | 2 +- docs/changelogs/v21.6.9.7-stable.md | 2 +- docs/changelogs/v21.7.10.4-stable.md | 2 +- docs/changelogs/v21.7.11.3-stable.md | 2 +- docs/changelogs/v21.7.9.7-stable.md | 2 +- docs/changelogs/v21.8.10.19-lts.md | 2 +- docs/changelogs/v21.8.11.4-lts.md | 4 ++-- docs/changelogs/v21.8.12.29-lts.md | 2 +- docs/changelogs/v21.8.13.6-lts.md | 4 ++-- docs/changelogs/v21.8.14.5-lts.md | 2 +- docs/changelogs/v21.8.15.7-lts.md | 2 +- docs/changelogs/v21.8.5.7-lts.md | 2 +- docs/changelogs/v21.8.6.15-lts.md | 2 +- docs/changelogs/v21.8.7.22-lts.md | 2 +- docs/changelogs/v21.8.8.29-lts.md | 2 +- docs/changelogs/v21.8.9.13-lts.md | 2 +- docs/changelogs/v21.9.2.17-stable.md | 2 +- docs/changelogs/v21.9.3.30-stable.md | 2 +- docs/changelogs/v21.9.4.35-stable.md | 2 +- docs/changelogs/v21.9.5.16-stable.md | 2 +- docs/changelogs/v21.9.6.24-stable.md | 4 ++-- docs/changelogs/v22.1.1.2542-prestable.md | 4 ++-- docs/changelogs/v22.1.3.7-stable.md | 2 +- docs/changelogs/v22.1.4.30-stable.md | 2 +- docs/changelogs/v22.10.1.1877-stable.md | 4 ++-- docs/changelogs/v22.10.2.11-stable.md | 2 +- docs/changelogs/v22.10.3.27-stable.md | 2 +- docs/changelogs/v22.10.4.23-stable.md | 2 +- docs/changelogs/v22.10.5.54-stable.md | 3 +-- docs/changelogs/v22.10.6.3-stable.md | 3 +-- docs/changelogs/v22.10.7.13-stable.md | 3 +-- docs/changelogs/v22.11.1.1360-stable.md | 2 +- docs/changelogs/v22.11.2.30-stable.md | 2 +- docs/changelogs/v22.11.3.47-stable.md | 3 +-- docs/changelogs/v22.11.4.3-stable.md | 3 +-- docs/changelogs/v22.11.5.15-stable.md | 3 +-- docs/changelogs/v22.11.6.44-stable.md | 3 +-- docs/changelogs/v22.12.1.1752-stable.md | 2 +- docs/changelogs/v22.12.2.25-stable.md | 2 +- docs/changelogs/v22.12.3.5-stable.md | 3 +-- docs/changelogs/v22.12.4.76-stable.md | 3 +-- docs/changelogs/v22.12.5.34-stable.md | 3 +-- docs/changelogs/v22.2.1.2139-prestable.md | 2 +- docs/changelogs/v22.2.3.5-stable.md | 2 +- docs/changelogs/v22.3.1.1262-prestable.md | 2 +- docs/changelogs/v22.3.10.22-lts.md | 4 ++-- docs/changelogs/v22.3.11.12-lts.md | 2 +- docs/changelogs/v22.3.12.19-lts.md | 2 +- docs/changelogs/v22.3.13.80-lts.md | 4 ++-- docs/changelogs/v22.3.14.18-lts.md | 2 +- docs/changelogs/v22.3.14.23-lts.md | 2 +- docs/changelogs/v22.3.15.33-lts.md | 2 +- docs/changelogs/v22.3.16.1190-lts.md | 3 +-- docs/changelogs/v22.3.18.37-lts.md | 3 +-- docs/changelogs/v22.3.19.6-lts.md | 3 +-- docs/changelogs/v22.3.3.44-lts.md | 2 +- docs/changelogs/v22.3.4.20-lts.md | 2 +- docs/changelogs/v22.3.5.5-lts.md | 2 +- docs/changelogs/v22.3.6.5-lts.md | 2 +- docs/changelogs/v22.3.7.28-lts.md | 4 ++-- docs/changelogs/v22.3.8.39-lts.md | 4 ++-- docs/changelogs/v22.3.9.19-lts.md | 2 +- docs/changelogs/v22.4.1.2305-prestable.md | 2 +- docs/changelogs/v22.4.3.3-stable.md | 2 +- docs/changelogs/v22.4.4.7-stable.md | 2 +- docs/changelogs/v22.4.5.9-stable.md | 2 +- docs/changelogs/v22.4.6.53-stable.md | 4 ++-- docs/changelogs/v22.5.1.2079-stable.md | 4 ++-- docs/changelogs/v22.5.2.53-stable.md | 2 +- docs/changelogs/v22.5.3.21-stable.md | 2 +- docs/changelogs/v22.5.4.19-stable.md | 2 +- docs/changelogs/v22.6.1.1985-stable.md | 4 ++-- docs/changelogs/v22.6.2.12-stable.md | 2 +- docs/changelogs/v22.6.3.35-stable.md | 2 +- docs/changelogs/v22.6.4.35-stable.md | 2 +- docs/changelogs/v22.6.5.22-stable.md | 2 +- docs/changelogs/v22.6.6.16-stable.md | 2 +- docs/changelogs/v22.6.7.7-stable.md | 2 +- docs/changelogs/v22.6.8.35-stable.md | 2 +- docs/changelogs/v22.6.9.11-stable.md | 2 +- docs/changelogs/v22.7.1.2484-stable.md | 4 ++-- docs/changelogs/v22.7.2.15-stable.md | 2 +- docs/changelogs/v22.7.3.5-stable.md | 2 +- docs/changelogs/v22.7.4.16-stable.md | 2 +- docs/changelogs/v22.7.5.13-stable.md | 2 +- docs/changelogs/v22.7.6.74-stable.md | 2 +- docs/changelogs/v22.7.7.24-stable.md | 2 +- docs/changelogs/v22.8.1.2097-lts.md | 2 +- docs/changelogs/v22.8.10.29-lts.md | 2 +- docs/changelogs/v22.8.11.15-lts.md | 2 +- docs/changelogs/v22.8.12.45-lts.md | 3 +-- docs/changelogs/v22.8.13.20-lts.md | 3 +-- docs/changelogs/v22.8.14.53-lts.md | 3 +-- docs/changelogs/v22.8.15.23-lts.md | 3 +-- docs/changelogs/v22.8.3.13-lts.md | 2 +- docs/changelogs/v22.8.4.7-lts.md | 2 +- docs/changelogs/v22.8.5.29-lts.md | 2 +- docs/changelogs/v22.8.6.71-lts.md | 2 +- docs/changelogs/v22.8.7.34-lts.md | 2 +- docs/changelogs/v22.8.8.3-lts.md | 2 +- docs/changelogs/v22.8.9.24-lts.md | 2 +- docs/changelogs/v22.9.2.7-stable.md | 2 +- docs/changelogs/v22.9.3.18-stable.md | 2 +- docs/changelogs/v22.9.4.32-stable.md | 2 +- docs/changelogs/v22.9.5.25-stable.md | 2 +- docs/changelogs/v22.9.6.20-stable.md | 2 +- docs/changelogs/v22.9.7.34-stable.md | 2 +- docs/changelogs/v23.1.1.3077-stable.md | 3 +-- docs/changelogs/v23.1.2.9-stable.md | 3 +-- docs/changelogs/v23.1.3.5-stable.md | 3 +-- docs/changelogs/v23.1.4.58-stable.md | 3 +-- docs/changelogs/v23.1.5.24-stable.md | 3 +-- docs/changelogs/v23.2.1.2537-stable.md | 3 +-- docs/changelogs/v23.2.2.20-stable.md | 3 +-- docs/changelogs/v23.2.3.17-stable.md | 3 +-- docs/changelogs/v23.2.4.12-stable.md | 3 +-- tests/ci/run_check.py | 4 ++-- utils/changelog/changelog.py | 8 ++++++++ 143 files changed, 170 insertions(+), 188 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index e22377e2332..01bbafe2f16 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -140,7 +140,7 @@ * Upgrade Intel QPL from v0.3.0 to v1.0.0 2. Build libaccel-config and link it statically to QPL library instead of dynamically. [#45809](https://github.com/ClickHouse/ClickHouse/pull/45809) ([jasperzhu](https://github.com/jinjunzh)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Flush data exactly by `rabbitmq_flush_interval_ms` or by `rabbitmq_max_block_size` in `StorageRabbitMQ`. Closes [#42389](https://github.com/ClickHouse/ClickHouse/issues/42389). Closes [#45160](https://github.com/ClickHouse/ClickHouse/issues/45160). [#44404](https://github.com/ClickHouse/ClickHouse/pull/44404) ([Kseniia Sumarokova](https://github.com/kssenii)). * Use PODArray to render in sparkBar function, so we can control the memory usage. Close [#44467](https://github.com/ClickHouse/ClickHouse/issues/44467). [#44489](https://github.com/ClickHouse/ClickHouse/pull/44489) ([Duc Canh Le](https://github.com/canhld94)). diff --git a/docs/changelogs/v21.10.1.8013-prestable.md b/docs/changelogs/v21.10.1.8013-prestable.md index 02ea593e02a..1c40aa67711 100644 --- a/docs/changelogs/v21.10.1.8013-prestable.md +++ b/docs/changelogs/v21.10.1.8013-prestable.md @@ -108,7 +108,7 @@ sidebar_label: 2022 * Print out git status information at CMake configure stage. [#28047](https://github.com/ClickHouse/ClickHouse/pull/28047) ([Braulio Valdivielso Martínez](https://github.com/BraulioVM)). * Add new log level `` for testing environments. [#28559](https://github.com/ClickHouse/ClickHouse/pull/28559) ([alesapin](https://github.com/alesapin)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Fix handling null value with type of Nullable(String) in function JSONExtract. This fixes [#27929](https://github.com/ClickHouse/ClickHouse/issues/27929) and [#27930](https://github.com/ClickHouse/ClickHouse/issues/27930) . This was introduced in https://github.com/ClickHouse/ClickHouse/pull/25452 . [#27939](https://github.com/ClickHouse/ClickHouse/pull/27939) ([Amos Bird](https://github.com/amosbird)). * Fix extremely rare segfaults on shutdown due to incorrect order of context/config reloader shutdown. [#28088](https://github.com/ClickHouse/ClickHouse/pull/28088) ([nvartolomei](https://github.com/nvartolomei)). diff --git a/docs/changelogs/v21.10.2.15-stable.md b/docs/changelogs/v21.10.2.15-stable.md index 4ae5c8f5072..42402808260 100644 --- a/docs/changelogs/v21.10.2.15-stable.md +++ b/docs/changelogs/v21.10.2.15-stable.md @@ -17,7 +17,7 @@ sidebar_label: 2022 * Backported in [#29970](https://github.com/ClickHouse/ClickHouse/issues/29970): Fix shutdown of `AccessControlManager`. Now there can't be reloading of the configuration after AccessControlManager has been destroyed. This PR fixes the flaky test [test_user_directories/test.py::test_relative_path](https://clickhouse-test-reports.s3.yandex.net/0/f0e3122507ed8bea3f177495531c7d56bcb32466/integration_tests_(thread).html). [#29951](https://github.com/ClickHouse/ClickHouse/pull/29951) ([Vitaly Baranov](https://github.com/vitlibar)). * Backported in [#30051](https://github.com/ClickHouse/ClickHouse/issues/30051): Fix releasing query ID and session ID at the end of query processing while handing gRPC call. This PR fixes flaky test [test_grpc_protocol/test.py::test_session](https://clickhouse-test-reports.s3.yandex.net/0/1ac03811a2df9717fa7c633d1af03def821d24b6/integration_tests_(memory).html). [#29954](https://github.com/ClickHouse/ClickHouse/pull/29954) ([Vitaly Baranov](https://github.com/vitlibar)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#29054](https://github.com/ClickHouse/ClickHouse/issues/29054): Fix invalid constant type conversion when nullable or lowcardinality primary key is used. [#28636](https://github.com/ClickHouse/ClickHouse/pull/28636) ([Amos Bird](https://github.com/amosbird)). * Backported in [#28795](https://github.com/ClickHouse/ClickHouse/issues/28795): - Fix the number of arguments required by s2RectAdd and s2RectContains functions. [#28663](https://github.com/ClickHouse/ClickHouse/pull/28663) ([Bharat Nallan](https://github.com/bharatnc)). diff --git a/docs/changelogs/v21.10.3.9-stable.md b/docs/changelogs/v21.10.3.9-stable.md index d0384d58e23..327e34ca64c 100644 --- a/docs/changelogs/v21.10.3.9-stable.md +++ b/docs/changelogs/v21.10.3.9-stable.md @@ -18,11 +18,11 @@ sidebar_label: 2022 * Backported in [#30620](https://github.com/ClickHouse/ClickHouse/issues/30620): Fix reading from empty file on encrypted disk. [#30494](https://github.com/ClickHouse/ClickHouse/pull/30494) ([Vitaly Baranov](https://github.com/vitlibar)). * Backported in [#31369](https://github.com/ClickHouse/ClickHouse/issues/31369): Fix SHOW GRANTS when partial revokes are used. This PR fixes [#31138](https://github.com/ClickHouse/ClickHouse/issues/31138). [#31249](https://github.com/ClickHouse/ClickHouse/pull/31249) ([Vitaly Baranov](https://github.com/vitlibar)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release +#### Bug Fix (user-visible misbehaviour in official stable release * Backported in [#30915](https://github.com/ClickHouse/ClickHouse/issues/30915): Fix `ORDER BY ... WITH FILL` with set `TO` and `FROM` and no rows in result set. [#30888](https://github.com/ClickHouse/ClickHouse/pull/30888) ([Anton Popov](https://github.com/CurtizJ)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#30824](https://github.com/ClickHouse/ClickHouse/issues/30824): Fix "Column is not under aggregate function and not in GROUP BY" with PREWHERE (Fixes: [#28461](https://github.com/ClickHouse/ClickHouse/issues/28461)). [#28502](https://github.com/ClickHouse/ClickHouse/pull/28502) ([Azat Khuzhin](https://github.com/azat)). * Backported in [#30766](https://github.com/ClickHouse/ClickHouse/issues/30766): Fix hanging DDL queries on Replicated database while adding a new replica. [#29328](https://github.com/ClickHouse/ClickHouse/pull/29328) ([Kevin Michel](https://github.com/kmichel-aiven)). diff --git a/docs/changelogs/v21.10.4.26-stable.md b/docs/changelogs/v21.10.4.26-stable.md index 7d1cc93bb98..267f2109f6f 100644 --- a/docs/changelogs/v21.10.4.26-stable.md +++ b/docs/changelogs/v21.10.4.26-stable.md @@ -13,7 +13,7 @@ sidebar_label: 2022 #### Bug Fix * Backported in [#31573](https://github.com/ClickHouse/ClickHouse/issues/31573): Quota limit was not reached, but the limit was exceeded. This PR fixes [#31174](https://github.com/ClickHouse/ClickHouse/issues/31174). [#31337](https://github.com/ClickHouse/ClickHouse/pull/31337) ([sunny](https://github.com/sunny19930321)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#31518](https://github.com/ClickHouse/ClickHouse/issues/31518): Remove not like function into RPNElement. [#31169](https://github.com/ClickHouse/ClickHouse/pull/31169) ([sundyli](https://github.com/sundy-li)). * Backported in [#31554](https://github.com/ClickHouse/ClickHouse/issues/31554): Resolve `nullptr` in STS credentials provider for S3. [#31409](https://github.com/ClickHouse/ClickHouse/pull/31409) ([Vladimir Chebotarev](https://github.com/excitoon)). diff --git a/docs/changelogs/v21.10.5.3-stable.md b/docs/changelogs/v21.10.5.3-stable.md index 88d3d70028e..7c717dfe838 100644 --- a/docs/changelogs/v21.10.5.3-stable.md +++ b/docs/changelogs/v21.10.5.3-stable.md @@ -11,7 +11,7 @@ sidebar_label: 2022 * Backported in [#32252](https://github.com/ClickHouse/ClickHouse/issues/32252): Fix skipping columns while writing protobuf. This PR fixes [#31160](https://github.com/ClickHouse/ClickHouse/issues/31160), see the comment [#31160](https://github.com/ClickHouse/ClickHouse/issues/31160)#issuecomment-980595318. [#31988](https://github.com/ClickHouse/ClickHouse/pull/31988) ([Vitaly Baranov](https://github.com/vitlibar)). * Backported in [#32346](https://github.com/ClickHouse/ClickHouse/issues/32346): Fix bug when remove unneeded columns in subquery. If there is an aggregation function in query without group by, do not remove if it is unneeded. [#32289](https://github.com/ClickHouse/ClickHouse/pull/32289) ([dongyifeng](https://github.com/dyf6372)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#32151](https://github.com/ClickHouse/ClickHouse/issues/32151): Fix crash when function `dictGet` with type is used for dictionary attribute when type is `Nullable`. Fixes [#30980](https://github.com/ClickHouse/ClickHouse/issues/30980). [#31800](https://github.com/ClickHouse/ClickHouse/pull/31800) ([Maksim Kita](https://github.com/kitaisreal)). * Backported in [#32093](https://github.com/ClickHouse/ClickHouse/issues/32093): Some `GET_PART` entry might hang in replication queue if part is lost on all replicas and there are no other parts in the same partition. It's fixed in cases when partition key contains only columns of integer types or `Date[Time]`. Fixes [#31485](https://github.com/ClickHouse/ClickHouse/issues/31485). [#31887](https://github.com/ClickHouse/ClickHouse/pull/31887) ([Alexander Tokmakov](https://github.com/tavplubix)). diff --git a/docs/changelogs/v21.10.6.2-stable.md b/docs/changelogs/v21.10.6.2-stable.md index 74f037b2f8f..0e8e934e2fa 100644 --- a/docs/changelogs/v21.10.6.2-stable.md +++ b/docs/changelogs/v21.10.6.2-stable.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Bug Fix * Backported in [#32692](https://github.com/ClickHouse/ClickHouse/issues/32692): Quota limit was not reached, but the limit was exceeded. This PR fixes [#31174](https://github.com/ClickHouse/ClickHouse/issues/31174). [#31656](https://github.com/ClickHouse/ClickHouse/pull/31656) ([sunny](https://github.com/sunny19930321)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#32680](https://github.com/ClickHouse/ClickHouse/issues/32680): Fix unexpected projection removal when detaching parts. [#32067](https://github.com/ClickHouse/ClickHouse/pull/32067) ([Amos Bird](https://github.com/amosbird)). * Backported in [#32285](https://github.com/ClickHouse/ClickHouse/issues/32285): Dictionaries fix cases when `{condition}` does not work for custom database queries. [#32117](https://github.com/ClickHouse/ClickHouse/pull/32117) ([Maksim Kita](https://github.com/kitaisreal)). @@ -23,7 +23,7 @@ sidebar_label: 2022 * Backported in [#33182](https://github.com/ClickHouse/ClickHouse/issues/33182): Server might fail to start if database with `MySQL` engine cannot connect to MySQL server, it's fixed. Fixes [#14441](https://github.com/ClickHouse/ClickHouse/issues/14441). [#32802](https://github.com/ClickHouse/ClickHouse/pull/32802) ([Alexander Tokmakov](https://github.com/tavplubix)). * Backported in [#33655](https://github.com/ClickHouse/ClickHouse/issues/33655): Fix hdfs url check that didn't allow using HA namenode address. Bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/31042. [#32976](https://github.com/ClickHouse/ClickHouse/pull/32976) ([Kruglov Pavel](https://github.com/Avogar)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release): +#### Bug Fix (user-visible misbehaviour in official stable release): * Backported in [#32657](https://github.com/ClickHouse/ClickHouse/issues/32657): Fix possible crash (or incorrect result) in case of `LowCardinality` arguments of window function. Fixes [#31114](https://github.com/ClickHouse/ClickHouse/issues/31114). [#31888](https://github.com/ClickHouse/ClickHouse/pull/31888) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). diff --git a/docs/changelogs/v21.11.1.8636-prestable.md b/docs/changelogs/v21.11.1.8636-prestable.md index 407a5196c1d..d6a435dd3ce 100644 --- a/docs/changelogs/v21.11.1.8636-prestable.md +++ b/docs/changelogs/v21.11.1.8636-prestable.md @@ -124,11 +124,11 @@ sidebar_label: 2022 * Recursive submodules are no longer needed for ClickHouse. [#30315](https://github.com/ClickHouse/ClickHouse/pull/30315) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Added docker image to build docs. [#30499](https://github.com/ClickHouse/ClickHouse/pull/30499) ([Ilya Yatsishin](https://github.com/qoega)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release +#### Bug Fix (user-visible misbehaviour in official stable release * Fix `ORDER BY ... WITH FILL` with set `TO` and `FROM` and no rows in result set. [#30888](https://github.com/ClickHouse/ClickHouse/pull/30888) ([Anton Popov](https://github.com/CurtizJ)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Fix hanging DDL queries on Replicated database while adding a new replica. [#29328](https://github.com/ClickHouse/ClickHouse/pull/29328) ([Kevin Michel](https://github.com/kmichel-aiven)). * Fix vertical merges of projection parts. This fixes [#29253](https://github.com/ClickHouse/ClickHouse/issues/29253) . This PR also fixes several projection merge/mutation issues introduced in https://github.com/ClickHouse/ClickHouse/pull/25165. [#29337](https://github.com/ClickHouse/ClickHouse/pull/29337) ([Amos Bird](https://github.com/amosbird)). diff --git a/docs/changelogs/v21.11.11.1-stable.md b/docs/changelogs/v21.11.11.1-stable.md index 85a8975c6e7..76cd5239cba 100644 --- a/docs/changelogs/v21.11.11.1-stable.md +++ b/docs/changelogs/v21.11.11.1-stable.md @@ -7,6 +7,6 @@ sidebar_label: 2022 ### ClickHouse release v21.11.11.1-stable FIXME as compared to v21.11.10.1-stable -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#33656](https://github.com/ClickHouse/ClickHouse/issues/33656): Fix hdfs url check that didn't allow using HA namenode address. Bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/31042. [#32976](https://github.com/ClickHouse/ClickHouse/pull/32976) ([Kruglov Pavel](https://github.com/Avogar)). diff --git a/docs/changelogs/v21.11.2.2-stable.md b/docs/changelogs/v21.11.2.2-stable.md index bf02de235e4..44938addad5 100644 --- a/docs/changelogs/v21.11.2.2-stable.md +++ b/docs/changelogs/v21.11.2.2-stable.md @@ -7,7 +7,7 @@ sidebar_label: 2022 ### ClickHouse release v21.11.2.2-stable FIXME as compared to v21.11.1.8636-prestable -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#31154](https://github.com/ClickHouse/ClickHouse/issues/31154): Skip max_partition_size_to_drop check in case of ATTACH PARTITION ... FROM and MOVE PARTITION ... [#30995](https://github.com/ClickHouse/ClickHouse/pull/30995) ([Amr Alaa](https://github.com/amralaa-MSFT)). * Backported in [#31027](https://github.com/ClickHouse/ClickHouse/issues/31027): Using `formatRow` function with not row formats led to segfault. Don't allow to use this function with such formats (because it doesn't make sense). [#31001](https://github.com/ClickHouse/ClickHouse/pull/31001) ([Kruglov Pavel](https://github.com/Avogar)). diff --git a/docs/changelogs/v21.11.3.6-stable.md b/docs/changelogs/v21.11.3.6-stable.md index e3886c4efac..f9cc64e2c2b 100644 --- a/docs/changelogs/v21.11.3.6-stable.md +++ b/docs/changelogs/v21.11.3.6-stable.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Bug Fix * Backported in [#31246](https://github.com/ClickHouse/ClickHouse/issues/31246): Memory amount was incorrectly estimated when ClickHouse is run in containers with cgroup limits. [#31157](https://github.com/ClickHouse/ClickHouse/pull/31157) ([Pavel Medvedev](https://github.com/pmed)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#31206](https://github.com/ClickHouse/ClickHouse/issues/31206): Fix possible assert in `hdfs` table function/engine, add test. [#31036](https://github.com/ClickHouse/ClickHouse/pull/31036) ([Kruglov Pavel](https://github.com/Avogar)). * Backported in [#31202](https://github.com/ClickHouse/ClickHouse/issues/31202): Fix abort in debug server and `DB::Exception: std::out_of_range: basic_string` error in release server in case of bad hdfs url by adding additional check of hdfs url structure. [#31042](https://github.com/ClickHouse/ClickHouse/pull/31042) ([Kruglov Pavel](https://github.com/Avogar)). diff --git a/docs/changelogs/v21.11.4.14-stable.md b/docs/changelogs/v21.11.4.14-stable.md index f05f43d9de1..8882832d1fe 100644 --- a/docs/changelogs/v21.11.4.14-stable.md +++ b/docs/changelogs/v21.11.4.14-stable.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Bug Fix * Backported in [#31370](https://github.com/ClickHouse/ClickHouse/issues/31370): Fix SHOW GRANTS when partial revokes are used. This PR fixes [#31138](https://github.com/ClickHouse/ClickHouse/issues/31138). [#31249](https://github.com/ClickHouse/ClickHouse/pull/31249) ([Vitaly Baranov](https://github.com/vitlibar)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#31282](https://github.com/ClickHouse/ClickHouse/issues/31282): Fix some corner cases with intersect/except. Closes [#30803](https://github.com/ClickHouse/ClickHouse/issues/30803). [#30965](https://github.com/ClickHouse/ClickHouse/pull/30965) ([Kseniia Sumarokova](https://github.com/kssenii)). * Backported in [#31237](https://github.com/ClickHouse/ClickHouse/issues/31237): Fix bug which broke select queries if they happened after dropping materialized view. Found in [#30691](https://github.com/ClickHouse/ClickHouse/issues/30691). [#30997](https://github.com/ClickHouse/ClickHouse/pull/30997) ([Kseniia Sumarokova](https://github.com/kssenii)). diff --git a/docs/changelogs/v21.11.5.33-stable.md b/docs/changelogs/v21.11.5.33-stable.md index 3780d5a2530..11e7f24dbb1 100644 --- a/docs/changelogs/v21.11.5.33-stable.md +++ b/docs/changelogs/v21.11.5.33-stable.md @@ -13,7 +13,7 @@ sidebar_label: 2022 #### Bug Fix * Backported in [#31572](https://github.com/ClickHouse/ClickHouse/issues/31572): Quota limit was not reached, but the limit was exceeded. This PR fixes [#31174](https://github.com/ClickHouse/ClickHouse/issues/31174). [#31337](https://github.com/ClickHouse/ClickHouse/pull/31337) ([sunny](https://github.com/sunny19930321)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#31517](https://github.com/ClickHouse/ClickHouse/issues/31517): Remove not like function into RPNElement. [#31169](https://github.com/ClickHouse/ClickHouse/pull/31169) ([sundyli](https://github.com/sundy-li)). * Backported in [#31551](https://github.com/ClickHouse/ClickHouse/issues/31551): Resolve `nullptr` in STS credentials provider for S3. [#31409](https://github.com/ClickHouse/ClickHouse/pull/31409) ([Vladimir Chebotarev](https://github.com/excitoon)). diff --git a/docs/changelogs/v21.11.6.7-stable.md b/docs/changelogs/v21.11.6.7-stable.md index 1f1935d1865..cddd472076a 100644 --- a/docs/changelogs/v21.11.6.7-stable.md +++ b/docs/changelogs/v21.11.6.7-stable.md @@ -11,7 +11,7 @@ sidebar_label: 2022 * Backported in [#32254](https://github.com/ClickHouse/ClickHouse/issues/32254): Fix skipping columns while writing protobuf. This PR fixes [#31160](https://github.com/ClickHouse/ClickHouse/issues/31160), see the comment [#31160](https://github.com/ClickHouse/ClickHouse/issues/31160)#issuecomment-980595318. [#31988](https://github.com/ClickHouse/ClickHouse/pull/31988) ([Vitaly Baranov](https://github.com/vitlibar)). * Backported in [#32345](https://github.com/ClickHouse/ClickHouse/issues/32345): Fix bug when remove unneeded columns in subquery. If there is an aggregation function in query without group by, do not remove if it is unneeded. [#32289](https://github.com/ClickHouse/ClickHouse/pull/32289) ([dongyifeng](https://github.com/dyf6372)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#32152](https://github.com/ClickHouse/ClickHouse/issues/32152): Fix crash when function `dictGet` with type is used for dictionary attribute when type is `Nullable`. Fixes [#30980](https://github.com/ClickHouse/ClickHouse/issues/30980). [#31800](https://github.com/ClickHouse/ClickHouse/pull/31800) ([Maksim Kita](https://github.com/kitaisreal)). * Backported in [#32298](https://github.com/ClickHouse/ClickHouse/issues/32298): Fix recursive user defined functions crash. Closes [#30856](https://github.com/ClickHouse/ClickHouse/issues/30856). [#31820](https://github.com/ClickHouse/ClickHouse/pull/31820) ([Maksim Kita](https://github.com/kitaisreal)). diff --git a/docs/changelogs/v21.11.7.9-stable.md b/docs/changelogs/v21.11.7.9-stable.md index baa6b0290a5..596d16a28ef 100644 --- a/docs/changelogs/v21.11.7.9-stable.md +++ b/docs/changelogs/v21.11.7.9-stable.md @@ -10,13 +10,13 @@ sidebar_label: 2022 #### Bug Fix * Backported in [#32691](https://github.com/ClickHouse/ClickHouse/issues/32691): Quota limit was not reached, but the limit was exceeded. This PR fixes [#31174](https://github.com/ClickHouse/ClickHouse/issues/31174). [#31656](https://github.com/ClickHouse/ClickHouse/pull/31656) ([sunny](https://github.com/sunny19930321)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#32711](https://github.com/ClickHouse/ClickHouse/issues/32711): Fix failures in queries that are trying to use skipping indices, which are not materialized yet. Fixes [#32292](https://github.com/ClickHouse/ClickHouse/issues/32292) and [#30343](https://github.com/ClickHouse/ClickHouse/issues/30343). [#32359](https://github.com/ClickHouse/ClickHouse/pull/32359) ([Anton Popov](https://github.com/CurtizJ)). * Backported in [#32568](https://github.com/ClickHouse/ClickHouse/issues/32568): Fix crash in `JoinCommon::removeColumnNullability`, close [#32458](https://github.com/ClickHouse/ClickHouse/issues/32458). [#32508](https://github.com/ClickHouse/ClickHouse/pull/32508) ([Vladimir C](https://github.com/vdimir)). * Backported in [#32732](https://github.com/ClickHouse/ClickHouse/issues/32732): Fix surprisingly bad code in function `file`. [#32640](https://github.com/ClickHouse/ClickHouse/pull/32640) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release): +#### Bug Fix (user-visible misbehaviour in official stable release): * Backported in [#32617](https://github.com/ClickHouse/ClickHouse/issues/32617): Fix possible crash (or incorrect result) in case of `LowCardinality` arguments of window function. Fixes [#31114](https://github.com/ClickHouse/ClickHouse/issues/31114). [#31888](https://github.com/ClickHouse/ClickHouse/pull/31888) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). diff --git a/docs/changelogs/v21.11.8.4-stable.md b/docs/changelogs/v21.11.8.4-stable.md index bd71374e870..28d413dd2c5 100644 --- a/docs/changelogs/v21.11.8.4-stable.md +++ b/docs/changelogs/v21.11.8.4-stable.md @@ -7,7 +7,7 @@ sidebar_label: 2022 ### ClickHouse release v21.11.8.4-stable FIXME as compared to v21.11.7.9-stable -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#32679](https://github.com/ClickHouse/ClickHouse/issues/32679): Fix unexpected projection removal when detaching parts. [#32067](https://github.com/ClickHouse/ClickHouse/pull/32067) ([Amos Bird](https://github.com/amosbird)). * Backported in [#32543](https://github.com/ClickHouse/ClickHouse/issues/32543): Some replication queue entries might hang for `temporary_directories_lifetime` (1 day by default) with `Directory tmp_merge_` or `Part ... (state Deleting) already exists, but it will be deleted soon` or similar error. It's fixed. Fixes [#29616](https://github.com/ClickHouse/ClickHouse/issues/29616). [#32201](https://github.com/ClickHouse/ClickHouse/pull/32201) ([Alexander Tokmakov](https://github.com/tavplubix)). diff --git a/docs/changelogs/v21.11.9.1-stable.md b/docs/changelogs/v21.11.9.1-stable.md index 1473a4a152c..ea36479c943 100644 --- a/docs/changelogs/v21.11.9.1-stable.md +++ b/docs/changelogs/v21.11.9.1-stable.md @@ -7,7 +7,7 @@ sidebar_label: 2022 ### ClickHouse release v21.11.9.1-stable FIXME as compared to v21.11.8.4-stable -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#33181](https://github.com/ClickHouse/ClickHouse/issues/33181): Server might fail to start if database with `MySQL` engine cannot connect to MySQL server, it's fixed. Fixes [#14441](https://github.com/ClickHouse/ClickHouse/issues/14441). [#32802](https://github.com/ClickHouse/ClickHouse/pull/32802) ([Alexander Tokmakov](https://github.com/tavplubix)). diff --git a/docs/changelogs/v21.12.1.9017-prestable.md b/docs/changelogs/v21.12.1.9017-prestable.md index e8f2ca283a4..88b8260e312 100644 --- a/docs/changelogs/v21.12.1.9017-prestable.md +++ b/docs/changelogs/v21.12.1.9017-prestable.md @@ -132,7 +132,7 @@ sidebar_label: 2022 * Build rpm and tgz packages in master and release branches workfolw. [#32048](https://github.com/ClickHouse/ClickHouse/pull/32048) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Fix broken symlink for sysroot/linux-riscv64/usr/lib. [#32071](https://github.com/ClickHouse/ClickHouse/pull/32071) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Fix some corner cases with intersect/except. Closes [#30803](https://github.com/ClickHouse/ClickHouse/issues/30803). [#30965](https://github.com/ClickHouse/ClickHouse/pull/30965) ([Kseniia Sumarokova](https://github.com/kssenii)). * Skip max_partition_size_to_drop check in case of ATTACH PARTITION ... FROM and MOVE PARTITION ... [#30995](https://github.com/ClickHouse/ClickHouse/pull/30995) ([Amr Alaa](https://github.com/amralaa-MSFT)). diff --git a/docs/changelogs/v21.12.2.17-stable.md b/docs/changelogs/v21.12.2.17-stable.md index 94cfc1b88a8..67761ce0e08 100644 --- a/docs/changelogs/v21.12.2.17-stable.md +++ b/docs/changelogs/v21.12.2.17-stable.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Bug Fix * Backported in [#32693](https://github.com/ClickHouse/ClickHouse/issues/32693): Quota limit was not reached, but the limit was exceeded. This PR fixes [#31174](https://github.com/ClickHouse/ClickHouse/issues/31174). [#31656](https://github.com/ClickHouse/ClickHouse/pull/31656) ([sunny](https://github.com/sunny19930321)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#32681](https://github.com/ClickHouse/ClickHouse/issues/32681): Fix unexpected projection removal when detaching parts. [#32067](https://github.com/ClickHouse/ClickHouse/pull/32067) ([Amos Bird](https://github.com/amosbird)). * Backported in [#32483](https://github.com/ClickHouse/ClickHouse/issues/32483): Fix 'APPLY lambda' parsing which could lead to client/server crash. [#32138](https://github.com/ClickHouse/ClickHouse/pull/32138) ([Kruglov Pavel](https://github.com/Avogar)). @@ -23,7 +23,7 @@ sidebar_label: 2022 * Backported in [#32733](https://github.com/ClickHouse/ClickHouse/issues/32733): Fix surprisingly bad code in function `file`. [#32640](https://github.com/ClickHouse/ClickHouse/pull/32640) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Backported in [#32793](https://github.com/ClickHouse/ClickHouse/issues/32793): fix crash when used fuzzBits with multiply same FixedString, Close [#32737](https://github.com/ClickHouse/ClickHouse/issues/32737). [#32755](https://github.com/ClickHouse/ClickHouse/pull/32755) ([SuperDJY](https://github.com/cmsxbc)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release): +#### Bug Fix (user-visible misbehaviour in official stable release): * Backported in [#32616](https://github.com/ClickHouse/ClickHouse/issues/32616): Fix possible crash (or incorrect result) in case of `LowCardinality` arguments of window function. Fixes [#31114](https://github.com/ClickHouse/ClickHouse/issues/31114). [#31888](https://github.com/ClickHouse/ClickHouse/pull/31888) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). diff --git a/docs/changelogs/v21.12.3.32-stable.md b/docs/changelogs/v21.12.3.32-stable.md index ea11efa46c5..c8c423a77b9 100644 --- a/docs/changelogs/v21.12.3.32-stable.md +++ b/docs/changelogs/v21.12.3.32-stable.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Bug Fix * Backported in [#33018](https://github.com/ClickHouse/ClickHouse/issues/33018): - ClickHouse Keeper handler should remove operation when response sent. [#32988](https://github.com/ClickHouse/ClickHouse/pull/32988) ([JackyWoo](https://github.com/JackyWoo)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#32890](https://github.com/ClickHouse/ClickHouse/issues/32890): Fix LOGICAL_ERROR when the target of a materialized view is a JOIN or a SET table. [#32669](https://github.com/ClickHouse/ClickHouse/pull/32669) ([Raúl Marín](https://github.com/Algunenano)). * Backported in [#33183](https://github.com/ClickHouse/ClickHouse/issues/33183): Server might fail to start if database with `MySQL` engine cannot connect to MySQL server, it's fixed. Fixes [#14441](https://github.com/ClickHouse/ClickHouse/issues/14441). [#32802](https://github.com/ClickHouse/ClickHouse/pull/32802) ([Alexander Tokmakov](https://github.com/tavplubix)). diff --git a/docs/changelogs/v21.12.4.1-stable.md b/docs/changelogs/v21.12.4.1-stable.md index bd38dbd0c59..3345f76b317 100644 --- a/docs/changelogs/v21.12.4.1-stable.md +++ b/docs/changelogs/v21.12.4.1-stable.md @@ -13,7 +13,7 @@ sidebar_label: 2022 #### Bug Fix * Backported in [#33551](https://github.com/ClickHouse/ClickHouse/issues/33551): Fix null pointer dereference in low cardinality data when deserializing LowCardinality data in the Native format. [#33021](https://github.com/ClickHouse/ClickHouse/pull/33021) ([Harry Lee](https://github.com/HarryLeeIBM)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#33537](https://github.com/ClickHouse/ClickHouse/issues/33537): Fix ORC stripe reading. [#32929](https://github.com/ClickHouse/ClickHouse/pull/32929) ([Ernest Zaslavsky](https://github.com/kreuzerkrieg)). * Backported in [#33654](https://github.com/ClickHouse/ClickHouse/issues/33654): Fix segfault in Avro that appears after the second insert into file. [#33566](https://github.com/ClickHouse/ClickHouse/pull/33566) ([Kruglov Pavel](https://github.com/Avogar)). diff --git a/docs/changelogs/v21.3.16.5-lts.md b/docs/changelogs/v21.3.16.5-lts.md index 123b27097d4..6aedeff5acb 100644 --- a/docs/changelogs/v21.3.16.5-lts.md +++ b/docs/changelogs/v21.3.16.5-lts.md @@ -25,7 +25,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#28075](https://github.com/ClickHouse/ClickHouse/issues/28075): Temporarily switched ubuntu apt repository to mirror ru.archive.ubuntu.com as default one(archive.ubuntu.com) is not responding from our CI. [#28016](https://github.com/ClickHouse/ClickHouse/pull/28016) ([Ilya Yatsishin](https://github.com/qoega)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#28181](https://github.com/ClickHouse/ClickHouse/issues/28181): Fixed possible excessive number of conditions moved from `WHERE` to `PREWHERE` (optimization controlled by settings `optimize_move_to_prewhere`). [#28139](https://github.com/ClickHouse/ClickHouse/pull/28139) ([lthaooo](https://github.com/lthaooo)). * Backported in [#28293](https://github.com/ClickHouse/ClickHouse/issues/28293): Fix inconsistent result in queries with `ORDER BY` and `Merge` tables with enabled setting `optimize_read_in_order`. [#28266](https://github.com/ClickHouse/ClickHouse/pull/28266) ([Anton Popov](https://github.com/CurtizJ)). diff --git a/docs/changelogs/v21.3.17.2-lts.md b/docs/changelogs/v21.3.17.2-lts.md index 6c288b5a0d8..9104ae7aa0a 100644 --- a/docs/changelogs/v21.3.17.2-lts.md +++ b/docs/changelogs/v21.3.17.2-lts.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Bug Fix * Backported in [#28647](https://github.com/ClickHouse/ClickHouse/issues/28647): Fix a rare bug in `DROP PART` which can lead to the error `Unexpected merged part intersects drop range`. [#27807](https://github.com/ClickHouse/ClickHouse/pull/27807) ([alesapin](https://github.com/alesapin)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#28569](https://github.com/ClickHouse/ClickHouse/issues/28569): Fix bug which can lead to error `Existing table metadata in ZooKeeper differs in sorting key expression.` after alter of `ReplicatedVersionedCollapsingMergeTree`. Fixes [#28515](https://github.com/ClickHouse/ClickHouse/issues/28515). [#28528](https://github.com/ClickHouse/ClickHouse/pull/28528) ([alesapin](https://github.com/alesapin)). * Backported in [#28857](https://github.com/ClickHouse/ClickHouse/issues/28857): Fix benign race condition in ReplicatedMergeTreeQueue. Shouldn't be visible for user, but can lead to subtle bugs. [#28734](https://github.com/ClickHouse/ClickHouse/pull/28734) ([alesapin](https://github.com/alesapin)). diff --git a/docs/changelogs/v21.3.18.4-lts.md b/docs/changelogs/v21.3.18.4-lts.md index d14dddfb1a5..33f4b86d81c 100644 --- a/docs/changelogs/v21.3.18.4-lts.md +++ b/docs/changelogs/v21.3.18.4-lts.md @@ -14,7 +14,7 @@ sidebar_label: 2022 #### Bug Fix * Backported in [#30041](https://github.com/ClickHouse/ClickHouse/issues/30041): Fix shutdown of `AccessControlManager`. Now there can't be reloading of the configuration after AccessControlManager has been destroyed. This PR fixes the flaky test [test_user_directories/test.py::test_relative_path](https://clickhouse-test-reports.s3.yandex.net/0/f0e3122507ed8bea3f177495531c7d56bcb32466/integration_tests_(thread).html). [#29951](https://github.com/ClickHouse/ClickHouse/pull/29951) ([Vitaly Baranov](https://github.com/vitlibar)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#29260](https://github.com/ClickHouse/ClickHouse/issues/29260): Fix invalid constant type conversion when nullable or lowcardinality primary key is used. [#28636](https://github.com/ClickHouse/ClickHouse/pull/28636) ([Amos Bird](https://github.com/amosbird)). * Backported in [#29026](https://github.com/ClickHouse/ClickHouse/issues/29026): Fix the number of threads used in `GLOBAL IN` subquery (it was executed in single threads since [#19414](https://github.com/ClickHouse/ClickHouse/issues/19414) bugfix). [#28997](https://github.com/ClickHouse/ClickHouse/pull/28997) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). diff --git a/docs/changelogs/v21.3.19.1-lts.md b/docs/changelogs/v21.3.19.1-lts.md index 4d4404077a5..26c36725610 100644 --- a/docs/changelogs/v21.3.19.1-lts.md +++ b/docs/changelogs/v21.3.19.1-lts.md @@ -14,11 +14,11 @@ sidebar_label: 2022 * Backported in [#31577](https://github.com/ClickHouse/ClickHouse/issues/31577): Quota limit was not reached, but the limit was exceeded. This PR fixes [#31174](https://github.com/ClickHouse/ClickHouse/issues/31174). [#31337](https://github.com/ClickHouse/ClickHouse/pull/31337) ([sunny](https://github.com/sunny19930321)). * Backported in [#32347](https://github.com/ClickHouse/ClickHouse/issues/32347): Fix bug when remove unneeded columns in subquery. If there is an aggregation function in query without group by, do not remove if it is unneeded. [#32289](https://github.com/ClickHouse/ClickHouse/pull/32289) ([dongyifeng](https://github.com/dyf6372)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release +#### Bug Fix (user-visible misbehaviour in official stable release * Backported in [#30913](https://github.com/ClickHouse/ClickHouse/issues/30913): Fix `ORDER BY ... WITH FILL` with set `TO` and `FROM` and no rows in result set. [#30888](https://github.com/ClickHouse/ClickHouse/pull/30888) ([Anton Popov](https://github.com/CurtizJ)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#30750](https://github.com/ClickHouse/ClickHouse/issues/30750): Functions for case-insensitive search in UTF8 strings like `positionCaseInsensitiveUTF8` and `countSubstringsCaseInsensitiveUTF8` might find substrings that actually does not match, it's fixed. [#30663](https://github.com/ClickHouse/ClickHouse/pull/30663) ([Alexander Tokmakov](https://github.com/tavplubix)). * Backported in [#31038](https://github.com/ClickHouse/ClickHouse/issues/31038): Using `formatRow` function with not row formats led to segfault. Don't allow to use this function with such formats (because it doesn't make sense). [#31001](https://github.com/ClickHouse/ClickHouse/pull/31001) ([Kruglov Pavel](https://github.com/Avogar)). diff --git a/docs/changelogs/v21.3.20.1-lts.md b/docs/changelogs/v21.3.20.1-lts.md index f9ce3cba78b..1b235556faf 100644 --- a/docs/changelogs/v21.3.20.1-lts.md +++ b/docs/changelogs/v21.3.20.1-lts.md @@ -11,7 +11,7 @@ sidebar_label: 2022 * Backported in [#32690](https://github.com/ClickHouse/ClickHouse/issues/32690): Quota limit was not reached, but the limit was exceeded. This PR fixes [#31174](https://github.com/ClickHouse/ClickHouse/issues/31174). [#31656](https://github.com/ClickHouse/ClickHouse/pull/31656) ([sunny](https://github.com/sunny19930321)). * Backported in [#33727](https://github.com/ClickHouse/ClickHouse/issues/33727): Fix null pointer dereference in low cardinality data when deserializing LowCardinality data in the Native format. [#33021](https://github.com/ClickHouse/ClickHouse/pull/33021) ([Harry Lee](https://github.com/HarryLeeIBM)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#32791](https://github.com/ClickHouse/ClickHouse/issues/32791): fix crash when used fuzzBits with multiply same FixedString, Close [#32737](https://github.com/ClickHouse/ClickHouse/issues/32737). [#32755](https://github.com/ClickHouse/ClickHouse/pull/32755) ([SuperDJY](https://github.com/cmsxbc)). diff --git a/docs/changelogs/v21.6.9.7-stable.md b/docs/changelogs/v21.6.9.7-stable.md index ca1edeb1722..0a989e4d6b7 100644 --- a/docs/changelogs/v21.6.9.7-stable.md +++ b/docs/changelogs/v21.6.9.7-stable.md @@ -40,7 +40,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#28030](https://github.com/ClickHouse/ClickHouse/issues/28030): Temporarily switched ubuntu apt repository to mirror ru.archive.ubuntu.com as default one(archive.ubuntu.com) is not responding from our CI. [#28016](https://github.com/ClickHouse/ClickHouse/pull/28016) ([Ilya Yatsishin](https://github.com/qoega)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#28119](https://github.com/ClickHouse/ClickHouse/issues/28119): Fix extremely rare segfaults on shutdown due to incorrect order of context/config reloader shutdown. [#28088](https://github.com/ClickHouse/ClickHouse/pull/28088) ([nvartolomei](https://github.com/nvartolomei)). * Backported in [#28179](https://github.com/ClickHouse/ClickHouse/issues/28179): Fixed possible excessive number of conditions moved from `WHERE` to `PREWHERE` (optimization controlled by settings `optimize_move_to_prewhere`). [#28139](https://github.com/ClickHouse/ClickHouse/pull/28139) ([lthaooo](https://github.com/lthaooo)). diff --git a/docs/changelogs/v21.7.10.4-stable.md b/docs/changelogs/v21.7.10.4-stable.md index daa063ebf5a..238dfd651eb 100644 --- a/docs/changelogs/v21.7.10.4-stable.md +++ b/docs/changelogs/v21.7.10.4-stable.md @@ -14,7 +14,7 @@ sidebar_label: 2022 * Backported in [#27925](https://github.com/ClickHouse/ClickHouse/issues/27925): Fix PostgreSQL-style cast (`::` operator) with negative numbers. [#27876](https://github.com/ClickHouse/ClickHouse/pull/27876) ([Anton Popov](https://github.com/CurtizJ)). * Backported in [#28752](https://github.com/ClickHouse/ClickHouse/issues/28752): Fix transformation of disjunctions chain to `IN` (controlled by settings `optimize_min_equality_disjunction_chain_length`) in distributed queries with settings `legacy_column_name_of_tuple_literal = 0`. [#28658](https://github.com/ClickHouse/ClickHouse/pull/28658) ([Anton Popov](https://github.com/CurtizJ)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#28509](https://github.com/ClickHouse/ClickHouse/issues/28509): Fixed possible ZooKeeper watches leak on background processing of distributed DDL queue. Closes [#26036](https://github.com/ClickHouse/ClickHouse/issues/26036). [#28446](https://github.com/ClickHouse/ClickHouse/pull/28446) ([Alexander Tokmakov](https://github.com/tavplubix)). * Backported in [#28570](https://github.com/ClickHouse/ClickHouse/issues/28570): Fix bug which can lead to error `Existing table metadata in ZooKeeper differs in sorting key expression.` after alter of `ReplicatedVersionedCollapsingMergeTree`. Fixes [#28515](https://github.com/ClickHouse/ClickHouse/issues/28515). [#28528](https://github.com/ClickHouse/ClickHouse/pull/28528) ([alesapin](https://github.com/alesapin)). diff --git a/docs/changelogs/v21.7.11.3-stable.md b/docs/changelogs/v21.7.11.3-stable.md index b3d1c9a44fd..8ccc31657de 100644 --- a/docs/changelogs/v21.7.11.3-stable.md +++ b/docs/changelogs/v21.7.11.3-stable.md @@ -7,7 +7,7 @@ sidebar_label: 2022 ### ClickHouse release v21.7.11.3-stable FIXME as compared to v21.7.10.4-stable -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#29024](https://github.com/ClickHouse/ClickHouse/issues/29024): Fix the number of threads used in `GLOBAL IN` subquery (it was executed in single threads since [#19414](https://github.com/ClickHouse/ClickHouse/issues/19414) bugfix). [#28997](https://github.com/ClickHouse/ClickHouse/pull/28997) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Backported in [#29195](https://github.com/ClickHouse/ClickHouse/issues/29195): Fix segfault while inserting into column with type LowCardinality(Nullable) in Avro input format. [#29132](https://github.com/ClickHouse/ClickHouse/pull/29132) ([Kruglov Pavel](https://github.com/Avogar)). diff --git a/docs/changelogs/v21.7.9.7-stable.md b/docs/changelogs/v21.7.9.7-stable.md index ac985f7af37..7aaab54af6b 100644 --- a/docs/changelogs/v21.7.9.7-stable.md +++ b/docs/changelogs/v21.7.9.7-stable.md @@ -23,7 +23,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#28032](https://github.com/ClickHouse/ClickHouse/issues/28032): Temporarily switched ubuntu apt repository to mirror ru.archive.ubuntu.com as default one(archive.ubuntu.com) is not responding from our CI. [#28016](https://github.com/ClickHouse/ClickHouse/pull/28016) ([Ilya Yatsishin](https://github.com/qoega)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#28116](https://github.com/ClickHouse/ClickHouse/issues/28116): Fix extremely rare segfaults on shutdown due to incorrect order of context/config reloader shutdown. [#28088](https://github.com/ClickHouse/ClickHouse/pull/28088) ([nvartolomei](https://github.com/nvartolomei)). * Backported in [#28183](https://github.com/ClickHouse/ClickHouse/issues/28183): Fixed possible excessive number of conditions moved from `WHERE` to `PREWHERE` (optimization controlled by settings `optimize_move_to_prewhere`). [#28139](https://github.com/ClickHouse/ClickHouse/pull/28139) ([lthaooo](https://github.com/lthaooo)). diff --git a/docs/changelogs/v21.8.10.19-lts.md b/docs/changelogs/v21.8.10.19-lts.md index 56c682ddbd2..5873f2b2ff9 100644 --- a/docs/changelogs/v21.8.10.19-lts.md +++ b/docs/changelogs/v21.8.10.19-lts.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Improvement * Backported in [#30452](https://github.com/ClickHouse/ClickHouse/issues/30452): Allow symlinks to files in user_files directory for file table function. [#30309](https://github.com/ClickHouse/ClickHouse/pull/30309) ([Kseniia Sumarokova](https://github.com/kssenii)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#29724](https://github.com/ClickHouse/ClickHouse/issues/29724): Fix null deference for `GROUP BY WITH TOTALS HAVING` (when the column from `HAVING` wasn't selected). [#29553](https://github.com/ClickHouse/ClickHouse/pull/29553) ([Azat Khuzhin](https://github.com/azat)). * Backported in [#30233](https://github.com/ClickHouse/ClickHouse/issues/30233): Fix INSERT SELECT incorrectly fills MATERIALIZED column based of Nullable column. [#30189](https://github.com/ClickHouse/ClickHouse/pull/30189) ([Azat Khuzhin](https://github.com/azat)). diff --git a/docs/changelogs/v21.8.11.4-lts.md b/docs/changelogs/v21.8.11.4-lts.md index d88d191bae2..81bbea4ee21 100644 --- a/docs/changelogs/v21.8.11.4-lts.md +++ b/docs/changelogs/v21.8.11.4-lts.md @@ -16,11 +16,11 @@ sidebar_label: 2022 #### Bug Fix * Backported in [#31368](https://github.com/ClickHouse/ClickHouse/issues/31368): Fix SHOW GRANTS when partial revokes are used. This PR fixes [#31138](https://github.com/ClickHouse/ClickHouse/issues/31138). [#31249](https://github.com/ClickHouse/ClickHouse/pull/31249) ([Vitaly Baranov](https://github.com/vitlibar)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release +#### Bug Fix (user-visible misbehaviour in official stable release * Backported in [#30914](https://github.com/ClickHouse/ClickHouse/issues/30914): Fix `ORDER BY ... WITH FILL` with set `TO` and `FROM` and no rows in result set. [#30888](https://github.com/ClickHouse/ClickHouse/pull/30888) ([Anton Popov](https://github.com/CurtizJ)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#28756](https://github.com/ClickHouse/ClickHouse/issues/28756): Fix NOT-IN index optimization when not all key columns are used. This fixes [#28120](https://github.com/ClickHouse/ClickHouse/issues/28120). [#28315](https://github.com/ClickHouse/ClickHouse/pull/28315) ([Amos Bird](https://github.com/amosbird)). * Backported in [#30825](https://github.com/ClickHouse/ClickHouse/issues/30825): Fix "Column is not under aggregate function and not in GROUP BY" with PREWHERE (Fixes: [#28461](https://github.com/ClickHouse/ClickHouse/issues/28461)). [#28502](https://github.com/ClickHouse/ClickHouse/pull/28502) ([Azat Khuzhin](https://github.com/azat)). diff --git a/docs/changelogs/v21.8.12.29-lts.md b/docs/changelogs/v21.8.12.29-lts.md index bd1f0c7fe60..8b68a6a3af8 100644 --- a/docs/changelogs/v21.8.12.29-lts.md +++ b/docs/changelogs/v21.8.12.29-lts.md @@ -13,7 +13,7 @@ sidebar_label: 2022 #### Bug Fix * Backported in [#31575](https://github.com/ClickHouse/ClickHouse/issues/31575): Quota limit was not reached, but the limit was exceeded. This PR fixes [#31174](https://github.com/ClickHouse/ClickHouse/issues/31174). [#31337](https://github.com/ClickHouse/ClickHouse/pull/31337) ([sunny](https://github.com/sunny19930321)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#31204](https://github.com/ClickHouse/ClickHouse/issues/31204): Fix abort in debug server and `DB::Exception: std::out_of_range: basic_string` error in release server in case of bad hdfs url by adding additional check of hdfs url structure. [#31042](https://github.com/ClickHouse/ClickHouse/pull/31042) ([Kruglov Pavel](https://github.com/Avogar)). * Backported in [#31253](https://github.com/ClickHouse/ClickHouse/issues/31253): Fix bug in Keeper which can lead to inability to start when some coordination logs was lost and we have more fresh snapshot than our latest log. [#31150](https://github.com/ClickHouse/ClickHouse/pull/31150) ([alesapin](https://github.com/alesapin)). diff --git a/docs/changelogs/v21.8.13.6-lts.md b/docs/changelogs/v21.8.13.6-lts.md index 63ac956c3d5..205628c6330 100644 --- a/docs/changelogs/v21.8.13.6-lts.md +++ b/docs/changelogs/v21.8.13.6-lts.md @@ -11,7 +11,7 @@ sidebar_label: 2022 * Backported in [#32688](https://github.com/ClickHouse/ClickHouse/issues/32688): Quota limit was not reached, but the limit was exceeded. This PR fixes [#31174](https://github.com/ClickHouse/ClickHouse/issues/31174). [#31656](https://github.com/ClickHouse/ClickHouse/pull/31656) ([sunny](https://github.com/sunny19930321)). * Backported in [#32343](https://github.com/ClickHouse/ClickHouse/issues/32343): Fix bug when remove unneeded columns in subquery. If there is an aggregation function in query without group by, do not remove if it is unneeded. [#32289](https://github.com/ClickHouse/ClickHouse/pull/32289) ([dongyifeng](https://github.com/dyf6372)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#32108](https://github.com/ClickHouse/ClickHouse/issues/32108): Fix crash with empty result on odbc query. Closes [#31465](https://github.com/ClickHouse/ClickHouse/issues/31465). [#31766](https://github.com/ClickHouse/ClickHouse/pull/31766) ([Kseniia Sumarokova](https://github.com/kssenii)). * Backported in [#32150](https://github.com/ClickHouse/ClickHouse/issues/32150): Fix crash when function `dictGet` with type is used for dictionary attribute when type is `Nullable`. Fixes [#30980](https://github.com/ClickHouse/ClickHouse/issues/30980). [#31800](https://github.com/ClickHouse/ClickHouse/pull/31800) ([Maksim Kita](https://github.com/kitaisreal)). @@ -26,7 +26,7 @@ sidebar_label: 2022 * Backported in [#33048](https://github.com/ClickHouse/ClickHouse/issues/33048): Fix possible exception at RabbitMQ storage startup by delaying channel creation. [#32584](https://github.com/ClickHouse/ClickHouse/pull/32584) ([Kseniia Sumarokova](https://github.com/kssenii)). * Backported in [#32795](https://github.com/ClickHouse/ClickHouse/issues/32795): fix crash when used fuzzBits with multiply same FixedString, Close [#32737](https://github.com/ClickHouse/ClickHouse/issues/32737). [#32755](https://github.com/ClickHouse/ClickHouse/pull/32755) ([SuperDJY](https://github.com/cmsxbc)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release): +#### Bug Fix (user-visible misbehaviour in official stable release): * Backported in [#32659](https://github.com/ClickHouse/ClickHouse/issues/32659): Fix possible crash (or incorrect result) in case of `LowCardinality` arguments of window function. Fixes [#31114](https://github.com/ClickHouse/ClickHouse/issues/31114). [#31888](https://github.com/ClickHouse/ClickHouse/pull/31888) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). diff --git a/docs/changelogs/v21.8.14.5-lts.md b/docs/changelogs/v21.8.14.5-lts.md index 1012d9c5784..75d966ec9e7 100644 --- a/docs/changelogs/v21.8.14.5-lts.md +++ b/docs/changelogs/v21.8.14.5-lts.md @@ -7,7 +7,7 @@ sidebar_label: 2022 ### ClickHouse release v21.8.14.5-lts FIXME as compared to v21.8.13.6-lts -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#33184](https://github.com/ClickHouse/ClickHouse/issues/33184): Server might fail to start if database with `MySQL` engine cannot connect to MySQL server, it's fixed. Fixes [#14441](https://github.com/ClickHouse/ClickHouse/issues/14441). [#32802](https://github.com/ClickHouse/ClickHouse/pull/32802) ([Alexander Tokmakov](https://github.com/tavplubix)). * Backported in [#33659](https://github.com/ClickHouse/ClickHouse/issues/33659): Fix hdfs url check that didn't allow using HA namenode address. Bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/31042. [#32976](https://github.com/ClickHouse/ClickHouse/pull/32976) ([Kruglov Pavel](https://github.com/Avogar)). diff --git a/docs/changelogs/v21.8.15.7-lts.md b/docs/changelogs/v21.8.15.7-lts.md index eb6bf39d7be..1522c28016a 100644 --- a/docs/changelogs/v21.8.15.7-lts.md +++ b/docs/changelogs/v21.8.15.7-lts.md @@ -7,7 +7,7 @@ sidebar_label: 2022 ### ClickHouse release v21.8.15.7-lts FIXME as compared to v21.8.14.5-lts -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#34121](https://github.com/ClickHouse/ClickHouse/issues/34121): Fix usage of functions `array` and `tuple` with literal arguments in distributed queries. Previously it could lead to `Not found columns` exception. [#33938](https://github.com/ClickHouse/ClickHouse/pull/33938) ([Anton Popov](https://github.com/CurtizJ)). * Backported in [#34097](https://github.com/ClickHouse/ClickHouse/issues/34097): Fix segfault while parsing ORC file with corrupted footer. Closes [#33797](https://github.com/ClickHouse/ClickHouse/issues/33797). [#33984](https://github.com/ClickHouse/ClickHouse/pull/33984) ([Kruglov Pavel](https://github.com/Avogar)). diff --git a/docs/changelogs/v21.8.5.7-lts.md b/docs/changelogs/v21.8.5.7-lts.md index 00c6c6e46a7..fa459e093f7 100644 --- a/docs/changelogs/v21.8.5.7-lts.md +++ b/docs/changelogs/v21.8.5.7-lts.md @@ -25,7 +25,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#28031](https://github.com/ClickHouse/ClickHouse/issues/28031): Temporarily switched ubuntu apt repository to mirror ru.archive.ubuntu.com as default one(archive.ubuntu.com) is not responding from our CI. [#28016](https://github.com/ClickHouse/ClickHouse/pull/28016) ([Ilya Yatsishin](https://github.com/qoega)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#27974](https://github.com/ClickHouse/ClickHouse/issues/27974): Fix handling null value with type of Nullable(String) in function JSONExtract. This fixes [#27929](https://github.com/ClickHouse/ClickHouse/issues/27929) and [#27930](https://github.com/ClickHouse/ClickHouse/issues/27930) . This was introduced in https://github.com/ClickHouse/ClickHouse/pull/25452 . [#27939](https://github.com/ClickHouse/ClickHouse/pull/27939) ([Amos Bird](https://github.com/amosbird)). * Backported in [#28117](https://github.com/ClickHouse/ClickHouse/issues/28117): Fix extremely rare segfaults on shutdown due to incorrect order of context/config reloader shutdown. [#28088](https://github.com/ClickHouse/ClickHouse/pull/28088) ([nvartolomei](https://github.com/nvartolomei)). diff --git a/docs/changelogs/v21.8.6.15-lts.md b/docs/changelogs/v21.8.6.15-lts.md index dc8ae4ec9b5..aa51dc1ac1e 100644 --- a/docs/changelogs/v21.8.6.15-lts.md +++ b/docs/changelogs/v21.8.6.15-lts.md @@ -16,7 +16,7 @@ sidebar_label: 2022 * Backported in [#27923](https://github.com/ClickHouse/ClickHouse/issues/27923): Fix PostgreSQL-style cast (`::` operator) with negative numbers. [#27876](https://github.com/ClickHouse/ClickHouse/pull/27876) ([Anton Popov](https://github.com/CurtizJ)). * Backported in [#28753](https://github.com/ClickHouse/ClickHouse/issues/28753): Fix transformation of disjunctions chain to `IN` (controlled by settings `optimize_min_equality_disjunction_chain_length`) in distributed queries with settings `legacy_column_name_of_tuple_literal = 0`. [#28658](https://github.com/ClickHouse/ClickHouse/pull/28658) ([Anton Popov](https://github.com/CurtizJ)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#28644](https://github.com/ClickHouse/ClickHouse/issues/28644): Fix rare case when changes of `clickhouse-keeper` settings may lead to lost logs and server hung. [#28360](https://github.com/ClickHouse/ClickHouse/pull/28360) ([alesapin](https://github.com/alesapin)). * Backported in [#28508](https://github.com/ClickHouse/ClickHouse/issues/28508): Fix lack of quotes for table names in MaterializedPostgreSQL engine. Closes [#28316](https://github.com/ClickHouse/ClickHouse/issues/28316). [#28433](https://github.com/ClickHouse/ClickHouse/pull/28433) ([Kseniia Sumarokova](https://github.com/kssenii)). diff --git a/docs/changelogs/v21.8.7.22-lts.md b/docs/changelogs/v21.8.7.22-lts.md index b6c5b70e096..7a751be4132 100644 --- a/docs/changelogs/v21.8.7.22-lts.md +++ b/docs/changelogs/v21.8.7.22-lts.md @@ -7,7 +7,7 @@ sidebar_label: 2022 ### ClickHouse release v21.8.7.22-lts FIXME as compared to v21.8.6.15-lts -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#29121](https://github.com/ClickHouse/ClickHouse/issues/29121): Better check for connection usability and also catch any exception in RabbitMQ shutdown just in case. [#28797](https://github.com/ClickHouse/ClickHouse/pull/28797) ([Kseniia Sumarokova](https://github.com/kssenii)). * Backported in [#29027](https://github.com/ClickHouse/ClickHouse/issues/29027): Fix the number of threads used in `GLOBAL IN` subquery (it was executed in single threads since [#19414](https://github.com/ClickHouse/ClickHouse/issues/19414) bugfix). [#28997](https://github.com/ClickHouse/ClickHouse/pull/28997) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). diff --git a/docs/changelogs/v21.8.8.29-lts.md b/docs/changelogs/v21.8.8.29-lts.md index 160d30a6aa9..e988c3c6801 100644 --- a/docs/changelogs/v21.8.8.29-lts.md +++ b/docs/changelogs/v21.8.8.29-lts.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Bug Fix * Backported in [#29128](https://github.com/ClickHouse/ClickHouse/issues/29128): Fix bug in `clickhouse-keeper-converter` which can lead to incorrect ZooKeeper log deserialization. [#29071](https://github.com/ClickHouse/ClickHouse/pull/29071) ([小路](https://github.com/nicelulu)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#29262](https://github.com/ClickHouse/ClickHouse/issues/29262): Fix invalid constant type conversion when nullable or lowcardinality primary key is used. [#28636](https://github.com/ClickHouse/ClickHouse/pull/28636) ([Amos Bird](https://github.com/amosbird)). * Backported in [#29106](https://github.com/ClickHouse/ClickHouse/issues/29106): Fix waiting for mutation with `mutations_sync=2`. [#28889](https://github.com/ClickHouse/ClickHouse/pull/28889) ([Azat Khuzhin](https://github.com/azat)). diff --git a/docs/changelogs/v21.8.9.13-lts.md b/docs/changelogs/v21.8.9.13-lts.md index a48ca30080e..71919c48c47 100644 --- a/docs/changelogs/v21.8.9.13-lts.md +++ b/docs/changelogs/v21.8.9.13-lts.md @@ -14,7 +14,7 @@ sidebar_label: 2022 * Backported in [#29817](https://github.com/ClickHouse/ClickHouse/issues/29817): Allow using a materialized column as the sharding key in a distributed table even if `insert_allow_materialized_columns=0`:. [#28637](https://github.com/ClickHouse/ClickHouse/pull/28637) ([Vitaly Baranov](https://github.com/vitlibar)). * Backported in [#29973](https://github.com/ClickHouse/ClickHouse/issues/29973): Fix shutdown of `AccessControlManager`. Now there can't be reloading of the configuration after AccessControlManager has been destroyed. This PR fixes the flaky test [test_user_directories/test.py::test_relative_path](https://clickhouse-test-reports.s3.yandex.net/0/f0e3122507ed8bea3f177495531c7d56bcb32466/integration_tests_(thread).html). [#29951](https://github.com/ClickHouse/ClickHouse/pull/29951) ([Vitaly Baranov](https://github.com/vitlibar)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#29676](https://github.com/ClickHouse/ClickHouse/issues/29676): Fix vertical merges of projection parts. This fixes [#29253](https://github.com/ClickHouse/ClickHouse/issues/29253) . This PR also fixes several projection merge/mutation issues introduced in https://github.com/ClickHouse/ClickHouse/pull/25165. [#29337](https://github.com/ClickHouse/ClickHouse/pull/29337) ([Amos Bird](https://github.com/amosbird)). * Backported in [#29538](https://github.com/ClickHouse/ClickHouse/issues/29538): Fix possible `Block structure mismatch` for subqueries with pushed-down `HAVING` predicate. Fixes [#29010](https://github.com/ClickHouse/ClickHouse/issues/29010). [#29475](https://github.com/ClickHouse/ClickHouse/pull/29475) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). diff --git a/docs/changelogs/v21.9.2.17-stable.md b/docs/changelogs/v21.9.2.17-stable.md index 39e3f627f4a..08d208ec97b 100644 --- a/docs/changelogs/v21.9.2.17-stable.md +++ b/docs/changelogs/v21.9.2.17-stable.md @@ -25,7 +25,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#28029](https://github.com/ClickHouse/ClickHouse/issues/28029): Temporarily switched ubuntu apt repository to mirror ru.archive.ubuntu.com as default one(archive.ubuntu.com) is not responding from our CI. [#28016](https://github.com/ClickHouse/ClickHouse/pull/28016) ([Ilya Yatsishin](https://github.com/qoega)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#27973](https://github.com/ClickHouse/ClickHouse/issues/27973): Fix handling null value with type of Nullable(String) in function JSONExtract. This fixes [#27929](https://github.com/ClickHouse/ClickHouse/issues/27929) and [#27930](https://github.com/ClickHouse/ClickHouse/issues/27930) . This was introduced in https://github.com/ClickHouse/ClickHouse/pull/25452 . [#27939](https://github.com/ClickHouse/ClickHouse/pull/27939) ([Amos Bird](https://github.com/amosbird)). * Backported in [#28118](https://github.com/ClickHouse/ClickHouse/issues/28118): Fix extremely rare segfaults on shutdown due to incorrect order of context/config reloader shutdown. [#28088](https://github.com/ClickHouse/ClickHouse/pull/28088) ([nvartolomei](https://github.com/nvartolomei)). diff --git a/docs/changelogs/v21.9.3.30-stable.md b/docs/changelogs/v21.9.3.30-stable.md index ee2dd24277d..28375c5588e 100644 --- a/docs/changelogs/v21.9.3.30-stable.md +++ b/docs/changelogs/v21.9.3.30-stable.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Improvement * Backported in [#28897](https://github.com/ClickHouse/ClickHouse/issues/28897): Use real tmp file instead of predefined "rows_sources" for vertical merges. This avoids generating garbage directories in tmp disks. [#28299](https://github.com/ClickHouse/ClickHouse/pull/28299) ([Amos Bird](https://github.com/amosbird)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#28815](https://github.com/ClickHouse/ClickHouse/issues/28815): Fix possible crash for `SELECT` with partially created aggregate projection in case of exception. [#28700](https://github.com/ClickHouse/ClickHouse/pull/28700) ([Amos Bird](https://github.com/amosbird)). * Backported in [#28789](https://github.com/ClickHouse/ClickHouse/issues/28789): Fix benign race condition in ReplicatedMergeTreeQueue. Shouldn't be visible for user, but can lead to subtle bugs. [#28734](https://github.com/ClickHouse/ClickHouse/pull/28734) ([alesapin](https://github.com/alesapin)). diff --git a/docs/changelogs/v21.9.4.35-stable.md b/docs/changelogs/v21.9.4.35-stable.md index 5a556df1b3a..0b300574559 100644 --- a/docs/changelogs/v21.9.4.35-stable.md +++ b/docs/changelogs/v21.9.4.35-stable.md @@ -7,7 +7,7 @@ sidebar_label: 2022 ### ClickHouse release v21.9.4.35-stable FIXME as compared to v21.9.3.30-stable -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#29191](https://github.com/ClickHouse/ClickHouse/issues/29191): Fix segfault while inserting into column with type LowCardinality(Nullable) in Avro input format. [#29132](https://github.com/ClickHouse/ClickHouse/pull/29132) ([Kruglov Pavel](https://github.com/Avogar)). diff --git a/docs/changelogs/v21.9.5.16-stable.md b/docs/changelogs/v21.9.5.16-stable.md index e681322eb45..895e882d257 100644 --- a/docs/changelogs/v21.9.5.16-stable.md +++ b/docs/changelogs/v21.9.5.16-stable.md @@ -17,7 +17,7 @@ sidebar_label: 2022 * Backported in [#29972](https://github.com/ClickHouse/ClickHouse/issues/29972): Fix shutdown of `AccessControlManager`. Now there can't be reloading of the configuration after AccessControlManager has been destroyed. This PR fixes the flaky test [test_user_directories/test.py::test_relative_path](https://clickhouse-test-reports.s3.yandex.net/0/f0e3122507ed8bea3f177495531c7d56bcb32466/integration_tests_(thread).html). [#29951](https://github.com/ClickHouse/ClickHouse/pull/29951) ([Vitaly Baranov](https://github.com/vitlibar)). * Backported in [#30052](https://github.com/ClickHouse/ClickHouse/issues/30052): Fix releasing query ID and session ID at the end of query processing while handing gRPC call. This PR fixes flaky test [test_grpc_protocol/test.py::test_session](https://clickhouse-test-reports.s3.yandex.net/0/1ac03811a2df9717fa7c633d1af03def821d24b6/integration_tests_(memory).html). [#29954](https://github.com/ClickHouse/ClickHouse/pull/29954) ([Vitaly Baranov](https://github.com/vitlibar)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#29055](https://github.com/ClickHouse/ClickHouse/issues/29055): Fix invalid constant type conversion when nullable or lowcardinality primary key is used. [#28636](https://github.com/ClickHouse/ClickHouse/pull/28636) ([Amos Bird](https://github.com/amosbird)). * Backported in [#29107](https://github.com/ClickHouse/ClickHouse/issues/29107): Fix waiting for mutation with `mutations_sync=2`. [#28889](https://github.com/ClickHouse/ClickHouse/pull/28889) ([Azat Khuzhin](https://github.com/azat)). diff --git a/docs/changelogs/v21.9.6.24-stable.md b/docs/changelogs/v21.9.6.24-stable.md index 2407c031873..890dc8d5d02 100644 --- a/docs/changelogs/v21.9.6.24-stable.md +++ b/docs/changelogs/v21.9.6.24-stable.md @@ -21,11 +21,11 @@ sidebar_label: 2022 * Backported in [#31371](https://github.com/ClickHouse/ClickHouse/issues/31371): Fix SHOW GRANTS when partial revokes are used. This PR fixes [#31138](https://github.com/ClickHouse/ClickHouse/issues/31138). [#31249](https://github.com/ClickHouse/ClickHouse/pull/31249) ([Vitaly Baranov](https://github.com/vitlibar)). * Backported in [#31576](https://github.com/ClickHouse/ClickHouse/issues/31576): Quota limit was not reached, but the limit was exceeded. This PR fixes [#31174](https://github.com/ClickHouse/ClickHouse/issues/31174). [#31337](https://github.com/ClickHouse/ClickHouse/pull/31337) ([sunny](https://github.com/sunny19930321)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release +#### Bug Fix (user-visible misbehaviour in official stable release * Backported in [#30916](https://github.com/ClickHouse/ClickHouse/issues/30916): Fix `ORDER BY ... WITH FILL` with set `TO` and `FROM` and no rows in result set. [#30888](https://github.com/ClickHouse/ClickHouse/pull/30888) ([Anton Popov](https://github.com/CurtizJ)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#30823](https://github.com/ClickHouse/ClickHouse/issues/30823): Fix "Column is not under aggregate function and not in GROUP BY" with PREWHERE (Fixes: [#28461](https://github.com/ClickHouse/ClickHouse/issues/28461)). [#28502](https://github.com/ClickHouse/ClickHouse/pull/28502) ([Azat Khuzhin](https://github.com/azat)). * Backported in [#30609](https://github.com/ClickHouse/ClickHouse/issues/30609): Fix bad optimizations of ORDER BY if it contains WITH FILL. This closes [#28908](https://github.com/ClickHouse/ClickHouse/issues/28908). This closes [#26049](https://github.com/ClickHouse/ClickHouse/issues/26049). [#28910](https://github.com/ClickHouse/ClickHouse/pull/28910) ([Alexey Milovidov](https://github.com/alexey-milovidov)). diff --git a/docs/changelogs/v22.1.1.2542-prestable.md b/docs/changelogs/v22.1.1.2542-prestable.md index 8d7bb015db6..cacd13c1e12 100644 --- a/docs/changelogs/v22.1.1.2542-prestable.md +++ b/docs/changelogs/v22.1.1.2542-prestable.md @@ -118,7 +118,7 @@ sidebar_label: 2022 * Remove editing /etc/hosts from Dockerfile. [#33635](https://github.com/ClickHouse/ClickHouse/pull/33635) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Properly separate thrift-cmake from arrow-cmake after https://github.com/ClickHouse/ClickHouse/pull/31104 . cc @taiyang-li. [#33661](https://github.com/ClickHouse/ClickHouse/pull/33661) ([Amos Bird](https://github.com/amosbird)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Fixed CAST from String to IPv4 or IPv6 and back. Fixed error message in case of failed conversion. [#29224](https://github.com/ClickHouse/ClickHouse/pull/29224) ([Dmitry Novik](https://github.com/novikd)). * Fix base64Encode adding trailing bytes on small strings. [#31797](https://github.com/ClickHouse/ClickHouse/pull/31797) ([Kevin Michel](https://github.com/kmichel-aiven)). @@ -185,7 +185,7 @@ sidebar_label: 2022 * Fix segfault in Avro that appears after the second insert into file. [#33566](https://github.com/ClickHouse/ClickHouse/pull/33566) ([Kruglov Pavel](https://github.com/Avogar)). * Fix wrong database for JOIN w/o explicit database in distributed queries (Fixes: [#10471](https://github.com/ClickHouse/ClickHouse/issues/10471)). [#33611](https://github.com/ClickHouse/ClickHouse/pull/33611) ([Azat Khuzhin](https://github.com/azat)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release): +#### Bug Fix (user-visible misbehaviour in official stable release): * Fix possible crash (or incorrect result) in case of `LowCardinality` arguments of window function. Fixes [#31114](https://github.com/ClickHouse/ClickHouse/issues/31114). [#31888](https://github.com/ClickHouse/ClickHouse/pull/31888) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). diff --git a/docs/changelogs/v22.1.3.7-stable.md b/docs/changelogs/v22.1.3.7-stable.md index a92a82be290..fd8787f0e75 100644 --- a/docs/changelogs/v22.1.3.7-stable.md +++ b/docs/changelogs/v22.1.3.7-stable.md @@ -10,6 +10,6 @@ sidebar_label: 2022 #### Improvement * Backported in [#33793](https://github.com/ClickHouse/ClickHouse/issues/33793): Create parent directories in DiskS3::restoreFileOperations method. [#33730](https://github.com/ClickHouse/ClickHouse/pull/33730) ([ianton-ru](https://github.com/ianton-ru)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#33898](https://github.com/ClickHouse/ClickHouse/issues/33898): Fix usage of sparse columns (which can be enabled by experimental setting `ratio_of_defaults_for_sparse_serialization`). [#33849](https://github.com/ClickHouse/ClickHouse/pull/33849) ([Anton Popov](https://github.com/CurtizJ)). diff --git a/docs/changelogs/v22.1.4.30-stable.md b/docs/changelogs/v22.1.4.30-stable.md index c4286d7b64d..b0437382a46 100644 --- a/docs/changelogs/v22.1.4.30-stable.md +++ b/docs/changelogs/v22.1.4.30-stable.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backport CI checks to 22.1 release branch. [#34897](https://github.com/ClickHouse/ClickHouse/pull/34897) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#34119](https://github.com/ClickHouse/ClickHouse/issues/34119): Fix usage of functions `array` and `tuple` with literal arguments in distributed queries. Previously it could lead to `Not found columns` exception. [#33938](https://github.com/ClickHouse/ClickHouse/pull/33938) ([Anton Popov](https://github.com/CurtizJ)). * Backported in [#34124](https://github.com/ClickHouse/ClickHouse/issues/34124): Fix crash while reading of nested tuples. Fixes [#33838](https://github.com/ClickHouse/ClickHouse/issues/33838). [#33956](https://github.com/ClickHouse/ClickHouse/pull/33956) ([Anton Popov](https://github.com/CurtizJ)). diff --git a/docs/changelogs/v22.10.1.1877-stable.md b/docs/changelogs/v22.10.1.1877-stable.md index 5b573a3faa4..23c58447c99 100644 --- a/docs/changelogs/v22.10.1.1877-stable.md +++ b/docs/changelogs/v22.10.1.1877-stable.md @@ -105,7 +105,7 @@ sidebar_label: 2022 * Update tzdata to 2022e to support the new timezone changes. Palestine transitions are now Saturdays at 02:00. Simplify three Ukraine zones into one. Jordan and Syria switch from +02/+03 with DST to year-round +03. (https://data.iana.org/time-zones/tzdb/NEWS). This closes [#42252](https://github.com/ClickHouse/ClickHouse/issues/42252). [#42327](https://github.com/ClickHouse/ClickHouse/pull/42327) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Fix power8 support. [#42462](https://github.com/ClickHouse/ClickHouse/pull/42462) ([Boris Kuschel](https://github.com/bkuschel)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Several fixes for DiskWeb. [#41652](https://github.com/ClickHouse/ClickHouse/pull/41652) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fixes issue when docker run will fail if "https_port" is not present in config. [#41693](https://github.com/ClickHouse/ClickHouse/pull/41693) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). @@ -147,7 +147,7 @@ sidebar_label: 2022 * This closes [#42453](https://github.com/ClickHouse/ClickHouse/issues/42453). [#42573](https://github.com/ClickHouse/ClickHouse/pull/42573) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Fix function `arrayElement` with type `Map` with `Nullable` values and `Nullable` index. [#42623](https://github.com/ClickHouse/ClickHouse/pull/42623) ([Anton Popov](https://github.com/CurtizJ)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Fix unexpected table loading error when partition key contains alias function names during server upgrade. [#36379](https://github.com/ClickHouse/ClickHouse/pull/36379) ([Amos Bird](https://github.com/amosbird)). diff --git a/docs/changelogs/v22.10.2.11-stable.md b/docs/changelogs/v22.10.2.11-stable.md index 4e3c382f5a7..196d3fbde80 100644 --- a/docs/changelogs/v22.10.2.11-stable.md +++ b/docs/changelogs/v22.10.2.11-stable.md @@ -7,7 +7,7 @@ sidebar_label: 2022 ### ClickHouse release v22.10.2.11-stable (d2bfcaba002) FIXME as compared to v22.10.1.1877-stable (98ab5a3c189) -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#42750](https://github.com/ClickHouse/ClickHouse/issues/42750): A segmentation fault related to DNS & c-ares has been reported. The below error ocurred in multiple threads: ``` 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008088 [ 356 ] {} BaseDaemon: ######################################## 2022-09-28 15:41:19.008,"2022.09.28 15:41:19.008147 [ 356 ] {} BaseDaemon: (version 22.8.5.29 (official build), build id: 92504ACA0B8E2267) (from thread 353) (no query) Received signal Segmentation fault (11)" 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008196 [ 356 ] {} BaseDaemon: Address: 0xf Access: write. Address not mapped to object. 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008216 [ 356 ] {} BaseDaemon: Stack trace: 0x188f8212 0x1626851b 0x1626a69e 0x16269b3f 0x16267eab 0x13cf8284 0x13d24afc 0x13c5217e 0x14ec2495 0x15ba440f 0x15b9d13b 0x15bb2699 0x1891ccb3 0x1891e00d 0x18ae0769 0x18ade022 0x7f76aa985609 0x7f76aa8aa133 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008274 [ 356 ] {} BaseDaemon: 2. Poco::Net::IPAddress::family() const @ 0x188f8212 in /usr/bin/clickhouse 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008297 [ 356 ] {} BaseDaemon: 3. ? @ 0x1626851b in /usr/bin/clickhouse 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008309 [ 356 ] {} BaseDaemon: 4. ? @ 0x1626a69e in /usr/bin/clickhouse ```. [#42234](https://github.com/ClickHouse/ClickHouse/pull/42234) ([Arthur Passos](https://github.com/arthurpassos)). * Backported in [#42793](https://github.com/ClickHouse/ClickHouse/issues/42793): Fix a bug in ParserFunction that could have led to a segmentation fault. [#42724](https://github.com/ClickHouse/ClickHouse/pull/42724) ([Nikolay Degterinsky](https://github.com/evillique)). diff --git a/docs/changelogs/v22.10.3.27-stable.md b/docs/changelogs/v22.10.3.27-stable.md index 6dc9fd7f3b9..6e0188ad619 100644 --- a/docs/changelogs/v22.10.3.27-stable.md +++ b/docs/changelogs/v22.10.3.27-stable.md @@ -14,7 +14,7 @@ sidebar_label: 2022 * Backported in [#42959](https://github.com/ClickHouse/ClickHouse/issues/42959): Before the fix, the user-defined config was preserved by RPM in `$file.rpmsave`. The PR fixes it and won't replace the user's files from packages. [#42936](https://github.com/ClickHouse/ClickHouse/pull/42936) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Backported in [#43042](https://github.com/ClickHouse/ClickHouse/issues/43042): Add a CI step to mark commits as ready for release; soft-forbid launching a release script from branches but master. [#43017](https://github.com/ClickHouse/ClickHouse/pull/43017) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#42864](https://github.com/ClickHouse/ClickHouse/issues/42864): Fix lowerUTF8()/upperUTF8() in case of symbol was in between 16-byte boundary (very frequent case of you have strings > 16 bytes long). [#42812](https://github.com/ClickHouse/ClickHouse/pull/42812) ([Azat Khuzhin](https://github.com/azat)). * Backported in [#43173](https://github.com/ClickHouse/ClickHouse/issues/43173): Fix rare possible hung on query cancellation. [#42874](https://github.com/ClickHouse/ClickHouse/pull/42874) ([Azat Khuzhin](https://github.com/azat)). diff --git a/docs/changelogs/v22.10.4.23-stable.md b/docs/changelogs/v22.10.4.23-stable.md index 04eb8be982f..4438a3470fc 100644 --- a/docs/changelogs/v22.10.4.23-stable.md +++ b/docs/changelogs/v22.10.4.23-stable.md @@ -13,7 +13,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#43053](https://github.com/ClickHouse/ClickHouse/issues/43053): Wait for all files are in sync before archiving them in integration tests. [#42891](https://github.com/ClickHouse/ClickHouse/pull/42891) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#43715](https://github.com/ClickHouse/ClickHouse/issues/43715): An issue with the following exception has been reported while trying to read a Parquet file from S3 into ClickHouse:. [#43297](https://github.com/ClickHouse/ClickHouse/pull/43297) ([Arthur Passos](https://github.com/arthurpassos)). * Backported in [#43576](https://github.com/ClickHouse/ClickHouse/issues/43576): Fix possible `Cannot create non-empty column with type Nothing` in functions if/multiIf. Closes [#43356](https://github.com/ClickHouse/ClickHouse/issues/43356). [#43368](https://github.com/ClickHouse/ClickHouse/pull/43368) ([Kruglov Pavel](https://github.com/Avogar)). diff --git a/docs/changelogs/v22.10.5.54-stable.md b/docs/changelogs/v22.10.5.54-stable.md index e372fb30618..73c15cf985d 100644 --- a/docs/changelogs/v22.10.5.54-stable.md +++ b/docs/changelogs/v22.10.5.54-stable.md @@ -17,7 +17,7 @@ sidebar_label: 2023 * Backported in [#44379](https://github.com/ClickHouse/ClickHouse/issues/44379): In rare cases, we don't rebuild binaries, because another task with a similar prefix succeeded. E.g. `binary_darwin` didn't restart because `binary_darwin_aarch64`. [#44311](https://github.com/ClickHouse/ClickHouse/pull/44311) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Backported in [#44559](https://github.com/ClickHouse/ClickHouse/issues/44559): Retry the integration tests on compressing errors. [#44529](https://github.com/ClickHouse/ClickHouse/pull/44529) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#44754](https://github.com/ClickHouse/ClickHouse/issues/44754): [#40651](https://github.com/ClickHouse/ClickHouse/issues/40651) [#41404](https://github.com/ClickHouse/ClickHouse/issues/41404). [#42126](https://github.com/ClickHouse/ClickHouse/pull/42126) ([Alexander Gololobov](https://github.com/davenger)). * Backported in [#43527](https://github.com/ClickHouse/ClickHouse/issues/43527): Fix incorrect UserTimeMicroseconds/SystemTimeMicroseconds accounting. [#42791](https://github.com/ClickHouse/ClickHouse/pull/42791) ([Azat Khuzhin](https://github.com/azat)). @@ -41,4 +41,3 @@ sidebar_label: 2023 * Implement a custom central checkout action [#44399](https://github.com/ClickHouse/ClickHouse/pull/44399) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Fix crash on delete from materialized view [#44705](https://github.com/ClickHouse/ClickHouse/pull/44705) ([Alexander Gololobov](https://github.com/davenger)). * Do not check read result consistency when unwinding [#44956](https://github.com/ClickHouse/ClickHouse/pull/44956) ([Alexander Gololobov](https://github.com/davenger)). - diff --git a/docs/changelogs/v22.10.6.3-stable.md b/docs/changelogs/v22.10.6.3-stable.md index b0e88c92cb0..c09e9a8670b 100644 --- a/docs/changelogs/v22.10.6.3-stable.md +++ b/docs/changelogs/v22.10.6.3-stable.md @@ -7,7 +7,6 @@ sidebar_label: 2023 ### ClickHouse release v22.10.6.3-stable (645a66d221f) FIXME as compared to v22.10.5.54-stable (dbc7984dc3b) -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#45084](https://github.com/ClickHouse/ClickHouse/issues/45084): fix alter table ttl error when wide part has light weight delete mask. [#44959](https://github.com/ClickHouse/ClickHouse/pull/44959) ([Mingliang Pan](https://github.com/liangliangpan)). - diff --git a/docs/changelogs/v22.10.7.13-stable.md b/docs/changelogs/v22.10.7.13-stable.md index c906e00e524..987bf6cf1e1 100644 --- a/docs/changelogs/v22.10.7.13-stable.md +++ b/docs/changelogs/v22.10.7.13-stable.md @@ -7,7 +7,7 @@ sidebar_label: 2023 ### ClickHouse release v22.10.7.13-stable (d261d9036cc) FIXME as compared to v22.10.6.3-stable (645a66d221f) -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#44998](https://github.com/ClickHouse/ClickHouse/issues/44998): Another fix for `Cannot read all data` error which could happen while reading `LowCardinality` dictionary from remote fs. Fixes [#44709](https://github.com/ClickHouse/ClickHouse/issues/44709). [#44875](https://github.com/ClickHouse/ClickHouse/pull/44875) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Backported in [#45551](https://github.com/ClickHouse/ClickHouse/issues/45551): Fix `SELECT ... FROM system.dictionaries` exception when there is a dictionary with a bad structure (e.g. incorrect type in xml config). [#45399](https://github.com/ClickHouse/ClickHouse/pull/45399) ([Aleksei Filatov](https://github.com/aalexfvk)). @@ -18,4 +18,3 @@ sidebar_label: 2023 * Improve release scripts [#45074](https://github.com/ClickHouse/ClickHouse/pull/45074) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Fix wrong approved_at, simplify conditions [#45302](https://github.com/ClickHouse/ClickHouse/pull/45302) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Get rid of artifactory in favor of r2 + ch-repos-manager [#45421](https://github.com/ClickHouse/ClickHouse/pull/45421) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v22.11.1.1360-stable.md b/docs/changelogs/v22.11.1.1360-stable.md index 1da53be02b7..4aa110484f8 100644 --- a/docs/changelogs/v22.11.1.1360-stable.md +++ b/docs/changelogs/v22.11.1.1360-stable.md @@ -81,7 +81,7 @@ sidebar_label: 2022 * Before the fix, the user-defined config was preserved by RPM in `$file.rpmsave`. The PR fixes it and won't replace the user's files from packages. [#42936](https://github.com/ClickHouse/ClickHouse/pull/42936) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Add a CI step to mark commits as ready for release; soft-forbid launching a release script from branches but master. [#43017](https://github.com/ClickHouse/ClickHouse/pull/43017) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Fix schema inference in s3Cluster and improve in hdfsCluster. [#41979](https://github.com/ClickHouse/ClickHouse/pull/41979) ([Kruglov Pavel](https://github.com/Avogar)). * Fix retries while reading from http table engines / table function. (retrtiable errors could be retries more times than needed, non-retrialble errors resulted in failed assertion in code). [#42224](https://github.com/ClickHouse/ClickHouse/pull/42224) ([Kseniia Sumarokova](https://github.com/kssenii)). diff --git a/docs/changelogs/v22.11.2.30-stable.md b/docs/changelogs/v22.11.2.30-stable.md index 7b2febe072a..4759aa4a503 100644 --- a/docs/changelogs/v22.11.2.30-stable.md +++ b/docs/changelogs/v22.11.2.30-stable.md @@ -13,7 +13,7 @@ sidebar_label: 2022 #### Improvement * Backported in [#43511](https://github.com/ClickHouse/ClickHouse/issues/43511): Restrict default access to named collections for user defined in config. It must have explicit `show_named_collections=1` to be able to see them. [#43325](https://github.com/ClickHouse/ClickHouse/pull/43325) ([Kseniia Sumarokova](https://github.com/kssenii)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#43716](https://github.com/ClickHouse/ClickHouse/issues/43716): An issue with the following exception has been reported while trying to read a Parquet file from S3 into ClickHouse:. [#43297](https://github.com/ClickHouse/ClickHouse/pull/43297) ([Arthur Passos](https://github.com/arthurpassos)). * Backported in [#43431](https://github.com/ClickHouse/ClickHouse/issues/43431): Fixed queries with `SAMPLE BY` with prewhere optimization on tables using `Merge` engine. [#43315](https://github.com/ClickHouse/ClickHouse/pull/43315) ([Antonio Andelic](https://github.com/antonio2368)). diff --git a/docs/changelogs/v22.11.3.47-stable.md b/docs/changelogs/v22.11.3.47-stable.md index d6451b853f7..a993ff8516f 100644 --- a/docs/changelogs/v22.11.3.47-stable.md +++ b/docs/changelogs/v22.11.3.47-stable.md @@ -13,7 +13,7 @@ sidebar_label: 2023 * Backported in [#44380](https://github.com/ClickHouse/ClickHouse/issues/44380): In rare cases, we don't rebuild binaries, because another task with a similar prefix succeeded. E.g. `binary_darwin` didn't restart because `binary_darwin_aarch64`. [#44311](https://github.com/ClickHouse/ClickHouse/pull/44311) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Backported in [#44560](https://github.com/ClickHouse/ClickHouse/issues/44560): Retry the integration tests on compressing errors. [#44529](https://github.com/ClickHouse/ClickHouse/pull/44529) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#44756](https://github.com/ClickHouse/ClickHouse/issues/44756): [#40651](https://github.com/ClickHouse/ClickHouse/issues/40651) [#41404](https://github.com/ClickHouse/ClickHouse/issues/41404). [#42126](https://github.com/ClickHouse/ClickHouse/pull/42126) ([Alexander Gololobov](https://github.com/davenger)). * Backported in [#43528](https://github.com/ClickHouse/ClickHouse/issues/43528): Fix incorrect UserTimeMicroseconds/SystemTimeMicroseconds accounting. [#42791](https://github.com/ClickHouse/ClickHouse/pull/42791) ([Azat Khuzhin](https://github.com/azat)). @@ -37,4 +37,3 @@ sidebar_label: 2023 * Implement a custom central checkout action [#44399](https://github.com/ClickHouse/ClickHouse/pull/44399) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Fix crash on delete from materialized view [#44705](https://github.com/ClickHouse/ClickHouse/pull/44705) ([Alexander Gololobov](https://github.com/davenger)). * Do not check read result consistency when unwinding [#44956](https://github.com/ClickHouse/ClickHouse/pull/44956) ([Alexander Gololobov](https://github.com/davenger)). - diff --git a/docs/changelogs/v22.11.4.3-stable.md b/docs/changelogs/v22.11.4.3-stable.md index 33780e848ef..b0e7586277a 100644 --- a/docs/changelogs/v22.11.4.3-stable.md +++ b/docs/changelogs/v22.11.4.3-stable.md @@ -7,7 +7,6 @@ sidebar_label: 2023 ### ClickHouse release v22.11.4.3-stable (7f4cf554f69) FIXME as compared to v22.11.3.47-stable (1c49d124a37) -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#45085](https://github.com/ClickHouse/ClickHouse/issues/45085): fix alter table ttl error when wide part has light weight delete mask. [#44959](https://github.com/ClickHouse/ClickHouse/pull/44959) ([Mingliang Pan](https://github.com/liangliangpan)). - diff --git a/docs/changelogs/v22.11.5.15-stable.md b/docs/changelogs/v22.11.5.15-stable.md index 742a8740514..f11a01c4b1c 100644 --- a/docs/changelogs/v22.11.5.15-stable.md +++ b/docs/changelogs/v22.11.5.15-stable.md @@ -7,7 +7,7 @@ sidebar_label: 2023 ### ClickHouse release v22.11.5.15-stable (d763e5a9239) FIXME as compared to v22.11.4.3-stable (7f4cf554f69) -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#44999](https://github.com/ClickHouse/ClickHouse/issues/44999): Another fix for `Cannot read all data` error which could happen while reading `LowCardinality` dictionary from remote fs. Fixes [#44709](https://github.com/ClickHouse/ClickHouse/issues/44709). [#44875](https://github.com/ClickHouse/ClickHouse/pull/44875) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Backported in [#45552](https://github.com/ClickHouse/ClickHouse/issues/45552): Fix `SELECT ... FROM system.dictionaries` exception when there is a dictionary with a bad structure (e.g. incorrect type in xml config). [#45399](https://github.com/ClickHouse/ClickHouse/pull/45399) ([Aleksei Filatov](https://github.com/aalexfvk)). @@ -19,4 +19,3 @@ sidebar_label: 2023 * Fix wrong approved_at, simplify conditions [#45302](https://github.com/ClickHouse/ClickHouse/pull/45302) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Get rid of artifactory in favor of r2 + ch-repos-manager [#45421](https://github.com/ClickHouse/ClickHouse/pull/45421) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Trim refs/tags/ from GITHUB_TAG in release workflow [#45636](https://github.com/ClickHouse/ClickHouse/pull/45636) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v22.11.6.44-stable.md b/docs/changelogs/v22.11.6.44-stable.md index 6e628b85150..db19e73c666 100644 --- a/docs/changelogs/v22.11.6.44-stable.md +++ b/docs/changelogs/v22.11.6.44-stable.md @@ -17,7 +17,7 @@ sidebar_label: 2023 * Backported in [#46483](https://github.com/ClickHouse/ClickHouse/issues/46483): Get rid of unnecessary build for standalone clickhouse-keeper. [#46367](https://github.com/ClickHouse/ClickHouse/pull/46367) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Backported in [#46507](https://github.com/ClickHouse/ClickHouse/issues/46507): Some time ago the ccache compression was changed to `zst`, but `gz` archives are downloaded by default. It fixes it by prioritizing zst archive. [#46490](https://github.com/ClickHouse/ClickHouse/pull/46490) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#45903](https://github.com/ClickHouse/ClickHouse/issues/45903): Fixed bug with non-parsable default value for EPHEMERAL column in table metadata. [#44026](https://github.com/ClickHouse/ClickHouse/pull/44026) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). * Backported in [#46239](https://github.com/ClickHouse/ClickHouse/issues/46239): A couple of seg faults have been reported around `c-ares`. All of the recent stack traces observed fail on inserting into `std::unodered_set<>`. I believe I have found the root cause of this, it seems to be unprocessed queries. Prior to this PR, CH calls `poll` to wait on the file descriptors in the `c-ares` channel. According to the [poll docs](https://man7.org/linux/man-pages/man2/poll.2.html), a negative return value means an error has ocurred. Because of this, we would abort the execution and return failure. The problem is that `poll` will also return a negative value if a system interrupt occurs. A system interrupt does not mean the processing has failed or ended, but we would abort it anyways because we were checking for negative values. Once the execution is aborted, the whole stack is destroyed, which includes the `std::unordered_set` passed to the `void *` parameter of the c-ares callback. Once c-ares completed the request, the callback would be invoked and would access an invalid memory address causing a segfault. [#45629](https://github.com/ClickHouse/ClickHouse/pull/45629) ([Arthur Passos](https://github.com/arthurpassos)). @@ -34,4 +34,3 @@ sidebar_label: 2023 * Add helping logging to auto-merge script [#46080](https://github.com/ClickHouse/ClickHouse/pull/46080) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Fix write buffer destruction order for vertical merge. [#46205](https://github.com/ClickHouse/ClickHouse/pull/46205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Get rid of legacy DocsReleaseChecks [#46665](https://github.com/ClickHouse/ClickHouse/pull/46665) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v22.12.1.1752-stable.md b/docs/changelogs/v22.12.1.1752-stable.md index 4f4c4b11150..1549af037f2 100644 --- a/docs/changelogs/v22.12.1.1752-stable.md +++ b/docs/changelogs/v22.12.1.1752-stable.md @@ -91,7 +91,7 @@ sidebar_label: 2022 * Bring sha512 sums back to the building step. [#44017](https://github.com/ClickHouse/ClickHouse/pull/44017) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Kill stress tests after 2.5h in case of hanging process. [#44214](https://github.com/ClickHouse/ClickHouse/pull/44214) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Fixed unable to log in (because of failure to create session_log entry) in rare case of messed up setting profiles. ... [#42641](https://github.com/ClickHouse/ClickHouse/pull/42641) ([Vasily Nemkov](https://github.com/Enmk)). * Fix incorrect UserTimeMicroseconds/SystemTimeMicroseconds accounting. [#42791](https://github.com/ClickHouse/ClickHouse/pull/42791) ([Azat Khuzhin](https://github.com/azat)). diff --git a/docs/changelogs/v22.12.2.25-stable.md b/docs/changelogs/v22.12.2.25-stable.md index 194d0fe3cbc..968854d0428 100644 --- a/docs/changelogs/v22.12.2.25-stable.md +++ b/docs/changelogs/v22.12.2.25-stable.md @@ -11,7 +11,7 @@ sidebar_label: 2023 * Backported in [#44381](https://github.com/ClickHouse/ClickHouse/issues/44381): In rare cases, we don't rebuild binaries, because another task with a similar prefix succeeded. E.g. `binary_darwin` didn't restart because `binary_darwin_aarch64`. [#44311](https://github.com/ClickHouse/ClickHouse/pull/44311) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Backported in [#44561](https://github.com/ClickHouse/ClickHouse/issues/44561): Retry the integration tests on compressing errors. [#44529](https://github.com/ClickHouse/ClickHouse/pull/44529) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#44739](https://github.com/ClickHouse/ClickHouse/issues/44739): [#40651](https://github.com/ClickHouse/ClickHouse/issues/40651) [#41404](https://github.com/ClickHouse/ClickHouse/issues/41404). [#42126](https://github.com/ClickHouse/ClickHouse/pull/42126) ([Alexander Gololobov](https://github.com/davenger)). * Backported in [#44764](https://github.com/ClickHouse/ClickHouse/issues/44764): Fix parsing of bad version from compatibility setting. [#44224](https://github.com/ClickHouse/ClickHouse/pull/44224) ([Kruglov Pavel](https://github.com/Avogar)). diff --git a/docs/changelogs/v22.12.3.5-stable.md b/docs/changelogs/v22.12.3.5-stable.md index 8cbcbc6a590..5f0cc9cebf3 100644 --- a/docs/changelogs/v22.12.3.5-stable.md +++ b/docs/changelogs/v22.12.3.5-stable.md @@ -7,11 +7,10 @@ sidebar_label: 2023 ### ClickHouse release v22.12.3.5-stable (893de538f02) FIXME as compared to v22.12.2.25-stable (c790cfd4465) -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#45086](https://github.com/ClickHouse/ClickHouse/issues/45086): fix alter table ttl error when wide part has light weight delete mask. [#44959](https://github.com/ClickHouse/ClickHouse/pull/44959) ([Mingliang Pan](https://github.com/liangliangpan)). #### NOT FOR CHANGELOG / INSIGNIFICANT * Do not check read result consistency when unwinding [#44956](https://github.com/ClickHouse/ClickHouse/pull/44956) ([Alexander Gololobov](https://github.com/davenger)). - diff --git a/docs/changelogs/v22.12.4.76-stable.md b/docs/changelogs/v22.12.4.76-stable.md index 79569ff841e..cdadaae7f7b 100644 --- a/docs/changelogs/v22.12.4.76-stable.md +++ b/docs/changelogs/v22.12.4.76-stable.md @@ -22,7 +22,7 @@ sidebar_label: 2023 * Backported in [#46509](https://github.com/ClickHouse/ClickHouse/issues/46509): Some time ago the ccache compression was changed to `zst`, but `gz` archives are downloaded by default. It fixes it by prioritizing zst archive. [#46490](https://github.com/ClickHouse/ClickHouse/pull/46490) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Backported in [#47058](https://github.com/ClickHouse/ClickHouse/issues/47058): Fix error during server startup on old distros (e.g. Amazon Linux 2) and on ARM that glibc 2.28 symbols are not found. [#47008](https://github.com/ClickHouse/ClickHouse/pull/47008) ([Robert Schulze](https://github.com/rschu1ze)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#45904](https://github.com/ClickHouse/ClickHouse/issues/45904): Fixed bug with non-parsable default value for EPHEMERAL column in table metadata. [#44026](https://github.com/ClickHouse/ClickHouse/pull/44026) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). * Backported in [#45321](https://github.com/ClickHouse/ClickHouse/issues/45321): Fixed a bug in normalization of a `DEFAULT` expression in `CREATE TABLE` statement. The second argument of function `in` (or the right argument of operator `IN`) might be replaced with the result of its evaluation during CREATE query execution. Fixes [#44496](https://github.com/ClickHouse/ClickHouse/issues/44496). [#44547](https://github.com/ClickHouse/ClickHouse/pull/44547) ([Alexander Tokmakov](https://github.com/tavplubix)). @@ -52,4 +52,3 @@ sidebar_label: 2023 * Fix dependencies for InstallPackagesTestAarch64 [#46597](https://github.com/ClickHouse/ClickHouse/pull/46597) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Get rid of legacy DocsReleaseChecks [#46665](https://github.com/ClickHouse/ClickHouse/pull/46665) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Reduce updates of Mergeable Check [#46781](https://github.com/ClickHouse/ClickHouse/pull/46781) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v22.12.5.34-stable.md b/docs/changelogs/v22.12.5.34-stable.md index 95befaa88ff..61f099462a0 100644 --- a/docs/changelogs/v22.12.5.34-stable.md +++ b/docs/changelogs/v22.12.5.34-stable.md @@ -10,7 +10,7 @@ sidebar_label: 2023 #### Improvement * Backported in [#46983](https://github.com/ClickHouse/ClickHouse/issues/46983): - Apply `ALTER TABLE table_name ON CLUSTER cluster MOVE PARTITION|PART partition_expr TO DISK|VOLUME 'disk_name'` to all replicas. Because `ALTER TABLE t MOVE` is not replicated. [#46402](https://github.com/ClickHouse/ClickHouse/pull/46402) ([lizhuoyu5](https://github.com/lzydmxy)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#45729](https://github.com/ClickHouse/ClickHouse/issues/45729): Fix key description when encountering duplicate primary keys. This can happen in projections. See [#45590](https://github.com/ClickHouse/ClickHouse/issues/45590) for details. [#45686](https://github.com/ClickHouse/ClickHouse/pull/45686) ([Amos Bird](https://github.com/amosbird)). * Backported in [#46398](https://github.com/ClickHouse/ClickHouse/issues/46398): Fix `SYSTEM UNFREEZE` queries failing with the exception `CANNOT_PARSE_INPUT_ASSERTION_FAILED`. [#46325](https://github.com/ClickHouse/ClickHouse/pull/46325) ([Aleksei Filatov](https://github.com/aalexfvk)). @@ -26,4 +26,3 @@ sidebar_label: 2023 * Update typing for a new PyGithub version [#47123](https://github.com/ClickHouse/ClickHouse/pull/47123) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Follow-up to [#46681](https://github.com/ClickHouse/ClickHouse/issues/46681) [#47284](https://github.com/ClickHouse/ClickHouse/pull/47284) ([Alexander Tokmakov](https://github.com/tavplubix)). * Add a manual trigger for release workflow [#47302](https://github.com/ClickHouse/ClickHouse/pull/47302) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v22.2.1.2139-prestable.md b/docs/changelogs/v22.2.1.2139-prestable.md index 67db2d9a18d..ca3a84ceaa2 100644 --- a/docs/changelogs/v22.2.1.2139-prestable.md +++ b/docs/changelogs/v22.2.1.2139-prestable.md @@ -141,7 +141,7 @@ sidebar_label: 2022 * - Rework version_helper, make it executable - Reimplement StorageSystemContributors.sh in version_helper - Create a release script. [#34641](https://github.com/ClickHouse/ClickHouse/pull/34641) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * - Fix path in workflows/release.yml - To be backported to branch 22.1. [#34646](https://github.com/ClickHouse/ClickHouse/pull/34646) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Fix lz4 compression for output. Closes [#31421](https://github.com/ClickHouse/ClickHouse/issues/31421). [#31862](https://github.com/ClickHouse/ClickHouse/pull/31862) ([Kruglov Pavel](https://github.com/Avogar)). * Create a function escapeForLDAPFilter and use it to escape characters '(' and ')' in a final_user_dn variable. [#33401](https://github.com/ClickHouse/ClickHouse/pull/33401) ([IlyaTsoi](https://github.com/IlyaTsoi)). diff --git a/docs/changelogs/v22.2.3.5-stable.md b/docs/changelogs/v22.2.3.5-stable.md index c433669049d..a4368e465aa 100644 --- a/docs/changelogs/v22.2.3.5-stable.md +++ b/docs/changelogs/v22.2.3.5-stable.md @@ -7,7 +7,7 @@ sidebar_label: 2022 ### ClickHouse release v22.2.3.5-stable FIXME as compared to v22.2.2.1-stable -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#34848](https://github.com/ClickHouse/ClickHouse/issues/34848): Fix possible failures in S2 functions when queries contain const columns. [#34745](https://github.com/ClickHouse/ClickHouse/pull/34745) ([Bharat Nallan](https://github.com/bharatnc)). diff --git a/docs/changelogs/v22.3.1.1262-prestable.md b/docs/changelogs/v22.3.1.1262-prestable.md index e6203ff18d1..385393cef17 100644 --- a/docs/changelogs/v22.3.1.1262-prestable.md +++ b/docs/changelogs/v22.3.1.1262-prestable.md @@ -95,7 +95,7 @@ sidebar_label: 2022 * Clion has the following problems "The breakpoint will not currently be hit. No executable code is associated with this line". [#35179](https://github.com/ClickHouse/ClickHouse/pull/35179) ([小路](https://github.com/nicelulu)). * Add an ability to build stripped binaries with cmake. [#35196](https://github.com/ClickHouse/ClickHouse/pull/35196) ([alesapin](https://github.com/alesapin)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Fix distributed subquery max_query_size limitation inconsistency. [#34078](https://github.com/ClickHouse/ClickHouse/pull/34078) ([Chao Ma](https://github.com/godliness)). * Fix incorrect trivial count result when part movement feature is used [#34089](https://github.com/ClickHouse/ClickHouse/issues/34089). [#34385](https://github.com/ClickHouse/ClickHouse/pull/34385) ([nvartolomei](https://github.com/nvartolomei)). diff --git a/docs/changelogs/v22.3.10.22-lts.md b/docs/changelogs/v22.3.10.22-lts.md index a43b8301aad..cc033eb707d 100644 --- a/docs/changelogs/v22.3.10.22-lts.md +++ b/docs/changelogs/v22.3.10.22-lts.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Bug Fix * Backported in [#39761](https://github.com/ClickHouse/ClickHouse/issues/39761): Fix seeking while reading from encrypted disk. This PR fixes [#38381](https://github.com/ClickHouse/ClickHouse/issues/38381). [#39687](https://github.com/ClickHouse/ClickHouse/pull/39687) ([Vitaly Baranov](https://github.com/vitlibar)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#39206](https://github.com/ClickHouse/ClickHouse/issues/39206): Fix reading of sparse columns from `MergeTree` tables that store their data in S3. [#37978](https://github.com/ClickHouse/ClickHouse/pull/37978) ([Anton Popov](https://github.com/CurtizJ)). * Backported in [#39381](https://github.com/ClickHouse/ClickHouse/issues/39381): Fixed error `Not found column Type in block` in selects with `PREWHERE` and read-in-order optimizations. [#39157](https://github.com/ClickHouse/ClickHouse/pull/39157) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). @@ -18,7 +18,7 @@ sidebar_label: 2022 * Backported in [#39610](https://github.com/ClickHouse/ClickHouse/issues/39610): Fix bug with maxsplit argument for splitByChar, which was not working correctly. [#39552](https://github.com/ClickHouse/ClickHouse/pull/39552) ([filimonov](https://github.com/filimonov)). * Backported in [#39834](https://github.com/ClickHouse/ClickHouse/issues/39834): Fix `CANNOT_READ_ALL_DATA` exception with `local_filesystem_read_method=pread_threadpool`. This bug affected only Linux kernel version 5.9 and 5.10 according to [man](https://manpages.debian.org/testing/manpages-dev/preadv2.2.en.html#BUGS). [#39800](https://github.com/ClickHouse/ClickHouse/pull/39800) ([Anton Popov](https://github.com/CurtizJ)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#39238](https://github.com/ClickHouse/ClickHouse/issues/39238): Fix performance regression of scalar query optimization. [#35986](https://github.com/ClickHouse/ClickHouse/pull/35986) ([Amos Bird](https://github.com/amosbird)). * Backported in [#39531](https://github.com/ClickHouse/ClickHouse/issues/39531): Fix some issues with async reads from remote filesystem which happened when reading low cardinality. [#36763](https://github.com/ClickHouse/ClickHouse/pull/36763) ([Kseniia Sumarokova](https://github.com/kssenii)). diff --git a/docs/changelogs/v22.3.11.12-lts.md b/docs/changelogs/v22.3.11.12-lts.md index e718493d9a0..58df0c0cadb 100644 --- a/docs/changelogs/v22.3.11.12-lts.md +++ b/docs/changelogs/v22.3.11.12-lts.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#39881](https://github.com/ClickHouse/ClickHouse/issues/39881): Former packages used to install systemd.service file to `/etc`. The files there are marked as `conf` and are not cleaned out, and not updated automatically. This PR cleans them out. [#39323](https://github.com/ClickHouse/ClickHouse/pull/39323) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#39336](https://github.com/ClickHouse/ClickHouse/issues/39336): Fix `parallel_view_processing=1` with `optimize_trivial_insert_select=1`. Fix `max_insert_threads` while pushing to views. [#38731](https://github.com/ClickHouse/ClickHouse/pull/38731) ([Azat Khuzhin](https://github.com/azat)). diff --git a/docs/changelogs/v22.3.12.19-lts.md b/docs/changelogs/v22.3.12.19-lts.md index 4f6342419f7..6ae342583a9 100644 --- a/docs/changelogs/v22.3.12.19-lts.md +++ b/docs/changelogs/v22.3.12.19-lts.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#40695](https://github.com/ClickHouse/ClickHouse/issues/40695): Fix TGZ packages. [#40681](https://github.com/ClickHouse/ClickHouse/pull/40681) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#40160](https://github.com/ClickHouse/ClickHouse/issues/40160): fix HashMethodOneNumber get wrong key value when column is const. [#40020](https://github.com/ClickHouse/ClickHouse/pull/40020) ([Duc Canh Le](https://github.com/canhld94)). * Backported in [#40122](https://github.com/ClickHouse/ClickHouse/issues/40122): Fix bug in collectFilesToSkip() by adding correct file extension(.idx or idx2) for indexes to be recalculated, avoid wrong hard links. Fixed [#39896](https://github.com/ClickHouse/ClickHouse/issues/39896). [#40095](https://github.com/ClickHouse/ClickHouse/pull/40095) ([Jianmei Zhang](https://github.com/zhangjmruc)). diff --git a/docs/changelogs/v22.3.13.80-lts.md b/docs/changelogs/v22.3.13.80-lts.md index bb84da9a96a..3d0a6a77276 100644 --- a/docs/changelogs/v22.3.13.80-lts.md +++ b/docs/changelogs/v22.3.13.80-lts.md @@ -17,7 +17,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#41557](https://github.com/ClickHouse/ClickHouse/issues/41557): Add `source` field to deb packages, update `nfpm`. [#41531](https://github.com/ClickHouse/ClickHouse/pull/41531) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#40745](https://github.com/ClickHouse/ClickHouse/issues/40745): * Fix cast lowcard of nullable in JoinSwitcher, close [#37385](https://github.com/ClickHouse/ClickHouse/issues/37385). [#37453](https://github.com/ClickHouse/ClickHouse/pull/37453) ([Vladimir C](https://github.com/vdimir)). * Backported in [#41812](https://github.com/ClickHouse/ClickHouse/issues/41812): Update `simdjson`. This fixes [#38621](https://github.com/ClickHouse/ClickHouse/issues/38621). [#38838](https://github.com/ClickHouse/ClickHouse/pull/38838) ([Alexey Milovidov](https://github.com/alexey-milovidov)). @@ -36,7 +36,7 @@ sidebar_label: 2022 * Backported in [#41639](https://github.com/ClickHouse/ClickHouse/issues/41639): Fix possible `pipeline stuck` exception for queries with `OFFSET`. The error was found with `enable_optimize_predicate_expression = 0` and always false condition in `WHERE`. Fixes [#41383](https://github.com/ClickHouse/ClickHouse/issues/41383). [#41588](https://github.com/ClickHouse/ClickHouse/pull/41588) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Backported in [#41899](https://github.com/ClickHouse/ClickHouse/issues/41899): Fix possible crash in `SELECT` from `Merge` table with enabled `optimize_monotonous_functions_in_order_by` setting. Fixes [#41269](https://github.com/ClickHouse/ClickHouse/issues/41269). [#41740](https://github.com/ClickHouse/ClickHouse/pull/41740) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#41321](https://github.com/ClickHouse/ClickHouse/issues/41321): Fix bug in function `if` when resulting column type differs with resulting data type that led to logical errors like `Logical error: 'Bad cast from type DB::ColumnVector to DB::ColumnVector'.`. Closes [#35367](https://github.com/ClickHouse/ClickHouse/issues/35367). [#35476](https://github.com/ClickHouse/ClickHouse/pull/35476) ([Kruglov Pavel](https://github.com/Avogar)). diff --git a/docs/changelogs/v22.3.14.18-lts.md b/docs/changelogs/v22.3.14.18-lts.md index 88801b268ce..235525bbe81 100644 --- a/docs/changelogs/v22.3.14.18-lts.md +++ b/docs/changelogs/v22.3.14.18-lts.md @@ -14,7 +14,7 @@ sidebar_label: 2022 * Backported in [#42328](https://github.com/ClickHouse/ClickHouse/issues/42328): Update cctz to the latest master, update tzdb to 2020e. [#42273](https://github.com/ClickHouse/ClickHouse/pull/42273) ([Dom Del Nano](https://github.com/ddelnano)). * Backported in [#42358](https://github.com/ClickHouse/ClickHouse/issues/42358): Update tzdata to 2022e to support the new timezone changes. Palestine transitions are now Saturdays at 02:00. Simplify three Ukraine zones into one. Jordan and Syria switch from +02/+03 with DST to year-round +03. (https://data.iana.org/time-zones/tzdb/NEWS). This closes [#42252](https://github.com/ClickHouse/ClickHouse/issues/42252). [#42327](https://github.com/ClickHouse/ClickHouse/pull/42327) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#42298](https://github.com/ClickHouse/ClickHouse/issues/42298): Fix a bug with projections and the `aggregate_functions_null_for_empty` setting. This bug is very rare and appears only if you enable the `aggregate_functions_null_for_empty` setting in the server's config. This closes [#41647](https://github.com/ClickHouse/ClickHouse/issues/41647). [#42198](https://github.com/ClickHouse/ClickHouse/pull/42198) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Backported in [#42592](https://github.com/ClickHouse/ClickHouse/issues/42592): This closes [#42453](https://github.com/ClickHouse/ClickHouse/issues/42453). [#42573](https://github.com/ClickHouse/ClickHouse/pull/42573) ([Alexey Milovidov](https://github.com/alexey-milovidov)). diff --git a/docs/changelogs/v22.3.14.23-lts.md b/docs/changelogs/v22.3.14.23-lts.md index 0a8c645702e..e086e223fb1 100644 --- a/docs/changelogs/v22.3.14.23-lts.md +++ b/docs/changelogs/v22.3.14.23-lts.md @@ -17,7 +17,7 @@ sidebar_label: 2022 * Backported in [#42328](https://github.com/ClickHouse/ClickHouse/issues/42328): Update cctz to the latest master, update tzdb to 2020e. [#42273](https://github.com/ClickHouse/ClickHouse/pull/42273) ([Dom Del Nano](https://github.com/ddelnano)). * Backported in [#42358](https://github.com/ClickHouse/ClickHouse/issues/42358): Update tzdata to 2022e to support the new timezone changes. Palestine transitions are now Saturdays at 02:00. Simplify three Ukraine zones into one. Jordan and Syria switch from +02/+03 with DST to year-round +03. (https://data.iana.org/time-zones/tzdb/NEWS). This closes [#42252](https://github.com/ClickHouse/ClickHouse/issues/42252). [#42327](https://github.com/ClickHouse/ClickHouse/pull/42327) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#42298](https://github.com/ClickHouse/ClickHouse/issues/42298): Fix a bug with projections and the `aggregate_functions_null_for_empty` setting. This bug is very rare and appears only if you enable the `aggregate_functions_null_for_empty` setting in the server's config. This closes [#41647](https://github.com/ClickHouse/ClickHouse/issues/41647). [#42198](https://github.com/ClickHouse/ClickHouse/pull/42198) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Backported in [#42592](https://github.com/ClickHouse/ClickHouse/issues/42592): This closes [#42453](https://github.com/ClickHouse/ClickHouse/issues/42453). [#42573](https://github.com/ClickHouse/ClickHouse/pull/42573) ([Alexey Milovidov](https://github.com/alexey-milovidov)). diff --git a/docs/changelogs/v22.3.15.33-lts.md b/docs/changelogs/v22.3.15.33-lts.md index e59bf8bd1cd..3f675bfba1e 100644 --- a/docs/changelogs/v22.3.15.33-lts.md +++ b/docs/changelogs/v22.3.15.33-lts.md @@ -18,7 +18,7 @@ sidebar_label: 2022 * Backported in [#42963](https://github.com/ClickHouse/ClickHouse/issues/42963): Before the fix, the user-defined config was preserved by RPM in `$file.rpmsave`. The PR fixes it and won't replace the user's files from packages. [#42936](https://github.com/ClickHouse/ClickHouse/pull/42936) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Backported in [#43039](https://github.com/ClickHouse/ClickHouse/issues/43039): Add a CI step to mark commits as ready for release; soft-forbid launching a release script from branches but master. [#43017](https://github.com/ClickHouse/ClickHouse/pull/43017) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#43427](https://github.com/ClickHouse/ClickHouse/issues/43427): Fixed queries with `SAMPLE BY` with prewhere optimization on tables using `Merge` engine. [#43315](https://github.com/ClickHouse/ClickHouse/pull/43315) ([Antonio Andelic](https://github.com/antonio2368)). * Backported in [#43720](https://github.com/ClickHouse/ClickHouse/issues/43720): Fixed primary key analysis with conditions involving `toString(enum)`. [#43596](https://github.com/ClickHouse/ClickHouse/pull/43596) ([Nikita Taranov](https://github.com/nickitat)). diff --git a/docs/changelogs/v22.3.16.1190-lts.md b/docs/changelogs/v22.3.16.1190-lts.md index a43d34551ca..b65a6484109 100644 --- a/docs/changelogs/v22.3.16.1190-lts.md +++ b/docs/changelogs/v22.3.16.1190-lts.md @@ -18,7 +18,7 @@ sidebar_label: 2023 * Backported in [#44431](https://github.com/ClickHouse/ClickHouse/issues/44431): Kill stress tests after 2.5h in case of hanging process. [#44214](https://github.com/ClickHouse/ClickHouse/pull/44214) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Backported in [#44557](https://github.com/ClickHouse/ClickHouse/issues/44557): Retry the integration tests on compressing errors. [#44529](https://github.com/ClickHouse/ClickHouse/pull/44529) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#43512](https://github.com/ClickHouse/ClickHouse/issues/43512): - Fix several buffer over-reads. [#43159](https://github.com/ClickHouse/ClickHouse/pull/43159) ([Raúl Marín](https://github.com/Algunenano)). * Backported in [#43750](https://github.com/ClickHouse/ClickHouse/issues/43750): An issue with the following exception has been reported while trying to read a Parquet file from S3 into ClickHouse:. [#43297](https://github.com/ClickHouse/ClickHouse/pull/43297) ([Arthur Passos](https://github.com/arthurpassos)). @@ -30,4 +30,3 @@ sidebar_label: 2023 #### NO CL ENTRY * NO CL ENTRY: 'Fix multipart upload for large S3 object, backport to 22.3'. [#44217](https://github.com/ClickHouse/ClickHouse/pull/44217) ([ianton-ru](https://github.com/ianton-ru)). - diff --git a/docs/changelogs/v22.3.18.37-lts.md b/docs/changelogs/v22.3.18.37-lts.md index ff6378f09ad..5a9004e7f98 100644 --- a/docs/changelogs/v22.3.18.37-lts.md +++ b/docs/changelogs/v22.3.18.37-lts.md @@ -14,7 +14,7 @@ sidebar_label: 2023 #### Build/Testing/Packaging Improvement * Backported in [#45856](https://github.com/ClickHouse/ClickHouse/issues/45856): Fix zookeeper downloading, update the version, and optimize the image size. [#44853](https://github.com/ClickHouse/ClickHouse/pull/44853) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#45620](https://github.com/ClickHouse/ClickHouse/issues/45620): Another fix for `Cannot read all data` error which could happen while reading `LowCardinality` dictionary from remote fs. Fixes [#44709](https://github.com/ClickHouse/ClickHouse/issues/44709). [#44875](https://github.com/ClickHouse/ClickHouse/pull/44875) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Backported in [#45549](https://github.com/ClickHouse/ClickHouse/issues/45549): Fix `SELECT ... FROM system.dictionaries` exception when there is a dictionary with a bad structure (e.g. incorrect type in xml config). [#45399](https://github.com/ClickHouse/ClickHouse/pull/45399) ([Aleksei Filatov](https://github.com/aalexfvk)). @@ -30,4 +30,3 @@ sidebar_label: 2023 * Get rid of progress timestamps in release publishing [#45818](https://github.com/ClickHouse/ClickHouse/pull/45818) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Add helping logging to auto-merge script [#46080](https://github.com/ClickHouse/ClickHouse/pull/46080) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Fix write buffer destruction order for vertical merge. [#46205](https://github.com/ClickHouse/ClickHouse/pull/46205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). - diff --git a/docs/changelogs/v22.3.19.6-lts.md b/docs/changelogs/v22.3.19.6-lts.md index d5b45f4ce66..dc34ece918d 100644 --- a/docs/changelogs/v22.3.19.6-lts.md +++ b/docs/changelogs/v22.3.19.6-lts.md @@ -7,11 +7,10 @@ sidebar_label: 2023 ### ClickHouse release v22.3.19.6-lts (467e0a7bd77) FIXME as compared to v22.3.18.37-lts (fe512717551) -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#46440](https://github.com/ClickHouse/ClickHouse/issues/46440): Fix possible `LOGICAL_ERROR` in asynchronous inserts with invalid data sent in format `VALUES`. [#46350](https://github.com/ClickHouse/ClickHouse/pull/46350) ([Anton Popov](https://github.com/CurtizJ)). #### NOT FOR CHANGELOG / INSIGNIFICANT * Get rid of legacy DocsReleaseChecks [#46665](https://github.com/ClickHouse/ClickHouse/pull/46665) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v22.3.3.44-lts.md b/docs/changelogs/v22.3.3.44-lts.md index 3d113d45e68..bf491e46915 100644 --- a/docs/changelogs/v22.3.3.44-lts.md +++ b/docs/changelogs/v22.3.3.44-lts.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Bug Fix * Backported in [#35928](https://github.com/ClickHouse/ClickHouse/issues/35928): Added settings `input_format_ipv4_default_on_conversion_error`, `input_format_ipv6_default_on_conversion_error` to allow insert of invalid ip address values as default into tables. Closes [#35726](https://github.com/ClickHouse/ClickHouse/issues/35726). [#35733](https://github.com/ClickHouse/ClickHouse/pull/35733) ([Maksim Kita](https://github.com/kitaisreal)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#35415](https://github.com/ClickHouse/ClickHouse/issues/35415): Fix possible deadlock in cache. [#35378](https://github.com/ClickHouse/ClickHouse/pull/35378) ([Kseniia Sumarokova](https://github.com/kssenii)). * Backported in [#35563](https://github.com/ClickHouse/ClickHouse/issues/35563): Fix cast into IPv4, IPv6 address in IN section. Fixes [#35528](https://github.com/ClickHouse/ClickHouse/issues/35528). [#35534](https://github.com/ClickHouse/ClickHouse/pull/35534) ([Maksim Kita](https://github.com/kitaisreal)). diff --git a/docs/changelogs/v22.3.4.20-lts.md b/docs/changelogs/v22.3.4.20-lts.md index 72d7b90c743..070ad961beb 100644 --- a/docs/changelogs/v22.3.4.20-lts.md +++ b/docs/changelogs/v22.3.4.20-lts.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * - Add `_le_` method for ClickHouseVersion - Fix auto_version for existing tag - docker_server now support getting version from tags - Add python unit tests to backport workflow. [#36028](https://github.com/ClickHouse/ClickHouse/pull/36028) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#36244](https://github.com/ClickHouse/ClickHouse/issues/36244): Fix usage of quota with asynchronous inserts. [#35645](https://github.com/ClickHouse/ClickHouse/pull/35645) ([Anton Popov](https://github.com/CurtizJ)). * Backported in [#36240](https://github.com/ClickHouse/ClickHouse/issues/36240): Fix possible loss of subcolumns in type `Object`. [#35682](https://github.com/ClickHouse/ClickHouse/pull/35682) ([Anton Popov](https://github.com/CurtizJ)). diff --git a/docs/changelogs/v22.3.5.5-lts.md b/docs/changelogs/v22.3.5.5-lts.md index b4205d5e1a9..4a5dc318b9f 100644 --- a/docs/changelogs/v22.3.5.5-lts.md +++ b/docs/changelogs/v22.3.5.5-lts.md @@ -7,7 +7,7 @@ sidebar_label: 2022 ### ClickHouse release v22.3.5.5-lts (438b4a81f77) FIXME as compared to v22.3.4.20-lts (ecbaf001f49) -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#36525](https://github.com/ClickHouse/ClickHouse/issues/36525): Queries with aliases inside special operators returned parsing error (was broken in 22.1). Example: `SELECT substring('test' AS t, 1, 1)`. [#36167](https://github.com/ClickHouse/ClickHouse/pull/36167) ([Maksim Kita](https://github.com/kitaisreal)). * Backported in [#36674](https://github.com/ClickHouse/ClickHouse/issues/36674): Fix merges of wide parts with type `Object`. [#36637](https://github.com/ClickHouse/ClickHouse/pull/36637) ([Anton Popov](https://github.com/CurtizJ)). diff --git a/docs/changelogs/v22.3.6.5-lts.md b/docs/changelogs/v22.3.6.5-lts.md index 4183332efb0..90e65c7445f 100644 --- a/docs/changelogs/v22.3.6.5-lts.md +++ b/docs/changelogs/v22.3.6.5-lts.md @@ -7,7 +7,7 @@ sidebar_label: 2022 ### ClickHouse release v22.3.6.5-lts (3e44e824cff) FIXME as compared to v22.3.5.5-lts (438b4a81f77) -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#36795](https://github.com/ClickHouse/ClickHouse/issues/36795): Fix vertical merges in wide parts. Previously an exception `There is no column` can be thrown during merge. [#36707](https://github.com/ClickHouse/ClickHouse/pull/36707) ([Anton Popov](https://github.com/CurtizJ)). diff --git a/docs/changelogs/v22.3.7.28-lts.md b/docs/changelogs/v22.3.7.28-lts.md index 7347e8e0705..a6a7885abc3 100644 --- a/docs/changelogs/v22.3.7.28-lts.md +++ b/docs/changelogs/v22.3.7.28-lts.md @@ -7,14 +7,14 @@ sidebar_label: 2022 ### ClickHouse release v22.3.7.28-lts (420bdfa2751) FIXME as compared to v22.3.6.5-lts (3e44e824cff) -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#37715](https://github.com/ClickHouse/ClickHouse/issues/37715): Fix unexpected errors with a clash of constant strings in aggregate function, prewhere and join. Close [#36891](https://github.com/ClickHouse/ClickHouse/issues/36891). [#37336](https://github.com/ClickHouse/ClickHouse/pull/37336) ([Vladimir C](https://github.com/vdimir)). * Backported in [#37511](https://github.com/ClickHouse/ClickHouse/issues/37511): Fix logical error in normalizeUTF8 functions. Closes [#37298](https://github.com/ClickHouse/ClickHouse/issues/37298). [#37443](https://github.com/ClickHouse/ClickHouse/pull/37443) ([Maksim Kita](https://github.com/kitaisreal)). * Backported in [#37843](https://github.com/ClickHouse/ClickHouse/issues/37843): Fix segmentation fault in `show create table` from mysql database when it is configured with named collections. Closes [#37683](https://github.com/ClickHouse/ClickHouse/issues/37683). [#37690](https://github.com/ClickHouse/ClickHouse/pull/37690) ([Kseniia Sumarokova](https://github.com/kssenii)). * Backported in [#37940](https://github.com/ClickHouse/ClickHouse/issues/37940): Fix setting cast_ipv4_ipv6_default_on_conversion_error for internal cast function. Closes [#35156](https://github.com/ClickHouse/ClickHouse/issues/35156). [#37761](https://github.com/ClickHouse/ClickHouse/pull/37761) ([Maksim Kita](https://github.com/kitaisreal)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#37926](https://github.com/ClickHouse/ClickHouse/issues/37926): Fix check asof join key nullability, close [#35565](https://github.com/ClickHouse/ClickHouse/issues/35565). [#35674](https://github.com/ClickHouse/ClickHouse/pull/35674) ([Vladimir C](https://github.com/vdimir)). * Backported in [#37172](https://github.com/ClickHouse/ClickHouse/issues/37172): Fix bug in indexes of not presented columns in -WithNames formats that led to error `INCORRECT_NUMBER_OF_COLUMNS ` when the number of columns is more than 256. Closes [#35793](https://github.com/ClickHouse/ClickHouse/issues/35793). [#35803](https://github.com/ClickHouse/ClickHouse/pull/35803) ([Kruglov Pavel](https://github.com/Avogar)). diff --git a/docs/changelogs/v22.3.8.39-lts.md b/docs/changelogs/v22.3.8.39-lts.md index 8fff7f00a01..7f9363207d3 100644 --- a/docs/changelogs/v22.3.8.39-lts.md +++ b/docs/changelogs/v22.3.8.39-lts.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#38826](https://github.com/ClickHouse/ClickHouse/issues/38826): - Change `all|noarch` packages to architecture-dependent - Fix some documentation for it - Push aarch64|arm64 packages to artifactory and release assets - Fixes [#36443](https://github.com/ClickHouse/ClickHouse/issues/36443). [#38580](https://github.com/ClickHouse/ClickHouse/pull/38580) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#38453](https://github.com/ClickHouse/ClickHouse/issues/38453): Fix bug with nested short-circuit functions that led to execution of arguments even if condition is false. Closes [#38040](https://github.com/ClickHouse/ClickHouse/issues/38040). [#38173](https://github.com/ClickHouse/ClickHouse/pull/38173) ([Kruglov Pavel](https://github.com/Avogar)). * Backported in [#38710](https://github.com/ClickHouse/ClickHouse/issues/38710): Fix incorrect result of distributed queries with `DISTINCT` and `LIMIT`. Fixes [#38282](https://github.com/ClickHouse/ClickHouse/issues/38282). [#38371](https://github.com/ClickHouse/ClickHouse/pull/38371) ([Anton Popov](https://github.com/CurtizJ)). @@ -18,7 +18,7 @@ sidebar_label: 2022 * Backported in [#38776](https://github.com/ClickHouse/ClickHouse/issues/38776): `rankCorr` function will work correctly if some arguments are NaNs. This closes [#38396](https://github.com/ClickHouse/ClickHouse/issues/38396). [#38722](https://github.com/ClickHouse/ClickHouse/pull/38722) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Backported in [#38780](https://github.com/ClickHouse/ClickHouse/issues/38780): Fix use-after-free for Map combinator that leads to incorrect result. [#38748](https://github.com/ClickHouse/ClickHouse/pull/38748) ([Azat Khuzhin](https://github.com/azat)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#36818](https://github.com/ClickHouse/ClickHouse/issues/36818): Fix projection analysis which might lead to wrong query result when IN subquery is used. This fixes [#35336](https://github.com/ClickHouse/ClickHouse/issues/35336). [#35631](https://github.com/ClickHouse/ClickHouse/pull/35631) ([Amos Bird](https://github.com/amosbird)). * Backported in [#38467](https://github.com/ClickHouse/ClickHouse/issues/38467): - Fix potential error with literals in `WHERE` for join queries. Close [#36279](https://github.com/ClickHouse/ClickHouse/issues/36279). [#36542](https://github.com/ClickHouse/ClickHouse/pull/36542) ([Vladimir C](https://github.com/vdimir)). diff --git a/docs/changelogs/v22.3.9.19-lts.md b/docs/changelogs/v22.3.9.19-lts.md index 084793f11cc..e0ebcd9b7b8 100644 --- a/docs/changelogs/v22.3.9.19-lts.md +++ b/docs/changelogs/v22.3.9.19-lts.md @@ -7,7 +7,7 @@ sidebar_label: 2022 ### ClickHouse release v22.3.9.19-lts (7976930b82e) FIXME as compared to v22.3.8.39-lts (6bcf982f58b) -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#39097](https://github.com/ClickHouse/ClickHouse/issues/39097): Any allocations inside OvercommitTracker may lead to deadlock. Logging was not very informative so it's easier just to remove logging. Fixes [#37794](https://github.com/ClickHouse/ClickHouse/issues/37794). [#39030](https://github.com/ClickHouse/ClickHouse/pull/39030) ([Dmitry Novik](https://github.com/novikd)). * Backported in [#39080](https://github.com/ClickHouse/ClickHouse/issues/39080): Fix bug in filesystem cache that could happen in some corner case which coincided with cache capacity hitting the limit. Closes [#39066](https://github.com/ClickHouse/ClickHouse/issues/39066). [#39070](https://github.com/ClickHouse/ClickHouse/pull/39070) ([Kseniia Sumarokova](https://github.com/kssenii)). diff --git a/docs/changelogs/v22.4.1.2305-prestable.md b/docs/changelogs/v22.4.1.2305-prestable.md index 04ad0bf8955..41f57454d0d 100644 --- a/docs/changelogs/v22.4.1.2305-prestable.md +++ b/docs/changelogs/v22.4.1.2305-prestable.md @@ -161,7 +161,7 @@ sidebar_label: 2022 * call RemoteQueryExecutor with original_query instead of an rewritten query, elimate the AMBIGUOUS_COLUMN_NAME exception. [#35748](https://github.com/ClickHouse/ClickHouse/pull/35748) ([lgbo](https://github.com/lgbo-ustc)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Disallow ALTER TTL for engines that does not support it, to avoid breaking ATTACH TABLE (closes [#33344](https://github.com/ClickHouse/ClickHouse/issues/33344)). [#33391](https://github.com/ClickHouse/ClickHouse/pull/33391) ([zhongyuankai](https://github.com/zhongyuankai)). * Do not delay final part writing by default (fixes possible `Memory limit exceeded` during `INSERT` by adding `max_insert_delayed_streams_for_parallel_write` with default to 1000 for writes to s3 and disabled as before otherwise). [#34780](https://github.com/ClickHouse/ClickHouse/pull/34780) ([Azat Khuzhin](https://github.com/azat)). diff --git a/docs/changelogs/v22.4.3.3-stable.md b/docs/changelogs/v22.4.3.3-stable.md index 69f95d8cd27..58d0f7b69a1 100644 --- a/docs/changelogs/v22.4.3.3-stable.md +++ b/docs/changelogs/v22.4.3.3-stable.md @@ -7,6 +7,6 @@ sidebar_label: 2022 ### ClickHouse release v22.4.3.3-stable (def956d6299) FIXME as compared to v22.4.2.1-stable (b34ebdc36ae) -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#36582](https://github.com/ClickHouse/ClickHouse/issues/36582): Fix nullptr dereference in JOIN and COLUMNS matcher. This fixes [#36416](https://github.com/ClickHouse/ClickHouse/issues/36416) . This is for https://github.com/ClickHouse/ClickHouse/pull/36417. [#36430](https://github.com/ClickHouse/ClickHouse/pull/36430) ([Amos Bird](https://github.com/amosbird)). diff --git a/docs/changelogs/v22.4.4.7-stable.md b/docs/changelogs/v22.4.4.7-stable.md index 1dce0c50e0a..af94ecafcf6 100644 --- a/docs/changelogs/v22.4.4.7-stable.md +++ b/docs/changelogs/v22.4.4.7-stable.md @@ -7,7 +7,7 @@ sidebar_label: 2022 ### ClickHouse release v22.4.4.7-stable (ba44414f9b3) FIXME as compared to v22.4.3.3-stable (def956d6299) -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#36524](https://github.com/ClickHouse/ClickHouse/issues/36524): Queries with aliases inside special operators returned parsing error (was broken in 22.1). Example: `SELECT substring('test' AS t, 1, 1)`. [#36167](https://github.com/ClickHouse/ClickHouse/pull/36167) ([Maksim Kita](https://github.com/kitaisreal)). * Backported in [#36673](https://github.com/ClickHouse/ClickHouse/issues/36673): Fix merges of wide parts with type `Object`. [#36637](https://github.com/ClickHouse/ClickHouse/pull/36637) ([Anton Popov](https://github.com/CurtizJ)). diff --git a/docs/changelogs/v22.4.5.9-stable.md b/docs/changelogs/v22.4.5.9-stable.md index 50cec91b12c..524b309f8ec 100644 --- a/docs/changelogs/v22.4.5.9-stable.md +++ b/docs/changelogs/v22.4.5.9-stable.md @@ -7,7 +7,7 @@ sidebar_label: 2022 ### ClickHouse release v22.4.5.9-stable (059ef6cadcd) FIXME as compared to v22.4.4.7-stable (ba44414f9b3) -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#36635](https://github.com/ClickHouse/ClickHouse/issues/36635): Fix `Missing column` exception which could happen while using `INTERPOLATE` with `ENGINE = MergeTree` table. [#36549](https://github.com/ClickHouse/ClickHouse/pull/36549) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). * Backported in [#36794](https://github.com/ClickHouse/ClickHouse/issues/36794): Fix vertical merges in wide parts. Previously an exception `There is no column` can be thrown during merge. [#36707](https://github.com/ClickHouse/ClickHouse/pull/36707) ([Anton Popov](https://github.com/CurtizJ)). diff --git a/docs/changelogs/v22.4.6.53-stable.md b/docs/changelogs/v22.4.6.53-stable.md index 5dc25697c20..b6380d9c7ba 100644 --- a/docs/changelogs/v22.4.6.53-stable.md +++ b/docs/changelogs/v22.4.6.53-stable.md @@ -13,7 +13,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#38828](https://github.com/ClickHouse/ClickHouse/issues/38828): - Change `all|noarch` packages to architecture-dependent - Fix some documentation for it - Push aarch64|arm64 packages to artifactory and release assets - Fixes [#36443](https://github.com/ClickHouse/ClickHouse/issues/36443). [#38580](https://github.com/ClickHouse/ClickHouse/pull/38580) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#37717](https://github.com/ClickHouse/ClickHouse/issues/37717): Fix unexpected errors with a clash of constant strings in aggregate function, prewhere and join. Close [#36891](https://github.com/ClickHouse/ClickHouse/issues/36891). [#37336](https://github.com/ClickHouse/ClickHouse/pull/37336) ([Vladimir C](https://github.com/vdimir)). * Backported in [#37512](https://github.com/ClickHouse/ClickHouse/issues/37512): Fix logical error in normalizeUTF8 functions. Closes [#37298](https://github.com/ClickHouse/ClickHouse/issues/37298). [#37443](https://github.com/ClickHouse/ClickHouse/pull/37443) ([Maksim Kita](https://github.com/kitaisreal)). @@ -27,7 +27,7 @@ sidebar_label: 2022 * Backported in [#38777](https://github.com/ClickHouse/ClickHouse/issues/38777): `rankCorr` function will work correctly if some arguments are NaNs. This closes [#38396](https://github.com/ClickHouse/ClickHouse/issues/38396). [#38722](https://github.com/ClickHouse/ClickHouse/pull/38722) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Backported in [#38781](https://github.com/ClickHouse/ClickHouse/issues/38781): Fix use-after-free for Map combinator that leads to incorrect result. [#38748](https://github.com/ClickHouse/ClickHouse/pull/38748) ([Azat Khuzhin](https://github.com/azat)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Backported in [#37456](https://github.com/ClickHouse/ClickHouse/issues/37456): Server might fail to start if it cannot resolve hostname of external ClickHouse dictionary. It's fixed. Fixes [#36451](https://github.com/ClickHouse/ClickHouse/issues/36451). [#36463](https://github.com/ClickHouse/ClickHouse/pull/36463) ([Alexander Tokmakov](https://github.com/tavplubix)). * Backported in [#38468](https://github.com/ClickHouse/ClickHouse/issues/38468): - Fix potential error with literals in `WHERE` for join queries. Close [#36279](https://github.com/ClickHouse/ClickHouse/issues/36279). [#36542](https://github.com/ClickHouse/ClickHouse/pull/36542) ([Vladimir C](https://github.com/vdimir)). diff --git a/docs/changelogs/v22.5.1.2079-stable.md b/docs/changelogs/v22.5.1.2079-stable.md index fdd6325fd1f..28dfa0825cf 100644 --- a/docs/changelogs/v22.5.1.2079-stable.md +++ b/docs/changelogs/v22.5.1.2079-stable.md @@ -104,7 +104,7 @@ sidebar_label: 2022 * ClickHouse builds for `PowerPC64LE` architecture are now available in universal installation script `curl https://clickhouse.com/ | sh` and by direct link `https://builds.clickhouse.com/master/powerpc64le/clickhouse`. [#37095](https://github.com/ClickHouse/ClickHouse/pull/37095) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * - Make cmake build scripts a bit more robust. [#37169](https://github.com/ClickHouse/ClickHouse/pull/37169) ([Robert Schulze](https://github.com/rschu1ze)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * The ilike() function on FixedString columns could have returned wrong results (i.e. match less than it should). [#37117](https://github.com/ClickHouse/ClickHouse/pull/37117) ([Robert Schulze](https://github.com/rschu1ze)). * Fix implicit cast for optimize_skip_unused_shards_rewrite_in. [#37153](https://github.com/ClickHouse/ClickHouse/pull/37153) ([Azat Khuzhin](https://github.com/azat)). @@ -120,7 +120,7 @@ sidebar_label: 2022 * Fix system.opentelemetry_span_log attribute.values alias to values instead of keys. [#37275](https://github.com/ClickHouse/ClickHouse/pull/37275) ([Aleksandr Razumov](https://github.com/ernado)). * Fix possible deadlock in OvercommitTracker during logging. cc @alesapin @tavplubix Fixes [#37272](https://github.com/ClickHouse/ClickHouse/issues/37272). [#37299](https://github.com/ClickHouse/ClickHouse/pull/37299) ([Dmitry Novik](https://github.com/novikd)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * - fix substring function range error length when `offset` and `length` is negative constant and `s` is not constant. [#33861](https://github.com/ClickHouse/ClickHouse/pull/33861) ([RogerYK](https://github.com/RogerYK)). * Accidentally ZSTD support for Arrow was not being built. This fixes [#35283](https://github.com/ClickHouse/ClickHouse/issues/35283). [#35486](https://github.com/ClickHouse/ClickHouse/pull/35486) ([Sean Lafferty](https://github.com/seanlaff)). diff --git a/docs/changelogs/v22.5.2.53-stable.md b/docs/changelogs/v22.5.2.53-stable.md index f2fb50b3e14..8af0c7dac45 100644 --- a/docs/changelogs/v22.5.2.53-stable.md +++ b/docs/changelogs/v22.5.2.53-stable.md @@ -13,7 +13,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#38827](https://github.com/ClickHouse/ClickHouse/issues/38827): - Change `all|noarch` packages to architecture-dependent - Fix some documentation for it - Push aarch64|arm64 packages to artifactory and release assets - Fixes [#36443](https://github.com/ClickHouse/ClickHouse/issues/36443). [#38580](https://github.com/ClickHouse/ClickHouse/pull/38580) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#37716](https://github.com/ClickHouse/ClickHouse/issues/37716): Fix unexpected errors with a clash of constant strings in aggregate function, prewhere and join. Close [#36891](https://github.com/ClickHouse/ClickHouse/issues/36891). [#37336](https://github.com/ClickHouse/ClickHouse/pull/37336) ([Vladimir C](https://github.com/vdimir)). * Backported in [#37408](https://github.com/ClickHouse/ClickHouse/issues/37408): Throw an exception when GROUPING SETS used with ROLLUP or CUBE. [#37367](https://github.com/ClickHouse/ClickHouse/pull/37367) ([Dmitry Novik](https://github.com/novikd)). diff --git a/docs/changelogs/v22.5.3.21-stable.md b/docs/changelogs/v22.5.3.21-stable.md index 994bf79f7a8..4915a41a10c 100644 --- a/docs/changelogs/v22.5.3.21-stable.md +++ b/docs/changelogs/v22.5.3.21-stable.md @@ -7,7 +7,7 @@ sidebar_label: 2022 ### ClickHouse release v22.5.3.21-stable (e03724efec5) FIXME as compared to v22.5.2.53-stable (5fd600fda9e) -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#38241](https://github.com/ClickHouse/ClickHouse/issues/38241): Fix possible crash in `Distributed` async insert in case of removing a replica from config. [#38029](https://github.com/ClickHouse/ClickHouse/pull/38029) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Backported in [#39098](https://github.com/ClickHouse/ClickHouse/issues/39098): Any allocations inside OvercommitTracker may lead to deadlock. Logging was not very informative so it's easier just to remove logging. Fixes [#37794](https://github.com/ClickHouse/ClickHouse/issues/37794). [#39030](https://github.com/ClickHouse/ClickHouse/pull/39030) ([Dmitry Novik](https://github.com/novikd)). diff --git a/docs/changelogs/v22.5.4.19-stable.md b/docs/changelogs/v22.5.4.19-stable.md index 24903260904..c90f45c770e 100644 --- a/docs/changelogs/v22.5.4.19-stable.md +++ b/docs/changelogs/v22.5.4.19-stable.md @@ -13,7 +13,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#39882](https://github.com/ClickHouse/ClickHouse/issues/39882): Former packages used to install systemd.service file to `/etc`. The files there are marked as `conf` and are not cleaned out, and not updated automatically. This PR cleans them out. [#39323](https://github.com/ClickHouse/ClickHouse/pull/39323) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#39209](https://github.com/ClickHouse/ClickHouse/issues/39209): Fix reading of sparse columns from `MergeTree` tables that store their data in S3. [#37978](https://github.com/ClickHouse/ClickHouse/pull/37978) ([Anton Popov](https://github.com/CurtizJ)). * Backported in [#39589](https://github.com/ClickHouse/ClickHouse/issues/39589): Fix data race and possible heap-buffer-overflow in Avro format. Closes [#39094](https://github.com/ClickHouse/ClickHouse/issues/39094) Closes [#33652](https://github.com/ClickHouse/ClickHouse/issues/33652). [#39498](https://github.com/ClickHouse/ClickHouse/pull/39498) ([Kruglov Pavel](https://github.com/Avogar)). diff --git a/docs/changelogs/v22.6.1.1985-stable.md b/docs/changelogs/v22.6.1.1985-stable.md index 0437f682789..c915d24fe00 100644 --- a/docs/changelogs/v22.6.1.1985-stable.md +++ b/docs/changelogs/v22.6.1.1985-stable.md @@ -119,7 +119,7 @@ sidebar_label: 2022 * Fix overly aggressive stripping which removed the embedded hash required for checking the consistency of the executable. [#37993](https://github.com/ClickHouse/ClickHouse/pull/37993) ([Robert Schulze](https://github.com/rschu1ze)). * fix MacOS build compressor faild. [#38007](https://github.com/ClickHouse/ClickHouse/pull/38007) ([chen](https://github.com/xiedeyantu)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Fix `GROUP BY` `AggregateFunction` (i.e. you `GROUP BY` by the column that has `AggregateFunction` type). [#37093](https://github.com/ClickHouse/ClickHouse/pull/37093) ([Azat Khuzhin](https://github.com/azat)). * Fix possible heap-use-after-free error when reading system.projection_parts and system.projection_parts_columns . This fixes [#37184](https://github.com/ClickHouse/ClickHouse/issues/37184). [#37185](https://github.com/ClickHouse/ClickHouse/pull/37185) ([Amos Bird](https://github.com/amosbird)). @@ -169,7 +169,7 @@ sidebar_label: 2022 * when open enable_filesystem_query_cache_limit, throw Reserved cache size exceeds the remaining cache size. [#38004](https://github.com/ClickHouse/ClickHouse/pull/38004) ([chen](https://github.com/xiedeyantu)). * Query, containing ORDER BY ... WITH FILL, can generate extra rows when multiple WITH FILL columns are present. [#38074](https://github.com/ClickHouse/ClickHouse/pull/38074) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Fix converting types for UNION queries (may produce LOGICAL_ERROR). [#34775](https://github.com/ClickHouse/ClickHouse/pull/34775) ([Azat Khuzhin](https://github.com/azat)). * TTL merge may not be scheduled again if BackgroundExecutor is busy. --merges_with_ttl_counter is increased in selectPartsToMerge() --merge task will be ignored if BackgroundExecutor is busy --merges_with_ttl_counter will not be decrease. [#36387](https://github.com/ClickHouse/ClickHouse/pull/36387) ([lthaooo](https://github.com/lthaooo)). diff --git a/docs/changelogs/v22.6.2.12-stable.md b/docs/changelogs/v22.6.2.12-stable.md index d8c1cd31936..3c0f2116f1d 100644 --- a/docs/changelogs/v22.6.2.12-stable.md +++ b/docs/changelogs/v22.6.2.12-stable.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Improvement * Backported in [#38484](https://github.com/ClickHouse/ClickHouse/issues/38484): Improve the stability for hive storage integration test. Move the data prepare step into test.py. [#38260](https://github.com/ClickHouse/ClickHouse/pull/38260) ([lgbo](https://github.com/lgbo-ustc)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#38404](https://github.com/ClickHouse/ClickHouse/issues/38404): Fix bug with nested short-circuit functions that led to execution of arguments even if condition is false. Closes [#38040](https://github.com/ClickHouse/ClickHouse/issues/38040). [#38173](https://github.com/ClickHouse/ClickHouse/pull/38173) ([Kruglov Pavel](https://github.com/Avogar)). diff --git a/docs/changelogs/v22.6.3.35-stable.md b/docs/changelogs/v22.6.3.35-stable.md index 0a86c136d35..86b8282e075 100644 --- a/docs/changelogs/v22.6.3.35-stable.md +++ b/docs/changelogs/v22.6.3.35-stable.md @@ -13,7 +13,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#38883](https://github.com/ClickHouse/ClickHouse/issues/38883): Add `clickhouse-diagnostics` binary to the packages. [#38647](https://github.com/ClickHouse/ClickHouse/pull/38647) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#38690](https://github.com/ClickHouse/ClickHouse/issues/38690): Fix incorrect columns order in subqueries of UNION (in case of duplicated columns in subselects may produce incorrect result). [#37887](https://github.com/ClickHouse/ClickHouse/pull/37887) ([Azat Khuzhin](https://github.com/azat)). * Backported in [#38500](https://github.com/ClickHouse/ClickHouse/issues/38500): Do not allow recursive usage of OvercommitTracker during logging. Fixes [#37794](https://github.com/ClickHouse/ClickHouse/issues/37794) cc @tavplubix @davenger. [#38246](https://github.com/ClickHouse/ClickHouse/pull/38246) ([Dmitry Novik](https://github.com/novikd)). diff --git a/docs/changelogs/v22.6.4.35-stable.md b/docs/changelogs/v22.6.4.35-stable.md index 5c4644f735a..2b5537b5bf9 100644 --- a/docs/changelogs/v22.6.4.35-stable.md +++ b/docs/changelogs/v22.6.4.35-stable.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#38822](https://github.com/ClickHouse/ClickHouse/issues/38822): - Change `all|noarch` packages to architecture-dependent - Fix some documentation for it - Push aarch64|arm64 packages to artifactory and release assets - Fixes [#36443](https://github.com/ClickHouse/ClickHouse/issues/36443). [#38580](https://github.com/ClickHouse/ClickHouse/pull/38580) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#38242](https://github.com/ClickHouse/ClickHouse/issues/38242): Fix possible crash in `Distributed` async insert in case of removing a replica from config. [#38029](https://github.com/ClickHouse/ClickHouse/pull/38029) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Backported in [#38865](https://github.com/ClickHouse/ClickHouse/issues/38865): Fix s3 seekable reads with parallel read buffer. (Affected memory usage during query). Closes [#38258](https://github.com/ClickHouse/ClickHouse/issues/38258). [#38802](https://github.com/ClickHouse/ClickHouse/pull/38802) ([Kseniia Sumarokova](https://github.com/kssenii)). diff --git a/docs/changelogs/v22.6.5.22-stable.md b/docs/changelogs/v22.6.5.22-stable.md index 5965f63df14..edb6bdf7c2d 100644 --- a/docs/changelogs/v22.6.5.22-stable.md +++ b/docs/changelogs/v22.6.5.22-stable.md @@ -13,7 +13,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#39883](https://github.com/ClickHouse/ClickHouse/issues/39883): Former packages used to install systemd.service file to `/etc`. The files there are marked as `conf` and are not cleaned out, and not updated automatically. This PR cleans them out. [#39323](https://github.com/ClickHouse/ClickHouse/pull/39323) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#39207](https://github.com/ClickHouse/ClickHouse/issues/39207): Fix reading of sparse columns from `MergeTree` tables that store their data in S3. [#37978](https://github.com/ClickHouse/ClickHouse/pull/37978) ([Anton Popov](https://github.com/CurtizJ)). * Backported in [#38932](https://github.com/ClickHouse/ClickHouse/issues/38932): Fix `parallel_view_processing=1` with `optimize_trivial_insert_select=1`. Fix `max_insert_threads` while pushing to views. [#38731](https://github.com/ClickHouse/ClickHouse/pull/38731) ([Azat Khuzhin](https://github.com/azat)). diff --git a/docs/changelogs/v22.6.6.16-stable.md b/docs/changelogs/v22.6.6.16-stable.md index 4d44621454b..d4d25771326 100644 --- a/docs/changelogs/v22.6.6.16-stable.md +++ b/docs/changelogs/v22.6.6.16-stable.md @@ -7,7 +7,7 @@ sidebar_label: 2022 ### ClickHouse release v22.6.6.16-stable (d2a33ebc822) FIXME as compared to v22.6.5.22-stable (47ca5f14a34) -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#40162](https://github.com/ClickHouse/ClickHouse/issues/40162): fix HashMethodOneNumber get wrong key value when column is const. [#40020](https://github.com/ClickHouse/ClickHouse/pull/40020) ([Duc Canh Le](https://github.com/canhld94)). * Backported in [#40124](https://github.com/ClickHouse/ClickHouse/issues/40124): Fix bug in collectFilesToSkip() by adding correct file extension(.idx or idx2) for indexes to be recalculated, avoid wrong hard links. Fixed [#39896](https://github.com/ClickHouse/ClickHouse/issues/39896). [#40095](https://github.com/ClickHouse/ClickHouse/pull/40095) ([Jianmei Zhang](https://github.com/zhangjmruc)). diff --git a/docs/changelogs/v22.6.7.7-stable.md b/docs/changelogs/v22.6.7.7-stable.md index f5351cc03ed..0b4cc6836f7 100644 --- a/docs/changelogs/v22.6.7.7-stable.md +++ b/docs/changelogs/v22.6.7.7-stable.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#40692](https://github.com/ClickHouse/ClickHouse/issues/40692): Fix TGZ packages. [#40681](https://github.com/ClickHouse/ClickHouse/pull/40681) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#40531](https://github.com/ClickHouse/ClickHouse/issues/40531): Proxy resolver stop on first successful request to endpoint. [#40353](https://github.com/ClickHouse/ClickHouse/pull/40353) ([Maksim Kita](https://github.com/kitaisreal)). * Backported in [#40623](https://github.com/ClickHouse/ClickHouse/issues/40623): Fix potential dataloss due to a bug in AWS SDK (https://github.com/aws/aws-sdk-cpp/issues/658). Bug can be triggered only when clickhouse is used over S3. [#40506](https://github.com/ClickHouse/ClickHouse/pull/40506) ([alesapin](https://github.com/alesapin)). diff --git a/docs/changelogs/v22.6.8.35-stable.md b/docs/changelogs/v22.6.8.35-stable.md index e68384d3d9c..b69cabffd4d 100644 --- a/docs/changelogs/v22.6.8.35-stable.md +++ b/docs/changelogs/v22.6.8.35-stable.md @@ -13,7 +13,7 @@ sidebar_label: 2022 #### Bug Fix * Backported in [#41274](https://github.com/ClickHouse/ClickHouse/issues/41274): Fix memory safety issues with functions `encrypt` and `contingency` if Array of Nullable is used as an argument. This fixes [#41004](https://github.com/ClickHouse/ClickHouse/issues/41004). [#40195](https://github.com/ClickHouse/ClickHouse/pull/40195) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#41282](https://github.com/ClickHouse/ClickHouse/issues/41282): Fix unused unknown columns introduced by WITH statement. This fixes [#37812](https://github.com/ClickHouse/ClickHouse/issues/37812) . [#39131](https://github.com/ClickHouse/ClickHouse/pull/39131) ([Amos Bird](https://github.com/amosbird)). * Backported in [#40905](https://github.com/ClickHouse/ClickHouse/issues/40905): Fix potential deadlock in WriteBufferFromS3 during task scheduling failure. [#40070](https://github.com/ClickHouse/ClickHouse/pull/40070) ([Maksim Kita](https://github.com/kitaisreal)). diff --git a/docs/changelogs/v22.6.9.11-stable.md b/docs/changelogs/v22.6.9.11-stable.md index 5beb9171d9e..42cf0948ecc 100644 --- a/docs/changelogs/v22.6.9.11-stable.md +++ b/docs/changelogs/v22.6.9.11-stable.md @@ -13,7 +13,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#41558](https://github.com/ClickHouse/ClickHouse/issues/41558): Add `source` field to deb packages, update `nfpm`. [#41531](https://github.com/ClickHouse/ClickHouse/pull/41531) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#41504](https://github.com/ClickHouse/ClickHouse/issues/41504): Writing data in Apache `ORC` format might lead to a buffer overrun. [#41458](https://github.com/ClickHouse/ClickHouse/pull/41458) ([Alexey Milovidov](https://github.com/alexey-milovidov)). diff --git a/docs/changelogs/v22.7.1.2484-stable.md b/docs/changelogs/v22.7.1.2484-stable.md index 3f90b3691ea..7464b0449ee 100644 --- a/docs/changelogs/v22.7.1.2484-stable.md +++ b/docs/changelogs/v22.7.1.2484-stable.md @@ -128,7 +128,7 @@ sidebar_label: 2022 * Fix LSan by fixing getauxval(). [#39299](https://github.com/ClickHouse/ClickHouse/pull/39299) ([Azat Khuzhin](https://github.com/azat)). * Adapt universal installation script for FreeBSD. [#39302](https://github.com/ClickHouse/ClickHouse/pull/39302) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Fix projection exception when aggregation keys are wrapped inside other functions. This fixes [#37151](https://github.com/ClickHouse/ClickHouse/issues/37151). [#37155](https://github.com/ClickHouse/ClickHouse/pull/37155) ([Amos Bird](https://github.com/amosbird)). * Fix possible logical error `... with argument with type Nothing and default implementation for Nothing is expected to return result with type Nothing, got ...` in some functions. Closes: [#37610](https://github.com/ClickHouse/ClickHouse/issues/37610) Closes: [#37741](https://github.com/ClickHouse/ClickHouse/issues/37741). [#37759](https://github.com/ClickHouse/ClickHouse/pull/37759) ([Kruglov Pavel](https://github.com/Avogar)). @@ -198,7 +198,7 @@ sidebar_label: 2022 * Fix UB (stack-use-after-scope) in extactAll(). [#39397](https://github.com/ClickHouse/ClickHouse/pull/39397) ([Azat Khuzhin](https://github.com/azat)). * Fix incorrect query result when trivial count optimization is in effect with array join. This fixes [#39431](https://github.com/ClickHouse/ClickHouse/issues/39431). [#39444](https://github.com/ClickHouse/ClickHouse/pull/39444) ([Amos Bird](https://github.com/amosbird)). -#### Bug Fix (user-visible misbehaviour in official stable or prestable release) +#### Bug Fix (user-visible misbehaviour in official stable release) * Disable send_logs_level for INSERT into Distributed to avoid possible hung. [#35075](https://github.com/ClickHouse/ClickHouse/pull/35075) ([Azat Khuzhin](https://github.com/azat)). diff --git a/docs/changelogs/v22.7.2.15-stable.md b/docs/changelogs/v22.7.2.15-stable.md index a9db8bcf10d..0a3748f90f7 100644 --- a/docs/changelogs/v22.7.2.15-stable.md +++ b/docs/changelogs/v22.7.2.15-stable.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Bug Fix * Backported in [#39750](https://github.com/ClickHouse/ClickHouse/issues/39750): Fix seeking while reading from encrypted disk. This PR fixes [#38381](https://github.com/ClickHouse/ClickHouse/issues/38381). [#39687](https://github.com/ClickHouse/ClickHouse/pull/39687) ([Vitaly Baranov](https://github.com/vitlibar)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#39591](https://github.com/ClickHouse/ClickHouse/issues/39591): Fix data race and possible heap-buffer-overflow in Avro format. Closes [#39094](https://github.com/ClickHouse/ClickHouse/issues/39094) Closes [#33652](https://github.com/ClickHouse/ClickHouse/issues/33652). [#39498](https://github.com/ClickHouse/ClickHouse/pull/39498) ([Kruglov Pavel](https://github.com/Avogar)). * Backported in [#39613](https://github.com/ClickHouse/ClickHouse/issues/39613): Fix bug with maxsplit argument for splitByChar, which was not working correctly. [#39552](https://github.com/ClickHouse/ClickHouse/pull/39552) ([filimonov](https://github.com/filimonov)). diff --git a/docs/changelogs/v22.7.3.5-stable.md b/docs/changelogs/v22.7.3.5-stable.md index 62a5dfee611..b55b16509d4 100644 --- a/docs/changelogs/v22.7.3.5-stable.md +++ b/docs/changelogs/v22.7.3.5-stable.md @@ -11,7 +11,7 @@ sidebar_label: 2022 * Backported in [#39884](https://github.com/ClickHouse/ClickHouse/issues/39884): Former packages used to install systemd.service file to `/etc`. The files there are marked as `conf` and are not cleaned out, and not updated automatically. This PR cleans them out. [#39323](https://github.com/ClickHouse/ClickHouse/pull/39323) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Backported in [#39884](https://github.com/ClickHouse/ClickHouse/issues/39884): Former packages used to install systemd.service file to `/etc`. The files there are marked as `conf` and are not cleaned out, and not updated automatically. This PR cleans them out. [#39323](https://github.com/ClickHouse/ClickHouse/pull/39323) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#40045](https://github.com/ClickHouse/ClickHouse/issues/40045): Fix big memory usage during fetches. Fixes [#39915](https://github.com/ClickHouse/ClickHouse/issues/39915). [#39990](https://github.com/ClickHouse/ClickHouse/pull/39990) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Backported in [#40045](https://github.com/ClickHouse/ClickHouse/issues/40045): Fix big memory usage during fetches. Fixes [#39915](https://github.com/ClickHouse/ClickHouse/issues/39915). [#39990](https://github.com/ClickHouse/ClickHouse/pull/39990) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). diff --git a/docs/changelogs/v22.7.4.16-stable.md b/docs/changelogs/v22.7.4.16-stable.md index 52d68283a2f..4847ef8cf64 100644 --- a/docs/changelogs/v22.7.4.16-stable.md +++ b/docs/changelogs/v22.7.4.16-stable.md @@ -7,7 +7,7 @@ sidebar_label: 2022 ### ClickHouse release v22.7.4.16-stable (0b9272f8fdc) FIXME as compared to v22.7.3.5-stable (e140b8b5f3a) -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#40163](https://github.com/ClickHouse/ClickHouse/issues/40163): fix HashMethodOneNumber get wrong key value when column is const. [#40020](https://github.com/ClickHouse/ClickHouse/pull/40020) ([Duc Canh Le](https://github.com/canhld94)). * Backported in [#40125](https://github.com/ClickHouse/ClickHouse/issues/40125): Fix bug in collectFilesToSkip() by adding correct file extension(.idx or idx2) for indexes to be recalculated, avoid wrong hard links. Fixed [#39896](https://github.com/ClickHouse/ClickHouse/issues/39896). [#40095](https://github.com/ClickHouse/ClickHouse/pull/40095) ([Jianmei Zhang](https://github.com/zhangjmruc)). diff --git a/docs/changelogs/v22.7.5.13-stable.md b/docs/changelogs/v22.7.5.13-stable.md index 0de9e9a26aa..24dbc5c9e7f 100644 --- a/docs/changelogs/v22.7.5.13-stable.md +++ b/docs/changelogs/v22.7.5.13-stable.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#40693](https://github.com/ClickHouse/ClickHouse/issues/40693): Fix TGZ packages. [#40681](https://github.com/ClickHouse/ClickHouse/pull/40681) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#40542](https://github.com/ClickHouse/ClickHouse/issues/40542): Fix potential deadlock in WriteBufferFromS3 during task scheduling failure. [#40070](https://github.com/ClickHouse/ClickHouse/pull/40070) ([Maksim Kita](https://github.com/kitaisreal)). * Backported in [#40450](https://github.com/ClickHouse/ClickHouse/issues/40450): Fix rare bug with column TTL for MergeTree engines family: In case of repeated vertical merge the error `Cannot unlink file ColumnName.bin ... No such file or directory.` could happen. [#40346](https://github.com/ClickHouse/ClickHouse/pull/40346) ([alesapin](https://github.com/alesapin)). diff --git a/docs/changelogs/v22.7.6.74-stable.md b/docs/changelogs/v22.7.6.74-stable.md index 9060375ce8e..3cf2edfccd1 100644 --- a/docs/changelogs/v22.7.6.74-stable.md +++ b/docs/changelogs/v22.7.6.74-stable.md @@ -16,7 +16,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#41559](https://github.com/ClickHouse/ClickHouse/issues/41559): Add `source` field to deb packages, update `nfpm`. [#41531](https://github.com/ClickHouse/ClickHouse/pull/41531) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#41283](https://github.com/ClickHouse/ClickHouse/issues/41283): Fix unused unknown columns introduced by WITH statement. This fixes [#37812](https://github.com/ClickHouse/ClickHouse/issues/37812) . [#39131](https://github.com/ClickHouse/ClickHouse/pull/39131) ([Amos Bird](https://github.com/amosbird)). * Backported in [#40865](https://github.com/ClickHouse/ClickHouse/issues/40865): - Fix crash while parsing values of type `Object` that contains arrays of variadic dimension. [#40483](https://github.com/ClickHouse/ClickHouse/pull/40483) ([Duc Canh Le](https://github.com/canhld94)). diff --git a/docs/changelogs/v22.7.7.24-stable.md b/docs/changelogs/v22.7.7.24-stable.md index cc3a83c5d4c..16e56156ff6 100644 --- a/docs/changelogs/v22.7.7.24-stable.md +++ b/docs/changelogs/v22.7.7.24-stable.md @@ -14,7 +14,7 @@ sidebar_label: 2022 * Backported in [#42329](https://github.com/ClickHouse/ClickHouse/issues/42329): Update cctz to the latest master, update tzdb to 2020e. [#42273](https://github.com/ClickHouse/ClickHouse/pull/42273) ([Dom Del Nano](https://github.com/ddelnano)). * Backported in [#42359](https://github.com/ClickHouse/ClickHouse/issues/42359): Update tzdata to 2022e to support the new timezone changes. Palestine transitions are now Saturdays at 02:00. Simplify three Ukraine zones into one. Jordan and Syria switch from +02/+03 with DST to year-round +03. (https://data.iana.org/time-zones/tzdb/NEWS). This closes [#42252](https://github.com/ClickHouse/ClickHouse/issues/42252). [#42327](https://github.com/ClickHouse/ClickHouse/pull/42327) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#42268](https://github.com/ClickHouse/ClickHouse/issues/42268): Fix reusing of files > 4GB from base backup. [#42146](https://github.com/ClickHouse/ClickHouse/pull/42146) ([Azat Khuzhin](https://github.com/azat)). * Backported in [#42299](https://github.com/ClickHouse/ClickHouse/issues/42299): Fix a bug with projections and the `aggregate_functions_null_for_empty` setting. This bug is very rare and appears only if you enable the `aggregate_functions_null_for_empty` setting in the server's config. This closes [#41647](https://github.com/ClickHouse/ClickHouse/issues/41647). [#42198](https://github.com/ClickHouse/ClickHouse/pull/42198) ([Alexey Milovidov](https://github.com/alexey-milovidov)). diff --git a/docs/changelogs/v22.8.1.2097-lts.md b/docs/changelogs/v22.8.1.2097-lts.md index ef454a7e283..b6b634f4826 100644 --- a/docs/changelogs/v22.8.1.2097-lts.md +++ b/docs/changelogs/v22.8.1.2097-lts.md @@ -106,7 +106,7 @@ sidebar_label: 2022 * Support build with `clang-16` (trunk). This closes [#39949](https://github.com/ClickHouse/ClickHouse/issues/39949). [#40181](https://github.com/ClickHouse/ClickHouse/pull/40181) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Prepare RISC-V 64 build to run in CI. This is for [#40141](https://github.com/ClickHouse/ClickHouse/issues/40141). [#40197](https://github.com/ClickHouse/ClickHouse/pull/40197) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Fixed query hanging for SELECT with ORDER BY WITH FILL with different date/time types. [#37849](https://github.com/ClickHouse/ClickHouse/pull/37849) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). * Fix ORDER BY that matches projections ORDER BY (before it simply returns unsorted result). [#38725](https://github.com/ClickHouse/ClickHouse/pull/38725) ([Azat Khuzhin](https://github.com/azat)). diff --git a/docs/changelogs/v22.8.10.29-lts.md b/docs/changelogs/v22.8.10.29-lts.md index ac41d71650c..33ae27b6da8 100644 --- a/docs/changelogs/v22.8.10.29-lts.md +++ b/docs/changelogs/v22.8.10.29-lts.md @@ -13,7 +13,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#43051](https://github.com/ClickHouse/ClickHouse/issues/43051): Wait for all files are in sync before archiving them in integration tests. [#42891](https://github.com/ClickHouse/ClickHouse/pull/42891) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#43513](https://github.com/ClickHouse/ClickHouse/issues/43513): - Fix several buffer over-reads. [#43159](https://github.com/ClickHouse/ClickHouse/pull/43159) ([Raúl Marín](https://github.com/Algunenano)). * Backported in [#43428](https://github.com/ClickHouse/ClickHouse/issues/43428): Fixed queries with `SAMPLE BY` with prewhere optimization on tables using `Merge` engine. [#43315](https://github.com/ClickHouse/ClickHouse/pull/43315) ([Antonio Andelic](https://github.com/antonio2368)). diff --git a/docs/changelogs/v22.8.11.15-lts.md b/docs/changelogs/v22.8.11.15-lts.md index 337eeba9187..dbe765ca4a2 100644 --- a/docs/changelogs/v22.8.11.15-lts.md +++ b/docs/changelogs/v22.8.11.15-lts.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Bug Fix * Backported in [#43098](https://github.com/ClickHouse/ClickHouse/issues/43098): Updated normaliser to clone the alias ast. resolves [#42452](https://github.com/ClickHouse/ClickHouse/issues/42452) Implementation: * Updated QueryNormalizer to clone alias ast, when its replaced. Previously just assigning the same leads to exception in LogicalExpressinsOptimizer as it would be the same parent being inserted again. * This bug is not seen with new analyser (allow_experimental_analyzer), so no changes for it. I added a test for the same. [#42827](https://github.com/ClickHouse/ClickHouse/pull/42827) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#43751](https://github.com/ClickHouse/ClickHouse/issues/43751): An issue with the following exception has been reported while trying to read a Parquet file from S3 into ClickHouse:. [#43297](https://github.com/ClickHouse/ClickHouse/pull/43297) ([Arthur Passos](https://github.com/arthurpassos)). * Backported in [#43617](https://github.com/ClickHouse/ClickHouse/issues/43617): Fix sumMap() for Nullable(Decimal()). [#43414](https://github.com/ClickHouse/ClickHouse/pull/43414) ([Azat Khuzhin](https://github.com/azat)). diff --git a/docs/changelogs/v22.8.12.45-lts.md b/docs/changelogs/v22.8.12.45-lts.md index 7412784419c..9fab9daeb95 100644 --- a/docs/changelogs/v22.8.12.45-lts.md +++ b/docs/changelogs/v22.8.12.45-lts.md @@ -16,7 +16,7 @@ sidebar_label: 2023 * Backported in [#44378](https://github.com/ClickHouse/ClickHouse/issues/44378): In rare cases, we don't rebuild binaries, because another task with a similar prefix succeeded. E.g. `binary_darwin` didn't restart because `binary_darwin_aarch64`. [#44311](https://github.com/ClickHouse/ClickHouse/pull/44311) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Backported in [#44558](https://github.com/ClickHouse/ClickHouse/issues/44558): Retry the integration tests on compressing errors. [#44529](https://github.com/ClickHouse/ClickHouse/pull/44529) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#44751](https://github.com/ClickHouse/ClickHouse/issues/44751): [#40651](https://github.com/ClickHouse/ClickHouse/issues/40651) [#41404](https://github.com/ClickHouse/ClickHouse/issues/41404). [#42126](https://github.com/ClickHouse/ClickHouse/pull/42126) ([Alexander Gololobov](https://github.com/davenger)). * Backported in [#43525](https://github.com/ClickHouse/ClickHouse/issues/43525): Fix incorrect UserTimeMicroseconds/SystemTimeMicroseconds accounting. [#42791](https://github.com/ClickHouse/ClickHouse/pull/42791) ([Azat Khuzhin](https://github.com/azat)). @@ -39,4 +39,3 @@ sidebar_label: 2023 * Add check for submodules sanity [#44386](https://github.com/ClickHouse/ClickHouse/pull/44386) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Implement a custom central checkout action [#44399](https://github.com/ClickHouse/ClickHouse/pull/44399) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Do not check read result consistency when unwinding [#44956](https://github.com/ClickHouse/ClickHouse/pull/44956) ([Alexander Gololobov](https://github.com/davenger)). - diff --git a/docs/changelogs/v22.8.13.20-lts.md b/docs/changelogs/v22.8.13.20-lts.md index d8dd1bd2b1c..0734f40bf3e 100644 --- a/docs/changelogs/v22.8.13.20-lts.md +++ b/docs/changelogs/v22.8.13.20-lts.md @@ -7,7 +7,7 @@ sidebar_label: 2023 ### ClickHouse release v22.8.13.20-lts (e4817946d18) FIXME as compared to v22.8.12.45-lts (86b0ecd5d51) -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#45565](https://github.com/ClickHouse/ClickHouse/issues/45565): Fix positional arguments exception Positional argument out of bounds. Closes [#40634](https://github.com/ClickHouse/ClickHouse/issues/40634). [#41189](https://github.com/ClickHouse/ClickHouse/pull/41189) ([Kseniia Sumarokova](https://github.com/kssenii)). * Backported in [#44997](https://github.com/ClickHouse/ClickHouse/issues/44997): Another fix for `Cannot read all data` error which could happen while reading `LowCardinality` dictionary from remote fs. Fixes [#44709](https://github.com/ClickHouse/ClickHouse/issues/44709). [#44875](https://github.com/ClickHouse/ClickHouse/pull/44875) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). @@ -21,4 +21,3 @@ sidebar_label: 2023 * Get rid of artifactory in favor of r2 + ch-repos-manager [#45421](https://github.com/ClickHouse/ClickHouse/pull/45421) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Trim refs/tags/ from GITHUB_TAG in release workflow [#45636](https://github.com/ClickHouse/ClickHouse/pull/45636) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Merge pull request [#38262](https://github.com/ClickHouse/ClickHouse/issues/38262) from PolyProgrammist/fix-ordinary-system-un… [#45650](https://github.com/ClickHouse/ClickHouse/pull/45650) ([alesapin](https://github.com/alesapin)). - diff --git a/docs/changelogs/v22.8.14.53-lts.md b/docs/changelogs/v22.8.14.53-lts.md index 5978080fa3a..3cceb3475b6 100644 --- a/docs/changelogs/v22.8.14.53-lts.md +++ b/docs/changelogs/v22.8.14.53-lts.md @@ -17,7 +17,7 @@ sidebar_label: 2023 * Backported in [#46482](https://github.com/ClickHouse/ClickHouse/issues/46482): Get rid of unnecessary build for standalone clickhouse-keeper. [#46367](https://github.com/ClickHouse/ClickHouse/pull/46367) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Backported in [#46505](https://github.com/ClickHouse/ClickHouse/issues/46505): Some time ago the ccache compression was changed to `zst`, but `gz` archives are downloaded by default. It fixes it by prioritizing zst archive. [#46490](https://github.com/ClickHouse/ClickHouse/pull/46490) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#45908](https://github.com/ClickHouse/ClickHouse/issues/45908): Fixed bug with non-parsable default value for EPHEMERAL column in table metadata. [#44026](https://github.com/ClickHouse/ClickHouse/pull/44026) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). * Backported in [#46238](https://github.com/ClickHouse/ClickHouse/issues/46238): A couple of seg faults have been reported around `c-ares`. All of the recent stack traces observed fail on inserting into `std::unodered_set<>`. I believe I have found the root cause of this, it seems to be unprocessed queries. Prior to this PR, CH calls `poll` to wait on the file descriptors in the `c-ares` channel. According to the [poll docs](https://man7.org/linux/man-pages/man2/poll.2.html), a negative return value means an error has ocurred. Because of this, we would abort the execution and return failure. The problem is that `poll` will also return a negative value if a system interrupt occurs. A system interrupt does not mean the processing has failed or ended, but we would abort it anyways because we were checking for negative values. Once the execution is aborted, the whole stack is destroyed, which includes the `std::unordered_set` passed to the `void *` parameter of the c-ares callback. Once c-ares completed the request, the callback would be invoked and would access an invalid memory address causing a segfault. [#45629](https://github.com/ClickHouse/ClickHouse/pull/45629) ([Arthur Passos](https://github.com/arthurpassos)). @@ -37,4 +37,3 @@ sidebar_label: 2023 * Add helping logging to auto-merge script [#46080](https://github.com/ClickHouse/ClickHouse/pull/46080) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Fix write buffer destruction order for vertical merge. [#46205](https://github.com/ClickHouse/ClickHouse/pull/46205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Get rid of legacy DocsReleaseChecks [#46665](https://github.com/ClickHouse/ClickHouse/pull/46665) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v22.8.15.23-lts.md b/docs/changelogs/v22.8.15.23-lts.md index 096a504c9c2..5f49dfb1757 100644 --- a/docs/changelogs/v22.8.15.23-lts.md +++ b/docs/changelogs/v22.8.15.23-lts.md @@ -13,7 +13,7 @@ sidebar_label: 2023 #### Bug Fix * Backported in [#47336](https://github.com/ClickHouse/ClickHouse/issues/47336): Sometimes after changing a role that could be not reflected on the access rights of a user who uses that role. This PR fixes that. [#46772](https://github.com/ClickHouse/ClickHouse/pull/46772) ([Vitaly Baranov](https://github.com/vitlibar)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#46901](https://github.com/ClickHouse/ClickHouse/issues/46901): - Fix incorrect alias recursion in QueryNormalizer. [#46609](https://github.com/ClickHouse/ClickHouse/pull/46609) ([Raúl Marín](https://github.com/Algunenano)). * Backported in [#47156](https://github.com/ClickHouse/ClickHouse/issues/47156): - Fix arithmetic operations in aggregate optimization with `min` and `max`. [#46705](https://github.com/ClickHouse/ClickHouse/pull/46705) ([Duc Canh Le](https://github.com/canhld94)). @@ -25,4 +25,3 @@ sidebar_label: 2023 * Reduce updates of Mergeable Check [#46781](https://github.com/ClickHouse/ClickHouse/pull/46781) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Update typing for a new PyGithub version [#47123](https://github.com/ClickHouse/ClickHouse/pull/47123) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Add a manual trigger for release workflow [#47302](https://github.com/ClickHouse/ClickHouse/pull/47302) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v22.8.3.13-lts.md b/docs/changelogs/v22.8.3.13-lts.md index 903f5b7a600..5f08bc9ee67 100644 --- a/docs/changelogs/v22.8.3.13-lts.md +++ b/docs/changelogs/v22.8.3.13-lts.md @@ -13,7 +13,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#40694](https://github.com/ClickHouse/ClickHouse/issues/40694): Fix TGZ packages. [#40681](https://github.com/ClickHouse/ClickHouse/pull/40681) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#40451](https://github.com/ClickHouse/ClickHouse/issues/40451): Fix rare bug with column TTL for MergeTree engines family: In case of repeated vertical merge the error `Cannot unlink file ColumnName.bin ... No such file or directory.` could happen. [#40346](https://github.com/ClickHouse/ClickHouse/pull/40346) ([alesapin](https://github.com/alesapin)). * Backported in [#40533](https://github.com/ClickHouse/ClickHouse/issues/40533): Proxy resolver stop on first successful request to endpoint. [#40353](https://github.com/ClickHouse/ClickHouse/pull/40353) ([Maksim Kita](https://github.com/kitaisreal)). diff --git a/docs/changelogs/v22.8.4.7-lts.md b/docs/changelogs/v22.8.4.7-lts.md index 93d9aa2bf1a..a852120ab8a 100644 --- a/docs/changelogs/v22.8.4.7-lts.md +++ b/docs/changelogs/v22.8.4.7-lts.md @@ -7,7 +7,7 @@ sidebar_label: 2022 ### ClickHouse release v22.8.4.7-lts (baad27bcd2f) FIXME as compared to v22.8.3.13-lts (6a15b73faea) -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#40760](https://github.com/ClickHouse/ClickHouse/issues/40760): Fix possible error 'Decimal math overflow' while parsing DateTime64. [#40546](https://github.com/ClickHouse/ClickHouse/pull/40546) ([Kruglov Pavel](https://github.com/Avogar)). * Backported in [#40811](https://github.com/ClickHouse/ClickHouse/issues/40811): In [#40595](https://github.com/ClickHouse/ClickHouse/issues/40595) it was reported that the `host_regexp` functionality was not working properly with a name to address resolution in `/etc/hosts`. It's fixed. [#40769](https://github.com/ClickHouse/ClickHouse/pull/40769) ([Arthur Passos](https://github.com/arthurpassos)). diff --git a/docs/changelogs/v22.8.5.29-lts.md b/docs/changelogs/v22.8.5.29-lts.md index b7ad3c11a46..1b59501cc28 100644 --- a/docs/changelogs/v22.8.5.29-lts.md +++ b/docs/changelogs/v22.8.5.29-lts.md @@ -17,7 +17,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#41157](https://github.com/ClickHouse/ClickHouse/issues/41157): Add macOS binaries to GH release assets, it fixes [#37718](https://github.com/ClickHouse/ClickHouse/issues/37718). [#41088](https://github.com/ClickHouse/ClickHouse/pull/41088) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#40866](https://github.com/ClickHouse/ClickHouse/issues/40866): - Fix crash while parsing values of type `Object` that contains arrays of variadic dimension. [#40483](https://github.com/ClickHouse/ClickHouse/pull/40483) ([Duc Canh Le](https://github.com/canhld94)). * Backported in [#40805](https://github.com/ClickHouse/ClickHouse/issues/40805): During insertion of a new query to the `ProcessList` allocations happen. If we reach the memory limit during these allocations we can not use `OvercommitTracker`, because `ProcessList::mutex` is already acquired. Fixes [#40611](https://github.com/ClickHouse/ClickHouse/issues/40611). [#40677](https://github.com/ClickHouse/ClickHouse/pull/40677) ([Dmitry Novik](https://github.com/novikd)). diff --git a/docs/changelogs/v22.8.6.71-lts.md b/docs/changelogs/v22.8.6.71-lts.md index 0337c5ba2e2..2ac6ef05bc4 100644 --- a/docs/changelogs/v22.8.6.71-lts.md +++ b/docs/changelogs/v22.8.6.71-lts.md @@ -16,7 +16,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#41560](https://github.com/ClickHouse/ClickHouse/issues/41560): Add `source` field to deb packages, update `nfpm`. [#41531](https://github.com/ClickHouse/ClickHouse/pull/41531) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#41284](https://github.com/ClickHouse/ClickHouse/issues/41284): Fix unused unknown columns introduced by WITH statement. This fixes [#37812](https://github.com/ClickHouse/ClickHouse/issues/37812) . [#39131](https://github.com/ClickHouse/ClickHouse/pull/39131) ([Amos Bird](https://github.com/amosbird)). * Backported in [#41837](https://github.com/ClickHouse/ClickHouse/issues/41837): Fix vertical merge of parts with lightweight deleted rows. [#40559](https://github.com/ClickHouse/ClickHouse/pull/40559) ([Alexander Gololobov](https://github.com/davenger)). diff --git a/docs/changelogs/v22.8.7.34-lts.md b/docs/changelogs/v22.8.7.34-lts.md index ee55f5d9f1f..56f03ecdf3b 100644 --- a/docs/changelogs/v22.8.7.34-lts.md +++ b/docs/changelogs/v22.8.7.34-lts.md @@ -17,7 +17,7 @@ sidebar_label: 2022 * Backported in [#42296](https://github.com/ClickHouse/ClickHouse/issues/42296): Update cctz to the latest master, update tzdb to 2020e. [#42273](https://github.com/ClickHouse/ClickHouse/pull/42273) ([Dom Del Nano](https://github.com/ddelnano)). * Backported in [#42360](https://github.com/ClickHouse/ClickHouse/issues/42360): Update tzdata to 2022e to support the new timezone changes. Palestine transitions are now Saturdays at 02:00. Simplify three Ukraine zones into one. Jordan and Syria switch from +02/+03 with DST to year-round +03. (https://data.iana.org/time-zones/tzdb/NEWS). This closes [#42252](https://github.com/ClickHouse/ClickHouse/issues/42252). [#42327](https://github.com/ClickHouse/ClickHouse/pull/42327) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#42489](https://github.com/ClickHouse/ClickHouse/issues/42489): Removed skipping of mutations in unaffected partitions of `MergeTree` tables, because this feature never worked correctly and might cause resurrection of finished mutations. [#40589](https://github.com/ClickHouse/ClickHouse/pull/40589) ([Alexander Tokmakov](https://github.com/tavplubix)). * Backported in [#42121](https://github.com/ClickHouse/ClickHouse/issues/42121): Fixed "Part ... intersects part ..." error that might happen in extremely rare cases if replica was restarted just after detaching some part as broken. [#41741](https://github.com/ClickHouse/ClickHouse/pull/41741) ([Alexander Tokmakov](https://github.com/tavplubix)). diff --git a/docs/changelogs/v22.8.8.3-lts.md b/docs/changelogs/v22.8.8.3-lts.md index b4673eb955a..d42f333ce3f 100644 --- a/docs/changelogs/v22.8.8.3-lts.md +++ b/docs/changelogs/v22.8.8.3-lts.md @@ -7,6 +7,6 @@ sidebar_label: 2022 ### ClickHouse release v22.8.8.3-lts (ac5a6cababc) FIXME as compared to v22.8.7.34-lts (3c38e5e8ab9) -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#42677](https://github.com/ClickHouse/ClickHouse/issues/42677): keeper-fix: fix race in accessing logs while snapshot is being installed. [#40627](https://github.com/ClickHouse/ClickHouse/pull/40627) ([Antonio Andelic](https://github.com/antonio2368)). diff --git a/docs/changelogs/v22.8.9.24-lts.md b/docs/changelogs/v22.8.9.24-lts.md index 585327b83a1..15935c4cf4e 100644 --- a/docs/changelogs/v22.8.9.24-lts.md +++ b/docs/changelogs/v22.8.9.24-lts.md @@ -17,7 +17,7 @@ sidebar_label: 2022 * Backported in [#42964](https://github.com/ClickHouse/ClickHouse/issues/42964): Before the fix, the user-defined config was preserved by RPM in `$file.rpmsave`. The PR fixes it and won't replace the user's files from packages. [#42936](https://github.com/ClickHouse/ClickHouse/pull/42936) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Backported in [#43040](https://github.com/ClickHouse/ClickHouse/issues/43040): Add a CI step to mark commits as ready for release; soft-forbid launching a release script from branches but master. [#43017](https://github.com/ClickHouse/ClickHouse/pull/43017) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#42720](https://github.com/ClickHouse/ClickHouse/issues/42720): Fixed `Unknown identifier (aggregate-function)` exception which appears when a user tries to calculate WINDOW ORDER BY/PARTITION BY expressions over aggregate functions: ``` CREATE TABLE default.tenk1 ( `unique1` Int32, `unique2` Int32, `ten` Int32 ) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192; SELECT ten, sum(unique1) + sum(unique2) AS res, rank() OVER (ORDER BY sum(unique1) + sum(unique2) ASC) AS rank FROM _complex GROUP BY ten ORDER BY ten ASC; ``` which gives: ``` Code: 47. DB::Exception: Received from localhost:9000. DB::Exception: Unknown identifier: sum(unique1); there are columns: unique1, unique2, ten: While processing sum(unique1) + sum(unique2) ASC. (UNKNOWN_IDENTIFIER) ```. [#39762](https://github.com/ClickHouse/ClickHouse/pull/39762) ([Vladimir Chebotaryov](https://github.com/quickhouse)). * Backported in [#42748](https://github.com/ClickHouse/ClickHouse/issues/42748): A segmentation fault related to DNS & c-ares has been reported. The below error ocurred in multiple threads: ``` 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008088 [ 356 ] {} BaseDaemon: ######################################## 2022-09-28 15:41:19.008,"2022.09.28 15:41:19.008147 [ 356 ] {} BaseDaemon: (version 22.8.5.29 (official build), build id: 92504ACA0B8E2267) (from thread 353) (no query) Received signal Segmentation fault (11)" 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008196 [ 356 ] {} BaseDaemon: Address: 0xf Access: write. Address not mapped to object. 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008216 [ 356 ] {} BaseDaemon: Stack trace: 0x188f8212 0x1626851b 0x1626a69e 0x16269b3f 0x16267eab 0x13cf8284 0x13d24afc 0x13c5217e 0x14ec2495 0x15ba440f 0x15b9d13b 0x15bb2699 0x1891ccb3 0x1891e00d 0x18ae0769 0x18ade022 0x7f76aa985609 0x7f76aa8aa133 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008274 [ 356 ] {} BaseDaemon: 2. Poco::Net::IPAddress::family() const @ 0x188f8212 in /usr/bin/clickhouse 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008297 [ 356 ] {} BaseDaemon: 3. ? @ 0x1626851b in /usr/bin/clickhouse 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008309 [ 356 ] {} BaseDaemon: 4. ? @ 0x1626a69e in /usr/bin/clickhouse ```. [#42234](https://github.com/ClickHouse/ClickHouse/pull/42234) ([Arthur Passos](https://github.com/arthurpassos)). diff --git a/docs/changelogs/v22.9.2.7-stable.md b/docs/changelogs/v22.9.2.7-stable.md index 5c4db4bfa96..bbd0a6cce32 100644 --- a/docs/changelogs/v22.9.2.7-stable.md +++ b/docs/changelogs/v22.9.2.7-stable.md @@ -10,7 +10,7 @@ sidebar_label: 2022 #### Improvement * Backported in [#41709](https://github.com/ClickHouse/ClickHouse/issues/41709): Check file path for path traversal attacks in errors logger for input formats. [#41694](https://github.com/ClickHouse/ClickHouse/pull/41694) ([Kruglov Pavel](https://github.com/Avogar)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#41696](https://github.com/ClickHouse/ClickHouse/issues/41696): Fixes issue when docker run will fail if "https_port" is not present in config. [#41693](https://github.com/ClickHouse/ClickHouse/pull/41693) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). diff --git a/docs/changelogs/v22.9.3.18-stable.md b/docs/changelogs/v22.9.3.18-stable.md index 656cb1dfc22..a46dba6718c 100644 --- a/docs/changelogs/v22.9.3.18-stable.md +++ b/docs/changelogs/v22.9.3.18-stable.md @@ -7,7 +7,7 @@ sidebar_label: 2022 ### ClickHouse release v22.9.3.18-stable (0cb4b15d2fa) FIXME as compared to v22.9.2.7-stable (362e2cefcef) -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#41902](https://github.com/ClickHouse/ClickHouse/issues/41902): Fix possible crash in `SELECT` from `Merge` table with enabled `optimize_monotonous_functions_in_order_by` setting. Fixes [#41269](https://github.com/ClickHouse/ClickHouse/issues/41269). [#41740](https://github.com/ClickHouse/ClickHouse/pull/41740) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Backported in [#41863](https://github.com/ClickHouse/ClickHouse/issues/41863): 22.9 might fail to startup `ReplicatedMergeTree` table if that table was created by 20.3 or older version and was never altered, it's fixed. Fixes [#41742](https://github.com/ClickHouse/ClickHouse/issues/41742). [#41796](https://github.com/ClickHouse/ClickHouse/pull/41796) ([Alexander Tokmakov](https://github.com/tavplubix)). diff --git a/docs/changelogs/v22.9.4.32-stable.md b/docs/changelogs/v22.9.4.32-stable.md index 658d39af079..92bcc01e408 100644 --- a/docs/changelogs/v22.9.4.32-stable.md +++ b/docs/changelogs/v22.9.4.32-stable.md @@ -14,7 +14,7 @@ sidebar_label: 2022 * Backported in [#42297](https://github.com/ClickHouse/ClickHouse/issues/42297): Update cctz to the latest master, update tzdb to 2020e. [#42273](https://github.com/ClickHouse/ClickHouse/pull/42273) ([Dom Del Nano](https://github.com/ddelnano)). * Backported in [#42361](https://github.com/ClickHouse/ClickHouse/issues/42361): Update tzdata to 2022e to support the new timezone changes. Palestine transitions are now Saturdays at 02:00. Simplify three Ukraine zones into one. Jordan and Syria switch from +02/+03 with DST to year-round +03. (https://data.iana.org/time-zones/tzdb/NEWS). This closes [#42252](https://github.com/ClickHouse/ClickHouse/issues/42252). [#42327](https://github.com/ClickHouse/ClickHouse/pull/42327) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#42122](https://github.com/ClickHouse/ClickHouse/issues/42122): Fixed "Part ... intersects part ..." error that might happen in extremely rare cases if replica was restarted just after detaching some part as broken. [#41741](https://github.com/ClickHouse/ClickHouse/pull/41741) ([Alexander Tokmakov](https://github.com/tavplubix)). * Backported in [#41938](https://github.com/ClickHouse/ClickHouse/issues/41938): Don't allow to create or alter merge tree tables with virtual column name _row_exists, which is reserved for lightweight delete. Fixed [#41716](https://github.com/ClickHouse/ClickHouse/issues/41716). [#41763](https://github.com/ClickHouse/ClickHouse/pull/41763) ([Jianmei Zhang](https://github.com/zhangjmruc)). diff --git a/docs/changelogs/v22.9.5.25-stable.md b/docs/changelogs/v22.9.5.25-stable.md index eb46fb893e7..90150726ace 100644 --- a/docs/changelogs/v22.9.5.25-stable.md +++ b/docs/changelogs/v22.9.5.25-stable.md @@ -14,7 +14,7 @@ sidebar_label: 2022 * Backported in [#42965](https://github.com/ClickHouse/ClickHouse/issues/42965): Before the fix, the user-defined config was preserved by RPM in `$file.rpmsave`. The PR fixes it and won't replace the user's files from packages. [#42936](https://github.com/ClickHouse/ClickHouse/pull/42936) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Backported in [#43041](https://github.com/ClickHouse/ClickHouse/issues/43041): Add a CI step to mark commits as ready for release; soft-forbid launching a release script from branches but master. [#43017](https://github.com/ClickHouse/ClickHouse/pull/43017) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#42749](https://github.com/ClickHouse/ClickHouse/issues/42749): A segmentation fault related to DNS & c-ares has been reported. The below error ocurred in multiple threads: ``` 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008088 [ 356 ] {} BaseDaemon: ######################################## 2022-09-28 15:41:19.008,"2022.09.28 15:41:19.008147 [ 356 ] {} BaseDaemon: (version 22.8.5.29 (official build), build id: 92504ACA0B8E2267) (from thread 353) (no query) Received signal Segmentation fault (11)" 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008196 [ 356 ] {} BaseDaemon: Address: 0xf Access: write. Address not mapped to object. 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008216 [ 356 ] {} BaseDaemon: Stack trace: 0x188f8212 0x1626851b 0x1626a69e 0x16269b3f 0x16267eab 0x13cf8284 0x13d24afc 0x13c5217e 0x14ec2495 0x15ba440f 0x15b9d13b 0x15bb2699 0x1891ccb3 0x1891e00d 0x18ae0769 0x18ade022 0x7f76aa985609 0x7f76aa8aa133 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008274 [ 356 ] {} BaseDaemon: 2. Poco::Net::IPAddress::family() const @ 0x188f8212 in /usr/bin/clickhouse 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008297 [ 356 ] {} BaseDaemon: 3. ? @ 0x1626851b in /usr/bin/clickhouse 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008309 [ 356 ] {} BaseDaemon: 4. ? @ 0x1626a69e in /usr/bin/clickhouse ```. [#42234](https://github.com/ClickHouse/ClickHouse/pull/42234) ([Arthur Passos](https://github.com/arthurpassos)). * Backported in [#42863](https://github.com/ClickHouse/ClickHouse/issues/42863): Fix lowerUTF8()/upperUTF8() in case of symbol was in between 16-byte boundary (very frequent case of you have strings > 16 bytes long). [#42812](https://github.com/ClickHouse/ClickHouse/pull/42812) ([Azat Khuzhin](https://github.com/azat)). diff --git a/docs/changelogs/v22.9.6.20-stable.md b/docs/changelogs/v22.9.6.20-stable.md index d450f285848..7abc4adc32e 100644 --- a/docs/changelogs/v22.9.6.20-stable.md +++ b/docs/changelogs/v22.9.6.20-stable.md @@ -13,7 +13,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#43052](https://github.com/ClickHouse/ClickHouse/issues/43052): Wait for all files are in sync before archiving them in integration tests. [#42891](https://github.com/ClickHouse/ClickHouse/pull/42891) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#43505](https://github.com/ClickHouse/ClickHouse/issues/43505): Fix a bug when row level filter uses default value of column. [#43387](https://github.com/ClickHouse/ClickHouse/pull/43387) ([Alexander Gololobov](https://github.com/davenger)). * Backported in [#43722](https://github.com/ClickHouse/ClickHouse/issues/43722): Fixed primary key analysis with conditions involving `toString(enum)`. [#43596](https://github.com/ClickHouse/ClickHouse/pull/43596) ([Nikita Taranov](https://github.com/nickitat)). diff --git a/docs/changelogs/v22.9.7.34-stable.md b/docs/changelogs/v22.9.7.34-stable.md index 83be449f70d..ed8173eaf50 100644 --- a/docs/changelogs/v22.9.7.34-stable.md +++ b/docs/changelogs/v22.9.7.34-stable.md @@ -13,7 +13,7 @@ sidebar_label: 2022 #### Build/Testing/Packaging Improvement * Backported in [#44111](https://github.com/ClickHouse/ClickHouse/issues/44111): Bring sha512 sums back to the building step. [#44017](https://github.com/ClickHouse/ClickHouse/pull/44017) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#43612](https://github.com/ClickHouse/ClickHouse/issues/43612): Fix bad inefficiency of `remote_filesystem_read_method=read` with filesystem cache. Closes [#42125](https://github.com/ClickHouse/ClickHouse/issues/42125). [#42129](https://github.com/ClickHouse/ClickHouse/pull/42129) ([Kseniia Sumarokova](https://github.com/kssenii)). * Backported in [#43526](https://github.com/ClickHouse/ClickHouse/issues/43526): Fix incorrect UserTimeMicroseconds/SystemTimeMicroseconds accounting. [#42791](https://github.com/ClickHouse/ClickHouse/pull/42791) ([Azat Khuzhin](https://github.com/azat)). diff --git a/docs/changelogs/v23.1.1.3077-stable.md b/docs/changelogs/v23.1.1.3077-stable.md index e218be62f09..53ca9e1831c 100644 --- a/docs/changelogs/v23.1.1.3077-stable.md +++ b/docs/changelogs/v23.1.1.3077-stable.md @@ -125,7 +125,7 @@ sidebar_label: 2023 * SQLite library is updated to the latest. It is used for the SQLite database and table integration engines. Also, fixed a false-positive TSan report. This closes [#45027](https://github.com/ClickHouse/ClickHouse/issues/45027). [#45031](https://github.com/ClickHouse/ClickHouse/pull/45031) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Fix report sending in the case when FastTest failed. [#45588](https://github.com/ClickHouse/ClickHouse/pull/45588) ([Dmitry Novik](https://github.com/novikd)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * #40651 [#41404](https://github.com/ClickHouse/ClickHouse/issues/41404). [#42126](https://github.com/ClickHouse/ClickHouse/pull/42126) ([Alexander Gololobov](https://github.com/davenger)). * Fix possible use-of-unitialized value after executing expressions after sorting. Closes [#43386](https://github.com/ClickHouse/ClickHouse/issues/43386) CC: @nickitat. [#43635](https://github.com/ClickHouse/ClickHouse/pull/43635) ([Kruglov Pavel](https://github.com/Avogar)). @@ -589,4 +589,3 @@ sidebar_label: 2023 * Resubmit "Fix possible in-use table after DETACH" [#45566](https://github.com/ClickHouse/ClickHouse/pull/45566) ([Alexander Tokmakov](https://github.com/tavplubix)). * Typo: "Granulesis" --> "Granules" [#45598](https://github.com/ClickHouse/ClickHouse/pull/45598) ([Robert Schulze](https://github.com/rschu1ze)). * Fix version in autogenerated_versions.txt [#45624](https://github.com/ClickHouse/ClickHouse/pull/45624) ([Dmitry Novik](https://github.com/novikd)). - diff --git a/docs/changelogs/v23.1.2.9-stable.md b/docs/changelogs/v23.1.2.9-stable.md index 272a2b95a86..7d34a6b9ec2 100644 --- a/docs/changelogs/v23.1.2.9-stable.md +++ b/docs/changelogs/v23.1.2.9-stable.md @@ -13,11 +13,10 @@ sidebar_label: 2023 #### Bug Fix * Backported in [#45673](https://github.com/ClickHouse/ClickHouse/issues/45673): Fix wiping sensitive info in logs. [#45603](https://github.com/ClickHouse/ClickHouse/pull/45603) ([Vitaly Baranov](https://github.com/vitlibar)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#45730](https://github.com/ClickHouse/ClickHouse/issues/45730): Fix key description when encountering duplicate primary keys. This can happen in projections. See [#45590](https://github.com/ClickHouse/ClickHouse/issues/45590) for details. [#45686](https://github.com/ClickHouse/ClickHouse/pull/45686) ([Amos Bird](https://github.com/amosbird)). #### NOT FOR CHANGELOG / INSIGNIFICANT * Trim refs/tags/ from GITHUB_TAG in release workflow [#45636](https://github.com/ClickHouse/ClickHouse/pull/45636) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v23.1.3.5-stable.md b/docs/changelogs/v23.1.3.5-stable.md index d4f39894bec..9f8ef928138 100644 --- a/docs/changelogs/v23.1.3.5-stable.md +++ b/docs/changelogs/v23.1.3.5-stable.md @@ -7,11 +7,10 @@ sidebar_label: 2023 ### ClickHouse release v23.1.3.5-stable (548b494bcce) FIXME as compared to v23.1.2.9-stable (8dfb1700858) -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#45896](https://github.com/ClickHouse/ClickHouse/issues/45896): Bugfix IPv6 parser for mixed ip4 address with missed first octet (like `::.1.2.3`). [#45871](https://github.com/ClickHouse/ClickHouse/pull/45871) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). #### NOT FOR CHANGELOG / INSIGNIFICANT * Get rid of progress timestamps in release publishing [#45818](https://github.com/ClickHouse/ClickHouse/pull/45818) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v23.1.4.58-stable.md b/docs/changelogs/v23.1.4.58-stable.md index d1ffe87f58e..9081d700308 100644 --- a/docs/changelogs/v23.1.4.58-stable.md +++ b/docs/changelogs/v23.1.4.58-stable.md @@ -20,7 +20,7 @@ sidebar_label: 2023 * Backported in [#46477](https://github.com/ClickHouse/ClickHouse/issues/46477): Get rid of unnecessary build for standalone clickhouse-keeper. [#46367](https://github.com/ClickHouse/ClickHouse/pull/46367) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Backported in [#46511](https://github.com/ClickHouse/ClickHouse/issues/46511): Some time ago the ccache compression was changed to `zst`, but `gz` archives are downloaded by default. It fixes it by prioritizing zst archive. [#46490](https://github.com/ClickHouse/ClickHouse/pull/46490) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#46228](https://github.com/ClickHouse/ClickHouse/issues/46228): A couple of seg faults have been reported around `c-ares`. All of the recent stack traces observed fail on inserting into `std::unodered_set<>`. I believe I have found the root cause of this, it seems to be unprocessed queries. Prior to this PR, CH calls `poll` to wait on the file descriptors in the `c-ares` channel. According to the [poll docs](https://man7.org/linux/man-pages/man2/poll.2.html), a negative return value means an error has ocurred. Because of this, we would abort the execution and return failure. The problem is that `poll` will also return a negative value if a system interrupt occurs. A system interrupt does not mean the processing has failed or ended, but we would abort it anyways because we were checking for negative values. Once the execution is aborted, the whole stack is destroyed, which includes the `std::unordered_set` passed to the `void *` parameter of the c-ares callback. Once c-ares completed the request, the callback would be invoked and would access an invalid memory address causing a segfault. [#45629](https://github.com/ClickHouse/ClickHouse/pull/45629) ([Arthur Passos](https://github.com/arthurpassos)). * Backported in [#46967](https://github.com/ClickHouse/ClickHouse/issues/46967): Backward compatibility - allow implicit narrowing conversion from UInt64 to IPv4 - required for "INSERT ... VALUES ..." expression. [#45865](https://github.com/ClickHouse/ClickHouse/pull/45865) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). @@ -44,4 +44,3 @@ sidebar_label: 2023 * Fix dependencies for InstallPackagesTestAarch64 [#46597](https://github.com/ClickHouse/ClickHouse/pull/46597) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Get rid of legacy DocsReleaseChecks [#46665](https://github.com/ClickHouse/ClickHouse/pull/46665) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Reduce updates of Mergeable Check [#46781](https://github.com/ClickHouse/ClickHouse/pull/46781) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v23.1.5.24-stable.md b/docs/changelogs/v23.1.5.24-stable.md index 1c2c127a8c3..934e97312c0 100644 --- a/docs/changelogs/v23.1.5.24-stable.md +++ b/docs/changelogs/v23.1.5.24-stable.md @@ -10,7 +10,7 @@ sidebar_label: 2023 #### Build/Testing/Packaging Improvement * Backported in [#47060](https://github.com/ClickHouse/ClickHouse/issues/47060): Fix error during server startup on old distros (e.g. Amazon Linux 2) and on ARM that glibc 2.28 symbols are not found. [#47008](https://github.com/ClickHouse/ClickHouse/pull/47008) ([Robert Schulze](https://github.com/rschu1ze)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#46401](https://github.com/ClickHouse/ClickHouse/issues/46401): Fix `SYSTEM UNFREEZE` queries failing with the exception `CANNOT_PARSE_INPUT_ASSERTION_FAILED`. [#46325](https://github.com/ClickHouse/ClickHouse/pull/46325) ([Aleksei Filatov](https://github.com/aalexfvk)). * Backported in [#46905](https://github.com/ClickHouse/ClickHouse/issues/46905): - Fix incorrect alias recursion in QueryNormalizer. [#46609](https://github.com/ClickHouse/ClickHouse/pull/46609) ([Raúl Marín](https://github.com/Algunenano)). @@ -25,4 +25,3 @@ sidebar_label: 2023 * Update typing for a new PyGithub version [#47123](https://github.com/ClickHouse/ClickHouse/pull/47123) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Follow-up to [#46681](https://github.com/ClickHouse/ClickHouse/issues/46681) [#47284](https://github.com/ClickHouse/ClickHouse/pull/47284) ([Alexander Tokmakov](https://github.com/tavplubix)). * Add a manual trigger for release workflow [#47302](https://github.com/ClickHouse/ClickHouse/pull/47302) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v23.2.1.2537-stable.md b/docs/changelogs/v23.2.1.2537-stable.md index 3fdcf6d6571..9da81c039ea 100644 --- a/docs/changelogs/v23.2.1.2537-stable.md +++ b/docs/changelogs/v23.2.1.2537-stable.md @@ -161,7 +161,7 @@ sidebar_label: 2023 * Some time ago the ccache compression was changed to `zst`, but `gz` archives are downloaded by default. It fixes it by prioritizing zst archive. [#46490](https://github.com/ClickHouse/ClickHouse/pull/46490) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Raised the minimum Clang version needed to build ClickHouse from 12 to 15. [#46710](https://github.com/ClickHouse/ClickHouse/pull/46710) ([Robert Schulze](https://github.com/rschu1ze)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Flush data exactly by `rabbitmq_flush_interval_ms` or by `rabbitmq_max_block_size` in `StorageRabbitMQ`. Closes [#42389](https://github.com/ClickHouse/ClickHouse/issues/42389). Closes [#45160](https://github.com/ClickHouse/ClickHouse/issues/45160). [#44404](https://github.com/ClickHouse/ClickHouse/pull/44404) ([Kseniia Sumarokova](https://github.com/kssenii)). * - Use PODArray to render in sparkBar function, so we can control the memory usage. Close [#44467](https://github.com/ClickHouse/ClickHouse/issues/44467). [#44489](https://github.com/ClickHouse/ClickHouse/pull/44489) ([Duc Canh Le](https://github.com/canhld94)). @@ -470,4 +470,3 @@ sidebar_label: 2023 #### Testing Improvement * Fixed functional test 00304_http_external_data for s390x. [#45807](https://github.com/ClickHouse/ClickHouse/pull/45807) ([Harry Lee](https://github.com/HarryLeeIBM)). - diff --git a/docs/changelogs/v23.2.2.20-stable.md b/docs/changelogs/v23.2.2.20-stable.md index 60aeaa66cbf..b92fbdebe33 100644 --- a/docs/changelogs/v23.2.2.20-stable.md +++ b/docs/changelogs/v23.2.2.20-stable.md @@ -17,7 +17,7 @@ sidebar_label: 2023 #### Build/Testing/Packaging Improvement * Backported in [#47062](https://github.com/ClickHouse/ClickHouse/issues/47062): Fix error during server startup on old distros (e.g. Amazon Linux 2) and on ARM that glibc 2.28 symbols are not found. [#47008](https://github.com/ClickHouse/ClickHouse/pull/47008) ([Robert Schulze](https://github.com/rschu1ze)). -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#46895](https://github.com/ClickHouse/ClickHouse/issues/46895): Fixed a bug in automatic retries of `DROP TABLE` query with `ReplicatedMergeTree` tables and `Atomic` databases. In rare cases it could lead to `Can't get data for node /zk_path/log_pointer` and `The specified key does not exist` errors if ZooKeeper session expired during DROP and a new replicated table with the same path in ZooKeeper was created in parallel. [#46384](https://github.com/ClickHouse/ClickHouse/pull/46384) ([Alexander Tokmakov](https://github.com/tavplubix)). * Backported in [#46865](https://github.com/ClickHouse/ClickHouse/issues/46865): Fix a bug in the `Map` data type. This closes [#46855](https://github.com/ClickHouse/ClickHouse/issues/46855). [#46856](https://github.com/ClickHouse/ClickHouse/pull/46856) ([Alexey Milovidov](https://github.com/alexey-milovidov)). @@ -27,4 +27,3 @@ sidebar_label: 2023 * More concise logging at trace level for PREWHERE steps [#46771](https://github.com/ClickHouse/ClickHouse/pull/46771) ([Alexander Gololobov](https://github.com/davenger)). * Reduce updates of Mergeable Check [#46781](https://github.com/ClickHouse/ClickHouse/pull/46781) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v23.2.3.17-stable.md b/docs/changelogs/v23.2.3.17-stable.md index fb2c4e394dc..75b7f8b2b20 100644 --- a/docs/changelogs/v23.2.3.17-stable.md +++ b/docs/changelogs/v23.2.3.17-stable.md @@ -7,7 +7,7 @@ sidebar_label: 2023 ### ClickHouse release v23.2.3.17-stable (dec18bf7281) FIXME as compared to v23.2.2.20-stable (f6c269c8df2) -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#46907](https://github.com/ClickHouse/ClickHouse/issues/46907): - Fix incorrect alias recursion in QueryNormalizer. [#46609](https://github.com/ClickHouse/ClickHouse/pull/46609) ([Raúl Marín](https://github.com/Algunenano)). * Backported in [#47091](https://github.com/ClickHouse/ClickHouse/issues/47091): - Fix arithmetic operations in aggregate optimization with `min` and `max`. [#46705](https://github.com/ClickHouse/ClickHouse/pull/46705) ([Duc Canh Le](https://github.com/canhld94)). @@ -20,4 +20,3 @@ sidebar_label: 2023 * Use /etc/default/clickhouse in systemd too [#47003](https://github.com/ClickHouse/ClickHouse/pull/47003) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * do flushUntrackedMemory when context switches [#47102](https://github.com/ClickHouse/ClickHouse/pull/47102) ([Sema Checherinda](https://github.com/CheSema)). * Update typing for a new PyGithub version [#47123](https://github.com/ClickHouse/ClickHouse/pull/47123) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v23.2.4.12-stable.md b/docs/changelogs/v23.2.4.12-stable.md index 2b6a689aee5..1542e3257ce 100644 --- a/docs/changelogs/v23.2.4.12-stable.md +++ b/docs/changelogs/v23.2.4.12-stable.md @@ -7,7 +7,7 @@ sidebar_label: 2023 ### ClickHouse release v23.2.4.12-stable (8fe866cb035) FIXME as compared to v23.2.3.17-stable (dec18bf7281) -#### Bug Fix (user-visible misbehavior in official stable or prestable release) +#### Bug Fix (user-visible misbehavior in official stable release) * Backported in [#47277](https://github.com/ClickHouse/ClickHouse/issues/47277): Fix IPv4/IPv6 serialization/deserialization in binary formats that was broken in https://github.com/ClickHouse/ClickHouse/pull/43221. Closes [#46522](https://github.com/ClickHouse/ClickHouse/issues/46522). [#46616](https://github.com/ClickHouse/ClickHouse/pull/46616) ([Kruglov Pavel](https://github.com/Avogar)). * Backported in [#47212](https://github.com/ClickHouse/ClickHouse/issues/47212): `INSERT` queries through native TCP protocol and HTTP protocol were not canceled correctly in some cases. It could lead to a partially applied query if a client canceled the query, or if a client died or, in rare cases, on network errors. As a result, it could lead to not working deduplication. Fixes [#27667](https://github.com/ClickHouse/ClickHouse/issues/27667) and [#45377](https://github.com/ClickHouse/ClickHouse/issues/45377). [#46681](https://github.com/ClickHouse/ClickHouse/pull/46681) ([Alexander Tokmakov](https://github.com/tavplubix)). @@ -17,4 +17,3 @@ sidebar_label: 2023 * Follow-up to [#46681](https://github.com/ClickHouse/ClickHouse/issues/46681) [#47284](https://github.com/ClickHouse/ClickHouse/pull/47284) ([Alexander Tokmakov](https://github.com/tavplubix)). * Add a manual trigger for release workflow [#47302](https://github.com/ClickHouse/ClickHouse/pull/47302) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 41574907492..8dc136e01f4 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -35,8 +35,8 @@ LABELS = { "pr-backward-incompatible": ["Backward Incompatible Change"], "pr-bugfix": [ "Bug Fix", - "Bug Fix (user-visible misbehaviour in official stable or prestable release)", - "Bug Fix (user-visible misbehavior in official stable or prestable release)", + "Bug Fix (user-visible misbehaviour in official stable release)", + "Bug Fix (user-visible misbehavior in official stable release)", ], "pr-build": [ "Build/Testing/Packaging Improvement", diff --git a/utils/changelog/changelog.py b/utils/changelog/changelog.py index 3b66b68193b..9972ce8a627 100755 --- a/utils/changelog/changelog.py +++ b/utils/changelog/changelog.py @@ -256,6 +256,14 @@ def generate_description(item: PullRequest, repo: Repository) -> Optional[Descri category = "NOT FOR CHANGELOG / INSIGNIFICANT" return Description(item.number, item.user, item.html_url, item.title, category) + # Normalize bug fixes + if re.match( + r"(?i)bug\Wfix", + category, + ): + category = "Bug Fix (user-visible misbehavior in an official stable release)" + return Description(item.number, item.user, item.html_url, item.title, category) + # Filter out documentations changelog if re.match( r"(?i)doc", From a71454f8ec53d56ca4332220843ea0540dcafca4 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Mon, 27 Mar 2023 12:53:31 +0200 Subject: [PATCH 403/418] Fix automatic indentation in the built-in UI SQL editor --- programs/server/play.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/play.html b/programs/server/play.html index a7fc951ec37..323ba7d0b9e 100644 --- a/programs/server/play.html +++ b/programs/server/play.html @@ -687,7 +687,7 @@ e.preventDefault(); return false; - } else if (e.key === 'Enter') { + } else if (e.key === 'Enter' && !(event.metaKey || event.ctrlKey)) { // If the user presses Enter, and the previous line starts with spaces, // then we will insert the same number of spaces. const elem = e.target; From b01f08ccb8958d1526cc53f6797bb9e3117a208d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 27 Mar 2023 13:23:56 +0200 Subject: [PATCH 404/418] rename marked_dropped_tables to dropped_tables --- ...roppedTables.cpp => StorageSystemDroppedTables.cpp} | 6 +++--- ...kedDroppedTables.h => StorageSystemDroppedTables.h} | 2 +- src/Storages/System/attachSystemTables.cpp | 4 ++-- .../0_stateless/25400_marked_dropped_tables.reference | 2 +- .../0_stateless/25400_marked_dropped_tables.sql | 10 +++++----- 5 files changed, 12 insertions(+), 12 deletions(-) rename src/Storages/System/{StorageSystemMarkedDroppedTables.cpp => StorageSystemDroppedTables.cpp} (91%) rename src/Storages/System/{StorageSystemMarkedDroppedTables.h => StorageSystemDroppedTables.h} (78%) diff --git a/src/Storages/System/StorageSystemMarkedDroppedTables.cpp b/src/Storages/System/StorageSystemDroppedTables.cpp similarity index 91% rename from src/Storages/System/StorageSystemMarkedDroppedTables.cpp rename to src/Storages/System/StorageSystemDroppedTables.cpp index fcdd6e1edcf..1d6c8824c76 100644 --- a/src/Storages/System/StorageSystemMarkedDroppedTables.cpp +++ b/src/Storages/System/StorageSystemDroppedTables.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -13,7 +13,7 @@ namespace DB { -NamesAndTypesList StorageSystemMarkedDroppedTables::getNamesAndTypes() +NamesAndTypesList StorageSystemDroppedTables::getNamesAndTypes() { NamesAndTypesList names_and_types{ {"index", std::make_shared()}, @@ -28,7 +28,7 @@ NamesAndTypesList StorageSystemMarkedDroppedTables::getNamesAndTypes() } -void StorageSystemMarkedDroppedTables::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemDroppedTables::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const { auto tables_mark_dropped = DatabaseCatalog::instance().getTablesMarkedDropped(); diff --git a/src/Storages/System/StorageSystemMarkedDroppedTables.h b/src/Storages/System/StorageSystemDroppedTables.h similarity index 78% rename from src/Storages/System/StorageSystemMarkedDroppedTables.h rename to src/Storages/System/StorageSystemDroppedTables.h index ea2a864311c..44cc8122603 100644 --- a/src/Storages/System/StorageSystemMarkedDroppedTables.h +++ b/src/Storages/System/StorageSystemDroppedTables.h @@ -6,7 +6,7 @@ namespace DB { -class StorageSystemMarkedDroppedTables final : public IStorageSystemOneBlock +class StorageSystemDroppedTables final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemMarkedDroppedTables"; } diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index fd1cf2f1623..d6982ba30d5 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -79,7 +79,7 @@ #include #include #include -#include +#include #ifdef OS_LINUX #include @@ -141,7 +141,7 @@ void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database) attach(context, system_database, "time_zones"); attach(context, system_database, "backups"); attach(context, system_database, "schema_inference_cache"); - attach(context, system_database, "marked_dropped_tables"); + attach(context, system_database, "dropped_tables"); #ifdef OS_LINUX attach(context, system_database, "stack_trace"); #endif diff --git a/tests/queries/0_stateless/25400_marked_dropped_tables.reference b/tests/queries/0_stateless/25400_marked_dropped_tables.reference index 6fc5caff0cb..44906da9527 100644 --- a/tests/queries/0_stateless/25400_marked_dropped_tables.reference +++ b/tests/queries/0_stateless/25400_marked_dropped_tables.reference @@ -1,4 +1,4 @@ -25400_marked_dropped_tables MergeTree +25400_dropped_tables MergeTree index UInt32 database String table String diff --git a/tests/queries/0_stateless/25400_marked_dropped_tables.sql b/tests/queries/0_stateless/25400_marked_dropped_tables.sql index 101642fa779..9bf6579b583 100644 --- a/tests/queries/0_stateless/25400_marked_dropped_tables.sql +++ b/tests/queries/0_stateless/25400_marked_dropped_tables.sql @@ -1,11 +1,11 @@ -- Tags: no-ordinary-database SET database_atomic_wait_for_drop_and_detach_synchronously = 0; -DROP TABLE IF EXISTS 25400_marked_dropped_tables; +DROP TABLE IF EXISTS 25400_dropped_tables; -CREATE TABLE 25400_marked_dropped_tables (id Int32) Engine=MergeTree() ORDER BY id; -DROP TABLE 25400_marked_dropped_tables; +CREATE TABLE 25400_dropped_tables (id Int32) Engine=MergeTree() ORDER BY id; +DROP TABLE 25400_dropped_tables; -SELECT table, engine FROM system.marked_dropped_tables WHERE database = currentDatabase() LIMIT 1; -DESCRIBE TABLE system.marked_dropped_tables; +SELECT table, engine FROM system.dropped_tables WHERE database = currentDatabase() LIMIT 1; +DESCRIBE TABLE system.dropped_tables; From 2773e5c35241d9f1cb53cbe46e90d4124725b6cb Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 27 Mar 2023 13:50:09 +0200 Subject: [PATCH 405/418] Fix test 02050_client_profile_events (#47951) --- src/Client/ClientBase.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index d3bac40d709..d3ba1d7e84c 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1131,6 +1131,8 @@ void ClientBase::onProfileEvents(Block & block) { if (profile_events.watch.elapsedMilliseconds() >= profile_events.delay_ms) { + /// We need to restart the watch each time we flushed these events + profile_events.watch.restart(); initLogsOutputStream(); if (need_render_progress && tty_buf) progress_indication.clearProgressOutput(*tty_buf); @@ -1144,7 +1146,6 @@ void ClientBase::onProfileEvents(Block & block) incrementProfileEventsBlock(profile_events.last_block, block); } } - profile_events.watch.restart(); } } From 4234c38a64a0ba7a7c2c152592b7f857813b30f1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Mar 2023 14:45:49 +0200 Subject: [PATCH 406/418] Automatically correct some mistakes in the changelog --- docs/changelogs/v22.10.5.54-stable.md | 1 - docs/changelogs/v22.10.6.3-stable.md | 1 - docs/changelogs/v22.10.7.13-stable.md | 1 - docs/changelogs/v22.11.3.47-stable.md | 1 - docs/changelogs/v22.11.4.3-stable.md | 1 - docs/changelogs/v22.11.5.15-stable.md | 1 - docs/changelogs/v22.11.6.44-stable.md | 1 - docs/changelogs/v22.12.3.5-stable.md | 1 - docs/changelogs/v22.12.4.76-stable.md | 1 - docs/changelogs/v22.12.5.34-stable.md | 1 - docs/changelogs/v22.3.16.1190-lts.md | 1 - docs/changelogs/v22.3.18.37-lts.md | 1 - docs/changelogs/v22.3.19.6-lts.md | 1 - docs/changelogs/v22.8.12.45-lts.md | 1 - docs/changelogs/v22.8.13.20-lts.md | 1 - docs/changelogs/v22.8.14.53-lts.md | 1 - docs/changelogs/v22.8.15.23-lts.md | 1 - docs/changelogs/v23.1.1.3077-stable.md | 1 - docs/changelogs/v23.1.2.9-stable.md | 1 - docs/changelogs/v23.1.3.5-stable.md | 1 - docs/changelogs/v23.1.4.58-stable.md | 1 - docs/changelogs/v23.1.5.24-stable.md | 1 - docs/changelogs/v23.2.1.2537-stable.md | 1 - docs/changelogs/v23.2.2.20-stable.md | 1 - docs/changelogs/v23.2.3.17-stable.md | 1 - docs/changelogs/v23.2.4.12-stable.md | 1 - utils/changelog/changelog.py | 20 ++++++++++++++++++-- 27 files changed, 18 insertions(+), 28 deletions(-) diff --git a/docs/changelogs/v22.10.5.54-stable.md b/docs/changelogs/v22.10.5.54-stable.md index e372fb30618..aa1173dc671 100644 --- a/docs/changelogs/v22.10.5.54-stable.md +++ b/docs/changelogs/v22.10.5.54-stable.md @@ -41,4 +41,3 @@ sidebar_label: 2023 * Implement a custom central checkout action [#44399](https://github.com/ClickHouse/ClickHouse/pull/44399) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Fix crash on delete from materialized view [#44705](https://github.com/ClickHouse/ClickHouse/pull/44705) ([Alexander Gololobov](https://github.com/davenger)). * Do not check read result consistency when unwinding [#44956](https://github.com/ClickHouse/ClickHouse/pull/44956) ([Alexander Gololobov](https://github.com/davenger)). - diff --git a/docs/changelogs/v22.10.6.3-stable.md b/docs/changelogs/v22.10.6.3-stable.md index b0e88c92cb0..43ca032c774 100644 --- a/docs/changelogs/v22.10.6.3-stable.md +++ b/docs/changelogs/v22.10.6.3-stable.md @@ -10,4 +10,3 @@ sidebar_label: 2023 #### Bug Fix (user-visible misbehavior in official stable or prestable release) * Backported in [#45084](https://github.com/ClickHouse/ClickHouse/issues/45084): fix alter table ttl error when wide part has light weight delete mask. [#44959](https://github.com/ClickHouse/ClickHouse/pull/44959) ([Mingliang Pan](https://github.com/liangliangpan)). - diff --git a/docs/changelogs/v22.10.7.13-stable.md b/docs/changelogs/v22.10.7.13-stable.md index c906e00e524..46e876f36b4 100644 --- a/docs/changelogs/v22.10.7.13-stable.md +++ b/docs/changelogs/v22.10.7.13-stable.md @@ -18,4 +18,3 @@ sidebar_label: 2023 * Improve release scripts [#45074](https://github.com/ClickHouse/ClickHouse/pull/45074) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Fix wrong approved_at, simplify conditions [#45302](https://github.com/ClickHouse/ClickHouse/pull/45302) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Get rid of artifactory in favor of r2 + ch-repos-manager [#45421](https://github.com/ClickHouse/ClickHouse/pull/45421) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v22.11.3.47-stable.md b/docs/changelogs/v22.11.3.47-stable.md index d6451b853f7..832591e68f3 100644 --- a/docs/changelogs/v22.11.3.47-stable.md +++ b/docs/changelogs/v22.11.3.47-stable.md @@ -37,4 +37,3 @@ sidebar_label: 2023 * Implement a custom central checkout action [#44399](https://github.com/ClickHouse/ClickHouse/pull/44399) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Fix crash on delete from materialized view [#44705](https://github.com/ClickHouse/ClickHouse/pull/44705) ([Alexander Gololobov](https://github.com/davenger)). * Do not check read result consistency when unwinding [#44956](https://github.com/ClickHouse/ClickHouse/pull/44956) ([Alexander Gololobov](https://github.com/davenger)). - diff --git a/docs/changelogs/v22.11.4.3-stable.md b/docs/changelogs/v22.11.4.3-stable.md index 33780e848ef..fbf2baa9f7b 100644 --- a/docs/changelogs/v22.11.4.3-stable.md +++ b/docs/changelogs/v22.11.4.3-stable.md @@ -10,4 +10,3 @@ sidebar_label: 2023 #### Bug Fix (user-visible misbehavior in official stable or prestable release) * Backported in [#45085](https://github.com/ClickHouse/ClickHouse/issues/45085): fix alter table ttl error when wide part has light weight delete mask. [#44959](https://github.com/ClickHouse/ClickHouse/pull/44959) ([Mingliang Pan](https://github.com/liangliangpan)). - diff --git a/docs/changelogs/v22.11.5.15-stable.md b/docs/changelogs/v22.11.5.15-stable.md index 742a8740514..6927dccf7b7 100644 --- a/docs/changelogs/v22.11.5.15-stable.md +++ b/docs/changelogs/v22.11.5.15-stable.md @@ -19,4 +19,3 @@ sidebar_label: 2023 * Fix wrong approved_at, simplify conditions [#45302](https://github.com/ClickHouse/ClickHouse/pull/45302) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Get rid of artifactory in favor of r2 + ch-repos-manager [#45421](https://github.com/ClickHouse/ClickHouse/pull/45421) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Trim refs/tags/ from GITHUB_TAG in release workflow [#45636](https://github.com/ClickHouse/ClickHouse/pull/45636) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v22.11.6.44-stable.md b/docs/changelogs/v22.11.6.44-stable.md index 6e628b85150..cb7b2667667 100644 --- a/docs/changelogs/v22.11.6.44-stable.md +++ b/docs/changelogs/v22.11.6.44-stable.md @@ -34,4 +34,3 @@ sidebar_label: 2023 * Add helping logging to auto-merge script [#46080](https://github.com/ClickHouse/ClickHouse/pull/46080) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Fix write buffer destruction order for vertical merge. [#46205](https://github.com/ClickHouse/ClickHouse/pull/46205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Get rid of legacy DocsReleaseChecks [#46665](https://github.com/ClickHouse/ClickHouse/pull/46665) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v22.12.3.5-stable.md b/docs/changelogs/v22.12.3.5-stable.md index 8cbcbc6a590..2770de71f0c 100644 --- a/docs/changelogs/v22.12.3.5-stable.md +++ b/docs/changelogs/v22.12.3.5-stable.md @@ -14,4 +14,3 @@ sidebar_label: 2023 #### NOT FOR CHANGELOG / INSIGNIFICANT * Do not check read result consistency when unwinding [#44956](https://github.com/ClickHouse/ClickHouse/pull/44956) ([Alexander Gololobov](https://github.com/davenger)). - diff --git a/docs/changelogs/v22.12.4.76-stable.md b/docs/changelogs/v22.12.4.76-stable.md index 79569ff841e..7f4334c565b 100644 --- a/docs/changelogs/v22.12.4.76-stable.md +++ b/docs/changelogs/v22.12.4.76-stable.md @@ -52,4 +52,3 @@ sidebar_label: 2023 * Fix dependencies for InstallPackagesTestAarch64 [#46597](https://github.com/ClickHouse/ClickHouse/pull/46597) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Get rid of legacy DocsReleaseChecks [#46665](https://github.com/ClickHouse/ClickHouse/pull/46665) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Reduce updates of Mergeable Check [#46781](https://github.com/ClickHouse/ClickHouse/pull/46781) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v22.12.5.34-stable.md b/docs/changelogs/v22.12.5.34-stable.md index 95befaa88ff..11613bd838b 100644 --- a/docs/changelogs/v22.12.5.34-stable.md +++ b/docs/changelogs/v22.12.5.34-stable.md @@ -26,4 +26,3 @@ sidebar_label: 2023 * Update typing for a new PyGithub version [#47123](https://github.com/ClickHouse/ClickHouse/pull/47123) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Follow-up to [#46681](https://github.com/ClickHouse/ClickHouse/issues/46681) [#47284](https://github.com/ClickHouse/ClickHouse/pull/47284) ([Alexander Tokmakov](https://github.com/tavplubix)). * Add a manual trigger for release workflow [#47302](https://github.com/ClickHouse/ClickHouse/pull/47302) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v22.3.16.1190-lts.md b/docs/changelogs/v22.3.16.1190-lts.md index a43d34551ca..5f38dc32c16 100644 --- a/docs/changelogs/v22.3.16.1190-lts.md +++ b/docs/changelogs/v22.3.16.1190-lts.md @@ -30,4 +30,3 @@ sidebar_label: 2023 #### NO CL ENTRY * NO CL ENTRY: 'Fix multipart upload for large S3 object, backport to 22.3'. [#44217](https://github.com/ClickHouse/ClickHouse/pull/44217) ([ianton-ru](https://github.com/ianton-ru)). - diff --git a/docs/changelogs/v22.3.18.37-lts.md b/docs/changelogs/v22.3.18.37-lts.md index ff6378f09ad..65c05b35149 100644 --- a/docs/changelogs/v22.3.18.37-lts.md +++ b/docs/changelogs/v22.3.18.37-lts.md @@ -30,4 +30,3 @@ sidebar_label: 2023 * Get rid of progress timestamps in release publishing [#45818](https://github.com/ClickHouse/ClickHouse/pull/45818) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Add helping logging to auto-merge script [#46080](https://github.com/ClickHouse/ClickHouse/pull/46080) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Fix write buffer destruction order for vertical merge. [#46205](https://github.com/ClickHouse/ClickHouse/pull/46205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). - diff --git a/docs/changelogs/v22.3.19.6-lts.md b/docs/changelogs/v22.3.19.6-lts.md index d5b45f4ce66..1cc4b9cc013 100644 --- a/docs/changelogs/v22.3.19.6-lts.md +++ b/docs/changelogs/v22.3.19.6-lts.md @@ -14,4 +14,3 @@ sidebar_label: 2023 #### NOT FOR CHANGELOG / INSIGNIFICANT * Get rid of legacy DocsReleaseChecks [#46665](https://github.com/ClickHouse/ClickHouse/pull/46665) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v22.8.12.45-lts.md b/docs/changelogs/v22.8.12.45-lts.md index 7412784419c..7fde9de07f2 100644 --- a/docs/changelogs/v22.8.12.45-lts.md +++ b/docs/changelogs/v22.8.12.45-lts.md @@ -39,4 +39,3 @@ sidebar_label: 2023 * Add check for submodules sanity [#44386](https://github.com/ClickHouse/ClickHouse/pull/44386) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Implement a custom central checkout action [#44399](https://github.com/ClickHouse/ClickHouse/pull/44399) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Do not check read result consistency when unwinding [#44956](https://github.com/ClickHouse/ClickHouse/pull/44956) ([Alexander Gololobov](https://github.com/davenger)). - diff --git a/docs/changelogs/v22.8.13.20-lts.md b/docs/changelogs/v22.8.13.20-lts.md index d8dd1bd2b1c..a2ed41fc97d 100644 --- a/docs/changelogs/v22.8.13.20-lts.md +++ b/docs/changelogs/v22.8.13.20-lts.md @@ -21,4 +21,3 @@ sidebar_label: 2023 * Get rid of artifactory in favor of r2 + ch-repos-manager [#45421](https://github.com/ClickHouse/ClickHouse/pull/45421) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Trim refs/tags/ from GITHUB_TAG in release workflow [#45636](https://github.com/ClickHouse/ClickHouse/pull/45636) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Merge pull request [#38262](https://github.com/ClickHouse/ClickHouse/issues/38262) from PolyProgrammist/fix-ordinary-system-un… [#45650](https://github.com/ClickHouse/ClickHouse/pull/45650) ([alesapin](https://github.com/alesapin)). - diff --git a/docs/changelogs/v22.8.14.53-lts.md b/docs/changelogs/v22.8.14.53-lts.md index 5978080fa3a..0b5170953eb 100644 --- a/docs/changelogs/v22.8.14.53-lts.md +++ b/docs/changelogs/v22.8.14.53-lts.md @@ -37,4 +37,3 @@ sidebar_label: 2023 * Add helping logging to auto-merge script [#46080](https://github.com/ClickHouse/ClickHouse/pull/46080) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Fix write buffer destruction order for vertical merge. [#46205](https://github.com/ClickHouse/ClickHouse/pull/46205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Get rid of legacy DocsReleaseChecks [#46665](https://github.com/ClickHouse/ClickHouse/pull/46665) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v22.8.15.23-lts.md b/docs/changelogs/v22.8.15.23-lts.md index 096a504c9c2..faa5dcd9860 100644 --- a/docs/changelogs/v22.8.15.23-lts.md +++ b/docs/changelogs/v22.8.15.23-lts.md @@ -25,4 +25,3 @@ sidebar_label: 2023 * Reduce updates of Mergeable Check [#46781](https://github.com/ClickHouse/ClickHouse/pull/46781) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Update typing for a new PyGithub version [#47123](https://github.com/ClickHouse/ClickHouse/pull/47123) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Add a manual trigger for release workflow [#47302](https://github.com/ClickHouse/ClickHouse/pull/47302) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v23.1.1.3077-stable.md b/docs/changelogs/v23.1.1.3077-stable.md index e218be62f09..025d1242b11 100644 --- a/docs/changelogs/v23.1.1.3077-stable.md +++ b/docs/changelogs/v23.1.1.3077-stable.md @@ -589,4 +589,3 @@ sidebar_label: 2023 * Resubmit "Fix possible in-use table after DETACH" [#45566](https://github.com/ClickHouse/ClickHouse/pull/45566) ([Alexander Tokmakov](https://github.com/tavplubix)). * Typo: "Granulesis" --> "Granules" [#45598](https://github.com/ClickHouse/ClickHouse/pull/45598) ([Robert Schulze](https://github.com/rschu1ze)). * Fix version in autogenerated_versions.txt [#45624](https://github.com/ClickHouse/ClickHouse/pull/45624) ([Dmitry Novik](https://github.com/novikd)). - diff --git a/docs/changelogs/v23.1.2.9-stable.md b/docs/changelogs/v23.1.2.9-stable.md index 272a2b95a86..825481d8382 100644 --- a/docs/changelogs/v23.1.2.9-stable.md +++ b/docs/changelogs/v23.1.2.9-stable.md @@ -20,4 +20,3 @@ sidebar_label: 2023 #### NOT FOR CHANGELOG / INSIGNIFICANT * Trim refs/tags/ from GITHUB_TAG in release workflow [#45636](https://github.com/ClickHouse/ClickHouse/pull/45636) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v23.1.3.5-stable.md b/docs/changelogs/v23.1.3.5-stable.md index d4f39894bec..bd4a59ac512 100644 --- a/docs/changelogs/v23.1.3.5-stable.md +++ b/docs/changelogs/v23.1.3.5-stable.md @@ -14,4 +14,3 @@ sidebar_label: 2023 #### NOT FOR CHANGELOG / INSIGNIFICANT * Get rid of progress timestamps in release publishing [#45818](https://github.com/ClickHouse/ClickHouse/pull/45818) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v23.1.4.58-stable.md b/docs/changelogs/v23.1.4.58-stable.md index d1ffe87f58e..035c55863a0 100644 --- a/docs/changelogs/v23.1.4.58-stable.md +++ b/docs/changelogs/v23.1.4.58-stable.md @@ -44,4 +44,3 @@ sidebar_label: 2023 * Fix dependencies for InstallPackagesTestAarch64 [#46597](https://github.com/ClickHouse/ClickHouse/pull/46597) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Get rid of legacy DocsReleaseChecks [#46665](https://github.com/ClickHouse/ClickHouse/pull/46665) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Reduce updates of Mergeable Check [#46781](https://github.com/ClickHouse/ClickHouse/pull/46781) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v23.1.5.24-stable.md b/docs/changelogs/v23.1.5.24-stable.md index 1c2c127a8c3..f490b92e725 100644 --- a/docs/changelogs/v23.1.5.24-stable.md +++ b/docs/changelogs/v23.1.5.24-stable.md @@ -25,4 +25,3 @@ sidebar_label: 2023 * Update typing for a new PyGithub version [#47123](https://github.com/ClickHouse/ClickHouse/pull/47123) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Follow-up to [#46681](https://github.com/ClickHouse/ClickHouse/issues/46681) [#47284](https://github.com/ClickHouse/ClickHouse/pull/47284) ([Alexander Tokmakov](https://github.com/tavplubix)). * Add a manual trigger for release workflow [#47302](https://github.com/ClickHouse/ClickHouse/pull/47302) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v23.2.1.2537-stable.md b/docs/changelogs/v23.2.1.2537-stable.md index 3fdcf6d6571..7feafe7af65 100644 --- a/docs/changelogs/v23.2.1.2537-stable.md +++ b/docs/changelogs/v23.2.1.2537-stable.md @@ -470,4 +470,3 @@ sidebar_label: 2023 #### Testing Improvement * Fixed functional test 00304_http_external_data for s390x. [#45807](https://github.com/ClickHouse/ClickHouse/pull/45807) ([Harry Lee](https://github.com/HarryLeeIBM)). - diff --git a/docs/changelogs/v23.2.2.20-stable.md b/docs/changelogs/v23.2.2.20-stable.md index 60aeaa66cbf..ae41fb24100 100644 --- a/docs/changelogs/v23.2.2.20-stable.md +++ b/docs/changelogs/v23.2.2.20-stable.md @@ -27,4 +27,3 @@ sidebar_label: 2023 * More concise logging at trace level for PREWHERE steps [#46771](https://github.com/ClickHouse/ClickHouse/pull/46771) ([Alexander Gololobov](https://github.com/davenger)). * Reduce updates of Mergeable Check [#46781](https://github.com/ClickHouse/ClickHouse/pull/46781) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v23.2.3.17-stable.md b/docs/changelogs/v23.2.3.17-stable.md index fb2c4e394dc..1735c0e5925 100644 --- a/docs/changelogs/v23.2.3.17-stable.md +++ b/docs/changelogs/v23.2.3.17-stable.md @@ -20,4 +20,3 @@ sidebar_label: 2023 * Use /etc/default/clickhouse in systemd too [#47003](https://github.com/ClickHouse/ClickHouse/pull/47003) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * do flushUntrackedMemory when context switches [#47102](https://github.com/ClickHouse/ClickHouse/pull/47102) ([Sema Checherinda](https://github.com/CheSema)). * Update typing for a new PyGithub version [#47123](https://github.com/ClickHouse/ClickHouse/pull/47123) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/docs/changelogs/v23.2.4.12-stable.md b/docs/changelogs/v23.2.4.12-stable.md index 2b6a689aee5..071fde68298 100644 --- a/docs/changelogs/v23.2.4.12-stable.md +++ b/docs/changelogs/v23.2.4.12-stable.md @@ -17,4 +17,3 @@ sidebar_label: 2023 * Follow-up to [#46681](https://github.com/ClickHouse/ClickHouse/issues/46681) [#47284](https://github.com/ClickHouse/ClickHouse/pull/47284) ([Alexander Tokmakov](https://github.com/tavplubix)). * Add a manual trigger for release workflow [#47302](https://github.com/ClickHouse/ClickHouse/pull/47302) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - diff --git a/utils/changelog/changelog.py b/utils/changelog/changelog.py index 3b66b68193b..6b70952eced 100755 --- a/utils/changelog/changelog.py +++ b/utils/changelog/changelog.py @@ -126,8 +126,8 @@ def get_descriptions(prs: PullRequests) -> Dict[str, List[Description]]: def parse_args() -> argparse.Namespace: parser = argparse.ArgumentParser( formatter_class=argparse.ArgumentDefaultsHelpFormatter, - description="Generate a changelog in MD format between given tags. " - "It fetches all tags and unshallow the git repositore automatically", + description="Generate a changelog in Markdown format between given tags. " + "It fetches all tags and unshallow the git repository automatically", ) parser.add_argument( "-v", @@ -243,6 +243,14 @@ def generate_description(item: PullRequest, repo: Repository) -> Optional[Descri else: i += 1 + # Remove excessive bullets from the entry. + if re.match(r"^[\-\*] ", entry): + entry = entry[2:] + + # Better style. + if re.match(r"^[a-z]", entry): + entry = entry.capitalize() + if not category: # Shouldn't happen, because description check in CI should catch such PRs. # Fall through, so that it shows up in output and the user can fix it. @@ -256,6 +264,14 @@ def generate_description(item: PullRequest, repo: Repository) -> Optional[Descri category = "NOT FOR CHANGELOG / INSIGNIFICANT" return Description(item.number, item.user, item.html_url, item.title, category) + # Normalize bug fixes + if re.match( + r"(?i)bug\Wfix", + category, + ): + category = "Bug Fix (user-visible misbehavior in an official stable release)" + return Description(item.number, item.user, item.html_url, item.title, category) + # Filter out documentations changelog if re.match( r"(?i)doc", From 64db4c478b76e179d067806795a3353d10372410 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 27 Mar 2023 13:19:45 +0000 Subject: [PATCH 407/418] Use one map --- src/Interpreters/ProcessList.cpp | 19 ++++++++++--------- src/Interpreters/ProcessList.h | 5 ++--- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index a86377a37d5..d66d4bdea64 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -362,7 +362,7 @@ QueryStatus::~QueryStatus() { #if !defined(NDEBUG) /// Check that all executors were invalidated. - for (const auto & e : executors) + for (const auto & [_, e] : executors) assert(!e->executor); #endif @@ -400,7 +400,9 @@ CancellationCode QueryStatus::cancelQuery(bool) { /// Create a snapshot of executors under a mutex. std::lock_guard lock(executors_mutex); - executors_snapshot = executors; + executors_snapshot.reserve(executors.size()); + for (const auto & [_, e] : executors) + executors_snapshot.push_back(e); } /// We should call cancel() for each executor with unlocked executors_mutex, because @@ -428,9 +430,8 @@ void QueryStatus::addPipelineExecutor(PipelineExecutor * e) throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled"); std::lock_guard lock(executors_mutex); - assert(!executor_indexes.contains(e)); - executors.push_back(std::make_shared(e)); - executor_indexes[e] = executors.size() - 1; + assert(!executors.contains(e)); + executors[e] = std::make_shared(e); } void QueryStatus::removePipelineExecutor(PipelineExecutor * e) @@ -439,12 +440,12 @@ void QueryStatus::removePipelineExecutor(PipelineExecutor * e) { std::lock_guard lock(executors_mutex); - assert(executor_indexes.contains(e)); - executor_holder = executors[executor_indexes[e]]; - executor_indexes.erase(e); + assert(executors.contains(e)); + executor_holder = executors[e]; + executors.erase(e); } - /// Invalidate executor pointer inside holder, but don't remove holder from the executors (to avoid race with cancelQuery) + /// Invalidate executor pointer inside holder. /// We should do it with released executors_mutex to avoid possible lock order inversion. executor_holder->remove(); } diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index 9e5c91bf2ed..a04beac4901 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -133,9 +133,8 @@ protected: using ExecutorHolderPtr = std::shared_ptr; - /// Array of PipelineExecutors to be cancelled when a cancelQuery is received - std::vector executors; - std::unordered_map executor_indexes; + /// Container of PipelineExecutors to be cancelled when a cancelQuery is received + std::unordered_map executors; enum QueryStreamsStatus { From 1845bfc6446f033475ae18799e20af4daface821 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 27 Mar 2023 14:13:32 +0200 Subject: [PATCH 408/418] Revert change to ld64.lld linker on Mac This reverts PRs #42470, #47673 and #47744. The problem was that after the switch to ld64.lld, server binaries build in Debug mode no longer came up, see the discussion after (*). My attempt to fix it with `-no_compact_unwind` didn't help. I also tried - `-keep_dwarf_unwind`, - `-unwindlib` (to use the OS unwinder), - the unwinder in contrib (CMake variable EXCEPTION_HANDLING_LIBRARY) but w/o success. Just tons of wasted time. Rolling back to lld as linker on Mac. This will bring back many linker warnings (#42282) but I rather accept that (temporarily, maybe someone can figure out how to fix them) than have a broken Debug binary. (*) https://github.com/ClickHouse/ClickHouse/pull/42470#issuecomment-1312344068 --- CMakeLists.txt | 14 ++------------ cmake/tools.cmake | 6 +----- 2 files changed, 3 insertions(+), 17 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 5550a19b699..bd7de46474b 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -178,7 +178,7 @@ endif () if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE") # Can be lld or ld-lld or lld-13 or /path/to/lld. - if (LINKER_NAME MATCHES "lld" AND OS_LINUX) + if (LINKER_NAME MATCHES "lld") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--gdb-index") message (STATUS "Adding .gdb-index via --gdb-index linker option.") endif () @@ -212,7 +212,7 @@ endif () # Create BuildID when using lld. For other linkers it is created by default. # (NOTE: LINKER_NAME can be either path or name, and in different variants) -if (LINKER_NAME MATCHES "lld" AND OS_LINUX) +if (LINKER_NAME MATCHES "lld") # SHA1 is not cryptographically secure but it is the best what lld is offering. set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--build-id=sha1") endif () @@ -349,17 +349,7 @@ set (CMAKE_ASM_FLAGS_DEBUG "${CMAKE_ASM_FLAGS_DEBUG} -O0 ${DEBUG_I if (COMPILER_CLANG) if (OS_DARWIN) set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") - set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,-U,_inside_main") - - # The LLVM MachO linker (ld64.lld, used in native builds) generates by default unwind info in 'compact' format which the internal - # unwinder doesn't support and the server will not come up ('invalid compact unwind encoding'). Disable it. You will see warning - # during the build "ld64.lld: warning: Option `-no_compact_unwind' is undocumented. Should lld implement it?". Yes, ld64.lld does - # not document the option, likely for compat with Apple's system ld after which ld64.lld is modeled after and which also does not - # document it. - if (NOT CMAKE_CROSSCOMPILING) - set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,-no_compact_unwind") - endif () endif() # Display absolute paths in error messages. Otherwise KDevelop fails to navigate to correct file and opens a new file instead. diff --git a/cmake/tools.cmake b/cmake/tools.cmake index 974b0bd1d3d..3572134d89f 100644 --- a/cmake/tools.cmake +++ b/cmake/tools.cmake @@ -61,17 +61,13 @@ if (NOT LINKER_NAME) if (COMPILER_GCC) find_program (LLD_PATH NAMES "ld.lld") elseif (COMPILER_CLANG) - # llvm lld is a generic driver. - # Invoke ld.lld (Unix), ld64.lld (macOS), lld-link (Windows), wasm-ld (WebAssembly) instead if (OS_LINUX) if (NOT ARCH_S390X) # s390x doesnt support lld find_program (LLD_PATH NAMES "ld.lld-${COMPILER_VERSION_MAJOR}" "ld.lld") endif () - elseif (OS_DARWIN) - find_program (LLD_PATH NAMES "ld64.lld-${COMPILER_VERSION_MAJOR}" "ld64.lld") endif () endif () - if (OS_LINUX OR OS_DARWIN) + if (OS_LINUX) if (LLD_PATH) if (COMPILER_GCC) # GCC driver requires one of supported linker names like "lld". From f1258a932fcaf34e5a6241501ffc9d5519f1fe72 Mon Sep 17 00:00:00 2001 From: nellicus Date: Mon, 27 Mar 2023 16:58:20 +0200 Subject: [PATCH 409/418] fix typo --- docs/en/operations/external-authenticators/ldap.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/external-authenticators/ldap.md b/docs/en/operations/external-authenticators/ldap.md index fa44e6e2978..ee2336e9378 100644 --- a/docs/en/operations/external-authenticators/ldap.md +++ b/docs/en/operations/external-authenticators/ldap.md @@ -120,7 +120,7 @@ Query: CREATE USER my_user IDENTIFIED WITH ldap SERVER 'my_ldap_server'; ``` -## LDAP Exernal User Directory {#ldap-external-user-directory} +## LDAP External User Directory {#ldap-external-user-directory} 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. From eac71397c4bb68b20bb071582ba33b71de84c60d Mon Sep 17 00:00:00 2001 From: rfraposa Date: Mon, 27 Mar 2023 09:19:50 -0600 Subject: [PATCH 410/418] Create marked_dropped_tables.md --- .../system-tables/marked_dropped_tables.md | 37 +++++++++++++++++++ 1 file changed, 37 insertions(+) create mode 100644 docs/en/operations/system-tables/marked_dropped_tables.md diff --git a/docs/en/operations/system-tables/marked_dropped_tables.md b/docs/en/operations/system-tables/marked_dropped_tables.md new file mode 100644 index 00000000000..a260aea1c5e --- /dev/null +++ b/docs/en/operations/system-tables/marked_dropped_tables.md @@ -0,0 +1,37 @@ +--- +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 +``` \ No newline at end of file From 246316dd49c5eca803e67b6018eea03756f3b4e1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 27 Mar 2023 15:23:30 +0000 Subject: [PATCH 411/418] Docs: Document [FULL] keyword in SHOW TABLES Follow-up to #43910 --- docs/en/sql-reference/statements/show.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index a9f0aedccdf..7d5e2845c4a 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -117,7 +117,7 @@ $ watch -n1 "clickhouse-client --query='SHOW PROCESSLIST'" Displays a list of tables. ```sql -SHOW [TEMPORARY] TABLES [{FROM | IN} ] [LIKE | ILIKE | NOT LIKE ''] [LIMIT ] [INTO OUTFILE ] [FORMAT ] +SHOW [FULL] [TEMPORARY] TABLES [{FROM | IN} ] [LIKE | ILIKE | NOT LIKE ''] [LIMIT ] [INTO OUTFILE ] [FORMAT ] ``` If the `FROM` clause is not specified, the query returns the list of tables from the current database. @@ -521,4 +521,4 @@ Outputs the content of the [system.table_engines](../../operations/system-tables **See Also** -- [system.table_engines](../../operations/system-tables/table_engines.md) table \ No newline at end of file +- [system.table_engines](../../operations/system-tables/table_engines.md) table From 15039d33271acfb93c393c3165a72cdf3beac259 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Mon, 27 Mar 2023 09:36:56 -0600 Subject: [PATCH 412/418] Rename file --- .../system-tables/{marked_dropped_tables.md => dropped_tables.md} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename docs/en/operations/system-tables/{marked_dropped_tables.md => dropped_tables.md} (100%) diff --git a/docs/en/operations/system-tables/marked_dropped_tables.md b/docs/en/operations/system-tables/dropped_tables.md similarity index 100% rename from docs/en/operations/system-tables/marked_dropped_tables.md rename to docs/en/operations/system-tables/dropped_tables.md From 1b63b0b043b0cd53569329d5089d29836f2200f2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 27 Mar 2023 17:51:22 +0200 Subject: [PATCH 413/418] Fix stateless tests numbers --- ..._bug.reference => 02695_logical_optimizer_alias_bug.reference} | 0 ...imizer_alias_bug.sql => 02695_logical_optimizer_alias_bug.sql} | 0 ...erence => 02695_storage_join_insert_select_deadlock.reference} | 0 ...deadlock.sql => 02695_storage_join_insert_select_deadlock.sql} | 0 ...checksums.reference => 02696_inverted_idx_checksums.reference} | 0 ...nverted_idx_checksums.sql => 02696_inverted_idx_checksums.sql} | 0 ...cel.reference => 02697_stop_reading_on_first_cancel.reference} | 0 ...g_on_first_cancel.sh => 02697_stop_reading_on_first_cancel.sh} | 0 ...ped_tables.reference => 02698_marked_dropped_tables.reference} | 0 ..._marked_dropped_tables.sql => 02698_marked_dropped_tables.sql} | 0 ...p.reference => 02699_polygons_sym_difference_rollup.reference} | 0 ...erence_rollup.sql => 02699_polygons_sym_difference_rollup.sql} | 0 ..._regexp_operator.reference => 02700_regexp_operator.reference} | 0 .../{25401_regexp_operator.sql => 02700_regexp_operator.sql} | 0 14 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{25340_logical_optimizer_alias_bug.reference => 02695_logical_optimizer_alias_bug.reference} (100%) rename tests/queries/0_stateless/{25340_logical_optimizer_alias_bug.sql => 02695_logical_optimizer_alias_bug.sql} (100%) rename tests/queries/0_stateless/{25340_storage_join_insert_select_deadlock.reference => 02695_storage_join_insert_select_deadlock.reference} (100%) rename tests/queries/0_stateless/{25340_storage_join_insert_select_deadlock.sql => 02695_storage_join_insert_select_deadlock.sql} (100%) rename tests/queries/0_stateless/{25341_inverted_idx_checksums.reference => 02696_inverted_idx_checksums.reference} (100%) rename tests/queries/0_stateless/{25341_inverted_idx_checksums.sql => 02696_inverted_idx_checksums.sql} (100%) rename tests/queries/0_stateless/{25341_stop_reading_on_first_cancel.reference => 02697_stop_reading_on_first_cancel.reference} (100%) rename tests/queries/0_stateless/{25341_stop_reading_on_first_cancel.sh => 02697_stop_reading_on_first_cancel.sh} (100%) rename tests/queries/0_stateless/{25400_marked_dropped_tables.reference => 02698_marked_dropped_tables.reference} (100%) rename tests/queries/0_stateless/{25400_marked_dropped_tables.sql => 02698_marked_dropped_tables.sql} (100%) rename tests/queries/0_stateless/{25401_polygons_sym_difference_rollup.reference => 02699_polygons_sym_difference_rollup.reference} (100%) rename tests/queries/0_stateless/{25401_polygons_sym_difference_rollup.sql => 02699_polygons_sym_difference_rollup.sql} (100%) rename tests/queries/0_stateless/{25401_regexp_operator.reference => 02700_regexp_operator.reference} (100%) rename tests/queries/0_stateless/{25401_regexp_operator.sql => 02700_regexp_operator.sql} (100%) diff --git a/tests/queries/0_stateless/25340_logical_optimizer_alias_bug.reference b/tests/queries/0_stateless/02695_logical_optimizer_alias_bug.reference similarity index 100% rename from tests/queries/0_stateless/25340_logical_optimizer_alias_bug.reference rename to tests/queries/0_stateless/02695_logical_optimizer_alias_bug.reference diff --git a/tests/queries/0_stateless/25340_logical_optimizer_alias_bug.sql b/tests/queries/0_stateless/02695_logical_optimizer_alias_bug.sql similarity index 100% rename from tests/queries/0_stateless/25340_logical_optimizer_alias_bug.sql rename to tests/queries/0_stateless/02695_logical_optimizer_alias_bug.sql diff --git a/tests/queries/0_stateless/25340_storage_join_insert_select_deadlock.reference b/tests/queries/0_stateless/02695_storage_join_insert_select_deadlock.reference similarity index 100% rename from tests/queries/0_stateless/25340_storage_join_insert_select_deadlock.reference rename to tests/queries/0_stateless/02695_storage_join_insert_select_deadlock.reference diff --git a/tests/queries/0_stateless/25340_storage_join_insert_select_deadlock.sql b/tests/queries/0_stateless/02695_storage_join_insert_select_deadlock.sql similarity index 100% rename from tests/queries/0_stateless/25340_storage_join_insert_select_deadlock.sql rename to tests/queries/0_stateless/02695_storage_join_insert_select_deadlock.sql diff --git a/tests/queries/0_stateless/25341_inverted_idx_checksums.reference b/tests/queries/0_stateless/02696_inverted_idx_checksums.reference similarity index 100% rename from tests/queries/0_stateless/25341_inverted_idx_checksums.reference rename to tests/queries/0_stateless/02696_inverted_idx_checksums.reference diff --git a/tests/queries/0_stateless/25341_inverted_idx_checksums.sql b/tests/queries/0_stateless/02696_inverted_idx_checksums.sql similarity index 100% rename from tests/queries/0_stateless/25341_inverted_idx_checksums.sql rename to tests/queries/0_stateless/02696_inverted_idx_checksums.sql diff --git a/tests/queries/0_stateless/25341_stop_reading_on_first_cancel.reference b/tests/queries/0_stateless/02697_stop_reading_on_first_cancel.reference similarity index 100% rename from tests/queries/0_stateless/25341_stop_reading_on_first_cancel.reference rename to tests/queries/0_stateless/02697_stop_reading_on_first_cancel.reference diff --git a/tests/queries/0_stateless/25341_stop_reading_on_first_cancel.sh b/tests/queries/0_stateless/02697_stop_reading_on_first_cancel.sh similarity index 100% rename from tests/queries/0_stateless/25341_stop_reading_on_first_cancel.sh rename to tests/queries/0_stateless/02697_stop_reading_on_first_cancel.sh diff --git a/tests/queries/0_stateless/25400_marked_dropped_tables.reference b/tests/queries/0_stateless/02698_marked_dropped_tables.reference similarity index 100% rename from tests/queries/0_stateless/25400_marked_dropped_tables.reference rename to tests/queries/0_stateless/02698_marked_dropped_tables.reference diff --git a/tests/queries/0_stateless/25400_marked_dropped_tables.sql b/tests/queries/0_stateless/02698_marked_dropped_tables.sql similarity index 100% rename from tests/queries/0_stateless/25400_marked_dropped_tables.sql rename to tests/queries/0_stateless/02698_marked_dropped_tables.sql diff --git a/tests/queries/0_stateless/25401_polygons_sym_difference_rollup.reference b/tests/queries/0_stateless/02699_polygons_sym_difference_rollup.reference similarity index 100% rename from tests/queries/0_stateless/25401_polygons_sym_difference_rollup.reference rename to tests/queries/0_stateless/02699_polygons_sym_difference_rollup.reference diff --git a/tests/queries/0_stateless/25401_polygons_sym_difference_rollup.sql b/tests/queries/0_stateless/02699_polygons_sym_difference_rollup.sql similarity index 100% rename from tests/queries/0_stateless/25401_polygons_sym_difference_rollup.sql rename to tests/queries/0_stateless/02699_polygons_sym_difference_rollup.sql diff --git a/tests/queries/0_stateless/25401_regexp_operator.reference b/tests/queries/0_stateless/02700_regexp_operator.reference similarity index 100% rename from tests/queries/0_stateless/25401_regexp_operator.reference rename to tests/queries/0_stateless/02700_regexp_operator.reference diff --git a/tests/queries/0_stateless/25401_regexp_operator.sql b/tests/queries/0_stateless/02700_regexp_operator.sql similarity index 100% rename from tests/queries/0_stateless/25401_regexp_operator.sql rename to tests/queries/0_stateless/02700_regexp_operator.sql From 86f682855687cda2494acf33fa9cc63183da0883 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 27 Mar 2023 15:56:31 +0000 Subject: [PATCH 414/418] Docs: Update syntax of some SHOW queries --- docs/en/sql-reference/statements/show.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index 7d5e2845c4a..544c556d4b3 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -21,13 +21,13 @@ Note that if you use this statement to get `CREATE` query of system tables, you Prints a list of all databases. ```sql -SHOW DATABASES [LIKE | ILIKE | NOT LIKE ''] [LIMIT ] [INTO OUTFILE filename] [FORMAT format] +SHOW DATABASES [[NOT] LIKE | ILIKE ''] [LIMIT ] [INTO OUTFILE filename] [FORMAT format] ``` This statement is identical to the query: ```sql -SELECT name FROM system.databases [WHERE name LIKE | ILIKE | NOT LIKE ''] [LIMIT ] [INTO OUTFILE filename] [FORMAT format] +SELECT name FROM system.databases [WHERE name [NOT] LIKE | ILIKE ''] [LIMIT ] [INTO OUTFILE filename] [FORMAT format] ``` ### Examples @@ -117,7 +117,7 @@ $ watch -n1 "clickhouse-client --query='SHOW PROCESSLIST'" Displays a list of tables. ```sql -SHOW [FULL] [TEMPORARY] TABLES [{FROM | IN} ] [LIKE | ILIKE | NOT LIKE ''] [LIMIT ] [INTO OUTFILE ] [FORMAT ] +SHOW [FULL] [TEMPORARY] TABLES [{FROM | IN} ] [[NOT] LIKE | ILIKE ''] [LIMIT ] [INTO OUTFILE ] [FORMAT ] ``` If the `FROM` clause is not specified, the query returns the list of tables from the current database. @@ -125,7 +125,7 @@ If the `FROM` clause is not specified, the query returns the list of tables from This statement is identical to the query: ```sql -SELECT name FROM system.tables [WHERE name LIKE | ILIKE | NOT LIKE ''] [LIMIT ] [INTO OUTFILE ] [FORMAT ] +SELECT name FROM system.tables [WHERE name [NOT] LIKE | ILIKE ''] [LIMIT ] [INTO OUTFILE ] [FORMAT ] ``` ### Examples @@ -370,7 +370,7 @@ Returns a list of clusters. All available clusters are listed in the [system.clu ``` sql SHOW CLUSTER '' -SHOW CLUSTERS [LIKE|NOT LIKE ''] [LIMIT ] +SHOW CLUSTERS [[NOT] LIKE|ILIKE ''] [LIMIT ] ``` ### Examples From feac46eac4b06ff21addd4985559dc7b53e28c2e Mon Sep 17 00:00:00 2001 From: Rich Raposa Date: Mon, 27 Mar 2023 11:16:28 -0600 Subject: [PATCH 415/418] Update dropped_tables.md (#48067) * Update dropped_tables.md * Update dropped_tables.md * Update docs/en/operations/system-tables/dropped_tables.md --------- Co-authored-by: Alexander Tokmakov --- docs/en/operations/system-tables/dropped_tables.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/operations/system-tables/dropped_tables.md b/docs/en/operations/system-tables/dropped_tables.md index a260aea1c5e..cb6cec0035a 100644 --- a/docs/en/operations/system-tables/dropped_tables.md +++ b/docs/en/operations/system-tables/dropped_tables.md @@ -1,7 +1,7 @@ --- -slug: /en/operations/system-tables/marked_dropped_tables +slug: /en/operations/system-tables/dropped_tables --- -# marked_dropped_tables +# dropped_tables Contains information about tables that drop table has been executed but data cleanup has not been actually performed. @@ -17,11 +17,11 @@ Columns: **Example** -The following example shows how to get information about marked_dropped_tables. +The following example shows how to get information about dropped_tables. ``` sql SELECT * -FROM system.marked_dropped_tables\G +FROM system.dropped_tables\G ``` ``` text @@ -34,4 +34,4 @@ 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 -``` \ No newline at end of file +``` From 98488f30ab25062e789acf40b6a8d13cc6b1e0cb Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 27 Mar 2023 14:40:45 -0300 Subject: [PATCH 416/418] Update settings.md --- docs/en/operations/settings/settings.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index f3c0f20f3a6..eb94836b3ba 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -988,6 +988,17 @@ The interval in microseconds for checking whether request execution has been can Default value: 100,000 (checks for cancelling and sends the progress ten times per second). +## idle_connection_timeout {#idle_connection_timeout} + +Timeout to close idle TCP connections after specified number of seconds. + +Possible values: + +- Positive integer. +- 0 — Never. + +Default value: 3600. + ## connect_timeout, receive_timeout, send_timeout {#connect-timeout-receive-timeout-send-timeout} Timeouts in seconds on the socket used for communicating with the client. From 295d3b751013f0f946e7d189d9054bfdef6d3ad4 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 27 Mar 2023 14:45:43 -0300 Subject: [PATCH 417/418] Update settings.md --- docs/en/operations/settings/settings.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index eb94836b3ba..4e5ae6d63cf 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -994,8 +994,7 @@ Timeout to close idle TCP connections after specified number of seconds. Possible values: -- Positive integer. -- 0 — Never. +- Positive integer (0 - close immediatly, after 0 seconds). Default value: 3600. From 6d8a2bbd48d7d753e52b9cfcc75b03bdb9b63dae Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Mon, 27 Mar 2023 14:54:05 -0400 Subject: [PATCH 418/418] standardize admonitions --- .../database-engines/materialized-mysql.md | 4 +-- .../materialized-postgresql.md | 2 +- docs/en/engines/database-engines/mysql.md | 2 +- .../table-engines/integrations/hdfs.md | 4 +-- .../table-engines/integrations/kafka.md | 2 +- .../integrations/materialized-postgresql.md | 2 +- .../table-engines/integrations/postgresql.md | 2 +- .../engines/table-engines/integrations/s3.md | 4 +-- .../mergetree-family/aggregatingmergetree.md | 2 +- .../mergetree-family/collapsingmergetree.md | 2 +- .../custom-partitioning-key.md | 4 +-- .../mergetree-family/graphitemergetree.md | 6 ++-- .../mergetree-family/invertedindexes.md | 2 +- .../mergetree-family/mergetree.md | 4 +-- .../mergetree-family/replacingmergetree.md | 4 +-- .../mergetree-family/replication.md | 2 +- .../mergetree-family/summingmergetree.md | 2 +- .../versionedcollapsingmergetree.md | 2 +- .../engines/table-engines/special/buffer.md | 2 +- docs/en/engines/table-engines/special/file.md | 2 +- .../example-datasets/star-schema.md | 2 +- docs/en/interfaces/formats.md | 28 +++++++++---------- docs/en/interfaces/http.md | 4 +-- docs/en/interfaces/postgresql.md | 2 +- .../third-party/client-libraries.md | 2 +- .../en/interfaces/third-party/integrations.md | 2 +- .../external-authenticators/kerberos.md | 6 ++-- docs/en/operations/opentelemetry.md | 2 +- docs/en/operations/query-cache.md | 2 +- .../settings.md | 4 +-- .../settings/merge-tree-settings.md | 2 +- .../operations/settings/settings-formats.md | 2 +- docs/en/operations/settings/settings-users.md | 2 +- docs/en/operations/settings/settings.md | 20 ++++++------- docs/en/operations/storing-data.md | 2 +- docs/en/operations/system-tables/parts.md | 2 +- .../operations/utilities/clickhouse-copier.md | 2 +- .../parametric-functions.md | 4 +-- docs/en/sql-reference/data-types/float.md | 2 +- docs/en/sql-reference/data-types/json.md | 4 +-- .../data-types/special-data-types/interval.md | 2 +- docs/en/sql-reference/dictionaries/index.md | 6 ++-- docs/en/sql-reference/distributed-ddl.md | 4 +-- .../sql-reference/functions/hash-functions.md | 4 +-- .../sql-reference/functions/introspection.md | 2 +- .../sql-reference/functions/nlp-functions.md | 2 +- .../functions/other-functions.md | 8 +++--- docs/en/sql-reference/operators/exists.md | 2 +- docs/en/sql-reference/operators/index.md | 2 +- .../sql-reference/statements/alter/column.md | 2 +- .../statements/alter/constraint.md | 2 +- .../statements/create/row-policy.md | 2 +- .../sql-reference/statements/create/table.md | 8 +++--- .../sql-reference/statements/create/user.md | 6 ++-- docs/en/sql-reference/statements/optimize.md | 2 +- docs/en/sql-reference/statements/system.md | 2 +- docs/en/sql-reference/statements/watch.md | 4 +-- docs/en/sql-reference/table-functions/file.md | 2 +- docs/en/sql-reference/table-functions/hdfs.md | 2 +- .../table-functions/hdfsCluster.md | 2 +- .../sql-reference/table-functions/iceberg.md | 2 +- .../en/sql-reference/table-functions/index.md | 2 +- docs/en/sql-reference/table-functions/s3.md | 2 +- .../table-functions/s3Cluster.md | 2 +- 64 files changed, 113 insertions(+), 113 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 899c8d024f1..45719b1340e 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -6,7 +6,7 @@ sidebar_position: 70 # [experimental] MaterializedMySQL -:::warning +:::note This is an experimental feature that should not be used in production. ::: @@ -245,7 +245,7 @@ extra care needs to be taken. You may specify overrides for tables that do not exist yet. -:::warning +:::important It is easy to break replication with table overrides if not used with care. For example: * If an ALIAS column is added with a table override, and a column with the same name is later added to the source diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index b43f71a7576..32e3435afa2 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -145,7 +145,7 @@ FROM pg_class WHERE oid = 'postgres_table'::regclass; ``` -:::warning +:::note Replication of [**TOAST**](https://www.postgresql.org/docs/9.5/storage-toast.html) values is not supported. The default value for the data type will be used. ::: diff --git a/docs/en/engines/database-engines/mysql.md b/docs/en/engines/database-engines/mysql.md index e4ff734d55f..7c8c3459ec5 100644 --- a/docs/en/engines/database-engines/mysql.md +++ b/docs/en/engines/database-engines/mysql.md @@ -60,7 +60,7 @@ These variables are supported: - `version` - `max_allowed_packet` -:::warning +:::note By now these variables are stubs and don't correspond to anything. ::: diff --git a/docs/en/engines/table-engines/integrations/hdfs.md b/docs/en/engines/table-engines/integrations/hdfs.md index 7c04a6594a6..1497ea47eca 100644 --- a/docs/en/engines/table-engines/integrations/hdfs.md +++ b/docs/en/engines/table-engines/integrations/hdfs.md @@ -64,7 +64,7 @@ SELECT * FROM hdfs_engine_table LIMIT 2 - Indexes. - [Zero-copy](../../../operations/storing-data.md#zero-copy) replication is possible, but not recommended. - :::warning Zero-copy replication is not ready for production + :::note Zero-copy replication is not ready for production Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use. ::: @@ -110,7 +110,7 @@ Table consists of all the files in both directories (all files should satisfy fo CREATE TABLE table_with_asterisk (name String, value UInt32) ENGINE = HDFS('hdfs://hdfs1:9000/{some,another}_dir/*', 'TSV') ``` -:::warning +:::note If the listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index 255ba06f056..e2a7304dc59 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -102,7 +102,7 @@ Examples: Deprecated Method for Creating a Table -:::warning +:::note Do not use this method in new projects. If possible, switch old projects to the method described above. ::: diff --git a/docs/en/engines/table-engines/integrations/materialized-postgresql.md b/docs/en/engines/table-engines/integrations/materialized-postgresql.md index 11e7928c3ed..3920b402a49 100644 --- a/docs/en/engines/table-engines/integrations/materialized-postgresql.md +++ b/docs/en/engines/table-engines/integrations/materialized-postgresql.md @@ -52,6 +52,6 @@ PRIMARY KEY key; SELECT key, value, _version FROM postgresql_db.postgresql_replica; ``` -:::warning +:::note Replication of [**TOAST**](https://www.postgresql.org/docs/9.5/storage-toast.html) values is not supported. The default value for the data type will be used. ::: diff --git a/docs/en/engines/table-engines/integrations/postgresql.md b/docs/en/engines/table-engines/integrations/postgresql.md index 18e884f3bcc..2222d1fc016 100644 --- a/docs/en/engines/table-engines/integrations/postgresql.md +++ b/docs/en/engines/table-engines/integrations/postgresql.md @@ -74,7 +74,7 @@ All joins, aggregations, sorting, `IN [ array ]` conditions and the `LIMIT` samp PostgreSQL `Array` types are converted into ClickHouse arrays. -:::warning +:::note Be careful - in PostgreSQL an array data, created like a `type_name[]`, may contain multi-dimensional arrays of different dimensions in different table rows in same column. But in ClickHouse it is only allowed to have multidimensional arrays of the same count of dimensions in all table rows in same column. ::: diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index dd843945e10..2901edb9f35 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -63,7 +63,7 @@ For more information about virtual columns see [here](../../../engines/table-eng - Indexes. - [Zero-copy](../../../operations/storing-data.md#zero-copy) replication is possible, but not supported. - :::warning Zero-copy replication is not ready for production + :::note Zero-copy replication is not ready for production Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use. ::: @@ -78,7 +78,7 @@ For more information about virtual columns see [here](../../../engines/table-eng Constructions with `{}` are similar to the [remote](../../../sql-reference/table-functions/remote.md) table function. -:::warning +:::note If the listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md index 9677f75a358..6591f666244 100644 --- a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md @@ -43,7 +43,7 @@ When creating an `AggregatingMergeTree` table the same [clauses](../../../engine Deprecated Method for Creating a Table -:::warning +:::note Do not use this method in new projects and, if possible, switch the old projects to the method described above. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md b/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md index 0bd665116f0..7e16f4926db 100644 --- a/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md @@ -45,7 +45,7 @@ When creating a `CollapsingMergeTree` table, the same [query clauses](../../../e Deprecated Method for Creating a Table -:::warning +:::note Do not use this method in new projects and, if possible, switch old projects to the method described above. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md b/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md index b1e79c4c3fd..f5b8436fdfe 100644 --- a/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md +++ b/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md @@ -6,7 +6,7 @@ sidebar_label: Custom Partitioning Key # Custom Partitioning Key -:::warning +:::note In most cases you do not need a partition key, and in most other cases you do not need a partition key more granular than by months. You should never use too granular of partitioning. Don't partition your data by client identifiers or names. Instead, make a client identifier or name the first column in the ORDER BY expression. @@ -159,7 +159,7 @@ FROM session_log GROUP BY UserID; ``` -:::warning +:::note Performance of such a query heavily depends on the table layout. Because of that the optimisation is not enabled by default. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/graphitemergetree.md b/docs/en/engines/table-engines/mergetree-family/graphitemergetree.md index 104ec049ec4..9577c8dc936 100644 --- a/docs/en/engines/table-engines/mergetree-family/graphitemergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/graphitemergetree.md @@ -55,7 +55,7 @@ When creating a `GraphiteMergeTree` table, the same [clauses](../../../engines/t Deprecated Method for Creating a Table -:::warning +:::note Do not use this method in new projects and, if possible, switch old projects to the method described above. ::: @@ -129,7 +129,7 @@ default ... ``` -:::warning +:::important Patterns must be strictly ordered: 1. Patterns without `function` or `retention`. @@ -263,6 +263,6 @@ Valid values: ``` -:::warning +:::note Data rollup is performed during merges. Usually, for old partitions, merges are not started, so for rollup it is necessary to trigger an unscheduled merge using [optimize](../../../sql-reference/statements/optimize.md). Or use additional tools, for example [graphite-ch-optimizer](https://github.com/innogames/graphite-ch-optimizer). ::: diff --git a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md index aa11258dc4a..701615495de 100644 --- a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md @@ -15,7 +15,7 @@ tokenized cells of the string column. For example, the string cell "I will be a " wi", "wil", "ill", "ll ", "l b", " be" etc. The more fine-granular the input strings are tokenized, the bigger but also the more useful the resulting inverted index will be. -:::warning +:::note Inverted indexes are experimental and should not be used in production environments yet. They may change in the future in backward-incompatible ways, for example with respect to their DDL/DQL syntax or performance/compression characteristics. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index f1b7a40094d..fa97579c33e 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -192,7 +192,7 @@ The `index_granularity` setting can be omitted because 8192 is the default value Deprecated Method for Creating a Table -:::warning +:::note Do not use this method in new projects. If possible, switch old projects to the method described above. ::: @@ -1092,7 +1092,7 @@ Other parameters: Examples of working configurations can be found in integration tests directory (see e.g. [test_merge_tree_azure_blob_storage](https://github.com/ClickHouse/ClickHouse/blob/master/tests/integration/test_merge_tree_azure_blob_storage/configs/config.d/storage_conf.xml) or [test_azure_blob_storage_zero_copy_replication](https://github.com/ClickHouse/ClickHouse/blob/master/tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml)). - :::warning Zero-copy replication is not ready for production + :::note Zero-copy replication is not ready for production Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md b/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md index f5d81182898..8351a31db55 100644 --- a/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md @@ -30,7 +30,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] For a description of request parameters, see [statement description](../../../sql-reference/statements/create/table.md). -:::warning +:::note Uniqueness of rows is determined by the `ORDER BY` table section, not `PRIMARY KEY`. ::: @@ -96,7 +96,7 @@ When creating a `ReplacingMergeTree` table the same [clauses](../../../engines/t Deprecated Method for Creating a Table -:::warning +:::note Do not use this method in new projects and, if possible, switch old projects to the method described above. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/replication.md b/docs/en/engines/table-engines/mergetree-family/replication.md index c50433f2aeb..b2b967c685a 100644 --- a/docs/en/engines/table-engines/mergetree-family/replication.md +++ b/docs/en/engines/table-engines/mergetree-family/replication.md @@ -43,7 +43,7 @@ ClickHouse uses [ClickHouse Keeper](/docs/en/guides/sre/keeper/index.md) for sto To use replication, set parameters in the [zookeeper](/docs/en/operations/server-configuration-parameters/settings.md/#server-settings_zookeeper) server configuration section. -:::warning +:::note Don’t neglect the security setting. ClickHouse supports the `digest` [ACL scheme](https://zookeeper.apache.org/doc/current/zookeeperProgrammers.html#sc_ZooKeeperAccessControl) of the ZooKeeper security subsystem. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/summingmergetree.md b/docs/en/engines/table-engines/mergetree-family/summingmergetree.md index b2b6272c58e..d0078656b5d 100644 --- a/docs/en/engines/table-engines/mergetree-family/summingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/summingmergetree.md @@ -44,7 +44,7 @@ When creating a `SummingMergeTree` table the same [clauses](../../../engines/tab Deprecated Method for Creating a Table -:::warning +:::note Do not use this method in new projects and, if possible, switch the old projects to the method described above. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md b/docs/en/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md index 2891907f79a..74ac9c97fc0 100644 --- a/docs/en/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md @@ -58,7 +58,7 @@ When creating a `VersionedCollapsingMergeTree` table, the same [clauses](../../. Deprecated Method for Creating a Table -:::warning +:::note Do not use this method in new projects. If possible, switch old projects to the method described above. ::: diff --git a/docs/en/engines/table-engines/special/buffer.md b/docs/en/engines/table-engines/special/buffer.md index 5f28fafbc26..f7d84b9b452 100644 --- a/docs/en/engines/table-engines/special/buffer.md +++ b/docs/en/engines/table-engines/special/buffer.md @@ -86,7 +86,7 @@ If the set of columns in the Buffer table does not match the set of columns in a If the types do not match for one of the columns in the Buffer table and a subordinate table, an error message is entered in the server log, and the buffer is cleared. The same happens if the subordinate table does not exist when the buffer is flushed. -:::warning +:::note Running ALTER on the Buffer table in releases made before 26 Oct 2021 will cause a `Block structure mismatch` error (see [#15117](https://github.com/ClickHouse/ClickHouse/issues/15117) and [#30565](https://github.com/ClickHouse/ClickHouse/pull/30565)), so deleting the Buffer table and then recreating is the only option. Check that this error is fixed in your release before trying to run ALTER on the Buffer table. ::: diff --git a/docs/en/engines/table-engines/special/file.md b/docs/en/engines/table-engines/special/file.md index 8314c511236..e99b1c83cbc 100644 --- a/docs/en/engines/table-engines/special/file.md +++ b/docs/en/engines/table-engines/special/file.md @@ -31,7 +31,7 @@ When creating table using `File(Format)` it creates empty subdirectory in that f You may manually create this subfolder and file in server filesystem and then [ATTACH](../../../sql-reference/statements/attach.md) it to table information with matching name, so you can query data from that file. -:::warning +:::note Be careful with this functionality, because ClickHouse does not keep track of external changes to such files. The result of simultaneous writes via ClickHouse and outside of ClickHouse is undefined. ::: diff --git a/docs/en/getting-started/example-datasets/star-schema.md b/docs/en/getting-started/example-datasets/star-schema.md index 1702be70410..72ced87ef55 100644 --- a/docs/en/getting-started/example-datasets/star-schema.md +++ b/docs/en/getting-started/example-datasets/star-schema.md @@ -18,7 +18,7 @@ $ make Generating data: -:::warning +:::note With `-s 100` dbgen generates 600 million rows (67 GB), while while `-s 1000` it generates 6 billion rows (which takes a lot of time) ::: diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index ae3756d5d41..8430946a6c6 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -205,7 +205,7 @@ Differs from the `TabSeparated` format in that the column names are written in t During parsing, the first row is expected to contain the column names. You can use column names to determine their position and to check their correctness. -:::warning +:::note If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from the input data will be mapped to the columns of the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -217,7 +217,7 @@ This format is also available under the name `TSVWithNames`. Differs from the `TabSeparated` format in that the column names are written to the first row, while the column types are in the second row. -:::warning +:::note If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from the input data will be mapped to the columns in the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -470,7 +470,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe Also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames). -:::warning +:::note If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -480,7 +480,7 @@ Otherwise, the first row will be skipped. Also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes). -:::warning +:::note If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -500,7 +500,7 @@ There is also `CustomSeparatedIgnoreSpaces` format, which is similar to [Templat Also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames). -:::warning +:::note If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -510,7 +510,7 @@ Otherwise, the first row will be skipped. Also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes). -:::warning +:::note If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -969,7 +969,7 @@ Differs from `JSONEachRow`/`JSONStringsEachRow` in that ClickHouse will also yie Differs from `JSONCompactEachRow` format in that it also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames). -:::warning +:::note If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -979,7 +979,7 @@ Otherwise, the first row will be skipped. Differs from `JSONCompactEachRow` format in that it also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes). -:::warning +:::note If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -991,7 +991,7 @@ the types from input data will be compared with the types of the corresponding c Differs from `JSONCompactStringsEachRow` in that in that it also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames). -:::warning +:::note If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -1001,7 +1001,7 @@ Otherwise, the first row will be skipped. Differs from `JSONCompactStringsEachRow` in that it also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes). -:::warning +:::note If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -1120,7 +1120,7 @@ CREATE TABLE IF NOT EXISTS example_table - If `input_format_defaults_for_omitted_fields = 0`, then the default value for `x` and `a` equals `0` (as the default value for the `UInt32` data type). - If `input_format_defaults_for_omitted_fields = 1`, then the default value for `x` equals `0`, but the default value of `a` equals `x * 2`. -:::warning +:::note When inserting data with `input_format_defaults_for_omitted_fields = 1`, ClickHouse consumes more computational resources, compared to insertion with `input_format_defaults_for_omitted_fields = 0`. ::: @@ -1450,7 +1450,7 @@ Similar to [RowBinary](#rowbinary), but with added header: - [LEB128](https://en.wikipedia.org/wiki/LEB128)-encoded number of columns (N) - N `String`s specifying column names -:::warning +:::note If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -1464,7 +1464,7 @@ Similar to [RowBinary](#rowbinary), but with added header: - N `String`s specifying column names - N `String`s specifying column types -:::warning +:::note If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -1915,7 +1915,7 @@ SET format_avro_schema_registry_url = 'http://schema-registry'; SELECT * FROM topic1_stream; ``` -:::warning +:::note Setting `format_avro_schema_registry_url` needs to be configured in `users.xml` to maintain it’s value after a restart. Also you can use the `format_avro_schema_registry_url` setting of the `Kafka` table engine. ::: diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 32f323a63d5..4bc108cac7c 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -446,7 +446,7 @@ Next are the configuration methods for different `type`. The following example defines the values of [max_threads](../operations/settings/settings.md#settings-max_threads) and `max_final_threads` settings, then queries the system table to check whether these settings were set successfully. -:::warning +:::note To keep the default `handlers` such as` query`, `play`,` ping`, add the `` rule. ::: @@ -477,7 +477,7 @@ $ curl -H 'XXX:TEST_HEADER_VALUE' -H 'PARAMS_XXX:max_threads' 'http://localhost: max_final_threads 2 ``` -:::warning +:::note In one `predefined_query_handler` only supports one `query` of an insert type. ::: diff --git a/docs/en/interfaces/postgresql.md b/docs/en/interfaces/postgresql.md index f7a619ca620..1146274b012 100644 --- a/docs/en/interfaces/postgresql.md +++ b/docs/en/interfaces/postgresql.md @@ -54,7 +54,7 @@ default=> And that's it! You now have a PostgreSQL client connected to ClickHouse, and all commands and queries are executed on ClickHouse. -:::caution +:::note The PostgreSQL protocol currently only supports plain-text passwords. ::: diff --git a/docs/en/interfaces/third-party/client-libraries.md b/docs/en/interfaces/third-party/client-libraries.md index 0e065cb7179..1069a04391f 100644 --- a/docs/en/interfaces/third-party/client-libraries.md +++ b/docs/en/interfaces/third-party/client-libraries.md @@ -6,7 +6,7 @@ sidebar_label: Client Libraries # Client Libraries from Third-party Developers -:::warning +:::note ClickHouse Inc does **not** maintain the libraries listed below and hasn’t done any extensive testing to ensure their quality. ::: diff --git a/docs/en/interfaces/third-party/integrations.md b/docs/en/interfaces/third-party/integrations.md index 90a4f088be7..a089b3eff17 100644 --- a/docs/en/interfaces/third-party/integrations.md +++ b/docs/en/interfaces/third-party/integrations.md @@ -6,7 +6,7 @@ sidebar_label: Integrations # Integration Libraries from Third-party Developers -:::warning Disclaimer +:::note Disclaimer ClickHouse, Inc. does **not** maintain the tools and libraries listed below and haven’t done extensive testing to ensure their quality. ::: diff --git a/docs/en/operations/external-authenticators/kerberos.md b/docs/en/operations/external-authenticators/kerberos.md index b7a11d7445b..27dc23c9792 100644 --- a/docs/en/operations/external-authenticators/kerberos.md +++ b/docs/en/operations/external-authenticators/kerberos.md @@ -59,11 +59,11 @@ With filtering by realm: ``` -:::warning +:::note You can define only one `kerberos` section. The presence of multiple `kerberos` sections will force ClickHouse to disable Kerberos authentication. ::: -:::warning +:::note `principal` and `realm` sections cannot be specified at the same time. The presence of both `principal` and `realm` sections will force ClickHouse to disable Kerberos authentication. ::: @@ -103,7 +103,7 @@ Example (goes into `users.xml`): ``` -:::warning +:::note Note that Kerberos authentication cannot be used alongside with any other authentication mechanism. The presence of any other sections like `password` alongside `kerberos` will force ClickHouse to shutdown. ::: diff --git a/docs/en/operations/opentelemetry.md b/docs/en/operations/opentelemetry.md index 1de5a09db0c..15185f7ae6b 100644 --- a/docs/en/operations/opentelemetry.md +++ b/docs/en/operations/opentelemetry.md @@ -7,7 +7,7 @@ title: "[experimental] Tracing ClickHouse with OpenTelemetry" [OpenTelemetry](https://opentelemetry.io/) is an open standard for collecting traces and metrics from the distributed application. ClickHouse has some support for OpenTelemetry. -:::warning +:::note This is an experimental feature that will change in backwards-incompatible ways in future releases. ::: diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index a5afee63e6e..ff4236ab8c3 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -29,7 +29,7 @@ Transactionally inconsistent caching is traditionally provided by client tools o the same caching logic and configuration is often duplicated. With ClickHouse's query cache, the caching logic moves to the server side. This reduces maintenance effort and avoids redundancy. -:::warning +:::note The query cache is an experimental feature that should not be used in production. There are known cases (e.g. in distributed query processing) where wrong results are returned. ::: diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index bd0fb03bad0..c2c577d3d79 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -25,7 +25,7 @@ Default value: 3600. Data compression settings for [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md)-engine tables. -:::warning +:::note Don’t use it if you have just started using ClickHouse. ::: @@ -1368,7 +1368,7 @@ The following settings are available: Changed settings take effect immediately. -:::warning +:::note Data for the query cache is allocated in DRAM. If memory is scarce, make sure to set a small value for `max_size` or disable the query cache altogether. ::: diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 5bc174727ad..7e77394729a 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -289,7 +289,7 @@ Default value: 0 (seconds) When this setting has a value greater than than zero only a single replica starts the merge immediately if merged part on shared storage and `allow_remote_fs_zero_copy_replication` is enabled. -:::warning Zero-copy replication is not ready for production +:::note Zero-copy replication is not ready for production Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use. ::: diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index aa991cd9f15..40ca914b738 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -142,7 +142,7 @@ y Nullable(String) z IPv4 ``` -:::warning +:::note If the `schema_inference_hints` is not formated properly, or if there is a typo or a wrong datatype, etc... the whole schema_inference_hints will be ignored. ::: diff --git a/docs/en/operations/settings/settings-users.md b/docs/en/operations/settings/settings-users.md index 9b27af61851..bafac853377 100644 --- a/docs/en/operations/settings/settings-users.md +++ b/docs/en/operations/settings/settings-users.md @@ -118,7 +118,7 @@ To open access for user from any network, specify: ::/0 ``` -:::warning +:::note It’s insecure to open access from any network unless you have a firewall properly configured or the server is not directly connected to Internet. ::: diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index f3c0f20f3a6..239931b6757 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -460,7 +460,7 @@ Possible values: Changes the behaviour of join operations with `ANY` strictness. -:::warning +:::note This setting applies only for `JOIN` operations with [Join](../../engines/table-engines/special/join.md) engine tables. ::: @@ -550,7 +550,7 @@ Default value: 64. Enables legacy ClickHouse server behaviour in `ANY INNER|LEFT JOIN` operations. -:::warning +:::note Use this setting only for backward compatibility if your use cases depend on legacy `JOIN` behaviour. ::: @@ -942,7 +942,7 @@ Higher values will lead to higher memory usage. The maximum size of blocks of uncompressed data before compressing for writing to a table. By default, 1,048,576 (1 MiB). Specifying a smaller block size generally leads to slightly reduced compression ratio, the compression and decompression speed increases slightly due to cache locality, and memory consumption is reduced. -:::warning +:::note This is an expert-level setting, and you shouldn't change it if you're just getting started with ClickHouse. ::: @@ -960,7 +960,7 @@ We are writing a UInt32-type column (4 bytes per value). When writing 8192 rows, We are writing a URL column with the String type (average size of 60 bytes per value). When writing 8192 rows, the average will be slightly less than 500 KB of data. Since this is more than 65,536, a compressed block will be formed for each mark. In this case, when reading data from the disk in the range of a single mark, extra data won’t be decompressed. -:::warning +:::note This is an expert-level setting, and you shouldn't change it if you're just getting started with ClickHouse. ::: @@ -1247,7 +1247,7 @@ Possible values: Default value: 1. -:::warning +:::note Disable this setting if you use [max_parallel_replicas](#settings-max_parallel_replicas) without [parallel_replicas_custom_key](#settings-parallel_replicas_custom_key). If [parallel_replicas_custom_key](#settings-parallel_replicas_custom_key) is set, disable this setting only if it's used on a cluster with multiple shards containing multiple replicas. If it's used on a cluster with a single shard and multiple replicas, disabling this setting will have negative effects. @@ -1277,7 +1277,7 @@ Default value: `1`. This options will produce different results depending on the settings used. -:::warning +:::note This setting will produce incorrect results when joins or subqueries are involved, and all tables don't meet certain requirements. See [Distributed Subqueries and max_parallel_replicas](../../sql-reference/operators/in.md/#max_parallel_replica-subqueries) for more details. ::: @@ -2186,7 +2186,7 @@ Default value: 0. This setting also affects broken batches (that may appears because of abnormal server (machine) termination and no `fsync_after_insert`/`fsync_directories` for [Distributed](../../engines/table-engines/special/distributed.md) table engine). ::: -:::warning +:::note You should not rely on automatic batch splitting, since this may hurt performance. ::: @@ -2194,7 +2194,7 @@ You should not rely on automatic batch splitting, since this may hurt performanc Sets the priority ([nice](https://en.wikipedia.org/wiki/Nice_(Unix))) for threads that execute queries. The OS scheduler considers this priority when choosing the next thread to run on each available CPU core. -:::warning +:::note To use this setting, you need to set the `CAP_SYS_NICE` capability. The `clickhouse-server` package sets it up during installation. Some virtual environments do not allow you to set the `CAP_SYS_NICE` capability. In this case, `clickhouse-server` shows a message about it at the start. ::: @@ -2858,11 +2858,11 @@ Possible values: Default value: `0`. -:::warning +:::note Nullable primary key usually indicates bad design. It is forbidden in almost all main stream DBMS. The feature is mainly for [AggregatingMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md) and is not heavily tested. Use with care. ::: -:::warning +:::note Do not enable this feature in version `<= 21.8`. It's not properly implemented and may lead to server crash. ::: diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index e019a3741cf..af7c526e29b 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -471,6 +471,6 @@ Use [http_max_single_read_retries](/docs/en/operations/settings/settings.md/#htt Zero-copy replication is possible, but not recommended, with `S3` and `HDFS` disks. Zero-copy replication means that if the data is stored remotely on several machines and needs to be synchronized, then only the metadata is replicated (paths to the data parts), but not the data itself. -:::warning Zero-copy replication is not ready for production +:::note Zero-copy replication is not ready for production Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use. ::: diff --git a/docs/en/operations/system-tables/parts.md b/docs/en/operations/system-tables/parts.md index 106d3c59dea..e7700562e35 100644 --- a/docs/en/operations/system-tables/parts.md +++ b/docs/en/operations/system-tables/parts.md @@ -99,7 +99,7 @@ Columns: - `move_ttl_info.expression` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Array of expressions. Each expression defines a [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-ttl). -:::warning +:::note The `move_ttl_info.expression` array is kept mostly for backward compatibility, now the simpliest way to check `TTL MOVE` rule is to use the `move_ttl_info.min` and `move_ttl_info.max` fields. ::: diff --git a/docs/en/operations/utilities/clickhouse-copier.md b/docs/en/operations/utilities/clickhouse-copier.md index 87280bc3ba8..ccce7ea1b79 100644 --- a/docs/en/operations/utilities/clickhouse-copier.md +++ b/docs/en/operations/utilities/clickhouse-copier.md @@ -8,7 +8,7 @@ sidebar_label: clickhouse-copier Copies data from the tables in one cluster to tables in another (or the same) cluster. -:::warning +:::note To get a consistent copy, the data in the source tables and partitions should not change during the entire process. ::: diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index 40184c0aa02..3b874dbe7cd 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -90,7 +90,7 @@ Checks whether the sequence contains an event chain that matches the pattern. sequenceMatch(pattern)(timestamp, cond1, cond2, ...) ``` -:::warning +:::note Events that occur at the same second may lay in the sequence in an undefined order affecting the result. ::: @@ -176,7 +176,7 @@ SELECT sequenceMatch('(?1)(?2)')(time, number = 1, number = 2, number = 4) FROM Counts the number of event chains that matched the pattern. The function searches event chains that do not overlap. It starts to search for the next chain after the current chain is matched. -:::warning +:::note Events that occur at the same second may lay in the sequence in an undefined order affecting the result. ::: diff --git a/docs/en/sql-reference/data-types/float.md b/docs/en/sql-reference/data-types/float.md index 38c414fa8cd..c89b24ad235 100644 --- a/docs/en/sql-reference/data-types/float.md +++ b/docs/en/sql-reference/data-types/float.md @@ -6,7 +6,7 @@ sidebar_label: Float32, Float64 # Float32, Float64 -:::warning +:::note If you need accurate calculations, in particular if you work with financial or business data requiring a high precision you should consider using Decimal instead. Floats might lead to inaccurate results as illustrated below: ``` diff --git a/docs/en/sql-reference/data-types/json.md b/docs/en/sql-reference/data-types/json.md index a21898de9a2..f727f0d75f7 100644 --- a/docs/en/sql-reference/data-types/json.md +++ b/docs/en/sql-reference/data-types/json.md @@ -6,7 +6,7 @@ sidebar_label: JSON # JSON -:::warning +:::note 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. ::: @@ -14,7 +14,7 @@ Stores JavaScript Object Notation (JSON) documents in a single column. `JSON` is an alias for `Object('json')`. -:::warning +:::note The JSON data type is an experimental feature. To use it, set `allow_experimental_object_type = 1`. ::: diff --git a/docs/en/sql-reference/data-types/special-data-types/interval.md b/docs/en/sql-reference/data-types/special-data-types/interval.md index 5169bc646c9..0ac5248e36c 100644 --- a/docs/en/sql-reference/data-types/special-data-types/interval.md +++ b/docs/en/sql-reference/data-types/special-data-types/interval.md @@ -8,7 +8,7 @@ sidebar_label: Interval The family of data types representing time and date intervals. The resulting types of the [INTERVAL](../../../sql-reference/operators/index.md#operator-interval) operator. -:::warning +:::note `Interval` data type values can’t be stored in tables. ::: diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 2185e2b31c1..f697b1ecdcf 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -417,7 +417,7 @@ Example: The table contains discounts for each advertiser in the format: 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 +:::note Values of `range_min` and `range_max` should fit in `Int64` type. ::: @@ -706,7 +706,7 @@ Set a large enough cache size. You need to experiment to select the number of ce 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 +:::note Do not use ClickHouse as a source, because it is slow to process queries with random reads. ::: @@ -1888,7 +1888,7 @@ ClickHouse supports the following types of keys: An xml structure can contain either `` or ``. DDL-query must contain single `PRIMARY KEY`. -:::warning +:::note You must not describe key as an attribute. ::: diff --git a/docs/en/sql-reference/distributed-ddl.md b/docs/en/sql-reference/distributed-ddl.md index ff5155391be..d170f3765c2 100644 --- a/docs/en/sql-reference/distributed-ddl.md +++ b/docs/en/sql-reference/distributed-ddl.md @@ -18,6 +18,6 @@ In order to run these queries correctly, each host must have the same cluster de The local version of the query will eventually be executed on each host in the cluster, even if some hosts are currently not available. -:::warning +:::important The order for executing queries within a single host is guaranteed. -::: \ No newline at end of file +::: diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 69dc73e2fb0..2943ba13861 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -125,7 +125,7 @@ SELECT sipHash64Keyed((506097522914230528, 1084818905618843912), array('e','x',' Like [sipHash64](#hash_functions-siphash64) but produces a 128-bit hash value, i.e. the final xor-folding state is done up to 128 bits. -:::warning +:::note This 128-bit variant differs from the reference implementation and it's weaker. This version exists because, when it was written, there was no official 128-bit extension for SipHash. New projects should probably use [sipHash128Reference](#hash_functions-siphash128reference). @@ -165,7 +165,7 @@ Result: Same as [sipHash128](#hash_functions-siphash128) but additionally takes an explicit key argument instead of using a fixed key. -:::warning +:::note This 128-bit variant differs from the reference implementation and it's weaker. This version exists because, when it was written, there was no official 128-bit extension for SipHash. New projects should probably use [sipHash128ReferenceKeyed](#hash_functions-siphash128referencekeyed). diff --git a/docs/en/sql-reference/functions/introspection.md b/docs/en/sql-reference/functions/introspection.md index 9357f75b8e6..c01abd6ed89 100644 --- a/docs/en/sql-reference/functions/introspection.md +++ b/docs/en/sql-reference/functions/introspection.md @@ -8,7 +8,7 @@ sidebar_label: Introspection You can use functions described in this chapter to introspect [ELF](https://en.wikipedia.org/wiki/Executable_and_Linkable_Format) and [DWARF](https://en.wikipedia.org/wiki/DWARF) for query profiling. -:::warning +:::note These functions are slow and may impose security considerations. ::: diff --git a/docs/en/sql-reference/functions/nlp-functions.md b/docs/en/sql-reference/functions/nlp-functions.md index f68448af2be..132a126edee 100644 --- a/docs/en/sql-reference/functions/nlp-functions.md +++ b/docs/en/sql-reference/functions/nlp-functions.md @@ -5,7 +5,7 @@ sidebar_label: NLP title: "[experimental] Natural Language Processing functions" --- -:::warning +:::note This is an experimental feature that is currently in development and is not ready for general use. It will change in unpredictable backwards-incompatible ways in future releases. Set `allow_experimental_nlp_functions = 1` to enable it. ::: diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 011b73405c5..2e44fa5e9f6 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 +:::tip 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 +:::tip 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 +:::tip 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 +:::tip The state is reset for each new data block. ::: diff --git a/docs/en/sql-reference/operators/exists.md b/docs/en/sql-reference/operators/exists.md index 4bc29389c9c..5e96e11b924 100644 --- a/docs/en/sql-reference/operators/exists.md +++ b/docs/en/sql-reference/operators/exists.md @@ -7,7 +7,7 @@ The `EXISTS` operator checks how many records are in the result of a subquery. I `EXISTS` can be used in a [WHERE](../../sql-reference/statements/select/where.md) clause. -:::warning +:::tip References to main query tables and columns are not supported in a subquery. ::: diff --git a/docs/en/sql-reference/operators/index.md b/docs/en/sql-reference/operators/index.md index 0fe7ebbf4b6..ade2d601f80 100644 --- a/docs/en/sql-reference/operators/index.md +++ b/docs/en/sql-reference/operators/index.md @@ -229,7 +229,7 @@ Types of intervals: You can also use a string literal when setting the `INTERVAL` value. For example, `INTERVAL 1 HOUR` is identical to the `INTERVAL '1 hour'` or `INTERVAL '1' hour`. -:::warning +:::tip Intervals with different types can’t be combined. You can’t use expressions like `INTERVAL 4 DAY 1 HOUR`. Specify intervals in units that are smaller or equal to the smallest unit of the interval, for example, `INTERVAL 25 HOUR`. You can use consecutive operations, like in the example below. ::: diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index d580efa4992..921785102a8 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -75,7 +75,7 @@ Deletes the column with the name `name`. If the `IF EXISTS` clause is specified, Deletes data from the file system. Since this deletes entire files, the query is completed almost instantly. -:::warning +:::tip You can’t delete a column if it is referenced by [materialized view](/docs/en/sql-reference/statements/create/view.md/#materialized). Otherwise, it returns an error. ::: diff --git a/docs/en/sql-reference/statements/alter/constraint.md b/docs/en/sql-reference/statements/alter/constraint.md index 844b24d7374..7a8f5809320 100644 --- a/docs/en/sql-reference/statements/alter/constraint.md +++ b/docs/en/sql-reference/statements/alter/constraint.md @@ -17,7 +17,7 @@ See more on [constraints](../../../sql-reference/statements/create/table.md#cons Queries will add or remove metadata about constraints from table so they are processed immediately. -:::warning +:::tip Constraint check **will not be executed** on existing data if it was added. ::: diff --git a/docs/en/sql-reference/statements/create/row-policy.md b/docs/en/sql-reference/statements/create/row-policy.md index 56a57534234..aa0a07747f2 100644 --- a/docs/en/sql-reference/statements/create/row-policy.md +++ b/docs/en/sql-reference/statements/create/row-policy.md @@ -7,7 +7,7 @@ title: "CREATE ROW POLICY" 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 +:::tip 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. ::: diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index b29bf31e26c..3432066864f 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -285,7 +285,7 @@ ENGINE = engine PRIMARY KEY(expr1[, expr2,...]); ``` -:::warning +:::tip You can't combine both ways in one query. ::: @@ -343,7 +343,7 @@ ALTER TABLE codec_example MODIFY COLUMN float_value CODEC(Default); Codecs can be combined in a pipeline, for example, `CODEC(Delta, Default)`. -:::warning +:::tip You can’t decompress ClickHouse database files with external utilities like `lz4`. Instead, use the special [clickhouse-compressor](https://github.com/ClickHouse/ClickHouse/tree/master/programs/compressor) utility. ::: @@ -438,11 +438,11 @@ Encryption codecs: These codecs use a fixed nonce and encryption is therefore deterministic. This makes it compatible with deduplicating engines such as [ReplicatedMergeTree](../../../engines/table-engines/mergetree-family/replication.md) but has a weakness: when the same data block is encrypted twice, the resulting ciphertext will be exactly the same so an adversary who can read the disk can see this equivalence (although only the equivalence, without getting its content). -:::warning +:::note Most engines including the "\*MergeTree" family create index files on disk without applying codecs. This means plaintext will appear on disk if an encrypted column is indexed. ::: -:::warning +:::note If you perform a SELECT query mentioning a specific value in an encrypted column (such as in its WHERE clause), the value may appear in [system.query_log](../../../operations/system-tables/query_log.md). You may want to disable the logging. ::: diff --git a/docs/en/sql-reference/statements/create/user.md b/docs/en/sql-reference/statements/create/user.md index a38523ee37b..b9bd2c1c507 100644 --- a/docs/en/sql-reference/statements/create/user.md +++ b/docs/en/sql-reference/statements/create/user.md @@ -50,7 +50,7 @@ There are multiple ways of user identification: CREATE USER name2 IDENTIFIED WITH plaintext_password BY 'my_password' ``` - :::warning + :::tip The password is stored in a SQL text file in `/var/lib/clickhouse/access`, so it's not a good idea to use `plaintext_password`. Try `sha256_password` instead, as demonstrated next... ::: @@ -76,7 +76,7 @@ There are multiple ways of user identification: ATTACH USER name3 IDENTIFIED WITH sha256_hash BY '0C268556C1680BEF0640AAC1E7187566704208398DA31F03D18C74F5C5BE5053' SALT '4FB16307F5E10048196966DD7E6876AE53DE6A1D1F625488482C75F14A5097C7'; ``` - :::note + :::tip If you have already created a hash value and corresponding salt value for a username, then you can use `IDENTIFIED WITH sha256_hash BY 'hash'` or `IDENTIFIED WITH sha256_hash BY 'hash' SALT 'salt'`. For identification with `sha256_hash` using `SALT` - hash must be calculated from concatenation of 'password' and 'salt'. ::: @@ -109,7 +109,7 @@ Another way of specifying host is to use `@` syntax following the username. Exam - `CREATE USER mira@'localhost'` — Equivalent to the `HOST LOCAL` syntax. - `CREATE USER mira@'192.168.%.%'` — Equivalent to the `HOST LIKE` syntax. -:::warning +:::tip ClickHouse treats `user_name@'address'` as a username as a whole. Thus, technically you can create multiple users with the same `user_name` and different constructions after `@`. However, we do not recommend to do so. ::: diff --git a/docs/en/sql-reference/statements/optimize.md b/docs/en/sql-reference/statements/optimize.md index 78615a2f9ad..427ee75cd5f 100644 --- a/docs/en/sql-reference/statements/optimize.md +++ b/docs/en/sql-reference/statements/optimize.md @@ -7,7 +7,7 @@ title: "OPTIMIZE Statement" This query tries to initialize an unscheduled merge of data parts for tables. -:::warning +:::note `OPTIMIZE` can’t fix the `Too many parts` error. ::: diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index d069ae8413a..b0877d94e8a 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -312,7 +312,7 @@ One may execute query after: Replica attaches locally found parts and sends info about them to Zookeeper. Parts present on a replica before metadata loss are not re-fetched from other ones if not being outdated (so replica restoration does not mean re-downloading all data over the network). -:::warning +:::note Parts in all states are moved to `detached/` folder. Parts active before data loss (committed) are attached. ::: diff --git a/docs/en/sql-reference/statements/watch.md b/docs/en/sql-reference/statements/watch.md index 90d19e6be0e..5230479cbd2 100644 --- a/docs/en/sql-reference/statements/watch.md +++ b/docs/en/sql-reference/statements/watch.md @@ -6,7 +6,7 @@ sidebar_label: WATCH # WATCH Statement (Experimental) -:::warning +:::note This is an experimental feature that may change in backwards-incompatible ways in the future releases. Enable live views and `WATCH` query using `set allow_experimental_live_view = 1`. ::: @@ -107,4 +107,4 @@ The `FORMAT` clause works the same way as for the [SELECT](../../sql-reference/s :::note The [JSONEachRowWithProgress](../../interfaces/formats.md#jsoneachrowwithprogress) format should be used when watching [LIVE VIEW](./create/view.md#live-view) tables over the HTTP interface. The progress messages will be added to the output to keep the long-lived HTTP connection alive until the query result changes. The interval between progress messages is controlled using the [live_view_heartbeat_interval](./create/view.md#live-view-settings) setting. -::: \ No newline at end of file +::: diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 594c328c3ff..48c2cadc62c 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -109,7 +109,7 @@ Query the number of rows in all files of these two directories: SELECT count(*) FROM file('{some,another}_dir/*', 'TSV', 'name String, value UInt32'); ``` -:::warning +:::note If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index 97a253a5356..1b4588a9b55 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -79,7 +79,7 @@ SELECT count(*) FROM hdfs('hdfs://hdfs1:9000/{some,another}_dir/*', 'TSV', 'name String, value UInt32') ``` -:::warning +:::note If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: diff --git a/docs/en/sql-reference/table-functions/hdfsCluster.md b/docs/en/sql-reference/table-functions/hdfsCluster.md index 231c552610f..546c6a3d1b1 100644 --- a/docs/en/sql-reference/table-functions/hdfsCluster.md +++ b/docs/en/sql-reference/table-functions/hdfsCluster.md @@ -50,7 +50,7 @@ SELECT count(*) FROM hdfsCluster('cluster_simple', 'hdfs://hdfs1:9000/{some,another}_dir/*', 'TSV', 'name String, value UInt32') ``` -:::warning +:::note If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: diff --git a/docs/en/sql-reference/table-functions/iceberg.md b/docs/en/sql-reference/table-functions/iceberg.md index 7c2648d3dcf..713b0f9bbf5 100644 --- a/docs/en/sql-reference/table-functions/iceberg.md +++ b/docs/en/sql-reference/table-functions/iceberg.md @@ -32,7 +32,7 @@ A table with the specified structure for reading data in the specified Iceberg t SELECT * FROM iceberg('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test') ``` -:::warning +:::important ClickHouse currently supports reading v1 (v2 support is coming soon!) of the Iceberg format via the `iceberg` table function and `Iceberg` table engine. ::: diff --git a/docs/en/sql-reference/table-functions/index.md b/docs/en/sql-reference/table-functions/index.md index b49c2f8da20..1010d53e86d 100644 --- a/docs/en/sql-reference/table-functions/index.md +++ b/docs/en/sql-reference/table-functions/index.md @@ -20,6 +20,6 @@ You can use table functions in: - [INSERT INTO TABLE FUNCTION](../../sql-reference/statements/insert-into.md#inserting-into-table-function) query. -:::warning +:::note You can’t use table functions if the [allow_ddl](../../operations/settings/permissions-for-queries.md#settings_allow_ddl) setting is disabled. ::: diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index 99b7832394d..44d1efdee7a 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -118,7 +118,7 @@ FROM s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/ └─────────┘ ``` -:::warning +:::tip If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: diff --git a/docs/en/sql-reference/table-functions/s3Cluster.md b/docs/en/sql-reference/table-functions/s3Cluster.md index f420a69596c..504f92b4dc0 100644 --- a/docs/en/sql-reference/table-functions/s3Cluster.md +++ b/docs/en/sql-reference/table-functions/s3Cluster.md @@ -42,7 +42,7 @@ SELECT * FROM s3Cluster( Count the total amount of rows in all files in the cluster `cluster_simple`: -:::warning +:::tip If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. :::