From dd5435501a510f591b0742c0a18426c25f0ecbac Mon Sep 17 00:00:00 2001 From: Javi santana bot Date: Fri, 20 Dec 2019 11:00:21 +0100 Subject: [PATCH 01/32] check for active replicas when waiting for commands --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index b3f69d463f2..88c731fb88f 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3844,9 +3844,15 @@ void StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry(const Repli { LOG_DEBUG(log, "Waiting for all replicas to process " << entry.znode_name); - Strings replicas = getZooKeeper()->getChildren(zookeeper_path + "/replicas"); + auto zookeeper = getZooKeeper() + Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); for (const String & replica : replicas) - waitForReplicaToProcessLogEntry(replica, entry); + { + if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")) + { + waitForReplicaToProcessLogEntry(replica, entry); + } + } LOG_DEBUG(log, "Finished waiting for all replicas to process " << entry.znode_name); } From 93643975970c05a6c2b53cbf9cb7af5bc0cafea3 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 18 Dec 2019 11:56:03 +0800 Subject: [PATCH 02/32] try fix some predicate optimizer bug --- .../Interpreters/DatabaseAndTableWithAlias.h | 2 + dbms/src/Interpreters/ExpressionAnalyzer.cpp | 1 - .../ExtractExpressionInfoVisitor.cpp | 59 ++ .../ExtractExpressionInfoVisitor.h | 39 ++ .../ExtractFunctionDataVisitor.cpp | 16 - .../Interpreters/ExtractFunctionDataVisitor.h | 25 - .../FindIdentifierBestTableVisitor.cpp | 40 -- .../FindIdentifierBestTableVisitor.h | 27 - dbms/src/Interpreters/InDepthNodeVisitor.h | 8 +- .../PredicateExpressionsOptimizer.cpp | 512 +++--------------- .../PredicateExpressionsOptimizer.h | 82 +-- .../Interpreters/PredicateRewriteVisitor.cpp | 141 +++++ .../Interpreters/PredicateRewriteVisitor.h | 41 ++ dbms/src/Interpreters/SyntaxAnalyzer.cpp | 2 +- dbms/src/Storages/StorageView.cpp | 21 +- 15 files changed, 405 insertions(+), 611 deletions(-) create mode 100644 dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp create mode 100644 dbms/src/Interpreters/ExtractExpressionInfoVisitor.h delete mode 100644 dbms/src/Interpreters/ExtractFunctionDataVisitor.cpp delete mode 100644 dbms/src/Interpreters/ExtractFunctionDataVisitor.h delete mode 100644 dbms/src/Interpreters/FindIdentifierBestTableVisitor.cpp delete mode 100644 dbms/src/Interpreters/FindIdentifierBestTableVisitor.h create mode 100644 dbms/src/Interpreters/PredicateRewriteVisitor.cpp create mode 100644 dbms/src/Interpreters/PredicateRewriteVisitor.h diff --git a/dbms/src/Interpreters/DatabaseAndTableWithAlias.h b/dbms/src/Interpreters/DatabaseAndTableWithAlias.h index 3567a351b14..ad1f747b6fd 100644 --- a/dbms/src/Interpreters/DatabaseAndTableWithAlias.h +++ b/dbms/src/Interpreters/DatabaseAndTableWithAlias.h @@ -72,4 +72,6 @@ private: std::vector getDatabaseAndTables(const ASTSelectQuery & select_query, const String & current_database); std::optional getDatabaseAndTable(const ASTSelectQuery & select, size_t table_number); +using TablesWithColumnNames = std::vector; + } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 59dff858cf0..fa199f94efc 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -26,7 +26,6 @@ #include #include #include -#include #include #include #include diff --git a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp new file mode 100644 index 00000000000..4bd0446cd97 --- /dev/null +++ b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp @@ -0,0 +1,59 @@ +#include +#include +#include +#include + + +namespace DB +{ + +void ExpressionInfoMatcher::visit(const ASTPtr & ast, Data & data) +{ + if (const auto * function = ast->as()) + visit(*function, ast, data); + else if (const auto * identifier = ast->as()) + visit(*identifier, ast, data); +} + +void ExpressionInfoMatcher::visit(const ASTFunction & ast_function, const ASTPtr &, Data & data) +{ + if (ast_function.name == "arrayJoin") + data.is_array_join = true; + else if (AggregateFunctionFactory::instance().isAggregateFunctionName(ast_function.name)) + data.is_aggregate_function = true; + else + { + const auto & function = FunctionFactory::instance().tryGet(ast_function.name, data.context); + + /// Skip lambda, tuple and other special functions + if (function && function->isStateful()) + data.is_stateful_function = true; + } +} + +void ExpressionInfoMatcher::visit(const ASTIdentifier & identifier, const ASTPtr &, Data & data) +{ + if (!identifier.compound()) + { + for (size_t index = 0; index < data.tables.size(); ++index) + { + const auto & columns = data.tables[index].columns; + + // TODO: make sure no collision ever happens + if (std::find(columns.begin(), columns.end(), identifier.name) != columns.end()) + { + data.unique_reference_tables_pos.emplace(index); + break; + } + } + } + else + { + size_t best_table_pos = 0; + if (IdentifierSemantic::chooseTable(identifier, data.tables, best_table_pos)) + data.unique_reference_tables_pos.emplace(best_table_pos); + } +} + +} + diff --git a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.h b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.h new file mode 100644 index 00000000000..6ef9960b918 --- /dev/null +++ b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.h @@ -0,0 +1,39 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +class Context; + +struct ExpressionInfoMatcher +{ + struct Data + { + const Context & context; + const std::vector & tables; + + bool is_array_join = false; + bool is_stateful_function = false; + bool is_aggregate_function = false; + std::unordered_set unique_reference_tables_pos; + }; + + static void visit(const ASTPtr & ast, Data & data); + + static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } + + static void visit(const ASTFunction & ast_function, const ASTPtr &, Data & data); + + static void visit(const ASTIdentifier & identifier, const ASTPtr &, Data & data); +}; + + +using ExpressionInfoVisitor = ConstInDepthNodeVisitor; + +} diff --git a/dbms/src/Interpreters/ExtractFunctionDataVisitor.cpp b/dbms/src/Interpreters/ExtractFunctionDataVisitor.cpp deleted file mode 100644 index d7a0d9001d5..00000000000 --- a/dbms/src/Interpreters/ExtractFunctionDataVisitor.cpp +++ /dev/null @@ -1,16 +0,0 @@ -#include -#include - - -namespace DB -{ - -void ExtractFunctionData::visit(ASTFunction & function, ASTPtr &) -{ - if (AggregateFunctionFactory::instance().isAggregateFunctionName(function.name)) - aggregate_functions.emplace_back(&function); - else - functions.emplace_back(&function); -} - -} diff --git a/dbms/src/Interpreters/ExtractFunctionDataVisitor.h b/dbms/src/Interpreters/ExtractFunctionDataVisitor.h deleted file mode 100644 index ed3dbb868c4..00000000000 --- a/dbms/src/Interpreters/ExtractFunctionDataVisitor.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include - -namespace DB -{ - -struct ExtractFunctionData -{ - using TypeToVisit = ASTFunction; - - std::vector functions; - std::vector aggregate_functions; - - void visit(ASTFunction & identifier, ASTPtr &); -}; - -using ExtractFunctionMatcher = OneTypeMatcher; -using ExtractFunctionVisitor = InDepthNodeVisitor; - -} diff --git a/dbms/src/Interpreters/FindIdentifierBestTableVisitor.cpp b/dbms/src/Interpreters/FindIdentifierBestTableVisitor.cpp deleted file mode 100644 index 56897ec15c7..00000000000 --- a/dbms/src/Interpreters/FindIdentifierBestTableVisitor.cpp +++ /dev/null @@ -1,40 +0,0 @@ -#include -#include - - -namespace DB -{ - -FindIdentifierBestTableData::FindIdentifierBestTableData(const std::vector & tables_) - : tables(tables_) -{ -} - -void FindIdentifierBestTableData::visit(ASTIdentifier & identifier, ASTPtr &) -{ - const DatabaseAndTableWithAlias * best_table = nullptr; - - if (!identifier.compound()) - { - for (const auto & table_names : tables) - { - auto & columns = table_names.columns; - if (std::find(columns.begin(), columns.end(), identifier.name) != columns.end()) - { - // TODO: make sure no collision ever happens - if (!best_table) - best_table = &table_names.table; - } - } - } - else - { - size_t best_table_pos = 0; - if (IdentifierSemantic::chooseTable(identifier, tables, best_table_pos)) - best_table = &tables[best_table_pos].table; - } - - identifier_table.emplace_back(&identifier, best_table); -} - -} diff --git a/dbms/src/Interpreters/FindIdentifierBestTableVisitor.h b/dbms/src/Interpreters/FindIdentifierBestTableVisitor.h deleted file mode 100644 index 498ee60ab0b..00000000000 --- a/dbms/src/Interpreters/FindIdentifierBestTableVisitor.h +++ /dev/null @@ -1,27 +0,0 @@ -#pragma once - -#include -#include -#include -#include - -namespace DB -{ - -struct FindIdentifierBestTableData -{ - using TypeToVisit = ASTIdentifier; - using IdentifierWithTable = std::pair; - - const std::vector & tables; - std::vector identifier_table; - - FindIdentifierBestTableData(const std::vector & tables_); - - void visit(ASTIdentifier & identifier, ASTPtr &); -}; - -using FindIdentifierBestTableMatcher = OneTypeMatcher; -using FindIdentifierBestTableVisitor = InDepthNodeVisitor; - -} diff --git a/dbms/src/Interpreters/InDepthNodeVisitor.h b/dbms/src/Interpreters/InDepthNodeVisitor.h index 18b84b11b24..7bb4f5e4d54 100644 --- a/dbms/src/Interpreters/InDepthNodeVisitor.h +++ b/dbms/src/Interpreters/InDepthNodeVisitor.h @@ -59,7 +59,13 @@ public: using Data = Data_; using TypeToVisit = typename Data::TypeToVisit; - static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return visit_children; } + static bool needChildVisit(const ASTPtr & node, const ASTPtr &) + { + if (node && node->as()) + return visit_children; + + return true; + } static void visit(T & ast, Data & data) { diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp index 050ee637d18..a5cc578a98a 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -2,9 +2,9 @@ #include #include -#include -#include #include +#include +#include #include #include #include @@ -13,9 +13,6 @@ #include #include #include -#include -#include -#include #include #include #include @@ -23,9 +20,9 @@ #include #include #include -#include -#include +#include #include +#include #include @@ -38,155 +35,99 @@ namespace ErrorCodes extern const int UNKNOWN_ELEMENT_IN_AST; } -namespace -{ - -constexpr auto and_function_name = "and"; - -String qualifiedName(ASTIdentifier * identifier, const String & prefix) -{ - if (identifier->isShort()) - return prefix + identifier->getAliasOrColumnName(); - return identifier->getAliasOrColumnName(); -} - -} - PredicateExpressionsOptimizer::PredicateExpressionsOptimizer( - ASTSelectQuery * ast_select_, ExtractedSettings && settings_, const Context & context_) - : ast_select(ast_select_), settings(settings_), context(context_) + const Context & context_, const TablesWithColumnNames & tables_with_columns_, const Settings & settings_) + : context(context_), tables_with_columns(tables_with_columns_), settings(settings_) { } -bool PredicateExpressionsOptimizer::optimize() +bool PredicateExpressionsOptimizer::optimize(ASTSelectQuery & select_query) { - if (!settings.enable_optimize_predicate_expression || !ast_select || !ast_select->tables() || ast_select->tables()->children.empty()) + if (!settings.enable_optimize_predicate_expression || !select_query.tables() || select_query.tables()->children.empty()) return false; - if (!ast_select->where() && !ast_select->prewhere()) + if ((!select_query.where() && !select_query.prewhere()) || select_query.array_join_expression_list()) return false; - if (ast_select->array_join_expression_list()) - return false; + const auto & tables_predicates = extractTablesPredicates(select_query.where(), select_query.prewhere()); - SubqueriesProjectionColumns all_subquery_projection_columns = getAllSubqueryProjectionColumns(); + if (!tables_predicates.empty()) + return tryRewritePredicatesToTables(select_query.refTables()->children, tables_predicates); - bool is_rewrite_subqueries = false; - if (!all_subquery_projection_columns.empty()) - { - is_rewrite_subqueries |= optimizeImpl(ast_select->where(), all_subquery_projection_columns, OptimizeKind::PUSH_TO_WHERE); - is_rewrite_subqueries |= optimizeImpl(ast_select->prewhere(), all_subquery_projection_columns, OptimizeKind::PUSH_TO_PREWHERE); - } - - return is_rewrite_subqueries; + return false; } -bool PredicateExpressionsOptimizer::optimizeImpl( - const ASTPtr & outer_expression, const SubqueriesProjectionColumns & subqueries_projection_columns, OptimizeKind expression_kind) +static ASTs splitConjunctionPredicate(const std::initializer_list & predicates) { - /// split predicate with `and` - std::vector outer_predicate_expressions = splitConjunctionPredicate(outer_expression); + std::vector res; - std::vector table_expressions = getTableExpressions(*ast_select); - std::vector tables_with_columns = getDatabaseAndTablesWithColumnNames(table_expressions, context); - - bool is_rewrite_subquery = false; - for (auto & outer_predicate : outer_predicate_expressions) + auto remove_expression_at_index = [&res] (const size_t index) { - if (isArrayJoinFunction(outer_predicate)) + if (index < res.size() - 1) + std::swap(res[index], res.back()); + res.pop_back(); + }; + + for (const auto & predicate : predicates) + { + if (!predicate) continue; - auto outer_predicate_dependencies = getDependenciesAndQualifiers(outer_predicate, tables_with_columns); + res.emplace_back(predicate); - /// TODO: remove origin expression - for (const auto & [subquery, projection_columns] : subqueries_projection_columns) + for (size_t idx = 0; idx < res.size();) { - OptimizeKind optimize_kind = OptimizeKind::NONE; - if (allowPushDown(subquery, outer_predicate, projection_columns, outer_predicate_dependencies, optimize_kind)) + const auto & expression = res.at(idx); + + if (const auto * function = expression->as(); function && function->name == "and") { - if (optimize_kind == OptimizeKind::NONE) - optimize_kind = expression_kind; + for (auto & child : function->arguments->children) + res.emplace_back(child); - ASTPtr inner_predicate = outer_predicate->clone(); - cleanExpressionAlias(inner_predicate); /// clears the alias name contained in the outer predicate + remove_expression_at_index(idx); + continue; + } + ++idx; + } + } - std::vector inner_predicate_dependencies = - getDependenciesAndQualifiers(inner_predicate, tables_with_columns); + return res; +} - setNewAliasesForInnerPredicate(projection_columns, inner_predicate_dependencies); +std::vector PredicateExpressionsOptimizer::extractTablesPredicates(const ASTPtr & where, const ASTPtr & prewhere) +{ + std::vector tables_predicates(tables_with_columns.size()); - switch (optimize_kind) - { - case OptimizeKind::NONE: continue; - case OptimizeKind::PUSH_TO_WHERE: - is_rewrite_subquery |= optimizeExpression(inner_predicate, subquery, ASTSelectQuery::Expression::WHERE); - continue; - case OptimizeKind::PUSH_TO_HAVING: - is_rewrite_subquery |= optimizeExpression(inner_predicate, subquery, ASTSelectQuery::Expression::HAVING); - continue; - case OptimizeKind::PUSH_TO_PREWHERE: - is_rewrite_subquery |= optimizeExpression(inner_predicate, subquery, ASTSelectQuery::Expression::PREWHERE); - continue; - } + for (const auto & predicate_expression : splitConjunctionPredicate({where, prewhere})) + { + ExpressionInfoVisitor::Data expression_info{.context = context, .tables = tables_with_columns}; + ExpressionInfoVisitor(expression_info).visit(predicate_expression); + + if (expression_info.is_stateful_function) + return {}; /// give up the optimization when the predicate contains stateful function + + if (!expression_info.is_array_join) + { + if (expression_info.unique_reference_tables_pos.size() == 1) + tables_predicates[*expression_info.unique_reference_tables_pos.begin()].emplace_back(predicate_expression); + else if (expression_info.unique_reference_tables_pos.size() == 0) + { + for (size_t index = 0; index < tables_predicates.size(); ++index) + tables_predicates[index].emplace_back(predicate_expression); } } } - return is_rewrite_subquery; + + return tables_predicates; /// everything is OK, it can be optimized } -bool PredicateExpressionsOptimizer::allowPushDown( - const ASTSelectQuery * subquery, - const ASTPtr &, - const std::vector & projection_columns, - const std::vector & dependencies, - OptimizeKind & optimize_kind) +bool PredicateExpressionsOptimizer::tryRewritePredicatesToTables(ASTs & tables_element, const std::vector & tables_predicates) { - if (!subquery - || (!settings.enable_optimize_predicate_expression_to_final_subquery && subquery->final()) - || subquery->limitBy() || subquery->limitLength() - || subquery->with() || subquery->withFill()) - return false; - else + bool is_rewrite_tables = false; + + for (size_t index = tables_element.size(); index > 0; --index) { - ASTPtr expr_list = ast_select->select(); - ExtractFunctionVisitor::Data extract_data; - ExtractFunctionVisitor(extract_data).visit(expr_list); - - for (const auto & subquery_function : extract_data.functions) - { - const auto & function = FunctionFactory::instance().tryGet(subquery_function->name, context); - - /// Skip lambda, tuple and other special functions - if (function && function->isStateful()) - return false; - } - } - - const auto * ast_join = ast_select->join(); - const ASTTableExpression * left_table_expr = nullptr; - const ASTTableExpression * right_table_expr = nullptr; - const ASTSelectQuery * left_subquery = nullptr; - const ASTSelectQuery * right_subquery = nullptr; - - if (ast_join) - { - left_table_expr = ast_select - ->tables()->as() - ->children[0]->as() - ->table_expression->as(); - right_table_expr = ast_select - ->tables()->as() - ->children[1]->as() - ->table_expression->as(); - - if (left_table_expr && left_table_expr->subquery) - left_subquery = left_table_expr->subquery - ->children[0]->as() - ->list_of_selects->children[0]->as(); - if (right_table_expr && right_table_expr->subquery) - right_subquery = right_table_expr->subquery - ->children[0]->as() - ->list_of_selects->children[0]->as(); + size_t table_pos = index - 1; /// NOTE: the syntactic way of pushdown has limitations and should be partially disabled in case of JOINs. /// Let's take a look at the query: @@ -201,326 +142,37 @@ bool PredicateExpressionsOptimizer::allowPushDown( /// It happens because the not-matching columns are replaced with a global default values on JOIN. /// Same is true for RIGHT JOIN and FULL JOIN. - /// Check right side for LEFT'o'FULL JOIN - if (isLeftOrFull(ast_join->table_join->as()->kind) && right_subquery == subquery) - return false; - - /// Check left side for RIGHT'o'FULL JOIN - if (isRightOrFull(ast_join->table_join->as()->kind) && left_subquery == subquery) - return false; - } - - return checkDependencies(projection_columns, dependencies, optimize_kind); -} - -bool PredicateExpressionsOptimizer::checkDependencies( - const std::vector & projection_columns, - const std::vector & dependencies, - OptimizeKind & optimize_kind) -{ - for (const auto & [identifier, prefix] : dependencies) - { - bool is_found = false; - String qualified_name = qualifiedName(identifier, prefix); - - for (const auto & [ast, alias] : projection_columns) + if (const auto & table_element = tables_element[table_pos]->as()) { - if (alias == qualified_name) - { - is_found = true; - ASTPtr projection_column = ast; - ExtractFunctionVisitor::Data extract_data; - ExtractFunctionVisitor(extract_data).visit(projection_column); + if (table_element->table_join && isLeft(table_element->table_join->as()->kind)) + continue; /// Skip right table optimization - if (!extract_data.aggregate_functions.empty()) - optimize_kind = OptimizeKind::PUSH_TO_HAVING; - } - } + if (table_element->table_join && isFull(table_element->table_join->as()->kind)) + break; /// Skip left and right table optimization - if (!is_found) - return false; - } + is_rewrite_tables |= tryRewritePredicatesToTable(tables_element[table_pos], tables_predicates[table_pos], + tables_with_columns[table_pos].columns); - return true; -} - -std::vector PredicateExpressionsOptimizer::splitConjunctionPredicate(const ASTPtr & predicate_expression) -{ - std::vector predicate_expressions; - - if (predicate_expression) - { - predicate_expressions.emplace_back(predicate_expression); - - auto remove_expression_at_index = [&predicate_expressions] (const size_t index) - { - if (index < predicate_expressions.size() - 1) - std::swap(predicate_expressions[index], predicate_expressions.back()); - predicate_expressions.pop_back(); - }; - - for (size_t idx = 0; idx < predicate_expressions.size();) - { - const auto expression = predicate_expressions.at(idx); - - if (const auto * function = expression->as()) - { - if (function->name == and_function_name) - { - for (auto & child : function->arguments->children) - predicate_expressions.emplace_back(child); - - remove_expression_at_index(idx); - continue; - } - } - ++idx; + if (table_element->table_join && isRight(table_element->table_join->as()->kind)) + break; /// Skip left table optimization } } - return predicate_expressions; + + return is_rewrite_tables; } -std::vector -PredicateExpressionsOptimizer::getDependenciesAndQualifiers(ASTPtr & expression, std::vector & tables) +bool PredicateExpressionsOptimizer::tryRewritePredicatesToTable(ASTPtr & table_element, const ASTs & table_predicates, const Names & table_column) const { - FindIdentifierBestTableVisitor::Data find_data(tables); - FindIdentifierBestTableVisitor(find_data).visit(expression); - - std::vector dependencies; - - for (const auto & [identifier, table] : find_data.identifier_table) + if (!table_predicates.empty()) { - String table_alias; - if (table) - table_alias = table->getQualifiedNamePrefix(); + PredicateRewriteVisitor::Data data( + context, table_predicates, table_column, settings.enable_optimize_predicate_expression_to_final_subquery); - dependencies.emplace_back(identifier, table_alias); + PredicateRewriteVisitor(data).visit(table_element); + return data.isRewrite(); } - return dependencies; -} - -void PredicateExpressionsOptimizer::setNewAliasesForInnerPredicate( - const std::vector & projection_columns, - const std::vector & dependencies) -{ - for (auto & [identifier, prefix] : dependencies) - { - String qualified_name = qualifiedName(identifier, prefix); - - for (auto & [ast, alias] : projection_columns) - { - if (alias == qualified_name) - { - String name; - if (auto * id = ast->as()) - { - name = id->tryGetAlias(); - if (name.empty()) - name = id->shortName(); - } - else - { - if (ast->tryGetAlias().empty()) - ast->setAlias(ast->getColumnName()); - name = ast->getAliasOrColumnName(); - } - - identifier->setShortName(name); - } - } - } -} - -bool PredicateExpressionsOptimizer::isArrayJoinFunction(const ASTPtr & node) -{ - if (const auto * function = node->as()) - { - if (function->name == "arrayJoin") - return true; - } - - for (auto & child : node->children) - if (isArrayJoinFunction(child)) - return true; - return false; } -bool PredicateExpressionsOptimizer::optimizeExpression(const ASTPtr & outer_expression, ASTSelectQuery * subquery, - ASTSelectQuery::Expression expr) -{ - ASTPtr subquery_expression = subquery->getExpression(expr, false); - subquery_expression = subquery_expression ? makeASTFunction(and_function_name, outer_expression, subquery_expression) : outer_expression; - - subquery->setExpression(expr, std::move(subquery_expression)); - return true; -} - -PredicateExpressionsOptimizer::SubqueriesProjectionColumns PredicateExpressionsOptimizer::getAllSubqueryProjectionColumns() -{ - SubqueriesProjectionColumns projection_columns; - - for (const auto & table_expression : getTableExpressions(*ast_select)) - if (table_expression->subquery) - getSubqueryProjectionColumns(table_expression->subquery, projection_columns); - - return projection_columns; -} - -void PredicateExpressionsOptimizer::getSubqueryProjectionColumns(const ASTPtr & subquery, SubqueriesProjectionColumns & projection_columns) -{ - String qualified_name_prefix = subquery->tryGetAlias(); - if (!qualified_name_prefix.empty()) - qualified_name_prefix += '.'; - - const ASTPtr & subselect = subquery->children[0]; - - ASTs select_with_union_projections; - const auto * select_with_union_query = subselect->as(); - - for (auto & select : select_with_union_query->list_of_selects->children) - { - std::vector subquery_projections; - auto select_projection_columns = getSelectQueryProjectionColumns(select); - - if (!select_projection_columns.empty()) - { - if (select_with_union_projections.empty()) - select_with_union_projections = select_projection_columns; - - for (size_t i = 0; i < select_projection_columns.size(); i++) - subquery_projections.emplace_back(std::pair(select_projection_columns[i], - qualified_name_prefix + select_with_union_projections[i]->getAliasOrColumnName())); - - projection_columns.insert(std::pair(select->as(), subquery_projections)); - } - } -} - -ASTs PredicateExpressionsOptimizer::getSelectQueryProjectionColumns(ASTPtr & ast) -{ - ASTs projection_columns; - auto * select_query = ast->as(); - - /// first should normalize query tree. - std::unordered_map aliases; - std::vector tables = getDatabaseAndTables(*select_query, context.getCurrentDatabase()); - - /// TODO: get tables from evaluateAsterisk instead of tablesOnly() to extract asterisks in general way - std::vector tables_with_columns = TranslateQualifiedNamesVisitor::Data::tablesOnly(tables); - TranslateQualifiedNamesVisitor::Data qn_visitor_data({}, std::move(tables_with_columns), false); - TranslateQualifiedNamesVisitor(qn_visitor_data).visit(ast); - - QueryAliasesVisitor::Data query_aliases_data{aliases}; - QueryAliasesVisitor(query_aliases_data).visit(ast); - - MarkTableIdentifiersVisitor::Data mark_tables_data{aliases}; - MarkTableIdentifiersVisitor(mark_tables_data).visit(ast); - - QueryNormalizer::Data normalizer_data(aliases, settings); - QueryNormalizer(normalizer_data).visit(ast); - - for (const auto & projection_column : select_query->select()->children) - { - if (projection_column->as() || projection_column->as() || projection_column->as()) - { - ASTs evaluated_columns = evaluateAsterisk(select_query, projection_column); - - for (const auto & column : evaluated_columns) - projection_columns.emplace_back(column); - - continue; - } - - projection_columns.emplace_back(projection_column); - } - return projection_columns; -} - -ASTs PredicateExpressionsOptimizer::evaluateAsterisk(ASTSelectQuery * select_query, const ASTPtr & asterisk) -{ - /// SELECT *, SELECT dummy, SELECT 1 AS id - if (!select_query->tables() || select_query->tables()->children.empty()) - return {}; - - std::vector tables_expression = getTableExpressions(*select_query); - - if (const auto * qualified_asterisk = asterisk->as()) - { - if (qualified_asterisk->children.size() != 1) - throw Exception("Logical error: qualified asterisk must have exactly one child", ErrorCodes::LOGICAL_ERROR); - - DatabaseAndTableWithAlias ident_db_and_name(qualified_asterisk->children[0]); - - for (auto it = tables_expression.begin(); it != tables_expression.end();) - { - const ASTTableExpression * table_expression = *it; - DatabaseAndTableWithAlias database_and_table_with_alias(*table_expression, context.getCurrentDatabase()); - - if (ident_db_and_name.satisfies(database_and_table_with_alias, true)) - ++it; - else - it = tables_expression.erase(it); /// It's not a required table - } - } - - ASTs projection_columns; - for (auto & table_expression : tables_expression) - { - if (table_expression->subquery) - { - const auto * subquery = table_expression->subquery->as(); - const auto * select_with_union_query = subquery->children[0]->as(); - const auto subquery_projections = getSelectQueryProjectionColumns(select_with_union_query->list_of_selects->children[0]); - projection_columns.insert(projection_columns.end(), subquery_projections.begin(), subquery_projections.end()); - } - else - { - StoragePtr storage; - - if (table_expression->table_function) - { - auto query_context = const_cast(&context.getQueryContext()); - storage = query_context->executeTableFunction(table_expression->table_function); - } - else if (table_expression->database_and_table_name) - { - const auto * database_and_table_ast = table_expression->database_and_table_name->as(); - DatabaseAndTableWithAlias database_and_table_name(*database_and_table_ast); - storage = context.getTable(database_and_table_name.database, database_and_table_name.table); - } - else - throw Exception("Logical error: unexpected table expression", ErrorCodes::LOGICAL_ERROR); - - const auto block = storage->getSampleBlock(); - if (const auto * asterisk_pattern = asterisk->as()) - { - for (size_t idx = 0; idx < block.columns(); ++idx) - { - auto & col = block.getByPosition(idx); - if (asterisk_pattern->isColumnMatching(col.name)) - projection_columns.emplace_back(std::make_shared(col.name)); - } - } - else - { - for (size_t idx = 0; idx < block.columns(); ++idx) - projection_columns.emplace_back(std::make_shared(block.getByPosition(idx).name)); - } - } - } - return projection_columns; -} - -void PredicateExpressionsOptimizer::cleanExpressionAlias(ASTPtr & expression) -{ - const auto my_alias = expression->tryGetAlias(); - if (!my_alias.empty()) - expression->setAlias(""); - - for (auto & child : expression->children) - cleanExpressionAlias(child); -} - } diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.h b/dbms/src/Interpreters/PredicateExpressionsOptimizer.h index ca2c8b8766d..19010c3a328 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.h +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.h @@ -1,15 +1,13 @@ #pragma once -#include "DatabaseAndTableWithAlias.h" #include -#include +#include namespace DB { -class ASTIdentifier; -class ASTSubquery; class Context; +class Settings; /** This class provides functions for Push-Down predicate expressions * @@ -24,87 +22,35 @@ class Context; */ class PredicateExpressionsOptimizer { - using ProjectionWithAlias = std::pair; - using SubqueriesProjectionColumns = std::map>; - using IdentifierWithQualifier = std::pair; +public: + PredicateExpressionsOptimizer(const Context & context_, const TablesWithColumnNames & tables_with_columns_, const Settings & settings_); + bool optimize(ASTSelectQuery & select_query); + +private: /// Extracts settings, mostly to show which are used and which are not. struct ExtractedSettings { - /// QueryNormalizer settings - const UInt64 max_ast_depth; - const UInt64 max_expanded_ast_elements; - const String count_distinct_implementation; - - /// for PredicateExpressionsOptimizer const bool enable_optimize_predicate_expression; const bool enable_optimize_predicate_expression_to_final_subquery; - const bool join_use_nulls; template ExtractedSettings(const T & settings_) - : max_ast_depth(settings_.max_ast_depth), - max_expanded_ast_elements(settings_.max_expanded_ast_elements), - count_distinct_implementation(settings_.count_distinct_implementation), - enable_optimize_predicate_expression(settings_.enable_optimize_predicate_expression), - enable_optimize_predicate_expression_to_final_subquery(settings_.enable_optimize_predicate_expression_to_final_subquery), - join_use_nulls(settings_.join_use_nulls) + : enable_optimize_predicate_expression(settings_.enable_optimize_predicate_expression), + enable_optimize_predicate_expression_to_final_subquery(settings_.enable_optimize_predicate_expression_to_final_subquery) {} }; -public: - PredicateExpressionsOptimizer(ASTSelectQuery * ast_select_, ExtractedSettings && settings_, const Context & context_); - - bool optimize(); - -private: - ASTSelectQuery * ast_select; - const ExtractedSettings settings; const Context & context; + const std::vector & tables_with_columns; - enum OptimizeKind - { - NONE, - PUSH_TO_PREWHERE, - PUSH_TO_WHERE, - PUSH_TO_HAVING, - }; + const ExtractedSettings settings; - bool isArrayJoinFunction(const ASTPtr & node); + std::vector extractTablesPredicates(const ASTPtr & where, const ASTPtr & prewhere); - std::vector splitConjunctionPredicate(const ASTPtr & predicate_expression); + bool tryRewritePredicatesToTables(ASTs & tables_element, const std::vector & tables_predicates); - std::vector getDependenciesAndQualifiers(ASTPtr & expression, - std::vector & tables_with_aliases); - - bool optimizeExpression(const ASTPtr & outer_expression, ASTSelectQuery * subquery, ASTSelectQuery::Expression expr); - - bool optimizeImpl(const ASTPtr & outer_expression, const SubqueriesProjectionColumns & subqueries_projection_columns, OptimizeKind optimize_kind); - - bool allowPushDown( - const ASTSelectQuery * subquery, - const ASTPtr & outer_predicate, - const std::vector & subquery_projection_columns, - const std::vector & outer_predicate_dependencies, - OptimizeKind & optimize_kind); - - bool checkDependencies( - const std::vector & projection_columns, - const std::vector & dependencies, - OptimizeKind & optimize_kind); - - void setNewAliasesForInnerPredicate(const std::vector & projection_columns, - const std::vector & inner_predicate_dependencies); - - SubqueriesProjectionColumns getAllSubqueryProjectionColumns(); - - void getSubqueryProjectionColumns(const ASTPtr & subquery, SubqueriesProjectionColumns & all_subquery_projection_columns); - - ASTs getSelectQueryProjectionColumns(ASTPtr & ast); - - ASTs evaluateAsterisk(ASTSelectQuery * select_query, const ASTPtr & asterisk); - - void cleanExpressionAlias(ASTPtr & expression); + bool tryRewritePredicatesToTable(ASTPtr & table_element, const ASTs & table_predicates, const Names & table_column) const; }; } diff --git a/dbms/src/Interpreters/PredicateRewriteVisitor.cpp b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp new file mode 100644 index 00000000000..c968f65b008 --- /dev/null +++ b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp @@ -0,0 +1,141 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +PredicateRewriteVisitorData::PredicateRewriteVisitorData( + const Context & context_, const ASTs & predicates_, const Names & colunm_names_, bool optimize_final_) + : context(context_), predicates(predicates_), column_names(colunm_names_), optimize_final(optimize_final_) +{ +} + +void PredicateRewriteVisitorData::visit(ASTSelectWithUnionQuery & union_select_query, ASTPtr &) +{ + auto & internal_select_list = union_select_query.list_of_selects->children; + + if (internal_select_list.size() > 0) + visitFirstInternalSelect(*internal_select_list[0]->as(), internal_select_list[0]); + + for (size_t index = 1; index < internal_select_list.size(); ++index) + visitOtherInternalSelect(*internal_select_list[index]->as(), internal_select_list[index]); +} + +void PredicateRewriteVisitorData::visitFirstInternalSelect(ASTSelectQuery & select_query, ASTPtr &) +{ + is_rewrite |= rewriteSubquery(select_query, column_names, column_names); +} + +void PredicateRewriteVisitorData::visitOtherInternalSelect(ASTSelectQuery & select_query, ASTPtr &) +{ + /// For non first select, its alias has no more significance, so we can set a temporary alias for them + ASTPtr temp_internal_select = select_query.clone(); + ASTSelectQuery * temp_select_query = temp_internal_select->as(); + + size_t alias_index = 0; + for (const auto ref_select : temp_select_query->refSelect()->children) + { + if (!ref_select->as() && !ref_select->as() && !ref_select->as()) + { + if (const auto & alias = ref_select->tryGetAlias(); alias.empty()) + ref_select->setAlias("--predicate_optimizer_" + toString(alias_index++)); + } + } + + const Names & internal_columns = InterpreterSelectQuery( + temp_internal_select, context, SelectQueryOptions().analyze()).getSampleBlock().getNames(); + + if ((is_rewrite |= rewriteSubquery(*temp_select_query, column_names, internal_columns))) + { + select_query.setExpression(ASTSelectQuery::Expression::SELECT, std::move(temp_select_query->refSelect())); + + if (temp_select_query->where()) + select_query.setExpression(ASTSelectQuery::Expression::WHERE, std::move(temp_select_query->refWhere())); + + if (temp_select_query->having()) + select_query.setExpression(ASTSelectQuery::Expression::HAVING, std::move(temp_select_query->refHaving())); + } +} + +static void cleanAliasAndCollectIdentifiers(ASTPtr & predicate, std::vector & identifiers) +{ + for (auto & children : predicate->children) + cleanAliasAndCollectIdentifiers(children, identifiers); + + if (const auto alias = predicate->tryGetAlias(); !alias.empty()) + predicate->setAlias(""); + + if (ASTIdentifier * identifier = predicate->as()) + identifiers.emplace_back(identifier); +} + +bool PredicateRewriteVisitorData::allowPushDown(const ASTSelectQuery &subquery, NameSet & aggregate_column) +{ + if ((!optimize_final && subquery.final()) + || subquery.limitBy() || subquery.limitLength() + || subquery.with() || subquery.withFill()) + return false; + + for (const auto & select_expression : subquery.select()->children) + { + ExpressionInfoVisitor::Data expression_info{.context = context, .tables = {}}; + ExpressionInfoVisitor(expression_info).visit(select_expression); + + if (expression_info.is_stateful_function) + return false; + else if (expression_info.is_aggregate_function) + aggregate_column.emplace(select_expression->getAliasOrColumnName()); + } + + return true; +} + +bool PredicateRewriteVisitorData::rewriteSubquery(ASTSelectQuery & subquery, const Names & outer_columns, const Names & inner_columns) +{ + NameSet aggregate_columns; + + if (!allowPushDown(subquery, aggregate_columns)) + return false; + + for (const auto & predicate : predicates) + { + std::vector identifiers; + ASTPtr optimize_predicate = predicate->clone(); + cleanAliasAndCollectIdentifiers(optimize_predicate, identifiers); + + ASTSelectQuery::Expression rewrite_to = ASTSelectQuery::Expression::WHERE; + + for (size_t index = 0; index < identifiers.size(); ++index) + { + const auto & column_name = IdentifierSemantic::getColumnName(*identifiers[index]); + + const auto & iterator = std::find(outer_columns.begin(), outer_columns.end(), column_name); + + if (iterator == outer_columns.end()) + throw Exception("", ErrorCodes::LOGICAL_ERROR); + + if (aggregate_columns.count(*column_name)) + rewrite_to = ASTSelectQuery::Expression::HAVING; + + identifiers[index]->setShortName(inner_columns[iterator - outer_columns.begin()]); + } + + ASTPtr optimize_expression = subquery.getExpression(rewrite_to, false); + subquery.setExpression(rewrite_to, + optimize_expression ? makeASTFunction("and", optimize_predicate, optimize_expression) : optimize_predicate); + } + return true; +} + +} diff --git a/dbms/src/Interpreters/PredicateRewriteVisitor.h b/dbms/src/Interpreters/PredicateRewriteVisitor.h new file mode 100644 index 00000000000..ea7656256fe --- /dev/null +++ b/dbms/src/Interpreters/PredicateRewriteVisitor.h @@ -0,0 +1,41 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class PredicateRewriteVisitorData +{ +public: + using TypeToVisit = ASTSelectWithUnionQuery; + + bool isRewrite() const { return is_rewrite; } + + void visit(ASTSelectWithUnionQuery & union_select_query, ASTPtr &); + + PredicateRewriteVisitorData(const Context & context_, const ASTs & predicates_, const Names & colunm_names_, bool optimize_final_); + +private: + const Context & context; + const ASTs & predicates; + const Names & column_names; + + bool optimize_final; + bool is_rewrite = false; + + void visitFirstInternalSelect(ASTSelectQuery & select_query, ASTPtr &); + + void visitOtherInternalSelect(ASTSelectQuery & select_query, ASTPtr &); + + bool allowPushDown(const ASTSelectQuery & subquery, NameSet & aggregate_column); + + bool rewriteSubquery(ASTSelectQuery & subquery, const Names & outer_columns, const Names & inner_columns); +}; + +using PredicateRewriteMatcher = OneTypeMatcher; +using PredicateRewriteVisitor = InDepthNodeVisitor; +} diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 85135c71c6f..678f42adf88 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -936,7 +936,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( getArrayJoinedColumns(query, result, select_query, result.source_columns, source_columns_set); /// Push the predicate expression down to the subqueries. - result.rewrite_subqueries = PredicateExpressionsOptimizer(select_query, settings, context).optimize(); + result.rewrite_subqueries = PredicateExpressionsOptimizer(context, tables_with_columns, settings).optimize(*select_query); setJoinStrictness(*select_query, settings.join_default_strictness, settings.any_join_distinct_right_table_keys, result.analyzed_join->table_join); diff --git a/dbms/src/Storages/StorageView.cpp b/dbms/src/Storages/StorageView.cpp index 824856dfc4e..5c8543bbb33 100644 --- a/dbms/src/Storages/StorageView.cpp +++ b/dbms/src/Storages/StorageView.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -23,6 +24,7 @@ namespace ErrorCodes { extern const int INCORRECT_QUERY; extern const int LOGICAL_ERROR; + extern const int ALIAS_REQUIRED; } @@ -62,8 +64,23 @@ BlockInputStreams StorageView::read( replaceTableNameWithSubquery(new_outer_select, new_inner_query); - if (PredicateExpressionsOptimizer(new_outer_select, context.getSettings(), context).optimize()) - current_inner_query = new_inner_query; + /// TODO: remove getTableExpressions and getTablesWithColumns + { + const auto & table_expressions = getTableExpressions(*new_outer_select); + const auto & tables_with_columns = getDatabaseAndTablesWithColumnNames(table_expressions, context); + + auto & settings = context.getSettingsRef(); + if (settings.joined_subquery_requires_alias && tables_with_columns.size() > 1) + { + for (auto & pr : tables_with_columns) + if (pr.table.table.empty() && pr.table.alias.empty()) + throw Exception("Not unique subquery in FROM requires an alias (or joined_subquery_requires_alias=0 to disable restriction).", + ErrorCodes::ALIAS_REQUIRED); + } + + if (PredicateExpressionsOptimizer(context, tables_with_columns, context.getSettings()).optimize(*new_outer_select)) + current_inner_query = new_inner_query; + } } QueryPipeline pipeline; From d43eae2db8fa91491c35bb4006f79cc17a5e2c6e Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Thu, 2 Jan 2020 12:24:55 -0500 Subject: [PATCH 03/32] First changes. --- .../src/Storages/LiveView/StorageLiveView.cpp | 114 +++++++++++------- dbms/src/Storages/LiveView/StorageLiveView.h | 7 ++ 2 files changed, 76 insertions(+), 45 deletions(-) diff --git a/dbms/src/Storages/LiveView/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp index db410eeb5e4..84ebb497243 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -95,6 +95,64 @@ static void extractDependentTable(ASTPtr & query, String & select_database_name, DB::ErrorCodes::LOGICAL_ERROR); } +BlocksPtrs StorageLiveView::collectMergeableBlocks(const Context & context) +{ + BlocksPtrs new_mergeable_blocks = std::make_shared>(); + BlocksPtr base_mergeable_blocks = std::make_shared(); + + InterpreterSelectQuery interpreter(mergeable_query, context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); + + auto view_mergeable_stream = std::make_shared(interpreter.execute().in); + + while (Block this_block = view_mergeable_stream->read()) + base_mergeable_blocks->push_back(this_block); + + new_mergeable_blocks->push_back(base_mergeable_blocks); + + mergeable_blocks = new_mergeable_blocks; +} + +BlockInputStreams blocksToInputStreams(BlocksPtrs blocks) +{ + BlockInputStreams streams; + + for (auto & blocks_ : *blocks) + { + if (blocks_->empty()) + continue; + + auto sample_block = blocks_->front().cloneEmpty(); + + BlockInputStreamPtr stream = std::make_shared(std::make_shared(blocks_), sample_block); + + streams.push_back(std::move(stream)); + } + return streams; +} + +/// Complete query using input streams from mergeable blocks +BlockInputStreamPtr StorageLiveView::completeQuery(BlockInputStreams from) +{ + auto block_context = std::make_unique(global_context); + block_context->makeQueryContext(); + + auto blocks_storage = StorageBlocks::createStorage(database_name, table_name, parent_storage->getColumns(), + std::move(from), QueryProcessingStage::WithMergeableState); + + block_context->addExternalTable(table_name + "_blocks", blocks_storage); + + InterpreterSelectQuery select(inner_blocks_query->clone(), *block_context, StoragePtr(), SelectQueryOptions(QueryProcessingStage::Complete)); + BlockInputStreamPtr data = std::make_shared(select.execute().in); + + /// Squashing is needed here because the view query can generate a lot of blocks + /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY + /// and two-level aggregation is triggered). + data = std::make_shared( + data, global_context.getSettingsRef().min_insert_block_size_rows, + global_context.getSettingsRef().min_insert_block_size_bytes); + + return data; +} void StorageLiveView::writeIntoLiveView( StorageLiveView & live_view, @@ -132,41 +190,25 @@ void StorageLiveView::writeIntoLiveView( mergeable_blocks = live_view.getMergeableBlocks(); if (!mergeable_blocks || mergeable_blocks->size() >= context.getGlobalContext().getSettingsRef().max_live_view_insert_blocks_before_refresh) { - mergeable_blocks = std::make_shared>(); - BlocksPtr base_mergeable_blocks = std::make_shared(); - InterpreterSelectQuery interpreter(mergeable_query, context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); - auto view_mergeable_stream = std::make_shared( - interpreter.execute().in); - while (Block this_block = view_mergeable_stream->read()) - base_mergeable_blocks->push_back(this_block); - mergeable_blocks->push_back(base_mergeable_blocks); - live_view.setMergeableBlocks(mergeable_blocks); - - /// Create from streams - for (auto & blocks_ : *mergeable_blocks) - { - if (blocks_->empty()) - continue; - auto sample_block = blocks_->front().cloneEmpty(); - BlockInputStreamPtr stream = std::make_shared(std::make_shared(blocks_), sample_block); - from.push_back(std::move(stream)); - } - + mergeable_blocks = live_view.collectMergeableBlocks(context); + from = live_view.blocksToInputStreams(mergeable_blocks); is_block_processed = true; } } - auto parent_storage = context.getTable(live_view.getSelectDatabaseName(), live_view.getSelectTableName()); - if (!is_block_processed) { BlockInputStreams streams = {std::make_shared(block)}; + auto blocks_storage = StorageBlocks::createStorage(live_view.database_name, live_view.table_name, - parent_storage->getColumns(), std::move(streams), QueryProcessingStage::FetchColumns); + live_view.getParentStorage()->getColumns(), std::move(streams), QueryProcessingStage::FetchColumns); + InterpreterSelectQuery select_block(mergeable_query, context, blocks_storage, QueryProcessingStage::WithMergeableState); + auto data_mergeable_stream = std::make_shared( select_block.execute().in); + while (Block this_block = data_mergeable_stream->read()) new_mergeable_blocks->push_back(this_block); @@ -178,31 +220,11 @@ void StorageLiveView::writeIntoLiveView( mergeable_blocks = live_view.getMergeableBlocks(); mergeable_blocks->push_back(new_mergeable_blocks); - - /// Create from streams - for (auto & blocks_ : *mergeable_blocks) - { - if (blocks_->empty()) - continue; - auto sample_block = blocks_->front().cloneEmpty(); - BlockInputStreamPtr stream = std::make_shared(std::make_shared(blocks_), sample_block); - from.push_back(std::move(stream)); - } + from = live_view.blocksToInputStreams(mergeable_blocks); } } - auto blocks_storage = StorageBlocks::createStorage(live_view.database_name, live_view.table_name, parent_storage->getColumns(), std::move(from), QueryProcessingStage::WithMergeableState); - block_context->addExternalTable(live_view.table_name + "_blocks", blocks_storage); - - InterpreterSelectQuery select(live_view.getInnerBlocksQuery(), *block_context, StoragePtr(), SelectQueryOptions(QueryProcessingStage::Complete)); - BlockInputStreamPtr data = std::make_shared(select.execute().in); - - /// Squashing is needed here because the view query can generate a lot of blocks - /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY - /// and two-level aggregation is triggered). - data = std::make_shared( - data, context.getGlobalContext().getSettingsRef().min_insert_block_size_rows, context.getGlobalContext().getSettingsRef().min_insert_block_size_bytes); - + BlockInputStreamPtr data = completeQuery(std::move(from)); copyData(*data, *output); } @@ -247,6 +269,8 @@ StorageLiveView::StorageLiveView( DatabaseAndTableName(select_database_name, select_table_name), DatabaseAndTableName(database_name, table_name)); + parent_storage = local_context.getTable(select_database_name, select_table_name); + is_temporary = query.temporary; temporary_live_view_timeout = local_context.getSettingsRef().temporary_live_view_timeout.totalSeconds(); diff --git a/dbms/src/Storages/LiveView/StorageLiveView.h b/dbms/src/Storages/LiveView/StorageLiveView.h index a5b0f15e879..46e5008ece3 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.h +++ b/dbms/src/Storages/LiveView/StorageLiveView.h @@ -45,6 +45,7 @@ public: String getDatabaseName() const override { return database_name; } String getSelectDatabaseName() const { return select_database_name; } String getSelectTableName() const { return select_table_name; } + StoragePtr getParentStorage() const { return parent_storage; } NameAndTypePair getColumn(const String & column_name) const override; bool hasColumn(const String & column_name) const override; @@ -139,6 +140,8 @@ public: std::shared_ptr getBlocksPtr() { return blocks_ptr; } BlocksPtrs getMergeableBlocks() { return mergeable_blocks; } + /// collect and set mergeable blocks. Must be called holding mutex + BlocksPtrs collectMergeableBlocks(const Context & context); void setMergeableBlocks(BlocksPtrs blocks) { mergeable_blocks = blocks; } std::shared_ptr getActivePtr() { return active_ptr; } @@ -147,6 +150,9 @@ public: Block getHeader() const; + /// convert blocks to input streams + static BlockInputStreams blocksToInputStreams(BlocksPtrs blocks); + static void writeIntoLiveView( StorageLiveView & live_view, const Block & block, @@ -162,6 +168,7 @@ private: ASTPtr inner_blocks_query; /// query over the mergeable blocks to produce final result Context & global_context; std::unique_ptr live_view_context; + StoragePtr parent_storage; bool is_temporary = false; /// Mutex to protect access to sample block From 2c4bf0581fa62ff63a552a4b20a7de0ce5dafac4 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Thu, 2 Jan 2020 20:31:04 +0100 Subject: [PATCH 04/32] Fixing compilation errors. --- dbms/src/Storages/LiveView/StorageLiveView.cpp | 13 ++++++++++--- dbms/src/Storages/LiveView/StorageLiveView.h | 4 ++++ 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/LiveView/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp index 84ebb497243..e847c9b8b93 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -97,6 +97,11 @@ static void extractDependentTable(ASTPtr & query, String & select_database_name, BlocksPtrs StorageLiveView::collectMergeableBlocks(const Context & context) { + ASTPtr mergeable_query = inner_query; + + if (inner_subquery) + mergeable_query = inner_subquery; + BlocksPtrs new_mergeable_blocks = std::make_shared>(); BlocksPtr base_mergeable_blocks = std::make_shared(); @@ -110,9 +115,11 @@ BlocksPtrs StorageLiveView::collectMergeableBlocks(const Context & context) new_mergeable_blocks->push_back(base_mergeable_blocks); mergeable_blocks = new_mergeable_blocks; + + return mergeable_blocks; } -BlockInputStreams blocksToInputStreams(BlocksPtrs blocks) +BlockInputStreams StorageLiveView::blocksToInputStreams(BlocksPtrs blocks) { BlockInputStreams streams; @@ -131,7 +138,7 @@ BlockInputStreams blocksToInputStreams(BlocksPtrs blocks) } /// Complete query using input streams from mergeable blocks -BlockInputStreamPtr StorageLiveView::completeQuery(BlockInputStreams from) +BlockInputStreamPtr StorageLiveView::completeQuery(BlockInputStreams & from) { auto block_context = std::make_unique(global_context); block_context->makeQueryContext(); @@ -224,7 +231,7 @@ void StorageLiveView::writeIntoLiveView( } } - BlockInputStreamPtr data = completeQuery(std::move(from)); + BlockInputStreamPtr data = live_view.completeQuery(from); copyData(*data, *output); } diff --git a/dbms/src/Storages/LiveView/StorageLiveView.h b/dbms/src/Storages/LiveView/StorageLiveView.h index 46e5008ece3..49bff4bf2ca 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.h +++ b/dbms/src/Storages/LiveView/StorageLiveView.h @@ -140,8 +140,12 @@ public: std::shared_ptr getBlocksPtr() { return blocks_ptr; } BlocksPtrs getMergeableBlocks() { return mergeable_blocks; } + /// collect and set mergeable blocks. Must be called holding mutex BlocksPtrs collectMergeableBlocks(const Context & context); + /// Complete query using input streams from mergeable blocks + BlockInputStreamPtr completeQuery(BlockInputStreams & from); + void setMergeableBlocks(BlocksPtrs blocks) { mergeable_blocks = blocks; } std::shared_ptr getActivePtr() { return active_ptr; } From 25458a486589290549d251eaa1b23218682a6853 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Fri, 3 Jan 2020 02:48:15 +0100 Subject: [PATCH 05/32] Updates to make all live view tests to pass. --- .../src/Storages/LiveView/StorageLiveView.cpp | 87 ++++++------------- dbms/src/Storages/LiveView/StorageLiveView.h | 21 +++-- 2 files changed, 42 insertions(+), 66 deletions(-) diff --git a/dbms/src/Storages/LiveView/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp index e847c9b8b93..75783364247 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -1,4 +1,4 @@ -/* iopyright (c) 2018 BlackBerry Limited +/* Copyright (c) 2018 BlackBerry Limited Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. @@ -95,50 +95,45 @@ static void extractDependentTable(ASTPtr & query, String & select_database_name, DB::ErrorCodes::LOGICAL_ERROR); } -BlocksPtrs StorageLiveView::collectMergeableBlocks(const Context & context) +MergeableBlocksPtr StorageLiveView::collectMergeableBlocks(const Context & context) { ASTPtr mergeable_query = inner_query; if (inner_subquery) mergeable_query = inner_subquery; - BlocksPtrs new_mergeable_blocks = std::make_shared>(); - BlocksPtr base_mergeable_blocks = std::make_shared(); + MergeableBlocksPtr new_mergeable_blocks = std::make_shared(); + BlocksPtrs new_blocks = std::make_shared>(); + BlocksPtr base_blocks = std::make_shared(); - InterpreterSelectQuery interpreter(mergeable_query, context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); + InterpreterSelectQuery interpreter(mergeable_query->clone(), context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); auto view_mergeable_stream = std::make_shared(interpreter.execute().in); while (Block this_block = view_mergeable_stream->read()) - base_mergeable_blocks->push_back(this_block); + base_blocks->push_back(this_block); - new_mergeable_blocks->push_back(base_mergeable_blocks); + new_blocks->push_back(base_blocks); - mergeable_blocks = new_mergeable_blocks; + new_mergeable_blocks->blocks = new_blocks; + new_mergeable_blocks->sample_block = view_mergeable_stream->getHeader(); - return mergeable_blocks; + return new_mergeable_blocks; } -BlockInputStreams StorageLiveView::blocksToInputStreams(BlocksPtrs blocks) +BlockInputStreams StorageLiveView::blocksToInputStreams(BlocksPtrs blocks, Block & sample_block) { BlockInputStreams streams; - for (auto & blocks_ : *blocks) { - if (blocks_->empty()) - continue; - - auto sample_block = blocks_->front().cloneEmpty(); - BlockInputStreamPtr stream = std::make_shared(std::make_shared(blocks_), sample_block); - streams.push_back(std::move(stream)); } return streams; } /// Complete query using input streams from mergeable blocks -BlockInputStreamPtr StorageLiveView::completeQuery(BlockInputStreams & from) +BlockInputStreamPtr StorageLiveView::completeQuery(BlockInputStreams from) { auto block_context = std::make_unique(global_context); block_context->makeQueryContext(); @@ -167,8 +162,6 @@ void StorageLiveView::writeIntoLiveView( const Context & context) { BlockOutputStreamPtr output = std::make_shared(live_view); - auto block_context = std::make_unique(context.getGlobalContext()); - block_context->makeQueryContext(); /// Check if live view has any readers if not /// just reset blocks to empty and do nothing else @@ -184,28 +177,30 @@ void StorageLiveView::writeIntoLiveView( bool is_block_processed = false; BlockInputStreams from; - BlocksPtrs mergeable_blocks; + MergeableBlocksPtr mergeable_blocks; BlocksPtr new_mergeable_blocks = std::make_shared(); - ASTPtr mergeable_query = live_view.getInnerQuery(); - - if (live_view.getInnerSubQuery()) - mergeable_query = live_view.getInnerSubQuery(); { std::lock_guard lock(live_view.mutex); mergeable_blocks = live_view.getMergeableBlocks(); - if (!mergeable_blocks || mergeable_blocks->size() >= context.getGlobalContext().getSettingsRef().max_live_view_insert_blocks_before_refresh) + if (!mergeable_blocks || mergeable_blocks->blocks->size() >= context.getGlobalContext().getSettingsRef().max_live_view_insert_blocks_before_refresh) { mergeable_blocks = live_view.collectMergeableBlocks(context); - from = live_view.blocksToInputStreams(mergeable_blocks); + live_view.setMergeableBlocks(mergeable_blocks); + from = live_view.blocksToInputStreams(mergeable_blocks->blocks, mergeable_blocks->sample_block); is_block_processed = true; } } if (!is_block_processed) { - BlockInputStreams streams = {std::make_shared(block)}; + ASTPtr mergeable_query = live_view.getInnerQuery(); + + if (live_view.getInnerSubQuery()) + mergeable_query = live_view.getInnerSubQuery(); + + BlockInputStreams streams = {std::make_shared(block)}; auto blocks_storage = StorageBlocks::createStorage(live_view.database_name, live_view.table_name, live_view.getParentStorage()->getColumns(), std::move(streams), QueryProcessingStage::FetchColumns); @@ -226,8 +221,8 @@ void StorageLiveView::writeIntoLiveView( std::lock_guard lock(live_view.mutex); mergeable_blocks = live_view.getMergeableBlocks(); - mergeable_blocks->push_back(new_mergeable_blocks); - from = live_view.blocksToInputStreams(mergeable_blocks); + mergeable_blocks->blocks->push_back(new_mergeable_blocks); + from = live_view.blocksToInputStreams(mergeable_blocks->blocks, mergeable_blocks->sample_block); } } @@ -329,36 +324,10 @@ bool StorageLiveView::getNewBlocks() UInt128 key; BlocksPtr new_blocks = std::make_shared(); BlocksMetadataPtr new_blocks_metadata = std::make_shared(); - BlocksPtr new_mergeable_blocks = std::make_shared(); - ASTPtr mergeable_query = inner_query; - if (inner_subquery) - mergeable_query = inner_subquery; - - InterpreterSelectQuery interpreter(mergeable_query->clone(), *live_view_context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); - auto mergeable_stream = std::make_shared(interpreter.execute().in); - - while (Block block = mergeable_stream->read()) - new_mergeable_blocks->push_back(block); - - auto block_context = std::make_unique(global_context); - block_context->makeQueryContext(); - - mergeable_blocks = std::make_shared>(); - mergeable_blocks->push_back(new_mergeable_blocks); - BlockInputStreamPtr from = std::make_shared(std::make_shared(new_mergeable_blocks), mergeable_stream->getHeader()); - - auto blocks_storage = StorageBlocks::createStorage(database_name, table_name, global_context.getTable(select_database_name, select_table_name)->getColumns(), {from}, QueryProcessingStage::WithMergeableState); - block_context->addExternalTable(table_name + "_blocks", blocks_storage); - - InterpreterSelectQuery select(inner_blocks_query->clone(), *block_context, StoragePtr(), SelectQueryOptions(QueryProcessingStage::Complete)); - BlockInputStreamPtr data = std::make_shared(select.execute().in); - - /// Squashing is needed here because the view query can generate a lot of blocks - /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY - /// and two-level aggregation is triggered). - data = std::make_shared( - data, global_context.getSettingsRef().min_insert_block_size_rows, global_context.getSettingsRef().min_insert_block_size_bytes); + mergeable_blocks = collectMergeableBlocks(*live_view_context); + BlockInputStreams from = blocksToInputStreams(mergeable_blocks->blocks, mergeable_blocks->sample_block); + BlockInputStreamPtr data = completeQuery({from}); while (Block block = data->read()) { diff --git a/dbms/src/Storages/LiveView/StorageLiveView.h b/dbms/src/Storages/LiveView/StorageLiveView.h index 49bff4bf2ca..916406a1dbd 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.h +++ b/dbms/src/Storages/LiveView/StorageLiveView.h @@ -27,9 +27,16 @@ struct BlocksMetadata UInt64 version; }; +struct MergeableBlocks +{ + BlocksPtrs blocks; + Block sample_block; +}; + class IAST; using ASTPtr = std::shared_ptr; using BlocksMetadataPtr = std::shared_ptr; +using MergeableBlocksPtr = std::shared_ptr; class StorageLiveView : public ext::shared_ptr_helper, public IStorage { @@ -139,14 +146,14 @@ public: unsigned num_streams) override; std::shared_ptr getBlocksPtr() { return blocks_ptr; } - BlocksPtrs getMergeableBlocks() { return mergeable_blocks; } + MergeableBlocksPtr getMergeableBlocks() { return mergeable_blocks; } - /// collect and set mergeable blocks. Must be called holding mutex - BlocksPtrs collectMergeableBlocks(const Context & context); + /// Collect mergeable blocks and their sample. Must be called holding mutex + MergeableBlocksPtr collectMergeableBlocks(const Context & context); /// Complete query using input streams from mergeable blocks - BlockInputStreamPtr completeQuery(BlockInputStreams & from); + BlockInputStreamPtr completeQuery(BlockInputStreams from); - void setMergeableBlocks(BlocksPtrs blocks) { mergeable_blocks = blocks; } + void setMergeableBlocks(MergeableBlocksPtr blocks) { mergeable_blocks = blocks; } std::shared_ptr getActivePtr() { return active_ptr; } /// Read new data blocks that store query result @@ -155,7 +162,7 @@ public: Block getHeader() const; /// convert blocks to input streams - static BlockInputStreams blocksToInputStreams(BlocksPtrs blocks); + static BlockInputStreams blocksToInputStreams(BlocksPtrs blocks, Block & sample_block); static void writeIntoLiveView( StorageLiveView & live_view, @@ -191,7 +198,7 @@ private: std::shared_ptr blocks_ptr; /// Current data blocks metadata std::shared_ptr blocks_metadata_ptr; - BlocksPtrs mergeable_blocks; + MergeableBlocksPtr mergeable_blocks; /// Background thread for temporary tables /// which drops this table if there are no users From 6a5e3e2dd9c585159264a2ac4ad2a6d5ab123655 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 4 Jan 2020 12:31:45 +0800 Subject: [PATCH 06/32] try fix failure test & fix build failure --- .../ExtractExpressionInfoVisitor.cpp | 14 ++++ .../ExtractExpressionInfoVisitor.h | 5 +- .../PredicateExpressionsOptimizer.cpp | 81 ++++++++++++++----- .../PredicateExpressionsOptimizer.h | 17 ++-- .../Interpreters/PredicateRewriteVisitor.cpp | 79 ++++++------------ .../Interpreters/PredicateRewriteVisitor.h | 9 +-- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 6 +- .../00597_push_down_predicate.reference | 2 +- 8 files changed, 115 insertions(+), 98 deletions(-) diff --git a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp index 4bd0446cd97..2bf954a7f9e 100644 --- a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp +++ b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp @@ -55,5 +55,19 @@ void ExpressionInfoMatcher::visit(const ASTIdentifier & identifier, const ASTPtr } } +bool hasStatefulFunction(const ASTPtr & node, const Context & context) +{ + for (const auto & select_expression : node->children) + { + ExpressionInfoVisitor::Data expression_info{.context = context, .tables = {}}; + ExpressionInfoVisitor(expression_info).visit(select_expression); + + if (expression_info.is_stateful_function) + return true; + } + + return false; +} + } diff --git a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.h b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.h index 6ef9960b918..e884a27775a 100644 --- a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.h +++ b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.h @@ -21,7 +21,7 @@ struct ExpressionInfoMatcher bool is_array_join = false; bool is_stateful_function = false; bool is_aggregate_function = false; - std::unordered_set unique_reference_tables_pos; + std::unordered_set unique_reference_tables_pos = {}; }; static void visit(const ASTPtr & ast, Data & data); @@ -33,7 +33,8 @@ struct ExpressionInfoMatcher static void visit(const ASTIdentifier & identifier, const ASTPtr &, Data & data); }; - using ExpressionInfoVisitor = ConstInDepthNodeVisitor; +bool hasStatefulFunction(const ASTPtr & node, const Context & context); + } diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp index a5cc578a98a..9927091874c 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -1,29 +1,13 @@ -#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 @@ -43,7 +27,13 @@ PredicateExpressionsOptimizer::PredicateExpressionsOptimizer( bool PredicateExpressionsOptimizer::optimize(ASTSelectQuery & select_query) { - if (!settings.enable_optimize_predicate_expression || !select_query.tables() || select_query.tables()->children.empty()) + if (!settings.enable_optimize_predicate_expression) + return false; + + if (select_query.having() && (!select_query.group_by_with_cube && !select_query.group_by_with_rollup && !select_query.group_by_with_totals)) + tryMovePredicatesFromHavingToWhere(select_query); + + if (!select_query.tables() || select_query.tables()->children.empty()) return false; if ((!select_query.where() && !select_query.prewhere()) || select_query.array_join_expression_list()) @@ -165,14 +155,61 @@ bool PredicateExpressionsOptimizer::tryRewritePredicatesToTable(ASTPtr & table_e { if (!table_predicates.empty()) { - PredicateRewriteVisitor::Data data( - context, table_predicates, table_column, settings.enable_optimize_predicate_expression_to_final_subquery); + auto optimize_final = settings.enable_optimize_predicate_expression_to_final_subquery; + PredicateRewriteVisitor::Data data(context, table_predicates, table_column, optimize_final); PredicateRewriteVisitor(data).visit(table_element); - return data.isRewrite(); + return data.is_rewrite; } return false; } +bool PredicateExpressionsOptimizer::tryMovePredicatesFromHavingToWhere(ASTSelectQuery & select_query) +{ + ASTs where_predicates; + ASTs having_predicates; + + const auto & reduce_predicates = [&](const ASTs & predicates) + { + ASTPtr res = predicates[0]; + for (size_t index = 1; index < predicates.size(); ++index) + res = makeASTFunction("and", res, predicates[index]); + + return res; + }; + + for (const auto & moving_predicate: splitConjunctionPredicate({select_query.having()})) + { + ExpressionInfoVisitor::Data expression_info{.context = context, .tables = {}}; + ExpressionInfoVisitor(expression_info).visit(moving_predicate); + + /// TODO: If there is no group by, where, and prewhere expression, we can push down the stateful function + if (expression_info.is_stateful_function) + return false; + + if (expression_info.is_aggregate_function) + having_predicates.emplace_back(moving_predicate); + else + where_predicates.emplace_back(moving_predicate); + } + + if (having_predicates.empty()) + select_query.setExpression(ASTSelectQuery::Expression::HAVING, {}); + else + { + auto having_predicate = reduce_predicates(having_predicates); + select_query.setExpression(ASTSelectQuery::Expression::HAVING, std::move(having_predicate)); + } + + if (!where_predicates.empty()) + { + auto moved_predicate = reduce_predicates(where_predicates); + moved_predicate = select_query.where() ? makeASTFunction("and", select_query.where(), moved_predicate) : moved_predicate; + select_query.setExpression(ASTSelectQuery::Expression::WHERE, std::move(moved_predicate)); + } + + return true; +} + } diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.h b/dbms/src/Interpreters/PredicateExpressionsOptimizer.h index 19010c3a328..da6b98987a6 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.h +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.h @@ -7,18 +7,13 @@ namespace DB { class Context; -class Settings; +struct Settings; -/** This class provides functions for Push-Down predicate expressions - * - * The Example: - * - Query before optimization : - * SELECT id_1, name_1 FROM (SELECT id_1, name_1 FROM table_a UNION ALL SELECT id_2, name_2 FROM table_b) - * WHERE id_1 = 1 - * - Query after optimization : - * SELECT id_1, name_1 FROM (SELECT id_1, name_1 FROM table_a WHERE id_1 = 1 UNION ALL SELECT id_2, name_2 FROM table_b WHERE id_2 = 1) - * WHERE id_1 = 1 +/** Predicate optimization based on rewriting ast rules * For more details : https://github.com/ClickHouse/ClickHouse/pull/2015#issuecomment-374283452 + * The optimizer does two different optimizations + * - Move predicates from having to where + * - Push the predicate down from the current query to the having of the subquery */ class PredicateExpressionsOptimizer { @@ -51,6 +46,8 @@ private: bool tryRewritePredicatesToTables(ASTs & tables_element, const std::vector & tables_predicates); bool tryRewritePredicatesToTable(ASTPtr & table_element, const ASTs & table_predicates, const Names & table_column) const; + + bool tryMovePredicatesFromHavingToWhere(ASTSelectQuery & select_query); }; } diff --git a/dbms/src/Interpreters/PredicateRewriteVisitor.cpp b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp index c968f65b008..f009edec937 100644 --- a/dbms/src/Interpreters/PredicateRewriteVisitor.cpp +++ b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp @@ -1,23 +1,23 @@ #include +#include #include +#include #include #include +#include #include #include -#include -#include -#include -#include #include +#include namespace DB { PredicateRewriteVisitorData::PredicateRewriteVisitorData( - const Context & context_, const ASTs & predicates_, const Names & colunm_names_, bool optimize_final_) - : context(context_), predicates(predicates_), column_names(colunm_names_), optimize_final(optimize_final_) + const Context & context_, const ASTs & predicates_, const Names & column_names_, bool optimize_final_) + : context(context_), predicates(predicates_), column_names(column_names_), optimize_final(optimize_final_) { } @@ -44,9 +44,10 @@ void PredicateRewriteVisitorData::visitOtherInternalSelect(ASTSelectQuery & sele ASTSelectQuery * temp_select_query = temp_internal_select->as(); size_t alias_index = 0; - for (const auto ref_select : temp_select_query->refSelect()->children) + for (auto & ref_select : temp_select_query->refSelect()->children) { - if (!ref_select->as() && !ref_select->as() && !ref_select->as()) + if (!ref_select->as() && !ref_select->as() && !ref_select->as() && + !ref_select->as()) { if (const auto & alias = ref_select->tryGetAlias(); alias.empty()) ref_select->setAlias("--predicate_optimizer_" + toString(alias_index++)); @@ -56,15 +57,11 @@ void PredicateRewriteVisitorData::visitOtherInternalSelect(ASTSelectQuery & sele const Names & internal_columns = InterpreterSelectQuery( temp_internal_select, context, SelectQueryOptions().analyze()).getSampleBlock().getNames(); - if ((is_rewrite |= rewriteSubquery(*temp_select_query, column_names, internal_columns))) + if (rewriteSubquery(*temp_select_query, column_names, internal_columns)) { + is_rewrite |= true; select_query.setExpression(ASTSelectQuery::Expression::SELECT, std::move(temp_select_query->refSelect())); - - if (temp_select_query->where()) - select_query.setExpression(ASTSelectQuery::Expression::WHERE, std::move(temp_select_query->refWhere())); - - if (temp_select_query->having()) - select_query.setExpression(ASTSelectQuery::Expression::HAVING, std::move(temp_select_query->refHaving())); + select_query.setExpression(ASTSelectQuery::Expression::HAVING, std::move(temp_select_query->refHaving())); } } @@ -80,32 +77,12 @@ static void cleanAliasAndCollectIdentifiers(ASTPtr & predicate, std::vectorchildren) - { - ExpressionInfoVisitor::Data expression_info{.context = context, .tables = {}}; - ExpressionInfoVisitor(expression_info).visit(select_expression); - - if (expression_info.is_stateful_function) - return false; - else if (expression_info.is_aggregate_function) - aggregate_column.emplace(select_expression->getAliasOrColumnName()); - } - - return true; -} - bool PredicateRewriteVisitorData::rewriteSubquery(ASTSelectQuery & subquery, const Names & outer_columns, const Names & inner_columns) { - NameSet aggregate_columns; - - if (!allowPushDown(subquery, aggregate_columns)) + if ((!optimize_final && subquery.final()) + || subquery.with() || subquery.withFill() + || subquery.limitBy() || subquery.limitLength() + || hasStatefulFunction(subquery.select(), context)) return false; for (const auto & predicate : predicates) @@ -114,27 +91,23 @@ bool PredicateRewriteVisitorData::rewriteSubquery(ASTSelectQuery & subquery, con ASTPtr optimize_predicate = predicate->clone(); cleanAliasAndCollectIdentifiers(optimize_predicate, identifiers); - ASTSelectQuery::Expression rewrite_to = ASTSelectQuery::Expression::WHERE; - for (size_t index = 0; index < identifiers.size(); ++index) { - const auto & column_name = IdentifierSemantic::getColumnName(*identifiers[index]); + const auto & column_name = identifiers[index]->shortName(); + const auto & outer_column_iterator = std::find(outer_columns.begin(), outer_columns.end(), column_name); - const auto & iterator = std::find(outer_columns.begin(), outer_columns.end(), column_name); + if (outer_column_iterator == outer_columns.end()) + throw Exception("LOGICAL ERROR: the column " + column_name + " does not exists.", ErrorCodes::LOGICAL_ERROR); - if (iterator == outer_columns.end()) - throw Exception("", ErrorCodes::LOGICAL_ERROR); - - if (aggregate_columns.count(*column_name)) - rewrite_to = ASTSelectQuery::Expression::HAVING; - - identifiers[index]->setShortName(inner_columns[iterator - outer_columns.begin()]); + identifiers[index]->setShortName(inner_columns[outer_column_iterator - outer_columns.begin()]); } - ASTPtr optimize_expression = subquery.getExpression(rewrite_to, false); - subquery.setExpression(rewrite_to, - optimize_expression ? makeASTFunction("and", optimize_predicate, optimize_expression) : optimize_predicate); + /// We only need to push all the predicates to subquery having + /// The subquery optimizer will move the appropriate predicates from having to where + subquery.setExpression(ASTSelectQuery::Expression::HAVING, + subquery.having() ? makeASTFunction("and", optimize_predicate, subquery.having()) : optimize_predicate); } + return true; } diff --git a/dbms/src/Interpreters/PredicateRewriteVisitor.h b/dbms/src/Interpreters/PredicateRewriteVisitor.h index ea7656256fe..e07df922c15 100644 --- a/dbms/src/Interpreters/PredicateRewriteVisitor.h +++ b/dbms/src/Interpreters/PredicateRewriteVisitor.h @@ -11,28 +11,23 @@ namespace DB class PredicateRewriteVisitorData { public: + bool is_rewrite = false; using TypeToVisit = ASTSelectWithUnionQuery; - bool isRewrite() const { return is_rewrite; } - void visit(ASTSelectWithUnionQuery & union_select_query, ASTPtr &); - PredicateRewriteVisitorData(const Context & context_, const ASTs & predicates_, const Names & colunm_names_, bool optimize_final_); + PredicateRewriteVisitorData(const Context & context_, const ASTs & predicates_, const Names & column_names_, bool optimize_final_); private: const Context & context; const ASTs & predicates; const Names & column_names; - bool optimize_final; - bool is_rewrite = false; void visitFirstInternalSelect(ASTSelectQuery & select_query, ASTPtr &); void visitOtherInternalSelect(ASTSelectQuery & select_query, ASTPtr &); - bool allowPushDown(const ASTSelectQuery & subquery, NameSet & aggregate_column); - bool rewriteSubquery(ASTSelectQuery & subquery, const Names & outer_columns, const Names & inner_columns); }; diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 678f42adf88..3680947c8f1 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -920,6 +920,9 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( if (select_query) { + /// Push the predicate expression down to the subqueries. + result.rewrite_subqueries = PredicateExpressionsOptimizer(context, tables_with_columns, settings).optimize(*select_query); + /// GROUP BY injective function elimination. optimizeGroupBy(select_query, source_columns_set, context); @@ -935,9 +938,6 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( /// array_join_alias_to_name, array_join_result_to_source. getArrayJoinedColumns(query, result, select_query, result.source_columns, source_columns_set); - /// Push the predicate expression down to the subqueries. - result.rewrite_subqueries = PredicateExpressionsOptimizer(context, tables_with_columns, settings).optimize(*select_query); - setJoinStrictness(*select_query, settings.join_default_strictness, settings.any_join_distinct_right_table_keys, result.analyzed_join->table_join); collectJoinedColumns(*result.analyzed_join, *select_query, tables_with_columns, result.aliases); diff --git a/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference b/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference index f64243e9be7..9fde80689f1 100644 --- a/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference +++ b/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference @@ -13,7 +13,7 @@ SELECT \n a, \n b\nFROM \n(\n SELECT \n 1 AS a, \n 1 AS b -------Need push down------- SELECT toString(value) AS value\nFROM \n(\n SELECT 1 AS value\n) 1 -SELECT id\nFROM \n(\n SELECT 1 AS id\n UNION ALL\n SELECT 2 AS `2`\n WHERE 0\n)\nWHERE id = 1 +SELECT id\nFROM \n(\n SELECT 1 AS id\n UNION ALL\n SELECT 2 AS `--predicate_optimizer_0`\n WHERE 0\n)\nWHERE id = 1 1 SELECT id\nFROM \n(\n SELECT arrayJoin([1, 2, 3]) AS id\n WHERE id = 1\n)\nWHERE id = 1 1 From f8aa0b90b1298a71e442039d1dd7887479aa5cc2 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 6 Jan 2020 11:55:07 +0800 Subject: [PATCH 07/32] add bugs test --- .../Interpreters/PredicateRewriteVisitor.cpp | 7 ++- .../01056_predicate_optimizer_bugs.reference | 18 ++++++++ .../01056_predicate_optimizer_bugs.sql | 45 +++++++++++++++++++ 3 files changed, 69 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference create mode 100644 dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql diff --git a/dbms/src/Interpreters/PredicateRewriteVisitor.cpp b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp index f009edec937..781c1510679 100644 --- a/dbms/src/Interpreters/PredicateRewriteVisitor.cpp +++ b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -68,7 +69,11 @@ void PredicateRewriteVisitorData::visitOtherInternalSelect(ASTSelectQuery & sele static void cleanAliasAndCollectIdentifiers(ASTPtr & predicate, std::vector & identifiers) { for (auto & children : predicate->children) - cleanAliasAndCollectIdentifiers(children, identifiers); + { + if (!children->as()) /// skip where x in (SELECT ...) + cleanAliasAndCollectIdentifiers(children, identifiers); + } + if (const auto alias = predicate->tryGetAlias(); !alias.empty()) predicate->setAlias(""); diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference new file mode 100644 index 00000000000..289639d35ab --- /dev/null +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference @@ -0,0 +1,18 @@ +SELECT count()\nFROM \n(\n SELECT \n a, \n runningDifference(a) AS rd\n FROM \n (\n SELECT a\n FROM \n (\n SELECT 1 AS a\n UNION ALL\n SELECT 2 AS a\n )\n ORDER BY a DESC\n )\n WHERE rd = -1\n) +1 +SELECT \n k, \n v, \n d, \n i\nFROM \n(\n SELECT \n t.1 AS k, \n t.2 AS v, \n runningDifference(v) AS d, \n runningDifference(xxHash32(t.1)) AS i\n FROM \n (\n SELECT arrayJoin([(\'a\', 1), (\'a\', 2), (\'a\', 3), (\'b\', 11), (\'b\', 13), (\'b\', 15)]) AS t\n )\n)\nWHERE i = 0 +a 1 0 0 +a 2 1 0 +a 3 1 0 +b 13 2 0 +b 15 2 0 +SELECT \n co, \n co2, \n co3, \n num\nFROM \n(\n SELECT \n co, \n co2, \n co3, \n count() AS num\n FROM \n (\n SELECT \n 1 AS co, \n 2 AS co2, \n 3 AS co3\n )\n GROUP BY \n co, \n co2, \n co3\n WITH CUBE\n HAVING (co != 0) AND (co2 != 2)\n)\nWHERE (co != 0) AND (co2 != 2) +1 0 3 1 +1 0 0 1 +SELECT alias AS name\nFROM \n(\n SELECT name AS alias\n FROM system.settings\n WHERE alias = \'enable_optimize_predicate_expression\'\n)\nANY INNER JOIN \n(\n SELECT name\n FROM system.settings\n) USING (name)\nWHERE name = \'enable_optimize_predicate_expression\' +enable_optimize_predicate_expression +1 val11 val21 val31 +SELECT ccc\nFROM \n(\n SELECT 1 AS ccc\n WHERE 0\n UNION ALL\n SELECT ccc\n FROM \n (\n SELECT 2 AS ccc\n )\n ANY INNER JOIN \n (\n SELECT 2 AS ccc\n ) USING (ccc)\n WHERE ccc > 1\n)\nWHERE ccc > 1 +2 +SELECT \n ts, \n id, \n id_b, \n b.ts, \n b.id, \n id_c\nFROM \n(\n SELECT \n ts, \n id, \n id_b\n FROM A\n WHERE ts <= toDateTime(\'1970-01-01 03:00:00\')\n) AS a\nALL LEFT JOIN B AS b ON b.id = id_b\nWHERE ts <= toDateTime(\'1970-01-01 03:00:00\') +SELECT \n ts AS `--a.ts`, \n id AS `--a.id`, \n id_b AS `--a.id_b`, \n b.ts AS `--b.ts`, \n b.id AS `--b.id`, \n id_c AS `--b.id_c`\nFROM \n(\n SELECT \n ts, \n id, \n id_b\n FROM A\n WHERE ts <= toDateTime(\'1970-01-01 03:00:00\')\n) AS a\nALL LEFT JOIN B AS b ON `--b.id` = `--a.id_b`\nWHERE `--a.ts` <= toDateTime(\'1970-01-01 03:00:00\') diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql new file mode 100644 index 00000000000..301ad2a707b --- /dev/null +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql @@ -0,0 +1,45 @@ +SET enable_debug_queries = 1; +SET enable_optimize_predicate_expression = 1; + +-- https://github.com/ClickHouse/ClickHouse/issues/3885 +-- https://github.com/ClickHouse/ClickHouse/issues/5485 +ANALYZE SELECT count() FROM (SELECT a, runningDifference(a) rd FROM ( SELECT * FROM ( SELECT 1 a UNION ALL SELECT 2 a ) ORDER BY a DESC ) WHERE rd=-1 ); +SELECT count() FROM (SELECT a, runningDifference(a) rd FROM ( SELECT * FROM ( SELECT 1 a UNION ALL SELECT 2 a ) ORDER BY a DESC ) WHERE rd=-1 ); + +ANALYZE SELECT k, v, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(xxHash32(t.1)) AS i FROM ( SELECT arrayJoin([('a', 1), ('a', 2), ('a', 3), ('b', 11), ('b', 13), ('b', 15)]) AS t)) WHERE i = 0; +SELECT k, v, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(xxHash32(t.1)) AS i FROM ( SELECT arrayJoin([('a', 1), ('a', 2), ('a', 3), ('b', 11), ('b', 13), ('b', 15)]) AS t)) WHERE i = 0; + +-- https://github.com/ClickHouse/ClickHouse/issues/5682 +ANALYZE SELECT co,co2,co3,num FROM ( SELECT co,co2,co3,count() AS num FROM ( SELECT 1 AS co,2 AS co2 ,3 AS co3 ) GROUP BY cube (co,co2,co3) ) WHERE co!=0 AND co2 !=2; +SELECT co,co2,co3,num FROM ( SELECT co,co2,co3,count() AS num FROM ( SELECT 1 AS co,2 AS co2 ,3 AS co3 ) GROUP BY cube (co,co2,co3) ) WHERE co!=0 AND co2 !=2; + +-- https://github.com/ClickHouse/ClickHouse/issues/6734 +ANALYZE SELECT alias AS name FROM ( SELECT name AS alias FROM system.settings ) ANY INNER JOIN ( SELECT name FROM system.settings ) USING (name) WHERE name = 'enable_optimize_predicate_expression'; +SELECT alias AS name FROM ( SELECT name AS alias FROM system.settings ) ANY INNER JOIN ( SELECT name FROM system.settings ) USING (name) WHERE name = 'enable_optimize_predicate_expression'; + +-- https://github.com/ClickHouse/ClickHouse/issues/6767 +CREATE TABLE t1 (id UInt32, value1 String ) ENGINE ReplacingMergeTree() ORDER BY id; +CREATE TABLE t2 (id UInt32, value2 String ) ENGINE ReplacingMergeTree() ORDER BY id; +CREATE TABLE t3 (id UInt32, value3 String ) ENGINE ReplacingMergeTree() ORDER BY id; + +INSERT INTO t1 (id, value1) VALUES (1, 'val11'); +INSERT INTO t2 (id, value2) VALUES (1, 'val21'); +INSERT INTO t3 (id, value3) VALUES (1, 'val31'); + +CREATE VIEW IF NOT EXISTS view1 AS SELECT t1.id AS id, t1.value1 AS value1, t2.value2 AS value2, t3.value3 AS value3 FROM t1 LEFT JOIN t2 ON t1.id = t2.id LEFT JOIN t3 ON t1.id = t3.id WHERE t1.id > 0; +SELECT * FROM view1 WHERE id = 1; + +-- https://github.com/ClickHouse/ClickHouse/issues/7136 +ANALYZE SELECT ccc FROM ( SELECT 1 AS ccc UNION ALL SELECT * FROM ( SELECT 2 AS ccc ) ANY INNER JOIN ( SELECT 2 AS ccc ) USING (ccc) ) WHERE ccc > 1; +SELECT ccc FROM ( SELECT 1 AS ccc UNION ALL SELECT * FROM ( SELECT 2 AS ccc ) ANY INNER JOIN ( SELECT 2 AS ccc ) USING (ccc) ) WHERE ccc > 1; + +-- https://github.com/ClickHouse/ClickHouse/issues/5674 +-- https://github.com/ClickHouse/ClickHouse/issues/4731 +-- https://github.com/ClickHouse/ClickHouse/issues/4904 +CREATE TABLE A (ts DateTime, id String, id_b String) ENGINE = MergeTree PARTITION BY toStartOfHour(ts) ORDER BY (ts,id); +CREATE TABLE B (ts DateTime, id String, id_c String) ENGINE = MergeTree PARTITION BY toStartOfHour(ts) ORDER BY (ts,id); + +ANALYZE SELECT ts, id, id_b, b.ts, b.id, id_c FROM (SELECT ts, id, id_b FROM A) AS a ALL LEFT JOIN B AS b ON b.id = a.id_b WHERE a.ts <= toDateTime('1970-01-01 03:00:00'); +ANALYZE SELECT ts AS `--a.ts`, id AS `--a.id`, id_b AS `--a.id_b`, b.ts AS `--b.ts`, b.id AS `--b.id`, id_c AS `--b.id_c` FROM (SELECT ts, id, id_b FROM A) AS a ALL LEFT JOIN B AS b ON `--b.id` = `--a.id_b` WHERE `--a.ts` <= toDateTime('1970-01-01 03:00:00'); + + From ce6e13549a0ff5f6176979bc8eb665e72327832d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 6 Jan 2020 12:18:56 +0800 Subject: [PATCH 08/32] fix neighbor is stateful function --- dbms/src/Functions/neighbor.cpp | 2 ++ .../01056_predicate_optimizer_bugs.reference | 8 +++++ .../01056_predicate_optimizer_bugs.sql | 29 +++++++++++++++++++ 3 files changed, 39 insertions(+) diff --git a/dbms/src/Functions/neighbor.cpp b/dbms/src/Functions/neighbor.cpp index 0253aed65d3..c37a3313a80 100644 --- a/dbms/src/Functions/neighbor.cpp +++ b/dbms/src/Functions/neighbor.cpp @@ -40,6 +40,8 @@ public: bool isVariadic() const override { return true; } + bool isStateful() const override { return true; } + bool isDeterministic() const override { return false; } bool isDeterministicInScopeOfQuery() const override { return false; } diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference index 289639d35ab..932651741e3 100644 --- a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference @@ -16,3 +16,11 @@ SELECT ccc\nFROM \n(\n SELECT 1 AS ccc\n WHERE 0\n UNION ALL\n SELEC 2 SELECT \n ts, \n id, \n id_b, \n b.ts, \n b.id, \n id_c\nFROM \n(\n SELECT \n ts, \n id, \n id_b\n FROM A\n WHERE ts <= toDateTime(\'1970-01-01 03:00:00\')\n) AS a\nALL LEFT JOIN B AS b ON b.id = id_b\nWHERE ts <= toDateTime(\'1970-01-01 03:00:00\') SELECT \n ts AS `--a.ts`, \n id AS `--a.id`, \n id_b AS `--a.id_b`, \n b.ts AS `--b.ts`, \n b.id AS `--b.id`, \n id_c AS `--b.id_c`\nFROM \n(\n SELECT \n ts, \n id, \n id_b\n FROM A\n WHERE ts <= toDateTime(\'1970-01-01 03:00:00\')\n) AS a\nALL LEFT JOIN B AS b ON `--b.id` = `--a.id_b`\nWHERE `--a.ts` <= toDateTime(\'1970-01-01 03:00:00\') +2 3 +3 4 +4 5 +5 0 +2 4 +4 0 +2 3 +4 5 diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql index 301ad2a707b..c6a79924291 100644 --- a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql @@ -18,6 +18,11 @@ ANALYZE SELECT alias AS name FROM ( SELECT name AS alias FROM system.settings ) SELECT alias AS name FROM ( SELECT name AS alias FROM system.settings ) ANY INNER JOIN ( SELECT name FROM system.settings ) USING (name) WHERE name = 'enable_optimize_predicate_expression'; -- https://github.com/ClickHouse/ClickHouse/issues/6767 +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS t3; +DROP TABLE IF EXISTS view1; + CREATE TABLE t1 (id UInt32, value1 String ) ENGINE ReplacingMergeTree() ORDER BY id; CREATE TABLE t2 (id UInt32, value2 String ) ENGINE ReplacingMergeTree() ORDER BY id; CREATE TABLE t3 (id UInt32, value3 String ) ENGINE ReplacingMergeTree() ORDER BY id; @@ -29,6 +34,11 @@ INSERT INTO t3 (id, value3) VALUES (1, 'val31'); CREATE VIEW IF NOT EXISTS view1 AS SELECT t1.id AS id, t1.value1 AS value1, t2.value2 AS value2, t3.value3 AS value3 FROM t1 LEFT JOIN t2 ON t1.id = t2.id LEFT JOIN t3 ON t1.id = t3.id WHERE t1.id > 0; SELECT * FROM view1 WHERE id = 1; +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS t3; +DROP TABLE IF EXISTS view1; + -- https://github.com/ClickHouse/ClickHouse/issues/7136 ANALYZE SELECT ccc FROM ( SELECT 1 AS ccc UNION ALL SELECT * FROM ( SELECT 2 AS ccc ) ANY INNER JOIN ( SELECT 2 AS ccc ) USING (ccc) ) WHERE ccc > 1; SELECT ccc FROM ( SELECT 1 AS ccc UNION ALL SELECT * FROM ( SELECT 2 AS ccc ) ANY INNER JOIN ( SELECT 2 AS ccc ) USING (ccc) ) WHERE ccc > 1; @@ -36,10 +46,29 @@ SELECT ccc FROM ( SELECT 1 AS ccc UNION ALL SELECT * FROM ( SELECT 2 AS ccc ) AN -- https://github.com/ClickHouse/ClickHouse/issues/5674 -- https://github.com/ClickHouse/ClickHouse/issues/4731 -- https://github.com/ClickHouse/ClickHouse/issues/4904 +DROP TABLE IF EXISTS A; +DROP TABLE IF EXISTS B; + CREATE TABLE A (ts DateTime, id String, id_b String) ENGINE = MergeTree PARTITION BY toStartOfHour(ts) ORDER BY (ts,id); CREATE TABLE B (ts DateTime, id String, id_c String) ENGINE = MergeTree PARTITION BY toStartOfHour(ts) ORDER BY (ts,id); ANALYZE SELECT ts, id, id_b, b.ts, b.id, id_c FROM (SELECT ts, id, id_b FROM A) AS a ALL LEFT JOIN B AS b ON b.id = a.id_b WHERE a.ts <= toDateTime('1970-01-01 03:00:00'); ANALYZE SELECT ts AS `--a.ts`, id AS `--a.id`, id_b AS `--a.id_b`, b.ts AS `--b.ts`, b.id AS `--b.id`, id_c AS `--b.id_c` FROM (SELECT ts, id, id_b FROM A) AS a ALL LEFT JOIN B AS b ON `--b.id` = `--a.id_b` WHERE `--a.ts` <= toDateTime('1970-01-01 03:00:00'); +DROP TABLE IF EXISTS A; +DROP TABLE IF EXISTS B; + +-- https://github.com/ClickHouse/ClickHouse/issues/7802 +DROP TABLE IF EXISTS test; + +CREATE TABLE test ( A Int32, B Int32 ) ENGINE = Memory(); + +INSERT INTO test VALUES(1, 2)(0, 3)(1, 4)(0, 5); + +SELECT B, neighbor(B, 1) AS next_B FROM (SELECT * FROM test ORDER BY B); +SELECT B, neighbor(B, 1) AS next_B FROM (SELECT * FROM test ORDER BY B) WHERE A == 1; +SELECT B, next_B FROM (SELECT A, B, neighbor(B, 1) AS next_B FROM (SELECT * FROM test ORDER BY B)) WHERE A == 1; + +DROP TABLE IF EXISTS test; + From 6d3e56def4fa50c6d94e9517dfabb4dc1aec8abf Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 6 Jan 2020 18:07:41 +0800 Subject: [PATCH 09/32] fix unbundled failure test --- .../0_stateless/01056_predicate_optimizer_bugs.reference | 2 +- .../queries/0_stateless/01056_predicate_optimizer_bugs.sql | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference index 932651741e3..3e1791fabb8 100644 --- a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference @@ -1,6 +1,6 @@ SELECT count()\nFROM \n(\n SELECT \n a, \n runningDifference(a) AS rd\n FROM \n (\n SELECT a\n FROM \n (\n SELECT 1 AS a\n UNION ALL\n SELECT 2 AS a\n )\n ORDER BY a DESC\n )\n WHERE rd = -1\n) 1 -SELECT \n k, \n v, \n d, \n i\nFROM \n(\n SELECT \n t.1 AS k, \n t.2 AS v, \n runningDifference(v) AS d, \n runningDifference(xxHash32(t.1)) AS i\n FROM \n (\n SELECT arrayJoin([(\'a\', 1), (\'a\', 2), (\'a\', 3), (\'b\', 11), (\'b\', 13), (\'b\', 15)]) AS t\n )\n)\nWHERE i = 0 +SELECT \n k, \n v, \n d, \n i\nFROM \n(\n SELECT \n t.1 AS k, \n t.2 AS v, \n runningDifference(v) AS d, \n runningDifference(cityHash64(t.1)) AS i\n FROM \n (\n SELECT arrayJoin([(\'a\', 1), (\'a\', 2), (\'a\', 3), (\'b\', 11), (\'b\', 13), (\'b\', 15)]) AS t\n )\n)\nWHERE i = 0 a 1 0 0 a 2 1 0 a 3 1 0 diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql index c6a79924291..59943742b3c 100644 --- a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql @@ -6,8 +6,8 @@ SET enable_optimize_predicate_expression = 1; ANALYZE SELECT count() FROM (SELECT a, runningDifference(a) rd FROM ( SELECT * FROM ( SELECT 1 a UNION ALL SELECT 2 a ) ORDER BY a DESC ) WHERE rd=-1 ); SELECT count() FROM (SELECT a, runningDifference(a) rd FROM ( SELECT * FROM ( SELECT 1 a UNION ALL SELECT 2 a ) ORDER BY a DESC ) WHERE rd=-1 ); -ANALYZE SELECT k, v, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(xxHash32(t.1)) AS i FROM ( SELECT arrayJoin([('a', 1), ('a', 2), ('a', 3), ('b', 11), ('b', 13), ('b', 15)]) AS t)) WHERE i = 0; -SELECT k, v, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(xxHash32(t.1)) AS i FROM ( SELECT arrayJoin([('a', 1), ('a', 2), ('a', 3), ('b', 11), ('b', 13), ('b', 15)]) AS t)) WHERE i = 0; +ANALYZE SELECT k, v, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(cityHash64(t.1)) AS i FROM ( SELECT arrayJoin([('a', 1), ('a', 2), ('a', 3), ('b', 11), ('b', 13), ('b', 15)]) AS t)) WHERE i = 0; +SELECT k, v, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(cityHash64(t.1)) AS i FROM ( SELECT arrayJoin([('a', 1), ('a', 2), ('a', 3), ('b', 11), ('b', 13), ('b', 15)]) AS t)) WHERE i = 0; -- https://github.com/ClickHouse/ClickHouse/issues/5682 ANALYZE SELECT co,co2,co3,num FROM ( SELECT co,co2,co3,count() AS num FROM ( SELECT 1 AS co,2 AS co2 ,3 AS co3 ) GROUP BY cube (co,co2,co3) ) WHERE co!=0 AND co2 !=2; From 2179935f1e7ccb772f3e89b1988b0962557e5e1f Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 6 Jan 2020 18:33:08 +0800 Subject: [PATCH 10/32] fix lambda function in predicate --- dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp | 6 ++++++ dbms/src/Interpreters/ExtractExpressionInfoVisitor.h | 2 +- dbms/src/Interpreters/PredicateRewriteVisitor.cpp | 9 +++++---- .../0_stateless/01056_predicate_optimizer_bugs.reference | 2 ++ .../0_stateless/01056_predicate_optimizer_bugs.sql | 2 ++ 5 files changed, 16 insertions(+), 5 deletions(-) diff --git a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp index 2bf954a7f9e..1240b6a09d6 100644 --- a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp +++ b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB @@ -55,6 +56,11 @@ void ExpressionInfoMatcher::visit(const ASTIdentifier & identifier, const ASTPtr } } +bool ExpressionInfoMatcher::needChildVisit(const ASTPtr & node, const ASTPtr &) +{ + return !node->as(); +} + bool hasStatefulFunction(const ASTPtr & node, const Context & context) { for (const auto & select_expression : node->children) diff --git a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.h b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.h index e884a27775a..65d23057e52 100644 --- a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.h +++ b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.h @@ -26,7 +26,7 @@ struct ExpressionInfoMatcher static void visit(const ASTPtr & ast, Data & data); - static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } + static bool needChildVisit(const ASTPtr & node, const ASTPtr &); static void visit(const ASTFunction & ast_function, const ASTPtr &, Data & data); diff --git a/dbms/src/Interpreters/PredicateRewriteVisitor.cpp b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp index 781c1510679..2575050825f 100644 --- a/dbms/src/Interpreters/PredicateRewriteVisitor.cpp +++ b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp @@ -101,10 +101,11 @@ bool PredicateRewriteVisitorData::rewriteSubquery(ASTSelectQuery & subquery, con const auto & column_name = identifiers[index]->shortName(); const auto & outer_column_iterator = std::find(outer_columns.begin(), outer_columns.end(), column_name); - if (outer_column_iterator == outer_columns.end()) - throw Exception("LOGICAL ERROR: the column " + column_name + " does not exists.", ErrorCodes::LOGICAL_ERROR); - - identifiers[index]->setShortName(inner_columns[outer_column_iterator - outer_columns.begin()]); + if (outer_column_iterator != outer_columns.end()) + { + /// Some temporary identifiers may be included in the predicate, for example: WHERE arrayMap(x -> x, [column_a]) = [column_a] + identifiers[index]->setShortName(inner_columns[outer_column_iterator - outer_columns.begin()]); + } } /// We only need to push all the predicates to subquery having diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference index 3e1791fabb8..34cab3eb9b8 100644 --- a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference @@ -24,3 +24,5 @@ SELECT \n ts AS `--a.ts`, \n id AS `--a.id`, \n id_b AS `--a.id_b`, \n 4 0 2 3 4 5 +SELECT dummy\nFROM \n(\n SELECT dummy\n FROM system.one\n WHERE arrayMap(x -> (x + 1), [dummy]) = [1]\n)\nWHERE arrayMap(x -> (x + 1), [dummy]) = [1] +0 diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql index 59943742b3c..b0da6a2d097 100644 --- a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql @@ -71,4 +71,6 @@ SELECT B, next_B FROM (SELECT A, B, neighbor(B, 1) AS next_B FROM (SELECT * FROM DROP TABLE IF EXISTS test; +ANALYZE SELECT * FROM (SELECT * FROM system.one) WHERE arrayMap(x -> x + 1, [dummy]) = [1]; +SELECT * FROM (SELECT * FROM system.one) WHERE arrayMap(x -> x + 1, [dummy]) = [1]; From 0f147e393f3bcab0984e593dc1d26624e3c0fdfa Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 6 Jan 2020 18:56:17 +0800 Subject: [PATCH 11/32] better skip lambda function --- dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp | 3 +++ dbms/src/Interpreters/PredicateRewriteVisitor.cpp | 10 ++++------ 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp index 1240b6a09d6..a5bcea73aac 100644 --- a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp +++ b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp @@ -58,6 +58,9 @@ void ExpressionInfoMatcher::visit(const ASTIdentifier & identifier, const ASTPtr bool ExpressionInfoMatcher::needChildVisit(const ASTPtr & node, const ASTPtr &) { + if (const auto & function = node->as(); function && function->name == "lambda") + return false; + return !node->as(); } diff --git a/dbms/src/Interpreters/PredicateRewriteVisitor.cpp b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp index 2575050825f..ec2088254c6 100644 --- a/dbms/src/Interpreters/PredicateRewriteVisitor.cpp +++ b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp @@ -68,13 +68,14 @@ void PredicateRewriteVisitorData::visitOtherInternalSelect(ASTSelectQuery & sele static void cleanAliasAndCollectIdentifiers(ASTPtr & predicate, std::vector & identifiers) { - for (auto & children : predicate->children) + /// skip WHERE x in (SELECT ...) AND arrayMap(x -> x, [column_a]) + const auto & function = predicate->as(); + if (!predicate->as() && !(function && function->name == "lambda")) { - if (!children->as()) /// skip where x in (SELECT ...) + for (auto & children : predicate->children) cleanAliasAndCollectIdentifiers(children, identifiers); } - if (const auto alias = predicate->tryGetAlias(); !alias.empty()) predicate->setAlias(""); @@ -102,10 +103,7 @@ bool PredicateRewriteVisitorData::rewriteSubquery(ASTSelectQuery & subquery, con const auto & outer_column_iterator = std::find(outer_columns.begin(), outer_columns.end(), column_name); if (outer_column_iterator != outer_columns.end()) - { - /// Some temporary identifiers may be included in the predicate, for example: WHERE arrayMap(x -> x, [column_a]) = [column_a] identifiers[index]->setShortName(inner_columns[outer_column_iterator - outer_columns.begin()]); - } } /// We only need to push all the predicates to subquery having From f0d2ef5bf7a1dd326965dfa1ee5e256e9497ba92 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Mon, 6 Jan 2020 08:23:05 -0500 Subject: [PATCH 12/32] Removing tabs. --- dbms/src/Storages/LiveView/StorageLiveView.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/LiveView/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp index 75783364247..eae8eaa1d3c 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -200,7 +200,7 @@ void StorageLiveView::writeIntoLiveView( if (live_view.getInnerSubQuery()) mergeable_query = live_view.getInnerSubQuery(); - BlockInputStreams streams = {std::make_shared(block)}; + BlockInputStreams streams = {std::make_shared(block)}; auto blocks_storage = StorageBlocks::createStorage(live_view.database_name, live_view.table_name, live_view.getParentStorage()->getColumns(), std::move(streams), QueryProcessingStage::FetchColumns); From b5035e8924b071d3cb11ef4efb519f30b4ef081f Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 7 Jan 2020 11:31:03 +0800 Subject: [PATCH 13/32] no skip lambda function --- dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp | 3 --- dbms/src/Interpreters/PredicateRewriteVisitor.cpp | 7 ++++--- .../0_stateless/01056_predicate_optimizer_bugs.reference | 2 ++ .../queries/0_stateless/01056_predicate_optimizer_bugs.sql | 2 ++ 4 files changed, 8 insertions(+), 6 deletions(-) diff --git a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp index a5bcea73aac..1240b6a09d6 100644 --- a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp +++ b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp @@ -58,9 +58,6 @@ void ExpressionInfoMatcher::visit(const ASTIdentifier & identifier, const ASTPtr bool ExpressionInfoMatcher::needChildVisit(const ASTPtr & node, const ASTPtr &) { - if (const auto & function = node->as(); function && function->name == "lambda") - return false; - return !node->as(); } diff --git a/dbms/src/Interpreters/PredicateRewriteVisitor.cpp b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp index ec2088254c6..6bd16ddc066 100644 --- a/dbms/src/Interpreters/PredicateRewriteVisitor.cpp +++ b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp @@ -68,9 +68,8 @@ void PredicateRewriteVisitorData::visitOtherInternalSelect(ASTSelectQuery & sele static void cleanAliasAndCollectIdentifiers(ASTPtr & predicate, std::vector & identifiers) { - /// skip WHERE x in (SELECT ...) AND arrayMap(x -> x, [column_a]) - const auto & function = predicate->as(); - if (!predicate->as() && !(function && function->name == "lambda")) + /// Skip WHERE x in (SELECT ...) + if (!predicate->as()) { for (auto & children : predicate->children) cleanAliasAndCollectIdentifiers(children, identifiers); @@ -102,6 +101,8 @@ bool PredicateRewriteVisitorData::rewriteSubquery(ASTSelectQuery & subquery, con const auto & column_name = identifiers[index]->shortName(); const auto & outer_column_iterator = std::find(outer_columns.begin(), outer_columns.end(), column_name); + /// For lambda functions, we can't always find them in the list of columns + /// For example: SELECT * FROM system.one WHERE arrayMap(x -> x, [dummy]) = [0] if (outer_column_iterator != outer_columns.end()) identifiers[index]->setShortName(inner_columns[outer_column_iterator - outer_columns.begin()]); } diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference index 34cab3eb9b8..50ee6cba252 100644 --- a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference @@ -26,3 +26,5 @@ SELECT \n ts AS `--a.ts`, \n id AS `--a.id`, \n id_b AS `--a.id_b`, \n 4 5 SELECT dummy\nFROM \n(\n SELECT dummy\n FROM system.one\n WHERE arrayMap(x -> (x + 1), [dummy]) = [1]\n)\nWHERE arrayMap(x -> (x + 1), [dummy]) = [1] 0 +SELECT \n id, \n value, \n value_1\nFROM \n(\n SELECT \n 1 AS id, \n 2 AS value\n)\nALL INNER JOIN \n(\n SELECT \n 1 AS id, \n 3 AS value_1\n) USING (id)\nWHERE arrayMap(x -> ((x + value) + value_1), [1]) = [6] +1 2 3 diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql index b0da6a2d097..549402243e7 100644 --- a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql @@ -74,3 +74,5 @@ DROP TABLE IF EXISTS test; ANALYZE SELECT * FROM (SELECT * FROM system.one) WHERE arrayMap(x -> x + 1, [dummy]) = [1]; SELECT * FROM (SELECT * FROM system.one) WHERE arrayMap(x -> x + 1, [dummy]) = [1]; +ANALYZE SELECT * FROM (SELECT 1 AS id, 2 AS value) INNER JOIN (SELECT 1 AS id, 3 AS value_1) USING id WHERE arrayMap(x -> x + value + value_1, [1]) = [6]; +SELECT * FROM (SELECT 1 AS id, 2 AS value) INNER JOIN (SELECT 1 AS id, 3 AS value_1) USING id WHERE arrayMap(x -> x + value + value_1, [1]) = [6]; From 776ceeb427ba461dcac4b19819df9959d5454b78 Mon Sep 17 00:00:00 2001 From: Javi santana bot Date: Tue, 7 Jan 2020 19:16:37 +0100 Subject: [PATCH 14/32] Wait only on optimize commands --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 8 ++++---- dbms/src/Storages/StorageReplicatedMergeTree.h | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 88c731fb88f..e0fe6d5708d 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3141,7 +3141,7 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p { /// NOTE Table lock must not be held while waiting. Some combination of R-W-R locks from different threads will yield to deadlock. for (auto & merge_entry : merge_entries) - waitForAllReplicasToProcessLogEntry(merge_entry); + waitForAllReplicasToProcessLogEntry(merge_entry, false); } return true; @@ -3840,15 +3840,15 @@ StorageReplicatedMergeTree::allocateBlockNumber( } -void StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry(const ReplicatedMergeTreeLogEntryData & entry) +void StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry(const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active) { LOG_DEBUG(log, "Waiting for all replicas to process " << entry.znode_name); - auto zookeeper = getZooKeeper() + auto zookeeper = getZooKeeper(); Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); for (const String & replica : replicas) { - if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")) + if (wait_for_non_active || zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")) { waitForReplicaToProcessLogEntry(replica, entry); } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 3727d5de0d8..c779b9a87d3 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -486,7 +486,7 @@ private: * Because it effectively waits for other thread that usually has to also acquire a lock to proceed and this yields deadlock. * TODO: There are wrong usages of this method that are not fixed yet. */ - void waitForAllReplicasToProcessLogEntry(const ReplicatedMergeTreeLogEntryData & entry); + void waitForAllReplicasToProcessLogEntry(const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active = true); /** Wait until the specified replica executes the specified action from the log. * NOTE: See comment about locks above. From 04f28e3aab7626dbd0012eb25116afb87b149af7 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 8 Jan 2020 11:22:26 +0800 Subject: [PATCH 15/32] remove union failure test --- .../0_stateless/01056_predicate_optimizer_bugs.reference | 2 -- .../queries/0_stateless/01056_predicate_optimizer_bugs.sql | 3 --- 2 files changed, 5 deletions(-) diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference index 50ee6cba252..019e95cb359 100644 --- a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference @@ -1,5 +1,3 @@ -SELECT count()\nFROM \n(\n SELECT \n a, \n runningDifference(a) AS rd\n FROM \n (\n SELECT a\n FROM \n (\n SELECT 1 AS a\n UNION ALL\n SELECT 2 AS a\n )\n ORDER BY a DESC\n )\n WHERE rd = -1\n) -1 SELECT \n k, \n v, \n d, \n i\nFROM \n(\n SELECT \n t.1 AS k, \n t.2 AS v, \n runningDifference(v) AS d, \n runningDifference(cityHash64(t.1)) AS i\n FROM \n (\n SELECT arrayJoin([(\'a\', 1), (\'a\', 2), (\'a\', 3), (\'b\', 11), (\'b\', 13), (\'b\', 15)]) AS t\n )\n)\nWHERE i = 0 a 1 0 0 a 2 1 0 diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql index 549402243e7..e1e185be076 100644 --- a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql @@ -3,9 +3,6 @@ SET enable_optimize_predicate_expression = 1; -- https://github.com/ClickHouse/ClickHouse/issues/3885 -- https://github.com/ClickHouse/ClickHouse/issues/5485 -ANALYZE SELECT count() FROM (SELECT a, runningDifference(a) rd FROM ( SELECT * FROM ( SELECT 1 a UNION ALL SELECT 2 a ) ORDER BY a DESC ) WHERE rd=-1 ); -SELECT count() FROM (SELECT a, runningDifference(a) rd FROM ( SELECT * FROM ( SELECT 1 a UNION ALL SELECT 2 a ) ORDER BY a DESC ) WHERE rd=-1 ); - ANALYZE SELECT k, v, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(cityHash64(t.1)) AS i FROM ( SELECT arrayJoin([('a', 1), ('a', 2), ('a', 3), ('b', 11), ('b', 13), ('b', 15)]) AS t)) WHERE i = 0; SELECT k, v, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(cityHash64(t.1)) AS i FROM ( SELECT arrayJoin([('a', 1), ('a', 2), ('a', 3), ('b', 11), ('b', 13), ('b', 15)]) AS t)) WHERE i = 0; From 2084d262389570f746fdd48eca89b63b8da83b6a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Jan 2020 13:20:55 +0300 Subject: [PATCH 16/32] Removed useless code, added comments --- dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index da3f1df8130..8041ad4dbe7 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -636,15 +636,16 @@ bool SplitTokenExtractor::next(const char * data, size_t len, size_t * pos, size { if (isASCII(data[*pos]) && !isAlphaNumericASCII(data[*pos])) { + /// Finish current token if any if (*token_len > 0) return true; *token_start = ++*pos; } else { - const size_t sz = UTF8::seqLength(static_cast(data[*pos])); - *pos += sz; - *token_len += sz; + /// Note that UTF-8 sequence is completely consisted of non-ASCII bytes. + ++*pos; + ++*token_len; } } return *token_len > 0; From da2783f21c021fbe747b9cfa8a277146ed7c7991 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Wed, 8 Jan 2020 17:35:28 +0300 Subject: [PATCH 17/32] Increased number of rows to make tests results noticeable. --- dbms/tests/performance/codecs_float_insert.xml | 12 +++++++++--- dbms/tests/performance/codecs_float_select.xml | 14 ++++++++++---- dbms/tests/performance/codecs_int_insert.xml | 12 +++++++++--- dbms/tests/performance/codecs_int_select.xml | 14 ++++++++++---- 4 files changed, 38 insertions(+), 14 deletions(-) diff --git a/dbms/tests/performance/codecs_float_insert.xml b/dbms/tests/performance/codecs_float_insert.xml index 2a39dfc48d6..0979ff79cce 100644 --- a/dbms/tests/performance/codecs_float_insert.xml +++ b/dbms/tests/performance/codecs_float_insert.xml @@ -37,14 +37,20 @@ rnd + + num_rows + + 10000000 + + CREATE TABLE IF NOT EXISTS codec_{seq_type}_{type}_{codec} (n {type} CODEC({codec})) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple(); - INSERT INTO codec_seq_Float64_{codec} (n) SELECT number/pi() FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 - INSERT INTO codec_mon_Float64_{codec} (n) SELECT number+sin(number) FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 - INSERT INTO codec_rnd_Float64_{codec} (n) SELECT (rand() - 4294967295)/pi() FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 + INSERT INTO codec_seq_Float64_{codec} (n) SELECT number/pi() FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 + INSERT INTO codec_mon_Float64_{codec} (n) SELECT number+sin(number) FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 + INSERT INTO codec_rnd_Float64_{codec} (n) SELECT (rand() - 4294967295)/pi() FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 DROP TABLE IF EXISTS codec_{seq_type}_{type}_{codec} diff --git a/dbms/tests/performance/codecs_float_select.xml b/dbms/tests/performance/codecs_float_select.xml index f23b363b914..db023f9f2e5 100644 --- a/dbms/tests/performance/codecs_float_select.xml +++ b/dbms/tests/performance/codecs_float_select.xml @@ -37,18 +37,24 @@ rnd + + num_rows + + 10000000 + + CREATE TABLE IF NOT EXISTS codec_{seq_type}_{type}_{codec} (n {type} CODEC({codec})) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple(); - INSERT INTO codec_seq_Float64_{codec} (n) SELECT number/pi() FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 - INSERT INTO codec_mon_Float64_{codec} (n) SELECT number+sin(number) FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 - INSERT INTO codec_rnd_Float64_{codec} (n) SELECT (rand() - 4294967295)/pi() FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 + INSERT INTO codec_seq_Float64_{codec} (n) SELECT number/pi() FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 + INSERT INTO codec_mon_Float64_{codec} (n) SELECT number+sin(number) FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 + INSERT INTO codec_rnd_Float64_{codec} (n) SELECT (rand() - 4294967295)/pi() FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 - SELECT count(n) FROM codec_{seq_type}_{type}_{codec} WHERE ignore(n) LIMIT 100000 SETTINGS max_threads=1 + SELECT count(n) FROM codec_{seq_type}_{type}_{codec} WHERE ignore(n) LIMIT {num_rows} SETTINGS max_threads=1 DROP TABLE IF EXISTS codec_{seq_type}_{type}_{codec} diff --git a/dbms/tests/performance/codecs_int_insert.xml b/dbms/tests/performance/codecs_int_insert.xml index 742693d49fe..1d3037cecb7 100644 --- a/dbms/tests/performance/codecs_int_insert.xml +++ b/dbms/tests/performance/codecs_int_insert.xml @@ -39,14 +39,20 @@ rnd + + num_rows + + 10000000 + + CREATE TABLE IF NOT EXISTS codec_{seq_type}_{type}_{codec} (n {type} CODEC({codec})) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple(); - INSERT INTO codec_seq_UInt64_{codec} (n) SELECT number FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 - INSERT INTO codec_mon_UInt64_{codec} (n) SELECT number*512+(rand()%512) FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 - INSERT INTO codec_rnd_UInt64_{codec} (n) SELECT rand() FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 + INSERT INTO codec_seq_UInt64_{codec} (n) SELECT number FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 + INSERT INTO codec_mon_UInt64_{codec} (n) SELECT number*512+(rand()%512) FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 + INSERT INTO codec_rnd_UInt64_{codec} (n) SELECT rand() FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 DROP TABLE IF EXISTS codec_{seq_type}_{type}_{codec} diff --git a/dbms/tests/performance/codecs_int_select.xml b/dbms/tests/performance/codecs_int_select.xml index 9c007863cd8..b3fc0834c2d 100644 --- a/dbms/tests/performance/codecs_int_select.xml +++ b/dbms/tests/performance/codecs_int_select.xml @@ -39,18 +39,24 @@ rnd + + num_rows + + 10000000 + + CREATE TABLE IF NOT EXISTS codec_{seq_type}_{type}_{codec} (n {type} CODEC({codec})) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple(); - INSERT INTO codec_seq_UInt64_{codec} (n) SELECT number FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 - INSERT INTO codec_mon_UInt64_{codec} (n) SELECT number*512+(rand()%512) FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 - INSERT INTO codec_rnd_UInt64_{codec} (n) SELECT rand() FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 + INSERT INTO codec_seq_UInt64_{codec} (n) SELECT number FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 + INSERT INTO codec_mon_UInt64_{codec} (n) SELECT number*512+(rand()%512) FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 + INSERT INTO codec_rnd_UInt64_{codec} (n) SELECT rand() FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 - SELECT count(n) FROM codec_{seq_type}_{type}_{codec} WHERE ignore(n) LIMIT 100000 SETTINGS max_threads=1 + SELECT count(n) FROM codec_{seq_type}_{type}_{codec} WHERE ignore(n) LIMIT {num_rows} SETTINGS max_threads=1 DROP TABLE IF EXISTS codec_{seq_type}_{type}_{codec} From 5a723fa76958e8ae4b6f1934e5172df1867bbe2f Mon Sep 17 00:00:00 2001 From: Dmitrii Raev Date: Wed, 8 Jan 2020 16:56:40 +0200 Subject: [PATCH 18/32] Update apply_catboost_model.md Fix CreateTable SQL. Storage MergeTree requires 3 to 4 parameters --- docs/ru/guides/apply_catboost_model.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/guides/apply_catboost_model.md b/docs/ru/guides/apply_catboost_model.md index 9f93aacbd22..65cb3f58508 100644 --- a/docs/ru/guides/apply_catboost_model.md +++ b/docs/ru/guides/apply_catboost_model.md @@ -74,7 +74,7 @@ $ clickhouse client ROLE_FAMILY UInt32, ROLE_CODE UInt32 ) -ENGINE = MergeTree() +ENGINE = MergeTree(date, date, 8192) ``` **3.** Выйдите из клиента ClickHouse: @@ -227,4 +227,4 @@ FROM ``` !!! note "Примечание" - Подробнее про функции [avg()](../query_language/agg_functions/reference.md#agg_function-avg), [log()](../query_language/functions/math_functions.md). \ No newline at end of file + Подробнее про функции [avg()](../query_language/agg_functions/reference.md#agg_function-avg), [log()](../query_language/functions/math_functions.md). From 746dd1a8b72dbaf0cde40d88434933c1513ec2a0 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 8 Jan 2020 22:03:10 +0700 Subject: [PATCH 19/32] PolicyInfo::setPolicy() now doesn't throw an excepion on empty filter. --- dbms/src/Access/RowPolicyContextFactory.cpp | 37 +++++++++++---------- 1 file changed, 19 insertions(+), 18 deletions(-) diff --git a/dbms/src/Access/RowPolicyContextFactory.cpp b/dbms/src/Access/RowPolicyContextFactory.cpp index e458f06ca94..77e5056e206 100644 --- a/dbms/src/Access/RowPolicyContextFactory.cpp +++ b/dbms/src/Access/RowPolicyContextFactory.cpp @@ -101,9 +101,6 @@ namespace public: void add(const ASTPtr & condition, bool is_restrictive) { - if (!condition) - return; - if (is_restrictive) restrictions.push_back(condition); else @@ -139,29 +136,32 @@ void RowPolicyContextFactory::PolicyInfo::setPolicy(const RowPolicyPtr & policy_ for (auto index : ext::range_with_static_cast(0, MAX_CONDITION_INDEX)) { + parsed_conditions[index] = nullptr; const String & condition = policy->conditions[index]; + if (condition.empty()) + continue; + auto previous_range = std::pair(std::begin(policy->conditions), std::begin(policy->conditions) + index); auto previous_it = std::find(previous_range.first, previous_range.second, condition); if (previous_it != previous_range.second) { /// The condition is already parsed before. parsed_conditions[index] = parsed_conditions[previous_it - previous_range.first]; + continue; } - else + + /// Try to parse the condition. + try { - /// Try to parse the condition. - try - { - ParserExpression parser; - parsed_conditions[index] = parseQuery(parser, condition, 0); - } - catch (...) - { - tryLogCurrentException( - &Poco::Logger::get("RowPolicy"), - String("Could not parse the condition ") + RowPolicy::conditionIndexToString(index) + " of row policy " - + backQuote(policy->getFullName())); - } + ParserExpression parser; + parsed_conditions[index] = parseQuery(parser, condition, 0); + } + catch (...) + { + tryLogCurrentException( + &Poco::Logger::get("RowPolicy"), + String("Could not parse the condition ") + RowPolicy::conditionIndexToString(index) + " of row policy " + + backQuote(policy->getFullName())); } } } @@ -290,7 +290,8 @@ void RowPolicyContextFactory::mixConditionsForContext(RowPolicyContext & context auto & mixers = map_of_mixers[std::pair{policy.getDatabase(), policy.getTableName()}]; mixers.policy_ids.push_back(policy_id); for (auto index : ext::range(0, MAX_CONDITION_INDEX)) - mixers.mixers[index].add(info.parsed_conditions[index], policy.isRestrictive()); + if (info.parsed_conditions[index]) + mixers.mixers[index].add(info.parsed_conditions[index], policy.isRestrictive()); } } From 189be696308b0874450ed932ce680c8bf011c1ce Mon Sep 17 00:00:00 2001 From: Dmitrii Raev Date: Wed, 8 Jan 2020 17:06:06 +0200 Subject: [PATCH 20/32] Update apply_catboost_model.md --- docs/en/guides/apply_catboost_model.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/guides/apply_catboost_model.md b/docs/en/guides/apply_catboost_model.md index 4665809bfa0..8329e4ed550 100644 --- a/docs/en/guides/apply_catboost_model.md +++ b/docs/en/guides/apply_catboost_model.md @@ -74,7 +74,7 @@ $ clickhouse client ROLE_FAMILY UInt32, ROLE_CODE UInt32 ) -ENGINE = MergeTree() +ENGINE = MergeTree(data, date, 8192) ``` **3.** Exit from ClickHouse console client: @@ -227,4 +227,4 @@ FROM ``` !!! note "Note" - More info about [avg()](../query_language/agg_functions/reference.md#agg_function-avg) and [log()](../query_language/functions/math_functions.md) functions. \ No newline at end of file + More info about [avg()](../query_language/agg_functions/reference.md#agg_function-avg) and [log()](../query_language/functions/math_functions.md) functions. From 79e9a1e3cbeec6b150f8c5c557aab4ba480fe89a Mon Sep 17 00:00:00 2001 From: Dmitrii Raev Date: Wed, 8 Jan 2020 17:07:01 +0200 Subject: [PATCH 21/32] Update apply_catboost_model.md Fix --- docs/en/guides/apply_catboost_model.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/guides/apply_catboost_model.md b/docs/en/guides/apply_catboost_model.md index 8329e4ed550..5e868bffe0d 100644 --- a/docs/en/guides/apply_catboost_model.md +++ b/docs/en/guides/apply_catboost_model.md @@ -74,7 +74,7 @@ $ clickhouse client ROLE_FAMILY UInt32, ROLE_CODE UInt32 ) -ENGINE = MergeTree(data, date, 8192) +ENGINE = MergeTree(date, date, 8192) ``` **3.** Exit from ClickHouse console client: From f56653f43ee40e626bb3b8e913735e60edf03d15 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 8 Jan 2020 14:13:14 +0800 Subject: [PATCH 22/32] ensure network be up before staring clickhouse --- debian/clickhouse-server.init | 4 ++-- debian/clickhouse-server.service | 2 ++ 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/debian/clickhouse-server.init b/debian/clickhouse-server.init index 32282756719..213da9affb9 100755 --- a/debian/clickhouse-server.init +++ b/debian/clickhouse-server.init @@ -3,8 +3,8 @@ # Provides: clickhouse-server # Default-Start: 2 3 4 5 # Default-Stop: 0 1 6 -# Required-Start: -# Required-Stop: +# Required-Start: $network +# Required-Stop: $network # Short-Description: Yandex clickhouse-server daemon ### END INIT INFO diff --git a/debian/clickhouse-server.service b/debian/clickhouse-server.service index 4543b304197..b9681f9279e 100644 --- a/debian/clickhouse-server.service +++ b/debian/clickhouse-server.service @@ -1,5 +1,7 @@ [Unit] Description=ClickHouse Server (analytic DBMS for big data) +Requires=network-online.target +After=network-online.target [Service] Type=simple From 5ed2a93748277667d16a3cbc80064622dde151a4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Jan 2020 12:03:08 +0300 Subject: [PATCH 23/32] Allow multiple edges for processors pipeline. --- dbms/src/Processors/Executors/PipelineExecutor.cpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index bc0de1fb81d..70cd2e2405f 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -64,13 +64,6 @@ bool PipelineExecutor::addEdges(UInt64 node) throwUnknownProcessor(to_proc, cur, true); UInt64 proc_num = it->second; - - for (auto & edge : edges) - { - if (edge.to == proc_num) - throw Exception("Multiple edges are not allowed for the same processors.", ErrorCodes::LOGICAL_ERROR); - } - auto & edge = edges.emplace_back(proc_num, is_backward, input_port_number, output_port_number, update_list); from_port.setUpdateInfo(&edge.update_info); From e3f0ea526ae47eaff69f2d82af7cb112c76f41bf Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 9 Jan 2020 12:52:40 +0300 Subject: [PATCH 24/32] update security changelog --- docs/en/security_changelog.md | 23 +++++++++++++++++++++++ docs/ru/security_changelog.md | 22 +++++++++++++++++++++- 2 files changed, 44 insertions(+), 1 deletion(-) diff --git a/docs/en/security_changelog.md b/docs/en/security_changelog.md index 0847300cc19..92b35868f94 100644 --- a/docs/en/security_changelog.md +++ b/docs/en/security_changelog.md @@ -1,6 +1,27 @@ +## Fixed in ClickHouse Release 19.14.3.3, 2019-09-10 + +### CVE-2019-15024 + +Аn attacker having write access to ZooKeeper and who is able to run a custom server available from the network where ClickHouse runs, can create a custom-built malicious server that will act as a ClickHouse replica and register it in ZooKeeper. When another replica will fetch data part from the malicious replica, it can force clickhouse-server to write to arbitrary path on filesystem. + +Credits: Eldar Zaitov of Yandex Information Security Team + +### CVE-2019-16535 + +Аn OOB read, OOB write and integer underflow in decompression algorithms can be used to achieve RCE or DoS via native protocol. + +Credits: Eldar Zaitov of Yandex Information Security Team + +### CVE-2019-16536 + +Stack overflow leading to DoS can be triggered by malicious authenticated client. + +Credits: Eldar Zaitov of Yandex Information Security Team + ## Fixed in ClickHouse Release 19.13.6.1, 2019-09-20 ### CVE-2019-18657 + Table function `url` had the vulnerability allowed the attacker to inject arbitrary HTTP headers in the request. Credits: [Nikita Tikhomirov](https://github.com/NSTikhomirov) @@ -24,6 +45,7 @@ Credits: Andrey Krasichkov and Evgeny Sidorov of Yandex Information Security Tea ## Fixed in ClickHouse Release 1.1.54388, 2018-06-28 ### CVE-2018-14668 + "remote" table function allowed arbitrary symbols in "user", "password" and "default_database" fields which led to Cross Protocol Request Forgery Attacks. Credits: Andrey Krasichkov of Yandex Information Security Team @@ -31,6 +53,7 @@ Credits: Andrey Krasichkov of Yandex Information Security Team ## Fixed in ClickHouse Release 1.1.54390, 2018-07-06 ### CVE-2018-14669 + ClickHouse MySQL client had "LOAD DATA LOCAL INFILE" functionality enabled that allowed a malicious MySQL database read arbitrary files from the connected ClickHouse server. Credits: Andrey Krasichkov and Evgeny Sidorov of Yandex Information Security Team diff --git a/docs/ru/security_changelog.md b/docs/ru/security_changelog.md index 17ae1eba19d..4e588386af2 100644 --- a/docs/ru/security_changelog.md +++ b/docs/ru/security_changelog.md @@ -1,3 +1,23 @@ +## Исправлено в релизе 19.14.3.3, 2019-09-10 + +### CVE-2019-15024 + +Злоумышленник с доступом на запись к ZooKeeper и возможностью запустить собственный сервер в сети доступной ClickHouse может создать вредоносный сервер, который будет вести себя как реплика ClickHouse и зарегистрируется в ZooKeeper. В процессе репликации вредоносный сервер может указать любой путь на файловой системе в который будут записаны данные. + +Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса + +### CVE-2019-16535 + +Интерфейс декомпрессии позволял совершать OOB чтения и записи данных в памяти, а также переполнение целочисленных переменных, что могло приводить к отказу в обслуживании. Также потенциально могло использоваьтся для удаленного выполнения кода. + +Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса + +### CVE-2019-16536 + +Аутентифицированный клиент злоумышленника имел возможность вызвать переполнение стека, что могло привести к отказу в обслуживании. + +Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса + ## Исправлено в релизе 19.13.6.1 от 20 сентября 2019 ### CVE-2019-18657 @@ -19,7 +39,7 @@ unixODBC позволял указать путь для подключения Обнаружено благодаря: Андрею Красичкову и Евгению Сидорову из Службы Информационной Безопасности Яндекса -## Исправлено в релизе 1.1.54388 от 28 июня 2018 +## Исправлено в релизе 1.1.54388 от 28 июня 2018 ### CVE-2018-14668 Табличная функция "remote" допускала произвольные символы в полях "user", "password" и "default_database", что позволяло производить атаки класса Cross Protocol Request Forgery. From c637396791f40c4a9d6d914534ff74eaf0910fd7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Jan 2020 13:37:27 +0300 Subject: [PATCH 25/32] Added DelayedPortsProcessor. --- dbms/src/Processors/DelayedPortsProcessor.cpp | 95 +++++++++++++++++++ dbms/src/Processors/DelayedPortsProcessor.h | 37 ++++++++ 2 files changed, 132 insertions(+) create mode 100644 dbms/src/Processors/DelayedPortsProcessor.cpp create mode 100644 dbms/src/Processors/DelayedPortsProcessor.h diff --git a/dbms/src/Processors/DelayedPortsProcessor.cpp b/dbms/src/Processors/DelayedPortsProcessor.cpp new file mode 100644 index 00000000000..d5a033dc7df --- /dev/null +++ b/dbms/src/Processors/DelayedPortsProcessor.cpp @@ -0,0 +1,95 @@ +#include + +namespace DB +{ + +DelayedPortsProcessor::DelayedPortsProcessor(const Block & header, size_t num_ports, const PortNumbers & delayed_ports) + : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)) + , num_delayed(delayed_ports.size()) +{ + port_pairs.resize(num_ports); + + auto input_it = inputs.begin(); + auto output_it = outputs.begin(); + for (size_t i = 0; i < num_ports; ++i) + { + port_pairs[i].input_port = &*input_it; + port_pairs[i].output_port = &*output_it; + ++input_it; + ++output_it; + } + + for (auto & delayed : delayed_ports) + port_pairs[delayed].is_delayed = true; +} + +bool DelayedPortsProcessor::processPair(PortsPair & pair) +{ + auto finish = [&]() + { + if (!pair.is_finished) + { + pair.input_port->close(); + + pair.is_finished = true; + ++num_finished; + } + }; + + if (pair.output_port->isFinished()) + { + finish(); + return false; + } + + if (pair.input_port->isFinished()) + { + finish(); + return false; + } + + if (!pair.output_port->canPush()) + return false; + + pair.input_port->setNeeded(); + if (pair.input_port->hasData()) + pair.output_port->pushData(pair.input_port->pullData()); + + return true; +} + +IProcessor::Status DelayedPortsProcessor::prepare(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) +{ + bool skip_delayed = (num_finished + num_delayed) < port_pairs.size(); + bool need_data = false; + + for (auto & output_number : updated_outputs) + { + if (!skip_delayed || !port_pairs[output_number].is_delayed) + need_data = processPair(port_pairs[output_number]) || need_data; + } + + for (auto & input_number : updated_inputs) + { + if (!skip_delayed || !port_pairs[input_number].is_delayed) + need_data = processPair(port_pairs[input_number]) || need_data; + } + + /// In case if main streams are finished at current iteration, start processing delayed streams. + if (skip_delayed && (num_finished + num_delayed) >= port_pairs.size()) + { + for (auto & pair : port_pairs) + if (pair.is_delayed) + need_data = processPair(pair) || need_data; + } + + if (num_finished == port_pairs.size()) + return Status::Finished; + + if (need_data) + return Status::NeedData; + + return Status::PortFull; +} + +} diff --git a/dbms/src/Processors/DelayedPortsProcessor.h b/dbms/src/Processors/DelayedPortsProcessor.h new file mode 100644 index 00000000000..44dd632f8a8 --- /dev/null +++ b/dbms/src/Processors/DelayedPortsProcessor.h @@ -0,0 +1,37 @@ +#pragma once +#include + +namespace DB +{ + +/// Processor with N inputs and N outputs. Only moves data from i-th input to i-th output as is. +/// Some ports are delayed. Delayed ports are processed after other outputs are all finished. +/// Data between ports is not mixed. It is important because this processor can be used before MergingSortedTransform. +/// Delayed ports are appeared after joins, when some non-matched data need to be processed at the end. +class DelayedPortsProcessor : public IProcessor +{ +public: + DelayedPortsProcessor(const Block & header, size_t num_ports, const PortNumbers & delayed_ports); + + String getName() const override { return "DelayedPorts"; } + + Status prepare(const PortNumbers &, const PortNumbers &) override; + +private: + + struct PortsPair + { + InputPort * input_port = nullptr; + OutputPort * output_port = nullptr; + bool is_delayed = false; + bool is_finished = false; + }; + + std::vector port_pairs; + size_t num_delayed; + size_t num_finished = 0; + + bool processPair(PortsPair & pair); +}; + +} From 39b91584fd385ab62a185c54ecbb2260b5209ce5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Jan 2020 14:52:34 +0300 Subject: [PATCH 26/32] Simplifu delayed stream in QueryPipeline. --- .../Interpreters/InterpreterSelectQuery.cpp | 27 ++++++--------- dbms/src/Processors/QueryPipeline.cpp | 34 +++---------------- dbms/src/Processors/QueryPipeline.h | 9 ++--- 3 files changed, 17 insertions(+), 53 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index f5971d7edbf..b764eed3fc0 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -1180,7 +1180,6 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS if (expressions.second_stage) { bool need_second_distinct_pass = false; - bool need_merge_streams = false; if (expressions.need_aggregate) { @@ -1241,13 +1240,11 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS executePreLimit(pipeline); } - if (need_second_distinct_pass - || query.limitLength() - || query.limitBy() - || pipeline.hasDelayedStream()) - { - need_merge_streams = true; - } + bool need_merge_streams = need_second_distinct_pass || query.limitLength() || query.limitBy(); + + if constexpr (!pipeline_with_processors) + if (pipeline.hasDelayedStream()) + need_merge_streams = true; if (need_merge_streams) { @@ -1933,7 +1930,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const * 1. Parallel aggregation is done, and the results should be merged in parallel. * 2. An aggregation is done with store of temporary data on the disk, and they need to be merged in a memory efficient way. */ - bool allow_to_use_two_level_group_by = pipeline.getNumMainStreams() > 1 || settings.max_bytes_before_external_group_by != 0; + bool allow_to_use_two_level_group_by = pipeline.getNumStreams() > 1 || settings.max_bytes_before_external_group_by != 0; Aggregator::Params params(header_before_aggregation, keys, aggregates, overflow_row, settings.max_rows_to_group_by, settings.group_by_overflow_mode, @@ -1947,12 +1944,12 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const pipeline.dropTotalsIfHas(); /// If there are several sources, then we perform parallel aggregation - if (pipeline.getNumMainStreams() > 1) + if (pipeline.getNumStreams() > 1) { /// Add resize transform to uniformly distribute data between aggregating streams. - pipeline.resize(pipeline.getNumMainStreams(), true); + pipeline.resize(pipeline.getNumStreams(), true); - auto many_data = std::make_shared(pipeline.getNumMainStreams()); + auto many_data = std::make_shared(pipeline.getNumStreams()); auto merge_threads = settings.aggregation_memory_efficient_merge_threads ? static_cast(settings.aggregation_memory_efficient_merge_threads) : static_cast(settings.max_threads); @@ -2807,11 +2804,7 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(Pipeline & pipeline void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPipeline & pipeline, SubqueriesForSets & subqueries_for_sets) { if (query_info.input_sorting_info) - { - if (pipeline.hasDelayedStream()) - throw Exception("Using read in order optimization, but has delayed stream in pipeline", ErrorCodes::LOGICAL_ERROR); executeMergeSorted(pipeline, query_info.input_sorting_info->order_key_prefix_descr, 0); - } const Settings & settings = context->getSettingsRef(); @@ -2828,7 +2821,7 @@ void InterpreterSelectQuery::unifyStreams(Pipeline & pipeline, Block header) { /// Unify streams in case they have different headers. - /// TODO: remove previos addition of _dummy column. + /// TODO: remove previous addition of _dummy column. if (header.columns() > 1 && header.has("_dummy")) header.erase("_dummy"); diff --git a/dbms/src/Processors/QueryPipeline.cpp b/dbms/src/Processors/QueryPipeline.cpp index 13e91ac718d..25abeb6c6d3 100644 --- a/dbms/src/Processors/QueryPipeline.cpp +++ b/dbms/src/Processors/QueryPipeline.cpp @@ -18,6 +18,7 @@ #include #include #include +#include namespace DB { @@ -165,7 +166,6 @@ void QueryPipeline::addSimpleTransformImpl(const TProcessorGetter & getter) for (size_t stream_num = 0; stream_num < streams.size(); ++stream_num) add_transform(streams[stream_num], StreamType::Main, stream_num); - add_transform(delayed_stream_port, StreamType::Main); add_transform(totals_having_port, StreamType::Totals); add_transform(extremes_port, StreamType::Extremes); @@ -185,7 +185,6 @@ void QueryPipeline::addSimpleTransform(const ProcessorGetterWithStreamKind & get void QueryPipeline::addPipe(Processors pipe) { checkInitialized(); - concatDelayedStream(); if (pipe.empty()) throw Exception("Can't add empty processors list to QueryPipeline.", ErrorCodes::LOGICAL_ERROR); @@ -224,41 +223,20 @@ void QueryPipeline::addDelayedStream(ProcessorPtr source) { checkInitialized(); - if (delayed_stream_port) - throw Exception("QueryPipeline already has stream with non joined data.", ErrorCodes::LOGICAL_ERROR); - checkSource(source, false); assertBlocksHaveEqualStructure(current_header, source->getOutputs().front().getHeader(), "QueryPipeline"); - delayed_stream_port = &source->getOutputs().front(); + IProcessor::PortNumbers delayed_streams = { streams.size() }; + streams.emplace_back(&source->getOutputs().front()); processors.emplace_back(std::move(source)); -} -void QueryPipeline::concatDelayedStream() -{ - if (!delayed_stream_port) - return; - - auto resize = std::make_shared(current_header, getNumMainStreams(), 1); - auto stream = streams.begin(); - for (auto & input : resize->getInputs()) - connect(**(stream++), input); - - auto concat = std::make_shared(current_header, 2); - connect(resize->getOutputs().front(), concat->getInputs().front()); - connect(*delayed_stream_port, concat->getInputs().back()); - - streams = { &concat->getOutputs().front() }; - processors.emplace_back(std::move(resize)); - processors.emplace_back(std::move(concat)); - - delayed_stream_port = nullptr; + auto processor = std::make_shared(current_header, streams.size(), delayed_streams); + addPipe({ std::move(processor) }); } void QueryPipeline::resize(size_t num_streams, bool force) { checkInitialized(); - concatDelayedStream(); if (!force && num_streams == getNumStreams()) return; @@ -443,7 +421,6 @@ void QueryPipeline::unitePipelines( std::vector && pipelines, const Block & common_header, const Context & context) { checkInitialized(); - concatDelayedStream(); addSimpleTransform([&](const Block & header) { @@ -456,7 +433,6 @@ void QueryPipeline::unitePipelines( for (auto & pipeline : pipelines) { pipeline.checkInitialized(); - pipeline.concatDelayedStream(); pipeline.addSimpleTransform([&](const Block & header) { diff --git a/dbms/src/Processors/QueryPipeline.h b/dbms/src/Processors/QueryPipeline.h index c27e570018f..29ebaf22955 100644 --- a/dbms/src/Processors/QueryPipeline.h +++ b/dbms/src/Processors/QueryPipeline.h @@ -57,7 +57,7 @@ public: /// Will read from this stream after all data was read from other streams. void addDelayedStream(ProcessorPtr source); - bool hasDelayedStream() const { return delayed_stream_port; } + /// Check if resize transform was used. (In that case another distinct transform will be added). bool hasMixedStreams() const { return has_resize || hasMoreThanOneStream(); } @@ -69,8 +69,7 @@ public: PipelineExecutorPtr execute(); - size_t getNumStreams() const { return streams.size() + (hasDelayedStream() ? 1 : 0); } - size_t getNumMainStreams() const { return streams.size(); } + size_t getNumStreams() const { return streams.size(); } bool hasMoreThanOneStream() const { return getNumStreams() > 1; } bool hasTotals() const { return totals_having_port != nullptr; } @@ -103,9 +102,6 @@ private: OutputPort * totals_having_port = nullptr; OutputPort * extremes_port = nullptr; - /// Special port for delayed stream. - OutputPort * delayed_stream_port = nullptr; - /// If resize processor was added to pipeline. bool has_resize = false; @@ -126,7 +122,6 @@ private: void checkInitialized(); void checkSource(const ProcessorPtr & source, bool can_have_totals); - void concatDelayedStream(); template void addSimpleTransformImpl(const TProcessorGetter & getter); From 0b2d238af9a490404216c92f6ac072afcabb3c3d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Jan 2020 15:24:56 +0300 Subject: [PATCH 27/32] Fix DelayedPortsProcessor. --- dbms/src/Processors/DelayedPortsProcessor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Processors/DelayedPortsProcessor.cpp b/dbms/src/Processors/DelayedPortsProcessor.cpp index d5a033dc7df..672f2645c16 100644 --- a/dbms/src/Processors/DelayedPortsProcessor.cpp +++ b/dbms/src/Processors/DelayedPortsProcessor.cpp @@ -29,8 +29,6 @@ bool DelayedPortsProcessor::processPair(PortsPair & pair) { if (!pair.is_finished) { - pair.input_port->close(); - pair.is_finished = true; ++num_finished; } @@ -38,12 +36,14 @@ bool DelayedPortsProcessor::processPair(PortsPair & pair) if (pair.output_port->isFinished()) { + pair.input_port->close(); finish(); return false; } if (pair.input_port->isFinished()) { + pair.output_port->finish(); finish(); return false; } From 5acbe8e5bd0c3d15aed9d6cdf54e2805ffb1c66c Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 9 Jan 2020 18:15:42 +0300 Subject: [PATCH 28/32] Update security_changelog.md --- docs/ru/security_changelog.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/security_changelog.md b/docs/ru/security_changelog.md index 4e588386af2..66d76849d15 100644 --- a/docs/ru/security_changelog.md +++ b/docs/ru/security_changelog.md @@ -2,7 +2,7 @@ ### CVE-2019-15024 -Злоумышленник с доступом на запись к ZooKeeper и возможностью запустить собственный сервер в сети доступной ClickHouse может создать вредоносный сервер, который будет вести себя как реплика ClickHouse и зарегистрируется в ZooKeeper. В процессе репликации вредоносный сервер может указать любой путь на файловой системе в который будут записаны данные. +Злоумышленник с доступом на запись к ZooKeeper и возможностью запустить собственный сервер в сети доступной ClickHouse может создать вредоносный сервер, который будет вести себя как реплика ClickHouse и зарегистрируется в ZooKeeper. В процессе репликации вредоносный сервер может указать любой путь на файловой системе в который будут записаны данные. Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса From 334b631759a685bf9ba34248c8b8370b9bba84ed Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 9 Jan 2020 18:16:15 +0300 Subject: [PATCH 29/32] Update security_changelog.md --- docs/ru/security_changelog.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/ru/security_changelog.md b/docs/ru/security_changelog.md index 66d76849d15..db742b5f990 100644 --- a/docs/ru/security_changelog.md +++ b/docs/ru/security_changelog.md @@ -4,19 +4,19 @@ Злоумышленник с доступом на запись к ZooKeeper и возможностью запустить собственный сервер в сети доступной ClickHouse может создать вредоносный сервер, который будет вести себя как реплика ClickHouse и зарегистрируется в ZooKeeper. В процессе репликации вредоносный сервер может указать любой путь на файловой системе в который будут записаны данные. -Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса +Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса ### CVE-2019-16535 Интерфейс декомпрессии позволял совершать OOB чтения и записи данных в памяти, а также переполнение целочисленных переменных, что могло приводить к отказу в обслуживании. Также потенциально могло использоваьтся для удаленного выполнения кода. -Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса +Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса ### CVE-2019-16536 Аутентифицированный клиент злоумышленника имел возможность вызвать переполнение стека, что могло привести к отказу в обслуживании. -Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса +Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса ## Исправлено в релизе 19.13.6.1 от 20 сентября 2019 From e594bc10618b6a94bb5222844ff88be358cd129a Mon Sep 17 00:00:00 2001 From: Sergei Semin Date: Thu, 9 Jan 2020 19:39:27 +0300 Subject: [PATCH 30/32] fix typo in tutorial.md replace ant with want (forgotten w) --- docs/en/getting_started/tutorial.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting_started/tutorial.md b/docs/en/getting_started/tutorial.md index acdd9074beb..bffee808122 100644 --- a/docs/en/getting_started/tutorial.md +++ b/docs/en/getting_started/tutorial.md @@ -444,7 +444,7 @@ SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192 ``` -You can execute those queries using interactive mode of `clickhouse-client` (just launch it in terminal without specifying a query in advance) or try some [alternative interface](../interfaces/index.md) if you ant. +You can execute those queries using interactive mode of `clickhouse-client` (just launch it in terminal without specifying a query in advance) or try some [alternative interface](../interfaces/index.md) if you want. As we can see, `hits_v1` uses the [basic MergeTree engine](../operations/table_engines/mergetree.md), while the `visits_v1` uses the [Collapsing](../operations/table_engines/collapsingmergetree.md) variant. From 38bcc5996b83ad84d438ad87ebeaba8e23cf0a3a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 9 Jan 2020 19:42:20 +0300 Subject: [PATCH 31/32] Update apply_catboost_model.md --- docs/en/guides/apply_catboost_model.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/guides/apply_catboost_model.md b/docs/en/guides/apply_catboost_model.md index 5e868bffe0d..06863bb48f9 100644 --- a/docs/en/guides/apply_catboost_model.md +++ b/docs/en/guides/apply_catboost_model.md @@ -74,7 +74,7 @@ $ clickhouse client ROLE_FAMILY UInt32, ROLE_CODE UInt32 ) -ENGINE = MergeTree(date, date, 8192) +ENGINE = MergeTree ORDER BY date ``` **3.** Exit from ClickHouse console client: From 2c6b00ab2854ddf2962867b18f21bb1de1b35807 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 9 Jan 2020 19:42:37 +0300 Subject: [PATCH 32/32] Update apply_catboost_model.md --- docs/ru/guides/apply_catboost_model.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/guides/apply_catboost_model.md b/docs/ru/guides/apply_catboost_model.md index 65cb3f58508..69aa0faccb2 100644 --- a/docs/ru/guides/apply_catboost_model.md +++ b/docs/ru/guides/apply_catboost_model.md @@ -74,7 +74,7 @@ $ clickhouse client ROLE_FAMILY UInt32, ROLE_CODE UInt32 ) -ENGINE = MergeTree(date, date, 8192) +ENGINE = MergeTree ORDER BY date ``` **3.** Выйдите из клиента ClickHouse: