From 9c6c6d98444d160608690634cf794195fc37eaae Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 17 Jan 2023 20:12:34 +0000 Subject: [PATCH 001/216] 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/216] 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 286a58801e0e2ec770b8169e9a85b65233a30fc4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 27 Jan 2023 18:38:14 +0000 Subject: [PATCH 003/216] 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 004/216] 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 005/216] 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 006/216] 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 007/216] 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 a542626fa35a7f5a37eb98358738554188f74a3c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 6 Feb 2023 18:42:58 +0000 Subject: [PATCH 008/216] 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 009/216] 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 010/216] 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 011/216] 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 012/216] 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 d6ea566b20673cc935823a7b7560dc76678f403c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Feb 2023 16:52:21 +0000 Subject: [PATCH 013/216] 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 014/216] 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 015/216] 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 62bce41d3d2a9bf49d231c381b12d9ab2d412bc8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Feb 2023 19:31:20 +0000 Subject: [PATCH 016/216] 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 f524dae924022d38c967b7edc172439e17b7b7ff Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 15 Feb 2023 19:17:24 +0000 Subject: [PATCH 017/216] 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 018/216] 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 019/216] 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 020/216] 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 021/216] 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 c5f93eb1081de7dfd026ca5ca57771f118cfaef9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 21 Feb 2023 15:44:50 +0000 Subject: [PATCH 022/216] 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 a47860af17be242fbfd492c1debaa3bba852ad91 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 21 Feb 2023 19:50:17 +0000 Subject: [PATCH 023/216] 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 024/216] 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 9749bd491bf8f34476994bdcd8c24308cddd6d60 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 22 Feb 2023 10:26:13 +0000 Subject: [PATCH 025/216] 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 67db9123d85b94ed5831dedd2427ce07ed1b2c38 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 22 Feb 2023 16:40:40 +0000 Subject: [PATCH 026/216] 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 027/216] 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 c63bbbb37401ff7cef7cd2b3355abcc2001b5d3f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 23 Feb 2023 20:22:12 +0000 Subject: [PATCH 028/216] 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 030a0ba7fb7feedefe1efa575cb4823c58fde472 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 24 Feb 2023 18:39:27 +0000 Subject: [PATCH 029/216] 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 53b006dd5cef3501d01e781025e99f097dfa7c36 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 24 Feb 2023 19:59:13 +0000 Subject: [PATCH 030/216] 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 031/216] 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 040067748387c1565e5e73399e61b31d85fa3104 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 27 Feb 2023 15:41:50 +0000 Subject: [PATCH 032/216] 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 9bf828cc98c490f3d405628fbaedd6641c85c737 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 27 Feb 2023 16:43:54 +0000 Subject: [PATCH 033/216] 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 034/216] 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 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 035/216] 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 82cc4afdd69efa5076579bbaf69043b99d698948 Mon Sep 17 00:00:00 2001 From: Aleksei Tikhomirov Date: Thu, 2 Mar 2023 19:25:58 +0300 Subject: [PATCH 036/216] fix_JSON_searchField --- base/base/JSON.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/base/base/JSON.cpp b/base/base/JSON.cpp index 315bcce38da..4c6d97b4444 100644 --- a/base/base/JSON.cpp +++ b/base/base/JSON.cpp @@ -466,9 +466,8 @@ JSON::Pos JSON::searchField(const char * data, size_t size) const { if (!it->hasEscapes()) { - if (static_cast(size) + 2 > it->dataEnd() - it->data()) - continue; - if (!strncmp(data, it->data() + 1, size)) + const auto current_name = it->getRawName(); + if (current_name.size() == size && 0 == memcmp(current_name.data(), data, size)) break; } else From c1e80683c444512a446e4579b000f512996237d5 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 3 Mar 2023 17:57:10 +0100 Subject: [PATCH 037/216] 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 420108a7a05f3cbc5d4230b2fcf1dad0168a9070 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 6 Mar 2023 19:10:36 +0100 Subject: [PATCH 038/216] 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 8186473cc28c730b1f150f62dac971ec0923e2fb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 7 Mar 2023 14:28:28 +0000 Subject: [PATCH 039/216] 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 040/216] 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 de8d0040a4452e72c7d8d00444ce787000e0ce21 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 13 Mar 2023 18:34:47 +0100 Subject: [PATCH 041/216] 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 042/216] 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 01be209e431a4921de7629c54ab069e3a92fc779 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 14 Mar 2023 17:44:02 +0100 Subject: [PATCH 043/216] 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 044/216] 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 1393e9ba035c932487d82b3d0b52085b017b1954 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 15 Mar 2023 01:09:54 +0000 Subject: [PATCH 045/216] 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 d78a9e03ad0381049b8d0322528b6d2261bce048 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 15 Mar 2023 15:38:11 +0100 Subject: [PATCH 046/216] 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 424e8df9ad8493f904f54ca2d9d63f0a32b6b8d7 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 15 Mar 2023 16:01:12 +0100 Subject: [PATCH 047/216] 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 e0954ce7beb9b38c16ebe222fc6abc354d1cd919 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 16 Mar 2023 00:22:05 +0100 Subject: [PATCH 048/216] 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 808f2c0cb4bc35427c91bf9fb81f089f38e7fc84 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Wed, 8 Mar 2023 07:38:52 -0800 Subject: [PATCH 049/216] Fix hashing tuples for s390x --- src/Functions/FunctionsHashing.h | 29 +++++++++++++++++++ .../00746_hashing_tuples.reference | 14 ++++----- .../0_stateless/00746_hashing_tuples.sql | 14 ++++----- 3 files changed, 43 insertions(+), 14 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 7b6f4213cd3..937e3b0023d 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -150,6 +150,13 @@ struct IntHash64Impl template T combineHashesFunc(T t1, T t2) { +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + T tmp; + reverseMemcpy(&tmp, &t1, sizeof(T)); + t1 = tmp; + reverseMemcpy(&tmp, &t2, sizeof(T)); + t2 = tmp; +#endif T hashes[] = {t1, t2}; return HashFunction::apply(reinterpret_cast(hashes), 2 * sizeof(T)); } @@ -183,6 +190,10 @@ struct HalfMD5Impl static UInt64 combineHashes(UInt64 h1, UInt64 h2) { +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + h1 = Poco::ByteOrder::flipBytes(static_cast(h1)); + h2 = Poco::ByteOrder::flipBytes(static_cast(h2)); +#endif UInt64 hashes[] = {h1, h2}; return apply(reinterpret_cast(hashes), 16); } @@ -322,6 +333,10 @@ struct SipHash64KeyedImpl static UInt64 combineHashesKeyed(const Key & key, UInt64 h1, UInt64 h2) { +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + h1 = Poco::ByteOrder::flipBytes(static_cast(h1)); + h2 = Poco::ByteOrder::flipBytes(static_cast(h2)); +#endif UInt64 hashes[] = {h1, h2}; return applyKeyed(key, reinterpret_cast(hashes), 2 * sizeof(UInt64)); } @@ -360,6 +375,13 @@ struct SipHash128KeyedImpl static UInt128 combineHashesKeyed(const Key & key, UInt128 h1, UInt128 h2) { +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + UInt128 tmp; + reverseMemcpy(&tmp, &h1, sizeof(UInt128)); + h1 = tmp; + reverseMemcpy(&tmp, &h2, sizeof(UInt128)); + h2 = tmp; +#endif UInt128 hashes[] = {h1, h2}; return applyKeyed(key, reinterpret_cast(hashes), 2 * sizeof(UInt128)); } @@ -395,6 +417,13 @@ struct SipHash128ReferenceKeyedImpl static UInt128 combineHashesKeyed(const Key & key, UInt128 h1, UInt128 h2) { +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + UInt128 tmp; + reverseMemcpy(&tmp, &h1, sizeof(UInt128)); + h1 = tmp; + reverseMemcpy(&tmp, &h2, sizeof(UInt128)); + h2 = tmp; +#endif UInt128 hashes[] = {h1, h2}; return applyKeyed(key, reinterpret_cast(hashes), 2 * sizeof(UInt128)); } diff --git a/tests/queries/0_stateless/00746_hashing_tuples.reference b/tests/queries/0_stateless/00746_hashing_tuples.reference index 71d45be5a54..e3b896f6077 100644 --- a/tests/queries/0_stateless/00746_hashing_tuples.reference +++ b/tests/queries/0_stateless/00746_hashing_tuples.reference @@ -1,11 +1,11 @@ 12940785793559895259 17926972817233444501 7456555839952096623 -CC45107CC4B79F62D831BEF2103C7CBF -DF2EC2F0669B000EDFF6ADEE264E7D68 -4CD1C30C38AB935D418B5269EF197B9E -9D78134EE48654D753CCA1B76185CF8E -389D16428D2AADEC9713905572F42864 +1 +1 +1 +1 +1 955237314186186656 8175794665478042155 9325786087413524176 @@ -18,8 +18,8 @@ DF2EC2F0669B000EDFF6ADEE264E7D68 8163029322371165472 8788309436660676487 236561483980029756 -8DD5527CC43D76F4760D26BE0F641F7E -F8F7AD9B6CD4CF117A71E277E2EC2931 +1 +1 12384823029245979431 4507350192761038840 1188926775431157506 diff --git a/tests/queries/0_stateless/00746_hashing_tuples.sql b/tests/queries/0_stateless/00746_hashing_tuples.sql index 466a2184c65..328ee5d6f05 100644 --- a/tests/queries/0_stateless/00746_hashing_tuples.sql +++ b/tests/queries/0_stateless/00746_hashing_tuples.sql @@ -4,11 +4,11 @@ SELECT sipHash64(1, 2, 3); SELECT sipHash64(1, 3, 2); SELECT sipHash64(('a', [1, 2, 3], 4, (4, ['foo', 'bar'], 1, (1, 2)))); -SELECT hex(sipHash128('foo')); -SELECT hex(sipHash128('\x01')); -SELECT hex(sipHash128('foo', 'foo')); -SELECT hex(sipHash128('foo', 'foo', 'foo')); -SELECT hex(sipHash128(1, 2, 3)); +SELECT hex(sipHash128('foo')) = hex(reverse(unhex('CC45107CC4B79F62D831BEF2103C7CBF'))) or hex(sipHash128('foo')) = 'CC45107CC4B79F62D831BEF2103C7CBF'; +SELECT hex(sipHash128('\x01')) = hex(reverse(unhex('DF2EC2F0669B000EDFF6ADEE264E7D68'))) or hex(sipHash128('foo')) = 'DF2EC2F0669B000EDFF6ADEE264E7D68'; +SELECT hex(sipHash128('foo', 'foo')) = hex(reverse(unhex('4CD1C30C38AB935D418B5269EF197B9E'))) or hex(sipHash128('foo')) = '4CD1C30C38AB935D418B5269EF197B9E'; +SELECT hex(sipHash128('foo', 'foo', 'foo')) = hex(reverse(unhex('9D78134EE48654D753CCA1B76185CF8E'))) or hex(sipHash128('foo')) = '9D78134EE48654D753CCA1B76185CF8E'; +SELECT hex(sipHash128(1, 2, 3)) = hex(reverse(unhex('389D16428D2AADEC9713905572F42864'))) or hex(sipHash128('foo')) = '389D16428D2AADEC9713905572F42864'; SELECT halfMD5(1, 2, 3); SELECT halfMD5(1, 3, 2); @@ -26,8 +26,8 @@ SELECT murmurHash3_64(1, 2, 3); SELECT murmurHash3_64(1, 3, 2); SELECT murmurHash3_64(('a', [1, 2, 3], 4, (4, ['foo', 'bar'], 1, (1, 2)))); -SELECT hex(murmurHash3_128('foo', 'foo')); -SELECT hex(murmurHash3_128('foo', 'foo', 'foo')); +SELECT hex(murmurHash3_128('foo', 'foo')) = hex(reverse(unhex('8DD5527CC43D76F4760D26BE0F641F7E'))) or hex(sipHash128('foo')) = '8DD5527CC43D76F4760D26BE0F641F7E'; +SELECT hex(murmurHash3_128('foo', 'foo', 'foo')) = hex(reverse(unhex('F8F7AD9B6CD4CF117A71E277E2EC2931'))) or hex(sipHash128('foo')) = 'F8F7AD9B6CD4CF117A71E277E2EC2931'; SELECT gccMurmurHash(1, 2, 3); SELECT gccMurmurHash(1, 3, 2); From 1ba75c6408830751b5d57625faa89a4164163ea4 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Thu, 16 Mar 2023 10:03:43 -0700 Subject: [PATCH 050/216] Refactored by using std::byteswap --- src/Functions/FunctionsHashing.h | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 937e3b0023d..a24b5109963 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -30,7 +30,7 @@ # include #endif -#include +#include #include #include #include @@ -184,15 +184,15 @@ struct HalfMD5Impl #if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ return buf.uint64_data; /// No need to flip bytes on big endian machines #else - return Poco::ByteOrder::flipBytes(static_cast(buf.uint64_data)); /// Compatibility with existing code. Cast need for old poco AND macos where UInt64 != uint64_t + return std::byteswap(buf.uint64_data); /// Compatibility with existing code. Cast need for old poco AND macos where UInt64 != uint64_t #endif } static UInt64 combineHashes(UInt64 h1, UInt64 h2) { #if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - h1 = Poco::ByteOrder::flipBytes(static_cast(h1)); - h2 = Poco::ByteOrder::flipBytes(static_cast(h2)); + h1 = std::byteswap(h1); + h2 = std::byteswap(h2); #endif UInt64 hashes[] = {h1, h2}; return apply(reinterpret_cast(hashes), 16); @@ -334,8 +334,8 @@ struct SipHash64KeyedImpl static UInt64 combineHashesKeyed(const Key & key, UInt64 h1, UInt64 h2) { #if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - h1 = Poco::ByteOrder::flipBytes(static_cast(h1)); - h2 = Poco::ByteOrder::flipBytes(static_cast(h2)); + h1 = std::byteswap(h1); + h2 = std::byteswap(h2); #endif UInt64 hashes[] = {h1, h2}; return applyKeyed(key, reinterpret_cast(hashes), 2 * sizeof(UInt64)); From f4b6205cab3d42b9058e9f594e911fb011325df2 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Thu, 16 Mar 2023 13:29:33 -0700 Subject: [PATCH 051/216] Fix sql in previous commit --- tests/queries/0_stateless/00746_hashing_tuples.sql | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/00746_hashing_tuples.sql b/tests/queries/0_stateless/00746_hashing_tuples.sql index 328ee5d6f05..f17ad6fa77f 100644 --- a/tests/queries/0_stateless/00746_hashing_tuples.sql +++ b/tests/queries/0_stateless/00746_hashing_tuples.sql @@ -5,10 +5,10 @@ SELECT sipHash64(1, 3, 2); SELECT sipHash64(('a', [1, 2, 3], 4, (4, ['foo', 'bar'], 1, (1, 2)))); SELECT hex(sipHash128('foo')) = hex(reverse(unhex('CC45107CC4B79F62D831BEF2103C7CBF'))) or hex(sipHash128('foo')) = 'CC45107CC4B79F62D831BEF2103C7CBF'; -SELECT hex(sipHash128('\x01')) = hex(reverse(unhex('DF2EC2F0669B000EDFF6ADEE264E7D68'))) or hex(sipHash128('foo')) = 'DF2EC2F0669B000EDFF6ADEE264E7D68'; -SELECT hex(sipHash128('foo', 'foo')) = hex(reverse(unhex('4CD1C30C38AB935D418B5269EF197B9E'))) or hex(sipHash128('foo')) = '4CD1C30C38AB935D418B5269EF197B9E'; -SELECT hex(sipHash128('foo', 'foo', 'foo')) = hex(reverse(unhex('9D78134EE48654D753CCA1B76185CF8E'))) or hex(sipHash128('foo')) = '9D78134EE48654D753CCA1B76185CF8E'; -SELECT hex(sipHash128(1, 2, 3)) = hex(reverse(unhex('389D16428D2AADEC9713905572F42864'))) or hex(sipHash128('foo')) = '389D16428D2AADEC9713905572F42864'; +SELECT hex(sipHash128('\x01')) = hex(reverse(unhex('DF2EC2F0669B000EDFF6ADEE264E7D68'))) or hex(sipHash128('\x01')) = 'DF2EC2F0669B000EDFF6ADEE264E7D68'; +SELECT hex(sipHash128('foo', 'foo')) = hex(reverse(unhex('4CD1C30C38AB935D418B5269EF197B9E'))) or hex(sipHash128('foo', 'foo')) = '4CD1C30C38AB935D418B5269EF197B9E'; +SELECT hex(sipHash128('foo', 'foo', 'foo')) = hex(reverse(unhex('9D78134EE48654D753CCA1B76185CF8E'))) or hex(sipHash128('foo', 'foo', 'foo')) = '9D78134EE48654D753CCA1B76185CF8E'; +SELECT hex(sipHash128(1, 2, 3)) = hex(reverse(unhex('389D16428D2AADEC9713905572F42864'))) or hex(sipHash128(1, 2, 3)) = '389D16428D2AADEC9713905572F42864'; SELECT halfMD5(1, 2, 3); SELECT halfMD5(1, 3, 2); @@ -26,8 +26,8 @@ SELECT murmurHash3_64(1, 2, 3); SELECT murmurHash3_64(1, 3, 2); SELECT murmurHash3_64(('a', [1, 2, 3], 4, (4, ['foo', 'bar'], 1, (1, 2)))); -SELECT hex(murmurHash3_128('foo', 'foo')) = hex(reverse(unhex('8DD5527CC43D76F4760D26BE0F641F7E'))) or hex(sipHash128('foo')) = '8DD5527CC43D76F4760D26BE0F641F7E'; -SELECT hex(murmurHash3_128('foo', 'foo', 'foo')) = hex(reverse(unhex('F8F7AD9B6CD4CF117A71E277E2EC2931'))) or hex(sipHash128('foo')) = 'F8F7AD9B6CD4CF117A71E277E2EC2931'; +SELECT hex(murmurHash3_128('foo', 'foo')) = hex(reverse(unhex('8DD5527CC43D76F4760D26BE0F641F7E'))) or hex(murmurHash3_128('foo', 'foo')) = '8DD5527CC43D76F4760D26BE0F641F7E'; +SELECT hex(murmurHash3_128('foo', 'foo', 'foo')) = hex(reverse(unhex('F8F7AD9B6CD4CF117A71E277E2EC2931'))) or hex(murmurHash3_128('foo', 'foo', 'foo')) = 'F8F7AD9B6CD4CF117A71E277E2EC2931'; SELECT gccMurmurHash(1, 2, 3); SELECT gccMurmurHash(1, 3, 2); From 6509d9e525c89987f14ae290962c865072499406 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 17 Mar 2023 18:36:49 +0000 Subject: [PATCH 052/216] 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 c14c083cdff487abfff299fe2b6c08e6a5b28d07 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 17 Mar 2023 19:30:12 -0400 Subject: [PATCH 053/216] 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 054/216] 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 dec59721944a2aeb152c7b1c5ca7de7ee1f3a107 Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Sun, 19 Mar 2023 20:35:24 +0100 Subject: [PATCH 055/216] 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 056/216] 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 057/216] 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 e805d75756fdb3650f4c2f31043286f44b3fbfef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Mar 2023 07:10:47 +0100 Subject: [PATCH 058/216] ClickHouse Obfuscator: add README --- programs/obfuscator/Obfuscator.cpp | 1 - programs/obfuscator/README.md | 354 +++++++++++++++++++++++++++++ 2 files changed, 354 insertions(+), 1 deletion(-) create mode 100644 programs/obfuscator/README.md diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 274ad29a174..add16ec5205 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -30,7 +30,6 @@ #include #include #include -#include #include #include #include diff --git a/programs/obfuscator/README.md b/programs/obfuscator/README.md new file mode 100644 index 00000000000..8c2aaafb3ac --- /dev/null +++ b/programs/obfuscator/README.md @@ -0,0 +1,354 @@ +## clickhouse-obfuscator — a tool for dataset anonymization + +### Installation And Usage + +``` +curl https://clickhouse.com/ | sh +./clickhouse obfuscator --help +``` + +### Example + +``` +./clickhouse obfuscator --seed 123 --input-format TSV --output-format TSV \ + --structure 'CounterID UInt32, URLDomain String, URL String, SearchPhrase String, Title String' \ + < source.tsv > result.tsv +``` + + +### A long, long time ago... + +ClickHouse users already know that its biggest advantage is its high-speed processing of analytical queries. But claims like this need to be confirmed with reliable performance testing. That's what we want to talk about today. + +![benchmarks.png](https://clickhouse.com/uploads/benchmarks_24f1904cc9.png) + +We started running tests in 2013, long before ClickHouse was available as open source. Back then, our main concern was data processing speed for a web analytics product. We started storing this data, which we would later store in ClickHouse, in January 2009. Part of the data had been written to a database starting in 2012, and part was converted from OLAPServer and Metrage (data structures previously used by the solution). For testing, we took the first subset at random from data for 1 billion pageviews. Our web analytics platform didn't have any queries at that point, so we came up with queries that interested us, using all the possible ways to filter, aggregate, and sort the data. + +ClickHouse performance was compared with similar systems like Vertica and MonetDB. To avoid bias, testing was performed by an employee who hadn't participated in ClickHouse development, and special cases in the code were not optimized until all the results were obtained. We used the same approach to get a data set for functional testing. + +After ClickHouse was released as open source in 2016, people began questioning these tests. + +## Shortcomings of tests on private data + +Our performance tests: + +- Couldn't be reproduced independently because they used private data that can't be published. Some of the functional tests are not available to external users for the same reason. +- Needed further development. The set of tests needed to be substantially expanded in order to isolate performance changes in individual parts of the system. +- Didn't run on a per-commit basis or for individual pull requests. External developers couldn't check their code for performance regressions. + +We could solve these problems by throwing out the old tests and writing new ones based on open data, like [flight data for the USA](https://clickhouse.com/docs/en/getting-started/example-datasets/ontime/) and [taxi rides in New York](https://clickhouse.com/docs/en/getting-started/example-datasets/nyc-taxi). Or we could use benchmarks like TPC-H, TPC-DS, and [Star Schema Benchmark](https://clickhouse.com/docs/en/getting-started/example-datasets/star-schema). The disadvantage is that this data was very different from web analytics data, and we would rather keep the test queries. + +### Why it's important to use real data + +Performance should only be tested on real data from a production environment. Let's look at some examples. + +### Example 1 + +Let's say you fill a database with evenly distributed pseudorandom numbers. Data compression isn't going to work in this case, although data compression is essential to analytical databases. There is no silver bullet solution to the challenge of choosing the right compression algorithm and the right way to integrate it into the system since data compression requires a compromise between the speed of compression and decompression and the potential compression efficiency. But systems that can't compress data are guaranteed losers. If your tests use evenly distributed pseudorandom numbers, this factor is ignored, and the results will be distorted. + +Bottom line: Test data must have a realistic compression ratio. + +### Example 2 + +Let's say we are interested in the execution speed of this SQL query: + +```sql +SELECT RegionID, uniq(UserID) AS visitors + FROM test.hits +GROUP BY RegionID +ORDER BY visitors DESC +LIMIT 10 +``` + +This was a typical query for web analytics product. What affects the processing speed? + +- How `GROUP BY` is executed. +- Which data structure is used for calculating the `uniq` aggregate function. +- How many different RegionIDs there are and how much RAM each state of the `uniq` function requires. + +But another important factor is that the amount of data is distributed unevenly between regions. (It probably follows a power law. I put the distribution on a log-log graph, but I can't say for sure.) If this is the case, the states of the `uniq` aggregate function with fewer values must use very little memory. When there are a lot of different aggregation keys, every single byte counts. How can we get generated data that has all these properties? The obvious solution is to use real data. + +Many DBMSs implement the HyperLogLog data structure for an approximation of COUNT(DISTINCT), but none of them work very well because this data structure uses a fixed amount of memory. ClickHouse has a function that uses [a combination of three different data structures](https://clickhouse.com/docs/en/sql-reference/aggregate-functions/reference/uniqcombined), depending on the size of the data set. + +Bottom line: Test data must represent distribution properties of the real data well enough, meaning cardinality (number of distinct values per column) and cross-column cardinality (number of different values counted across several different columns). + +### Example 3 + +Instead of testing the performance of the ClickHouse DBMS, let's take something simpler, like hash tables. For hash tables, it's essential to choose the right hash function. This is not as important for `std::unordered_map`, because it's a hash table based on chaining, and a prime number is used as the array size. The standard library implementation in GCC and Clang uses a trivial hash function as the default hash function for numeric types. However, `std::unordered_map` is not the best choice when we are looking for maximum speed. With an open-addressing hash table, we can't just use a standard hash function. Choosing the right hash function becomes the deciding factor. + +It's easy to find hash table performance tests using random data that don't take the hash functions used into account. Many hash function tests also focus on the calculation speed and certain quality criteria, even though they ignore the data structures used. But the fact is that hash tables and HyperLogLog require different hash function quality criteria. + +![alexey_chat.png](https://clickhouse.com/uploads/alexey_chat_3f8db88301.png) + +## Challenge + +Our goal was to obtain data for testing performance that had the same structure as our web analytics data with all the properties that are important for benchmarks, but in such a way that there remain no traces of real website users in this data. In other words, the data must be anonymized and still preserve its: + +* Compression ratio. +* Cardinality (the number of distinct values). +* Mutual cardinality between several different columns. +* Properties of probability distributions that can be used for data modeling (for example, if we believe that regions are distributed according to a power law, then the exponent — the distribution parameter — should be approximately the same for artificial data and for real data). + +How can we get a similar compression ratio for the data? If LZ4 is used, substrings in binary data must be repeated at approximately the same distance, and the repetitions must be approximately the same length. For ZSTD, entropy per byte must also coincide. + +The ultimate goal was to create a publicly available tool that anyone can use to anonymize their data sets for publication. This would allow us to debug and test performance on other people's data similar to our production data. We would also like the generated data to be interesting. + +However, these are very loosely-defined requirements, and we aren't planning to write up a formal problem statement or specification for this task. + +## Possible solutions + +I don't want to make it sound like this problem was particularly important. It was never actually included in planning, and no one had intentions to work on it. I hoped that an idea would come up someday, and suddenly I would be in a good mood and be able to put everything else off until later. + +### Explicit probabilistic models + +- We want to preserve the continuity of time series data. This means that for some types of data, we need to model the difference between neighboring values rather than the value itself. +- To model "joint cardinality" of columns, we would also have to explicitly reflect dependencies between columns. For instance, there are usually very few IP addresses per user ID, so to generate an IP address, we would have to use a hash value of the user ID as a seed and add a small amount of other pseudorandom data. +- We weren't sure how to express the dependency that the same user frequently visits URLs with matching domains at approximately the same time. + +All this can be written in a C++ "script" with the distributions and dependencies hard coded. However, Markov models are obtained from a combination of statistics with smoothing and adding noise. I started writing a script like this, but after writing explicit models for ten columns, it became unbearably boring — and the "hits" table in the web analytics product had more than 100 columns way back in 2012. + +```c++ +EventTime.day(std::discrete_distribution<>({ + 0, 0, 13, 30, 0, 14, 42, 5, 6, 31, 17, 0, 0, 0, 0, 23, 10, ...})(random)); +EventTime.hour(std::discrete_distribution<>({ + 13, 7, 4, 3, 2, 3, 4, 6, 10, 16, 20, 23, 24, 23, 18, 19, 19, ...})(random)); +EventTime.minute(std::uniform_int_distribution(0, 59)(random)); +EventTime.second(std::uniform_int_distribution(0, 59)(random)); + +UInt64 UserID = hash(4, powerLaw(5000, 1.1)); +UserID = UserID / 10000000000ULL * 10000000000ULL + + static_cast(EventTime) + UserID % 1000000; + +random_with_seed.seed(powerLaw(5000, 1.1)); +auto get_random_with_seed = [&]{ return random_with_seed(); }; +``` + +Advantages: + +- Conceptual simplicity. + +Disadvantages: + +- A large amount of work is required. +- The solution only applies to one type of data. + +And I preferred a more general solution that can be used for obfuscating any dataset. + +In any case, this solution could be improved. Instead of manually selecting models, we could implement a catalog of models and choose the best among them (best fit plus some form of regularization). Or maybe we could use Markov models for all types of fields, not just for text. Dependencies between data could also be extracted automatically. This would require calculating the [relative entropy](https://en.wikipedia.org/wiki/Kullback%E2%80%93Leibler_divergence) (the relative amount of information) between columns. A simpler alternative is to calculate relative cardinalities for each pair of columns (something like "how many different values of A are there on average for a fixed value B"). For instance, this will make it clear that `URLDomain` fully depends on the `URL`, and not vice versa. + +But I also rejected this idea because there are too many factors to consider, and it would take too long to write. + +### Neural networks + +As I've already mentioned, this task wasn't high on the priority list — no one was even thinking about trying to solve it. But as luck would have it, our colleague Ivan Puzirevsky was teaching at the Higher School of Economics. He asked me if I had any interesting problems that would work as suitable thesis topics for his students. When I offered him this one, he assured me it had potential. So I handed this challenge off to a nice guy "off the street" Sharif (he did have to sign an NDA to access the data, though). + +I shared all my ideas with him but emphasized that there were no restrictions on how the problem could be solved, and a good option would be to try approaches that I know nothing about, like using LSTM to generate a text dump of data. This seemed promising after coming across the article [The Unreasonable Effectiveness of Recurrent Neural Networks](http://karpathy.github.io/2015/05/21/rnn-effectiveness/). + +The first challenge is that we need to generate structured data, not just text. But it wasn't clear whether a recurrent neural network could generate data with the desired structure. There are two ways to solve this. The first solution is to use separate models for generating the structure and the "filler", and only use the neural network for generating values. But this approach was postponed and then never completed. The second solution is to simply generate a TSV dump as text. Experience has shown that some of the rows in the text won't match the structure, but these rows can be thrown out when loading the data. + +The second challenge is that the recurrent neural network generates a sequence of data, and thus dependencies in data must follow in the order of the sequence. But in our data, the order of columns can potentially be in reverse to dependencies between them. We didn't do anything to resolve this problem. + +As summer approached, we had the first working Python script that generated data. The data quality seemed decent at first glance: + +![python_script.jpg](https://clickhouse.com/uploads/python_script_810d491dfb.jpg) + +However, we did run into some difficulties: + +1. The size of the model was about a gigabyte. We tried to create a model for data that was several gigabytes in size (for a start). The fact that the resulting model is so large raised concerns. Would it be possible to extract the real data that it was trained on? Unlikely. But I don't know much about machine learning and neural networks, and I haven't read this developer's Python code, so how can I be sure? There were several articles published at the time about how to compress neural networks without loss of quality, but it wasn't implemented. On the one hand, this doesn't seem to be a serious problem since we can opt out of publishing the model and just publish the generated data. On the other hand, if overfitting occurs, the generated data may contain some part of the source data. + +2. On a machine with a single CPU, the data generation speed is approximately 100 rows per second. Our goal was to generate at least a billion rows. Calculations showed that this wouldn't be completed before the date of the thesis defense. It didn't make sense to use additional hardware because the goal was to make a data generation tool that anyone could use. + +Sharif tried to analyze the quality of data by comparing statistics. Among other things, he calculated the frequency of different characters occurring in the source data and in the generated data. The result was stunning: the most frequent characters were Ð and Ñ. + +Don't worry about Sharif, though. He successfully defended his thesis, and we happily forgot about the whole thing. + +### Mutation of compressed data + +Let's assume that the problem statement has been reduced to a single point: we need to generate data that has the same compression ratio as the source data, and the data must decompress at the same speed. How can we achieve this? We need to edit compressed data bytes directly! This allows us to change the data without changing the size of the compressed data, plus everything will work fast. I wanted to try out this idea right away, despite the fact that the problem it solves is different from what we started with. But that's how it always is. + +So how do we edit a compressed file? Let's say we are only interested in LZ4. LZ4 compressed data is composed of sequences, which in turn are strings of not-compressed bytes (literals), followed by a match copy: + +1. Literals (copy the following N bytes as is). +2. Matches with a minimum repeat length of 4 (repeat N bytes in the file at a distance of M). + +Source data: + +`Hello world Hello.` + +Compressed data (arbitrary example): + +`literals 12 "Hello world " match 5 12.` + +In the compressed file, we leave "match" as-is and change the byte values in "literals". As a result, after decompressing, we get a file in which all repeating sequences at least 4 bytes long are also repeated at the same distance, but they consist of a different set of bytes (basically, the modified file doesn't contain a single byte that was taken from the source file). + +But how do we change the bytes? The answer isn't obvious because, in addition to the column types, the data also has its own internal, implicit structure that we would like to preserve. For example, text data is often stored in UTF-8 encoding, and we want the generated data also to be valid UTF-8. I developed a simple heuristic that involves meeting several criteria: + +- Null bytes and ASCII control characters are kept as-is. +- Some punctuation characters remain as-is. +- ASCII is converted to ASCII, and for everything else, the most significant bit is preserved (or an explicit set of "if" statements is written for different UTF-8 lengths). In one byte class, a new value is picked uniformly at random. +- Fragments like `https://` are preserved; otherwise, it looks a bit silly. + +The only caveat to this approach is that the data model is the source data itself, which means it cannot be published. The model is only fit for generating amounts of data no larger than the source. On the contrary, the previous approaches provide models allowing the generation of data of arbitrary size. + +``` +http://ljc.she/kdoqdqwpgafe/klwlpm&qw=962788775I0E7bs7OXeAyAx +http://ljc.she/kdoqdqwdffhant.am/wcpoyodjit/cbytjgeoocvdtclac +http://ljc.she/kdoqdqwpgafe/klwlpm&qw=962788775I0E7bs7OXe +http://ljc.she/kdoqdqwdffhant.am/wcpoyodjit/cbytjgeoocvdtclac +http://ljc.she/kdoqdqwdbknvj.s/hmqhpsavon.yf#aortxqdvjja +http://ljc.she/kdoqdqw-bknvj.s/hmqhpsavon.yf#aortxqdvjja +http://ljc.she/kdoqdqwpdtu-Unu-Rjanjna-bbcohu_qxht +http://ljc.she/kdoqdqw-bknvj.s/hmqhpsavon.yf#aortxqdvjja +http://ljc.she/kdoqdqwpdtu-Unu-Rjanjna-bbcohu_qxht +http://ljc.she/kdoqdqw-bknvj.s/hmqhpsavon.yf#aortxqdvjja +http://ljc.she/kdoqdqwpdtu-Unu-Rjanjna-bbcohu-702130 +``` + +The results were positive, and the data was interesting, but something wasn't quite right. The URLs kept the same structure, but in some of them, it was too easy to recognize the original terms, such as "avito" (a popular marketplace in Russia), so I created a heuristic that swapped some of the bytes around. + +There were other concerns as well. For example, sensitive information could possibly reside in a FixedString column in binary representation and potentially consist of ASCII control characters and punctuation, which I decided to preserve. However, I didn't take data types into consideration. + +Another problem is that if a column stores data in the "length, value" format (this is how String columns are stored), how do I ensure that the length remains correct after the mutation? When I tried to fix this, I immediately lost interest. + +### Random permutations + +Unfortunately, the problem wasn't solved. We performed a few experiments, and it just got worse. The only thing left was to sit around doing nothing and surf the web randomly since the magic was gone. Luckily, I came across a page that [explained the algorithm](http://fabiensanglard.net/fizzlefade/index.php) for rendering the death of the main character in the game Wolfenstein 3D. + +wolfenstein.gif + +
+ +The animation is really well done — the screen fills up with blood. The article explains that this is actually a pseudorandom permutation. A random permutation of a set of elements is a randomly picked bijective (one-to-one) transformation of the set. In other words, a mapping where each and every derived element corresponds to exactly one original element (and vice versa). In other words, it is a way to randomly iterate through all the elements of a data set. And that is exactly the process shown in the picture: each pixel is filled in random order, without any repetition. If we were to just choose a random pixel at each step, it would take a long time to get to the last one. + +The game uses a very simple algorithm for pseudorandom permutation called linear feedback shift register ([LFSR](https://en.wikipedia.org/wiki/Linear-feedback_shift_register)). Similar to pseudorandom number generators, random permutations, or rather their families, can be cryptographically strong when parametrized by a key. This is exactly what we needed for our data transformation. However, the details were trickier. For example, cryptographically strong encryption of N bytes to N bytes with a pre-determined key and initialization vector seems like it would work for a pseudorandom permutation of a set of N-byte strings. Indeed, this is a one-to-one transformation, and it appears to be random. But if we use the same transformation for all of our data, the result may be susceptible to cryptoanalysis because the same initialization vector and key value are used multiple times. This is similar to the [Electronic Codebook](https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#ECB) mode of operation for a block cipher. + +For example, three multiplications and two xorshift operations are used for the [murmurhash](https://github.com/ClickHouse/ClickHouse/blob/master/dbms/src/Common/HashTable/Hash.h#L18) finalizer. This operation is a pseudorandom permutation. However, I should point out that hash functions don't have to be one-to-one (even hashes of N bits to N bits). + +Or here's another interesting [example from elementary number theory](https://preshing.com/20121224/how-to-generate-a-sequence-of-unique-random-integers/) from Jeff Preshing's website. + +How can we use pseudorandom permutations to solve our problem? We can use them to transform all numeric fields so we can preserve the cardinalities and mutual cardinalities of all combinations of fields. In other words, COUNT(DISTINCT) will return the same value as before the transformation and, furthermore, with any GROUP BY. + +It is worth noting that preserving all cardinalities somewhat contradicts our goal of data anonymization. Let's say someone knows that the source data for site sessions contains a user who visited sites from 10 different countries, and they want to find that user in the transformed data. The transformed data also shows that the user visited sites from 10 different countries, which makes it easy to narrow down the search. However, even if they find out what the user was transformed into, it won't be very useful; all of the other data has also been transformed, so they won't be able to figure out what sites the user visited or anything else. But these rules can be applied in a chain. For example, suppose someone knows that the most frequently occurring website in our data is Google, with Yahoo in second place. In that case, they can use the ranking to determine which transformed site identifiers actually mean Yahoo and Google. There's nothing surprising about this since we are working with an informal problem statement, and we are trying to find a balance between the anonymization of data (hiding information) and preserving data properties (disclosure of information). For information about how to approach the data anonymization issue more reliably, read this [article](https://medium.com/georgian-impact-blog/a-brief-introduction-to-differential-privacy-eacf8722283b). + +In addition to keeping the original cardinality of values, I also wanted to keep the order of magnitude of the values. What I mean is that if the source data contained numbers under 10, then I want the transformed numbers to also be small. How can we achieve this? + +For example, we can divide a set of possible values into size classes and perform permutations within each class separately (maintaining the size classes). The easiest way to do this is to take the nearest power of two or the position of the most significant bit in the number as the size class (these are the same thing). The numbers 0 and 1 will always remain as is. The numbers 2 and 3 will sometimes remain as is (with a probability of 1/2) and will sometimes be swapped (with a probability of 1/2). The set of numbers 1024..2047 will be mapped to one of 1024! (factorial) variants, and so on. For signed numbers, we will keep the sign. + +It's also doubtful whether we need a one-to-one function. We can probably just use a cryptographically strong hash function. The transformation won't be one-to-one, but the cardinality will be close to the same. + +However, we need a cryptographically strong random permutation so that when we define a key and derive a permutation with that key, restoring the original data from the rearranged data without knowing the key would be difficult. + +There is one problem: in addition to knowing nothing about neural networks and machine learning, I am also quite ignorant when it comes to cryptography. That leaves just my courage. I was still reading random web pages and found a link on [Hackers News](https://news.ycombinator.com/item?id=15122540) to a discussion on Fabien Sanglard's page. It had a link to a [blog post](http://antirez.com/news/113) by Redis developer Salvatore Sanfilippo that talked about using a wonderful generic way of getting random permutations, known as a [Feistel network](https://en.wikipedia.org/wiki/Feistel_cipher). + +The Feistel network is iterative, consisting of rounds. Each round is a remarkable transformation that allows you to get a one-to-one function from any function. Let's look at how it works. + +1. The argument's bits are divided into two halves: +``` + arg: xxxxyyyy + arg_l: xxxx + arg_r: yyyy +``` +2. The right half replaces the left. In its place, we put the result of XOR on the initial value of the left half and the result of the function applied to the initial value of the right half, like this: + + ``` + res: yyyyzzzz + res_l = yyyy = arg_r + res_r = zzzz = arg_l ^ F(arg_r) +``` + +There is also a claim that if we use a cryptographically strong pseudorandom function for F and apply a Feistel round at least four times, we'll get a cryptographically strong pseudorandom permutation. + +This is like a miracle: we take a function that produces random garbage based on data, insert it into the Feistel network, and we now have a function that produces random garbage based on data, but yet is invertible! + +The Feistel network is at the heart of several data encryption algorithms. What we're going to do is something like encryption, only it's really bad. There are two reasons for this: + +1. We are encrypting individual values independently and in the same way, similar to the Electronic Codebook mode of operation. +2. We are storing information about the order of magnitude (the nearest power of two) and the sign of the value, which means that some values do not change at all. + +This way, we can obfuscate numeric fields while preserving the properties we need. For example, after using LZ4, the compression ratio should remain approximately the same because the duplicate values in the source data will be repeated in the converted data and at the same distances from each other. + +### Markov models + +Text models are used for data compression, predictive input, speech recognition, and random string generation. A text model is a probability distribution of all possible strings. Let's say we have an imaginary probability distribution of the texts of all the books that humanity could ever write. To generate a string, we just take a random value with this distribution and return the resulting string (a random book that humanity could write). But how do we find out the probability distribution of all possible strings? + +First, this would require too much information. There are 256^10 possible strings that are 10 bytes in length, and it would take quite a lot of memory to explicitly write a table with the probability of each string. Second, we don't have enough statistics to accurately assess the distribution. + +This is why we use a probability distribution obtained from rough statistics as the text model. For example, we could calculate the probability of each letter occurring in the text and then generate strings by selecting each next letter with the same probability. This primitive model works, but the strings are still very unnatural. + +To improve the model slightly, we could also make use of the conditional probability of the letter's occurrence if it is preceded by N-specific letters. N is a pre-set constant. Let's say N = 5, and we are calculating the probability of the letter "e" occurring after the letters "compr". This text model is called an Order-N Markov model. + +``` +P(cata | cat) = 0.8 +P(catb | cat) = 0.05 +P(catc | cat) = 0.1 +... +``` + +Let's look at how Markov models work on the website [of Hay Kranen](https://projects.haykranen.nl/markov/demo/). Unlike LSTM neural networks, the models only have enough memory for a small context of fixed-length N, so they generate funny nonsensical texts. Markov models are also used in primitive methods for generating spam, and the generated texts can be easily distinguished from real ones by counting statistics that don't fit the model. There is one advantage: Markov models work much faster than neural networks, which is exactly what we need. + +Example for Title (our examples are in Turkish because of the data used): + +
+

Hyunday Butter'dan anket shluha — Politika head manşetleri | STALKER BOXER Çiftede book — Yanudistkarışmanlı Mı Kanal | League el Digitalika Haberler Haberleri — Haberlerisi — Hotels with Centry'ler Neden babah.com

+
+ +We can calculate statistics from the source data, create a Markov model, and generate new data. Note that the model needs smoothing to avoid disclosing information about rare combinations in the source data, but this is not a problem. We use a combination of models from 0 to N. If statistics are insufficient for order N, the N−1 model is used instead. + +But we still want to preserve the cardinality of data. In other words, if the source data had 123456 unique URL values, the result should have approximately the same number of unique values. We can use a deterministically initialized random number generator to achieve this. The easiest way is to use a hash function and apply it to the original value. In other words, we get a pseudorandom result that is explicitly determined by the original value. + +Another requirement is that the source data may have many different URLs that start with the same prefix but aren't identical. For example: `https://www.clickhouse.com/images/cats/?id=xxxxxx`. We want the result to also have URLs that all start with the same prefix, but a different one. For example: http://ftp.google.kz/cgi-bin/index.phtml?item=xxxxxx. As a random number generator for generating the next character using a Markov model, we'll take a hash function from a moving window of 8 bytes at the specified position (instead of taking it from the entire string). + +
+
+https://www.clickhouse.com/images/cats/?id=12345 + ^^^^^^^^ + +distribution: [aaaa][b][cc][dddd][e][ff][ggggg][h]... +hash("images/c") % total_count: ^ +
+
+ + It turns out to be exactly what we need. Here's the example of page titles: + +
+
+PhotoFunia - Haber7 - Hava mükemment.net Oynamak içinde şaşıracak haber, Oyunu Oynanılmaz • apród.hu kínálatában - RT Arabic +PhotoFunia - Kinobar.Net - apród: Ingyenes | Posti +PhotoFunia - Peg Perfeo - Castika, Sıradışı Deniz Lokoning Your Code, sire Eminema.tv/ +PhotoFunia - TUT.BY - Your Ayakkanın ve Son Dakika Spor, +PhotoFunia - big film izle, Del Meireles offilim, Samsung DealeXtreme Değerler NEWSru.com.tv, Smotri.com Mobile yapmak Okey +PhotoFunia 5 | Galaxy, gt, după ce anal bilgi yarak Ceza RE050A V-Stranç +PhotoFunia :: Miami olacaksını yerel Haberler Oyun Young video +PhotoFunia Monstelli'nin En İyi kisa.com.tr –Star Thunder Ekranı +PhotoFunia Seks - Politika,Ekonomi,Spor GTA SANAYİ VE +PhotoFunia Taker-Rating Star TV Resmi Söylenen Yatağa każdy dzież wierzchnie +PhotoFunia TourIndex.Marketime oyunu Oyna Geldolları Mynet Spor,Magazin,Haberler yerel Haberleri ve Solvia, korkusuz Ev SahneTv +PhotoFunia todo in the Gratis Perky Parti'nin yapıyı bu fotogram +PhotoFunian Dünyasın takımız halles en kulları - TEZ +
+
+ +## Results + +After trying four methods, I got so tired of this problem that it was time just to choose something, make it into a usable tool, and announce the solution. I chose the solution that uses random permutations and Markov models parametrized by a key. It is implemented as the clickhouse-obfuscator program, which is very easy to use. The input is a table dump in any supported format (such as CSV or JSONEachRow), and the command line parameters specify the table structure (column names and types) and the secret key (any string, which you can forget immediately after use). The output is the same number of rows of obfuscated data. + +The program is installed with `clickhouse-client`, has no dependencies, and works on almost any flavor of Linux. You can apply it to any database dump, not just ClickHouse. For instance, you can generate test data from MySQL or PostgreSQL databases or create development databases that are similar to your production databases. + +```bash +clickhouse-obfuscator \ + --seed "$(head -c16 /dev/urandom | base64)" \ + --input-format TSV --output-format TSV \ + --structure 'CounterID UInt32, URLDomain String, \ + URL String, SearchPhrase String, Title String' \ + < table.tsv > result.tsv +``` + +```bash + clickhouse-obfuscator --help +``` + +Of course, everything isn't so cut and dry because data transformed by this program is almost completely reversible. The question is whether it is possible to perform the reverse transformation without knowing the key. If the transformation used a cryptographic algorithm, this operation would be as difficult as a brute-force search. Although the transformation uses some cryptographic primitives, they are not used in the correct way, and the data is susceptible to certain methods of analysis. To avoid problems, these issues are covered in the documentation for the program (access it using --help). + +In the end, we transformed the data set we needed [for functional and performance testing](https://clickhouse.com/docs/en/getting-started/example-datasets/metrica/), and received approval from our data security team to publish. + +Our developers and members of our community use this data for real performance testing when optimizing algorithms inside ClickHouse. Third-party users can provide us with their obfuscated data so that we can make ClickHouse even faster for them. We also released an independent open benchmark for hardware and cloud providers on top of this data: [https://benchmark.clickhouse.com/](https://benchmark.clickhouse.com/) From 10980554091440304b8a235079947e685f2c3ad5 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 20 Mar 2023 13:21:10 +0000 Subject: [PATCH 059/216] 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 060/216] 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 061/216] 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 3cafb9747a75860c7c6300401be896d68983107a Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 20 Mar 2023 19:42:11 +0100 Subject: [PATCH 062/216] 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 c99e74248287fe890381b766f9ecc3fa65bf6fc2 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 20 Mar 2023 20:45:31 +0100 Subject: [PATCH 063/216] 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 064/216] 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 038bfb40ab7fd66c693317f957dd4be1a43eb280 Mon Sep 17 00:00:00 2001 From: AVMusorin Date: Mon, 27 Feb 2023 15:42:38 +0100 Subject: [PATCH 065/216] 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 db7582e0605e44c90b268bc8ba65e2a3b3a9113d Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 21 Mar 2023 14:26:39 +0000 Subject: [PATCH 066/216] 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 49ffda181b3e57aff8d63f8636241b01159bd652 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 21 Mar 2023 14:52:13 +0000 Subject: [PATCH 067/216] 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 068/216] 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 fbef6ec05a56c7d5753510130c01a9ef118b736f Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 15 Mar 2023 10:43:10 +0000 Subject: [PATCH 069/216] 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 070/216] 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 ef6b891f289c5248a18de60381387335a2f0a678 Mon Sep 17 00:00:00 2001 From: flynn Date: Wed, 22 Mar 2023 02:47:04 +0000 Subject: [PATCH 071/216] 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 023d14a8943c1de58b3499a3357e8d3591f659e1 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 7 Mar 2023 20:39:26 +0000 Subject: [PATCH 072/216] 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 073/216] 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 074/216] 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 075/216] 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 076/216] 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 077/216] 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 078/216] 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 079/216] 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 d7d3722b76123ec906a07ca60f28ee5833f977ee Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 22 Mar 2023 15:41:54 +0000 Subject: [PATCH 080/216] 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 081/216] 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 575c4263a3dea3865b6e87bbe54d3f64ca6faa73 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 22 Mar 2023 17:47:25 +0100 Subject: [PATCH 082/216] 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 083/216] 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 084/216] 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 8139d7d4b33e3a2f5b0b92be3347767ec3f88cae Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 22 Mar 2023 18:24:20 +0000 Subject: [PATCH 085/216] 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 62c8eeb25cf5bd55b60c3cfbaf1ce1a92fb893cd Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Wed, 22 Mar 2023 20:13:30 +0000 Subject: [PATCH 086/216] Ignoring inaccessible tables for attaching to the materilized view --- src/Core/Settings.h | 1 + src/Interpreters/InterpreterCreateQuery.cpp | 75 +++++++++++++------ ...re_inacc_tables_mat_view_atttach.reference | 0 ...6_ignore_inacc_tables_mat_view_atttach.sql | 22 ++++++ 4 files changed, 74 insertions(+), 24 deletions(-) create mode 100644 tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.reference create mode 100644 tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index accfe387137..644234248f1 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -467,6 +467,7 @@ class IColumn; M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited.", 0) \ M(Bool, check_query_single_value_result, true, "Return check query result as single 1/0 value", 0) \ M(Bool, allow_drop_detached, false, "Allow ALTER TABLE ... DROP DETACHED PART[ITION] ... queries", 0) \ + M(Bool, ignore_inaccessible_tables_mat_view_attach, false, "Ignoring inaccessible tables for attaching to the materilized view.", 0) \ \ M(UInt64, postgresql_connection_pool_size, 16, "Connection pool size for PostgreSQL table engine and database engine.", 0) \ M(UInt64, postgresql_connection_pool_wait_timeout, 5000, "Connection pool push/pop timeout on empty pool for PostgreSQL table engine and database engine. By default it will block on empty pool.", 0) \ diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 7a4d65a4d57..c15607bc273 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -97,6 +97,7 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int LOGICAL_ERROR; extern const int UNKNOWN_DATABASE; + extern const int UNKNOWN_TABLE; extern const int PATH_ACCESS_DENIED; extern const int NOT_IMPLEMENTED; extern const int ENGINE_REQUIRED; @@ -1210,35 +1211,61 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) { Block input_block; - if (getContext()->getSettingsRef().allow_experimental_analyzer) + auto check_type_compatible_for_materialize_view = [&]() { - input_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); + if (getContext()->getSettingsRef().allow_experimental_analyzer) + { + input_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); + } + else + { + input_block = InterpreterSelectWithUnionQuery(create.select->clone(), + getContext(), + SelectQueryOptions().analyze()).getSampleBlock(); + } + + Block output_block = to_table->getInMemoryMetadataPtr()->getSampleBlock(); + + ColumnsWithTypeAndName input_columns; + ColumnsWithTypeAndName output_columns; + for (const auto & input_column : input_block) + { + if (const auto * output_column = output_block.findByName(input_column.name)) + { + input_columns.push_back(input_column.cloneEmpty()); + output_columns.push_back(output_column->cloneEmpty()); + } + } + + ActionsDAG::makeConvertingActions( + input_columns, + output_columns, + ActionsDAG::MatchColumnsMode::Position + ); + }; + + if (getContext()->getSettingsRef().ignore_inaccessible_tables_mat_view_attach) + { + try + { + check_type_compatible_for_materialize_view(); + } catch (const Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_TABLE && create.attach) + { + auto * log = &Poco::Logger::get("InterpreterSelectQuery"); + LOG_WARNING(log, "{}", e.message()); + } + else + { + throw; + } + } } else { - input_block = InterpreterSelectWithUnionQuery(create.select->clone(), - getContext(), - SelectQueryOptions().analyze()).getSampleBlock(); + check_type_compatible_for_materialize_view(); } - - Block output_block = to_table->getInMemoryMetadataPtr()->getSampleBlock(); - - ColumnsWithTypeAndName input_columns; - ColumnsWithTypeAndName output_columns; - for (const auto & input_column : input_block) - { - if (const auto * output_column = output_block.findByName(input_column.name)) - { - input_columns.push_back(input_column.cloneEmpty()); - output_columns.push_back(output_column->cloneEmpty()); - } - } - - ActionsDAG::makeConvertingActions( - input_columns, - output_columns, - ActionsDAG::MatchColumnsMode::Position - ); } } diff --git a/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.reference b/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql b/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql new file mode 100644 index 00000000000..2c81eb245ee --- /dev/null +++ b/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql @@ -0,0 +1,22 @@ +DROP DATABASE IF EXISTS test_db; + +SET ignore_inaccessible_tables_mat_view_attach = 1; +SET send_logs_level = 'fatal'; + +CREATE DATABASE test_db; + +CREATE TABLE test_db.table (n Int32, s String) ENGINE MergeTree PARTITION BY n ORDER BY n; + +CREATE TABLE test_db.mview_backend (n Int32, n2 Int64) ENGINE MergeTree PARTITION BY n ORDER BY n; + +CREATE MATERIALIZED VIEW test_db.mview TO test_db.mview_backend AS SELECT n, n * n AS "n2" FROM test_db.table; + +DROP TABLE test_db.table; + +DETACH TABLE test_db.mview; + +ATTACH TABLE test_db.mview; + +SET ignore_inaccessible_tables_mat_view_attach = 0; + +DROP DATABASE test_db; \ No newline at end of file From f047e1b6682b9f01fa7bac038b8092e9d4369b0b Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Wed, 22 Mar 2023 20:18:40 +0000 Subject: [PATCH 087/216] Fixed new line --- .../0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql b/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql index 2c81eb245ee..1b11ec9a9c8 100644 --- a/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql +++ b/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql @@ -19,4 +19,5 @@ ATTACH TABLE test_db.mview; SET ignore_inaccessible_tables_mat_view_attach = 0; -DROP DATABASE test_db; \ No newline at end of file +DROP DATABASE test_db; + From f69c432f5f65ae07742e14f3306c6999738aba1b Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 22 Mar 2023 18:23:11 -0400 Subject: [PATCH 088/216] 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 e7afe0f60fc53c2882a5109e70ffa0dfbbd2b16b Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 23 Mar 2023 04:08:36 +0000 Subject: [PATCH 089/216] 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 b5e17cc2c0b6fba5e955dc4d7b9631b0993147b4 Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 23 Mar 2023 06:24:58 +0000 Subject: [PATCH 090/216] 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 02c666d06d279ceb9f94abc5811de35d9860041c Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 23 Mar 2023 11:14:12 +0000 Subject: [PATCH 091/216] 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 679fe4c9cd60d4b302c8f37206b41fe4a674fa74 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Thu, 23 Mar 2023 14:57:52 +0100 Subject: [PATCH 092/216] Added test. Closes #12264 --- tests/queries/0_stateless/25402_fail_on_invalid_having.reference | 0 tests/queries/0_stateless/25402_fail_on_invalid_having.sql | 1 + 2 files changed, 1 insertion(+) create mode 100644 tests/queries/0_stateless/25402_fail_on_invalid_having.reference create mode 100644 tests/queries/0_stateless/25402_fail_on_invalid_having.sql diff --git a/tests/queries/0_stateless/25402_fail_on_invalid_having.reference b/tests/queries/0_stateless/25402_fail_on_invalid_having.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/25402_fail_on_invalid_having.sql b/tests/queries/0_stateless/25402_fail_on_invalid_having.sql new file mode 100644 index 00000000000..beb72218685 --- /dev/null +++ b/tests/queries/0_stateless/25402_fail_on_invalid_having.sql @@ -0,0 +1 @@ +SELECT a, sum(b) FROM (SELECT 1 AS a, 1 AS b, 0 AS c) GROUP BY a HAVING c -- {{ serverError NOT_AN_AGGREGATE}} \ No newline at end of file From 5e95a37c526875ad1791240895f0f5cb72632e41 Mon Sep 17 00:00:00 2001 From: MeenaRenganathan22 Date: Thu, 23 Mar 2023 07:05:51 -0700 Subject: [PATCH 093/216] 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 094/216] 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 8ff01429e1e75b6d773c154e743b1a61a3d6ed4c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 23 Mar 2023 15:49:59 +0100 Subject: [PATCH 095/216] 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 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 096/216] 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 9a2a091c76e3c13c57b970eda072141b5413f27f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 23 Mar 2023 16:39:16 +0100 Subject: [PATCH 097/216] 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 098/216] 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 e9bb7efef6dda22d4a74ac5e0feecbabaf1d4266 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 23 Mar 2023 18:27:30 +0000 Subject: [PATCH 099/216] 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 100/216] 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 101/216] 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 f46970671d1586312e67d55a49ffc66e5955546f Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 9 Feb 2023 15:42:08 -0800 Subject: [PATCH 102/216] Implement tokenbf_v1 index utilization for hasTokenCaseInsensitive --- .../mergetree-family/mergetree.md | 52 +++++---- .../MergeTree/MergeTreeIndexFullText.cpp | 103 ++++++++---------- .../MergeTree/MergeTreeIndexFullText.h | 2 +- .../MergeTree/MergeTreeIndexInverted.cpp | 3 +- .../00990_hasToken_and_tokenbf.reference | 6 + .../00990_hasToken_and_tokenbf.sql | 25 ++++- 6 files changed, 104 insertions(+), 87 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..004eaf64b0b 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -454,32 +454,36 @@ Conditions in the `WHERE` clause contains calls of the functions that operate wi Indexes of type `set` can be utilized by all functions. The other index types are supported as follows: -| Function (operator) / Index | primary key | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter | -|------------------------------------------------------------------------------------------------------------|-------------|--------|------------|------------|--------------| -| [equals (=, ==)](/docs/en/sql-reference/functions/comparison-functions.md/#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [notEquals(!=, <>)](/docs/en/sql-reference/functions/comparison-functions.md/#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [like](/docs/en/sql-reference/functions/string-search-functions.md/#function-like) | ✔ | ✔ | ✔ | ✔ | ✗ | -| [notLike](/docs/en/sql-reference/functions/string-search-functions.md/#function-notlike) | ✔ | ✔ | ✔ | ✔ | ✗ | -| [startsWith](/docs/en/sql-reference/functions/string-functions.md/#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | -| [endsWith](/docs/en/sql-reference/functions/string-functions.md/#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ | -| [multiSearchAny](/docs/en/sql-reference/functions/string-search-functions.md/#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | -| [in](/docs/en/sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [notIn](/docs/en/sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [less (<)](/docs/en/sql-reference/functions/comparison-functions.md/#function-less) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [greater (>)](/docs/en/sql-reference/functions/comparison-functions.md/#function-greater) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [lessOrEquals (<=)](/docs/en/sql-reference/functions/comparison-functions.md/#function-lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [greaterOrEquals (>=)](/docs/en/sql-reference/functions/comparison-functions.md/#function-greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [empty](/docs/en/sql-reference/functions/array-functions#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [notEmpty](/docs/en/sql-reference/functions/array-functions#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [has](/docs/en/sql-reference/functions/array-functions#function-has) | ✗ | ✗ | ✔ | ✔ | ✔ | -| [hasAny](/docs/en/sql-reference/functions/array-functions#function-hasAny) | ✗ | ✗ | ✗ | ✗ | ✔ | -| [hasAll](/docs/en/sql-reference/functions/array-functions#function-hasAll) | ✗ | ✗ | ✗ | ✗ | ✔ | -| hasToken | ✗ | ✗ | ✗ | ✔ | ✗ | -| hasTokenOrNull | ✗ | ✗ | ✗ | ✔ | ✗ | -| hasTokenCaseInsensitive | ✗ | ✗ | ✗ | ✔ | ✗ | -| hasTokenCaseInsensitiveOrNull | ✗ | ✗ | ✗ | ✔ | ✗ | +| Function (operator) / Index | primary key | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter | inverted | +|------------------------------------------------------------------------------------------------------------|-------------|--------|------------|------------|--------------|----------| +| [equals (=, ==)](/docs/en/sql-reference/functions/comparison-functions.md/#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | +| [notEquals(!=, <>)](/docs/en/sql-reference/functions/comparison-functions.md/#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | +| [like](/docs/en/sql-reference/functions/string-search-functions.md/#function-like) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ | +| [notLike](/docs/en/sql-reference/functions/string-search-functions.md/#function-notlike) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ | +| [startsWith](/docs/en/sql-reference/functions/string-functions.md/#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ | +| [endsWith](/docs/en/sql-reference/functions/string-functions.md/#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ | ✔ | +| [multiSearchAny](/docs/en/sql-reference/functions/string-search-functions.md/#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | ✔ | +| [in](/docs/en/sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | +| [notIn](/docs/en/sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | +| [less (<)](/docs/en/sql-reference/functions/comparison-functions.md/#function-less) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | +| [greater (>)](/docs/en/sql-reference/functions/comparison-functions.md/#function-greater) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | +| [lessOrEquals (<=)](/docs/en/sql-reference/functions/comparison-functions.md/#function-lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | +| [greaterOrEquals (>=)](/docs/en/sql-reference/functions/comparison-functions.md/#function-greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | +| [empty](/docs/en/sql-reference/functions/array-functions#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | +| [notEmpty](/docs/en/sql-reference/functions/array-functions#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | +| [has](/docs/en/sql-reference/functions/array-functions#function-has) | ✗ | ✗ | ✔ | ✔ | ✔ | ✔ | +| [hasAny](/docs/en/sql-reference/functions/array-functions#function-hasAny) | ✗ | ✗ | ✗ | ✗ | ✔ | ✗ | +| [hasAll](/docs/en/sql-reference/functions/array-functions#function-hasAll) | ✗ | ✗ | ✗ | ✗ | ✔ | ✗ | +| hasToken | ✗ | ✗ | ✗ | ✔ | ✗ | ✔ | +| hasTokenOrNull | ✗ | ✗ | ✗ | ✔ | ✗ | ✔ | +| hasTokenCaseInsensitive (*) | ✗ | ✗ | ✗ | ✔ | ✗ | ✗ | +| hasTokenCaseInsensitiveOrNull (*) | ✗ | ✗ | ✗ | ✔ | ✗ | ✗ | Functions with a constant argument that is less than ngram size can’t be used by `ngrambf_v1` for query optimization. +(*) For `hasTokenCaseInsensitve` and `hasTokenCaseInsensitive` to be effective, the data skipping index of type `tokenbf_v1` must be created on lowercased data, for example: +``` +CREATE TABLE tab (id UInt64, s String, INDEX tok_bf_idx (s, lower(s)) TYPE tokenbf_v1(512, 3, 0) GRANULARITY 1) ... . ) ENGINE = MergeTree() +``` :::note Bloom filters can have false positive matches, so the `ngrambf_v1`, `tokenbf_v1`, and `bloom_filter` indexes can not be used for optimizing queries where the result of a function is expected to be false. diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index fa1bd36f863..06fddd51cb8 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -322,14 +322,10 @@ bool MergeTreeConditionFullText::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx return rpn_stack[0].can_be_true; } -bool MergeTreeConditionFullText::getKey(const std::string & key_column_name, size_t & key_column_num) +std::optional MergeTreeConditionFullText::getKeyIndex(const std::string & key_column_name) { - auto it = std::find(index_columns.begin(), index_columns.end(), key_column_name); - if (it == index_columns.end()) - return false; - - key_column_num = static_cast(it - index_columns.begin()); - return true; + const auto it = std::ranges::find(index_columns, key_column_name); + return it == index_columns.end() ? std::nullopt : std::make_optional(std::ranges::distance(index_columns.cbegin(), it)); } bool MergeTreeConditionFullText::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out) @@ -389,7 +385,7 @@ bool MergeTreeConditionFullText::extractAtomFromTree(const RPNBuilderTreeNode & function_name == "mapContains" || function_name == "like" || function_name == "notLike" || - function_name == "hasToken" || + function_name.starts_with("hasToken") || function_name == "startsWith" || function_name == "endsWith" || function_name == "multiSearchAny") @@ -426,10 +422,9 @@ bool MergeTreeConditionFullText::traverseTreeEquals( Field const_value = value_field; - auto column_name = key_node.getColumnName(); - size_t key_column_num = 0; - bool key_exists = getKey(column_name, key_column_num); - bool map_key_exists = getKey(fmt::format("mapKeys({})", column_name), key_column_num); + const auto column_name = key_node.getColumnName(); + auto key_index = getKeyIndex(column_name); + const auto map_key_index = getKeyIndex(fmt::format("mapKeys({})", column_name)); if (key_node.isFunction()) { @@ -450,24 +445,14 @@ bool MergeTreeConditionFullText::traverseTreeEquals( auto first_argument = key_function_node.getArgumentAt(0); const auto map_column_name = first_argument.getColumnName(); - - size_t map_keys_key_column_num = 0; - auto map_keys_index_column_name = fmt::format("mapKeys({})", map_column_name); - bool map_keys_exists = getKey(map_keys_index_column_name, map_keys_key_column_num); - - size_t map_values_key_column_num = 0; - auto map_values_index_column_name = fmt::format("mapValues({})", map_column_name); - bool map_values_exists = getKey(map_values_index_column_name, map_values_key_column_num); - - if (map_keys_exists) + if (const auto map_keys_index = getKeyIndex(fmt::format("mapKeys({})", map_column_name))) { auto second_argument = key_function_node.getArgumentAt(1); DataTypePtr const_type; if (second_argument.tryGetConstant(const_value, const_type)) { - key_column_num = map_keys_key_column_num; - key_exists = true; + key_index = map_keys_index; auto const_data_type = WhichDataType(const_type); if (!const_data_type.isStringOrFixedString() && !const_data_type.isArray()) @@ -478,10 +463,9 @@ bool MergeTreeConditionFullText::traverseTreeEquals( return false; } } - else if (map_values_exists) + else if (const auto map_values_exists = getKeyIndex(fmt::format("mapValues({})", map_column_name))) { - key_column_num = map_values_key_column_num; - key_exists = true; + key_index = map_values_exists; } else { @@ -490,12 +474,29 @@ bool MergeTreeConditionFullText::traverseTreeEquals( } } - if (!key_exists && !map_key_exists) + const auto lowercase_key_index = getKeyIndex(fmt::format("lower({})", column_name)); + const auto is_has_token_case_insensitive = function_name.starts_with("hasTokenCaseInsensitive"); + if (const auto is_case_insensitive_scenario = is_has_token_case_insensitive && lowercase_key_index; + function_name.starts_with("hasToken") && ((!is_has_token_case_insensitive && key_index) || is_case_insensitive_scenario)) + { + out.key_column = is_case_insensitive_scenario ? *lowercase_key_index : *key_index; + out.function = RPNElement::FUNCTION_EQUALS; + out.bloom_filter = std::make_unique(params); + + auto value = const_value.get(); + if (is_case_insensitive_scenario) + std::ranges::transform(value, value.begin(), [](const auto & c) { return static_cast(std::tolower(c)); }); + + token_extractor->stringToBloomFilter(value.data(), value.size(), *out.bloom_filter); + return true; + } + + if (!key_index && !map_key_index) return false; - if (map_key_exists && (function_name == "has" || function_name == "mapContains")) + if (map_key_index && (function_name == "has" || function_name == "mapContains")) { - out.key_column = key_column_num; + out.key_column = *key_index; out.function = RPNElement::FUNCTION_HAS; out.bloom_filter = std::make_unique(params); auto & value = const_value.get(); @@ -504,7 +505,7 @@ bool MergeTreeConditionFullText::traverseTreeEquals( } else if (function_name == "has") { - out.key_column = key_column_num; + out.key_column = *key_index; out.function = RPNElement::FUNCTION_HAS; out.bloom_filter = std::make_unique(params); auto & value = const_value.get(); @@ -514,7 +515,7 @@ bool MergeTreeConditionFullText::traverseTreeEquals( if (function_name == "notEquals") { - out.key_column = key_column_num; + out.key_column = *key_index; out.function = RPNElement::FUNCTION_NOT_EQUALS; out.bloom_filter = std::make_unique(params); const auto & value = const_value.get(); @@ -523,7 +524,7 @@ bool MergeTreeConditionFullText::traverseTreeEquals( } else if (function_name == "equals") { - out.key_column = key_column_num; + out.key_column = *key_index; out.function = RPNElement::FUNCTION_EQUALS; out.bloom_filter = std::make_unique(params); const auto & value = const_value.get(); @@ -532,7 +533,7 @@ bool MergeTreeConditionFullText::traverseTreeEquals( } else if (function_name == "like") { - out.key_column = key_column_num; + out.key_column = *key_index; out.function = RPNElement::FUNCTION_EQUALS; out.bloom_filter = std::make_unique(params); const auto & value = const_value.get(); @@ -541,25 +542,16 @@ bool MergeTreeConditionFullText::traverseTreeEquals( } else if (function_name == "notLike") { - out.key_column = key_column_num; + out.key_column = *key_index; out.function = RPNElement::FUNCTION_NOT_EQUALS; out.bloom_filter = std::make_unique(params); const auto & value = const_value.get(); token_extractor->stringLikeToBloomFilter(value.data(), value.size(), *out.bloom_filter); return true; } - else if (function_name == "hasToken") - { - out.key_column = key_column_num; - out.function = RPNElement::FUNCTION_EQUALS; - out.bloom_filter = std::make_unique(params); - const auto & value = const_value.get(); - token_extractor->stringToBloomFilter(value.data(), value.size(), *out.bloom_filter); - return true; - } else if (function_name == "startsWith") { - out.key_column = key_column_num; + out.key_column = *key_index; out.function = RPNElement::FUNCTION_EQUALS; out.bloom_filter = std::make_unique(params); const auto & value = const_value.get(); @@ -568,7 +560,7 @@ bool MergeTreeConditionFullText::traverseTreeEquals( } else if (function_name == "endsWith") { - out.key_column = key_column_num; + out.key_column = *key_index; out.function = RPNElement::FUNCTION_EQUALS; out.bloom_filter = std::make_unique(params); const auto & value = const_value.get(); @@ -577,7 +569,7 @@ bool MergeTreeConditionFullText::traverseTreeEquals( } else if (function_name == "multiSearchAny") { - out.key_column = key_column_num; + out.key_column = *key_index; out.function = RPNElement::FUNCTION_MULTI_SEARCH; /// 2d vector is not needed here but is used because already exists for FUNCTION_IN @@ -616,22 +608,17 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter( for (size_t i = 0; i < left_argument_function_node_arguments_size; ++i) { - size_t key = 0; - if (getKey(left_argument_function_node.getArgumentAt(i).getColumnName(), key)) + if (const auto key = getKeyIndex(left_argument_function_node.getArgumentAt(i).getColumnName())) { - key_tuple_mapping.emplace_back(i, key); - data_types.push_back(index_data_types[key]); + key_tuple_mapping.emplace_back(i, *key); + data_types.push_back(index_data_types[*key]); } } } - else + else if (const auto key = getKeyIndex(left_argument.getColumnName())) { - size_t key = 0; - if (getKey(left_argument.getColumnName(), key)) - { - key_tuple_mapping.emplace_back(0, key); - data_types.push_back(index_data_types[key]); - } + key_tuple_mapping.emplace_back(0, *key); + data_types.push_back(index_data_types[*key]); } if (key_tuple_mapping.empty()) diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.h b/src/Storages/MergeTree/MergeTreeIndexFullText.h index ad487816aef..c9b24f7086b 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.h +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.h @@ -131,7 +131,7 @@ private: const Field & value_field, RPNElement & out); - bool getKey(const std::string & key_column_name, size_t & key_column_num); + std::optional getKeyIndex(const std::string & key_column_name); bool tryPrepareSetBloomFilter(const RPNBuilderTreeNode & left_argument, const RPNBuilderTreeNode & right_argument, RPNElement & out); static bool createFunctionEqualsCondition( diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp index 8e8409f3868..baa11368c8b 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp @@ -426,6 +426,7 @@ bool MergeTreeConditionInverted::traverseAtomAST(const RPNBuilderTreeNode & node function_name == "like" || function_name == "notLike" || function_name == "hasToken" || + function_name == "hasTokenOrNull" || function_name == "startsWith" || function_name == "endsWith" || function_name == "multiSearchAny") @@ -568,7 +569,7 @@ bool MergeTreeConditionInverted::traverseASTEquals( token_extractor->stringLikeToGinFilter(value.data(), value.size(), *out.gin_filter); return true; } - else if (function_name == "hasToken") + else if (function_name == "hasToken" || function_name == "hasTokenOrNull") { out.key_column = key_column_num; out.function = RPNElement::FUNCTION_EQUALS; diff --git a/tests/queries/0_stateless/00990_hasToken_and_tokenbf.reference b/tests/queries/0_stateless/00990_hasToken_and_tokenbf.reference index 4b3beccf5f1..c462030edbf 100644 --- a/tests/queries/0_stateless/00990_hasToken_and_tokenbf.reference +++ b/tests/queries/0_stateless/00990_hasToken_and_tokenbf.reference @@ -2,6 +2,12 @@ 0 2007 2007 +0 +2007 +2007 +2007 +2007 +2007 2007 0 2007 diff --git a/tests/queries/0_stateless/00990_hasToken_and_tokenbf.sql b/tests/queries/0_stateless/00990_hasToken_and_tokenbf.sql index 41676905771..1361ce0f3c3 100644 --- a/tests/queries/0_stateless/00990_hasToken_and_tokenbf.sql +++ b/tests/queries/0_stateless/00990_hasToken_and_tokenbf.sql @@ -12,21 +12,40 @@ insert into bloom_filter select number+2000, 'abc,def,zzz' from numbers(8); insert into bloom_filter select number+3000, 'yyy,uuu' from numbers(1024); insert into bloom_filter select number+3000, 'abcdefzzz' from numbers(1024); +drop table if exists bloom_filter2; +create table bloom_filter2 +( + id UInt64, + s String, + index tok_bf3 (s, lower(s)) type tokenbf_v1(512, 3, 0) GRANULARITY 1 +) engine = MergeTree() order by id settings index_granularity = 8; + +insert into bloom_filter2 select number, 'yyy,uuu' from numbers(1024); +insert into bloom_filter2 select number+2000, 'ABC,def,zzz' from numbers(8); +insert into bloom_filter2 select number+3000, 'yyy,uuu' from numbers(1024); +insert into bloom_filter2 select number+3000, 'abcdefzzz' from numbers(1024); + SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'abc,def,zzz'); -- { serverError BAD_ARGUMENTS } SELECT max(id) FROM bloom_filter WHERE hasTokenCaseInsensitive(s, 'abc,def,zzz'); -- { serverError BAD_ARGUMENTS } SELECT max(id) FROM bloom_filter WHERE hasTokenOrNull(s, 'abc,def,zzz'); SELECT max(id) FROM bloom_filter WHERE hasTokenCaseInsensitiveOrNull(s, 'abc,def,zzz'); -select max(id) from bloom_filter where hasTokenCaseInsensitive(s, 'ABC'); -select max(id) from bloom_filter where hasTokenCaseInsensitive(s, 'zZz'); - set max_rows_to_read = 16; SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'abc'); +SELECT max(id) FROM bloom_filter WHERE hasTokenOrNull(s, 'abc'); SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'ABC'); +select max(id) from bloom_filter where hasTokenCaseInsensitive(s, 'ABC'); +select max(id) from bloom_filter where hasTokenCaseInsensitiveOrNull(s, 'ABC'); SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'def'); SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'zzz'); +select max(id) from bloom_filter where hasTokenCaseInsensitive(s, 'zZz'); + +select max(id) from bloom_filter2 where hasToken(s, 'ABC'); +select max(id) from bloom_filter2 where hasToken(s, 'abc'); +select max(id) from bloom_filter2 where hasTokenCaseInsensitive(s, 'abc'); +select max(id) from bloom_filter2 where hasTokenCaseInsensitive(s, 'ABC'); -- invert result -- this does not work as expected, reading more rows that it should From 2491128b087baaf3bb5ef7c6605555ce1d2c2130 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 23 Mar 2023 17:40:04 -0300 Subject: [PATCH 103/216] Add unit test to assert arrow lib does not abort on fatal logs --- src/CMakeLists.txt | 1 + .../gtest_assert_arrow_log_does_not_abort.cpp | 15 +++++++++++++++ 2 files changed, 16 insertions(+) create mode 100644 src/Processors/tests/gtest_assert_arrow_log_does_not_abort.cpp diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index edf638c5350..e5cdd14d826 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -608,6 +608,7 @@ if (ENABLE_TESTS) dbms clickhouse_common_config clickhouse_common_zookeeper + ch_contrib::parquet string_utils) if (TARGET ch_contrib::simdjson) diff --git a/src/Processors/tests/gtest_assert_arrow_log_does_not_abort.cpp b/src/Processors/tests/gtest_assert_arrow_log_does_not_abort.cpp new file mode 100644 index 00000000000..93bcdb29f73 --- /dev/null +++ b/src/Processors/tests/gtest_assert_arrow_log_does_not_abort.cpp @@ -0,0 +1,15 @@ +#include +#include +#include +#include + +using namespace DB; + +TEST(ChunkedArray, ChunkedArrayWithZeroChunksShouldNotAbort) +{ + std::vector> empty_chunks_vector; + + EXPECT_ANY_THROW(::arrow::ChunkedArray{empty_chunks_vector}); + + ::arrow::util::ArrowLog(__FILE__, __LINE__, ::arrow::util::ArrowLogLevel::ARROW_FATAL); +} From d13aa3c02962c90790a3705b6b4898f4370e87f5 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 23 Mar 2023 17:42:55 -0300 Subject: [PATCH 104/216] missing change --- .../tests/gtest_assert_arrow_log_does_not_abort.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Processors/tests/gtest_assert_arrow_log_does_not_abort.cpp b/src/Processors/tests/gtest_assert_arrow_log_does_not_abort.cpp index 93bcdb29f73..fb13f4664d9 100644 --- a/src/Processors/tests/gtest_assert_arrow_log_does_not_abort.cpp +++ b/src/Processors/tests/gtest_assert_arrow_log_does_not_abort.cpp @@ -10,6 +10,9 @@ TEST(ChunkedArray, ChunkedArrayWithZeroChunksShouldNotAbort) std::vector> empty_chunks_vector; EXPECT_ANY_THROW(::arrow::ChunkedArray{empty_chunks_vector}); - - ::arrow::util::ArrowLog(__FILE__, __LINE__, ::arrow::util::ArrowLogLevel::ARROW_FATAL); +} + +TEST(ArrowLog, FatalLogShouldThrow) +{ + EXPECT_ANY_THROW(::arrow::util::ArrowLog(__FILE__, __LINE__, ::arrow::util::ArrowLogLevel::ARROW_FATAL)); } From a4771812cb64c8d8a515ba5ee6ae4121439a41f1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 23 Mar 2023 20:46:42 +0000 Subject: [PATCH 105/216] 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 81bef1cb4b78d5f30f41a3a13b784f900e69bd81 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 23 Mar 2023 21:18:30 +0000 Subject: [PATCH 106/216] 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 fd1d8d5c55fcd1afd48b4195aa68b31050affaaa Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 23 Mar 2023 19:00:20 -0400 Subject: [PATCH 107/216] 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 108/216] 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 109/216] 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 c4870ad7bc04732280afde192cf3e0337d240566 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 23 Mar 2023 16:56:20 +0100 Subject: [PATCH 110/216] 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 68118388e6a3f73a1091a29a353d5525953f7cb1 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Fri, 24 Mar 2023 08:23:22 +0000 Subject: [PATCH 111/216] Rename opt key, added test and refactored code. --- src/Interpreters/InterpreterCreateQuery.cpp | 126 +++++++++--------- src/Interpreters/InterpreterCreateQuery.h | 3 + ...6_ignore_inacc_tables_mat_view_atttach.sql | 7 +- ...re_inacc_tables_mat_view_atttach.reference | 0 ...t_ignore_inacc_tables_mat_view_atttach.sql | 20 +++ ..._view_state_after_droping_tables.reference | 2 + ...re_mat_view_state_after_droping_tables.sql | 27 ++++ 7 files changed, 118 insertions(+), 67 deletions(-) create mode 100644 tests/queries/0_stateless/02697_do_not_ignore_inacc_tables_mat_view_atttach.reference create mode 100644 tests/queries/0_stateless/02697_do_not_ignore_inacc_tables_mat_view_atttach.sql create mode 100644 tests/queries/0_stateless/02698_restore_mat_view_state_after_droping_tables.reference create mode 100644 tests/queries/0_stateless/02698_restore_mat_view_state_after_droping_tables.sql diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index c15607bc273..67dd053ef19 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1077,6 +1077,68 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data } } +Block InterpreterCreateQuery::getSampleBlockFromCreteQuery(const ASTCreateQuery & create) +{ + Block result_block; + if (getContext()->getSettingsRef().allow_experimental_analyzer) + { + result_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); + } + else + { + result_block = InterpreterSelectWithUnionQuery(create.select->clone(), + getContext(), + SelectQueryOptions().analyze()).getSampleBlock(); + } + return result_block; +} + +void InterpreterCreateQuery::checkTypecompatibleForMaterializeView(const ASTCreateQuery & create) +{ + if (StoragePtr to_table = DatabaseCatalog::instance().tryGetTable( + {create.to_table_id.database_name, create.to_table_id.table_name, create.to_table_id.uuid}, + getContext() + )) + { + Block input_block; + try + { + input_block = getSampleBlockFromCreteQuery(create); + } + catch (const Exception & e) + { + if (getContext()->getSettingsRef().skip_materialized_view_checking_if_source_table_not_exist && + e.code() == ErrorCodes::UNKNOWN_TABLE && create.attach + ) + { + LOG_WARNING(&Poco::Logger::get("InterpreterSelectQuery"), "{}", e.message()); + } + else + { + throw; + } + } + + Block output_block = to_table->getInMemoryMetadataPtr()->getSampleBlock(); + + ColumnsWithTypeAndName input_columns; + ColumnsWithTypeAndName output_columns; + for (const auto & input_column : input_block) + { + if (const auto * output_column = output_block.findByName(input_column.name)) + { + input_columns.push_back(input_column.cloneEmpty()); + output_columns.push_back(output_column->cloneEmpty()); + } + } + + ActionsDAG::makeConvertingActions( + input_columns, + output_columns, + ActionsDAG::MatchColumnsMode::Position + ); + } +} BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) { @@ -1204,69 +1266,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) /// Check type compatible for materialized dest table and select columns if (create.select && create.is_materialized_view && create.to_table_id) { - if (StoragePtr to_table = DatabaseCatalog::instance().tryGetTable( - {create.to_table_id.database_name, create.to_table_id.table_name, create.to_table_id.uuid}, - getContext() - )) - { - Block input_block; - - auto check_type_compatible_for_materialize_view = [&]() - { - if (getContext()->getSettingsRef().allow_experimental_analyzer) - { - input_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); - } - else - { - input_block = InterpreterSelectWithUnionQuery(create.select->clone(), - getContext(), - SelectQueryOptions().analyze()).getSampleBlock(); - } - - Block output_block = to_table->getInMemoryMetadataPtr()->getSampleBlock(); - - ColumnsWithTypeAndName input_columns; - ColumnsWithTypeAndName output_columns; - for (const auto & input_column : input_block) - { - if (const auto * output_column = output_block.findByName(input_column.name)) - { - input_columns.push_back(input_column.cloneEmpty()); - output_columns.push_back(output_column->cloneEmpty()); - } - } - - ActionsDAG::makeConvertingActions( - input_columns, - output_columns, - ActionsDAG::MatchColumnsMode::Position - ); - }; - - if (getContext()->getSettingsRef().ignore_inaccessible_tables_mat_view_attach) - { - try - { - check_type_compatible_for_materialize_view(); - } catch (const Exception & e) - { - if (e.code() == ErrorCodes::UNKNOWN_TABLE && create.attach) - { - auto * log = &Poco::Logger::get("InterpreterSelectQuery"); - LOG_WARNING(log, "{}", e.message()); - } - else - { - throw; - } - } - } - else - { - check_type_compatible_for_materialize_view(); - } - } + checkTypecompatibleForMaterializeView(create); } DatabasePtr database; diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index 4d11387f44c..0d3ce92a4ff 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -95,6 +95,9 @@ private: BlockIO doCreateOrReplaceTable(ASTCreateQuery & create, const InterpreterCreateQuery::TableProperties & properties); /// Inserts data in created table if it's CREATE ... SELECT BlockIO fillTableIfNeeded(const ASTCreateQuery & create); + /// Check type compatible for materialized dest table and select columns + void checkTypecompatibleForMaterializeView(const ASTCreateQuery & create); + Block getSampleBlockFromCreteQuery(const ASTCreateQuery & create); void assertOrSetUUID(ASTCreateQuery & create, const DatabasePtr & database) const; diff --git a/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql b/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql index 1b11ec9a9c8..854a074c234 100644 --- a/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql +++ b/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql @@ -1,6 +1,6 @@ DROP DATABASE IF EXISTS test_db; -SET ignore_inaccessible_tables_mat_view_attach = 1; +SET skip_materialized_view_checking_if_source_table_not_exist = 1; SET send_logs_level = 'fatal'; CREATE DATABASE test_db; @@ -17,7 +17,6 @@ DETACH TABLE test_db.mview; ATTACH TABLE test_db.mview; -SET ignore_inaccessible_tables_mat_view_attach = 0; - -DROP DATABASE test_db; +SET skip_materialized_view_checking_if_source_table_not_exist = 0; +DROP DATABASE test_db; \ No newline at end of file diff --git a/tests/queries/0_stateless/02697_do_not_ignore_inacc_tables_mat_view_atttach.reference b/tests/queries/0_stateless/02697_do_not_ignore_inacc_tables_mat_view_atttach.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02697_do_not_ignore_inacc_tables_mat_view_atttach.sql b/tests/queries/0_stateless/02697_do_not_ignore_inacc_tables_mat_view_atttach.sql new file mode 100644 index 00000000000..146952b263b --- /dev/null +++ b/tests/queries/0_stateless/02697_do_not_ignore_inacc_tables_mat_view_atttach.sql @@ -0,0 +1,20 @@ +DROP DATABASE IF EXISTS test_db; + +SET skip_materialized_view_checking_if_source_table_not_exist = 0; +SET send_logs_level = 'fatal'; + +CREATE DATABASE test_db; + +CREATE TABLE test_db.table (n Int32, s String) ENGINE MergeTree PARTITION BY n ORDER BY n; + +CREATE TABLE test_db.mview_backend (n Int32, n2 Int64) ENGINE MergeTree PARTITION BY n ORDER BY n; + +CREATE MATERIALIZED VIEW test_db.mview TO test_db.mview_backend AS SELECT n, n * n AS "n2" FROM test_db.table; + +DROP TABLE test_db.table; + +DETACH TABLE test_db.mview; + +ATTACH TABLE test_db.mview; --{serverError 60} + +DROP DATABASE test_db; \ No newline at end of file diff --git a/tests/queries/0_stateless/02698_restore_mat_view_state_after_droping_tables.reference b/tests/queries/0_stateless/02698_restore_mat_view_state_after_droping_tables.reference new file mode 100644 index 00000000000..bb3ee860aec --- /dev/null +++ b/tests/queries/0_stateless/02698_restore_mat_view_state_after_droping_tables.reference @@ -0,0 +1,2 @@ +3 some_val +3 9 diff --git a/tests/queries/0_stateless/02698_restore_mat_view_state_after_droping_tables.sql b/tests/queries/0_stateless/02698_restore_mat_view_state_after_droping_tables.sql new file mode 100644 index 00000000000..43c4efb8843 --- /dev/null +++ b/tests/queries/0_stateless/02698_restore_mat_view_state_after_droping_tables.sql @@ -0,0 +1,27 @@ +DROP DATABASE IF EXISTS test_db; + +SET skip_materialized_view_checking_if_source_table_not_exist = 1; +SET send_logs_level = 'fatal'; + +CREATE DATABASE test_db; + +CREATE TABLE test_db.table (n Int32, s String) ENGINE MergeTree PARTITION BY n ORDER BY n; + +CREATE TABLE test_db.mview_backend (n Int32, n2 Int64) ENGINE MergeTree PARTITION BY n ORDER BY n; + +CREATE MATERIALIZED VIEW test_db.mview TO test_db.mview_backend AS SELECT n, n * n AS "n2" FROM test_db.table; + +DROP TABLE test_db.table; + +DETACH TABLE test_db.mview; + +ATTACH TABLE test_db.mview; + +CREATE TABLE test_db.table (n Int32, s String) ENGINE MergeTree PARTITION BY n ORDER BY n; + +INSERT INTO test_db.table VALUES (3,'some_val'); + +SELECT n,s FROM test_db.table ORDER BY n; +SELECT n,n2 FROM test_db.mview ORDER by n; + +DROP DATABASE test_db; \ No newline at end of file From abd9a5bef60c0767fd3c1801cb5ee064fa7c9160 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Fri, 24 Mar 2023 08:25:48 +0000 Subject: [PATCH 112/216] Fixed new lines in test --- .../0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql | 3 ++- .../02697_do_not_ignore_inacc_tables_mat_view_atttach.sql | 3 ++- .../02698_restore_mat_view_state_after_droping_tables.sql | 3 ++- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql b/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql index 854a074c234..cfb363053e5 100644 --- a/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql +++ b/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql @@ -19,4 +19,5 @@ ATTACH TABLE test_db.mview; SET skip_materialized_view_checking_if_source_table_not_exist = 0; -DROP DATABASE test_db; \ No newline at end of file +DROP DATABASE test_db; + diff --git a/tests/queries/0_stateless/02697_do_not_ignore_inacc_tables_mat_view_atttach.sql b/tests/queries/0_stateless/02697_do_not_ignore_inacc_tables_mat_view_atttach.sql index 146952b263b..492c2ed1fa2 100644 --- a/tests/queries/0_stateless/02697_do_not_ignore_inacc_tables_mat_view_atttach.sql +++ b/tests/queries/0_stateless/02697_do_not_ignore_inacc_tables_mat_view_atttach.sql @@ -17,4 +17,5 @@ DETACH TABLE test_db.mview; ATTACH TABLE test_db.mview; --{serverError 60} -DROP DATABASE test_db; \ No newline at end of file +DROP DATABASE test_db; + diff --git a/tests/queries/0_stateless/02698_restore_mat_view_state_after_droping_tables.sql b/tests/queries/0_stateless/02698_restore_mat_view_state_after_droping_tables.sql index 43c4efb8843..b733f42270a 100644 --- a/tests/queries/0_stateless/02698_restore_mat_view_state_after_droping_tables.sql +++ b/tests/queries/0_stateless/02698_restore_mat_view_state_after_droping_tables.sql @@ -24,4 +24,5 @@ INSERT INTO test_db.table VALUES (3,'some_val'); SELECT n,s FROM test_db.table ORDER BY n; SELECT n,n2 FROM test_db.mview ORDER by n; -DROP DATABASE test_db; \ No newline at end of file +DROP DATABASE test_db; + From 8917e1ad5714fe9fee4bfce13c7ea4bad2d7b880 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 24 Mar 2023 09:40:12 +0000 Subject: [PATCH 113/216] 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 114/216] 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 3628d114059a0f189f12b1bb9b6202b89b833220 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 24 Mar 2023 10:56:02 +0000 Subject: [PATCH 115/216] 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 cc92eacb3e4bdfb1292c465d32d533f9e6630330 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Fri, 24 Mar 2023 11:51:09 +0000 Subject: [PATCH 116/216] Merged several tests into one and rm unnecessary func --- src/Core/Settings.h | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 28 ++++++++----------- src/Interpreters/InterpreterCreateQuery.h | 1 - ...re_inacc_tables_mat_view_atttach.reference | 2 ++ ...6_ignore_inacc_tables_mat_view_atttach.sql | 21 ++++++++++---- ...re_inacc_tables_mat_view_atttach.reference | 0 ...t_ignore_inacc_tables_mat_view_atttach.sql | 21 -------------- ..._view_state_after_droping_tables.reference | 2 -- ...re_mat_view_state_after_droping_tables.sql | 28 ------------------- 9 files changed, 30 insertions(+), 75 deletions(-) delete mode 100644 tests/queries/0_stateless/02697_do_not_ignore_inacc_tables_mat_view_atttach.reference delete mode 100644 tests/queries/0_stateless/02697_do_not_ignore_inacc_tables_mat_view_atttach.sql delete mode 100644 tests/queries/0_stateless/02698_restore_mat_view_state_after_droping_tables.reference delete mode 100644 tests/queries/0_stateless/02698_restore_mat_view_state_after_droping_tables.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 644234248f1..6670920f613 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -467,7 +467,7 @@ class IColumn; M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited.", 0) \ M(Bool, check_query_single_value_result, true, "Return check query result as single 1/0 value", 0) \ M(Bool, allow_drop_detached, false, "Allow ALTER TABLE ... DROP DETACHED PART[ITION] ... queries", 0) \ - M(Bool, ignore_inaccessible_tables_mat_view_attach, false, "Ignoring inaccessible tables for attaching to the materilized view.", 0) \ + M(Bool, skip_materialized_view_checking_if_source_table_not_exist, false, "Allow attaching to a materialized view even if dependent tables are inaccessible.", 0) \ \ M(UInt64, postgresql_connection_pool_size, 16, "Connection pool size for PostgreSQL table engine and database engine.", 0) \ M(UInt64, postgresql_connection_pool_wait_timeout, 5000, "Connection pool push/pop timeout on empty pool for PostgreSQL table engine and database engine. By default it will block on empty pool.", 0) \ diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 67dd053ef19..4e07a6ab017 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1077,22 +1077,6 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data } } -Block InterpreterCreateQuery::getSampleBlockFromCreteQuery(const ASTCreateQuery & create) -{ - Block result_block; - if (getContext()->getSettingsRef().allow_experimental_analyzer) - { - result_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); - } - else - { - result_block = InterpreterSelectWithUnionQuery(create.select->clone(), - getContext(), - SelectQueryOptions().analyze()).getSampleBlock(); - } - return result_block; -} - void InterpreterCreateQuery::checkTypecompatibleForMaterializeView(const ASTCreateQuery & create) { if (StoragePtr to_table = DatabaseCatalog::instance().tryGetTable( @@ -1103,7 +1087,16 @@ void InterpreterCreateQuery::checkTypecompatibleForMaterializeView(const ASTCrea Block input_block; try { - input_block = getSampleBlockFromCreteQuery(create); + if (getContext()->getSettingsRef().allow_experimental_analyzer) + { + input_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); + } + else + { + input_block = InterpreterSelectWithUnionQuery(create.select->clone(), + getContext(), + SelectQueryOptions().analyze()).getSampleBlock(); + } } catch (const Exception & e) { @@ -1112,6 +1105,7 @@ void InterpreterCreateQuery::checkTypecompatibleForMaterializeView(const ASTCrea ) { LOG_WARNING(&Poco::Logger::get("InterpreterSelectQuery"), "{}", e.message()); + return; } else { diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index 0d3ce92a4ff..22e71715bcf 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -97,7 +97,6 @@ private: BlockIO fillTableIfNeeded(const ASTCreateQuery & create); /// Check type compatible for materialized dest table and select columns void checkTypecompatibleForMaterializeView(const ASTCreateQuery & create); - Block getSampleBlockFromCreteQuery(const ASTCreateQuery & create); void assertOrSetUUID(ASTCreateQuery & create, const DatabasePtr & database) const; diff --git a/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.reference b/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.reference index e69de29bb2d..bb3ee860aec 100644 --- a/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.reference +++ b/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.reference @@ -0,0 +1,2 @@ +3 some_val +3 9 diff --git a/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql b/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql index cfb363053e5..48f4bf2b2e2 100644 --- a/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql +++ b/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql @@ -1,8 +1,7 @@ -DROP DATABASE IF EXISTS test_db; - -SET skip_materialized_view_checking_if_source_table_not_exist = 1; SET send_logs_level = 'fatal'; +DROP DATABASE IF EXISTS test_db; + CREATE DATABASE test_db; CREATE TABLE test_db.table (n Int32, s String) ENGINE MergeTree PARTITION BY n ORDER BY n; @@ -15,9 +14,21 @@ DROP TABLE test_db.table; DETACH TABLE test_db.mview; -ATTACH TABLE test_db.mview; +/* Check that we get an exception with the option. */ SET skip_materialized_view_checking_if_source_table_not_exist = 0; +ATTACH TABLE test_db.mview; --{serverError 60} + +/* Check that we don't get an exception with the option. */ +SET skip_materialized_view_checking_if_source_table_not_exist = 1; +ATTACH TABLE test_db.mview; + +/* Check if the data in the materialized view is updated after the restore.*/ +CREATE TABLE test_db.table (n Int32, s String) ENGINE MergeTree PARTITION BY n ORDER BY n; + +INSERT INTO test_db.table VALUES (3,'some_val'); + +SELECT n,s FROM test_db.table ORDER BY n; +SELECT n,n2 FROM test_db.mview ORDER by n; DROP DATABASE test_db; - diff --git a/tests/queries/0_stateless/02697_do_not_ignore_inacc_tables_mat_view_atttach.reference b/tests/queries/0_stateless/02697_do_not_ignore_inacc_tables_mat_view_atttach.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02697_do_not_ignore_inacc_tables_mat_view_atttach.sql b/tests/queries/0_stateless/02697_do_not_ignore_inacc_tables_mat_view_atttach.sql deleted file mode 100644 index 492c2ed1fa2..00000000000 --- a/tests/queries/0_stateless/02697_do_not_ignore_inacc_tables_mat_view_atttach.sql +++ /dev/null @@ -1,21 +0,0 @@ -DROP DATABASE IF EXISTS test_db; - -SET skip_materialized_view_checking_if_source_table_not_exist = 0; -SET send_logs_level = 'fatal'; - -CREATE DATABASE test_db; - -CREATE TABLE test_db.table (n Int32, s String) ENGINE MergeTree PARTITION BY n ORDER BY n; - -CREATE TABLE test_db.mview_backend (n Int32, n2 Int64) ENGINE MergeTree PARTITION BY n ORDER BY n; - -CREATE MATERIALIZED VIEW test_db.mview TO test_db.mview_backend AS SELECT n, n * n AS "n2" FROM test_db.table; - -DROP TABLE test_db.table; - -DETACH TABLE test_db.mview; - -ATTACH TABLE test_db.mview; --{serverError 60} - -DROP DATABASE test_db; - diff --git a/tests/queries/0_stateless/02698_restore_mat_view_state_after_droping_tables.reference b/tests/queries/0_stateless/02698_restore_mat_view_state_after_droping_tables.reference deleted file mode 100644 index bb3ee860aec..00000000000 --- a/tests/queries/0_stateless/02698_restore_mat_view_state_after_droping_tables.reference +++ /dev/null @@ -1,2 +0,0 @@ -3 some_val -3 9 diff --git a/tests/queries/0_stateless/02698_restore_mat_view_state_after_droping_tables.sql b/tests/queries/0_stateless/02698_restore_mat_view_state_after_droping_tables.sql deleted file mode 100644 index b733f42270a..00000000000 --- a/tests/queries/0_stateless/02698_restore_mat_view_state_after_droping_tables.sql +++ /dev/null @@ -1,28 +0,0 @@ -DROP DATABASE IF EXISTS test_db; - -SET skip_materialized_view_checking_if_source_table_not_exist = 1; -SET send_logs_level = 'fatal'; - -CREATE DATABASE test_db; - -CREATE TABLE test_db.table (n Int32, s String) ENGINE MergeTree PARTITION BY n ORDER BY n; - -CREATE TABLE test_db.mview_backend (n Int32, n2 Int64) ENGINE MergeTree PARTITION BY n ORDER BY n; - -CREATE MATERIALIZED VIEW test_db.mview TO test_db.mview_backend AS SELECT n, n * n AS "n2" FROM test_db.table; - -DROP TABLE test_db.table; - -DETACH TABLE test_db.mview; - -ATTACH TABLE test_db.mview; - -CREATE TABLE test_db.table (n Int32, s String) ENGINE MergeTree PARTITION BY n ORDER BY n; - -INSERT INTO test_db.table VALUES (3,'some_val'); - -SELECT n,s FROM test_db.table ORDER BY n; -SELECT n,n2 FROM test_db.mview ORDER by n; - -DROP DATABASE test_db; - From 0712627c058cb924676ea741cbe9a2f29a405730 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 16 Mar 2023 15:47:01 +0000 Subject: [PATCH 117/216] 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 118/216] 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 119/216] 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 120/216] 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 121/216] 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 f353561204c5707fd229fcaeefc880096006cff2 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 24 Mar 2023 14:37:40 +0000 Subject: [PATCH 122/216] 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 123/216] 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 ef4f860b62078e7fa19167209449f80e2448cf28 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 24 Mar 2023 11:51:22 -0300 Subject: [PATCH 124/216] namespace fix --- .../tests/gtest_assert_arrow_log_does_not_abort.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Processors/tests/gtest_assert_arrow_log_does_not_abort.cpp b/src/Processors/tests/gtest_assert_arrow_log_does_not_abort.cpp index fb13f4664d9..203f2d7cef6 100644 --- a/src/Processors/tests/gtest_assert_arrow_log_does_not_abort.cpp +++ b/src/Processors/tests/gtest_assert_arrow_log_does_not_abort.cpp @@ -3,7 +3,8 @@ #include #include -using namespace DB; +namespace DB +{ TEST(ChunkedArray, ChunkedArrayWithZeroChunksShouldNotAbort) { @@ -16,3 +17,5 @@ TEST(ArrowLog, FatalLogShouldThrow) { EXPECT_ANY_THROW(::arrow::util::ArrowLog(__FILE__, __LINE__, ::arrow::util::ArrowLogLevel::ARROW_FATAL)); } + +} From 912fe0115e45f00972f352fb7e9189e7f309f446 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 24 Mar 2023 14:57:49 +0000 Subject: [PATCH 125/216] 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 126/216] 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 127/216] 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 128/216] 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 16ebfcc3f8b080968107751b46a7150c13d5680f Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Mar 2023 16:15:52 +0100 Subject: [PATCH 129/216] Better delegate disks handle mutual path --- src/Disks/DiskEncrypted.h | 4 +++ src/Disks/IDisk.h | 2 ++ src/Storages/MergeTree/MergeTreeData.cpp | 33 +++++++++---------- .../test_encrypted_disk/configs/storage.xml | 1 - tests/integration/test_encrypted_disk/test.py | 23 +++++++++++++ 5 files changed, 45 insertions(+), 18 deletions(-) diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index d38c916ee6e..d37e3a560bc 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -276,6 +276,10 @@ public: return delegate->getMetadataStorage(); } + DiskPtr getDelegateDiskIfExists() const override + { + return delegate; + } private: String wrappedPath(const String & path) const diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index ea117c0e0c6..4e488bbb39a 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -423,6 +423,8 @@ public: void markDiskAsCustom() { is_custom_disk = true; } + virtual DiskPtr getDelegateDiskIfExists() const { return nullptr; } + protected: friend class DiskDecorator; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 84c301e5986..f7814fcf276 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1517,24 +1517,26 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) { /// Check extra parts at different disks, in order to not allow to miss data parts at undefined disks. std::unordered_set defined_disk_names; - /// If disk is wrapped into cached disk, it will not be defined in storage policy. - std::unordered_set disk_names_wrapped_in_cache; for (const auto & disk_ptr : disks) + { defined_disk_names.insert(disk_ptr->getName()); - for (const auto & [disk_name, disk_ptr] : getContext()->getDisksMap()) - { - /// In composable cache with the underlying source disk there might the following structure: + /// As encrypted disk can use the same path of its nested disk, + /// we need to take it into account here. + const auto & delegate = disk_ptr->getDelegateDiskIfExists(); + if (delegate && disk_ptr->getPath() == delegate->getPath()) + defined_disk_names.insert(delegate->getName()); + + /// As cache is implemented on object storage layer, not on disk level, e.g. + /// we have such structure: /// DiskObjectStorage(CachedObjectStorage(...(CachedObjectStored(ObjectStorage)...))) - /// In configuration file each of these layers has a different name, but data path - /// (getPath() result) is the same. We need to take it into account here. - if (disk_ptr->supportsCache() && defined_disk_names.contains(disk_ptr->getName())) + /// and disk_ptr->getName() here is the name of last delegate - ObjectStorage. + /// So now we need to add cache layers to defined disk names. + if (disk_ptr->supportsCache()) { auto caches = disk_ptr->getCacheLayersNames(); - disk_names_wrapped_in_cache.insert(caches.begin(), caches.end()); - LOG_TEST(log, "Cache layers for cache disk `{}`, inner disk `{}`: {}", - disk_name, disk_ptr->getName(), fmt::join(caches, ", ")); + defined_disk_names.insert(caches.begin(), caches.end()); } } @@ -1543,9 +1545,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) if (disk->isBroken() || disk->isCustomDisk()) continue; - if (!defined_disk_names.contains(disk_name) - && disk->exists(relative_data_path) - && !disk_names_wrapped_in_cache.contains(disk_name)) + if (!defined_disk_names.contains(disk_name) && disk->exists(relative_data_path)) { for (const auto it = disk->iterateDirectory(relative_data_path); it->isValid(); it->next()) { @@ -1553,9 +1553,8 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) { throw Exception( ErrorCodes::UNKNOWN_DISK, - "Part {} ({}) was found on disk {} which is not defined in the storage policy (defined disks: {}, wrapped disks: {})", - backQuote(it->name()), backQuote(it->path()), backQuote(disk_name), - fmt::join(defined_disk_names, ", "), fmt::join(disk_names_wrapped_in_cache, ", ")); + "Part {} ({}) was found on disk {} which is not defined in the storage policy (defined disks: {})", + backQuote(it->name()), backQuote(it->path()), backQuote(disk_name), fmt::join(defined_disk_names, ", ")); } } } diff --git a/tests/integration/test_encrypted_disk/configs/storage.xml b/tests/integration/test_encrypted_disk/configs/storage.xml index 5cba95a89a0..a6fff813f2e 100644 --- a/tests/integration/test_encrypted_disk/configs/storage.xml +++ b/tests/integration/test_encrypted_disk/configs/storage.xml @@ -14,7 +14,6 @@ encrypted disk_s3 - encrypted/ 1234567812345678 diff --git a/tests/integration/test_encrypted_disk/test.py b/tests/integration/test_encrypted_disk/test.py index 42ce7aab906..5b85888201b 100644 --- a/tests/integration/test_encrypted_disk/test.py +++ b/tests/integration/test_encrypted_disk/test.py @@ -12,6 +12,7 @@ node = cluster.add_instance( main_configs=["configs/storage.xml"], tmpfs=["/disk:size=100M"], with_minio=True, + stay_alive=True, ) @@ -269,3 +270,25 @@ def test_read_in_order(): node.query( "SELECT * FROM encrypted_test ORDER BY a, b SETTINGS optimize_read_in_order=0 FORMAT Null" ) + + +def test_restart(): + node.query( + """ + DROP TABLE IF EXISTS encrypted_test; + CREATE TABLE encrypted_test ( + id Int64, + data String + ) ENGINE=MergeTree() + ORDER BY id + SETTINGS disk='disk_s3_encrypted' + """ + ) + + node.query("INSERT INTO encrypted_test VALUES (0,'data'),(1,'data')") + select_query = "SELECT * FROM encrypted_test ORDER BY id FORMAT Values" + assert node.query(select_query) == "(0,'data'),(1,'data')" + + node.restart_clickhouse() + + assert node.query(select_query) == "(0,'data'),(1,'data')" From 1e462e02ed56652cd3f1fc636d2ae407aa150477 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 24 Mar 2023 17:00:04 +0100 Subject: [PATCH 130/216] add ut --- src/Common/tests/gtest_base_json.cpp | 14 ++++++++++++++ 1 file changed, 14 insertions(+) create mode 100644 src/Common/tests/gtest_base_json.cpp diff --git a/src/Common/tests/gtest_base_json.cpp b/src/Common/tests/gtest_base_json.cpp new file mode 100644 index 00000000000..2a359bd43b4 --- /dev/null +++ b/src/Common/tests/gtest_base_json.cpp @@ -0,0 +1,14 @@ +#include + +#include + +TEST(JSON, searchField) +{ + const JSON json = JSON(std::string_view(R"({"k1":1,"k2":{"k3":2,"k4":3,"k":4},"k":5})")); + ASSERT_EQ(json["k1"], 1); + ASSERT_EQ(json["k2"].toString(), R"({"k3":2,"k4":3,"k":4})"); + ASSERT_EQ(json["k2"]["k3"].getUInt(), 2); + ASSERT_EQ(json["k2"]["k4"].getUInt(), 3); + ASSERT_EQ(json["k2"]["k"].getUInt(), 4); + ASSERT_EQ(json["k"].getUInt(), 5); +} From bf7b0c5cfe5bcdf61a0ce774bb1850daba7aa2e9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 24 Mar 2023 17:07:48 +0100 Subject: [PATCH 131/216] Different approach to inject timeouts --- src/Common/ZooKeeper/ZooKeeperArgs.cpp | 20 ++++++ src/Common/ZooKeeper/ZooKeeperArgs.h | 5 ++ src/Common/ZooKeeper/ZooKeeperImpl.cpp | 65 +++++++++++++++---- src/Common/ZooKeeper/ZooKeeperImpl.h | 7 ++ .../config.d/zookeeper_fault_injection.xml | 4 ++ 5 files changed, 87 insertions(+), 14 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.cpp b/src/Common/ZooKeeper/ZooKeeperArgs.cpp index fe2f6957490..5cf7e927434 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.cpp +++ b/src/Common/ZooKeeper/ZooKeeperArgs.cpp @@ -42,6 +42,10 @@ ZooKeeperArgs::ZooKeeperArgs(const Poco::Util::AbstractConfiguration & config, c { connection_timeout_ms = config.getInt(config_name + "." + key); } + else if (key == "enable_fault_injections_during_startup") + { + enable_fault_injections_during_startup = config.getBool(config_name + "." + key); + } else if (key == "send_fault_probability") { send_fault_probability = config.getDouble(config_name + "." + key); @@ -50,6 +54,22 @@ ZooKeeperArgs::ZooKeeperArgs(const Poco::Util::AbstractConfiguration & config, c { recv_fault_probability = config.getDouble(config_name + "." + key); } + else if (key == "send_sleep_probability") + { + send_sleep_probability = config.getDouble(config_name + "." + key); + } + else if (key == "recv_sleep_probability") + { + recv_sleep_probability = config.getDouble(config_name + "." + key); + } + else if (key == "send_sleep_ms") + { + send_sleep_ms = config.getUInt64(config_name + "." + key); + } + else if (key == "recv_sleep_ms") + { + recv_sleep_ms = config.getUInt64(config_name + "." + key); + } else if (key == "identity") { identity = config.getString(config_name + "." + key); diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.h b/src/Common/ZooKeeper/ZooKeeperArgs.h index 48c8e4386f8..e1f8fe16aee 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.h +++ b/src/Common/ZooKeeper/ZooKeeperArgs.h @@ -28,8 +28,13 @@ struct ZooKeeperArgs int32_t connection_timeout_ms = Coordination::DEFAULT_CONNECTION_TIMEOUT_MS; int32_t session_timeout_ms = Coordination::DEFAULT_SESSION_TIMEOUT_MS; int32_t operation_timeout_ms = Coordination::DEFAULT_OPERATION_TIMEOUT_MS; + bool enable_fault_injections_during_startup = false; double send_fault_probability = 0.0; double recv_fault_probability = 0.0; + double send_sleep_probability = 0.0; + double recv_sleep_probability = 0.0; + UInt64 send_sleep_ms = 0; + UInt64 recv_sleep_ms = 0; DB::GetPriorityForLoadBalancing get_priority_load_balancing; }; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index b637bdea835..99e1e0d3e7f 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1,19 +1,21 @@ #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 "Coordination/KeeperConstants.h" #include "config.h" @@ -352,6 +354,14 @@ ZooKeeper::ZooKeeper( { recv_inject_fault.emplace(args.recv_fault_probability); } + if (0 < args.send_sleep_probability && args.send_sleep_probability <= 1) + { + send_inject_sleep.emplace(args.send_sleep_probability); + } + if (0 < args.recv_sleep_probability && args.recv_sleep_probability <= 1) + { + recv_inject_sleep.emplace(args.recv_sleep_probability); + } connect(nodes, args.connection_timeout_ms * 1000); @@ -571,7 +581,6 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data) static_cast(err), errorMessage(err)); } - void ZooKeeper::sendThread() { setThreadName("ZooKeeperSend"); @@ -587,6 +596,8 @@ void ZooKeeper::sendThread() auto now = clock::now(); auto next_heartbeat_time = prev_heartbeat_time + std::chrono::milliseconds(args.session_timeout_ms / 3); + maybeInjectSendSleep(); + if (next_heartbeat_time > now) { /// Wait for the next request in queue. No more than operation timeout. No more than until next heartbeat time. @@ -659,6 +670,7 @@ void ZooKeeper::receiveThread() Int64 waited_us = 0; while (!requests_queue.isFinished()) { + maybeInjectRecvSleep(); auto prev_bytes_received = in->count(); clock::time_point now = clock::now(); @@ -728,8 +740,7 @@ void ZooKeeper::receiveEvent() ZooKeeperResponsePtr response; UInt64 elapsed_ms = 0; - if (unlikely(recv_inject_fault) && recv_inject_fault.value()(thread_local_rng)) - throw Exception(Error::ZSESSIONEXPIRED, "Session expired (fault injected on recv)"); + maybeInjectRecvFault(); if (xid == PING_XID) { @@ -1078,8 +1089,7 @@ void ZooKeeper::pushRequest(RequestInfo && info) } } - if (unlikely(send_inject_fault) && send_inject_fault.value()(thread_local_rng)) - throw Exception(Error::ZSESSIONEXPIRED, "Session expired (fault injected on send)"); + maybeInjectSendFault(); if (!requests_queue.tryPush(std::move(info), args.operation_timeout_ms)) { @@ -1403,4 +1413,31 @@ void ZooKeeper::logOperationIfNeeded(const ZooKeeperRequestPtr &, const ZooKeepe {} #endif +void ZooKeeper::maybeInjectSendFault() +{ + if (unlikely(send_inject_fault) && send_inject_fault.value()(thread_local_rng) + && (args.enable_fault_injections_during_startup || Context::getGlobalContextInstance()->isServerCompletelyStarted())) + throw Exception(Error::ZSESSIONEXPIRED, "Session expired (fault injected on recv)"); +} + +void ZooKeeper::maybeInjectRecvFault() +{ + if (unlikely(recv_inject_fault) && recv_inject_fault.value()(thread_local_rng) + && (args.enable_fault_injections_during_startup || Context::getGlobalContextInstance()->isServerCompletelyStarted())) + throw Exception(Error::ZSESSIONEXPIRED, "Session expired (fault injected on recv)"); +} + +void ZooKeeper::maybeInjectSendSleep() +{ + if (unlikely(send_inject_sleep) && send_inject_sleep.value()(thread_local_rng) + && (args.enable_fault_injections_during_startup || Context::getGlobalContextInstance()->isServerCompletelyStarted())) + sleepForMilliseconds(args.send_sleep_ms); +} + +void ZooKeeper::maybeInjectRecvSleep() +{ + if (unlikely(recv_inject_sleep) && recv_inject_sleep.value()(thread_local_rng) + && (args.enable_fault_injections_during_startup || Context::getGlobalContextInstance()->isServerCompletelyStarted())) + sleepForMilliseconds(args.recv_sleep_ms); +} } diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 6b70f8bc753..2939d7bfaf8 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -202,8 +202,15 @@ private: zkutil::ZooKeeperArgs args; + /// Fault injection + void maybeInjectSendFault(); + void maybeInjectRecvFault(); + void maybeInjectSendSleep(); + void maybeInjectRecvSleep(); std::optional send_inject_fault; std::optional recv_inject_fault; + std::optional send_inject_sleep; + std::optional recv_inject_sleep; Poco::Net::StreamSocket socket; /// To avoid excessive getpeername(2) calls. diff --git a/tests/config/config.d/zookeeper_fault_injection.xml b/tests/config/config.d/zookeeper_fault_injection.xml index 1f13155a130..a339e1f0fba 100644 --- a/tests/config/config.d/zookeeper_fault_injection.xml +++ b/tests/config/config.d/zookeeper_fault_injection.xml @@ -12,7 +12,11 @@ In other words, session will expire 4 times per 99996 successful requests or approximately each 25000 requests (on average). --> + 0 0.00002 0.00002 + + 0.00001 + 10000 From 1badc3cba05015972ea2690ba2caca4b4b3a7cef Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 24 Mar 2023 17:38:19 +0100 Subject: [PATCH 132/216] 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 5a4525742d86e8a2be642eaf7c141e95927ebdd3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Mar 2023 18:53:00 +0100 Subject: [PATCH 133/216] Drop table after test --- tests/integration/test_encrypted_disk/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_encrypted_disk/test.py b/tests/integration/test_encrypted_disk/test.py index 5b85888201b..8e2935a1603 100644 --- a/tests/integration/test_encrypted_disk/test.py +++ b/tests/integration/test_encrypted_disk/test.py @@ -292,3 +292,5 @@ def test_restart(): node.restart_clickhouse() assert node.query(select_query) == "(0,'data'),(1,'data')" + + node.query("DROP TABLE IF EXISTS encrypted_test NO DELAY;") From 601aaf4d2f4ce862c96d6472e8fde9d0bdb10c7b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 24 Mar 2023 21:33:24 +0300 Subject: [PATCH 134/216] 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 135/216] 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 136/216] 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 137/216] 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 138/216] 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 139/216] 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 140/216] 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 141/216] 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 57ceb3436669ae60ac046eb562cd2627a5a98eb5 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Sat, 25 Mar 2023 13:36:58 +0000 Subject: [PATCH 142/216] Remove database from test --- ...6_ignore_inacc_tables_mat_view_atttach.sql | 27 ++++++++----------- 1 file changed, 11 insertions(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql b/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql index 48f4bf2b2e2..615737eacf0 100644 --- a/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql +++ b/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql @@ -1,34 +1,29 @@ SET send_logs_level = 'fatal'; -DROP DATABASE IF EXISTS test_db; +CREATE TABLE test_table (n Int32, s String) ENGINE MergeTree PARTITION BY n ORDER BY n; -CREATE DATABASE test_db; +CREATE TABLE mview_backend (n Int32, n2 Int64) ENGINE MergeTree PARTITION BY n ORDER BY n; -CREATE TABLE test_db.table (n Int32, s String) ENGINE MergeTree PARTITION BY n ORDER BY n; +CREATE MATERIALIZED VIEW mview TO mview_backend AS SELECT n, n * n AS "n2" FROM test_table; -CREATE TABLE test_db.mview_backend (n Int32, n2 Int64) ENGINE MergeTree PARTITION BY n ORDER BY n; +DROP TABLE test_table; -CREATE MATERIALIZED VIEW test_db.mview TO test_db.mview_backend AS SELECT n, n * n AS "n2" FROM test_db.table; - -DROP TABLE test_db.table; - -DETACH TABLE test_db.mview; +DETACH TABLE mview; /* Check that we get an exception with the option. */ SET skip_materialized_view_checking_if_source_table_not_exist = 0; -ATTACH TABLE test_db.mview; --{serverError 60} +ATTACH TABLE mview; --{serverError 60} /* Check that we don't get an exception with the option. */ SET skip_materialized_view_checking_if_source_table_not_exist = 1; -ATTACH TABLE test_db.mview; +ATTACH TABLE mview; /* Check if the data in the materialized view is updated after the restore.*/ -CREATE TABLE test_db.table (n Int32, s String) ENGINE MergeTree PARTITION BY n ORDER BY n; +CREATE TABLE test_table (n Int32, s String) ENGINE MergeTree PARTITION BY n ORDER BY n; -INSERT INTO test_db.table VALUES (3,'some_val'); +INSERT INTO test_table VALUES (3,'some_val'); -SELECT n,s FROM test_db.table ORDER BY n; -SELECT n,n2 FROM test_db.mview ORDER by n; +SELECT n,s FROM test_table ORDER BY n; +SELECT n,n2 FROM mview ORDER by n; -DROP DATABASE test_db; From 2c8916a00573dd40e38db57b15ca656f83d4d793 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 25 Mar 2023 18:05:09 +0000 Subject: [PATCH 143/216] 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 144/216] 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 145/216] 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 146/216] 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 147/216] 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 148/216] 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 149/216] 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 150/216] 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 151/216] 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 152/216] 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 153/216] 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 154/216] 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 155/216] 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 156/216] 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 157/216] 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 158/216] 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 159/216] 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 160/216] 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 161/216] 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 162/216] 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 163/216] 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 164/216] 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 165/216] 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 db55fa864acade1a27c6e1987c4135e85a08dffb Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 27 Mar 2023 10:47:34 +0200 Subject: [PATCH 166/216] fix build --- src/Common/tests/gtest_base_json.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/tests/gtest_base_json.cpp b/src/Common/tests/gtest_base_json.cpp index 2a359bd43b4..60c469f1542 100644 --- a/src/Common/tests/gtest_base_json.cpp +++ b/src/Common/tests/gtest_base_json.cpp @@ -5,7 +5,7 @@ TEST(JSON, searchField) { const JSON json = JSON(std::string_view(R"({"k1":1,"k2":{"k3":2,"k4":3,"k":4},"k":5})")); - ASSERT_EQ(json["k1"], 1); + ASSERT_EQ(json["k1"].getUInt(), 1); ASSERT_EQ(json["k2"].toString(), R"({"k3":2,"k4":3,"k":4})"); ASSERT_EQ(json["k2"]["k3"].getUInt(), 2); ASSERT_EQ(json["k2"]["k4"].getUInt(), 3); From 2981890ab4d583216f2a28722a0faddaef9a0329 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Mon, 27 Mar 2023 12:03:47 +0200 Subject: [PATCH 167/216] 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 168/216] 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 169/216] 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 170/216] 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 171/216] 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 172/216] 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 65c496176963fe6c04b73c4232f38602c65f9cf1 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Mon, 27 Mar 2023 13:47:35 +0200 Subject: [PATCH 173/216] Update tests/queries/0_stateless/25402_fail_on_invalid_having.sql --- tests/queries/0_stateless/25402_fail_on_invalid_having.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/25402_fail_on_invalid_having.sql b/tests/queries/0_stateless/25402_fail_on_invalid_having.sql index beb72218685..2d46a335dc1 100644 --- a/tests/queries/0_stateless/25402_fail_on_invalid_having.sql +++ b/tests/queries/0_stateless/25402_fail_on_invalid_having.sql @@ -1 +1 @@ -SELECT a, sum(b) FROM (SELECT 1 AS a, 1 AS b, 0 AS c) GROUP BY a HAVING c -- {{ serverError NOT_AN_AGGREGATE}} \ No newline at end of file +SELECT a, sum(b) FROM (SELECT 1 AS a, 1 AS b, 0 AS c) GROUP BY a HAVING c SETTINGS allow_experimental_analyzer=1 -- {{ serverError NOT_AN_AGGREGATE}} \ No newline at end of file From 2773e5c35241d9f1cb53cbe46e90d4124725b6cb Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 27 Mar 2023 13:50:09 +0200 Subject: [PATCH 174/216] 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 08fa1846b1e0f23f4714edc28e5aedf98d2ca094 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 27 Mar 2023 11:11:42 +0000 Subject: [PATCH 175/216] Add merge tree setting max_number_of_mutations_for_replica --- src/Core/ServerSettings.h | 2 +- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 5 +++++ src/Storages/MergeTree/MergeTreeSettings.h | 1 + 3 files changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 4fc721e70a2..e780424507c 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -61,7 +61,7 @@ namespace DB M(UInt64, concurrent_threads_soft_limit_ratio_to_cores, 0, "Same as concurrent_threads_soft_limit_num, but with ratio to cores.", 0) \ \ M(UInt64, background_pool_size, 16, "The maximum number of threads what will be used for merging or mutating data parts for *MergeTree-engine tables in a background.", 0) \ - M(UInt64, background_merges_mutations_concurrency_ratio, 2, "The multiplier which shows the relation between the number of tasks that could be executed concurrently and the number of threads being used.", 0) \ + M(UInt64, background_merges_mutations_concurrency_ratio, 2, "The number of part mutation tasks that can be executed concurrently by each thread in background pool.", 0) \ M(String, background_merges_mutations_scheduling_policy, "round_robin", "The policy on how to perform a scheduling for background merges and mutations. Possible values are: `round_robin` and `shortest_task_first`. ", 0) \ M(UInt64, background_move_pool_size, 8, "The maximum number of threads that will be used for moving data parts to another disk or volume for *MergeTree-engine tables in a background.", 0) \ M(UInt64, background_fetches_pool_size, 8, "The maximum number of threads that will be used for fetching data parts from another replica for *MergeTree-engine tables in a background.", 0) \ diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index cf009a10c27..49289df6161 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -113,9 +113,14 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartSizeForMutation() const const auto data_settings = data.getSettings(); size_t occupied = CurrentMetrics::values[CurrentMetrics::BackgroundMergesAndMutationsPoolTask].load(std::memory_order_relaxed); + if (data_settings->max_number_of_mutations_for_replica > 0 && + occupied >= data_settings->max_number_of_mutations_for_replica) + return 0; + /// DataPart can be store only at one disk. Get maximum reservable free space at all disks. UInt64 disk_space = data.getStoragePolicy()->getMaxUnreservedFreeSpace(); auto max_tasks_count = data.getContext()->getMergeMutateExecutor()->getMaxTasksCount(); + /// Allow mutations only if there are enough threads, leave free threads for merges else if (occupied <= 1 || max_tasks_count - occupied >= data_settings->number_of_free_entries_in_pool_to_execute_mutation) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 6d280f3aaec..4aabcbbf865 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -48,6 +48,7 @@ struct Settings; M(UInt64, max_replicated_merges_with_ttl_in_queue, 1, "How many tasks of merging parts with TTL are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ M(UInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge, 8, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.", 0) \ M(UInt64, number_of_free_entries_in_pool_to_execute_mutation, 20, "When there is less than specified number of free entries in pool, do not execute part mutations. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ + M(UInt64, max_number_of_mutations_for_replica, 0, "Limit the number of part mutations per replica to the specified amount. Zero means no limit on the number of mutations per replica (the execution can still be constrained by other settings).", 0) \ M(UInt64, max_number_of_merges_with_ttl_in_pool, 2, "When there is more than specified number of merges with TTL entries in pool, do not assign new merge with TTL. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ M(Seconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \ M(Seconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories. You should not lower this value because merges and mutations may not be able to work with low value of this setting.", 0) \ From cfd10dafe47b07a573e15015c9aa1b907e9fce62 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 27 Mar 2023 09:22:07 -0300 Subject: [PATCH 176/216] only link arrow lib if it exists, same for ut --- src/CMakeLists.txt | 4 +++- .../tests/gtest_assert_arrow_log_does_not_abort.cpp | 6 ++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index e5cdd14d826..62005c76ce0 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -608,7 +608,6 @@ if (ENABLE_TESTS) dbms clickhouse_common_config clickhouse_common_zookeeper - ch_contrib::parquet string_utils) if (TARGET ch_contrib::simdjson) @@ -623,6 +622,9 @@ if (ENABLE_TESTS) target_link_libraries(unit_tests_dbms PRIVATE ch_contrib::azure_sdk) endif() + if (TARGET ch_contrib::parquet) + target_link_libraries(unit_tests_dbms PRIVATE ch_contrib::parquet) + endif() add_check(unit_tests_dbms) endif () diff --git a/src/Processors/tests/gtest_assert_arrow_log_does_not_abort.cpp b/src/Processors/tests/gtest_assert_arrow_log_does_not_abort.cpp index 203f2d7cef6..8152ced287e 100644 --- a/src/Processors/tests/gtest_assert_arrow_log_does_not_abort.cpp +++ b/src/Processors/tests/gtest_assert_arrow_log_does_not_abort.cpp @@ -1,3 +1,7 @@ +#include "config.h" + +#if USE_ARROW || USE_PARQUET + #include #include #include @@ -19,3 +23,5 @@ TEST(ArrowLog, FatalLogShouldThrow) } } + +#endif From 4234c38a64a0ba7a7c2c152592b7f857813b30f1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Mar 2023 14:45:49 +0200 Subject: [PATCH 177/216] 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 714b54b322c9d7908d32c64f17f7d3e1cd0981b9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 27 Mar 2023 15:06:46 +0200 Subject: [PATCH 178/216] Reimplement enable_fault_injections_during_startup --- src/Common/ZooKeeper/ZooKeeper.cpp | 6 +++ src/Common/ZooKeeper/ZooKeeper.h | 2 + src/Common/ZooKeeper/ZooKeeperImpl.cpp | 66 ++++++++++++++++---------- src/Common/ZooKeeper/ZooKeeperImpl.h | 4 ++ src/Interpreters/Context.cpp | 25 ++++++++-- 5 files changed, 73 insertions(+), 30 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index aa6ed5b2090..67ad553bbd3 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -1163,6 +1163,12 @@ void ZooKeeper::setZooKeeperLog(std::shared_ptr zk_log_) zk->setZooKeeperLog(zk_log); } +void ZooKeeper::setServerCompletelyStarted() +{ + if (auto * zk = dynamic_cast(impl.get())) + zk->setServerCompletelyStarted(); +} + size_t getFailedOpIndex(Coordination::Error exception_code, const Coordination::Responses & responses) { diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index d20d036f04e..acd6750fced 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -520,6 +520,8 @@ public: UInt32 getSessionUptime() const { return static_cast(session_uptime.elapsedSeconds()); } + void setServerCompletelyStarted(); + private: friend class EphemeralNodeHolder; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 99e1e0d3e7f..8183569a718 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -344,24 +344,8 @@ ZooKeeper::ZooKeeper( default_acls.emplace_back(std::move(acl)); } - /// It makes sense (especially, for async requests) to inject a fault in two places: - /// pushRequest (before request is sent) and receiveEvent (after request was executed). - if (0 < args.send_fault_probability && args.send_fault_probability <= 1) - { - send_inject_fault.emplace(args.send_fault_probability); - } - if (0 < args.recv_fault_probability && args.recv_fault_probability <= 1) - { - recv_inject_fault.emplace(args.recv_fault_probability); - } - if (0 < args.send_sleep_probability && args.send_sleep_probability <= 1) - { - send_inject_sleep.emplace(args.send_sleep_probability); - } - if (0 < args.recv_sleep_probability && args.recv_sleep_probability <= 1) - { - recv_inject_sleep.emplace(args.recv_sleep_probability); - } + if (args.enable_fault_injections_during_startup) + setupFaultDistributions(); connect(nodes, args.connection_timeout_ms * 1000); @@ -1413,31 +1397,61 @@ void ZooKeeper::logOperationIfNeeded(const ZooKeeperRequestPtr &, const ZooKeepe {} #endif + +void ZooKeeper::setServerCompletelyStarted() +{ + if (!args.enable_fault_injections_during_startup) + setupFaultDistributions(); +} + +void ZooKeeper::setupFaultDistributions() +{ + /// It makes sense (especially, for async requests) to inject a fault in two places: + /// pushRequest (before request is sent) and receiveEvent (after request was executed). + if (0 < args.send_fault_probability && args.send_fault_probability <= 1) + { + LOG_INFO(log, "ZK send fault: {}%", args.send_fault_probability * 100); + send_inject_fault.emplace(args.send_fault_probability); + } + if (0 < args.recv_fault_probability && args.recv_fault_probability <= 1) + { + LOG_INFO(log, "ZK recv fault: {}%", args.recv_fault_probability * 100); + recv_inject_fault.emplace(args.recv_fault_probability); + } + if (0 < args.send_sleep_probability && args.send_sleep_probability <= 1) + { + LOG_INFO(log, "ZK send sleep: {}% -> {}ms", args.send_sleep_probability * 100, args.send_sleep_ms); + send_inject_sleep.emplace(args.send_sleep_probability); + } + if (0 < args.recv_sleep_probability && args.recv_sleep_probability <= 1) + { + LOG_INFO(log, "ZK recv sleep: {}% -> {}ms", args.recv_sleep_probability * 100, args.recv_sleep_ms); + recv_inject_sleep.emplace(args.recv_sleep_probability); + } + inject_setup.test_and_set(); +} + void ZooKeeper::maybeInjectSendFault() { - if (unlikely(send_inject_fault) && send_inject_fault.value()(thread_local_rng) - && (args.enable_fault_injections_during_startup || Context::getGlobalContextInstance()->isServerCompletelyStarted())) + if (unlikely(inject_setup.test() && send_inject_fault && send_inject_fault.value()(thread_local_rng))) throw Exception(Error::ZSESSIONEXPIRED, "Session expired (fault injected on recv)"); } void ZooKeeper::maybeInjectRecvFault() { - if (unlikely(recv_inject_fault) && recv_inject_fault.value()(thread_local_rng) - && (args.enable_fault_injections_during_startup || Context::getGlobalContextInstance()->isServerCompletelyStarted())) + if (unlikely(inject_setup.test() && recv_inject_fault && recv_inject_fault.value()(thread_local_rng))) throw Exception(Error::ZSESSIONEXPIRED, "Session expired (fault injected on recv)"); } void ZooKeeper::maybeInjectSendSleep() { - if (unlikely(send_inject_sleep) && send_inject_sleep.value()(thread_local_rng) - && (args.enable_fault_injections_during_startup || Context::getGlobalContextInstance()->isServerCompletelyStarted())) + if (unlikely(inject_setup.test() && send_inject_sleep && send_inject_sleep.value()(thread_local_rng))) sleepForMilliseconds(args.send_sleep_ms); } void ZooKeeper::maybeInjectRecvSleep() { - if (unlikely(recv_inject_sleep) && recv_inject_sleep.value()(thread_local_rng) - && (args.enable_fault_injections_during_startup || Context::getGlobalContextInstance()->isServerCompletelyStarted())) + if (unlikely(inject_setup.test() && recv_inject_sleep && recv_inject_sleep.value()(thread_local_rng))) sleepForMilliseconds(args.recv_sleep_ms); } } diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 2939d7bfaf8..91c5083bda1 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -197,6 +197,8 @@ public: void setZooKeeperLog(std::shared_ptr zk_log_); + void setServerCompletelyStarted(); + private: ACLs default_acls; @@ -207,6 +209,8 @@ private: void maybeInjectRecvFault(); void maybeInjectSendSleep(); void maybeInjectRecvSleep(); + void setupFaultDistributions(); + std::atomic_flag inject_setup = ATOMIC_FLAG_INIT; std::optional send_inject_fault; std::optional recv_inject_fault; std::optional send_inject_sleep; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5b412416747..286e91f4a40 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2588,8 +2588,12 @@ void Context::resetZooKeeper() const shared->zookeeper.reset(); } -static void reloadZooKeeperIfChangedImpl(const ConfigurationPtr & config, const std::string & config_name, zkutil::ZooKeeperPtr & zk, - std::shared_ptr zk_log) +static void reloadZooKeeperIfChangedImpl( + const ConfigurationPtr & config, + const std::string & config_name, + zkutil::ZooKeeperPtr & zk, + std::shared_ptr zk_log, + bool server_started) { if (!zk || zk->configChanged(*config, config_name)) { @@ -2597,18 +2601,22 @@ static void reloadZooKeeperIfChangedImpl(const ConfigurationPtr & config, const zk->finalize("Config changed"); zk = std::make_shared(*config, config_name, std::move(zk_log)); + if (server_started) + zk->setServerCompletelyStarted(); } } void Context::reloadZooKeeperIfChanged(const ConfigurationPtr & config) const { + bool server_started = isServerCompletelyStarted(); std::lock_guard lock(shared->zookeeper_mutex); shared->zookeeper_config = config; - reloadZooKeeperIfChangedImpl(config, "zookeeper", shared->zookeeper, getZooKeeperLog()); + reloadZooKeeperIfChangedImpl(config, "zookeeper", shared->zookeeper, getZooKeeperLog(), server_started); } void Context::reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & config) { + bool server_started = isServerCompletelyStarted(); std::lock_guard lock(shared->auxiliary_zookeepers_mutex); shared->auxiliary_zookeepers_config = config; @@ -2619,7 +2627,7 @@ void Context::reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & it = shared->auxiliary_zookeepers.erase(it); else { - reloadZooKeeperIfChangedImpl(config, "auxiliary_zookeepers." + it->first, it->second, getZooKeeperLog()); + reloadZooKeeperIfChangedImpl(config, "auxiliary_zookeepers." + it->first, it->second, getZooKeeperLog(), server_started); ++it; } } @@ -3695,6 +3703,15 @@ bool Context::isServerCompletelyStarted() const void Context::setServerCompletelyStarted() { + { + std::lock_guard lock(shared->zookeeper_mutex); + if (shared->zookeeper) + shared->zookeeper->setServerCompletelyStarted(); + + for (auto & zk : shared->auxiliary_zookeepers) + zk.second->setServerCompletelyStarted(); + } + auto lock = getLock(); assert(global_context.lock().get() == this); assert(!shared->is_server_completely_started); From c47dcb0805b0787abc6db6134aed935f6a3e708f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 27 Mar 2023 15:07:59 +0200 Subject: [PATCH 179/216] Enable ZK fault injection on new servers (but not during startup) --- docker/test/upgrade/run.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index b9abe5b51fe..fce90ca2537 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -109,8 +109,7 @@ mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/c # Install and start new server install_packages package_folder -# Disable fault injections on start (we don't test them here, and it can lead to tons of requests in case of huge number of tables). -export ZOOKEEPER_FAULT_INJECTION=0 +export ZOOKEEPER_FAULT_INJECTION=1 configure start 500 clickhouse-client --query "SELECT 'Server successfully started', 'OK', NULL, ''" >> /test_output/test_results.tsv \ From 64db4c478b76e179d067806795a3353d10372410 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 27 Mar 2023 13:19:45 +0000 Subject: [PATCH 180/216] 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 181/216] 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 cb04c6301c51ebf92aef27fc70bede97cae710b3 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 27 Mar 2023 16:30:53 +0200 Subject: [PATCH 182/216] =?UTF-8?q?Support=20for=20cte=20in=20parameterize?= =?UTF-8?q?d=20views=E2=80=A8Implementation:=20*=20Updated=20to=20allow=20?= =?UTF-8?q?query=20parameters=20while=20evaluating=20scalar=20subqueries.?= =?UTF-8?q?=20Testing:=20*=20Added=20test=20case=20with=20cute=20for=20par?= =?UTF-8?q?ameterized=20view.?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../ExecuteScalarSubqueriesVisitor.cpp | 1 + .../ExecuteScalarSubqueriesVisitor.h | 1 + src/Interpreters/TreeRewriter.cpp | 8 +++---- .../02428_parameterized_view.reference | 1 + .../0_stateless/02428_parameterized_view.sh | 22 ++++++++++++++++++- 5 files changed, 28 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index 7a5fc67596f..3d5f14f4723 100644 --- a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -98,6 +98,7 @@ static auto getQueryInterpreter(const ASTSubquery & subquery, ExecuteScalarSubqu ASTPtr subquery_select = subquery.children.at(0); auto options = SelectQueryOptions(QueryProcessingStage::Complete, data.subquery_depth + 1, true); + options.is_create_parameterized_view = data.is_create_parameterized_view; options.analyze(data.only_analyze); return std::make_unique(subquery_select, subquery_context, options); diff --git a/src/Interpreters/ExecuteScalarSubqueriesVisitor.h b/src/Interpreters/ExecuteScalarSubqueriesVisitor.h index d702404dab6..577140f731f 100644 --- a/src/Interpreters/ExecuteScalarSubqueriesVisitor.h +++ b/src/Interpreters/ExecuteScalarSubqueriesVisitor.h @@ -36,6 +36,7 @@ public: Scalars & scalars; Scalars & local_scalars; bool only_analyze; + bool is_create_parameterized_view; }; static bool needChildVisit(ASTPtr & node, const ASTPtr &); diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 716d63f03e4..bf684077942 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -452,10 +452,10 @@ void removeUnneededColumnsFromSelectClause(ASTSelectQuery * select_query, const /// Replacing scalar subqueries with constant values. void executeScalarSubqueries( - ASTPtr & query, ContextPtr context, size_t subquery_depth, Scalars & scalars, Scalars & local_scalars, bool only_analyze) + ASTPtr & query, ContextPtr context, size_t subquery_depth, Scalars & scalars, Scalars & local_scalars, bool only_analyze, bool is_create_parameterized_view) { LogAST log; - ExecuteScalarSubqueriesVisitor::Data visitor_data{WithContext{context}, subquery_depth, scalars, local_scalars, only_analyze}; + ExecuteScalarSubqueriesVisitor::Data visitor_data{WithContext{context}, subquery_depth, scalars, local_scalars, only_analyze, is_create_parameterized_view}; ExecuteScalarSubqueriesVisitor(visitor_data, log.stream()).visit(query); } @@ -1222,7 +1222,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( removeUnneededColumnsFromSelectClause(select_query, required_result_columns, remove_duplicates); /// Executing scalar subqueries - replacing them with constant values. - executeScalarSubqueries(query, getContext(), subquery_depth, result.scalars, result.local_scalars, select_options.only_analyze); + executeScalarSubqueries(query, getContext(), subquery_depth, result.scalars, result.local_scalars, select_options.only_analyze, select_options.is_create_parameterized_view); if (settings.legacy_column_name_of_tuple_literal) markTupleLiteralsAsLegacy(query); @@ -1336,7 +1336,7 @@ TreeRewriterResultPtr TreeRewriter::analyze( normalize(query, result.aliases, result.source_columns_set, false, settings, allow_self_aliases, getContext(), is_create_parameterized_view); /// Executing scalar subqueries. Column defaults could be a scalar subquery. - executeScalarSubqueries(query, getContext(), 0, result.scalars, result.local_scalars, !execute_scalar_subqueries); + executeScalarSubqueries(query, getContext(), 0, result.scalars, result.local_scalars, !execute_scalar_subqueries, is_create_parameterized_view); if (settings.legacy_column_name_of_tuple_literal) markTupleLiteralsAsLegacy(query); diff --git a/tests/queries/0_stateless/02428_parameterized_view.reference b/tests/queries/0_stateless/02428_parameterized_view.reference index 5af9e75680c..59a3f4d67dd 100644 --- a/tests/queries/0_stateless/02428_parameterized_view.reference +++ b/tests/queries/0_stateless/02428_parameterized_view.reference @@ -38,3 +38,4 @@ ERROR 10 10 10 +1 diff --git a/tests/queries/0_stateless/02428_parameterized_view.sh b/tests/queries/0_stateless/02428_parameterized_view.sh index 274ee43e00d..57a43ce08a0 100755 --- a/tests/queries/0_stateless/02428_parameterized_view.sh +++ b/tests/queries/0_stateless/02428_parameterized_view.sh @@ -23,6 +23,9 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_02428_Catalog" $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 "DROP VIEW IF EXISTS 02428_trace_view" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS 02428_otel_traces_trace_id_ts" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS 02428_otel_traces" $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" @@ -95,6 +98,20 @@ $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 "CREATE TABLE 02428_otel_traces (TraceId String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/otel_traces', 'r1') ORDER BY TraceId" +$CLICKHOUSE_CLIENT -q "CREATE TABLE 02428_otel_traces_trace_id_ts (TraceId String, Start Timestamp) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/otel_traces_trace_id_ts', 'r1') ORDER BY TraceId" + +$CLICKHOUSE_CLIENT -q "INSERT INTO 02428_otel_traces(TraceId) VALUES ('1')" +$CLICKHOUSE_CLIENT -q "INSERT INTO 02428_otel_traces_trace_id_ts(TraceId, Start) VALUES('1', now())" + +$CLICKHOUSE_CLIENT -q "CREATE VIEW 02428_trace_view AS WITH {trace_id:String} AS trace_id, + ( SELECT min(Start) FROM 02428_otel_traces_trace_id_ts WHERE TraceId = trace_id + ) AS start SELECT + TraceId AS traceID + FROM 02428_otel_traces" +$CLICKHOUSE_CLIENT -q "SELECT * FROM 02428_trace_view(trace_id='1')" + + $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv1" $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv2" $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv3" @@ -110,4 +127,7 @@ $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" $CLICKHOUSE_CLIENT -q "DROP TABLE ${CLICKHOUSE_TEST_UNIQUE_NAME}.Catalog" -$CLICKHOUSE_CLIENT -q "DROP DATABASE ${CLICKHOUSE_TEST_UNIQUE_NAME}" \ No newline at end of file +$CLICKHOUSE_CLIENT -q "DROP DATABASE ${CLICKHOUSE_TEST_UNIQUE_NAME}" +$CLICKHOUSE_CLIENT -q "DROP VIEW 02428_trace_view" +$CLICKHOUSE_CLIENT -q "DROP TABLE 02428_otel_traces_trace_id_ts" +$CLICKHOUSE_CLIENT -q "DROP TABLE 02428_otel_traces" \ No newline at end of file From 61d41e8152cc79166de54c11e10ccf50b1418fe4 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 27 Mar 2023 14:41:36 +0000 Subject: [PATCH 183/216] Use uniq names for Records in Avro to avoid reusing its schema --- src/Processors/Formats/Impl/AvroRowOutputFormat.cpp | 4 +++- .../02592_avro_records_with_same_names.reference | 1 + .../0_stateless/02592_avro_records_with_same_names.sh | 9 +++++++++ 3 files changed, 13 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02592_avro_records_with_same_names.reference create mode 100755 tests/queries/0_stateless/02592_avro_records_with_same_names.sh diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp index f1b42147cd6..8a851b511da 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp @@ -386,7 +386,9 @@ AvroSerializer::SchemaWithSerializeFn AvroSerializer::createSchemaWithSerializeF const auto & nested_names = tuple_type.getElementNames(); std::vector nested_serializers; nested_serializers.reserve(nested_types.size()); - auto schema = avro::RecordSchema(column_name); + /// We should use unique names for records. Otherwise avro will reuse schema of this record later + /// for all records with the same name. + auto schema = avro::RecordSchema(column_name + "_" + std::to_string(type_name_increment)); for (size_t i = 0; i != nested_types.size(); ++i) { auto nested_mapping = createSchemaWithSerializeFn(nested_types[i], type_name_increment, nested_names[i]); diff --git a/tests/queries/0_stateless/02592_avro_records_with_same_names.reference b/tests/queries/0_stateless/02592_avro_records_with_same_names.reference new file mode 100644 index 00000000000..7237be8884e --- /dev/null +++ b/tests/queries/0_stateless/02592_avro_records_with_same_names.reference @@ -0,0 +1 @@ +((1,2)) ((3,4,5)) diff --git a/tests/queries/0_stateless/02592_avro_records_with_same_names.sh b/tests/queries/0_stateless/02592_avro_records_with_same_names.sh new file mode 100755 index 00000000000..d13d36230e3 --- /dev/null +++ b/tests/queries/0_stateless/02592_avro_records_with_same_names.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +SCHEMADIR=$CURDIR/format_schemas +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "select tuple(tuple(1, 2))::Tuple(x Tuple(a UInt32, b UInt32)) as c1, tuple(tuple(3, 4, 5))::Tuple(x Tuple(c UInt32, d UInt32, e UInt32)) as c2 format Avro" | $CLICKHOUSE_LOCAL --input-format Avro --structure 'c1 Tuple(x Tuple(a UInt32, b UInt32)), c2 Tuple(x Tuple(c UInt32, d UInt32, e UInt32))' -q "select * from table" From 3701fda6b1ab05542117f7773ffd05e6dc4154ba Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 26 Mar 2023 13:43:15 +0000 Subject: [PATCH 184/216] Add str_to_date() alias for MySQL compat Fixes #43755 --- .../functions/type-conversion-functions.md | 18 +++ src/Functions/parseDateTime.cpp | 135 +++++++++++++++--- ...new_functions_must_be_documented.reference | 4 + .../02668_parse_datetime.reference | 13 ++ .../0_stateless/02668_parse_datetime.sql | 10 +- ...68_parse_datetime_in_joda_syntax.reference | 9 ++ .../02668_parse_datetime_in_joda_syntax.sql | 6 + 7 files changed, 172 insertions(+), 23 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 90f6cf0aa7d..6e4686a0b93 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1188,6 +1188,16 @@ SELECT parseDateTime('2021-01-04+23:00:00', '%Y-%m-%d+%H:%i:%s') Alias: `TO_TIMESTAMP`. +## parseDateTimeOrZero + +Same as for [parseDateTime](#type_conversion_functions-parseDateTime) except that it returns zero date when it encounters a date format that cannot be processed. + +## parseDateTimeOrNull + +Same as for [parseDateTime](#type_conversion_functions-parseDateTime) except that it returns `NULL` when it encounters a date format that cannot be processed. + +Alias: `str_to_date`. + ## 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. @@ -1227,6 +1237,14 @@ SELECT parseDateTimeInJodaSyntax('2023-02-24 14:53:31', 'yyyy-MM-dd HH:mm:ss', ' └─────────────────────────────────────────────────────────────────────────────────────────┘ ``` +## parseDateTimeInJodaSyntaxOrZero + +Same as for [parseDateTimeInJodaSyntax](#type_conversion_functions-parseDateTimeInJodaSyntax) except that it returns zero date when it encounters a date format that cannot be processed. + +## parseDateTimeInJodaSyntaxOrNull + +Same as for [parseDateTimeInJodaSyntax](#type_conversion_functions-parseDateTimeInJodaSyntax) except that it returns `NULL` when it encounters a date format that cannot be processed. + ## parseDateTimeBestEffort ## parseDateTime32BestEffort diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index abee7e0d8f8..6a7a6010d4b 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -1,3 +1,5 @@ +#include +#include #include #include #include @@ -452,8 +454,15 @@ namespace Joda }; + enum class ErrorHandling + { + Exception, + Zero, + Null + }; + /// _FUNC_(str[, format, timezone]) - template + template class FunctionParseDateTimeImpl : public IFunction { public: @@ -500,11 +509,14 @@ namespace getName()); String time_zone_name = getTimeZone(arguments).getTimeZone(); - return std::make_shared(time_zone_name); + DataTypePtr date_type = std::make_shared(time_zone_name); + if (error_handling == ErrorHandling::Null) + return std::make_shared(date_type); + else + return date_type; } - ColumnPtr - executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override { const auto * col_str = checkAndGetColumn(arguments[0].column.get()); if (!col_str) @@ -518,8 +530,12 @@ namespace const auto & time_zone = getTimeZone(arguments); std::vector instructions = parseFormat(format); - auto col_res = ColumnDateTime::create(); - col_res->reserve(input_rows_count); + auto col_res = ColumnDateTime::create(input_rows_count); + + ColumnUInt8::MutablePtr col_null_map; + if constexpr (error_handling == ErrorHandling::Null) + col_null_map = ColumnUInt8::create(input_rows_count, 0); + auto & res_data = col_res->getData(); /// Make datetime fit in a cache line. @@ -527,29 +543,77 @@ namespace for (size_t i = 0; i < input_rows_count; ++i) { datetime.reset(); - StringRef str_ref = col_str->getDataAt(i); Pos cur = str_ref.data; Pos end = str_ref.data + str_ref.size; + bool error = false; + for (const auto & instruction : instructions) { - cur = instruction.perform(cur, end, datetime); + try + { + cur = instruction.perform(cur, end, datetime); + } + catch (...) + { + if constexpr (error_handling == ErrorHandling::Zero) + { + res_data[i] = 0; + error = true; + break; + } + else if constexpr (error_handling == ErrorHandling::Null) + { + res_data[i] = 0; + col_null_map->getData()[i] = 1; + error = true; + break; + } + else + { + static_assert(error_handling == ErrorHandling::Exception); + throw; + } + } } - // Ensure all input was consumed. - if (cur < end) - throw Exception( - ErrorCodes::CANNOT_PARSE_DATETIME, - "Invalid format input {} is malformed at {}", - str_ref.toView(), - std::string_view(cur, end - cur)); + if (error) + continue; - Int64 time = datetime.buildDateTime(time_zone); - res_data.push_back(static_cast(time)); + try + { + /// Ensure all input was consumed + if (cur < end) + throw Exception( + ErrorCodes::CANNOT_PARSE_DATETIME, + "Invalid format input {} is malformed at {}", + str_ref.toView(), + std::string_view(cur, end - cur)); + Int64 time = datetime.buildDateTime(time_zone); + res_data[i] = static_cast(time); + } + catch (...) + { + if constexpr (error_handling == ErrorHandling::Zero) + res_data[i] = 0; + else if constexpr (error_handling == ErrorHandling::Null) + { + res_data[i] = 0; + col_null_map->getData()[i] = 1; + } + else + { + static_assert(error_handling == ErrorHandling::Exception); + throw; + } + } } - return col_res; - } + if constexpr (error_handling == ErrorHandling::Null) + return ColumnNullable::create(std::move(col_res), std::move(col_null_map)); + else + return col_res; + } private: @@ -1753,23 +1817,50 @@ namespace static constexpr auto name = "parseDateTime"; }; + struct NameParseDateTimeOrZero + { + static constexpr auto name = "parseDateTimeOrZero"; + }; + + struct NameParseDateTimeOrNull + { + static constexpr auto name = "parseDateTimeOrNull"; + }; + struct NameParseDateTimeInJodaSyntax { static constexpr auto name = "parseDateTimeInJodaSyntax"; }; + struct NameParseDateTimeInJodaSyntaxOrZero + { + static constexpr auto name = "parseDateTimeInJodaSyntaxOrZero"; + }; - using FunctionParseDateTime = FunctionParseDateTimeImpl; - using FunctionParseDateTimeInJodaSyntax - = FunctionParseDateTimeImpl; + struct NameParseDateTimeInJodaSyntaxOrNull + { + static constexpr auto name = "parseDateTimeInJodaSyntaxOrNull"; + }; + + using FunctionParseDateTime = FunctionParseDateTimeImpl; + using FunctionParseDateTimeOrZero = FunctionParseDateTimeImpl; + using FunctionParseDateTimeOrNull = FunctionParseDateTimeImpl; + using FunctionParseDateTimeInJodaSyntax = FunctionParseDateTimeImpl; + using FunctionParseDateTimeInJodaSyntaxOrZero = FunctionParseDateTimeImpl; + using FunctionParseDateTimeInJodaSyntaxOrNull = FunctionParseDateTimeImpl; } REGISTER_FUNCTION(ParseDateTime) { factory.registerFunction(); factory.registerAlias("TO_UNIXTIME", FunctionParseDateTime::name); + factory.registerFunction(); + factory.registerFunction(); + factory.registerAlias("str_to_date", FunctionParseDateTimeOrNull::name); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); } 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 4ae98bda16d..6682f5668ea 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 @@ -530,6 +530,10 @@ parseDateTimeBestEffortUS parseDateTimeBestEffortUSOrNull parseDateTimeBestEffortUSOrZero parseDateTimeInJodaSyntax +parseDateTimeInJodaSyntaxOrNull +parseDateTimeInJodaSyntaxOrZero +parseDateTimeOrNull +parseDateTimeOrZero parseTimeDelta partitionId path diff --git a/tests/queries/0_stateless/02668_parse_datetime.reference b/tests/queries/0_stateless/02668_parse_datetime.reference index b893c1bc6e9..f39655c6a41 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.reference +++ b/tests/queries/0_stateless/02668_parse_datetime.reference @@ -197,3 +197,16 @@ select parseDateTime('2019-07-03 11:04:10', '%Y-%m-%d %H:%i:%s', 'UTC') = toDate 1 select parseDateTime('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); 1 +-- *OrZero, *OrNull, str_to_date +select parseDateTimeOrZero('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); +1 +select parseDateTimeOrZero('10:04:11 invalid 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTimeOrNull('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); +1 +select parseDateTimeOrNull('10:04:11 invalid 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') IS NULL; +1 +select str_to_date('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); +1 +select str_to_date('10:04:11 invalid 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') IS NULL; +1 diff --git a/tests/queries/0_stateless/02668_parse_datetime.sql b/tests/queries/0_stateless/02668_parse_datetime.sql index 86e8877eedc..757c4fe2efe 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.sql +++ b/tests/queries/0_stateless/02668_parse_datetime.sql @@ -132,4 +132,12 @@ select parseDateTime('2021-01-04+23:00:00', '%Y-%m-%d+%H:%i:%s', 'UTC') = toDate 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 } \ No newline at end of file +-- *OrZero, *OrNull, str_to_date +select parseDateTimeOrZero('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); +select parseDateTimeOrZero('10:04:11 invalid 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTimeOrNull('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); +select parseDateTimeOrNull('10:04:11 invalid 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') IS NULL; +select str_to_date('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); +select str_to_date('10:04:11 invalid 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') IS NULL; + +-- { echoOff } 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 7402f104ae4..124836d6118 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 @@ -344,3 +344,12 @@ select parseDateTimeInJodaSyntax('-1', 's', 'UTC'); -- { serverError CANNOT_PARS select parseDateTimeInJodaSyntax('123456789', 's', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- integer overflow in AST Fuzzer select parseDateTimeInJodaSyntax('19191919191919191919191919191919', 'CCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCC', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +-- *OrZero, *OrNull +select parseDateTimeInJodaSyntaxOrZero('2001 366 2000', 'yyyy D yyyy', 'UTC') = toDateTime('2000-12-31', 'UTC'); +1 +select parseDateTimeInJodaSyntaxOrZero('2001 invalid 366 2000', 'yyyy D yyyy', 'UTC') = toDateTime('1970-01-01', 'UTC'); +1 +select parseDateTimeInJodaSyntaxOrNull('2001 366 2000', 'yyyy D yyyy', 'UTC') = toDateTime('2000-12-31', 'UTC'); +1 +select parseDateTimeInJodaSyntaxOrNull('2001 invalid 366 2000', 'yyyy D yyyy', 'UTC') IS NULL; +1 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 7ce5c1a4fdd..b2c781432d1 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 @@ -232,4 +232,10 @@ select parseDateTimeInJodaSyntax('123456789', 's', 'UTC'); -- { serverError CANN -- integer overflow in AST Fuzzer select parseDateTimeInJodaSyntax('19191919191919191919191919191919', 'CCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCC', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +-- *OrZero, *OrNull +select parseDateTimeInJodaSyntaxOrZero('2001 366 2000', 'yyyy D yyyy', 'UTC') = toDateTime('2000-12-31', 'UTC'); +select parseDateTimeInJodaSyntaxOrZero('2001 invalid 366 2000', 'yyyy D yyyy', 'UTC') = toDateTime('1970-01-01', 'UTC'); +select parseDateTimeInJodaSyntaxOrNull('2001 366 2000', 'yyyy D yyyy', 'UTC') = toDateTime('2000-12-31', 'UTC'); +select parseDateTimeInJodaSyntaxOrNull('2001 invalid 366 2000', 'yyyy D yyyy', 'UTC') IS NULL; + -- { echoOff } From f1258a932fcaf34e5a6241501ffc9d5519f1fe72 Mon Sep 17 00:00:00 2001 From: nellicus Date: Mon, 27 Mar 2023 16:58:20 +0200 Subject: [PATCH 185/216] 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 186/216] 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 187/216] 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 188/216] 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 ceefe77ce2395f4522498a4012486efba8b960f4 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 27 Mar 2023 17:40:17 +0200 Subject: [PATCH 189/216] Fix style --- tests/queries/0_stateless/02592_avro_records_with_same_names.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02592_avro_records_with_same_names.sh b/tests/queries/0_stateless/02592_avro_records_with_same_names.sh index d13d36230e3..92a7846d3bd 100755 --- a/tests/queries/0_stateless/02592_avro_records_with_same_names.sh +++ b/tests/queries/0_stateless/02592_avro_records_with_same_names.sh @@ -2,7 +2,6 @@ # Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -SCHEMADIR=$CURDIR/format_schemas # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh 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 190/216] 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 191/216] 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 c39d931ef7db67a76aff3787f402b9ad42105010 Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Fri, 24 Mar 2023 09:07:33 +0100 Subject: [PATCH 192/216] Fix wide_integer_impl.h compilation in macOS M1 --- base/base/wide_integer_impl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/base/wide_integer_impl.h b/base/base/wide_integer_impl.h index 30d08be2e4f..712c8516dad 100644 --- a/base/base/wide_integer_impl.h +++ b/base/base/wide_integer_impl.h @@ -1242,7 +1242,7 @@ constexpr integer::operator long double() const noexcept for (unsigned i = 0; i < _impl::item_count; ++i) { long double t = res; - res *= std::numeric_limits::max(); + res *= std::numeric_limits::max(); res += t; res += tmp.items[_impl::big(i)]; } From 20fa73042d1f351a7aca1d7545fd16525d4fa5cb Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Fri, 24 Mar 2023 11:26:35 +0100 Subject: [PATCH 193/216] Fix AggregateFunctionQuantile.h compilation in macOS M1 --- src/AggregateFunctions/AggregateFunctionQuantile.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionQuantile.h b/src/AggregateFunctions/AggregateFunctionQuantile.h index 49157acf690..21799ccd65a 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantile.h +++ b/src/AggregateFunctions/AggregateFunctionQuantile.h @@ -125,7 +125,7 @@ public: if constexpr (std::is_same_v>) { /// QuantileTiming only supports unsigned integers. Too large values are also meaningless. - if (isNaN(value) || value > std::numeric_limits::max() || value < 0) + if (isNaN(value) || value > std::numeric_limits::max() || value < 0) return; } From 4eafb1ae6b03c20624da97260ad8af17cfe30841 Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Fri, 24 Mar 2023 23:58:59 +0100 Subject: [PATCH 194/216] Fix tests --- base/base/wide_integer_impl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/base/wide_integer_impl.h b/base/base/wide_integer_impl.h index 712c8516dad..7cdb527f9cf 100644 --- a/base/base/wide_integer_impl.h +++ b/base/base/wide_integer_impl.h @@ -1242,7 +1242,7 @@ constexpr integer::operator long double() const noexcept for (unsigned i = 0; i < _impl::item_count; ++i) { long double t = res; - res *= std::numeric_limits::max(); + res *= static_cast(std::numeric_limits::max()); res += t; res += tmp.items[_impl::big(i)]; } From b64cb10d205d0d0b8339dfba4beedca5359fc786 Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Mon, 27 Mar 2023 13:23:12 +0200 Subject: [PATCH 195/216] Rework AggregateFunctionQuantile.h fix --- src/AggregateFunctions/AggregateFunctionQuantile.h | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionQuantile.h b/src/AggregateFunctions/AggregateFunctionQuantile.h index 21799ccd65a..01ff019f221 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantile.h +++ b/src/AggregateFunctions/AggregateFunctionQuantile.h @@ -125,8 +125,15 @@ public: if constexpr (std::is_same_v>) { /// QuantileTiming only supports unsigned integers. Too large values are also meaningless. - if (isNaN(value) || value > std::numeric_limits::max() || value < 0) + #ifdef OS_DARWIN + #pragma clang diagnostic push + #pragma clang diagnostic ignored "-Wimplicit-const-int-float-conversion" + #endif + if (isNaN(value) || value > std::numeric_limits::max() || value < 0) return; + #ifdef OS_DARWIN + #pragma clang diagnostic pop + #endif } if constexpr (has_second_arg) From feac46eac4b06ff21addd4985559dc7b53e28c2e Mon Sep 17 00:00:00 2001 From: Rich Raposa Date: Mon, 27 Mar 2023 11:16:28 -0600 Subject: [PATCH 196/216] 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 314ee1244229ee6adfb7340ea29158b5fbbd8b6e Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 27 Mar 2023 16:54:40 +0200 Subject: [PATCH 197/216] Fix checks --- src/Storages/MergeTree/MergeTreeData.cpp | 24 ++++++++----- .../test_encrypted_disk/configs/storage.xml | 30 ++++++++++++++++ tests/integration/test_encrypted_disk/test.py | 35 ++++++++++--------- 3 files changed, 63 insertions(+), 26 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f7814fcf276..4d04b8ba69b 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1521,21 +1521,27 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) for (const auto & disk_ptr : disks) { defined_disk_names.insert(disk_ptr->getName()); + } + /// In case of delegate disks it is not enough to traverse `disks`, + /// because for example cache or encrypted disk which wrap s3 disk and s3 disk itself can be put into different storage policies. + /// But disk->exists returns the same thing for both disks. + for (const auto & [disk_name, disk] : getContext()->getDisksMap()) + { /// As encrypted disk can use the same path of its nested disk, /// we need to take it into account here. - const auto & delegate = disk_ptr->getDelegateDiskIfExists(); - if (delegate && disk_ptr->getPath() == delegate->getPath()) + const auto & delegate = disk->getDelegateDiskIfExists(); + if (delegate && disk->getPath() == delegate->getPath()) defined_disk_names.insert(delegate->getName()); - /// As cache is implemented on object storage layer, not on disk level, e.g. - /// we have such structure: - /// DiskObjectStorage(CachedObjectStorage(...(CachedObjectStored(ObjectStorage)...))) - /// and disk_ptr->getName() here is the name of last delegate - ObjectStorage. - /// So now we need to add cache layers to defined disk names. - if (disk_ptr->supportsCache()) + if (disk->supportsCache()) { - auto caches = disk_ptr->getCacheLayersNames(); + /// As cache is implemented on object storage layer, not on disk level, e.g. + /// we have such structure: + /// DiskObjectStorage(CachedObjectStorage(...(CachedObjectStored(ObjectStorage)...))) + /// and disk_ptr->getName() here is the name of last delegate - ObjectStorage. + /// So now we need to add cache layers to defined disk names. + auto caches = disk->getCacheLayersNames(); defined_disk_names.insert(caches.begin(), caches.end()); } } diff --git a/tests/integration/test_encrypted_disk/configs/storage.xml b/tests/integration/test_encrypted_disk/configs/storage.xml index a6fff813f2e..1e48c80d50f 100644 --- a/tests/integration/test_encrypted_disk/configs/storage.xml +++ b/tests/integration/test_encrypted_disk/configs/storage.xml @@ -15,7 +15,23 @@ encrypted disk_s3 1234567812345678 + encrypted/ + + encrypted + disk_s3 + 1234567812345678 + + + disk_s3 + s3_cache/ + 1Gi + + + encrypted + s3_cache + 1234567812345678 + encrypted disk_local @@ -73,6 +89,20 @@ + + +
+ disk_s3_encrypted_default_path +
+
+
_ + + +
+ encrypted_s3_cache +
+
+
diff --git a/tests/integration/test_encrypted_disk/test.py b/tests/integration/test_encrypted_disk/test.py index 8e2935a1603..681df89dd0f 100644 --- a/tests/integration/test_encrypted_disk/test.py +++ b/tests/integration/test_encrypted_disk/test.py @@ -273,24 +273,25 @@ def test_read_in_order(): def test_restart(): - node.query( - """ - DROP TABLE IF EXISTS encrypted_test; - CREATE TABLE encrypted_test ( - id Int64, - data String - ) ENGINE=MergeTree() - ORDER BY id - SETTINGS disk='disk_s3_encrypted' - """ - ) + for policy in ["disk_s3_encrypted_default_path", "encrypted_s3_cache"]: + node.query( + f""" + DROP TABLE IF EXISTS encrypted_test; + CREATE TABLE encrypted_test ( + id Int64, + data String + ) ENGINE=MergeTree() + ORDER BY id + SETTINGS disk='{policy}' + """ + ) - node.query("INSERT INTO encrypted_test VALUES (0,'data'),(1,'data')") - select_query = "SELECT * FROM encrypted_test ORDER BY id FORMAT Values" - assert node.query(select_query) == "(0,'data'),(1,'data')" + node.query("INSERT INTO encrypted_test VALUES (0,'data'),(1,'data')") + select_query = "SELECT * FROM encrypted_test ORDER BY id FORMAT Values" + assert node.query(select_query) == "(0,'data'),(1,'data')" - node.restart_clickhouse() + node.restart_clickhouse() - assert node.query(select_query) == "(0,'data'),(1,'data')" + assert node.query(select_query) == "(0,'data'),(1,'data')" - node.query("DROP TABLE IF EXISTS encrypted_test NO DELAY;") + node.query("DROP TABLE encrypted_test NO DELAY;") From 98488f30ab25062e789acf40b6a8d13cc6b1e0cb Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 27 Mar 2023 14:40:45 -0300 Subject: [PATCH 198/216] 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 199/216] 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 c296c2e299c1dd31b9bdf49a8f5d1125f7b86f97 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Mon, 27 Mar 2023 18:34:12 +0000 Subject: [PATCH 200/216] Review changes --- src/Core/Settings.h | 1 - src/Interpreters/InterpreterCreateQuery.cpp | 97 ++++++++----------- src/Interpreters/InterpreterCreateQuery.h | 2 - ...6_ignore_inacc_tables_mat_view_atttach.sql | 6 -- 4 files changed, 38 insertions(+), 68 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6670920f613..accfe387137 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -467,7 +467,6 @@ class IColumn; M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited.", 0) \ M(Bool, check_query_single_value_result, true, "Return check query result as single 1/0 value", 0) \ M(Bool, allow_drop_detached, false, "Allow ALTER TABLE ... DROP DETACHED PART[ITION] ... queries", 0) \ - M(Bool, skip_materialized_view_checking_if_source_table_not_exist, false, "Allow attaching to a materialized view even if dependent tables are inaccessible.", 0) \ \ M(UInt64, postgresql_connection_pool_size, 16, "Connection pool size for PostgreSQL table engine and database engine.", 0) \ M(UInt64, postgresql_connection_pool_wait_timeout, 5000, "Connection pool push/pop timeout on empty pool for PostgreSQL table engine and database engine. By default it will block on empty pool.", 0) \ diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 4e07a6ab017..b142e4c3450 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -97,7 +97,6 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int LOGICAL_ERROR; extern const int UNKNOWN_DATABASE; - extern const int UNKNOWN_TABLE; extern const int PATH_ACCESS_DENIED; extern const int NOT_IMPLEMENTED; extern const int ENGINE_REQUIRED; @@ -1077,62 +1076,6 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data } } -void InterpreterCreateQuery::checkTypecompatibleForMaterializeView(const ASTCreateQuery & create) -{ - if (StoragePtr to_table = DatabaseCatalog::instance().tryGetTable( - {create.to_table_id.database_name, create.to_table_id.table_name, create.to_table_id.uuid}, - getContext() - )) - { - Block input_block; - try - { - if (getContext()->getSettingsRef().allow_experimental_analyzer) - { - input_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); - } - else - { - input_block = InterpreterSelectWithUnionQuery(create.select->clone(), - getContext(), - SelectQueryOptions().analyze()).getSampleBlock(); - } - } - catch (const Exception & e) - { - if (getContext()->getSettingsRef().skip_materialized_view_checking_if_source_table_not_exist && - e.code() == ErrorCodes::UNKNOWN_TABLE && create.attach - ) - { - LOG_WARNING(&Poco::Logger::get("InterpreterSelectQuery"), "{}", e.message()); - return; - } - else - { - throw; - } - } - - Block output_block = to_table->getInMemoryMetadataPtr()->getSampleBlock(); - - ColumnsWithTypeAndName input_columns; - ColumnsWithTypeAndName output_columns; - for (const auto & input_column : input_block) - { - if (const auto * output_column = output_block.findByName(input_column.name)) - { - input_columns.push_back(input_column.cloneEmpty()); - output_columns.push_back(output_column->cloneEmpty()); - } - } - - ActionsDAG::makeConvertingActions( - input_columns, - output_columns, - ActionsDAG::MatchColumnsMode::Position - ); - } -} BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) { @@ -1258,9 +1201,45 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) TableProperties properties = getTablePropertiesAndNormalizeCreateQuery(create); /// Check type compatible for materialized dest table and select columns - if (create.select && create.is_materialized_view && create.to_table_id) + if (create.select && create.is_materialized_view && create.to_table_id && !create.attach) { - checkTypecompatibleForMaterializeView(create); + if (StoragePtr to_table = DatabaseCatalog::instance().tryGetTable( + {create.to_table_id.database_name, create.to_table_id.table_name, create.to_table_id.uuid}, + getContext() + )) + { + Block input_block; + + if (getContext()->getSettingsRef().allow_experimental_analyzer) + { + input_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); + } + else + { + input_block = InterpreterSelectWithUnionQuery(create.select->clone(), + getContext(), + SelectQueryOptions().analyze()).getSampleBlock(); + } + + Block output_block = to_table->getInMemoryMetadataPtr()->getSampleBlock(); + + ColumnsWithTypeAndName input_columns; + ColumnsWithTypeAndName output_columns; + for (const auto & input_column : input_block) + { + if (const auto * output_column = output_block.findByName(input_column.name)) + { + input_columns.push_back(input_column.cloneEmpty()); + output_columns.push_back(output_column->cloneEmpty()); + } + } + + ActionsDAG::makeConvertingActions( + input_columns, + output_columns, + ActionsDAG::MatchColumnsMode::Position + ); + } } DatabasePtr database; diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index 22e71715bcf..4d11387f44c 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -95,8 +95,6 @@ private: BlockIO doCreateOrReplaceTable(ASTCreateQuery & create, const InterpreterCreateQuery::TableProperties & properties); /// Inserts data in created table if it's CREATE ... SELECT BlockIO fillTableIfNeeded(const ASTCreateQuery & create); - /// Check type compatible for materialized dest table and select columns - void checkTypecompatibleForMaterializeView(const ASTCreateQuery & create); void assertOrSetUUID(ASTCreateQuery & create, const DatabasePtr & database) const; diff --git a/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql b/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql index 615737eacf0..25e0ddf2ef6 100644 --- a/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql +++ b/tests/queries/0_stateless/02696_ignore_inacc_tables_mat_view_atttach.sql @@ -10,13 +10,7 @@ DROP TABLE test_table; DETACH TABLE mview; -/* Check that we get an exception with the option. */ - -SET skip_materialized_view_checking_if_source_table_not_exist = 0; -ATTACH TABLE mview; --{serverError 60} - /* Check that we don't get an exception with the option. */ -SET skip_materialized_view_checking_if_source_table_not_exist = 1; ATTACH TABLE mview; /* Check if the data in the materialized view is updated after the restore.*/ From 6d8a2bbd48d7d753e52b9cfcc75b03bdb9b63dae Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Mon, 27 Mar 2023 14:54:05 -0400 Subject: [PATCH 201/216] 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 `?`. ::: From c7c9d8a92adc44636eb0ba9c5b041bcfa39a8fb7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 28 Mar 2023 01:05:10 +0300 Subject: [PATCH 202/216] Update README.md --- programs/obfuscator/README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/obfuscator/README.md b/programs/obfuscator/README.md index 8c2aaafb3ac..371af6344ef 100644 --- a/programs/obfuscator/README.md +++ b/programs/obfuscator/README.md @@ -312,7 +312,7 @@ hash("images/c") % total_count: ^
 
-PhotoFunia - Haber7 - Hava mükemment.net Oynamak içinde şaşıracak haber, Oyunu Oynanılmaz • apród.hu kínálatában - RT Arabic +PhotoFunia - Haber7 - Have mükemment.net Oynamak içinde şaşıracak haber, Oyunu Oynanılmaz • apród.hu kínálatában - RT Arabic PhotoFunia - Kinobar.Net - apród: Ingyenes | Posti PhotoFunia - Peg Perfeo - Castika, Sıradışı Deniz Lokoning Your Code, sire Eminema.tv/ PhotoFunia - TUT.BY - Your Ayakkanın ve Son Dakika Spor, @@ -323,7 +323,7 @@ PhotoFunia Monstelli'nin En İyi kisa.com.tr –Star Thunder Ekranı PhotoFunia Seks - Politika,Ekonomi,Spor GTA SANAYİ VE PhotoFunia Taker-Rating Star TV Resmi Söylenen Yatağa każdy dzież wierzchnie PhotoFunia TourIndex.Marketime oyunu Oyna Geldolları Mynet Spor,Magazin,Haberler yerel Haberleri ve Solvia, korkusuz Ev SahneTv -PhotoFunia todo in the Gratis Perky Parti'nin yapıyı bu fotogram +PhotoFunia todo in the Gratis Perky Parti'nin yapıyı by fotogram PhotoFunian Dünyasın takımız halles en kulları - TEZ
From 3946e66272caf05cb162e453f41b98318dfe7966 Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Tue, 28 Mar 2023 08:11:37 +0200 Subject: [PATCH 203/216] Comestic PR changes --- src/AggregateFunctions/AggregateFunctionQuantile.h | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionQuantile.h b/src/AggregateFunctions/AggregateFunctionQuantile.h index 01ff019f221..85eaf6ceba6 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantile.h +++ b/src/AggregateFunctions/AggregateFunctionQuantile.h @@ -125,15 +125,15 @@ public: if constexpr (std::is_same_v>) { /// QuantileTiming only supports unsigned integers. Too large values are also meaningless. - #ifdef OS_DARWIN - #pragma clang diagnostic push - #pragma clang diagnostic ignored "-Wimplicit-const-int-float-conversion" - #endif +#ifdef OS_DARWIN +# pragma clang diagnostic push +# pragma clang diagnostic ignored "-Wimplicit-const-int-float-conversion" +#endif if (isNaN(value) || value > std::numeric_limits::max() || value < 0) return; - #ifdef OS_DARWIN - #pragma clang diagnostic pop - #endif +#ifdef OS_DARWIN +# pragma clang diagnostic pop +#endif } if constexpr (has_second_arg) From 943a4f75f85006a9e6c0c07132318b30d992e1c2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 28 Mar 2023 12:00:56 +0200 Subject: [PATCH 204/216] Catch all exceptions to avoid replication stuck --- .../PostgreSQL/PostgreSQLReplicationHandler.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index f9bfe1d174a..322ad3c78c0 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -417,7 +417,15 @@ void PostgreSQLReplicationHandler::consumerFunc() { assertInitialized(); - bool schedule_now = getConsumer()->consume(); + bool schedule_now = true; + try + { + schedule_now = getConsumer()->consume(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } if (stop_synchronization) { From e5994e873e8aa8d44213e49956b64704b292daec Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 28 Mar 2023 14:03:50 +0200 Subject: [PATCH 205/216] Added documentation for new settings to disallow concurrent backup/restore --- docs/en/operations/backup.md | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index d58dd1376eb..46c3a52b5c7 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -383,3 +383,19 @@ Data can be restored from backup using the `ALTER TABLE ... ATTACH PARTITION ... For more information about queries related to partition manipulations, see the [ALTER documentation](../sql-reference/statements/alter/partition.md#alter_manipulations-with-partitions). A third-party tool is available to automate this approach: [clickhouse-backup](https://github.com/AlexAkulov/clickhouse-backup). + +## Settings to disallow concurrent backup/restore + +To disallow concurrent backup/restore, you can use these settings respectively. + +```xml + + + false + false + + +``` + +The default value for both is true, so by default concurrent backup/restores are allowed. +When this setting is used on a cluster, only 1 backup/restore is allowed to run on a cluster at a time. \ No newline at end of file From aace3cb34cd2aca833fb1faa2180dd4ed974b389 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 28 Mar 2023 13:55:16 +0200 Subject: [PATCH 206/216] Try to prevent large test numbers --- utils/check-style/check-style | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 946e29856d1..7a1fa6ce123 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -165,7 +165,7 @@ find $ROOT_PATH/tests/queries -iname '*fail*' | grep . && echo 'Tests should not be named with "fail" in their names. It makes looking at the results less convenient when you search for "fail" substring in browser.' # Queries to system.query_log/system.query_thread_log should have current_database = currentDatabase() condition -# NOTE: it is not that accuate, but at least something. +# NOTE: it is not that accurate, but at least something. tests_with_query_log=( $( find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' -or -iname '*.j2' | grep -vP $EXCLUDE_DIRS | @@ -177,6 +177,17 @@ for test_case in "${tests_with_query_log[@]}"; do } || echo "Queries to system.query_log/system.query_thread_log does not have current_database = currentDatabase() condition in $test_case" done +# There shouldn't be large jumps between test numbers (since they should be consecutive) +max_diff=$( + find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' -or -iname '*.j2' | + grep -oP '\d+\D+\K\d+' | sort -n -u | awk 's{print ($0-s) " diff " s " and " $0 }{s=$0}' | sort -n | tail -n 1 +) +max_diff_value=( $(echo $max_diff) ) +if [[ $max_diff_value -ge 100 ]]; +then + echo "Too big of a difference between test numbers: $max_diff" +fi + # Queries to: tables_with_database_column=( system.tables From 07cc2a7cb25c560377437beb3e4854b4206ae93e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 28 Mar 2023 14:06:24 +0200 Subject: [PATCH 207/216] Fix tests style --- ...id_having.reference => 02701_fail_on_invalid_having.reference} | 0 ...ail_on_invalid_having.sql => 02701_fail_on_invalid_having.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{25402_fail_on_invalid_having.reference => 02701_fail_on_invalid_having.reference} (100%) rename tests/queries/0_stateless/{25402_fail_on_invalid_having.sql => 02701_fail_on_invalid_having.sql} (100%) diff --git a/tests/queries/0_stateless/25402_fail_on_invalid_having.reference b/tests/queries/0_stateless/02701_fail_on_invalid_having.reference similarity index 100% rename from tests/queries/0_stateless/25402_fail_on_invalid_having.reference rename to tests/queries/0_stateless/02701_fail_on_invalid_having.reference diff --git a/tests/queries/0_stateless/25402_fail_on_invalid_having.sql b/tests/queries/0_stateless/02701_fail_on_invalid_having.sql similarity index 100% rename from tests/queries/0_stateless/25402_fail_on_invalid_having.sql rename to tests/queries/0_stateless/02701_fail_on_invalid_having.sql From 95a1799a006e5842be08138416d9f95234ec6df3 Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Tue, 28 Mar 2023 14:34:48 +0200 Subject: [PATCH 208/216] Fix add-test for mac --- tests/queries/0_stateless/add-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/add-test b/tests/queries/0_stateless/add-test index e8e68cf174e..dbff8212020 100755 --- a/tests/queries/0_stateless/add-test +++ b/tests/queries/0_stateless/add-test @@ -10,7 +10,7 @@ TESTS_PATH=$(dirname ${BASH_SOURCE[0]}) set -ue # shellcheck disable=SC2010 -LAST_TEST_NO=$(ls -1 ${TESTS_PATH} | grep -P -o '^\d+' | sort -nr | head -1) +LAST_TEST_NO=$(ls -1 ${TESTS_PATH} | grep -E -o '^[0-9]+' | sort -nr | head -1) # remove leading zeros, increment and add padding zeros to 5 digits NEW_TEST_NO=$(printf "%05d\n" $((10#$LAST_TEST_NO + 1))) From c8ec900d905ae662a5ac6c57c8eaafa67e0a0d18 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 28 Mar 2023 15:37:49 +0200 Subject: [PATCH 209/216] Fixed comment for usage on cluster --- 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 46c3a52b5c7..000c3551fda 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -398,4 +398,4 @@ To disallow concurrent backup/restore, you can use these settings respectively. ``` The default value for both is true, so by default concurrent backup/restores are allowed. -When this setting is used on a cluster, only 1 backup/restore is allowed to run on a cluster at a time. \ No newline at end of file +When these setting are true on a cluster, only 1 backup/restore is allowed to run on a cluster at a time. \ No newline at end of file From afe5982c4ee8cae2531178b30bd3141efade81e7 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 28 Mar 2023 15:39:28 +0200 Subject: [PATCH 210/216] Fixed typo - settings --- 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 000c3551fda..9005432f1bd 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -398,4 +398,4 @@ To disallow concurrent backup/restore, you can use these settings respectively. ``` The default value for both is true, so by default concurrent backup/restores are allowed. -When these setting are true on a cluster, only 1 backup/restore is allowed to run on a cluster at a time. \ No newline at end of file +When these settings are true on a cluster, only 1 backup/restore is allowed to run on a cluster at a time. \ No newline at end of file From a26cfd523c25ae4af7ef43bb552cbcdaa6bef115 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 28 Mar 2023 15:42:21 +0200 Subject: [PATCH 211/216] Updated right bool for disallowing concurrent backups and restores --- 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 9005432f1bd..a31a52f509e 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -398,4 +398,4 @@ To disallow concurrent backup/restore, you can use these settings respectively. ``` The default value for both is true, so by default concurrent backup/restores are allowed. -When these settings are true on a cluster, only 1 backup/restore is allowed to run on a cluster at a time. \ No newline at end of file +When these settings are false on a cluster, only 1 backup/restore is allowed to run on a cluster at a time. \ No newline at end of file From 29d640aa48296c7fb5d79285d977fd45105ee525 Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Tue, 28 Mar 2023 07:11:21 -0700 Subject: [PATCH 212/216] s390x reinterpret as float64 --- src/Functions/reinterpretAs.cpp | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Functions/reinterpretAs.cpp b/src/Functions/reinterpretAs.cpp index 76afedb4f06..1481f73fbbf 100644 --- a/src/Functions/reinterpretAs.cpp +++ b/src/Functions/reinterpretAs.cpp @@ -180,9 +180,14 @@ public: size_t offset = 0; for (size_t i = 0; i < size; ++i) { - memcpy(&vec_res[i], - &data_from[offset], - std::min(static_cast(sizeof(ToFieldType)), offsets_from[i] - offset - 1)); + if constexpr (std::endian::native == std::endian::little) + memcpy(&vec_res[i], + &data_from[offset], + std::min(static_cast(sizeof(ToFieldType)), offsets_from[i] - offset - 1)); + else + reverseMemcpy(&vec_res[i], + &data_from[offset], + std::min(static_cast(sizeof(ToFieldType)), offsets_from[i] - offset - 1)); offset = offsets_from[i]; } From bc143b35ceb1206c1040eb3b1dd9a36eead229a7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 28 Mar 2023 16:44:44 +0200 Subject: [PATCH 213/216] Remove slow outdated test --- ...ition_with_different_granularity.reference | 3 -- ...e_partition_with_different_granularity.sql | 41 ------------------- 2 files changed, 44 deletions(-) delete mode 100644 tests/queries/1_stateful/00151_replace_partition_with_different_granularity.reference delete mode 100644 tests/queries/1_stateful/00151_replace_partition_with_different_granularity.sql diff --git a/tests/queries/1_stateful/00151_replace_partition_with_different_granularity.reference b/tests/queries/1_stateful/00151_replace_partition_with_different_granularity.reference deleted file mode 100644 index 31d3e6d14da..00000000000 --- a/tests/queries/1_stateful/00151_replace_partition_with_different_granularity.reference +++ /dev/null @@ -1,3 +0,0 @@ -8873898 -8873899 -8873899 diff --git a/tests/queries/1_stateful/00151_replace_partition_with_different_granularity.sql b/tests/queries/1_stateful/00151_replace_partition_with_different_granularity.sql deleted file mode 100644 index c1a2001e2a5..00000000000 --- a/tests/queries/1_stateful/00151_replace_partition_with_different_granularity.sql +++ /dev/null @@ -1,41 +0,0 @@ --- Tags: no-tsan - -DROP TABLE IF EXISTS mixed_granularity_table; - -CREATE TABLE mixed_granularity_table (`WatchID` UInt64, `JavaEnable` UInt8, `Title` String, `GoodEvent` Int16, `EventTime` DateTime, `EventDate` Date, `CounterID` UInt32, `ClientIP` UInt32, `ClientIP6` FixedString(16), `RegionID` UInt32, `UserID` UInt64, `CounterClass` Int8, `OS` UInt8, `UserAgent` UInt8, `URL` String, `Referer` String, `URLDomain` String, `RefererDomain` String, `Refresh` UInt8, `IsRobot` UInt8, `RefererCategories` Array(UInt16), `URLCategories` Array(UInt16), `URLRegions` Array(UInt32), `RefererRegions` Array(UInt32), `ResolutionWidth` UInt16, `ResolutionHeight` UInt16, `ResolutionDepth` UInt8, `FlashMajor` UInt8, `FlashMinor` UInt8, `FlashMinor2` String, `NetMajor` UInt8, `NetMinor` UInt8, `UserAgentMajor` UInt16, `UserAgentMinor` FixedString(2), `CookieEnable` UInt8, `JavascriptEnable` UInt8, `IsMobile` UInt8, `MobilePhone` UInt8, `MobilePhoneModel` String, `Params` String, `IPNetworkID` UInt32, `TraficSourceID` Int8, `SearchEngineID` UInt16, `SearchPhrase` String, `AdvEngineID` UInt8, `IsArtifical` UInt8, `WindowClientWidth` UInt16, `WindowClientHeight` UInt16, `ClientTimeZone` Int16, `ClientEventTime` DateTime, `SilverlightVersion1` UInt8, `SilverlightVersion2` UInt8, `SilverlightVersion3` UInt32, `SilverlightVersion4` UInt16, `PageCharset` String, `CodeVersion` UInt32, `IsLink` UInt8, `IsDownload` UInt8, `IsNotBounce` UInt8, `FUniqID` UInt64, `HID` UInt32, `IsOldCounter` UInt8, `IsEvent` UInt8, `IsParameter` UInt8, `DontCountHits` UInt8, `WithHash` UInt8, `HitColor` FixedString(1), `UTCEventTime` DateTime, `Age` UInt8, `Sex` UInt8, `Income` UInt8, `Interests` UInt16, `Robotness` UInt8, `GeneralInterests` Array(UInt16), `RemoteIP` UInt32, `RemoteIP6` FixedString(16), `WindowName` Int32, `OpenerName` Int32, `HistoryLength` Int16, `BrowserLanguage` FixedString(2), `BrowserCountry` FixedString(2), `SocialNetwork` String, `SocialAction` String, `HTTPError` UInt16, `SendTiming` Int32, `DNSTiming` Int32, `ConnectTiming` Int32, `ResponseStartTiming` Int32, `ResponseEndTiming` Int32, `FetchTiming` Int32, `RedirectTiming` Int32, `DOMInteractiveTiming` Int32, `DOMContentLoadedTiming` Int32, `DOMCompleteTiming` Int32, `LoadEventStartTiming` Int32, `LoadEventEndTiming` Int32, `NSToDOMContentLoadedTiming` Int32, `FirstPaintTiming` Int32, `RedirectCount` Int8, `SocialSourceNetworkID` UInt8, `SocialSourcePage` String, `ParamPrice` Int64, `ParamOrderID` String, `ParamCurrency` FixedString(3), `ParamCurrencyID` UInt16, `GoalsReached` Array(UInt32), `OpenstatServiceName` String, `OpenstatCampaignID` String, `OpenstatAdID` String, `OpenstatSourceID` String, `UTMSource` String, `UTMMedium` String, `UTMCampaign` String, `UTMContent` String, `UTMTerm` String, `FromTag` String, `HasGCLID` UInt8, `RefererHash` UInt64, `URLHash` UInt64, `CLID` UInt32, `YCLID` UInt64, `ShareService` String, `ShareURL` String, `ShareTitle` String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `IslandID` FixedString(16), `RequestNum` UInt32, `RequestTry` UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192, enable_mixed_granularity_parts=1; -- same with hits, but enabled mixed granularity - -INSERT INTO mixed_granularity_table SELECT * FROM test.hits LIMIT 10; - -ALTER TABLE mixed_granularity_table REPLACE PARTITION 201403 FROM test.hits; - -SELECT COUNT() FROM mixed_granularity_table; - -INSERT INTO mixed_granularity_table SELECT * FROM test.hits LIMIT 1; - -SELECT COUNT() FROM mixed_granularity_table; - -OPTIMIZE TABLE mixed_granularity_table FINAL; - -SELECT COUNT() FROM mixed_granularity_table; - -CREATE TABLE non_mixed_granularity_non_adaptive_table (`WatchID` UInt64, `JavaEnable` UInt8, `Title` String, `GoodEvent` Int16, `EventTime` DateTime, `EventDate` Date, `CounterID` UInt32, `ClientIP` UInt32, `ClientIP6` FixedString(16), `RegionID` UInt32, `UserID` UInt64, `CounterClass` Int8, `OS` UInt8, `UserAgent` UInt8, `URL` String, `Referer` String, `URLDomain` String, `RefererDomain` String, `Refresh` UInt8, `IsRobot` UInt8, `RefererCategories` Array(UInt16), `URLCategories` Array(UInt16), `URLRegions` Array(UInt32), `RefererRegions` Array(UInt32), `ResolutionWidth` UInt16, `ResolutionHeight` UInt16, `ResolutionDepth` UInt8, `FlashMajor` UInt8, `FlashMinor` UInt8, `FlashMinor2` String, `NetMajor` UInt8, `NetMinor` UInt8, `UserAgentMajor` UInt16, `UserAgentMinor` FixedString(2), `CookieEnable` UInt8, `JavascriptEnable` UInt8, `IsMobile` UInt8, `MobilePhone` UInt8, `MobilePhoneModel` String, `Params` String, `IPNetworkID` UInt32, `TraficSourceID` Int8, `SearchEngineID` UInt16, `SearchPhrase` String, `AdvEngineID` UInt8, `IsArtifical` UInt8, `WindowClientWidth` UInt16, `WindowClientHeight` UInt16, `ClientTimeZone` Int16, `ClientEventTime` DateTime, `SilverlightVersion1` UInt8, `SilverlightVersion2` UInt8, `SilverlightVersion3` UInt32, `SilverlightVersion4` UInt16, `PageCharset` String, `CodeVersion` UInt32, `IsLink` UInt8, `IsDownload` UInt8, `IsNotBounce` UInt8, `FUniqID` UInt64, `HID` UInt32, `IsOldCounter` UInt8, `IsEvent` UInt8, `IsParameter` UInt8, `DontCountHits` UInt8, `WithHash` UInt8, `HitColor` FixedString(1), `UTCEventTime` DateTime, `Age` UInt8, `Sex` UInt8, `Income` UInt8, `Interests` UInt16, `Robotness` UInt8, `GeneralInterests` Array(UInt16), `RemoteIP` UInt32, `RemoteIP6` FixedString(16), `WindowName` Int32, `OpenerName` Int32, `HistoryLength` Int16, `BrowserLanguage` FixedString(2), `BrowserCountry` FixedString(2), `SocialNetwork` String, `SocialAction` String, `HTTPError` UInt16, `SendTiming` Int32, `DNSTiming` Int32, `ConnectTiming` Int32, `ResponseStartTiming` Int32, `ResponseEndTiming` Int32, `FetchTiming` Int32, `RedirectTiming` Int32, `DOMInteractiveTiming` Int32, `DOMContentLoadedTiming` Int32, `DOMCompleteTiming` Int32, `LoadEventStartTiming` Int32, `LoadEventEndTiming` Int32, `NSToDOMContentLoadedTiming` Int32, `FirstPaintTiming` Int32, `RedirectCount` Int8, `SocialSourceNetworkID` UInt8, `SocialSourcePage` String, `ParamPrice` Int64, `ParamOrderID` String, `ParamCurrency` FixedString(3), `ParamCurrencyID` UInt16, `GoalsReached` Array(UInt32), `OpenstatServiceName` String, `OpenstatCampaignID` String, `OpenstatAdID` String, `OpenstatSourceID` String, `UTMSource` String, `UTMMedium` String, `UTMCampaign` String, `UTMContent` String, `UTMTerm` String, `FromTag` String, `HasGCLID` UInt8, `RefererHash` UInt64, `URLHash` UInt64, `CLID` UInt32, `YCLID` UInt64, `ShareService` String, `ShareURL` String, `ShareTitle` String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `IslandID` FixedString(16), `RequestNum` UInt32, `RequestTry` UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192, index_granularity_bytes=0; -- same with hits, but enabled mixed granularity and fixed_granularity - -INSERT INTO non_mixed_granularity_non_adaptive_table SELECT * FROM test.hits LIMIT 10; - --- after optimize mixed_granularity_table will have .mrk2 parts -ALTER TABLE non_mixed_granularity_non_adaptive_table REPLACE PARTITION 201403 FROM mixed_granularity_table; -- { serverError 36 } - -DROP TABLE IF EXISTS non_mixed_granularity_non_adaptive_table; - - -DROP TABLE IF EXISTS mixed_granularity_strictly_non_adaptive_table; - -CREATE TABLE mixed_granularity_strictly_non_adaptive_table (`WatchID` UInt64, `JavaEnable` UInt8, `Title` String, `GoodEvent` Int16, `EventTime` DateTime, `EventDate` Date, `CounterID` UInt32, `ClientIP` UInt32, `ClientIP6` FixedString(16), `RegionID` UInt32, `UserID` UInt64, `CounterClass` Int8, `OS` UInt8, `UserAgent` UInt8, `URL` String, `Referer` String, `URLDomain` String, `RefererDomain` String, `Refresh` UInt8, `IsRobot` UInt8, `RefererCategories` Array(UInt16), `URLCategories` Array(UInt16), `URLRegions` Array(UInt32), `RefererRegions` Array(UInt32), `ResolutionWidth` UInt16, `ResolutionHeight` UInt16, `ResolutionDepth` UInt8, `FlashMajor` UInt8, `FlashMinor` UInt8, `FlashMinor2` String, `NetMajor` UInt8, `NetMinor` UInt8, `UserAgentMajor` UInt16, `UserAgentMinor` FixedString(2), `CookieEnable` UInt8, `JavascriptEnable` UInt8, `IsMobile` UInt8, `MobilePhone` UInt8, `MobilePhoneModel` String, `Params` String, `IPNetworkID` UInt32, `TraficSourceID` Int8, `SearchEngineID` UInt16, `SearchPhrase` String, `AdvEngineID` UInt8, `IsArtifical` UInt8, `WindowClientWidth` UInt16, `WindowClientHeight` UInt16, `ClientTimeZone` Int16, `ClientEventTime` DateTime, `SilverlightVersion1` UInt8, `SilverlightVersion2` UInt8, `SilverlightVersion3` UInt32, `SilverlightVersion4` UInt16, `PageCharset` String, `CodeVersion` UInt32, `IsLink` UInt8, `IsDownload` UInt8, `IsNotBounce` UInt8, `FUniqID` UInt64, `HID` UInt32, `IsOldCounter` UInt8, `IsEvent` UInt8, `IsParameter` UInt8, `DontCountHits` UInt8, `WithHash` UInt8, `HitColor` FixedString(1), `UTCEventTime` DateTime, `Age` UInt8, `Sex` UInt8, `Income` UInt8, `Interests` UInt16, `Robotness` UInt8, `GeneralInterests` Array(UInt16), `RemoteIP` UInt32, `RemoteIP6` FixedString(16), `WindowName` Int32, `OpenerName` Int32, `HistoryLength` Int16, `BrowserLanguage` FixedString(2), `BrowserCountry` FixedString(2), `SocialNetwork` String, `SocialAction` String, `HTTPError` UInt16, `SendTiming` Int32, `DNSTiming` Int32, `ConnectTiming` Int32, `ResponseStartTiming` Int32, `ResponseEndTiming` Int32, `FetchTiming` Int32, `RedirectTiming` Int32, `DOMInteractiveTiming` Int32, `DOMContentLoadedTiming` Int32, `DOMCompleteTiming` Int32, `LoadEventStartTiming` Int32, `LoadEventEndTiming` Int32, `NSToDOMContentLoadedTiming` Int32, `FirstPaintTiming` Int32, `RedirectCount` Int8, `SocialSourceNetworkID` UInt8, `SocialSourcePage` String, `ParamPrice` Int64, `ParamOrderID` String, `ParamCurrency` FixedString(3), `ParamCurrencyID` UInt16, `GoalsReached` Array(UInt32), `OpenstatServiceName` String, `OpenstatCampaignID` String, `OpenstatAdID` String, `OpenstatSourceID` String, `UTMSource` String, `UTMMedium` String, `UTMCampaign` String, `UTMContent` String, `UTMTerm` String, `FromTag` String, `HasGCLID` UInt8, `RefererHash` UInt64, `URLHash` UInt64, `CLID` UInt32, `YCLID` UInt64, `ShareService` String, `ShareURL` String, `ShareTitle` String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `IslandID` FixedString(16), `RequestNum` UInt32, `RequestTry` UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192, enable_mixed_granularity_parts=1, index_granularity_bytes=0; -- same with hits, but enabled mixed granularity and fixed_granularity - -INSERT INTO mixed_granularity_strictly_non_adaptive_table SELECT * FROM test.hits LIMIT 10; - -ALTER TABLE mixed_granularity_strictly_non_adaptive_table REPLACE PARTITION 201403 FROM mixed_granularity_table; -- { serverError 36 } - -DROP TABLE IF EXISTS mixed_granularity_table; - -DROP TABLE IF EXISTS mixed_granularity_strictly_non_adaptive_table; From 0fad5bdc7d6debb2e0da7ca890449022c50d713e Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 28 Mar 2023 19:52:51 +0200 Subject: [PATCH 214/216] Add a script to clone or update clickhouse-docs --- docs/.gitignore | 1 + docs/README.md | 2 ++ docs/get-clickhouse-docs.sh | 31 +++++++++++++++++++++++++++++++ 3 files changed, 34 insertions(+) create mode 100644 docs/get-clickhouse-docs.sh diff --git a/docs/.gitignore b/docs/.gitignore index 378eac25d31..509538d9051 100644 --- a/docs/.gitignore +++ b/docs/.gitignore @@ -1 +1,2 @@ build +clickhouse-docs diff --git a/docs/README.md b/docs/README.md index 9bfd3d2b897..0cd35a4e3ec 100644 --- a/docs/README.md +++ b/docs/README.md @@ -40,6 +40,8 @@ The documentation contains information about all the aspects of the ClickHouse l At the moment, [documentation](https://clickhouse.com/docs) exists in English, Russian, and Chinese. We store the reference documentation besides the ClickHouse source code in the [GitHub repository](https://github.com/ClickHouse/ClickHouse/tree/master/docs), and user guides in a separate repo [Clickhouse/clickhouse-docs](https://github.com/ClickHouse/clickhouse-docs). +To get the latter launch the `get-clickhouse-docs.sh` script. + Each language lies in the corresponding folder. Files that are not translated from English are symbolic links to the English ones. diff --git a/docs/get-clickhouse-docs.sh b/docs/get-clickhouse-docs.sh new file mode 100644 index 00000000000..1ba0dae9844 --- /dev/null +++ b/docs/get-clickhouse-docs.sh @@ -0,0 +1,31 @@ +#!/usr/bin/env bash +set -e +# The script to clone or update the user-guides documentation repo +# https://github.com/ClickHouse/clickhouse-docs + +WORKDIR=$(dirname "$0") +WORKDIR=$(readlink -f "${WORKDIR}") +cd "$WORKDIR" + +if [ -d "clickhouse-docs" ]; then + git -C clickhouse-docs pull +else + if [ -n "$1" ]; then + url_type="$1" + else + read -rp "Enter the URL type (ssh | https): " url_type + fi + case "$url_type" in + ssh) + git_url=git@github.com:ClickHouse/clickhouse-docs.git + ;; + https) + git_url=https://github.com/ClickHouse/clickhouse-docs.git + ;; + *) + echo "Url type must be 'ssh' or 'https'" + exit 1 + ;; + esac + git clone "$git_url" "clickhouse-docs" +fi From 44d0a8075de482c72c76d3624e43882bb5408127 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 28 Mar 2023 20:38:43 +0000 Subject: [PATCH 215/216] Small follow-up to #46252 --- .../mergetree-family/mergetree.md | 50 +++++++++---------- .../00990_hasToken_and_tokenbf.sql | 13 ++--- 2 files changed, 31 insertions(+), 32 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 4f3c9c0cb69..1c16ffe9db5 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -458,34 +458,32 @@ Indexes of type `set` can be utilized by all functions. The other index types ar | Function (operator) / Index | primary key | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter | inverted | |------------------------------------------------------------------------------------------------------------|-------------|--------|------------|------------|--------------|----------| -| [equals (=, ==)](/docs/en/sql-reference/functions/comparison-functions.md/#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | -| [notEquals(!=, <>)](/docs/en/sql-reference/functions/comparison-functions.md/#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | -| [like](/docs/en/sql-reference/functions/string-search-functions.md/#function-like) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ | -| [notLike](/docs/en/sql-reference/functions/string-search-functions.md/#function-notlike) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ | -| [startsWith](/docs/en/sql-reference/functions/string-functions.md/#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ | -| [endsWith](/docs/en/sql-reference/functions/string-functions.md/#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ | ✔ | -| [multiSearchAny](/docs/en/sql-reference/functions/string-search-functions.md/#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | ✔ | -| [in](/docs/en/sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | -| [notIn](/docs/en/sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | -| [less (<)](/docs/en/sql-reference/functions/comparison-functions.md/#function-less) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | -| [greater (>)](/docs/en/sql-reference/functions/comparison-functions.md/#function-greater) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | -| [lessOrEquals (<=)](/docs/en/sql-reference/functions/comparison-functions.md/#function-lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | -| [greaterOrEquals (>=)](/docs/en/sql-reference/functions/comparison-functions.md/#function-greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | -| [empty](/docs/en/sql-reference/functions/array-functions#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | -| [notEmpty](/docs/en/sql-reference/functions/array-functions#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | -| [has](/docs/en/sql-reference/functions/array-functions#function-has) | ✗ | ✗ | ✔ | ✔ | ✔ | ✔ | -| [hasAny](/docs/en/sql-reference/functions/array-functions#function-hasAny) | ✗ | ✗ | ✗ | ✗ | ✔ | ✗ | -| [hasAll](/docs/en/sql-reference/functions/array-functions#function-hasAll) | ✗ | ✗ | ✗ | ✗ | ✔ | ✗ | -| hasToken | ✗ | ✗ | ✗ | ✔ | ✗ | ✔ | -| hasTokenOrNull | ✗ | ✗ | ✗ | ✔ | ✗ | ✔ | -| hasTokenCaseInsensitive (*) | ✗ | ✗ | ✗ | ✔ | ✗ | ✗ | -| hasTokenCaseInsensitiveOrNull (*) | ✗ | ✗ | ✗ | ✔ | ✗ | ✗ | +| [equals (=, ==)](/docs/en/sql-reference/functions/comparison-functions.md/#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | +| [notEquals(!=, <>)](/docs/en/sql-reference/functions/comparison-functions.md/#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | +| [like](/docs/en/sql-reference/functions/string-search-functions.md/#function-like) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ | +| [notLike](/docs/en/sql-reference/functions/string-search-functions.md/#function-notlike) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ | +| [startsWith](/docs/en/sql-reference/functions/string-functions.md/#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ | +| [endsWith](/docs/en/sql-reference/functions/string-functions.md/#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ | ✔ | +| [multiSearchAny](/docs/en/sql-reference/functions/string-search-functions.md/#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | ✔ | +| [in](/docs/en/sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | +| [notIn](/docs/en/sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | +| [less (<)](/docs/en/sql-reference/functions/comparison-functions.md/#function-less) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | +| [greater (>)](/docs/en/sql-reference/functions/comparison-functions.md/#function-greater) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | +| [lessOrEquals (<=)](/docs/en/sql-reference/functions/comparison-functions.md/#function-lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | +| [greaterOrEquals (>=)](/docs/en/sql-reference/functions/comparison-functions.md/#function-greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | +| [empty](/docs/en/sql-reference/functions/array-functions#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | +| [notEmpty](/docs/en/sql-reference/functions/array-functions#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | +| [has](/docs/en/sql-reference/functions/array-functions#function-has) | ✗ | ✗ | ✔ | ✔ | ✔ | ✔ | +| [hasAny](/docs/en/sql-reference/functions/array-functions#function-hasAny) | ✗ | ✗ | ✗ | ✗ | ✔ | ✗ | +| [hasAll](/docs/en/sql-reference/functions/array-functions#function-hasAll) | ✗ | ✗ | ✗ | ✗ | ✔ | ✗ | +| hasToken | ✗ | ✗ | ✗ | ✔ | ✗ | ✔ | +| hasTokenOrNull | ✗ | ✗ | ✗ | ✔ | ✗ | ✔ | +| hasTokenCaseInsensitive (*) | ✗ | ✗ | ✗ | ✔ | ✗ | ✗ | +| hasTokenCaseInsensitiveOrNull (*) | ✗ | ✗ | ✗ | ✔ | ✗ | ✗ | Functions with a constant argument that is less than ngram size can’t be used by `ngrambf_v1` for query optimization. -(*) For `hasTokenCaseInsensitve` and `hasTokenCaseInsensitive` to be effective, the data skipping index of type `tokenbf_v1` must be created on lowercased data, for example: -``` -CREATE TABLE tab (id UInt64, s String, INDEX tok_bf_idx (s, lower(s)) TYPE tokenbf_v1(512, 3, 0) GRANULARITY 1) ... . ) ENGINE = MergeTree() -``` + +(*) For `hasTokenCaseInsensitve` and `hasTokenCaseInsensitive` to be effective, the `tokenbf_v1` index must be created on lowercased data, for example `INDEX idx (lower(str_col)) TYPE tokenbf_v1(512, 3, 0)`. :::note Bloom filters can have false positive matches, so the `ngrambf_v1`, `tokenbf_v1`, and `bloom_filter` indexes can not be used for optimizing queries where the result of a function is expected to be false. diff --git a/tests/queries/0_stateless/00990_hasToken_and_tokenbf.sql b/tests/queries/0_stateless/00990_hasToken_and_tokenbf.sql index 1361ce0f3c3..8e88af40046 100644 --- a/tests/queries/0_stateless/00990_hasToken_and_tokenbf.sql +++ b/tests/queries/0_stateless/00990_hasToken_and_tokenbf.sql @@ -12,6 +12,13 @@ insert into bloom_filter select number+2000, 'abc,def,zzz' from numbers(8); insert into bloom_filter select number+3000, 'yyy,uuu' from numbers(1024); insert into bloom_filter select number+3000, 'abcdefzzz' from numbers(1024); +SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'abc,def,zzz'); -- { serverError BAD_ARGUMENTS } +SELECT max(id) FROM bloom_filter WHERE hasTokenCaseInsensitive(s, 'abc,def,zzz'); -- { serverError BAD_ARGUMENTS } + +SELECT max(id) FROM bloom_filter WHERE hasTokenOrNull(s, 'abc,def,zzz'); +SELECT max(id) FROM bloom_filter WHERE hasTokenCaseInsensitiveOrNull(s, 'abc,def,zzz'); + +-- as table "bloom_filter" but w/o index_granularity_bytes drop table if exists bloom_filter2; create table bloom_filter2 ( @@ -25,12 +32,6 @@ insert into bloom_filter2 select number+2000, 'ABC,def,zzz' from numbers(8); insert into bloom_filter2 select number+3000, 'yyy,uuu' from numbers(1024); insert into bloom_filter2 select number+3000, 'abcdefzzz' from numbers(1024); -SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'abc,def,zzz'); -- { serverError BAD_ARGUMENTS } -SELECT max(id) FROM bloom_filter WHERE hasTokenCaseInsensitive(s, 'abc,def,zzz'); -- { serverError BAD_ARGUMENTS } - -SELECT max(id) FROM bloom_filter WHERE hasTokenOrNull(s, 'abc,def,zzz'); -SELECT max(id) FROM bloom_filter WHERE hasTokenCaseInsensitiveOrNull(s, 'abc,def,zzz'); - set max_rows_to_read = 16; SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'abc'); From b8edcd90ccfee98411248c442baf3589cf8a85dd Mon Sep 17 00:00:00 2001 From: rfraposa Date: Tue, 28 Mar 2023 15:49:58 -0600 Subject: [PATCH 216/216] Update stack_trace.md --- docs/en/operations/system-tables/stack_trace.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/operations/system-tables/stack_trace.md b/docs/en/operations/system-tables/stack_trace.md index c64cf067220..d6963160399 100644 --- a/docs/en/operations/system-tables/stack_trace.md +++ b/docs/en/operations/system-tables/stack_trace.md @@ -14,6 +14,10 @@ Columns: - `query_id` ([String](../../sql-reference/data-types/string.md)) — Query identifier that can be used to get details about a query that was running from the [query_log](../system-tables/query_log.md) system table. - `trace` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — A [stack trace](https://en.wikipedia.org/wiki/Stack_trace) which represents a list of physical addresses where the called methods are stored. +:::tip +Check out the Knowledge Base for some handy queries, including [how to see what threads are currently running](https://clickhouse.com/docs/knowledgebase/find-expensive-queries) and [useful queries for troubleshooting](https://clickhouse.com/docs/knowledgebase/useful-queries-for-troubleshooting). +::: + **Example** Enabling introspection functions: