From bcd6e032571448ca8017fda4144503f83c580bdb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 2 Nov 2018 21:53:23 +0300 Subject: [PATCH 01/46] SyntaxAnalyzer (in progress). --- dbms/src/Interpreters/ActionsVisitor.cpp | 8 +- dbms/src/Interpreters/AnalyzedJoin.cpp | 133 +++ dbms/src/Interpreters/AnalyzedJoin.h | 84 ++ .../Interpreters/ArrayJoinedColumnsVisitor.h | 9 + dbms/src/Interpreters/ExpressionAnalyzer.cpp | 1001 +---------------- dbms/src/Interpreters/ExpressionAnalyzer.h | 64 +- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 904 +++++++++++++++ dbms/src/Interpreters/SyntaxAnalyzer.h | 50 + .../TranslateQualifiedNamesVisitor.cpp | 5 +- .../TranslateQualifiedNamesVisitor.h | 16 +- 10 files changed, 1232 insertions(+), 1042 deletions(-) create mode 100644 dbms/src/Interpreters/AnalyzedJoin.cpp create mode 100644 dbms/src/Interpreters/AnalyzedJoin.h create mode 100644 dbms/src/Interpreters/SyntaxAnalyzer.cpp create mode 100644 dbms/src/Interpreters/SyntaxAnalyzer.h diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index 8eebf869303..7d4ee8e0bc4 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -49,9 +49,11 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -/// defined in ExpressionAnalyser.cpp -NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypesList & cols); - +NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypesList & cols) +{ + return std::find_if(cols.begin(), cols.end(), + [&](const NamesAndTypesList::value_type & val) { return val.name == name; }); +} void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool create_ordered_set, const Context & context, const SizeLimits & size_limits, PreparedSets & prepared_sets) diff --git a/dbms/src/Interpreters/AnalyzedJoin.cpp b/dbms/src/Interpreters/AnalyzedJoin.cpp new file mode 100644 index 00000000000..d1542362ca7 --- /dev/null +++ b/dbms/src/Interpreters/AnalyzedJoin.cpp @@ -0,0 +1,133 @@ +#include +#include +#include +#include + +#include +#include +#include +#include + +#include + +namespace DB +{ + +void AnalyzedJoin::createJoinedBlockActions(const NameSet & source_columns, + const ASTSelectQuery * select_query_with_join, + const Context & context) +{ + if (!select_query_with_join) + return; + + const ASTTablesInSelectQueryElement * join = select_query_with_join->join(); + + if (!join) + return; + + const auto & join_params = static_cast(*join->table_join); + + /// Create custom expression list with join keys from right table. + auto expression_list = std::make_shared(); + ASTs & children = expression_list->children; + + if (join_params.on_expression) + for (const auto & join_right_key : key_asts_right) + children.emplace_back(join_right_key); + + NameSet required_columns_set(key_names_right.begin(), key_names_right.end()); + for (const auto & joined_column : columns_added_by_join) + required_columns_set.insert(joined_column.name_and_type.name); + Names required_columns(required_columns_set.begin(), required_columns_set.end()); + + const auto & columns_from_joined_table = getColumnsFromJoinedTable(source_columns, context, select_query_with_join); + NamesAndTypesList source_column_names; + for (auto & column : columns_from_joined_table) + source_column_names.emplace_back(column.name_and_type); + + ExpressionAnalyzer analyzer(expression_list, context, nullptr, source_column_names, required_columns); + joined_block_actions = analyzer.getActions(false); + + auto required_action_columns = joined_block_actions->getRequiredColumns(); + required_columns_from_joined_table.insert(required_action_columns.begin(), required_action_columns.end()); + auto sample = joined_block_actions->getSampleBlock(); + + for (auto & column : key_names_right) + if (!sample.has(column)) + required_columns_from_joined_table.insert(column); + + for (auto & column : columns_added_by_join) + if (!sample.has(column.name_and_type.name)) + required_columns_from_joined_table.insert(column.name_and_type.name); +} + +NamesAndTypesList AnalyzedJoin::getColumnsAddedByJoin() const +{ + NamesAndTypesList result; + for (const auto & joined_column : columns_added_by_join) + result.push_back(joined_column.name_and_type); + + return result; +} + +const JoinedColumnsList & AnalyzedJoin::getColumnsFromJoinedTable( + const NameSet & source_columns, const Context & context, const ASTSelectQuery * select_query_with_join) +{ + if (select_query_with_join && columns_from_joined_table.empty()) + { + if (const ASTTablesInSelectQueryElement * node = select_query_with_join->join()) + { + const auto & table_expression = static_cast(*node->table_expression); + DatabaseAndTableWithAlias table_name_with_alias(table_expression, context.getCurrentDatabase()); + + auto columns = getNamesAndTypeListFromTableExpression(table_expression, context); + + for (auto & column : columns) + { + JoinedColumn joined_column(column, column.name); + + if (source_columns.count(column.name)) + { + auto qualified_name = table_name_with_alias.getQualifiedNamePrefix() + column.name; + joined_column.name_and_type.name = qualified_name; + } + + /// We don't want to select duplicate columns from the joined subquery if they appear + if (std::find(columns_from_joined_table.begin(), columns_from_joined_table.end(), joined_column) == columns_from_joined_table.end()) + columns_from_joined_table.push_back(joined_column); + + } + } + } + + return columns_from_joined_table; +} + + +NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpression & table_expression, const Context & context) +{ + NamesAndTypesList names_and_type_list; + if (table_expression.subquery) + { + const auto & subquery = table_expression.subquery->children.at(0); + names_and_type_list = InterpreterSelectWithUnionQuery::getSampleBlock(subquery, context).getNamesAndTypesList(); + } + else if (table_expression.table_function) + { + const auto table_function = table_expression.table_function; + auto query_context = const_cast(&context.getQueryContext()); + const auto & function_storage = query_context->executeTableFunction(table_function); + names_and_type_list = function_storage->getSampleBlockNonMaterialized().getNamesAndTypesList(); + } + else if (table_expression.database_and_table_name) + { + const auto & identifier = static_cast(*table_expression.database_and_table_name); + DatabaseAndTableWithAlias database_table(identifier); + const auto & table = context.getTable(database_table.database, database_table.table); + names_and_type_list = table->getSampleBlockNonMaterialized().getNamesAndTypesList(); + } + + return names_and_type_list; +} + +} diff --git a/dbms/src/Interpreters/AnalyzedJoin.h b/dbms/src/Interpreters/AnalyzedJoin.h new file mode 100644 index 00000000000..d74c6801dfe --- /dev/null +++ b/dbms/src/Interpreters/AnalyzedJoin.h @@ -0,0 +1,84 @@ +#pragma once + +#include +#include +#include + +#include +#include + +namespace DB +{ + +class Context; +class ASTSelectQuery; + +class ExpressionActions; +using ExpressionActionsPtr = std::shared_ptr; + +struct JoinedColumn +{ + /// Column will be joined to block. + NameAndTypePair name_and_type; + /// original column name from joined source. + String original_name; + + JoinedColumn(NameAndTypePair name_and_type_, String original_name_) + : name_and_type(std::move(name_and_type_)), original_name(std::move(original_name_)) {} + + bool operator==(const JoinedColumn & o) const + { + return name_and_type == o.name_and_type && original_name == o.original_name; + } +}; + +using JoinedColumnsList = std::list; + +struct AnalyzedJoin +{ + + /// NOTE: So far, only one JOIN per query is supported. + + /** Query of the form `SELECT expr(x) AS k FROM t1 ANY LEFT JOIN (SELECT expr(x) AS k FROM t2) USING k` + * The join is made by column k. + * During the JOIN, + * - in the "right" table, it will be available by alias `k`, since `Project` action for the subquery was executed. + * - in the "left" table, it will be accessible by the name `expr(x)`, since `Project` action has not been executed yet. + * You must remember both of these options. + * + * Query of the form `SELECT ... from t1 ANY LEFT JOIN (SELECT ... from t2) ON expr(t1 columns) = expr(t2 columns)` + * to the subquery will be added expression `expr(t2 columns)`. + * It's possible to use name `expr(t2 columns)`. + */ + Names key_names_left; + Names key_names_right; /// Duplicating names are qualified. + ASTs key_asts_left; + ASTs key_asts_right; + + /// All columns which can be read from joined table. Duplicating names are qualified. + JoinedColumnsList columns_from_joined_table; + /// Columns which will be used in query to the joined query. Duplicating names are qualified. + NameSet required_columns_from_joined_table; + + /// Columns which will be added to block, possible including some columns from right join key. + JoinedColumnsList columns_added_by_join; + /// Such columns will be copied from left join keys during join. + NameSet columns_added_by_join_from_right_keys; + /// Actions which need to be calculated on joined block. + ExpressionActionsPtr joined_block_actions; + + void createJoinedBlockActions(const NameSet & source_columns, + const ASTSelectQuery * select_query_with_join, + const Context & context); + + NamesAndTypesList getColumnsAddedByJoin() const; + + const JoinedColumnsList & getColumnsFromJoinedTable(const NameSet & source_columns, + const Context & context, + const ASTSelectQuery * select_query_with_join); +}; + +class ASTTableExpression; +NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpression & table_expression, const Context & context); + +} diff --git a/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h b/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h index 427811e2c18..b1543c99e3a 100644 --- a/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h +++ b/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h @@ -1,5 +1,14 @@ #pragma once +#include +#include + +#include +#include +#include + +#include + namespace DB { diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index baa8c5d8e91..9aeb885a78c 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -98,61 +98,22 @@ namespace ErrorCodes } -/** Calls to these functions in the GROUP BY statement would be - * replaced by their immediate argument. - */ -const std::unordered_set injective_function_names -{ - "negate", - "bitNot", - "reverse", - "reverseUTF8", - "toString", - "toFixedString", - "IPv4NumToString", - "IPv4StringToNum", - "hex", - "unhex", - "bitmaskToList", - "bitmaskToArray", - "tuple", - "regionToName", - "concatAssumeInjective", -}; - -const std::unordered_set possibly_injective_function_names -{ - "dictGetString", - "dictGetUInt8", - "dictGetUInt16", - "dictGetUInt32", - "dictGetUInt64", - "dictGetInt8", - "dictGetInt16", - "dictGetInt32", - "dictGetInt64", - "dictGetFloat32", - "dictGetFloat64", - "dictGetDate", - "dictGetDateTime" -}; - -namespace -{ - -void removeDuplicateColumns(NamesAndTypesList & columns) -{ - std::set names; - for (auto it = columns.begin(); it != columns.end();) - { - if (names.emplace(it->name).second) - ++it; - else - columns.erase(it++); - } -} - -} +//namespace +//{ +// +//void removeDuplicateColumns(NamesAndTypesList & columns) +//{ +// std::set names; +// for (auto it = columns.begin(); it != columns.end();) +// { +// if (names.emplace(it->name).second) +// ++it; +// else +// columns.erase(it++); +// } +//} +// +//} ExpressionAnalyzer::ExpressionAnalyzer( @@ -168,79 +129,21 @@ ExpressionAnalyzer::ExpressionAnalyzer( query(query_), context(context_), settings(context.getSettings()), storage(storage_), subquery_depth(subquery_depth_), do_global(do_global_) { + auto syntax_analyzer_result = SyntaxAnalyzer() + .analyze(query, context, storage, source_columns, required_result_columns_, subquery_depth); + query = syntax_analyzer_result.query; + source_columns = syntax_analyzer_result.source_columns; + aliases = syntax_analyzer_result.aliases; + array_join_result_to_source = syntax_analyzer_result.array_join_result_to_source; + array_join_alias_to_name = syntax_analyzer_result.array_join_alias_to_name; + array_join_name_to_alias = syntax_analyzer_result.array_join_name_to_alias; + analyzed_join = syntax_analyzer_result.analyzed_join; + rewrite_subqueries = syntax_analyzer_result.rewrite_subqueries; + storage = syntax_analyzer_result.storage; + select_query = typeid_cast(query.get()); - if (!storage && select_query) - { - if (auto db_and_table = getDatabaseAndTable(*select_query, 0)) - storage = context.tryGetTable(db_and_table->database, db_and_table->table); - } - - if (storage) - { - auto physical_columns = storage->getColumns().getAllPhysical(); - if (source_columns.empty()) - source_columns.swap(physical_columns); - else - source_columns.insert(source_columns.end(), physical_columns.begin(), physical_columns.end()); - - if (select_query) - { - const auto & storage_aliases = storage->getColumns().aliases; - source_columns.insert(source_columns.end(), storage_aliases.begin(), storage_aliases.end()); - } - } - - removeDuplicateColumns(source_columns); - - translateQualifiedNames(); - - /// Depending on the user's profile, check for the execution rights - /// distributed subqueries inside the IN or JOIN sections and process these subqueries. - InJoinSubqueriesPreprocessor(context).process(select_query); - - /// Optimizes logical expressions. - LogicalExpressionsOptimizer(select_query, settings.min_equality_disjunction_chain_length).perform(); - - /// Creates a dictionary `aliases`: alias -> ASTPtr - { - LogAST log; - QueryAliasesVisitor query_aliases_visitor(log.stream()); - query_aliases_visitor.visit(query, aliases); - } - - /// Common subexpression elimination. Rewrite rules. - normalizeTree(); - - /// Remove unneeded columns according to 'required_result_columns'. - /// Leave all selected columns in case of DISTINCT; columns that contain arrayJoin function inside. - /// Must be after 'normalizeTree' (after expanding aliases, for aliases not get lost) - /// and before 'executeScalarSubqueries', 'analyzeAggregation', etc. to avoid excessive calculations. - removeUnneededColumnsFromSelectClause(); - - /// Executing scalar subqueries - replacing them with constant values. - executeScalarSubqueries(); - - /// Optimize if with constant condition after constants was substituted instead of sclalar subqueries. - optimizeIfWithConstantCondition(); - - /// GROUP BY injective function elimination. - optimizeGroupBy(); - - /// Remove duplicate items from ORDER BY. - optimizeOrderBy(); - - // Remove duplicated elements from LIMIT BY clause. - optimizeLimitBy(); - - /// Remove duplicated columns from USING(...). - optimizeUsing(); - - /// array_join_alias_to_name, array_join_result_to_source. - getArrayJoinedColumns(); - - /// Push the predicate expression down to the subqueries. - rewrite_subqueries = PredicateExpressionsOptimizer(select_query, settings, context).optimize(); + // removeDuplicateColumns(source_columns); /// Delete the unnecessary from `source_columns` list. Create `unknown_required_source_columns`. Form `columns_added_by_join`. collectUsedColumns(); @@ -265,115 +168,6 @@ bool ExpressionAnalyzer::isRemoteStorage() const } -void ExpressionAnalyzer::translateQualifiedNames() -{ - if (!select_query || !select_query->tables || select_query->tables->children.empty()) - return; - - std::vector tables = getDatabaseAndTables(*select_query, context.getCurrentDatabase()); - - LogAST log; - TranslateQualifiedNamesVisitor visitor(source_columns, tables, log.stream()); - visitor.visit(query); -} - -void ExpressionAnalyzer::optimizeIfWithConstantCondition() -{ - optimizeIfWithConstantConditionImpl(query); -} - -bool ExpressionAnalyzer::tryExtractConstValueFromCondition(const ASTPtr & condition, bool & value) const -{ - /// numeric constant in condition - if (const ASTLiteral * literal = typeid_cast(condition.get())) - { - if (literal->value.getType() == Field::Types::Int64 || - literal->value.getType() == Field::Types::UInt64) - { - value = literal->value.get(); - return true; - } - } - - /// cast of numeric constant in condition to UInt8 - if (const ASTFunction * function = typeid_cast(condition.get())) - { - if (function->name == "CAST") - { - if (ASTExpressionList * expr_list = typeid_cast(function->arguments.get())) - { - const ASTPtr & type_ast = expr_list->children.at(1); - if (const ASTLiteral * type_literal = typeid_cast(type_ast.get())) - { - if (type_literal->value.getType() == Field::Types::String && - type_literal->value.get() == "UInt8") - return tryExtractConstValueFromCondition(expr_list->children.at(0), value); - } - } - } - } - - return false; -} - -void ExpressionAnalyzer::optimizeIfWithConstantConditionImpl(ASTPtr & current_ast) -{ - if (!current_ast) - return; - - for (ASTPtr & child : current_ast->children) - { - ASTFunction * function_node = typeid_cast(child.get()); - if (!function_node || function_node->name != "if") - { - optimizeIfWithConstantConditionImpl(child); - continue; - } - - optimizeIfWithConstantConditionImpl(function_node->arguments); - ASTExpressionList * args = typeid_cast(function_node->arguments.get()); - - if (args->children.size() != 3) - throw Exception("Wrong number of arguments for function 'if' (" + toString(args->children.size()) + " instead of 3)", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - ASTPtr condition_expr = args->children[0]; - ASTPtr then_expr = args->children[1]; - ASTPtr else_expr = args->children[2]; - - bool condition; - if (tryExtractConstValueFromCondition(condition_expr, condition)) - { - ASTPtr replace_ast = condition ? then_expr : else_expr; - ASTPtr child_copy = child; - String replace_alias = replace_ast->tryGetAlias(); - String if_alias = child->tryGetAlias(); - - if (replace_alias.empty()) - { - replace_ast->setAlias(if_alias); - child = replace_ast; - } - else - { - /// Only copy of one node is required here. - /// But IAST has only method for deep copy of subtree. - /// This can be a reason of performance degradation in case of deep queries. - ASTPtr replace_ast_deep_copy = replace_ast->clone(); - replace_ast_deep_copy->setAlias(if_alias); - child = replace_ast_deep_copy; - } - - if (!if_alias.empty()) - { - auto alias_it = aliases.find(if_alias); - if (alias_it != aliases.end() && alias_it->second.get() == child_copy.get()) - alias_it->second = child; - } - } - } -} - void ExpressionAnalyzer::analyzeAggregation() { /** Find aggregation keys (aggregation_keys), information about aggregate functions (aggregate_descriptions), @@ -499,295 +293,6 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables() } -NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypesList & cols) -{ - return std::find_if(cols.begin(), cols.end(), - [&](const NamesAndTypesList::value_type & val) { return val.name == name; }); -} - -static NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpression & table_expression, const Context & context) -{ - NamesAndTypesList names_and_type_list; - if (table_expression.subquery) - { - const auto & subquery = table_expression.subquery->children.at(0); - names_and_type_list = InterpreterSelectWithUnionQuery::getSampleBlock(subquery, context).getNamesAndTypesList(); - } - else if (table_expression.table_function) - { - const auto table_function = table_expression.table_function; - auto query_context = const_cast(&context.getQueryContext()); - const auto & function_storage = query_context->executeTableFunction(table_function); - names_and_type_list = function_storage->getSampleBlockNonMaterialized().getNamesAndTypesList(); - } - else if (table_expression.database_and_table_name) - { - const auto & identifier = static_cast(*table_expression.database_and_table_name); - DatabaseAndTableWithAlias database_table(identifier); - const auto & table = context.getTable(database_table.database, database_table.table); - names_and_type_list = table->getSampleBlockNonMaterialized().getNamesAndTypesList(); - } - - return names_and_type_list; -} - -void ExpressionAnalyzer::normalizeTree() -{ - Names all_columns_name; - - auto columns_name = storage ? storage->getColumns().ordinary.getNames() : source_columns.getNames(); - all_columns_name.insert(all_columns_name.begin(), columns_name.begin(), columns_name.end()); - - if (!settings.asterisk_left_columns_only) - { - auto columns_from_joined_table = analyzed_join.getColumnsFromJoinedTable(source_columns, context, select_query); - for (auto & column : columns_from_joined_table) - all_columns_name.emplace_back(column.name_and_type.name); - } - - if (all_columns_name.empty()) - throw Exception("An asterisk cannot be replaced with empty columns.", ErrorCodes::LOGICAL_ERROR); - - TableNamesAndColumnNames table_names_and_column_names; - if (select_query && select_query->tables && !select_query->tables->children.empty()) - { - std::vector tables_expression = getSelectTablesExpression(*select_query); - - bool first = true; - for (const auto * table_expression : tables_expression) - { - DatabaseAndTableWithAlias table_name(*table_expression, context.getCurrentDatabase()); - NamesAndTypesList names_and_types = getNamesAndTypeListFromTableExpression(*table_expression, context); - - if (!first) - { - /// For joined tables qualify duplicating names. - for (auto & name_and_type : names_and_types) - if (source_columns.contains(name_and_type.name)) - name_and_type.name = table_name.getQualifiedNamePrefix() + name_and_type.name; - } - - first = false; - - table_names_and_column_names.emplace_back(std::pair(table_name, names_and_types.getNames())); - } - } - - QueryNormalizer(query, aliases, settings, all_columns_name, table_names_and_column_names).perform(); -} - - -void ExpressionAnalyzer::executeScalarSubqueries() -{ - LogAST log; - - if (!select_query) - { - ExecuteScalarSubqueriesVisitor visitor(context, subquery_depth, log.stream()); - visitor.visit(query); - } - else - { - for (auto & child : query->children) - { - /// Do not go to FROM, JOIN, UNION. - if (!typeid_cast(child.get()) - && !typeid_cast(child.get())) - { - ExecuteScalarSubqueriesVisitor visitor(context, subquery_depth, log.stream()); - visitor.visit(child); - } - } - } -} - - -void ExpressionAnalyzer::optimizeGroupBy() -{ - if (!(select_query && select_query->group_expression_list)) - return; - - const auto is_literal = [] (const ASTPtr & ast) - { - return typeid_cast(ast.get()); - }; - - auto & group_exprs = select_query->group_expression_list->children; - - /// removes expression at index idx by making it last one and calling .pop_back() - const auto remove_expr_at_index = [&group_exprs] (const size_t idx) - { - if (idx < group_exprs.size() - 1) - std::swap(group_exprs[idx], group_exprs.back()); - - group_exprs.pop_back(); - }; - - /// iterate over each GROUP BY expression, eliminate injective function calls and literals - for (size_t i = 0; i < group_exprs.size();) - { - if (const auto function = typeid_cast(group_exprs[i].get())) - { - /// assert function is injective - if (possibly_injective_function_names.count(function->name)) - { - /// do not handle semantic errors here - if (function->arguments->children.size() < 2) - { - ++i; - continue; - } - - const auto & dict_name = typeid_cast(*function->arguments->children[0]) - .value.safeGet(); - - const auto & dict_ptr = context.getExternalDictionaries().getDictionary(dict_name); - - const auto & attr_name = typeid_cast(*function->arguments->children[1]) - .value.safeGet(); - - if (!dict_ptr->isInjective(attr_name)) - { - ++i; - continue; - } - } - else if (!injective_function_names.count(function->name)) - { - ++i; - continue; - } - - /// copy shared pointer to args in order to ensure lifetime - auto args_ast = function->arguments; - - /** remove function call and take a step back to ensure - * next iteration does not skip not yet processed data - */ - remove_expr_at_index(i); - - /// copy non-literal arguments - std::remove_copy_if( - std::begin(args_ast->children), std::end(args_ast->children), - std::back_inserter(group_exprs), is_literal - ); - } - else if (is_literal(group_exprs[i])) - { - remove_expr_at_index(i); - } - else - { - /// if neither a function nor literal - advance to next expression - ++i; - } - } - - if (group_exprs.empty()) - { - /** You can not completely remove GROUP BY. Because if there were no aggregate functions, then it turns out that there will be no aggregation. - * Instead, leave `GROUP BY const`. - * Next, see deleting the constants in the analyzeAggregation method. - */ - - /// You must insert a constant that is not the name of the column in the table. Such a case is rare, but it happens. - UInt64 unused_column = 0; - String unused_column_name = toString(unused_column); - - while (source_columns.end() != std::find_if(source_columns.begin(), source_columns.end(), - [&unused_column_name](const NameAndTypePair & name_type) { return name_type.name == unused_column_name; })) - { - ++unused_column; - unused_column_name = toString(unused_column); - } - - select_query->group_expression_list = std::make_shared(); - select_query->group_expression_list->children.emplace_back(std::make_shared(UInt64(unused_column))); - } -} - - -void ExpressionAnalyzer::optimizeOrderBy() -{ - if (!(select_query && select_query->order_expression_list)) - return; - - /// Make unique sorting conditions. - using NameAndLocale = std::pair; - std::set elems_set; - - ASTs & elems = select_query->order_expression_list->children; - ASTs unique_elems; - unique_elems.reserve(elems.size()); - - for (const auto & elem : elems) - { - String name = elem->children.front()->getColumnName(); - const ASTOrderByElement & order_by_elem = typeid_cast(*elem); - - if (elems_set.emplace(name, order_by_elem.collation ? order_by_elem.collation->getColumnName() : "").second) - unique_elems.emplace_back(elem); - } - - if (unique_elems.size() < elems.size()) - elems = unique_elems; -} - - -void ExpressionAnalyzer::optimizeLimitBy() -{ - if (!(select_query && select_query->limit_by_expression_list)) - return; - - std::set elems_set; - - ASTs & elems = select_query->limit_by_expression_list->children; - ASTs unique_elems; - unique_elems.reserve(elems.size()); - - for (const auto & elem : elems) - { - if (elems_set.emplace(elem->getColumnName()).second) - unique_elems.emplace_back(elem); - } - - if (unique_elems.size() < elems.size()) - elems = unique_elems; -} - -void ExpressionAnalyzer::optimizeUsing() -{ - if (!select_query) - return; - - auto node = const_cast(select_query->join()); - if (!node) - return; - - auto table_join = static_cast(&*node->table_join); - if (!(table_join && table_join->using_expression_list)) - return; - - ASTs & expression_list = table_join->using_expression_list->children; - ASTs uniq_expressions_list; - - std::set expressions_names; - - for (const auto & expression : expression_list) - { - auto expression_name = expression->getAliasOrColumnName(); - if (expressions_names.find(expression_name) == expressions_names.end()) - { - uniq_expressions_list.push_back(expression); - expressions_names.insert(expression_name); - } - } - - if (uniq_expressions_list.size() < expression_list.size()) - expression_list = uniq_expressions_list; -} - - void ExpressionAnalyzer::makeSetsForIndex() { if (storage && select_query && storage->supportsIndexForIn()) @@ -869,66 +374,6 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block & } } - -void ExpressionAnalyzer::getArrayJoinedColumns() -{ - if (select_query && select_query->array_join_expression_list()) - { - ASTs & array_join_asts = select_query->array_join_expression_list()->children; - for (const auto & ast : array_join_asts) - { - const String nested_table_name = ast->getColumnName(); - const String nested_table_alias = ast->getAliasOrColumnName(); - - if (nested_table_alias == nested_table_name && !typeid_cast(ast.get())) - throw Exception("No alias for non-trivial value in ARRAY JOIN: " + nested_table_name, ErrorCodes::ALIAS_REQUIRED); - - if (array_join_alias_to_name.count(nested_table_alias) || aliases.count(nested_table_alias)) - throw Exception("Duplicate alias in ARRAY JOIN: " + nested_table_alias, ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS); - - array_join_alias_to_name[nested_table_alias] = nested_table_name; - array_join_name_to_alias[nested_table_name] = nested_table_alias; - } - - { - ArrayJoinedColumnsVisitor visitor(array_join_name_to_alias, array_join_alias_to_name, array_join_result_to_source); - visitor.visit(query); - } - - /// If the result of ARRAY JOIN is not used, it is necessary to ARRAY-JOIN any column, - /// to get the correct number of rows. - if (array_join_result_to_source.empty()) - { - ASTPtr expr = select_query->array_join_expression_list()->children.at(0); - String source_name = expr->getColumnName(); - String result_name = expr->getAliasOrColumnName(); - - /// This is an array. - if (!typeid_cast(expr.get()) || findColumn(source_name, source_columns) != source_columns.end()) - { - array_join_result_to_source[result_name] = source_name; - } - else /// This is a nested table. - { - bool found = false; - for (const auto & column_name_type : source_columns) - { - auto splitted = Nested::splitName(column_name_type.name); - if (splitted.first == source_name && !splitted.second.empty()) - { - array_join_result_to_source[Nested::concatenateName(result_name, splitted.second)] = column_name_type.name; - found = true; - break; - } - } - if (!found) - throw Exception("No columns in nested table " + source_name, ErrorCodes::EMPTY_NESTED_TABLE); - } - } - } -} - - bool ExpressionAnalyzer::isThereArrayJoin(const ASTPtr & ast) { if (typeid_cast(ast.get())) @@ -1154,99 +599,6 @@ void ExpressionAnalyzer::addJoinAction(ExpressionActionsPtr & actions, bool only analyzed_join.getColumnsAddedByJoin())); } - -void ExpressionAnalyzer::AnalyzedJoin::createJoinedBlockActions(const NamesAndTypesList & source_columns, - const ASTSelectQuery * select_query_with_join, - const Context & context) -{ - if (!select_query_with_join) - return; - - const ASTTablesInSelectQueryElement * join = select_query_with_join->join(); - - if (!join) - return; - - const auto & join_params = static_cast(*join->table_join); - - /// Create custom expression list with join keys from right table. - auto expression_list = std::make_shared(); - ASTs & children = expression_list->children; - - if (join_params.on_expression) - for (const auto & join_right_key : key_asts_right) - children.emplace_back(join_right_key); - - NameSet required_columns_set(key_names_right.begin(), key_names_right.end()); - for (const auto & joined_column : columns_added_by_join) - required_columns_set.insert(joined_column.name_and_type.name); - Names required_columns(required_columns_set.begin(), required_columns_set.end()); - - const auto & columns_from_joined_table = getColumnsFromJoinedTable(source_columns, context, select_query_with_join); - NamesAndTypesList source_column_names; - for (auto & column : columns_from_joined_table) - source_column_names.emplace_back(column.name_and_type); - - ExpressionAnalyzer analyzer(expression_list, context, nullptr, source_column_names, required_columns); - joined_block_actions = analyzer.getActions(false); - - auto required_action_columns = joined_block_actions->getRequiredColumns(); - required_columns_from_joined_table.insert(required_action_columns.begin(), required_action_columns.end()); - auto sample = joined_block_actions->getSampleBlock(); - - for (auto & column : key_names_right) - if (!sample.has(column)) - required_columns_from_joined_table.insert(column); - - for (auto & column : columns_added_by_join) - if (!sample.has(column.name_and_type.name)) - required_columns_from_joined_table.insert(column.name_and_type.name); -} - - -NamesAndTypesList ExpressionAnalyzer::AnalyzedJoin::getColumnsAddedByJoin() const -{ - NamesAndTypesList result; - for (const auto & joined_column : columns_added_by_join) - result.push_back(joined_column.name_and_type); - - return result; -} - -const ExpressionAnalyzer::AnalyzedJoin::JoinedColumnsList & ExpressionAnalyzer::AnalyzedJoin::getColumnsFromJoinedTable( - const NamesAndTypesList & source_columns, const Context & context, const ASTSelectQuery * select_query_with_join) -{ - if (select_query_with_join && columns_from_joined_table.empty()) - { - if (const ASTTablesInSelectQueryElement * node = select_query_with_join->join()) - { - const auto & table_expression = static_cast(*node->table_expression); - DatabaseAndTableWithAlias table_name_with_alias(table_expression, context.getCurrentDatabase()); - - auto columns = getNamesAndTypeListFromTableExpression(table_expression, context); - - for (auto & column : columns) - { - JoinedColumn joined_column(column, column.name); - - if (source_columns.contains(column.name)) - { - auto qualified_name = table_name_with_alias.getQualifiedNamePrefix() + column.name; - joined_column.name_and_type.name = qualified_name; - } - - /// We don't want to select duplicate columns from the joined subquery if they appear - if (std::find(columns_from_joined_table.begin(), columns_from_joined_table.end(), joined_column) == columns_from_joined_table.end()) - columns_from_joined_table.push_back(joined_column); - - } - } - } - - return columns_from_joined_table; -} - - bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_types) { assertSelect(); @@ -1326,7 +678,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty table = table_to_join.database_and_table_name; Names original_columns; - for (const auto & column : analyzed_join.getColumnsFromJoinedTable(source_columns, context, select_query)) + for (const auto & column : analyzed_join.columns_from_joined_table) if (analyzed_join.required_columns_from_joined_table.count(column.name_and_type.name)) original_columns.emplace_back(column.original_name); @@ -1337,7 +689,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty } /// Alias duplicating columns as qualified. - for (const auto & column : analyzed_join.getColumnsFromJoinedTable(source_columns, context, select_query)) + for (const auto & column : analyzed_join.columns_from_joined_table) if (analyzed_join.required_columns_from_joined_table.count(column.name_and_type.name)) subquery_for_set.joined_block_aliases.emplace_back(column.original_name, column.name_and_type.name); @@ -1736,7 +1088,8 @@ void ExpressionAnalyzer::collectUsedColumns() * (Do not assume that they are required for reading from the "left" table). */ NameSet available_joined_columns; - collectJoinedColumns(available_joined_columns); + for (const auto & joined_column : analyzed_join.columns_added_by_join) + available_joined_columns.insert(joined_column.name_and_type.name); NameSet required_joined_columns; @@ -1758,7 +1111,10 @@ void ExpressionAnalyzer::collectUsedColumns() analyzed_join.columns_added_by_join.erase(it++); } - analyzed_join.createJoinedBlockActions(source_columns, select_query, context); + NameSet source_columns_set; + for (const auto & type_name : source_columns) + source_columns_set.insert(type_name.name); + analyzed_join.createJoinedBlockActions(source_columns_set, select_query, context); /// Some columns from right join key may be used in query. This columns will be appended to block during join. for (const auto & right_key_name : analyzed_join.key_names_right) @@ -1811,292 +1167,9 @@ void ExpressionAnalyzer::collectUsedColumns() } -void ExpressionAnalyzer::collectJoinedColumnsFromJoinOnExpr() -{ - const auto & tables = static_cast(*select_query->tables); - const auto * left_tables_element = static_cast(tables.children.at(0).get()); - const auto * right_tables_element = select_query->join(); - - if (!left_tables_element || !right_tables_element) - return; - - const auto & table_join = static_cast(*right_tables_element->table_join); - if (!table_join.on_expression) - return; - - const auto & left_table_expression = static_cast(*left_tables_element->table_expression); - const auto & right_table_expression = static_cast(*right_tables_element->table_expression); - - DatabaseAndTableWithAlias left_source_names(left_table_expression, context.getCurrentDatabase()); - DatabaseAndTableWithAlias right_source_names(right_table_expression, context.getCurrentDatabase()); - - /// Stores examples of columns which are only from one table. - struct TableBelonging - { - const ASTIdentifier * example_only_from_left = nullptr; - const ASTIdentifier * example_only_from_right = nullptr; - }; - - /// Check all identifiers in ast and decide their possible table belonging. - /// Throws if there are two identifiers definitely from different tables. - std::function get_table_belonging; - get_table_belonging = [&](const ASTPtr & ast) -> TableBelonging - { - auto * identifier = typeid_cast(ast.get()); - if (identifier) - { - if (identifier->general()) - { - auto left_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, left_source_names); - auto right_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, right_source_names); - - /// Assume that component from definite table if num_components is greater than for the other table. - if (left_num_components > right_num_components) - return {identifier, nullptr}; - if (left_num_components < right_num_components) - return {nullptr, identifier}; - } - return {}; - } - - TableBelonging table_belonging; - for (const auto & child : ast->children) - { - auto children_belonging = get_table_belonging(child); - if (!table_belonging.example_only_from_left) - table_belonging.example_only_from_left = children_belonging.example_only_from_left; - if (!table_belonging.example_only_from_right) - table_belonging.example_only_from_right = children_belonging.example_only_from_right; - } - - if (table_belonging.example_only_from_left && table_belonging.example_only_from_right) - throw Exception("Invalid columns in JOIN ON section. Columns " - + table_belonging.example_only_from_left->getAliasOrColumnName() + " and " - + table_belonging.example_only_from_right->getAliasOrColumnName() - + " are from different tables.", ErrorCodes::INVALID_JOIN_ON_EXPRESSION); - - return table_belonging; - }; - - std::function translate_qualified_names; - translate_qualified_names = [&](ASTPtr & ast, const DatabaseAndTableWithAlias & source_names, bool right_table) - { - if (auto * identifier = typeid_cast(ast.get())) - { - if (identifier->general()) - { - auto num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, source_names); - stripIdentifier(ast, num_components); - - if (right_table && source_columns.contains(ast->getColumnName())) - source_names.makeQualifiedName(ast); - - } - return; - } - - for (auto & child : ast->children) - translate_qualified_names(child, source_names, right_table); - }; - - const auto supported_syntax = " Supported syntax: JOIN ON Expr([table.]column, ...) = Expr([table.]column, ...) " - "[AND Expr([table.]column, ...) = Expr([table.]column, ...) ...]"; - auto throwSyntaxException = [&](const String & msg) - { - throw Exception("Invalid expression for JOIN ON. " + msg + supported_syntax, ErrorCodes::INVALID_JOIN_ON_EXPRESSION); - }; - - /// For equal expression find out corresponding table for each part, translate qualified names and add asts to join keys. - auto add_columns_from_equals_expr = [&](const ASTPtr & expr) - { - auto * func_equals = typeid_cast(expr.get()); - if (!func_equals || func_equals->name != "equals") - throwSyntaxException("Expected equals expression, got " + queryToString(expr) + "."); - - ASTPtr left_ast = func_equals->arguments->children.at(0)->clone(); - ASTPtr right_ast = func_equals->arguments->children.at(1)->clone(); - - auto left_table_belonging = get_table_belonging(left_ast); - auto right_table_belonging = get_table_belonging(right_ast); - - bool can_be_left_part_from_left_table = left_table_belonging.example_only_from_right == nullptr; - bool can_be_left_part_from_right_table = left_table_belonging.example_only_from_left == nullptr; - bool can_be_right_part_from_left_table = right_table_belonging.example_only_from_right == nullptr; - bool can_be_right_part_from_right_table = right_table_belonging.example_only_from_left == nullptr; - - auto add_join_keys = [&](ASTPtr & ast_to_left_table, ASTPtr & ast_to_right_table) - { - translate_qualified_names(ast_to_left_table, left_source_names, false); - translate_qualified_names(ast_to_right_table, right_source_names, true); - - analyzed_join.key_asts_left.push_back(ast_to_left_table); - analyzed_join.key_names_left.push_back(ast_to_left_table->getColumnName()); - analyzed_join.key_asts_right.push_back(ast_to_right_table); - analyzed_join.key_names_right.push_back(ast_to_right_table->getAliasOrColumnName()); - }; - - /// Default variant when all identifiers may be from any table. - if (can_be_left_part_from_left_table && can_be_right_part_from_right_table) - add_join_keys(left_ast, right_ast); - else if (can_be_left_part_from_right_table && can_be_right_part_from_left_table) - add_join_keys(right_ast, left_ast); - else - { - auto * left_example = left_table_belonging.example_only_from_left ? - left_table_belonging.example_only_from_left : - left_table_belonging.example_only_from_right; - - auto * right_example = right_table_belonging.example_only_from_left ? - right_table_belonging.example_only_from_left : - right_table_belonging.example_only_from_right; - - auto left_name = queryToString(*left_example); - auto right_name = queryToString(*right_example); - auto expr_name = queryToString(expr); - - throwSyntaxException("In expression " + expr_name + " columns " + left_name + " and " + right_name - + " are from the same table but from different arguments of equal function."); - } - }; - - auto * func = typeid_cast(table_join.on_expression.get()); - if (func && func->name == "and") - { - for (const auto & expr : func->arguments->children) - add_columns_from_equals_expr(expr); - } - else - add_columns_from_equals_expr(table_join.on_expression); -} - -void ExpressionAnalyzer::collectJoinedColumns(NameSet & joined_columns) -{ - if (!select_query) - return; - - const ASTTablesInSelectQueryElement * node = select_query->join(); - - if (!node) - return; - - const auto & table_join = static_cast(*node->table_join); - const auto & table_expression = static_cast(*node->table_expression); - DatabaseAndTableWithAlias joined_table_name(table_expression, context.getCurrentDatabase()); - - auto add_name_to_join_keys = [&](Names & join_keys, ASTs & join_asts, const ASTPtr & ast, bool right_table) - { - String name; - if (right_table) - { - name = ast->getAliasOrColumnName(); - if (source_columns.contains(name)) - name = joined_table_name.getQualifiedNamePrefix() + name; - } - else - name = ast->getColumnName(); - - join_keys.push_back(name); - join_asts.push_back(ast); - }; - - if (table_join.using_expression_list) - { - auto & keys = typeid_cast(*table_join.using_expression_list); - for (const auto & key : keys.children) - { - add_name_to_join_keys(analyzed_join.key_names_left, analyzed_join.key_asts_left, key, false); - add_name_to_join_keys(analyzed_join.key_names_right, analyzed_join.key_asts_right, key, true); - } - } - else if (table_join.on_expression) - collectJoinedColumnsFromJoinOnExpr(); - - /// When we use JOIN ON syntax, non_joined_columns are columns from join_key_names_left, - /// because even if a column from join_key_names_right, we may need to join it if it has different name. - /// If we use USING syntax, join_key_names_left and join_key_names_right are almost the same, but we need to use - /// join_key_names_right in order to support aliases in USING list. Example: - /// SELECT x FROM tab1 ANY LEFT JOIN tab2 USING (x as y) - will join column x from tab1 with column y from tab2. - //auto & not_joined_columns = table_join.using_expression_list ? analyzed_join.key_names_right : analyzed_join.key_names_left; - auto & columns_from_joined_table = analyzed_join.getColumnsFromJoinedTable(source_columns, context, select_query); - - for (auto & column : columns_from_joined_table) - { - auto & column_name = column.name_and_type.name; - auto & column_type = column.name_and_type.type; - auto & original_name = column.original_name; - //if (table_join.on_expression - // || not_joined_columns.end() == std::find(not_joined_columns.begin(), not_joined_columns.end(), column_name)) - { - if (joined_columns.count(column_name)) /// Duplicate columns in the subquery for JOIN do not make sense. - continue; - - joined_columns.insert(column_name); - - bool make_nullable = settings.join_use_nulls && (table_join.kind == ASTTableJoin::Kind::Left || - table_join.kind == ASTTableJoin::Kind::Full); - auto type = make_nullable ? makeNullable(column_type) : column_type; - analyzed_join.columns_added_by_join.emplace_back(NameAndTypePair(column_name, std::move(type)), original_name); - } - } -} - - Names ExpressionAnalyzer::getRequiredSourceColumns() const { return source_columns.getNames(); } - -static bool hasArrayJoin(const ASTPtr & ast) -{ - if (const ASTFunction * function = typeid_cast(&*ast)) - if (function->name == "arrayJoin") - return true; - - for (const auto & child : ast->children) - if (!typeid_cast(child.get()) && hasArrayJoin(child)) - return true; - - return false; -} - - -void ExpressionAnalyzer::removeUnneededColumnsFromSelectClause() -{ - if (!select_query) - return; - - if (required_result_columns.empty()) - return; - - ASTs & elements = select_query->select_expression_list->children; - - ASTs new_elements; - new_elements.reserve(elements.size()); - - /// Some columns may be queried multiple times, like SELECT x, y, y FROM table. - /// In that case we keep them exactly same number of times. - std::map required_columns_with_duplicate_count; - for (const auto & name : required_result_columns) - ++required_columns_with_duplicate_count[name]; - - for (const auto & elem : elements) - { - String name = elem->getAliasOrColumnName(); - - auto it = required_columns_with_duplicate_count.find(name); - if (required_columns_with_duplicate_count.end() != it && it->second) - { - new_elements.push_back(elem); - --it->second; - } - else if (select_query->distinct || hasArrayJoin(elem)) - { - new_elements.push_back(elem); - } - } - - elements = std::move(new_elements); -} - } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index 4f0ff267eae..333ff26456d 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -229,71 +230,8 @@ private: size_t subquery_depth; bool do_global; /// Do I need to prepare for execution global subqueries when analyzing the query. - struct AnalyzedJoin - { - - /// NOTE: So far, only one JOIN per query is supported. - - /** Query of the form `SELECT expr(x) AS k FROM t1 ANY LEFT JOIN (SELECT expr(x) AS k FROM t2) USING k` - * The join is made by column k. - * During the JOIN, - * - in the "right" table, it will be available by alias `k`, since `Project` action for the subquery was executed. - * - in the "left" table, it will be accessible by the name `expr(x)`, since `Project` action has not been executed yet. - * You must remember both of these options. - * - * Query of the form `SELECT ... from t1 ANY LEFT JOIN (SELECT ... from t2) ON expr(t1 columns) = expr(t2 columns)` - * to the subquery will be added expression `expr(t2 columns)`. - * It's possible to use name `expr(t2 columns)`. - */ - Names key_names_left; - Names key_names_right; /// Duplicating names are qualified. - ASTs key_asts_left; - ASTs key_asts_right; - - struct JoinedColumn - { - /// Column will be joined to block. - NameAndTypePair name_and_type; - /// original column name from joined source. - String original_name; - - JoinedColumn(const NameAndTypePair & name_and_type_, const String & original_name_) - : name_and_type(name_and_type_), original_name(original_name_) {} - - bool operator==(const JoinedColumn & o) const - { - return name_and_type == o.name_and_type && original_name == o.original_name; - } - }; - - using JoinedColumnsList = std::list; - - /// All columns which can be read from joined table. Duplicating names are qualified. - JoinedColumnsList columns_from_joined_table; - /// Columns which will be used in query to the joined query. Duplicating names are qualified. - NameSet required_columns_from_joined_table; - - /// Columns which will be added to block, possible including some columns from right join key. - JoinedColumnsList columns_added_by_join; - /// Such columns will be copied from left join keys during join. - NameSet columns_added_by_join_from_right_keys; - /// Actions which need to be calculated on joined block. - ExpressionActionsPtr joined_block_actions; - - void createJoinedBlockActions(const NamesAndTypesList & source_columns, - const ASTSelectQuery * select_query_with_join, - const Context & context); - - NamesAndTypesList getColumnsAddedByJoin() const; - - const JoinedColumnsList & getColumnsFromJoinedTable(const NamesAndTypesList & source_columns, - const Context & context, - const ASTSelectQuery * select_query_with_join); - }; - AnalyzedJoin analyzed_join; - /** Remove all unnecessary columns from the list of all available columns of the table (`columns`). * At the same time, form a set of unknown columns (`unknown_required_source_columns`), * as well as the columns added by JOIN (`columns_added_by_join`). diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp new file mode 100644 index 00000000000..68a55ecb13e --- /dev/null +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -0,0 +1,904 @@ +#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 +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ALIAS_REQUIRED; + extern const int MULTIPLE_EXPRESSIONS_FOR_ALIAS; + extern const int EMPTY_NESTED_TABLE; + extern const int LOGICAL_ERROR; + extern const int INVALID_JOIN_ON_EXPRESSION; +} + +namespace +{ + +using LogAST = DebugASTLog; /// set to true to enable logs +using Aliases = std::unordered_map; + +void translateQualifiedNames(ASTPtr & query, ASTSelectQuery * select_query, + const NameSet & source_columns, const Context & context) +{ + if (!select_query || !select_query->tables || select_query->tables->children.empty()) + return; + + std::vector tables = getDatabaseAndTables(*select_query, context.getCurrentDatabase()); + + LogAST log; + TranslateQualifiedNamesVisitor visitor(source_columns, tables, log.stream()); + visitor.visit(query); +} + +void normalizeTree(SyntaxAnalyzerResult & result, const Names & source_columns, const NameSet & source_columns_set, + const Context & context, const ASTSelectQuery * select_query, bool asterisk_left_columns_only) +{ + Names all_columns_name(source_columns.begin(), source_columns.end()); + + if (!asterisk_left_columns_only) + { + auto columns_from_joined_table = result.analyzed_join.getColumnsFromJoinedTable(source_columns_set, context, select_query); + for (auto & column : columns_from_joined_table) + all_columns_name.emplace_back(column.name_and_type.name); + } + + if (all_columns_name.empty()) + throw Exception("An asterisk cannot be replaced with empty columns.", ErrorCodes::LOGICAL_ERROR); + + TableNamesAndColumnNames table_names_and_column_names; + if (select_query && select_query->tables && !select_query->tables->children.empty()) + { + std::vector tables_expression = getSelectTablesExpression(*select_query); + + bool first = true; + for (const auto * table_expression : tables_expression) + { + DatabaseAndTableWithAlias table_name(*table_expression, context.getCurrentDatabase()); + NamesAndTypesList names_and_types = getNamesAndTypeListFromTableExpression(*table_expression, context); + + if (!first) + { + /// For joined tables qualify duplicating names. + for (auto & name_and_type : names_and_types) + if (source_columns_set.count(name_and_type.name)) + name_and_type.name = table_name.getQualifiedNamePrefix() + name_and_type.name; + } + + first = false; + + table_names_and_column_names.emplace_back(std::pair(table_name, names_and_types.getNames())); + } + } + + auto & settings = context.getSettingsRef(); + QueryNormalizer(result.query, result.aliases, settings, all_columns_name, table_names_and_column_names).perform(); +} + +bool hasArrayJoin(const ASTPtr & ast) +{ + if (const ASTFunction * function = typeid_cast(&*ast)) + if (function->name == "arrayJoin") + return true; + + for (const auto & child : ast->children) + if (!typeid_cast(child.get()) && hasArrayJoin(child)) + return true; + + return false; +} + +void removeUnneededColumnsFromSelectClause(const ASTSelectQuery * select_query, const Names & required_result_columns) +{ + if (!select_query) + return; + + if (required_result_columns.empty()) + return; + + ASTs & elements = select_query->select_expression_list->children; + + ASTs new_elements; + new_elements.reserve(elements.size()); + + /// Some columns may be queried multiple times, like SELECT x, y, y FROM table. + /// In that case we keep them exactly same number of times. + std::map required_columns_with_duplicate_count; + for (const auto & name : required_result_columns) + ++required_columns_with_duplicate_count[name]; + + for (const auto & elem : elements) + { + String name = elem->getAliasOrColumnName(); + + auto it = required_columns_with_duplicate_count.find(name); + if (required_columns_with_duplicate_count.end() != it && it->second) + { + new_elements.push_back(elem); + --it->second; + } + else if (select_query->distinct || hasArrayJoin(elem)) + { + new_elements.push_back(elem); + } + } + + elements = std::move(new_elements); +} + +void executeScalarSubqueries(SyntaxAnalyzerResult & result, const ASTSelectQuery * select_query, + const Context & context, size_t subquery_depth) +{ + LogAST log; + + if (!select_query) + { + ExecuteScalarSubqueriesVisitor visitor(context, subquery_depth, log.stream()); + visitor.visit(result.query); + } + else + { + for (auto & child : result.query->children) + { + /// Do not go to FROM, JOIN, UNION. + if (!typeid_cast(child.get()) + && !typeid_cast(child.get())) + { + ExecuteScalarSubqueriesVisitor visitor(context, subquery_depth, log.stream()); + visitor.visit(child); + } + } + } +} + +bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & value) +{ + /// numeric constant in condition + if (const ASTLiteral * literal = typeid_cast(condition.get())) + { + if (literal->value.getType() == Field::Types::Int64 || + literal->value.getType() == Field::Types::UInt64) + { + value = literal->value.get(); + return true; + } + } + + /// cast of numeric constant in condition to UInt8 + if (const ASTFunction * function = typeid_cast(condition.get())) + { + if (function->name == "CAST") + { + if (ASTExpressionList * expr_list = typeid_cast(function->arguments.get())) + { + const ASTPtr & type_ast = expr_list->children.at(1); + if (const ASTLiteral * type_literal = typeid_cast(type_ast.get())) + { + if (type_literal->value.getType() == Field::Types::String && + type_literal->value.get() == "UInt8") + return tryExtractConstValueFromCondition(expr_list->children.at(0), value); + } + } + } + } + + return false; +} + +void optimizeIfWithConstantCondition(ASTPtr & current_ast, Aliases & aliases) +{ + if (!current_ast) + return; + + for (ASTPtr & child : current_ast->children) + { + auto * function_node = typeid_cast(child.get()); + if (!function_node || function_node->name != "if") + { + optimizeIfWithConstantCondition(child, aliases); + continue; + } + + optimizeIfWithConstantCondition(function_node->arguments, aliases); + auto * args = typeid_cast(function_node->arguments.get()); + + if (args->children.size() != 3) + throw Exception("Wrong number of arguments for function 'if' (" + toString(args->children.size()) + " instead of 3)", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + ASTPtr condition_expr = args->children[0]; + ASTPtr then_expr = args->children[1]; + ASTPtr else_expr = args->children[2]; + + bool condition; + if (tryExtractConstValueFromCondition(condition_expr, condition)) + { + ASTPtr replace_ast = condition ? then_expr : else_expr; + ASTPtr child_copy = child; + String replace_alias = replace_ast->tryGetAlias(); + String if_alias = child->tryGetAlias(); + + if (replace_alias.empty()) + { + replace_ast->setAlias(if_alias); + child = replace_ast; + } + else + { + /// Only copy of one node is required here. + /// But IAST has only method for deep copy of subtree. + /// This can be a reason of performance degradation in case of deep queries. + ASTPtr replace_ast_deep_copy = replace_ast->clone(); + replace_ast_deep_copy->setAlias(if_alias); + child = replace_ast_deep_copy; + } + + if (!if_alias.empty()) + { + auto alias_it = aliases.find(if_alias); + if (alias_it != aliases.end() && alias_it->second.get() == child_copy.get()) + alias_it->second = child; + } + } + } +} + +/** Calls to these functions in the GROUP BY statement would be + * replaced by their immediate argument. + */ +const std::unordered_set injective_function_names +{ + "negate", + "bitNot", + "reverse", + "reverseUTF8", + "toString", + "toFixedString", + "IPv4NumToString", + "IPv4StringToNum", + "hex", + "unhex", + "bitmaskToList", + "bitmaskToArray", + "tuple", + "regionToName", + "concatAssumeInjective", +}; + +const std::unordered_set possibly_injective_function_names +{ + "dictGetString", + "dictGetUInt8", + "dictGetUInt16", + "dictGetUInt32", + "dictGetUInt64", + "dictGetInt8", + "dictGetInt16", + "dictGetInt32", + "dictGetInt64", + "dictGetFloat32", + "dictGetFloat64", + "dictGetDate", + "dictGetDateTime" +}; + +void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_columns, const Context & context) +{ + if (!(select_query && select_query->group_expression_list)) + return; + + const auto is_literal = [] (const ASTPtr & ast) + { + return typeid_cast(ast.get()); + }; + + auto & group_exprs = select_query->group_expression_list->children; + + /// removes expression at index idx by making it last one and calling .pop_back() + const auto remove_expr_at_index = [&group_exprs] (const size_t idx) + { + if (idx < group_exprs.size() - 1) + std::swap(group_exprs[idx], group_exprs.back()); + + group_exprs.pop_back(); + }; + + /// iterate over each GROUP BY expression, eliminate injective function calls and literals + for (size_t i = 0; i < group_exprs.size();) + { + if (const auto function = typeid_cast(group_exprs[i].get())) + { + /// assert function is injective + if (possibly_injective_function_names.count(function->name)) + { + /// do not handle semantic errors here + if (function->arguments->children.size() < 2) + { + ++i; + continue; + } + + const auto & dict_name = typeid_cast(*function->arguments->children[0]) + .value.safeGet(); + + const auto & dict_ptr = context.getExternalDictionaries().getDictionary(dict_name); + + const auto & attr_name = typeid_cast(*function->arguments->children[1]) + .value.safeGet(); + + if (!dict_ptr->isInjective(attr_name)) + { + ++i; + continue; + } + } + else if (!injective_function_names.count(function->name)) + { + ++i; + continue; + } + + /// copy shared pointer to args in order to ensure lifetime + auto args_ast = function->arguments; + + /** remove function call and take a step back to ensure + * next iteration does not skip not yet processed data + */ + remove_expr_at_index(i); + + /// copy non-literal arguments + std::remove_copy_if( + std::begin(args_ast->children), std::end(args_ast->children), + std::back_inserter(group_exprs), is_literal + ); + } + else if (is_literal(group_exprs[i])) + { + remove_expr_at_index(i); + } + else + { + /// if neither a function nor literal - advance to next expression + ++i; + } + } + + if (group_exprs.empty()) + { + /** You can not completely remove GROUP BY. Because if there were no aggregate functions, then it turns out that there will be no aggregation. + * Instead, leave `GROUP BY const`. + * Next, see deleting the constants in the analyzeAggregation method. + */ + + /// You must insert a constant that is not the name of the column in the table. Such a case is rare, but it happens. + UInt64 unused_column = 0; + String unused_column_name = toString(unused_column); + + while (source_columns.count(unused_column_name)) + { + ++unused_column; + unused_column_name = toString(unused_column); + } + + select_query->group_expression_list = std::make_shared(); + select_query->group_expression_list->children.emplace_back(std::make_shared(UInt64(unused_column))); + } +} + +void optimizeOrderBy(const ASTSelectQuery * select_query) +{ + if (!(select_query && select_query->order_expression_list)) + return; + + /// Make unique sorting conditions. + using NameAndLocale = std::pair; + std::set elems_set; + + ASTs & elems = select_query->order_expression_list->children; + ASTs unique_elems; + unique_elems.reserve(elems.size()); + + for (const auto & elem : elems) + { + String name = elem->children.front()->getColumnName(); + const ASTOrderByElement & order_by_elem = typeid_cast(*elem); + + if (elems_set.emplace(name, order_by_elem.collation ? order_by_elem.collation->getColumnName() : "").second) + unique_elems.emplace_back(elem); + } + + if (unique_elems.size() < elems.size()) + elems = unique_elems; +} + +void optimizeLimitBy(const ASTSelectQuery * select_query) +{ + if (!(select_query && select_query->limit_by_expression_list)) + return; + + std::set elems_set; + + ASTs & elems = select_query->limit_by_expression_list->children; + ASTs unique_elems; + unique_elems.reserve(elems.size()); + + for (const auto & elem : elems) + { + if (elems_set.emplace(elem->getColumnName()).second) + unique_elems.emplace_back(elem); + } + + if (unique_elems.size() < elems.size()) + elems = unique_elems; +} + +void optimizeUsing(const ASTSelectQuery * select_query) +{ + if (!select_query) + return; + + auto node = const_cast(select_query->join()); + if (!node) + return; + + auto table_join = static_cast(&*node->table_join); + if (!(table_join && table_join->using_expression_list)) + return; + + ASTs & expression_list = table_join->using_expression_list->children; + ASTs uniq_expressions_list; + + std::set expressions_names; + + for (const auto & expression : expression_list) + { + auto expression_name = expression->getAliasOrColumnName(); + if (expressions_names.find(expression_name) == expressions_names.end()) + { + uniq_expressions_list.push_back(expression); + expressions_names.insert(expression_name); + } + } + + if (uniq_expressions_list.size() < expression_list.size()) + expression_list = uniq_expressions_list; +} + +void getArrayJoinedColumns(SyntaxAnalyzerResult & result, const ASTSelectQuery * select_query, const NameSet & source_columns) +{ + if (select_query && select_query->array_join_expression_list()) + { + ASTs & array_join_asts = select_query->array_join_expression_list()->children; + for (const auto & ast : array_join_asts) + { + const String nested_table_name = ast->getColumnName(); + const String nested_table_alias = ast->getAliasOrColumnName(); + + if (nested_table_alias == nested_table_name && !typeid_cast(ast.get())) + throw Exception("No alias for non-trivial value in ARRAY JOIN: " + nested_table_name, + ErrorCodes::ALIAS_REQUIRED); + + if (result.array_join_alias_to_name.count(nested_table_alias) || result.aliases.count(nested_table_alias)) + throw Exception("Duplicate alias in ARRAY JOIN: " + nested_table_alias, + ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS); + + result.array_join_alias_to_name[nested_table_alias] = nested_table_name; + result.array_join_name_to_alias[nested_table_name] = nested_table_alias; + } + + { + ArrayJoinedColumnsVisitor visitor(result.array_join_name_to_alias, + result.array_join_alias_to_name, + result.array_join_result_to_source); + visitor.visit(result.query); + } + + /// If the result of ARRAY JOIN is not used, it is necessary to ARRAY-JOIN any column, + /// to get the correct number of rows. + if (result.array_join_result_to_source.empty()) + { + ASTPtr expr = select_query->array_join_expression_list()->children.at(0); + String source_name = expr->getColumnName(); + String result_name = expr->getAliasOrColumnName(); + + /// This is an array. + if (!typeid_cast(expr.get()) || source_columns.count(source_name)) + { + result.array_join_result_to_source[result_name] = source_name; + } + else /// This is a nested table. + { + bool found = false; + for (const auto & column_name : source_columns) + { + auto splitted = Nested::splitName(column_name); + if (splitted.first == source_name && !splitted.second.empty()) + { + result.array_join_result_to_source[Nested::concatenateName(result_name, splitted.second)] = column_name; + found = true; + break; + } + } + if (!found) + throw Exception("No columns in nested table " + source_name, ErrorCodes::EMPTY_NESTED_TABLE); + } + } + } +} + +void collectJoinedColumnsFromJoinOnExpr(AnalyzedJoin & analyzed_join, const ASTSelectQuery * select_query, + const NameSet & source_columns, const Context & context) +{ + const auto & tables = static_cast(*select_query->tables); + const auto * left_tables_element = static_cast(tables.children.at(0).get()); + const auto * right_tables_element = select_query->join(); + + if (!left_tables_element || !right_tables_element) + return; + + const auto & table_join = static_cast(*right_tables_element->table_join); + if (!table_join.on_expression) + return; + + const auto & left_table_expression = static_cast(*left_tables_element->table_expression); + const auto & right_table_expression = static_cast(*right_tables_element->table_expression); + + DatabaseAndTableWithAlias left_source_names(left_table_expression, context.getCurrentDatabase()); + DatabaseAndTableWithAlias right_source_names(right_table_expression, context.getCurrentDatabase()); + + /// Stores examples of columns which are only from one table. + struct TableBelonging + { + const ASTIdentifier * example_only_from_left = nullptr; + const ASTIdentifier * example_only_from_right = nullptr; + }; + + /// Check all identifiers in ast and decide their possible table belonging. + /// Throws if there are two identifiers definitely from different tables. + std::function get_table_belonging; + get_table_belonging = [&](const ASTPtr & ast) -> TableBelonging + { + auto * identifier = typeid_cast(ast.get()); + if (identifier) + { + if (identifier->general()) + { + auto left_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, left_source_names); + auto right_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, right_source_names); + + /// Assume that component from definite table if num_components is greater than for the other table. + if (left_num_components > right_num_components) + return {identifier, nullptr}; + if (left_num_components < right_num_components) + return {nullptr, identifier}; + } + return {}; + } + + TableBelonging table_belonging; + for (const auto & child : ast->children) + { + auto children_belonging = get_table_belonging(child); + if (!table_belonging.example_only_from_left) + table_belonging.example_only_from_left = children_belonging.example_only_from_left; + if (!table_belonging.example_only_from_right) + table_belonging.example_only_from_right = children_belonging.example_only_from_right; + } + + if (table_belonging.example_only_from_left && table_belonging.example_only_from_right) + throw Exception("Invalid columns in JOIN ON section. Columns " + + table_belonging.example_only_from_left->getAliasOrColumnName() + " and " + + table_belonging.example_only_from_right->getAliasOrColumnName() + + " are from different tables.", ErrorCodes::INVALID_JOIN_ON_EXPRESSION); + + return table_belonging; + }; + + std::function translate_qualified_names; + translate_qualified_names = [&](ASTPtr & ast, const DatabaseAndTableWithAlias & source_names, bool right_table) + { + if (auto * identifier = typeid_cast(ast.get())) + { + if (identifier->general()) + { + auto num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, source_names); + stripIdentifier(ast, num_components); + + if (right_table && source_columns.count(ast->getColumnName())) + source_names.makeQualifiedName(ast); + + } + return; + } + + for (auto & child : ast->children) + translate_qualified_names(child, source_names, right_table); + }; + + const auto supported_syntax = " Supported syntax: JOIN ON Expr([table.]column, ...) = Expr([table.]column, ...) " + "[AND Expr([table.]column, ...) = Expr([table.]column, ...) ...]"; + auto throwSyntaxException = [&](const String & msg) + { + throw Exception("Invalid expression for JOIN ON. " + msg + supported_syntax, ErrorCodes::INVALID_JOIN_ON_EXPRESSION); + }; + + /// For equal expression find out corresponding table for each part, translate qualified names and add asts to join keys. + auto add_columns_from_equals_expr = [&](const ASTPtr & expr) + { + auto * func_equals = typeid_cast(expr.get()); + if (!func_equals || func_equals->name != "equals") + throwSyntaxException("Expected equals expression, got " + queryToString(expr) + "."); + + ASTPtr left_ast = func_equals->arguments->children.at(0)->clone(); + ASTPtr right_ast = func_equals->arguments->children.at(1)->clone(); + + auto left_table_belonging = get_table_belonging(left_ast); + auto right_table_belonging = get_table_belonging(right_ast); + + bool can_be_left_part_from_left_table = left_table_belonging.example_only_from_right == nullptr; + bool can_be_left_part_from_right_table = left_table_belonging.example_only_from_left == nullptr; + bool can_be_right_part_from_left_table = right_table_belonging.example_only_from_right == nullptr; + bool can_be_right_part_from_right_table = right_table_belonging.example_only_from_left == nullptr; + + auto add_join_keys = [&](ASTPtr & ast_to_left_table, ASTPtr & ast_to_right_table) + { + translate_qualified_names(ast_to_left_table, left_source_names, false); + translate_qualified_names(ast_to_right_table, right_source_names, true); + + analyzed_join.key_asts_left.push_back(ast_to_left_table); + analyzed_join.key_names_left.push_back(ast_to_left_table->getColumnName()); + analyzed_join.key_asts_right.push_back(ast_to_right_table); + analyzed_join.key_names_right.push_back(ast_to_right_table->getAliasOrColumnName()); + }; + + /// Default variant when all identifiers may be from any table. + if (can_be_left_part_from_left_table && can_be_right_part_from_right_table) + add_join_keys(left_ast, right_ast); + else if (can_be_left_part_from_right_table && can_be_right_part_from_left_table) + add_join_keys(right_ast, left_ast); + else + { + auto * left_example = left_table_belonging.example_only_from_left ? + left_table_belonging.example_only_from_left : + left_table_belonging.example_only_from_right; + + auto * right_example = right_table_belonging.example_only_from_left ? + right_table_belonging.example_only_from_left : + right_table_belonging.example_only_from_right; + + auto left_name = queryToString(*left_example); + auto right_name = queryToString(*right_example); + auto expr_name = queryToString(expr); + + throwSyntaxException("In expression " + expr_name + " columns " + left_name + " and " + right_name + + " are from the same table but from different arguments of equal function."); + } + }; + + auto * func = typeid_cast(table_join.on_expression.get()); + if (func && func->name == "and") + { + for (const auto & expr : func->arguments->children) + add_columns_from_equals_expr(expr); + } + else + add_columns_from_equals_expr(table_join.on_expression); +} + +void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery * select_query, + const NameSet & source_columns, const Context & context) +{ + if (!select_query) + return; + + const ASTTablesInSelectQueryElement * node = select_query->join(); + + if (!node) + return; + + const auto & table_join = static_cast(*node->table_join); + const auto & table_expression = static_cast(*node->table_expression); + DatabaseAndTableWithAlias joined_table_name(table_expression, context.getCurrentDatabase()); + + auto add_name_to_join_keys = [&](Names & join_keys, ASTs & join_asts, const ASTPtr & ast, bool right_table) + { + String name; + if (right_table) + { + name = ast->getAliasOrColumnName(); + if (source_columns.count(name)) + name = joined_table_name.getQualifiedNamePrefix() + name; + } + else + name = ast->getColumnName(); + + join_keys.push_back(name); + join_asts.push_back(ast); + }; + + if (table_join.using_expression_list) + { + auto & keys = typeid_cast(*table_join.using_expression_list); + for (const auto & key : keys.children) + { + add_name_to_join_keys(analyzed_join.key_names_left, analyzed_join.key_asts_left, key, false); + add_name_to_join_keys(analyzed_join.key_names_right, analyzed_join.key_asts_right, key, true); + } + } + else if (table_join.on_expression) + collectJoinedColumnsFromJoinOnExpr(analyzed_join, select_query, source_columns, context); + + auto & columns_from_joined_table = analyzed_join.getColumnsFromJoinedTable(source_columns, context, select_query); + + NameSet joined_columns; + + auto & settings = context.getSettingsRef(); + + for (auto & column : columns_from_joined_table) + { + auto & column_name = column.name_and_type.name; + auto & column_type = column.name_and_type.type; + auto & original_name = column.original_name; + { + if (joined_columns.count(column_name)) /// Duplicate columns in the subquery for JOIN do not make sense. + continue; + + joined_columns.insert(column_name); + + bool make_nullable = settings.join_use_nulls && (table_join.kind == ASTTableJoin::Kind::Left || + table_join.kind == ASTTableJoin::Kind::Full); + auto type = make_nullable ? makeNullable(column_type) : column_type; + analyzed_join.columns_added_by_join.emplace_back(NameAndTypePair(column_name, std::move(type)), original_name); + } + } +} + +void removeDuplicateColumns(NamesAndTypesList & columns) +{ + std::set names; + for (auto it = columns.begin(); it != columns.end();) + { + if (names.emplace(it->name).second) + ++it; + else + columns.erase(it++); + } +} + +NamesAndTypesList collectSourceColumns(NamesAndTypesList source_columns, ASTSelectQuery * select_query, + const Context & context, StoragePtr & storage) +{ + if (!storage && select_query) + { + if (auto db_and_table = getDatabaseAndTable(*select_query, 0)) + storage = context.tryGetTable(db_and_table->database, db_and_table->table); + } + + if (storage) + { + auto physical_columns = storage->getColumns().getAllPhysical(); + if (source_columns.empty()) + source_columns.swap(physical_columns); + else + source_columns.insert(source_columns.end(), physical_columns.begin(), physical_columns.end()); + + if (select_query) + { + const auto & storage_aliases = storage->getColumns().aliases; + source_columns.insert(source_columns.end(), storage_aliases.begin(), storage_aliases.end()); + } + } + + removeDuplicateColumns(source_columns); + return source_columns; +} + +} + +SyntaxAnalyzerResult SyntaxAnalyzer::analyze(const ASTPtr & query, + const Context & context, + const StoragePtr & storage, + NamesAndTypesList source_columns, + const Names & required_result_columns, + size_t subquery_depth) const +{ + + SyntaxAnalyzerResult result; + result.storage = storage; + result.query = query->clone(); + auto * select_query = typeid_cast(result.query.get()); + result.source_columns = collectSourceColumns(std::move(source_columns), select_query, context, result.storage); + + const auto & settings = context.getSettingsRef(); + + Names source_columns_list; + source_columns_list.reserve(result.source_columns.size()); + for (const auto & type_name : result.source_columns) + source_columns_list.emplace_back(type_name.name); + NameSet source_columns_set(source_columns_list.begin(), source_columns_list.end()); + + translateQualifiedNames(result.query, select_query, source_columns_set, context); + + /// Depending on the user's profile, check for the execution rights + /// distributed subqueries inside the IN or JOIN sections and process these subqueries. + InJoinSubqueriesPreprocessor(context).process(select_query); + + /// Optimizes logical expressions. + LogicalExpressionsOptimizer(select_query, settings.optimize_min_equality_disjunction_chain_length.value).perform(); + + /// Creates a dictionary `aliases`: alias -> ASTPtr + { + LogAST log; + QueryAliasesVisitor query_aliases_visitor(log.stream()); + query_aliases_visitor.visit(result.query, result.aliases); + } + + /// Common subexpression elimination. Rewrite rules. + normalizeTree(result, source_columns_list, source_columns_set, context, select_query, settings.asterisk_left_columns_only != 0); + + /// Remove unneeded columns according to 'required_result_columns'. + /// Leave all selected columns in case of DISTINCT; columns that contain arrayJoin function inside. + /// Must be after 'normalizeTree' (after expanding aliases, for aliases not get lost) + /// and before 'executeScalarSubqueries', 'analyzeAggregation', etc. to avoid excessive calculations. + removeUnneededColumnsFromSelectClause(select_query, required_result_columns); + + /// Executing scalar subqueries - replacing them with constant values. + executeScalarSubqueries(result, select_query, context, subquery_depth); + + /// Optimize if with constant condition after constants was substituted instead of sclalar subqueries. + optimizeIfWithConstantCondition(result.query, result.aliases); + + /// GROUP BY injective function elimination. + optimizeGroupBy(select_query, source_columns_set, context); + + /// Remove duplicate items from ORDER BY. + optimizeOrderBy(select_query); + + // Remove duplicated elements from LIMIT BY clause. + optimizeLimitBy(select_query); + + /// Remove duplicated columns from USING(...). + optimizeUsing(select_query); + + /// array_join_alias_to_name, array_join_result_to_source. + getArrayJoinedColumns(result, select_query, source_columns_set); + + /// Push the predicate expression down to the subqueries. + result.rewrite_subqueries = PredicateExpressionsOptimizer(select_query, settings, context).optimize(); + + collectJoinedColumns(result.analyzed_join, select_query, source_columns_set, context); + + return result; +} + +} diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.h b/dbms/src/Interpreters/SyntaxAnalyzer.h new file mode 100644 index 00000000000..1eef07e0cf6 --- /dev/null +++ b/dbms/src/Interpreters/SyntaxAnalyzer.h @@ -0,0 +1,50 @@ +#pragma once + +#include + +namespace DB +{ + +class IStorage; +using StoragePtr = std::shared_ptr; + +struct SyntaxAnalyzerResult +{ + ASTPtr query; + + StoragePtr storage; + + NamesAndTypesList source_columns; + + using Aliases = std::unordered_map; + Aliases aliases; + + /// Which column is needed to be ARRAY-JOIN'ed to get the specified. + /// For example, for `SELECT s.v ... ARRAY JOIN a AS s` will get "s.v" -> "a.v". + NameToNameMap array_join_result_to_source; + + /// For the ARRAY JOIN section, mapping from the alias to the full column name. + /// For example, for `ARRAY JOIN [1,2] AS b` "b" -> "array(1,2)" will enter here. + NameToNameMap array_join_alias_to_name; + + /// The backward mapping for array_join_alias_to_name. + NameToNameMap array_join_name_to_alias; + + AnalyzedJoin analyzed_join; + + /// Predicate optimizer overrides the sub queries + bool rewrite_subqueries = false; +}; + +class SyntaxAnalyzer +{ +public: + SyntaxAnalyzerResult analyze(const ASTPtr & query, + const Context & context, + const StoragePtr & storage, + NamesAndTypesList source_columns, + const Names & required_result_columns = {}, + size_t subquery_depth = 0) const; +}; + +} diff --git a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index ebd575d314d..becc6727dc1 100644 --- a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include @@ -8,6 +8,7 @@ #include #include + namespace DB { @@ -43,7 +44,7 @@ void TranslateQualifiedNamesVisitor::visit(ASTIdentifier * identifier, ASTPtr & } /// In case if column from the joined table are in source columns, change it's name to qualified. - if (best_table_pos && source_columns.contains(ast->getColumnName())) + if (best_table_pos && source_columns.count(ast->getColumnName())) { const DatabaseAndTableWithAlias & table = tables[best_table_pos]; table.makeQualifiedName(ast); diff --git a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h index eaf5a4f7ba0..716f0163f95 100644 --- a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h +++ b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h @@ -5,24 +5,20 @@ #include #include +#include +#include +#include +#include #include namespace DB { -class ASTIdentifier; -class ASTQualifiedAsterisk; -class ASTSelectQuery; -struct ASTTableJoin; - -class NamesAndTypesList; - - /// It visits nodes, find columns (general identifiers and asterisks) and translate their names according to tables' names. class TranslateQualifiedNamesVisitor { public: - TranslateQualifiedNamesVisitor(const NamesAndTypesList & source_columns_, const std::vector & tables_, + TranslateQualifiedNamesVisitor(const NameSet & source_columns_, const std::vector & tables_, std::ostream * ostr_ = nullptr) : source_columns(source_columns_), tables(tables_), @@ -42,7 +38,7 @@ public: } private: - const NamesAndTypesList & source_columns; + const NameSet & source_columns; const std::vector & tables; mutable size_t visit_depth; std::ostream * ostr; From 748b0f70774729afbdf83c48a3aa38c8a3913122 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 7 Nov 2018 15:44:05 +0300 Subject: [PATCH 02/46] SyntaxAnalyzer (in progress). --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 2 +- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 9aeb885a78c..0553f7616eb 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -132,6 +132,7 @@ ExpressionAnalyzer::ExpressionAnalyzer( auto syntax_analyzer_result = SyntaxAnalyzer() .analyze(query, context, storage, source_columns, required_result_columns_, subquery_depth); query = syntax_analyzer_result.query; + storage = syntax_analyzer_result.storage; source_columns = syntax_analyzer_result.source_columns; aliases = syntax_analyzer_result.aliases; array_join_result_to_source = syntax_analyzer_result.array_join_result_to_source; @@ -139,7 +140,6 @@ ExpressionAnalyzer::ExpressionAnalyzer( array_join_name_to_alias = syntax_analyzer_result.array_join_name_to_alias; analyzed_join = syntax_analyzer_result.analyzed_join; rewrite_subqueries = syntax_analyzer_result.rewrite_subqueries; - storage = syntax_analyzer_result.storage; select_query = typeid_cast(query.get()); diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 68a55ecb13e..4c921ab6490 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -832,10 +832,9 @@ SyntaxAnalyzerResult SyntaxAnalyzer::analyze(const ASTPtr & query, const Names & required_result_columns, size_t subquery_depth) const { - SyntaxAnalyzerResult result; result.storage = storage; - result.query = query->clone(); + result.query = query; // ->clone(); auto * select_query = typeid_cast(result.query.get()); result.source_columns = collectSourceColumns(std::move(source_columns), select_query, context, result.storage); From 6ebfd2c42fe8402a78f12b7a8a0e551a21962f32 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 7 Nov 2018 15:55:19 +0300 Subject: [PATCH 03/46] Syntax analyzer (in progress). --- dbms/src/Interpreters/AnalyzedJoin.cpp | 10 +------ dbms/src/Interpreters/AnalyzedJoin.h | 7 +++-- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 21 ++++++++------- dbms/src/Interpreters/ExpressionAnalyzer.h | 4 +-- dbms/src/Interpreters/QueryAliasesVisitor.h | 1 + dbms/src/Interpreters/SyntaxAnalyzer.cpp | 26 ++++++++++++------- dbms/src/Interpreters/SyntaxAnalyzer.h | 1 + .../tests/expression_analyzer.cpp | 12 ++++----- 8 files changed, 41 insertions(+), 41 deletions(-) diff --git a/dbms/src/Interpreters/AnalyzedJoin.cpp b/dbms/src/Interpreters/AnalyzedJoin.cpp index d1542362ca7..6f61a4cab24 100644 --- a/dbms/src/Interpreters/AnalyzedJoin.cpp +++ b/dbms/src/Interpreters/AnalyzedJoin.cpp @@ -14,6 +14,7 @@ namespace DB { void AnalyzedJoin::createJoinedBlockActions(const NameSet & source_columns, + const JoinedColumnsList & columns_added_by_join, const ASTSelectQuery * select_query_with_join, const Context & context) { @@ -61,15 +62,6 @@ void AnalyzedJoin::createJoinedBlockActions(const NameSet & source_columns, required_columns_from_joined_table.insert(column.name_and_type.name); } -NamesAndTypesList AnalyzedJoin::getColumnsAddedByJoin() const -{ - NamesAndTypesList result; - for (const auto & joined_column : columns_added_by_join) - result.push_back(joined_column.name_and_type); - - return result; -} - const JoinedColumnsList & AnalyzedJoin::getColumnsFromJoinedTable( const NameSet & source_columns, const Context & context, const ASTSelectQuery * select_query_with_join) { diff --git a/dbms/src/Interpreters/AnalyzedJoin.h b/dbms/src/Interpreters/AnalyzedJoin.h index d74c6801dfe..cc1f6e223a9 100644 --- a/dbms/src/Interpreters/AnalyzedJoin.h +++ b/dbms/src/Interpreters/AnalyzedJoin.h @@ -61,24 +61,23 @@ struct AnalyzedJoin NameSet required_columns_from_joined_table; /// Columns which will be added to block, possible including some columns from right join key. - JoinedColumnsList columns_added_by_join; + JoinedColumnsList available_joined_columns; /// Such columns will be copied from left join keys during join. NameSet columns_added_by_join_from_right_keys; /// Actions which need to be calculated on joined block. ExpressionActionsPtr joined_block_actions; void createJoinedBlockActions(const NameSet & source_columns, + const JoinedColumnsList & columns_added_by_join, // Subset of available_joined_columns const ASTSelectQuery * select_query_with_join, const Context & context); - NamesAndTypesList getColumnsAddedByJoin() const; - const JoinedColumnsList & getColumnsFromJoinedTable(const NameSet & source_columns, const Context & context, const ASTSelectQuery * select_query_with_join); }; -class ASTTableExpression; +struct ASTTableExpression; NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpression & table_expression, const Context & context); } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 0553f7616eb..0e40aa45350 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -134,7 +134,6 @@ ExpressionAnalyzer::ExpressionAnalyzer( query = syntax_analyzer_result.query; storage = syntax_analyzer_result.storage; source_columns = syntax_analyzer_result.source_columns; - aliases = syntax_analyzer_result.aliases; array_join_result_to_source = syntax_analyzer_result.array_join_result_to_source; array_join_alias_to_name = syntax_analyzer_result.array_join_alias_to_name; array_join_name_to_alias = syntax_analyzer_result.array_join_name_to_alias; @@ -359,7 +358,7 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block & { NamesAndTypesList temp_columns = source_columns; temp_columns.insert(temp_columns.end(), array_join_columns.begin(), array_join_columns.end()); - for (const auto & joined_column : analyzed_join.columns_added_by_join) + for (const auto & joined_column : columns_added_by_join) temp_columns.push_back(joined_column.name_and_type); ExpressionActionsPtr temp_actions = std::make_shared(temp_columns, context); getRootActions(func->arguments->children.at(0), true, temp_actions); @@ -589,14 +588,17 @@ bool ExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, bool on void ExpressionAnalyzer::addJoinAction(ExpressionActionsPtr & actions, bool only_types) const { + NamesAndTypesList columns_added_by_join_list; + for (const auto & joined_column : columns_added_by_join) + columns_added_by_join_list.push_back(joined_column.name_and_type); + if (only_types) - actions->add(ExpressionAction::ordinaryJoin(nullptr, analyzed_join.key_names_left, - analyzed_join.getColumnsAddedByJoin())); + actions->add(ExpressionAction::ordinaryJoin(nullptr, analyzed_join.key_names_left, columns_added_by_join_list)); else for (auto & subquery_for_set : subqueries_for_sets) if (subquery_for_set.second.join) actions->add(ExpressionAction::ordinaryJoin(subquery_for_set.second.join, analyzed_join.key_names_left, - analyzed_join.getColumnsAddedByJoin())); + columns_added_by_join_list)); } bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_types) @@ -1088,7 +1090,7 @@ void ExpressionAnalyzer::collectUsedColumns() * (Do not assume that they are required for reading from the "left" table). */ NameSet available_joined_columns; - for (const auto & joined_column : analyzed_join.columns_added_by_join) + for (const auto & joined_column : analyzed_join.available_joined_columns) available_joined_columns.insert(joined_column.name_and_type.name); NameSet required_joined_columns; @@ -1103,18 +1105,19 @@ void ExpressionAnalyzer::collectUsedColumns() RequiredSourceColumnsVisitor columns_visitor(available_columns, required, ignored, available_joined_columns, required_joined_columns); columns_visitor.visit(query); - for (auto it = analyzed_join.columns_added_by_join.begin(); it != analyzed_join.columns_added_by_join.end();) + columns_added_by_join = analyzed_join.available_joined_columns; + for (auto it = columns_added_by_join.begin(); it != columns_added_by_join.end();) { if (required_joined_columns.count(it->name_and_type.name)) ++it; else - analyzed_join.columns_added_by_join.erase(it++); + columns_added_by_join.erase(it++); } NameSet source_columns_set; for (const auto & type_name : source_columns) source_columns_set.insert(type_name.name); - analyzed_join.createJoinedBlockActions(source_columns_set, select_query, context); + analyzed_join.createJoinedBlockActions(source_columns_set, columns_added_by_join, select_query, context); /// Some columns from right join key may be used in query. This columns will be appended to block during join. for (const auto & right_key_name : analyzed_join.key_names_right) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index 333ff26456d..b1d43806c94 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -56,9 +56,6 @@ struct ExpressionAnalyzerData bool has_global_subqueries = false; - using Aliases = std::unordered_map; - Aliases aliases; - /// Which column is needed to be ARRAY-JOIN'ed to get the specified. /// For example, for `SELECT s.v ... ARRAY JOIN a AS s` will get "s.v" -> "a.v". NameToNameMap array_join_result_to_source; @@ -231,6 +228,7 @@ private: bool do_global; /// Do I need to prepare for execution global subqueries when analyzing the query. AnalyzedJoin analyzed_join; + JoinedColumnsList columns_added_by_join; /// Subset of analyzed_join.available_joined_columns /** Remove all unnecessary columns from the list of all available columns of the table (`columns`). * At the same time, form a set of unknown columns (`unknown_required_source_columns`), diff --git a/dbms/src/Interpreters/QueryAliasesVisitor.h b/dbms/src/Interpreters/QueryAliasesVisitor.h index efcc2290b88..87e5af688da 100644 --- a/dbms/src/Interpreters/QueryAliasesVisitor.h +++ b/dbms/src/Interpreters/QueryAliasesVisitor.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 3331b210064..088a2895cb0 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -61,10 +61,16 @@ void translateQualifiedNames(ASTPtr & query, ASTSelectQuery * select_query, visitor.visit(query); } -void normalizeTree(SyntaxAnalyzerResult & result, const Names & source_columns, const NameSet & source_columns_set, - const Context & context, const ASTSelectQuery * select_query, bool asterisk_left_columns_only) +void normalizeTree( + SyntaxAnalyzerResult & result, + const Names & source_columns, + const NameSet & source_columns_set, + const StoragePtr & storage, + const Context & context, + const ASTSelectQuery * select_query, + bool asterisk_left_columns_only) { - Names all_columns_name(source_columns.begin(), source_columns.end()); + Names all_columns_name = storage ? storage->getColumns().ordinary.getNames() : source_columns; if (!asterisk_left_columns_only) { @@ -494,7 +500,8 @@ void optimizeUsing(const ASTSelectQuery * select_query) expression_list = uniq_expressions_list; } -void getArrayJoinedColumns(SyntaxAnalyzerResult & result, const ASTSelectQuery * select_query, const NameSet & source_columns) +void getArrayJoinedColumns(SyntaxAnalyzerResult & result, const ASTSelectQuery * select_query, + const Names & source_columns, const NameSet & source_columns_set) { if (select_query && select_query->array_join_expression_list()) { @@ -532,7 +539,7 @@ void getArrayJoinedColumns(SyntaxAnalyzerResult & result, const ASTSelectQuery * String result_name = expr->getAliasOrColumnName(); /// This is an array. - if (!typeid_cast(expr.get()) || source_columns.count(source_name)) + if (!typeid_cast(expr.get()) || source_columns_set.count(source_name)) { result.array_join_result_to_source[result_name] = source_name; } @@ -778,7 +785,7 @@ void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery * s bool make_nullable = settings.join_use_nulls && (table_join.kind == ASTTableJoin::Kind::Left || table_join.kind == ASTTableJoin::Kind::Full); auto type = make_nullable ? makeNullable(column_type) : column_type; - analyzed_join.columns_added_by_join.emplace_back(NameAndTypePair(column_name, std::move(type)), original_name); + analyzed_join.available_joined_columns.emplace_back(NameAndTypePair(column_name, std::move(type)), original_name); } } } @@ -858,12 +865,13 @@ SyntaxAnalyzerResult SyntaxAnalyzer::analyze(const ASTPtr & query, /// Creates a dictionary `aliases`: alias -> ASTPtr { LogAST log; - QueryAliasesVisitor query_aliases_visitor(aliases, log.stream()); + QueryAliasesVisitor query_aliases_visitor(result.aliases, log.stream()); query_aliases_visitor.visit(query); } /// Common subexpression elimination. Rewrite rules. - normalizeTree(result, source_columns_list, source_columns_set, context, select_query, settings.asterisk_left_columns_only != 0); + normalizeTree(result, source_columns_list, source_columns_set, storage, + context, select_query, settings.asterisk_left_columns_only != 0); /// Remove unneeded columns according to 'required_result_columns'. /// Leave all selected columns in case of DISTINCT; columns that contain arrayJoin function inside. @@ -890,7 +898,7 @@ SyntaxAnalyzerResult SyntaxAnalyzer::analyze(const ASTPtr & query, optimizeUsing(select_query); /// array_join_alias_to_name, array_join_result_to_source. - getArrayJoinedColumns(result, select_query, source_columns_set); + getArrayJoinedColumns(result, select_query, source_columns_list, source_columns_set); /// Push the predicate expression down to the subqueries. result.rewrite_subqueries = PredicateExpressionsOptimizer(select_query, settings, context).optimize(); diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.h b/dbms/src/Interpreters/SyntaxAnalyzer.h index 1eef07e0cf6..c7bc921265f 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.h +++ b/dbms/src/Interpreters/SyntaxAnalyzer.h @@ -16,6 +16,7 @@ struct SyntaxAnalyzerResult NamesAndTypesList source_columns; + /// Note: used only in tests. using Aliases = std::unordered_map; Aliases aliases; diff --git a/dbms/src/Interpreters/tests/expression_analyzer.cpp b/dbms/src/Interpreters/tests/expression_analyzer.cpp index a523e3f43b8..eaa7731dd69 100644 --- a/dbms/src/Interpreters/tests/expression_analyzer.cpp +++ b/dbms/src/Interpreters/tests/expression_analyzer.cpp @@ -8,7 +8,7 @@ #include #include -#include +#include #include #include @@ -35,11 +35,9 @@ struct TestEntry { ASTPtr ast = parse(query); - ExpressionAnalyzer analyzer(ast, context, {}, source_columns, required_result_columns); + auto res = SyntaxAnalyzer().analyze(ast, context, {}, source_columns, required_result_columns); - const ExpressionAnalyzerData & data = analyzer.getAnalyzedData(); - - if (!checkAliases(data)) + if (!checkAliases(res)) { collectWithAnalysers(context, ast); return false; @@ -49,9 +47,9 @@ struct TestEntry } private: - bool checkAliases(const ExpressionAnalyzerData & data) + bool checkAliases(const SyntaxAnalyzerResult & res) { - for (const auto & alias : data.aliases) + for (const auto & alias : res.aliases) { const String & alias_name = alias.first; if (expected_aliases.count(alias_name) == 0 || From a5a36d007832e5b2c7a41309ed4d24f8c3b4af15 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 8 Nov 2018 12:00:25 +0300 Subject: [PATCH 04/46] Syntax analyzer (in progress). --- dbms/src/Interpreters/AnalyzedJoin.cpp | 19 +- dbms/src/Interpreters/AnalyzedJoin.h | 21 +- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 11 +- dbms/src/Interpreters/ExpressionAnalyzer.h | 61 +--- .../Interpreters/GlobalSubqueriesVisitor.h | 3 + dbms/src/Interpreters/SyntaxAnalyzer.cpp | 290 +++++++++++------- 6 files changed, 216 insertions(+), 189 deletions(-) diff --git a/dbms/src/Interpreters/AnalyzedJoin.cpp b/dbms/src/Interpreters/AnalyzedJoin.cpp index 6f61a4cab24..9c44593ac55 100644 --- a/dbms/src/Interpreters/AnalyzedJoin.cpp +++ b/dbms/src/Interpreters/AnalyzedJoin.cpp @@ -13,18 +13,20 @@ namespace DB { -void AnalyzedJoin::createJoinedBlockActions(const NameSet & source_columns, - const JoinedColumnsList & columns_added_by_join, - const ASTSelectQuery * select_query_with_join, - const Context & context) +ExpressionActionsPtr AnalyzedJoin::createJoinedBlockActions( + const NameSet & source_columns, + const JoinedColumnsList & columns_added_by_join, + const ASTSelectQuery * select_query_with_join, + const Context & context, + NameSet & required_columns_from_joined_table) const { if (!select_query_with_join) - return; + return nullptr; const ASTTablesInSelectQueryElement * join = select_query_with_join->join(); if (!join) - return; + return nullptr; const auto & join_params = static_cast(*join->table_join); @@ -41,13 +43,12 @@ void AnalyzedJoin::createJoinedBlockActions(const NameSet & source_columns, required_columns_set.insert(joined_column.name_and_type.name); Names required_columns(required_columns_set.begin(), required_columns_set.end()); - const auto & columns_from_joined_table = getColumnsFromJoinedTable(source_columns, context, select_query_with_join); NamesAndTypesList source_column_names; for (auto & column : columns_from_joined_table) source_column_names.emplace_back(column.name_and_type); ExpressionAnalyzer analyzer(expression_list, context, nullptr, source_column_names, required_columns); - joined_block_actions = analyzer.getActions(false); + auto joined_block_actions = analyzer.getActions(false); auto required_action_columns = joined_block_actions->getRequiredColumns(); required_columns_from_joined_table.insert(required_action_columns.begin(), required_action_columns.end()); @@ -60,6 +61,8 @@ void AnalyzedJoin::createJoinedBlockActions(const NameSet & source_columns, for (auto & column : columns_added_by_join) if (!sample.has(column.name_and_type.name)) required_columns_from_joined_table.insert(column.name_and_type.name); + + return joined_block_actions; } const JoinedColumnsList & AnalyzedJoin::getColumnsFromJoinedTable( diff --git a/dbms/src/Interpreters/AnalyzedJoin.h b/dbms/src/Interpreters/AnalyzedJoin.h index cc1f6e223a9..6b8105d9184 100644 --- a/dbms/src/Interpreters/AnalyzedJoin.h +++ b/dbms/src/Interpreters/AnalyzedJoin.h @@ -57,20 +57,17 @@ struct AnalyzedJoin /// All columns which can be read from joined table. Duplicating names are qualified. JoinedColumnsList columns_from_joined_table; - /// Columns which will be used in query to the joined query. Duplicating names are qualified. - NameSet required_columns_from_joined_table; - - /// Columns which will be added to block, possible including some columns from right join key. + /// Columns from joined table which may be added to block. + /// It's columns_from_joined_table without duplicate columns and possibly modified types. JoinedColumnsList available_joined_columns; - /// Such columns will be copied from left join keys during join. - NameSet columns_added_by_join_from_right_keys; - /// Actions which need to be calculated on joined block. - ExpressionActionsPtr joined_block_actions; - void createJoinedBlockActions(const NameSet & source_columns, - const JoinedColumnsList & columns_added_by_join, // Subset of available_joined_columns - const ASTSelectQuery * select_query_with_join, - const Context & context); + ExpressionActionsPtr createJoinedBlockActions( + const NameSet & source_columns, + const JoinedColumnsList & columns_added_by_join, /// Subset of available_joined_columns. + const ASTSelectQuery * select_query_with_join, + const Context & context, + NameSet & required_columns_from_joined_table /// Columns which will be used in query from joined table. + ) const; const JoinedColumnsList & getColumnsFromJoinedTable(const NameSet & source_columns, const Context & context, diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 0e40aa45350..68cee7d86da 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -681,7 +681,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty Names original_columns; for (const auto & column : analyzed_join.columns_from_joined_table) - if (analyzed_join.required_columns_from_joined_table.count(column.name_and_type.name)) + if (required_columns_from_joined_table.count(column.name_and_type.name)) original_columns.emplace_back(column.original_name); auto interpreter = interpretSubquery(table, context, subquery_depth, original_columns); @@ -692,7 +692,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty /// Alias duplicating columns as qualified. for (const auto & column : analyzed_join.columns_from_joined_table) - if (analyzed_join.required_columns_from_joined_table.count(column.name_and_type.name)) + if (required_columns_from_joined_table.count(column.name_and_type.name)) subquery_for_set.joined_block_aliases.emplace_back(column.original_name, column.name_and_type.name); auto sample_block = subquery_for_set.source->getHeader(); @@ -708,12 +708,12 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty } } - analyzed_join.joined_block_actions->execute(sample_block); + joined_block_actions->execute(sample_block); /// TODO You do not need to set this up when JOIN is only needed on remote servers. subquery_for_set.join = join; subquery_for_set.join->setSampleBlock(sample_block); - subquery_for_set.joined_block_actions = analyzed_join.joined_block_actions; + subquery_for_set.joined_block_actions = joined_block_actions; } addJoinAction(step.actions, false); @@ -1117,7 +1117,8 @@ void ExpressionAnalyzer::collectUsedColumns() NameSet source_columns_set; for (const auto & type_name : source_columns) source_columns_set.insert(type_name.name); - analyzed_join.createJoinedBlockActions(source_columns_set, columns_added_by_join, select_query, context); + joined_block_actions = analyzed_join.createJoinedBlockActions( + source_columns_set, columns_added_by_join, select_query, context, required_columns_from_joined_table); /// Some columns from right join key may be used in query. This columns will be appended to block during join. for (const auto & right_key_name : analyzed_join.key_names_right) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index b1d43806c94..caa9c5008f8 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -73,6 +73,19 @@ struct ExpressionAnalyzerData /// Predicate optimizer overrides the sub queries bool rewrite_subqueries = false; + /// Columns will be added to block by join. + JoinedColumnsList columns_added_by_join; /// Subset of analyzed_join.available_joined_columns + + /// Actions which need to be calculated on joined block. + ExpressionActionsPtr joined_block_actions; + + /// Columns which will be used in query from joined table. Duplicate names are qualified. + NameSet required_columns_from_joined_table; + + /// Such columns will be copied from left join keys during join. + /// Example: select right from tab1 join tab2 on left + 1 = right + NameSet columns_added_by_join_from_right_keys; + protected: ExpressionAnalyzerData(const NamesAndTypesList & source_columns_, const Names & required_result_columns_, @@ -228,7 +241,6 @@ private: bool do_global; /// Do I need to prepare for execution global subqueries when analyzing the query. AnalyzedJoin analyzed_join; - JoinedColumnsList columns_added_by_join; /// Subset of analyzed_join.available_joined_columns /** Remove all unnecessary columns from the list of all available columns of the table (`columns`). * At the same time, form a set of unknown columns (`unknown_required_source_columns`), @@ -236,45 +248,9 @@ private: */ void collectUsedColumns(); - /** Find the columns that are obtained by JOIN. - */ - void collectJoinedColumns(NameSet & joined_columns); - /// Parse JOIN ON expression and collect ASTs for joined columns. - void collectJoinedColumnsFromJoinOnExpr(); - - /** For star nodes(`*`), expand them to a list of all columns. - * For literal nodes, substitute aliases. - */ - void normalizeTree(); - - /// Eliminates injective function calls and constant expressions from group by statement - void optimizeGroupBy(); - - /// Remove duplicate items from ORDER BY. - void optimizeOrderBy(); - - void optimizeLimitBy(); - - /// Remove duplicated columns from USING(...). - void optimizeUsing(); - - /// remove Function_if AST if condition is constant - void optimizeIfWithConstantCondition(); - void optimizeIfWithConstantConditionImpl(ASTPtr & current_ast); - bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & value) const; - - /// Replacing scalar subqueries with constant values. - void executeScalarSubqueries(); - /// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables. void initGlobalSubqueriesAndExternalTables(); - /** Initialize InterpreterSelectQuery for a subquery in the GLOBAL IN/JOIN section, - * create a temporary table of type Memory and store it in the external_tables dictionary. - */ - void addExternalStorage(ASTPtr & subquery_or_table_name); - - void getArrayJoinedColumns(); void addMultipleArrayJoinAction(ExpressionActionsPtr & actions) const; void addJoinAction(ExpressionActionsPtr & actions, bool only_types) const; @@ -311,17 +287,6 @@ private: void makeSetsForIndexImpl(const ASTPtr & node, const Block & sample_block); - /** Translate qualified names such as db.table.column, table.column, table_alias.column - * to unqualified names. This is done in a poor transitional way: - * only one ("main") table is supported. Ambiguity is not detected or resolved. - */ - void translateQualifiedNames(); - - /** Sometimes we have to calculate more columns in SELECT clause than will be returned from query. - * This is the case when we have DISTINCT or arrayJoin: we require more columns in SELECT even if we need less columns in result. - */ - void removeUnneededColumnsFromSelectClause(); - bool isRemoteStorage() const; }; diff --git a/dbms/src/Interpreters/GlobalSubqueriesVisitor.h b/dbms/src/Interpreters/GlobalSubqueriesVisitor.h index 635a635755a..ce84e3b81fa 100644 --- a/dbms/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/dbms/src/Interpreters/GlobalSubqueriesVisitor.h @@ -75,6 +75,9 @@ private: return false; } + /** Initialize InterpreterSelectQuery for a subquery in the GLOBAL IN/JOIN section, + * create a temporary table of type Memory and store it in the external_tables dictionary. + */ void addExternalStorage(ASTPtr & subquery_or_table_name_or_table_expression) const { /// With nondistributed queries, creating temporary tables does not make sense. diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 088a2895cb0..b356568e00e 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -48,6 +48,180 @@ namespace using LogAST = DebugASTLog; /// set to true to enable logs using Aliases = std::unordered_map; +/// Add columns from storage to source_columns list. +NamesAndTypesList collectSourceColumns(NamesAndTypesList source_columns, ASTSelectQuery * select_query, + const Context & context, StoragePtr & storage); + +/// Translate qualified names such as db.table.column, table.column, table_alias.column to unqualified names. +void translateQualifiedNames(ASTPtr & query, ASTSelectQuery * select_query, + const NameSet & source_columns, const Context & context); + +/// For star nodes(`*`), expand them to a list of all columns. For literal nodes, substitute aliases. +void normalizeTree( + SyntaxAnalyzerResult & result, + const Names & source_columns, + const NameSet & source_columns_set, + const StoragePtr & storage, + const Context & context, + const ASTSelectQuery * select_query, + bool asterisk_left_columns_only); + +/// Sometimes we have to calculate more columns in SELECT clause than will be returned from query. +/// This is the case when we have DISTINCT or arrayJoin: we require more columns in SELECT even if we need less columns in result. +void removeUnneededColumnsFromSelectClause(const ASTSelectQuery * select_query, const Names & required_result_columns); + +/// Replacing scalar subqueries with constant values. +void executeScalarSubqueries(SyntaxAnalyzerResult & result, const ASTSelectQuery * select_query, + const Context & context, size_t subquery_depth); + +/// Remove Function_if AST if condition is constant. +void optimizeIfWithConstantCondition(ASTPtr & current_ast, Aliases & aliases); + +/// Eliminates injective function calls and constant expressions from group by statement. +void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_columns, const Context & context); + +/// Remove duplicate items from ORDER BY. +void optimizeOrderBy(const ASTSelectQuery * select_query); + +/// Remove duplicate items from LIMIT BY. +void optimizeLimitBy(const ASTSelectQuery * select_query); + +/// Remove duplicated columns from USING(...). +void optimizeUsing(const ASTSelectQuery * select_query); + +void getArrayJoinedColumns(SyntaxAnalyzerResult & result, const ASTSelectQuery * select_query, + const Names & source_columns, const NameSet & source_columns_set); + +/// Parse JOIN ON expression and collect ASTs for joined columns. +void collectJoinedColumnsFromJoinOnExpr(AnalyzedJoin & analyzed_join, const ASTSelectQuery * select_query, + const NameSet & source_columns, const Context & context); + +/// Find the columns that are obtained by JOIN. +void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery * select_query, + const NameSet & source_columns, const Context & context); +} + +SyntaxAnalyzerResult SyntaxAnalyzer::analyze( + const ASTPtr & query, + const Context & context, + const StoragePtr & storage, + NamesAndTypesList source_columns, + const Names & required_result_columns, + size_t subquery_depth) const +{ + SyntaxAnalyzerResult result; + result.storage = storage; + result.query = query; // ->clone(); + auto * select_query = typeid_cast(result.query.get()); + result.source_columns = collectSourceColumns(std::move(source_columns), select_query, context, result.storage); + + const auto & settings = context.getSettingsRef(); + + Names source_columns_list; + source_columns_list.reserve(result.source_columns.size()); + for (const auto & type_name : result.source_columns) + source_columns_list.emplace_back(type_name.name); + NameSet source_columns_set(source_columns_list.begin(), source_columns_list.end()); + + translateQualifiedNames(result.query, select_query, source_columns_set, context); + + /// Depending on the user's profile, check for the execution rights + /// distributed subqueries inside the IN or JOIN sections and process these subqueries. + InJoinSubqueriesPreprocessor(context).process(select_query); + + /// Optimizes logical expressions. + LogicalExpressionsOptimizer(select_query, settings.optimize_min_equality_disjunction_chain_length.value).perform(); + + /// Creates a dictionary `aliases`: alias -> ASTPtr + { + LogAST log; + QueryAliasesVisitor query_aliases_visitor(result.aliases, log.stream()); + query_aliases_visitor.visit(query); + } + + /// Common subexpression elimination. Rewrite rules. + normalizeTree(result, source_columns_list, source_columns_set, storage, + context, select_query, settings.asterisk_left_columns_only != 0); + + /// Remove unneeded columns according to 'required_result_columns'. + /// Leave all selected columns in case of DISTINCT; columns that contain arrayJoin function inside. + /// Must be after 'normalizeTree' (after expanding aliases, for aliases not get lost) + /// and before 'executeScalarSubqueries', 'analyzeAggregation', etc. to avoid excessive calculations. + removeUnneededColumnsFromSelectClause(select_query, required_result_columns); + + /// Executing scalar subqueries - replacing them with constant values. + executeScalarSubqueries(result, select_query, context, subquery_depth); + + /// Optimize if with constant condition after constants was substituted instead of sclalar subqueries. + optimizeIfWithConstantCondition(result.query, result.aliases); + + /// GROUP BY injective function elimination. + optimizeGroupBy(select_query, source_columns_set, context); + + /// Remove duplicate items from ORDER BY. + optimizeOrderBy(select_query); + + // Remove duplicated elements from LIMIT BY clause. + optimizeLimitBy(select_query); + + /// Remove duplicated columns from USING(...). + optimizeUsing(select_query); + + /// array_join_alias_to_name, array_join_result_to_source. + getArrayJoinedColumns(result, select_query, source_columns_list, source_columns_set); + + /// Push the predicate expression down to the subqueries. + result.rewrite_subqueries = PredicateExpressionsOptimizer(select_query, settings, context).optimize(); + + collectJoinedColumns(result.analyzed_join, select_query, source_columns_set, context); + + return result; +} + + +namespace +{ + +void removeDuplicateColumns(NamesAndTypesList & columns) +{ + std::set names; + for (auto it = columns.begin(); it != columns.end();) + { + if (names.emplace(it->name).second) + ++it; + else + columns.erase(it++); + } +} + +NamesAndTypesList collectSourceColumns(NamesAndTypesList source_columns, ASTSelectQuery * select_query, + const Context & context, StoragePtr & storage) +{ + if (!storage && select_query) + { + if (auto db_and_table = getDatabaseAndTable(*select_query, 0)) + storage = context.tryGetTable(db_and_table->database, db_and_table->table); + } + + if (storage) + { + auto physical_columns = storage->getColumns().getAllPhysical(); + if (source_columns.empty()) + source_columns.swap(physical_columns); + else + source_columns.insert(source_columns.end(), physical_columns.begin(), physical_columns.end()); + + if (select_query) + { + const auto & storage_aliases = storage->getColumns().aliases; + source_columns.insert(source_columns.end(), storage_aliases.begin(), storage_aliases.end()); + } + } + + removeDuplicateColumns(source_columns); + return source_columns; +} + void translateQualifiedNames(ASTPtr & query, ASTSelectQuery * select_query, const NameSet & source_columns, const Context & context) { @@ -790,122 +964,6 @@ void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery * s } } -void removeDuplicateColumns(NamesAndTypesList & columns) -{ - std::set names; - for (auto it = columns.begin(); it != columns.end();) - { - if (names.emplace(it->name).second) - ++it; - else - columns.erase(it++); - } -} - -NamesAndTypesList collectSourceColumns(NamesAndTypesList source_columns, ASTSelectQuery * select_query, - const Context & context, StoragePtr & storage) -{ - if (!storage && select_query) - { - if (auto db_and_table = getDatabaseAndTable(*select_query, 0)) - storage = context.tryGetTable(db_and_table->database, db_and_table->table); - } - - if (storage) - { - auto physical_columns = storage->getColumns().getAllPhysical(); - if (source_columns.empty()) - source_columns.swap(physical_columns); - else - source_columns.insert(source_columns.end(), physical_columns.begin(), physical_columns.end()); - - if (select_query) - { - const auto & storage_aliases = storage->getColumns().aliases; - source_columns.insert(source_columns.end(), storage_aliases.begin(), storage_aliases.end()); - } - } - - removeDuplicateColumns(source_columns); - return source_columns; -} - -} - -SyntaxAnalyzerResult SyntaxAnalyzer::analyze(const ASTPtr & query, - const Context & context, - const StoragePtr & storage, - NamesAndTypesList source_columns, - const Names & required_result_columns, - size_t subquery_depth) const -{ - SyntaxAnalyzerResult result; - result.storage = storage; - result.query = query; // ->clone(); - auto * select_query = typeid_cast(result.query.get()); - result.source_columns = collectSourceColumns(std::move(source_columns), select_query, context, result.storage); - - const auto & settings = context.getSettingsRef(); - - Names source_columns_list; - source_columns_list.reserve(result.source_columns.size()); - for (const auto & type_name : result.source_columns) - source_columns_list.emplace_back(type_name.name); - NameSet source_columns_set(source_columns_list.begin(), source_columns_list.end()); - - translateQualifiedNames(result.query, select_query, source_columns_set, context); - - /// Depending on the user's profile, check for the execution rights - /// distributed subqueries inside the IN or JOIN sections and process these subqueries. - InJoinSubqueriesPreprocessor(context).process(select_query); - - /// Optimizes logical expressions. - LogicalExpressionsOptimizer(select_query, settings.optimize_min_equality_disjunction_chain_length.value).perform(); - - /// Creates a dictionary `aliases`: alias -> ASTPtr - { - LogAST log; - QueryAliasesVisitor query_aliases_visitor(result.aliases, log.stream()); - query_aliases_visitor.visit(query); - } - - /// Common subexpression elimination. Rewrite rules. - normalizeTree(result, source_columns_list, source_columns_set, storage, - context, select_query, settings.asterisk_left_columns_only != 0); - - /// Remove unneeded columns according to 'required_result_columns'. - /// Leave all selected columns in case of DISTINCT; columns that contain arrayJoin function inside. - /// Must be after 'normalizeTree' (after expanding aliases, for aliases not get lost) - /// and before 'executeScalarSubqueries', 'analyzeAggregation', etc. to avoid excessive calculations. - removeUnneededColumnsFromSelectClause(select_query, required_result_columns); - - /// Executing scalar subqueries - replacing them with constant values. - executeScalarSubqueries(result, select_query, context, subquery_depth); - - /// Optimize if with constant condition after constants was substituted instead of sclalar subqueries. - optimizeIfWithConstantCondition(result.query, result.aliases); - - /// GROUP BY injective function elimination. - optimizeGroupBy(select_query, source_columns_set, context); - - /// Remove duplicate items from ORDER BY. - optimizeOrderBy(select_query); - - // Remove duplicated elements from LIMIT BY clause. - optimizeLimitBy(select_query); - - /// Remove duplicated columns from USING(...). - optimizeUsing(select_query); - - /// array_join_alias_to_name, array_join_result_to_source. - getArrayJoinedColumns(result, select_query, source_columns_list, source_columns_set); - - /// Push the predicate expression down to the subqueries. - result.rewrite_subqueries = PredicateExpressionsOptimizer(select_query, settings, context).optimize(); - - collectJoinedColumns(result.analyzed_join, select_query, source_columns_set, context); - - return result; } } From 90120493bf4a2805b8830163fb60a9e853f64c3b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 8 Nov 2018 12:06:11 +0300 Subject: [PATCH 05/46] Syntax analyzer (in progress). --- dbms/src/Interpreters/AnalyzedJoin.cpp | 1 - dbms/src/Interpreters/AnalyzedJoin.h | 1 - dbms/src/Interpreters/ExpressionAnalyzer.cpp | 9 +++------ 3 files changed, 3 insertions(+), 8 deletions(-) diff --git a/dbms/src/Interpreters/AnalyzedJoin.cpp b/dbms/src/Interpreters/AnalyzedJoin.cpp index 9c44593ac55..5d1a031e560 100644 --- a/dbms/src/Interpreters/AnalyzedJoin.cpp +++ b/dbms/src/Interpreters/AnalyzedJoin.cpp @@ -14,7 +14,6 @@ namespace DB { ExpressionActionsPtr AnalyzedJoin::createJoinedBlockActions( - const NameSet & source_columns, const JoinedColumnsList & columns_added_by_join, const ASTSelectQuery * select_query_with_join, const Context & context, diff --git a/dbms/src/Interpreters/AnalyzedJoin.h b/dbms/src/Interpreters/AnalyzedJoin.h index 6b8105d9184..4c215821755 100644 --- a/dbms/src/Interpreters/AnalyzedJoin.h +++ b/dbms/src/Interpreters/AnalyzedJoin.h @@ -62,7 +62,6 @@ struct AnalyzedJoin JoinedColumnsList available_joined_columns; ExpressionActionsPtr createJoinedBlockActions( - const NameSet & source_columns, const JoinedColumnsList & columns_added_by_join, /// Subset of available_joined_columns. const ASTSelectQuery * select_query_with_join, const Context & context, diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 68cee7d86da..66fd6492095 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -659,7 +659,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty if (!subquery_for_set.join) { JoinPtr join = std::make_shared( - analyzed_join.key_names_left, analyzed_join.key_names_right, analyzed_join.columns_added_by_join_from_right_keys, + analyzed_join.key_names_left, analyzed_join.key_names_right, columns_added_by_join_from_right_keys, settings.join_use_nulls, settings.size_limits_for_join, join_params.kind, join_params.strictness); @@ -1114,16 +1114,13 @@ void ExpressionAnalyzer::collectUsedColumns() columns_added_by_join.erase(it++); } - NameSet source_columns_set; - for (const auto & type_name : source_columns) - source_columns_set.insert(type_name.name); joined_block_actions = analyzed_join.createJoinedBlockActions( - source_columns_set, columns_added_by_join, select_query, context, required_columns_from_joined_table); + columns_added_by_join, select_query, context, required_columns_from_joined_table); /// Some columns from right join key may be used in query. This columns will be appended to block during join. for (const auto & right_key_name : analyzed_join.key_names_right) if (required_joined_columns.count(right_key_name)) - analyzed_join.columns_added_by_join_from_right_keys.insert(right_key_name); + columns_added_by_join_from_right_keys.insert(right_key_name); /// Insert the columns required for the ARRAY JOIN calculation into the required columns list. NameSet array_join_sources; From 29fe5ea5b6922af8dbdc1b14c4ce9ee025ecbb21 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 8 Nov 2018 14:17:31 +0300 Subject: [PATCH 06/46] Review fixes. --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 39 +------------------- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 4 +- dbms/src/Interpreters/SyntaxAnalyzer.h | 19 +++++++++- 3 files changed, 20 insertions(+), 42 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 66fd6492095..23efe225c57 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -77,45 +77,12 @@ using LogAST = DebugASTLog; /// set to true to enable logs namespace ErrorCodes { - extern const int BAD_ARGUMENTS; - extern const int MULTIPLE_EXPRESSIONS_FOR_ALIAS; extern const int UNKNOWN_IDENTIFIER; - extern const int CYCLIC_ALIASES; - extern const int NOT_FOUND_COLUMN_IN_BLOCK; - extern const int INCORRECT_ELEMENT_OF_SET; - extern const int ALIAS_REQUIRED; - extern const int EMPTY_NESTED_TABLE; - extern const int DUPLICATE_COLUMN; - extern const int FUNCTION_CANNOT_HAVE_PARAMETERS; extern const int ILLEGAL_AGGREGATION; - extern const int SUPPORT_IS_DISABLED; - extern const int TOO_DEEP_AST; - extern const int TOO_BIG_AST; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int CONDITIONAL_TREE_PARENT_NOT_FOUND; - extern const int INVALID_JOIN_ON_EXPRESSION; extern const int EXPECTED_ALL_OR_ANY; } -//namespace -//{ -// -//void removeDuplicateColumns(NamesAndTypesList & columns) -//{ -// std::set names; -// for (auto it = columns.begin(); it != columns.end();) -// { -// if (names.emplace(it->name).second) -// ++it; -// else -// columns.erase(it++); -// } -//} -// -//} - - ExpressionAnalyzer::ExpressionAnalyzer( const ASTPtr & query_, const Context & context_, @@ -129,8 +96,8 @@ ExpressionAnalyzer::ExpressionAnalyzer( query(query_), context(context_), settings(context.getSettings()), storage(storage_), subquery_depth(subquery_depth_), do_global(do_global_) { - auto syntax_analyzer_result = SyntaxAnalyzer() - .analyze(query, context, storage, source_columns, required_result_columns_, subquery_depth); + auto syntax_analyzer_result = SyntaxAnalyzer(context, storage) + .analyze(query, source_columns, required_result_columns_, subquery_depth); query = syntax_analyzer_result.query; storage = syntax_analyzer_result.storage; source_columns = syntax_analyzer_result.source_columns; @@ -142,8 +109,6 @@ ExpressionAnalyzer::ExpressionAnalyzer( select_query = typeid_cast(query.get()); - // removeDuplicateColumns(source_columns); - /// Delete the unnecessary from `source_columns` list. Create `unknown_required_source_columns`. Form `columns_added_by_join`. collectUsedColumns(); diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index b356568e00e..49e849a5f3c 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -103,8 +103,6 @@ void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery * s SyntaxAnalyzerResult SyntaxAnalyzer::analyze( const ASTPtr & query, - const Context & context, - const StoragePtr & storage, NamesAndTypesList source_columns, const Names & required_result_columns, size_t subquery_depth) const @@ -140,7 +138,7 @@ SyntaxAnalyzerResult SyntaxAnalyzer::analyze( } /// Common subexpression elimination. Rewrite rules. - normalizeTree(result, source_columns_list, source_columns_set, storage, + normalizeTree(result, source_columns_list, source_columns_set, result.storage, context, select_query, settings.asterisk_left_columns_only != 0); /// Remove unneeded columns according to 'required_result_columns'. diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.h b/dbms/src/Interpreters/SyntaxAnalyzer.h index c7bc921265f..fed4655c36d 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.h +++ b/dbms/src/Interpreters/SyntaxAnalyzer.h @@ -37,15 +37,30 @@ struct SyntaxAnalyzerResult bool rewrite_subqueries = false; }; +/// AST syntax analysis. +/// Optimises AST tree and collect information for further expression analysis. +/// Result AST has the following invariants: +/// * all aliases are substituted +/// * qualified names are translated +/// * scalar subqueries are executed replaced with constants +/// * unneeded columns are removed from SELECT clause +/// * duplicated columns are removed from ORDER BY, LIMIT BY, USING(...). +/// Motivation: +/// * group most of the AST-changing operations in single place +/// * avoid AST rewriting in ExpressionAnalyzer +/// * decompose ExpressionAnalyzer class SyntaxAnalyzer { public: + SyntaxAnalyzer(const Context & context, const StoragePtr & storage) : context(context), storage(storage) {} + SyntaxAnalyzerResult analyze(const ASTPtr & query, - const Context & context, - const StoragePtr & storage, NamesAndTypesList source_columns, const Names & required_result_columns = {}, size_t subquery_depth = 0) const; + + const Context & context; + StoragePtr storage; }; } From aca55a651a05f64cdb1e4ce4cd1063534a20e9eb Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Thu, 8 Nov 2018 14:52:08 +0300 Subject: [PATCH 07/46] Add ru changelog for v18.14.13 --- CHANGELOG_RU.md | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/CHANGELOG_RU.md b/CHANGELOG_RU.md index 18d23e3a6ce..077701230b8 100644 --- a/CHANGELOG_RU.md +++ b/CHANGELOG_RU.md @@ -1,3 +1,17 @@ +## ClickHouse release 18.14.13, 2018-11-08 + +### Исправления ошибок: +* Исправлена ошибка `Block structure mismatch in MergingSorted stream`. [#3162](https://github.com/yandex/ClickHouse/issues/3162) +* Исправлена работа запросов `ON CLUSTER` в случае, когда в конфигурации кластера включено шифрование (флаг ``). [#3465](https://github.com/yandex/ClickHouse/pull/3465) +* Исправлена ошибка при использовании `SAMPLE`, `PREWHERE` и столбцов-алиасов. [#3543](https://github.com/yandex/ClickHouse/pull/3543) +* Исправлена редкая ошибка `unknown compression method` при использовании настройки `min_bytes_to_use_direct_io`. [3544](https://github.com/yandex/ClickHouse/pull/3544) + +### Улучшения производительности +* Исправлена деградация производительности запросов с `GROUP BY` столбцов типа Int16, Date на процессорах AMD EPYC. [Игорь Лапко](https://github.com/yandex/ClickHouse/pull/3512) + +### Улучшения процесса сборки ClickHouse: +* Доработки для упрощения сборки в Arcadia. [#3475](https://github.com/yandex/ClickHouse/pull/3475), [#3535](https://github.com/yandex/ClickHouse/pull/3535) + ## ClickHouse release 18.14.12, 2018-11-02 ### Исправления ошибок: From 6aec41a2bddeeb2fcaf528e7059e3d3f94ee0f6e Mon Sep 17 00:00:00 2001 From: Alex Zatelepin Date: Thu, 8 Nov 2018 15:04:52 +0300 Subject: [PATCH 08/46] Update CHANGELOG_RU.md --- CHANGELOG_RU.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG_RU.md b/CHANGELOG_RU.md index 077701230b8..a6b9ca92989 100644 --- a/CHANGELOG_RU.md +++ b/CHANGELOG_RU.md @@ -6,8 +6,9 @@ * Исправлена ошибка при использовании `SAMPLE`, `PREWHERE` и столбцов-алиасов. [#3543](https://github.com/yandex/ClickHouse/pull/3543) * Исправлена редкая ошибка `unknown compression method` при использовании настройки `min_bytes_to_use_direct_io`. [3544](https://github.com/yandex/ClickHouse/pull/3544) -### Улучшения производительности +### Улучшения производительности: * Исправлена деградация производительности запросов с `GROUP BY` столбцов типа Int16, Date на процессорах AMD EPYC. [Игорь Лапко](https://github.com/yandex/ClickHouse/pull/3512) +* Исправлена деградация производительности при обработке длинных строк. [#3530](https://github.com/yandex/ClickHouse/pull/3530) ### Улучшения процесса сборки ClickHouse: * Доработки для упрощения сборки в Arcadia. [#3475](https://github.com/yandex/ClickHouse/pull/3475), [#3535](https://github.com/yandex/ClickHouse/pull/3535) From baef5abc12c569dcf35506bbccbf07a9c2cdb22e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 8 Nov 2018 15:06:35 +0300 Subject: [PATCH 09/46] Review fixes. --- dbms/src/Interpreters/tests/expression_analyzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/tests/expression_analyzer.cpp b/dbms/src/Interpreters/tests/expression_analyzer.cpp index eaa7731dd69..b737a7af7a4 100644 --- a/dbms/src/Interpreters/tests/expression_analyzer.cpp +++ b/dbms/src/Interpreters/tests/expression_analyzer.cpp @@ -35,7 +35,7 @@ struct TestEntry { ASTPtr ast = parse(query); - auto res = SyntaxAnalyzer().analyze(ast, context, {}, source_columns, required_result_columns); + auto res = SyntaxAnalyzer(context, {}).analyze(ast, source_columns, required_result_columns); if (!checkAliases(res)) { From 726ae0cb9a5f0542b64d33d14e538fb23e44964c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 8 Nov 2018 18:43:14 +0300 Subject: [PATCH 10/46] Split ExpressionAnalyzer and SyntaxAnalyzer. --- .../DataStreams/tests/expression_stream.cpp | 5 +- dbms/src/DataStreams/tests/filter_stream.cpp | 5 +- dbms/src/Interpreters/AnalyzedJoin.cpp | 5 +- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 41 +++++----- dbms/src/Interpreters/ExpressionAnalyzer.h | 16 ++-- .../Interpreters/InterpreterCreateQuery.cpp | 5 +- .../Interpreters/InterpreterSelectQuery.cpp | 22 ++++-- .../src/Interpreters/InterpreterSelectQuery.h | 4 + .../src/Interpreters/MutationsInterpreter.cpp | 10 ++- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 75 +++++++++---------- dbms/src/Interpreters/SyntaxAnalyzer.h | 19 ++--- .../evaluateConstantExpression.cpp | 8 +- .../Interpreters/evaluateMissingDefaults.cpp | 4 +- dbms/src/Interpreters/tests/expression.cpp | 4 +- .../tests/expression_analyzer.cpp | 2 +- dbms/src/Storages/AlterCommands.cpp | 10 ++- dbms/src/Storages/MergeTree/KeyCondition.cpp | 9 ++- dbms/src/Storages/MergeTree/KeyCondition.h | 3 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 27 +++++-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 10 ++- .../MergeTree/MergeTreeWhereOptimizer.cpp | 3 +- dbms/src/Storages/SelectQueryInfo.h | 4 + dbms/src/Storages/StorageDistributed.cpp | 11 ++- dbms/src/Storages/StorageMerge.cpp | 4 +- dbms/src/Storages/VirtualColumnUtils.cpp | 5 +- .../transformQueryForExternalDatabase.cpp | 10 ++- 26 files changed, 206 insertions(+), 115 deletions(-) diff --git a/dbms/src/DataStreams/tests/expression_stream.cpp b/dbms/src/DataStreams/tests/expression_stream.cpp index 91028638e08..75f999c7344 100644 --- a/dbms/src/DataStreams/tests/expression_stream.cpp +++ b/dbms/src/DataStreams/tests/expression_stream.cpp @@ -16,6 +16,7 @@ #include #include +#include #include #include #include @@ -35,7 +36,9 @@ try Context context = Context::createGlobal(); - ExpressionAnalyzer analyzer(ast, context, {}, {NameAndTypePair("number", std::make_shared())}); + NamesAndTypesList source_columns = {{"number", std::make_shared()}}; + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(ast, source_columns); + ExpressionAnalyzer analyzer(ast, syntax_result, context, source_columns); ExpressionActionsChain chain(context); analyzer.appendSelect(chain, false); analyzer.appendProjectResult(chain); diff --git a/dbms/src/DataStreams/tests/filter_stream.cpp b/dbms/src/DataStreams/tests/filter_stream.cpp index 015c976c00b..3e13a9b7966 100644 --- a/dbms/src/DataStreams/tests/filter_stream.cpp +++ b/dbms/src/DataStreams/tests/filter_stream.cpp @@ -18,6 +18,7 @@ #include #include +#include #include #include #include @@ -40,7 +41,9 @@ try Context context = Context::createGlobal(); - ExpressionAnalyzer analyzer(ast, context, {}, {NameAndTypePair("number", std::make_shared())}); + NamesAndTypesList source_columns = {{"number", std::make_shared()}}; + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(ast, source_columns); + ExpressionAnalyzer analyzer(ast, syntax_result, context, source_columns); ExpressionActionsChain chain(context); analyzer.appendSelect(chain, false); analyzer.appendProjectResult(chain); diff --git a/dbms/src/Interpreters/AnalyzedJoin.cpp b/dbms/src/Interpreters/AnalyzedJoin.cpp index 5d1a031e560..7642e3d6503 100644 --- a/dbms/src/Interpreters/AnalyzedJoin.cpp +++ b/dbms/src/Interpreters/AnalyzedJoin.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -46,7 +47,9 @@ ExpressionActionsPtr AnalyzedJoin::createJoinedBlockActions( for (auto & column : columns_from_joined_table) source_column_names.emplace_back(column.name_and_type); - ExpressionAnalyzer analyzer(expression_list, context, nullptr, source_column_names, required_columns); + ASTPtr query = expression_list; + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, source_column_names, required_columns); + ExpressionAnalyzer analyzer(query, syntax_result, context, source_column_names, required_columns); auto joined_block_actions = analyzer.getActions(false); auto required_action_columns = joined_block_actions->getRequiredColumns(); diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 23efe225c57..4055e73cea6 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -82,33 +82,30 @@ namespace ErrorCodes extern const int EXPECTED_ALL_OR_ANY; } +/// From SyntaxAnalyzer.cpp +extern void removeDuplicateColumns(NamesAndTypesList & columns); ExpressionAnalyzer::ExpressionAnalyzer( const ASTPtr & query_, + const SyntaxAnalyzerResultPtr & syntax_analyzer_result_, const Context & context_, - const StoragePtr & storage_, const NamesAndTypesList & source_columns_, const Names & required_result_columns_, size_t subquery_depth_, bool do_global_, const SubqueriesForSets & subqueries_for_sets_) - : ExpressionAnalyzerData(source_columns_, required_result_columns_, subqueries_for_sets_), - query(query_), context(context_), settings(context.getSettings()), storage(storage_), - subquery_depth(subquery_depth_), do_global(do_global_) + : ExpressionAnalyzerData(source_columns_, required_result_columns_, subqueries_for_sets_) + , query(query_), context(context_), settings(context.getSettings()) + , subquery_depth(subquery_depth_), do_global(do_global_) + , syntax(syntax_analyzer_result_), analyzed_join(syntax->analyzed_join) { - auto syntax_analyzer_result = SyntaxAnalyzer(context, storage) - .analyze(query, source_columns, required_result_columns_, subquery_depth); - query = syntax_analyzer_result.query; - storage = syntax_analyzer_result.storage; - source_columns = syntax_analyzer_result.source_columns; - array_join_result_to_source = syntax_analyzer_result.array_join_result_to_source; - array_join_alias_to_name = syntax_analyzer_result.array_join_alias_to_name; - array_join_name_to_alias = syntax_analyzer_result.array_join_name_to_alias; - analyzed_join = syntax_analyzer_result.analyzed_join; - rewrite_subqueries = syntax_analyzer_result.rewrite_subqueries; + storage = syntax->storage; + rewrite_subqueries = syntax->rewrite_subqueries; select_query = typeid_cast(query.get()); + removeDuplicateColumns(source_columns); + /// Delete the unnecessary from `source_columns` list. Create `unknown_required_source_columns`. Form `columns_added_by_join`. collectUsedColumns(); @@ -521,7 +518,7 @@ void ExpressionAnalyzer::initChain(ExpressionActionsChain & chain, const NamesAn void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActionsPtr & actions) const { NameSet result_columns; - for (const auto & result_source : array_join_result_to_source) + for (const auto & result_source : syntax->array_join_result_to_source) { /// Assign new names to columns, if needed. if (result_source.first != result_source.second) @@ -696,10 +693,18 @@ bool ExpressionAnalyzer::appendPrewhere(ExpressionActionsChain & chain, bool onl Names additional_required_mergetree_columns; if (sampling_expression) - additional_required_mergetree_columns = ExpressionAnalyzer(sampling_expression, context, storage).getRequiredSourceColumns(); + { + NamesAndTypesList columns; + auto ast = sampling_expression; + auto syntax_result = SyntaxAnalyzer(context, storage).analyze(ast, columns); + additional_required_mergetree_columns = ExpressionAnalyzer(ast, syntax_result, context, columns).getRequiredSourceColumns(); + } if (primary_expression) { - auto required_primary_columns = ExpressionAnalyzer(primary_expression, context, storage).getRequiredSourceColumns(); + NamesAndTypesList columns; + auto ast = primary_expression; + auto syntax_result = SyntaxAnalyzer(context, storage).analyze(ast, columns); + auto required_primary_columns = ExpressionAnalyzer(ast, syntax_result, context, columns).getRequiredSourceColumns(); additional_required_mergetree_columns.insert(additional_required_mergetree_columns.end(), required_primary_columns.begin(), required_primary_columns.end()); } @@ -1089,7 +1094,7 @@ void ExpressionAnalyzer::collectUsedColumns() /// Insert the columns required for the ARRAY JOIN calculation into the required columns list. NameSet array_join_sources; - for (const auto & result_source : array_join_result_to_source) + for (const auto & result_source : syntax->array_join_result_to_source) array_join_sources.insert(result_source.second); for (const auto & column_name_type : source_columns) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index caa9c5008f8..3dd21b2f82f 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -31,6 +31,8 @@ class ASTFunction; class ASTExpressionList; class ASTSelectQuery; +struct SyntaxAnalyzerResult; +using SyntaxAnalyzerResultPtr = std::shared_ptr; /// ExpressionAnalyzers sources, intermediates and results. It splits data and logic, allows to test them separately. /// If you are not writing a test you probably don't need it. Use ExpressionAnalyzer itself. @@ -60,13 +62,6 @@ struct ExpressionAnalyzerData /// For example, for `SELECT s.v ... ARRAY JOIN a AS s` will get "s.v" -> "a.v". NameToNameMap array_join_result_to_source; - /// For the ARRAY JOIN section, mapping from the alias to the full column name. - /// For example, for `ARRAY JOIN [1,2] AS b` "b" -> "array(1,2)" will enter here. - NameToNameMap array_join_alias_to_name; - - /// The backward mapping for array_join_alias_to_name. - NameToNameMap array_join_name_to_alias; - /// All new temporary tables obtained by performing the GLOBAL IN/JOIN subqueries. Tables external_tables; @@ -144,9 +139,9 @@ private: public: ExpressionAnalyzer( const ASTPtr & query_, + const SyntaxAnalyzerResultPtr & syntax_analyzer_result_, const Context & context_, - const StoragePtr & storage_, - const NamesAndTypesList & source_columns_ = {}, + const NamesAndTypesList & source_columns_, const Names & required_result_columns_ = {}, size_t subquery_depth_ = 0, bool do_global_ = false, @@ -240,7 +235,8 @@ private: size_t subquery_depth; bool do_global; /// Do I need to prepare for execution global subqueries when analyzing the query. - AnalyzedJoin analyzed_join; + SyntaxAnalyzerResultPtr syntax; + const AnalyzedJoin & analyzed_join; /** Remove all unnecessary columns from the list of all available columns of the table (`columns`). * At the same time, form a set of unknown columns (`unknown_required_source_columns`), diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 6dfb99bd4ac..923093f5b12 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include #include @@ -223,7 +224,9 @@ static ColumnsAndDefaults parseColumns(const ASTExpressionList & column_list_ast /// set missing types and wrap default_expression's in a conversion-function if necessary if (!defaulted_columns.empty()) { - const auto actions = ExpressionAnalyzer{default_expr_list, context, {}, columns}.getActions(true); + auto source_columns = columns; + auto syntax_analyzer_result = SyntaxAnalyzer(context, {}).analyze(default_expr_list, source_columns); + const auto actions = ExpressionAnalyzer(default_expr_list, syntax_analyzer_result, context, source_columns).getActions(true); const auto block = actions->getSampleBlock(); for (auto action : actions->getActions()) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 3a381e3f05f..998544c0e67 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -184,8 +184,11 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (storage) table_lock = storage->lockStructure(false, __PRETTY_FUNCTION__); + auto source_columns = source_header.getNamesAndTypesList(); + syntax_analyzer_result = SyntaxAnalyzer(context, storage) + .analyze(query_ptr, source_columns, required_result_column_names, subquery_depth); query_analyzer = std::make_unique( - query_ptr, context, storage, source_header.getNamesAndTypesList(), required_result_column_names, subquery_depth, !only_analyze); + query_ptr, syntax_analyzer_result, context, source_columns, required_result_column_names, subquery_depth, !only_analyze); if (!only_analyze) { @@ -460,6 +463,7 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt { SelectQueryInfo query_info; query_info.query = query_ptr; + query_info.syntax_analyzer_result = syntax_analyzer_result; query_info.sets = query_analyzer->getPreparedSets(); /// Try transferring some condition from WHERE to PREWHERE if enabled and viable @@ -752,7 +756,8 @@ void InterpreterSelectQuery::executeFetchColumns( } /// Columns which we will get after prewhere execution. - NamesAndTypesList additional_source_columns; + NamesAndTypesList alias_source_columns = source_columns; + NameSet additional_source_columns; /// Add columns which will be added by prewhere (otherwise we will remove them in project action). NameSet columns_to_remove(columns_to_remove_after_prewhere.begin(), columns_to_remove_after_prewhere.end()); for (const auto & column : prewhere_actions_result) @@ -764,11 +769,11 @@ void InterpreterSelectQuery::executeFetchColumns( continue; required_columns_expr_list->children.emplace_back(std::make_shared(column.name)); - additional_source_columns.emplace_back(column.name, column.type); + alias_source_columns.emplace_back(column.name, column.type); + additional_source_columns.insert(column.name); } - auto additional_source_columns_set = ext::map(additional_source_columns, [] (const auto & it) { return it.name; }); - alias_actions = ExpressionAnalyzer(required_columns_expr_list, context, storage, additional_source_columns).getActions(true); + alias_actions = ExpressionAnalyzer(required_columns_expr_list, syntax_analyzer_result, context, alias_source_columns).getActions(true); /// The set of required columns could be added as a result of adding an action to calculate ALIAS. required_columns = alias_actions->getRequiredColumns(); @@ -783,7 +788,7 @@ void InterpreterSelectQuery::executeFetchColumns( size_t next_req_column_pos = 0; for (size_t i = 0; i < required_columns.size(); ++i) { - if (!additional_source_columns_set.count(required_columns[i])) + if (!additional_source_columns.count(required_columns[i])) { if (next_req_column_pos < i) std::swap(required_columns[i], required_columns[next_req_column_pos]); @@ -804,7 +809,9 @@ void InterpreterSelectQuery::executeFetchColumns( } prewhere_info->prewhere_actions = std::move(new_actions); - prewhere_info->alias_actions = ExpressionAnalyzer(required_prewhere_columns_expr_list, context, storage).getActions(true, false); + prewhere_info->alias_actions = + ExpressionAnalyzer(required_prewhere_columns_expr_list, syntax_analyzer_result, context, source_columns) + .getActions(true, false); /// Add columns required by alias actions. auto required_aliased_columns = prewhere_info->alias_actions->getRequiredColumns(); @@ -909,6 +916,7 @@ void InterpreterSelectQuery::executeFetchColumns( SelectQueryInfo query_info; query_info.query = query_ptr; + query_info.syntax_analyzer_result = syntax_analyzer_result; query_info.sets = query_analyzer->getPreparedSets(); query_info.prewhere_info = prewhere_info; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index d593c49e4ae..7bbb0271f7e 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -20,6 +20,8 @@ class ASTSelectQuery; struct SubqueryForSet; class InterpreterSelectWithUnionQuery; +struct SyntaxAnalyzerResult; +using SyntaxAnalyzerResultPtr = std::shared_ptr; /** Interprets the SELECT query. Returns the stream of blocks with the results of the query before `to_stage` stage. */ @@ -218,6 +220,8 @@ private: Context context; QueryProcessingStage::Enum to_stage; size_t subquery_depth = 0; + NamesAndTypesList source_columns; + SyntaxAnalyzerResultPtr syntax_analyzer_result; std::unique_ptr query_analyzer; /// How many streams we ask for storage to produce, and in how many threads we will do further processing. diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index 16517532e38..e63d9137705 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -195,7 +196,10 @@ void MutationsInterpreter::prepare(bool dry_run) const ColumnDefault & col_default = kv.second; if (col_default.kind == ColumnDefaultKind::Materialized) { - ExpressionAnalyzer analyzer(col_default.expression->clone(), context, nullptr, all_columns); + NamesAndTypesList source_columns = all_columns; + auto query = col_default.expression->clone(); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, source_columns); + ExpressionAnalyzer analyzer(query, syntax_result, context, source_columns); for (const String & dependency : analyzer.getRequiredSourceColumns()) { if (updated_columns.count(dependency)) @@ -301,7 +305,9 @@ void MutationsInterpreter::prepare(bool dry_run) for (const String & column : stage.output_columns) all_asts->children.push_back(std::make_shared(column)); - stage.analyzer = std::make_unique(all_asts, context, nullptr, all_columns); + NamesAndTypesList source_columns = all_columns; + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(all_asts, source_columns); + stage.analyzer = std::make_unique(all_asts, syntax_result, context, all_columns); ExpressionActionsChain & actions_chain = stage.expressions_chain; diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 49e849a5f3c..49bb7e7c306 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -49,8 +49,8 @@ using LogAST = DebugASTLog; /// set to true to enable logs using Aliases = std::unordered_map; /// Add columns from storage to source_columns list. -NamesAndTypesList collectSourceColumns(NamesAndTypesList source_columns, ASTSelectQuery * select_query, - const Context & context, StoragePtr & storage); +void collectSourceColumns(NamesAndTypesList & source_columns, ASTSelectQuery * select_query, + const Context & context, StoragePtr & storage); /// Translate qualified names such as db.table.column, table.column, table_alias.column to unqualified names. void translateQualifiedNames(ASTPtr & query, ASTSelectQuery * select_query, @@ -58,20 +58,21 @@ void translateQualifiedNames(ASTPtr & query, ASTSelectQuery * select_query, /// For star nodes(`*`), expand them to a list of all columns. For literal nodes, substitute aliases. void normalizeTree( - SyntaxAnalyzerResult & result, - const Names & source_columns, - const NameSet & source_columns_set, - const StoragePtr & storage, - const Context & context, - const ASTSelectQuery * select_query, - bool asterisk_left_columns_only); + ASTPtr & query, + SyntaxAnalyzerResult & result, + const Names & source_columns, + const NameSet & source_columns_set, + const StoragePtr & storage, + const Context & context, + const ASTSelectQuery * select_query, + bool asterisk_left_columns_only); /// Sometimes we have to calculate more columns in SELECT clause than will be returned from query. /// This is the case when we have DISTINCT or arrayJoin: we require more columns in SELECT even if we need less columns in result. void removeUnneededColumnsFromSelectClause(const ASTSelectQuery * select_query, const Names & required_result_columns); /// Replacing scalar subqueries with constant values. -void executeScalarSubqueries(SyntaxAnalyzerResult & result, const ASTSelectQuery * select_query, +void executeScalarSubqueries(ASTPtr & query, const ASTSelectQuery * select_query, const Context & context, size_t subquery_depth); /// Remove Function_if AST if condition is constant. @@ -89,7 +90,7 @@ void optimizeLimitBy(const ASTSelectQuery * select_query); /// Remove duplicated columns from USING(...). void optimizeUsing(const ASTSelectQuery * select_query); -void getArrayJoinedColumns(SyntaxAnalyzerResult & result, const ASTSelectQuery * select_query, +void getArrayJoinedColumns(ASTPtr & query, SyntaxAnalyzerResult & result, const ASTSelectQuery * select_query, const Names & source_columns, const NameSet & source_columns_set); /// Parse JOIN ON expression and collect ASTs for joined columns. @@ -101,27 +102,26 @@ void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery * s const NameSet & source_columns, const Context & context); } -SyntaxAnalyzerResult SyntaxAnalyzer::analyze( - const ASTPtr & query, - NamesAndTypesList source_columns, +SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( + ASTPtr & query, + NamesAndTypesList & source_columns, const Names & required_result_columns, size_t subquery_depth) const { SyntaxAnalyzerResult result; result.storage = storage; - result.query = query; // ->clone(); - auto * select_query = typeid_cast(result.query.get()); - result.source_columns = collectSourceColumns(std::move(source_columns), select_query, context, result.storage); + auto * select_query = typeid_cast(query.get()); + collectSourceColumns(source_columns, select_query, context, result.storage); const auto & settings = context.getSettingsRef(); Names source_columns_list; - source_columns_list.reserve(result.source_columns.size()); - for (const auto & type_name : result.source_columns) + source_columns_list.reserve(source_columns.size()); + for (const auto & type_name : source_columns) source_columns_list.emplace_back(type_name.name); NameSet source_columns_set(source_columns_list.begin(), source_columns_list.end()); - translateQualifiedNames(result.query, select_query, source_columns_set, context); + translateQualifiedNames(query, select_query, source_columns_set, context); /// Depending on the user's profile, check for the execution rights /// distributed subqueries inside the IN or JOIN sections and process these subqueries. @@ -138,7 +138,7 @@ SyntaxAnalyzerResult SyntaxAnalyzer::analyze( } /// Common subexpression elimination. Rewrite rules. - normalizeTree(result, source_columns_list, source_columns_set, result.storage, + normalizeTree(query, result, source_columns_list, source_columns_set, result.storage, context, select_query, settings.asterisk_left_columns_only != 0); /// Remove unneeded columns according to 'required_result_columns'. @@ -148,10 +148,10 @@ SyntaxAnalyzerResult SyntaxAnalyzer::analyze( removeUnneededColumnsFromSelectClause(select_query, required_result_columns); /// Executing scalar subqueries - replacing them with constant values. - executeScalarSubqueries(result, select_query, context, subquery_depth); + executeScalarSubqueries(query, select_query, context, subquery_depth); /// Optimize if with constant condition after constants was substituted instead of sclalar subqueries. - optimizeIfWithConstantCondition(result.query, result.aliases); + optimizeIfWithConstantCondition(query, result.aliases); /// GROUP BY injective function elimination. optimizeGroupBy(select_query, source_columns_set, context); @@ -166,20 +166,16 @@ SyntaxAnalyzerResult SyntaxAnalyzer::analyze( optimizeUsing(select_query); /// array_join_alias_to_name, array_join_result_to_source. - getArrayJoinedColumns(result, select_query, source_columns_list, source_columns_set); + getArrayJoinedColumns(query, result, select_query, source_columns_list, source_columns_set); /// Push the predicate expression down to the subqueries. result.rewrite_subqueries = PredicateExpressionsOptimizer(select_query, settings, context).optimize(); collectJoinedColumns(result.analyzed_join, select_query, source_columns_set, context); - return result; + return std::make_shared(result); } - -namespace -{ - void removeDuplicateColumns(NamesAndTypesList & columns) { std::set names; @@ -192,8 +188,11 @@ void removeDuplicateColumns(NamesAndTypesList & columns) } } -NamesAndTypesList collectSourceColumns(NamesAndTypesList source_columns, ASTSelectQuery * select_query, - const Context & context, StoragePtr & storage) +namespace +{ + +void collectSourceColumns(NamesAndTypesList & source_columns, ASTSelectQuery * select_query, + const Context & context, StoragePtr & storage) { if (!storage && select_query) { @@ -217,7 +216,6 @@ NamesAndTypesList collectSourceColumns(NamesAndTypesList source_columns, ASTSele } removeDuplicateColumns(source_columns); - return source_columns; } void translateQualifiedNames(ASTPtr & query, ASTSelectQuery * select_query, @@ -234,6 +232,7 @@ void translateQualifiedNames(ASTPtr & query, ASTSelectQuery * select_query, } void normalizeTree( + ASTPtr & query, SyntaxAnalyzerResult & result, const Names & source_columns, const NameSet & source_columns_set, @@ -280,7 +279,7 @@ void normalizeTree( } auto & settings = context.getSettingsRef(); - QueryNormalizer(result.query, result.aliases, settings, all_columns_name, table_names_and_column_names).perform(); + QueryNormalizer(query, result.aliases, settings, all_columns_name, table_names_and_column_names).perform(); } bool hasArrayJoin(const ASTPtr & ast) @@ -334,7 +333,7 @@ void removeUnneededColumnsFromSelectClause(const ASTSelectQuery * select_query, elements = std::move(new_elements); } -void executeScalarSubqueries(SyntaxAnalyzerResult & result, const ASTSelectQuery * select_query, +void executeScalarSubqueries(ASTPtr & query, const ASTSelectQuery * select_query, const Context & context, size_t subquery_depth) { LogAST log; @@ -342,11 +341,11 @@ void executeScalarSubqueries(SyntaxAnalyzerResult & result, const ASTSelectQuery if (!select_query) { ExecuteScalarSubqueriesVisitor visitor(context, subquery_depth, log.stream()); - visitor.visit(result.query); + visitor.visit(query); } else { - for (auto & child : result.query->children) + for (auto & child : query->children) { /// Do not go to FROM, JOIN, UNION. if (!typeid_cast(child.get()) @@ -672,7 +671,7 @@ void optimizeUsing(const ASTSelectQuery * select_query) expression_list = uniq_expressions_list; } -void getArrayJoinedColumns(SyntaxAnalyzerResult & result, const ASTSelectQuery * select_query, +void getArrayJoinedColumns(ASTPtr & query, SyntaxAnalyzerResult & result, const ASTSelectQuery * select_query, const Names & source_columns, const NameSet & source_columns_set) { if (select_query && select_query->array_join_expression_list()) @@ -699,7 +698,7 @@ void getArrayJoinedColumns(SyntaxAnalyzerResult & result, const ASTSelectQuery * ArrayJoinedColumnsVisitor visitor(result.array_join_name_to_alias, result.array_join_alias_to_name, result.array_join_result_to_source); - visitor.visit(result.query); + visitor.visit(query); } /// If the result of ARRAY JOIN is not used, it is necessary to ARRAY-JOIN any column, diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.h b/dbms/src/Interpreters/SyntaxAnalyzer.h index fed4655c36d..c03b10d3cd2 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.h +++ b/dbms/src/Interpreters/SyntaxAnalyzer.h @@ -10,12 +10,8 @@ using StoragePtr = std::shared_ptr; struct SyntaxAnalyzerResult { - ASTPtr query; - StoragePtr storage; - NamesAndTypesList source_columns; - /// Note: used only in tests. using Aliases = std::unordered_map; Aliases aliases; @@ -26,9 +22,11 @@ struct SyntaxAnalyzerResult /// For the ARRAY JOIN section, mapping from the alias to the full column name. /// For example, for `ARRAY JOIN [1,2] AS b` "b" -> "array(1,2)" will enter here. + /// Note: not used further. NameToNameMap array_join_alias_to_name; /// The backward mapping for array_join_alias_to_name. + /// Note: not used further. NameToNameMap array_join_name_to_alias; AnalyzedJoin analyzed_join; @@ -37,6 +35,8 @@ struct SyntaxAnalyzerResult bool rewrite_subqueries = false; }; +using SyntaxAnalyzerResultPtr = std::shared_ptr; + /// AST syntax analysis. /// Optimises AST tree and collect information for further expression analysis. /// Result AST has the following invariants: @@ -52,12 +52,13 @@ struct SyntaxAnalyzerResult class SyntaxAnalyzer { public: - SyntaxAnalyzer(const Context & context, const StoragePtr & storage) : context(context), storage(storage) {} + SyntaxAnalyzer(const Context & context, StoragePtr storage) : context(context), storage(std::move(storage)) {} - SyntaxAnalyzerResult analyze(const ASTPtr & query, - NamesAndTypesList source_columns, - const Names & required_result_columns = {}, - size_t subquery_depth = 0) const; + SyntaxAnalyzerResultPtr analyze( + ASTPtr & query, + NamesAndTypesList & source_columns, + const Names & required_result_columns = {}, + size_t subquery_depth = 0) const; const Context & context; StoragePtr storage; diff --git a/dbms/src/Interpreters/evaluateConstantExpression.cpp b/dbms/src/Interpreters/evaluateConstantExpression.cpp index 481ad8ee0b2..be94f04ff3c 100644 --- a/dbms/src/Interpreters/evaluateConstantExpression.cpp +++ b/dbms/src/Interpreters/evaluateConstantExpression.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -26,8 +27,11 @@ namespace ErrorCodes std::pair> evaluateConstantExpression(const ASTPtr & node, const Context & context) { - ExpressionActionsPtr expr_for_constant_folding = ExpressionAnalyzer( - node, context, nullptr, NamesAndTypesList{{ "_dummy", std::make_shared() }}, Names()).getConstActions(); + NamesAndTypesList source_columns = {{ "_dummy", std::make_shared() }}; + auto query = node->clone(); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, source_columns, {}); + ExpressionActionsPtr expr_for_constant_folding + = ExpressionAnalyzer(node, syntax_result, context, source_columns).getConstActions(); /// There must be at least one column in the block so that it knows the number of rows. Block block_with_constants{{ ColumnConst::create(ColumnUInt8::create(1, 0), 1), std::make_shared(), "_dummy" }}; diff --git a/dbms/src/Interpreters/evaluateMissingDefaults.cpp b/dbms/src/Interpreters/evaluateMissingDefaults.cpp index c9758ab0cd0..34bd252f95d 100644 --- a/dbms/src/Interpreters/evaluateMissingDefaults.cpp +++ b/dbms/src/Interpreters/evaluateMissingDefaults.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -47,7 +48,8 @@ void evaluateMissingDefaults(Block & block, for (size_t i = 0, size = block.columns(); i < size; ++i) available_columns.emplace_back(block.getByPosition(i).name, block.getByPosition(i).type); - ExpressionAnalyzer{default_expr_list, context, {}, available_columns}.getActions(true)->execute(copy_block); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(default_expr_list, available_columns); + ExpressionAnalyzer{default_expr_list, syntax_result, context, available_columns}.getActions(true)->execute(copy_block); /// move evaluated columns to the original block, materializing them at the same time size_t pos = 0; diff --git a/dbms/src/Interpreters/tests/expression.cpp b/dbms/src/Interpreters/tests/expression.cpp index aea1ccce48c..7b5203ec01f 100644 --- a/dbms/src/Interpreters/tests/expression.cpp +++ b/dbms/src/Interpreters/tests/expression.cpp @@ -19,6 +19,7 @@ #include #include +#include #include #include #include @@ -53,7 +54,8 @@ int main(int argc, char ** argv) {"s2", std::make_shared()} }; - ExpressionAnalyzer analyzer(ast, context, {}, columns); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(ast, columns); + ExpressionAnalyzer analyzer(ast, syntax_result, context, columns); ExpressionActionsChain chain(context); analyzer.appendSelect(chain, false); analyzer.appendProjectResult(chain); diff --git a/dbms/src/Interpreters/tests/expression_analyzer.cpp b/dbms/src/Interpreters/tests/expression_analyzer.cpp index b737a7af7a4..3079e1b8c21 100644 --- a/dbms/src/Interpreters/tests/expression_analyzer.cpp +++ b/dbms/src/Interpreters/tests/expression_analyzer.cpp @@ -37,7 +37,7 @@ struct TestEntry auto res = SyntaxAnalyzer(context, {}).analyze(ast, source_columns, required_result_columns); - if (!checkAliases(res)) + if (!checkAliases(*res)) { collectWithAnalysers(context, ast); return false; diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index 08e24d98c0e..e117c066af9 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -320,7 +321,10 @@ void AlterCommands::validate(const IStorage & table, const Context & context) for (const auto & default_column : defaults) { const auto & default_expression = default_column.second.expression; - const auto actions = ExpressionAnalyzer{default_expression, context, {}, all_columns}.getActions(true); + NamesAndTypesList source_columns = all_columns; + ASTPtr query = default_expression; + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, source_columns); + const auto actions = ExpressionAnalyzer(query, syntax_result, context, all_columns).getActions(true); const auto required_columns = actions->getRequiredColumns(); if (required_columns.end() != std::find(required_columns.begin(), required_columns.end(), command.column_name)) @@ -376,7 +380,9 @@ void AlterCommands::validate(const IStorage & table, const Context & context) defaulted_columns.emplace_back(NameAndTypePair{column_name, column_type_ptr}, nullptr); } - const auto actions = ExpressionAnalyzer{default_expr_list, context, {}, all_columns}.getActions(true); + ASTPtr query = default_expr_list; + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, all_columns); + const auto actions = ExpressionAnalyzer(query, syntax_result, context, all_columns).getActions(true); const auto block = actions->getSampleBlock(); /// set deduced types, modify default expression if necessary diff --git a/dbms/src/Storages/MergeTree/KeyCondition.cpp b/dbms/src/Storages/MergeTree/KeyCondition.cpp index 745e1112a1f..4978e23183d 100644 --- a/dbms/src/Storages/MergeTree/KeyCondition.cpp +++ b/dbms/src/Storages/MergeTree/KeyCondition.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -247,14 +248,15 @@ bool FieldWithInfinity::operator==(const FieldWithInfinity & other) const * For index to work when something like "WHERE Date = toDate(now())" is written. */ Block KeyCondition::getBlockWithConstants( - const ASTPtr & query, const Context & context, const NamesAndTypesList & all_columns) + const ASTPtr & query, const SyntaxAnalyzerResultPtr & syntax_analyzer_result, + const Context & context, const NamesAndTypesList & all_columns) { Block result { { DataTypeUInt8().createColumnConstWithDefaultValue(1), std::make_shared(), "_dummy" } }; - const auto expr_for_constant_folding = ExpressionAnalyzer{query, context, nullptr, all_columns}.getConstActions(); + const auto expr_for_constant_folding = ExpressionAnalyzer(query, syntax_analyzer_result, context, all_columns).getConstActions(); expr_for_constant_folding->execute(result); @@ -280,7 +282,8 @@ KeyCondition::KeyCondition( /** Evaluation of expressions that depend only on constants. * For the index to be used, if it is written, for example `WHERE Date = toDate(now())`. */ - Block block_with_constants = getBlockWithConstants(query_info.query, context, all_columns); + Block block_with_constants = getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, + context, all_columns); /// Trasform WHERE section to Reverse Polish notation const ASTSelectQuery & select = typeid_cast(*query_info.query); diff --git a/dbms/src/Storages/MergeTree/KeyCondition.h b/dbms/src/Storages/MergeTree/KeyCondition.h index cb06909eb87..d6db68ef738 100644 --- a/dbms/src/Storages/MergeTree/KeyCondition.h +++ b/dbms/src/Storages/MergeTree/KeyCondition.h @@ -269,7 +269,8 @@ public: static Block getBlockWithConstants( - const ASTPtr & query, const Context & context, const NamesAndTypesList & all_columns); + const ASTPtr & query, const SyntaxAnalyzerResultPtr & syntax_analyzer_result, + const Context & context, const NamesAndTypesList & all_columns); static std::optional applyMonotonicFunctionsChainToRange( Range key_range, diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 1c959f41759..c80464e93ac 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -226,11 +227,13 @@ void MergeTreeData::initPrimaryKey() primary_sort_columns.clear(); addSortColumns(primary_sort_columns, primary_expr_ast); - primary_expr = ExpressionAnalyzer(primary_expr_ast, context, nullptr, getColumns().getAllPhysical()).getActions(false); - { + auto source_columns = getColumns().getAllPhysical(); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(primary_expr_ast, source_columns); + primary_expr = ExpressionAnalyzer(primary_expr_ast, syntax_result, context, source_columns).getActions(false); + ExpressionActionsPtr projected_expr = - ExpressionAnalyzer(primary_expr_ast, context, nullptr, getColumns().getAllPhysical()).getActions(true); + ExpressionAnalyzer(primary_expr_ast, syntax_result, context, source_columns).getActions(true); primary_key_sample = projected_expr->getSampleBlock(); } @@ -245,10 +248,12 @@ void MergeTreeData::initPrimaryKey() if (secondary_sort_expr_ast) { addSortColumns(sort_columns, secondary_sort_expr_ast); - secondary_sort_expr = ExpressionAnalyzer(secondary_sort_expr_ast, context, nullptr, getColumns().getAllPhysical()).getActions(false); + NamesAndTypesList source_columns = getColumns().getAllPhysical(); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(secondary_sort_expr_ast, source_columns); + secondary_sort_expr = ExpressionAnalyzer(secondary_sort_expr_ast, syntax_result, context, source_columns).getActions(false); ExpressionActionsPtr projected_expr = - ExpressionAnalyzer(secondary_sort_expr_ast, context, nullptr, getColumns().getAllPhysical()).getActions(true); + ExpressionAnalyzer(secondary_sort_expr_ast, syntax_result, context, source_columns).getActions(true); auto secondary_key_sample = projected_expr->getSampleBlock(); checkKeyExpression(*secondary_sort_expr, secondary_key_sample, "Secondary"); @@ -261,7 +266,12 @@ void MergeTreeData::initPartitionKey() if (!partition_expr_ast || partition_expr_ast->children.empty()) return; - partition_expr = ExpressionAnalyzer(partition_expr_ast, context, nullptr, getColumns().getAllPhysical()).getActions(false); + { + NamesAndTypesList source_columns = getColumns().getAllPhysical(); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(partition_expr_ast, source_columns); + partition_expr = ExpressionAnalyzer(partition_expr_ast, syntax_result, context, source_columns).getActions(false); + } + for (const ASTPtr & ast : partition_expr_ast->children) { String col_name = ast->getColumnName(); @@ -1147,7 +1157,10 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart( /// TODO: Check the order of secondary sorting key columns. if (new_primary_key.get() != primary_expr_ast.get()) { - ExpressionActionsPtr new_primary_expr = ExpressionAnalyzer(new_primary_key, context, nullptr, new_columns).getActions(true); + NamesAndTypesList source_columns = new_columns; + ASTPtr query = new_primary_key; + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, source_columns); + ExpressionActionsPtr new_primary_expr = ExpressionAnalyzer(query, syntax_result, context, source_columns).getActions(true); Block new_primary_key_sample = new_primary_expr->getSampleBlock(); size_t new_key_size = new_primary_key_sample.columns(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 4d2bb671af1..ecc6b6ecf33 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -495,7 +495,10 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( filter_function->children.push_back(filter_function->arguments); } - filter_expression = ExpressionAnalyzer(filter_function, context, nullptr, available_real_columns).getActions(false); + NamesAndTypesList source_columns = available_real_columns; + ASTPtr query = filter_function; + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, source_columns); + filter_expression = ExpressionAnalyzer(filter_function, syntax_result, context, source_columns).getActions(false); /// Add columns needed for `sampling_expression` to `column_names_to_read`. std::vector add_columns = filter_expression->getRequiredColumns(); @@ -853,7 +856,10 @@ void MergeTreeDataSelectExecutor::createPositiveSignCondition( arguments->children.push_back(sign); arguments->children.push_back(one); - out_expression = ExpressionAnalyzer(function, context, {}, data.getColumns().getAllPhysical()).getActions(false); + ASTPtr query = function; + NamesAndTypesList source_columns = data.getColumns().getAllPhysical(); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, source_columns); + out_expression = ExpressionAnalyzer(query, syntax_result, context, source_columns).getActions(false); out_column = function->getColumnName(); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 998084381cb..14efe1484b2 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -43,7 +43,8 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( : primary_key_columns{ext::collection_cast(data.getPrimarySortColumns())}, table_columns{ext::map(data.getColumns().getAllPhysical(), [] (const NameAndTypePair & col) { return col.name; })}, - block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, context, data.getColumns().getAllPhysical())}, + block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, + context, data.getColumns().getAllPhysical())}, prepared_sets(query_info.sets), log{log} { diff --git a/dbms/src/Storages/SelectQueryInfo.h b/dbms/src/Storages/SelectQueryInfo.h index 107a45b18c2..1aed0a98f8f 100644 --- a/dbms/src/Storages/SelectQueryInfo.h +++ b/dbms/src/Storages/SelectQueryInfo.h @@ -37,6 +37,8 @@ struct PrewhereInfo using PrewhereInfoPtr = std::shared_ptr; +struct SyntaxAnalyzerResult; +using SyntaxAnalyzerResultPtr = std::shared_ptr; /** Query along with some additional data, * that can be used during query processing @@ -46,6 +48,8 @@ struct SelectQueryInfo { ASTPtr query; + SyntaxAnalyzerResultPtr syntax_analyzer_result; + PrewhereInfoPtr prewhere_info; /// Prepared sets are used for indices by storage engine. diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 8144639d998..5bdd2cc7417 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -27,6 +27,7 @@ #include #include #include +#include #include #include #include @@ -140,6 +141,12 @@ void initializeFileNamesIncrement(const std::string & path, SimpleIncrement & in /// For destruction of std::unique_ptr of type that is incomplete in class definition. StorageDistributed::~StorageDistributed() = default; +static ExpressionActionsPtr buildShardingKeyExpression(const ASTPtr & sharding_key, const Context & context, NamesAndTypesList columns, bool project) +{ + ASTPtr query = sharding_key; + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, columns); + return ExpressionAnalyzer(query, syntax_result, context, columns).getActions(project); +} StorageDistributed::StorageDistributed( const String & database_name, @@ -156,7 +163,7 @@ StorageDistributed::StorageDistributed( table_name(table_name_), remote_database(remote_database_), remote_table(remote_table_), context(context_), cluster_name(context.getMacros()->expand(cluster_name_)), has_sharding_key(sharding_key_), - sharding_key_expr(sharding_key_ ? ExpressionAnalyzer(sharding_key_, context, nullptr, getColumns().getAllPhysical()).getActions(false) : nullptr), + sharding_key_expr(sharding_key_ ? buildShardingKeyExpression(sharding_key_, context, getColumns().getAllPhysical(), false) : nullptr), sharding_key_column_name(sharding_key_ ? sharding_key_->getColumnName() : String{}), path(data_path_.empty() ? "" : (data_path_ + escapeForFileName(table_name) + '/')) { @@ -461,7 +468,7 @@ void registerStorageDistributed(StorageFactory & factory) /// Check that sharding_key exists in the table and has numeric type. if (sharding_key) { - auto sharding_expr = ExpressionAnalyzer(sharding_key, args.context, nullptr, args.columns.getAllPhysical()).getActions(true); + auto sharding_expr = buildShardingKeyExpression(sharding_key, args.context, args.columns.getAllPhysical(), true); const Block & block = sharding_expr->getSampleBlock(); if (block.columns() != 1) diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 0bb5c832cfe..3730ed92849 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -458,7 +459,8 @@ void StorageMerge::convertingSourceStream(const Block & header, const Context & NamesAndTypesList source_columns = getSampleBlock().getNamesAndTypesList(); NameAndTypePair virtual_column = getColumn("_table"); source_columns.insert(source_columns.end(), virtual_column); - ExpressionActionsPtr actions = ExpressionAnalyzer{where_expression, context, {}, source_columns}.getActions(false, false); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(where_expression, source_columns); + ExpressionActionsPtr actions = ExpressionAnalyzer{where_expression, syntax_result, context, source_columns}.getActions(false, false); Names required_columns = actions->getRequiredColumns(); for (const auto required_column : required_columns) diff --git a/dbms/src/Storages/VirtualColumnUtils.cpp b/dbms/src/Storages/VirtualColumnUtils.cpp index 71e26c323a4..5e32ca14fa2 100644 --- a/dbms/src/Storages/VirtualColumnUtils.cpp +++ b/dbms/src/Storages/VirtualColumnUtils.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include @@ -156,7 +157,9 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & c return; /// Let's analyze and calculate the expression. - ExpressionAnalyzer analyzer(expression_ast, context, {}, block.getNamesAndTypesList()); + NamesAndTypesList source_columns = block.getNamesAndTypesList(); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(expression_ast, source_columns); + ExpressionAnalyzer analyzer(expression_ast, syntax_result, context, source_columns); ExpressionActionsPtr actions = analyzer.getActions(false); Block block_with_filter = block; diff --git a/dbms/src/Storages/transformQueryForExternalDatabase.cpp b/dbms/src/Storages/transformQueryForExternalDatabase.cpp index d143cb32ff8..dabd6268aa7 100644 --- a/dbms/src/Storages/transformQueryForExternalDatabase.cpp +++ b/dbms/src/Storages/transformQueryForExternalDatabase.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -25,7 +26,10 @@ static void replaceConstFunction(IAST & node, const Context & context, const Nam if (ASTFunction * function = typeid_cast(&*child)) { - auto result_block = KeyCondition::getBlockWithConstants(function->ptr(), context, all_columns); + NamesAndTypesList source_columns = all_columns; + ASTPtr query = function->ptr(); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, source_columns); + auto result_block = KeyCondition::getBlockWithConstants(query, syntax_result, context, all_columns); if (!result_block.has(child->getColumnName())) return; @@ -88,7 +92,9 @@ String transformQueryForExternalDatabase( const Context & context) { auto clone_query = query.clone(); - ExpressionAnalyzer analyzer(clone_query, context, {}, available_columns); + NamesAndTypesList source_columns = available_columns; + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(clone_query, source_columns); + ExpressionAnalyzer analyzer(clone_query, syntax_result, context, available_columns); const Names & used_columns = analyzer.getRequiredSourceColumns(); auto select = std::make_shared(); From 076ec440897055f1b3c78999a0e391ea0fdef6c9 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 9 Nov 2018 01:24:30 +0800 Subject: [PATCH 11/46] Prioritize rvalue creator for ColumnTuple This validates `ColumnTuple::create(Columns({std::move(s_c0), std::move(s_c1)})); ` --- dbms/src/Columns/ColumnTuple.h | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Columns/ColumnTuple.h b/dbms/src/Columns/ColumnTuple.h index b9fefd26db4..d146c8bff6c 100644 --- a/dbms/src/Columns/ColumnTuple.h +++ b/dbms/src/Columns/ColumnTuple.h @@ -31,6 +31,7 @@ public: */ using Base = COWPtrHelper; static Ptr create(const Columns & columns); + static Ptr create(Columns && arg) { return create(arg); } template ::value>::type> static MutablePtr create(Arg && arg) { return Base::create(std::forward(arg)); } From 1e9aeba60367155bd2ecfdfcff93c65463ef8be5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 8 Nov 2018 20:28:52 +0300 Subject: [PATCH 12/46] Split ExpressionAnalyzer and SyntaxAnalyzer. --- .../DataStreams/tests/expression_stream.cpp | 2 +- dbms/src/DataStreams/tests/filter_stream.cpp | 2 +- dbms/src/Interpreters/AnalyzedJoin.cpp | 2 +- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 20 +++++++------- dbms/src/Interpreters/ExpressionAnalyzer.h | 2 +- .../Interpreters/InterpreterCreateQuery.cpp | 5 ++-- .../Interpreters/InterpreterSelectQuery.cpp | 20 +++++++------- .../src/Interpreters/MutationsInterpreter.cpp | 10 +++---- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 9 ++++--- dbms/src/Interpreters/SyntaxAnalyzer.h | 4 ++- .../evaluateConstantExpression.cpp | 7 +++-- .../Interpreters/evaluateMissingDefaults.cpp | 2 +- dbms/src/Interpreters/tests/expression.cpp | 2 +- dbms/src/Storages/AlterCommands.cpp | 7 +++-- dbms/src/Storages/MergeTree/KeyCondition.cpp | 9 +++---- dbms/src/Storages/MergeTree/KeyCondition.h | 4 +-- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 26 +++++++------------ .../MergeTree/MergeTreeDataSelectExecutor.cpp | 22 +++++----------- .../MergeTree/MergeTreeWhereOptimizer.cpp | 3 +-- dbms/src/Storages/StorageDistributed.cpp | 2 +- dbms/src/Storages/StorageMerge.cpp | 6 ++--- dbms/src/Storages/VirtualColumnUtils.cpp | 5 ++-- .../transformQueryForExternalDatabase.cpp | 7 +++-- 23 files changed, 76 insertions(+), 102 deletions(-) diff --git a/dbms/src/DataStreams/tests/expression_stream.cpp b/dbms/src/DataStreams/tests/expression_stream.cpp index 75f999c7344..3cbce14649d 100644 --- a/dbms/src/DataStreams/tests/expression_stream.cpp +++ b/dbms/src/DataStreams/tests/expression_stream.cpp @@ -38,7 +38,7 @@ try NamesAndTypesList source_columns = {{"number", std::make_shared()}}; auto syntax_result = SyntaxAnalyzer(context, {}).analyze(ast, source_columns); - ExpressionAnalyzer analyzer(ast, syntax_result, context, source_columns); + ExpressionAnalyzer analyzer(ast, syntax_result, context); ExpressionActionsChain chain(context); analyzer.appendSelect(chain, false); analyzer.appendProjectResult(chain); diff --git a/dbms/src/DataStreams/tests/filter_stream.cpp b/dbms/src/DataStreams/tests/filter_stream.cpp index 3e13a9b7966..ed12c09dc99 100644 --- a/dbms/src/DataStreams/tests/filter_stream.cpp +++ b/dbms/src/DataStreams/tests/filter_stream.cpp @@ -43,7 +43,7 @@ try NamesAndTypesList source_columns = {{"number", std::make_shared()}}; auto syntax_result = SyntaxAnalyzer(context, {}).analyze(ast, source_columns); - ExpressionAnalyzer analyzer(ast, syntax_result, context, source_columns); + ExpressionAnalyzer analyzer(ast, syntax_result, context); ExpressionActionsChain chain(context); analyzer.appendSelect(chain, false); analyzer.appendProjectResult(chain); diff --git a/dbms/src/Interpreters/AnalyzedJoin.cpp b/dbms/src/Interpreters/AnalyzedJoin.cpp index 7642e3d6503..83dfebcd852 100644 --- a/dbms/src/Interpreters/AnalyzedJoin.cpp +++ b/dbms/src/Interpreters/AnalyzedJoin.cpp @@ -49,7 +49,7 @@ ExpressionActionsPtr AnalyzedJoin::createJoinedBlockActions( ASTPtr query = expression_list; auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, source_column_names, required_columns); - ExpressionAnalyzer analyzer(query, syntax_result, context, source_column_names, required_columns); + ExpressionAnalyzer analyzer(query, syntax_result, context, {}, required_columns); auto joined_block_actions = analyzer.getActions(false); auto required_action_columns = joined_block_actions->getRequiredColumns(); diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 4055e73cea6..d9b2163c0b7 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -89,12 +89,12 @@ ExpressionAnalyzer::ExpressionAnalyzer( const ASTPtr & query_, const SyntaxAnalyzerResultPtr & syntax_analyzer_result_, const Context & context_, - const NamesAndTypesList & source_columns_, + const NamesAndTypesList & additional_source_columns, const Names & required_result_columns_, size_t subquery_depth_, bool do_global_, const SubqueriesForSets & subqueries_for_sets_) - : ExpressionAnalyzerData(source_columns_, required_result_columns_, subqueries_for_sets_) + : ExpressionAnalyzerData(syntax_analyzer_result_->source_columns, required_result_columns_, subqueries_for_sets_) , query(query_), context(context_), settings(context.getSettings()) , subquery_depth(subquery_depth_), do_global(do_global_) , syntax(syntax_analyzer_result_), analyzed_join(syntax->analyzed_join) @@ -104,7 +104,11 @@ ExpressionAnalyzer::ExpressionAnalyzer( select_query = typeid_cast(query.get()); - removeDuplicateColumns(source_columns); + if (!additional_source_columns.empty()) + { + source_columns.insert(source_columns.end(), additional_source_columns.begin(), additional_source_columns.end()); + removeDuplicateColumns(source_columns); + } /// Delete the unnecessary from `source_columns` list. Create `unknown_required_source_columns`. Form `columns_added_by_join`. collectUsedColumns(); @@ -694,17 +698,15 @@ bool ExpressionAnalyzer::appendPrewhere(ExpressionActionsChain & chain, bool onl Names additional_required_mergetree_columns; if (sampling_expression) { - NamesAndTypesList columns; auto ast = sampling_expression; - auto syntax_result = SyntaxAnalyzer(context, storage).analyze(ast, columns); - additional_required_mergetree_columns = ExpressionAnalyzer(ast, syntax_result, context, columns).getRequiredSourceColumns(); + auto syntax_result = SyntaxAnalyzer(context, storage).analyze(ast, {}); + additional_required_mergetree_columns = ExpressionAnalyzer(ast, syntax_result, context).getRequiredSourceColumns(); } if (primary_expression) { - NamesAndTypesList columns; auto ast = primary_expression; - auto syntax_result = SyntaxAnalyzer(context, storage).analyze(ast, columns); - auto required_primary_columns = ExpressionAnalyzer(ast, syntax_result, context, columns).getRequiredSourceColumns(); + auto syntax_result = SyntaxAnalyzer(context, storage).analyze(ast, {}); + auto required_primary_columns = ExpressionAnalyzer(ast, syntax_result, context).getRequiredSourceColumns(); additional_required_mergetree_columns.insert(additional_required_mergetree_columns.end(), required_primary_columns.begin(), required_primary_columns.end()); } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index 3dd21b2f82f..c2eddcb820c 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -141,7 +141,7 @@ public: const ASTPtr & query_, const SyntaxAnalyzerResultPtr & syntax_analyzer_result_, const Context & context_, - const NamesAndTypesList & source_columns_, + const NamesAndTypesList & additional_source_columns = {}, const Names & required_result_columns_ = {}, size_t subquery_depth_ = 0, bool do_global_ = false, diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 923093f5b12..af2b51e58e8 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -224,9 +224,8 @@ static ColumnsAndDefaults parseColumns(const ASTExpressionList & column_list_ast /// set missing types and wrap default_expression's in a conversion-function if necessary if (!defaulted_columns.empty()) { - auto source_columns = columns; - auto syntax_analyzer_result = SyntaxAnalyzer(context, {}).analyze(default_expr_list, source_columns); - const auto actions = ExpressionAnalyzer(default_expr_list, syntax_analyzer_result, context, source_columns).getActions(true); + auto syntax_analyzer_result = SyntaxAnalyzer(context, {}).analyze(default_expr_list, columns); + const auto actions = ExpressionAnalyzer(default_expr_list, syntax_analyzer_result, context).getActions(true); const auto block = actions->getSampleBlock(); for (auto action : actions->getActions()) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 998544c0e67..5ee3b1cc5bf 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -184,11 +184,10 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (storage) table_lock = storage->lockStructure(false, __PRETTY_FUNCTION__); - auto source_columns = source_header.getNamesAndTypesList(); syntax_analyzer_result = SyntaxAnalyzer(context, storage) - .analyze(query_ptr, source_columns, required_result_column_names, subquery_depth); + .analyze(query_ptr, source_header.getNamesAndTypesList(), required_result_column_names, subquery_depth); query_analyzer = std::make_unique( - query_ptr, syntax_analyzer_result, context, source_columns, required_result_column_names, subquery_depth, !only_analyze); + query_ptr, syntax_analyzer_result, context, NamesAndTypesList(), required_result_column_names, subquery_depth, !only_analyze); if (!only_analyze) { @@ -756,8 +755,7 @@ void InterpreterSelectQuery::executeFetchColumns( } /// Columns which we will get after prewhere execution. - NamesAndTypesList alias_source_columns = source_columns; - NameSet additional_source_columns; + NamesAndTypesList additional_source_columns; /// Add columns which will be added by prewhere (otherwise we will remove them in project action). NameSet columns_to_remove(columns_to_remove_after_prewhere.begin(), columns_to_remove_after_prewhere.end()); for (const auto & column : prewhere_actions_result) @@ -769,11 +767,13 @@ void InterpreterSelectQuery::executeFetchColumns( continue; required_columns_expr_list->children.emplace_back(std::make_shared(column.name)); - alias_source_columns.emplace_back(column.name, column.type); - additional_source_columns.insert(column.name); + additional_source_columns.emplace_back(column.name, column.type); } + auto additional_source_columns_set = ext::map(additional_source_columns, [] (const auto & it) { return it.name; }); - alias_actions = ExpressionAnalyzer(required_columns_expr_list, syntax_analyzer_result, context, alias_source_columns).getActions(true); + ASTPtr query = required_columns_expr_list; + auto syntax_result = SyntaxAnalyzer(context, storage).analyze(query, additional_source_columns); + alias_actions = ExpressionAnalyzer(query, syntax_result, context).getActions(true); /// The set of required columns could be added as a result of adding an action to calculate ALIAS. required_columns = alias_actions->getRequiredColumns(); @@ -788,7 +788,7 @@ void InterpreterSelectQuery::executeFetchColumns( size_t next_req_column_pos = 0; for (size_t i = 0; i < required_columns.size(); ++i) { - if (!additional_source_columns.count(required_columns[i])) + if (!additional_source_columns_set.count(required_columns[i])) { if (next_req_column_pos < i) std::swap(required_columns[i], required_columns[next_req_column_pos]); @@ -810,7 +810,7 @@ void InterpreterSelectQuery::executeFetchColumns( prewhere_info->prewhere_actions = std::move(new_actions); prewhere_info->alias_actions = - ExpressionAnalyzer(required_prewhere_columns_expr_list, syntax_analyzer_result, context, source_columns) + ExpressionAnalyzer(required_prewhere_columns_expr_list, syntax_analyzer_result, context) .getActions(true, false); /// Add columns required by alias actions. diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index e63d9137705..60672d62cc6 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -196,10 +196,9 @@ void MutationsInterpreter::prepare(bool dry_run) const ColumnDefault & col_default = kv.second; if (col_default.kind == ColumnDefaultKind::Materialized) { - NamesAndTypesList source_columns = all_columns; auto query = col_default.expression->clone(); - auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, source_columns); - ExpressionAnalyzer analyzer(query, syntax_result, context, source_columns); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, all_columns); + ExpressionAnalyzer analyzer(query, syntax_result, context); for (const String & dependency : analyzer.getRequiredSourceColumns()) { if (updated_columns.count(dependency)) @@ -305,9 +304,8 @@ void MutationsInterpreter::prepare(bool dry_run) for (const String & column : stage.output_columns) all_asts->children.push_back(std::make_shared(column)); - NamesAndTypesList source_columns = all_columns; - auto syntax_result = SyntaxAnalyzer(context, {}).analyze(all_asts, source_columns); - stage.analyzer = std::make_unique(all_asts, syntax_result, context, all_columns); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(all_asts, all_columns); + stage.analyzer = std::make_unique(all_asts, syntax_result, context); ExpressionActionsChain & actions_chain = stage.expressions_chain; diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 49bb7e7c306..8724f632f8c 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -104,20 +104,21 @@ void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery * s SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( ASTPtr & query, - NamesAndTypesList & source_columns, + const NamesAndTypesList & source_columns_, const Names & required_result_columns, size_t subquery_depth) const { SyntaxAnalyzerResult result; result.storage = storage; + result.source_columns = source_columns_; auto * select_query = typeid_cast(query.get()); - collectSourceColumns(source_columns, select_query, context, result.storage); + collectSourceColumns(result.source_columns, select_query, context, result.storage); const auto & settings = context.getSettingsRef(); Names source_columns_list; - source_columns_list.reserve(source_columns.size()); - for (const auto & type_name : source_columns) + source_columns_list.reserve(result.source_columns.size()); + for (const auto & type_name : result.source_columns) source_columns_list.emplace_back(type_name.name); NameSet source_columns_set(source_columns_list.begin(), source_columns_list.end()); diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.h b/dbms/src/Interpreters/SyntaxAnalyzer.h index c03b10d3cd2..38595917917 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.h +++ b/dbms/src/Interpreters/SyntaxAnalyzer.h @@ -12,6 +12,8 @@ struct SyntaxAnalyzerResult { StoragePtr storage; + NamesAndTypesList source_columns; + /// Note: used only in tests. using Aliases = std::unordered_map; Aliases aliases; @@ -56,7 +58,7 @@ public: SyntaxAnalyzerResultPtr analyze( ASTPtr & query, - NamesAndTypesList & source_columns, + const NamesAndTypesList & source_columns_, const Names & required_result_columns = {}, size_t subquery_depth = 0) const; diff --git a/dbms/src/Interpreters/evaluateConstantExpression.cpp b/dbms/src/Interpreters/evaluateConstantExpression.cpp index be94f04ff3c..adb60d34eaa 100644 --- a/dbms/src/Interpreters/evaluateConstantExpression.cpp +++ b/dbms/src/Interpreters/evaluateConstantExpression.cpp @@ -28,10 +28,9 @@ namespace ErrorCodes std::pair> evaluateConstantExpression(const ASTPtr & node, const Context & context) { NamesAndTypesList source_columns = {{ "_dummy", std::make_shared() }}; - auto query = node->clone(); - auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, source_columns, {}); - ExpressionActionsPtr expr_for_constant_folding - = ExpressionAnalyzer(node, syntax_result, context, source_columns).getConstActions(); + auto ast = node->clone(); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(ast, source_columns); + ExpressionActionsPtr expr_for_constant_folding = ExpressionAnalyzer(ast, syntax_result, context).getConstActions(); /// There must be at least one column in the block so that it knows the number of rows. Block block_with_constants{{ ColumnConst::create(ColumnUInt8::create(1, 0), 1), std::make_shared(), "_dummy" }}; diff --git a/dbms/src/Interpreters/evaluateMissingDefaults.cpp b/dbms/src/Interpreters/evaluateMissingDefaults.cpp index 34bd252f95d..e0574b31333 100644 --- a/dbms/src/Interpreters/evaluateMissingDefaults.cpp +++ b/dbms/src/Interpreters/evaluateMissingDefaults.cpp @@ -49,7 +49,7 @@ void evaluateMissingDefaults(Block & block, available_columns.emplace_back(block.getByPosition(i).name, block.getByPosition(i).type); auto syntax_result = SyntaxAnalyzer(context, {}).analyze(default_expr_list, available_columns); - ExpressionAnalyzer{default_expr_list, syntax_result, context, available_columns}.getActions(true)->execute(copy_block); + ExpressionAnalyzer{default_expr_list, syntax_result, context}.getActions(true)->execute(copy_block); /// move evaluated columns to the original block, materializing them at the same time size_t pos = 0; diff --git a/dbms/src/Interpreters/tests/expression.cpp b/dbms/src/Interpreters/tests/expression.cpp index 7b5203ec01f..73502d9067e 100644 --- a/dbms/src/Interpreters/tests/expression.cpp +++ b/dbms/src/Interpreters/tests/expression.cpp @@ -55,7 +55,7 @@ int main(int argc, char ** argv) }; auto syntax_result = SyntaxAnalyzer(context, {}).analyze(ast, columns); - ExpressionAnalyzer analyzer(ast, syntax_result, context, columns); + ExpressionAnalyzer analyzer(ast, syntax_result, context); ExpressionActionsChain chain(context); analyzer.appendSelect(chain, false); analyzer.appendProjectResult(chain); diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index e117c066af9..fefd96262c2 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -321,10 +321,9 @@ void AlterCommands::validate(const IStorage & table, const Context & context) for (const auto & default_column : defaults) { const auto & default_expression = default_column.second.expression; - NamesAndTypesList source_columns = all_columns; ASTPtr query = default_expression; - auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, source_columns); - const auto actions = ExpressionAnalyzer(query, syntax_result, context, all_columns).getActions(true); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, all_columns); + const auto actions = ExpressionAnalyzer(query, syntax_result, context).getActions(true); const auto required_columns = actions->getRequiredColumns(); if (required_columns.end() != std::find(required_columns.begin(), required_columns.end(), command.column_name)) @@ -382,7 +381,7 @@ void AlterCommands::validate(const IStorage & table, const Context & context) ASTPtr query = default_expr_list; auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, all_columns); - const auto actions = ExpressionAnalyzer(query, syntax_result, context, all_columns).getActions(true); + const auto actions = ExpressionAnalyzer(query, syntax_result, context).getActions(true); const auto block = actions->getSampleBlock(); /// set deduced types, modify default expression if necessary diff --git a/dbms/src/Storages/MergeTree/KeyCondition.cpp b/dbms/src/Storages/MergeTree/KeyCondition.cpp index 4978e23183d..37610f59f1f 100644 --- a/dbms/src/Storages/MergeTree/KeyCondition.cpp +++ b/dbms/src/Storages/MergeTree/KeyCondition.cpp @@ -248,15 +248,14 @@ bool FieldWithInfinity::operator==(const FieldWithInfinity & other) const * For index to work when something like "WHERE Date = toDate(now())" is written. */ Block KeyCondition::getBlockWithConstants( - const ASTPtr & query, const SyntaxAnalyzerResultPtr & syntax_analyzer_result, - const Context & context, const NamesAndTypesList & all_columns) + const ASTPtr & query, const SyntaxAnalyzerResultPtr & syntax_analyzer_result, const Context & context) { Block result { { DataTypeUInt8().createColumnConstWithDefaultValue(1), std::make_shared(), "_dummy" } }; - const auto expr_for_constant_folding = ExpressionAnalyzer(query, syntax_analyzer_result, context, all_columns).getConstActions(); + const auto expr_for_constant_folding = ExpressionAnalyzer(query, syntax_analyzer_result, context).getConstActions(); expr_for_constant_folding->execute(result); @@ -267,7 +266,6 @@ Block KeyCondition::getBlockWithConstants( KeyCondition::KeyCondition( const SelectQueryInfo & query_info, const Context & context, - const NamesAndTypesList & all_columns, const Names & key_column_names, const ExpressionActionsPtr & key_expr_) : key_expr(key_expr_), prepared_sets(query_info.sets) @@ -282,8 +280,7 @@ KeyCondition::KeyCondition( /** Evaluation of expressions that depend only on constants. * For the index to be used, if it is written, for example `WHERE Date = toDate(now())`. */ - Block block_with_constants = getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, - context, all_columns); + Block block_with_constants = getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context); /// Trasform WHERE section to Reverse Polish notation const ASTSelectQuery & select = typeid_cast(*query_info.query); diff --git a/dbms/src/Storages/MergeTree/KeyCondition.h b/dbms/src/Storages/MergeTree/KeyCondition.h index d6db68ef738..d025f70bf09 100644 --- a/dbms/src/Storages/MergeTree/KeyCondition.h +++ b/dbms/src/Storages/MergeTree/KeyCondition.h @@ -232,7 +232,6 @@ public: KeyCondition( const SelectQueryInfo & query_info, const Context & context, - const NamesAndTypesList & all_columns, const Names & key_column_names, const ExpressionActionsPtr & key_expr); @@ -269,8 +268,7 @@ public: static Block getBlockWithConstants( - const ASTPtr & query, const SyntaxAnalyzerResultPtr & syntax_analyzer_result, - const Context & context, const NamesAndTypesList & all_columns); + const ASTPtr & query, const SyntaxAnalyzerResultPtr & syntax_analyzer_result, const Context & context); static std::optional applyMonotonicFunctionsChainToRange( Range key_range, diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index c80464e93ac..524b8bfe8bf 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -228,12 +228,10 @@ void MergeTreeData::initPrimaryKey() addSortColumns(primary_sort_columns, primary_expr_ast); { - auto source_columns = getColumns().getAllPhysical(); - auto syntax_result = SyntaxAnalyzer(context, {}).analyze(primary_expr_ast, source_columns); - primary_expr = ExpressionAnalyzer(primary_expr_ast, syntax_result, context, source_columns).getActions(false); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(primary_expr_ast, getColumns().getAllPhysical()); + primary_expr = ExpressionAnalyzer(primary_expr_ast, syntax_result, context).getActions(false); - ExpressionActionsPtr projected_expr = - ExpressionAnalyzer(primary_expr_ast, syntax_result, context, source_columns).getActions(true); + ExpressionActionsPtr projected_expr = ExpressionAnalyzer(primary_expr_ast, syntax_result, context).getActions(true); primary_key_sample = projected_expr->getSampleBlock(); } @@ -248,12 +246,10 @@ void MergeTreeData::initPrimaryKey() if (secondary_sort_expr_ast) { addSortColumns(sort_columns, secondary_sort_expr_ast); - NamesAndTypesList source_columns = getColumns().getAllPhysical(); - auto syntax_result = SyntaxAnalyzer(context, {}).analyze(secondary_sort_expr_ast, source_columns); - secondary_sort_expr = ExpressionAnalyzer(secondary_sort_expr_ast, syntax_result, context, source_columns).getActions(false); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(secondary_sort_expr_ast, getColumns().getAllPhysical()); + secondary_sort_expr = ExpressionAnalyzer(secondary_sort_expr_ast, syntax_result, context).getActions(false); - ExpressionActionsPtr projected_expr = - ExpressionAnalyzer(secondary_sort_expr_ast, syntax_result, context, source_columns).getActions(true); + ExpressionActionsPtr projected_expr = ExpressionAnalyzer(secondary_sort_expr_ast, syntax_result, context).getActions(true); auto secondary_key_sample = projected_expr->getSampleBlock(); checkKeyExpression(*secondary_sort_expr, secondary_key_sample, "Secondary"); @@ -267,9 +263,8 @@ void MergeTreeData::initPartitionKey() return; { - NamesAndTypesList source_columns = getColumns().getAllPhysical(); - auto syntax_result = SyntaxAnalyzer(context, {}).analyze(partition_expr_ast, source_columns); - partition_expr = ExpressionAnalyzer(partition_expr_ast, syntax_result, context, source_columns).getActions(false); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(partition_expr_ast, getColumns().getAllPhysical()); + partition_expr = ExpressionAnalyzer(partition_expr_ast, syntax_result, context).getActions(false); } for (const ASTPtr & ast : partition_expr_ast->children) @@ -1157,10 +1152,9 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart( /// TODO: Check the order of secondary sorting key columns. if (new_primary_key.get() != primary_expr_ast.get()) { - NamesAndTypesList source_columns = new_columns; ASTPtr query = new_primary_key; - auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, source_columns); - ExpressionActionsPtr new_primary_expr = ExpressionAnalyzer(query, syntax_result, context, source_columns).getActions(true); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, new_columns); + ExpressionActionsPtr new_primary_expr = ExpressionAnalyzer(query, syntax_result, context).getActions(true); Block new_primary_key_sample = new_primary_expr->getSampleBlock(); size_t new_key_size = new_primary_key_sample.columns(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index ecc6b6ecf33..23f147cc47f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -194,10 +194,6 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( NamesAndTypesList available_real_columns = data.getColumns().getAllPhysical(); - NamesAndTypesList available_real_and_virtual_columns = available_real_columns; - for (const auto & name : virt_column_names) - available_real_and_virtual_columns.emplace_back(data.getColumn(name)); - /// If there are only virtual columns in the query, you must request at least one non-virtual one. if (real_column_names.empty()) real_column_names.push_back(ExpressionActions::getSmallestColumn(available_real_columns)); @@ -214,9 +210,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( const Settings & settings = context.getSettingsRef(); Names primary_sort_columns = data.getPrimarySortColumns(); - KeyCondition key_condition( - query_info, context, available_real_and_virtual_columns, - primary_sort_columns, data.getPrimaryExpression()); + KeyCondition key_condition(query_info, context, primary_sort_columns, data.getPrimaryExpression()); if (settings.force_primary_key && key_condition.alwaysUnknownOrTrue()) { @@ -232,9 +226,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( std::optional minmax_idx_condition; if (data.minmax_idx_expr) { - minmax_idx_condition.emplace( - query_info, context, available_real_and_virtual_columns, - data.minmax_idx_columns, data.minmax_idx_expr); + minmax_idx_condition.emplace(query_info, context, data.minmax_idx_columns, data.minmax_idx_expr); if (settings.force_index_by_date && minmax_idx_condition->alwaysUnknownOrTrue()) { @@ -495,10 +487,9 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( filter_function->children.push_back(filter_function->arguments); } - NamesAndTypesList source_columns = available_real_columns; ASTPtr query = filter_function; - auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, source_columns); - filter_expression = ExpressionAnalyzer(filter_function, syntax_result, context, source_columns).getActions(false); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, available_real_columns); + filter_expression = ExpressionAnalyzer(filter_function, syntax_result, context).getActions(false); /// Add columns needed for `sampling_expression` to `column_names_to_read`. std::vector add_columns = filter_expression->getRequiredColumns(); @@ -857,9 +848,8 @@ void MergeTreeDataSelectExecutor::createPositiveSignCondition( arguments->children.push_back(one); ASTPtr query = function; - NamesAndTypesList source_columns = data.getColumns().getAllPhysical(); - auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, source_columns); - out_expression = ExpressionAnalyzer(query, syntax_result, context, source_columns).getActions(false); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, data.getColumns().getAllPhysical()); + out_expression = ExpressionAnalyzer(query, syntax_result, context).getActions(false); out_column = function->getColumnName(); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 14efe1484b2..c7b3148390e 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -43,8 +43,7 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( : primary_key_columns{ext::collection_cast(data.getPrimarySortColumns())}, table_columns{ext::map(data.getColumns().getAllPhysical(), [] (const NameAndTypePair & col) { return col.name; })}, - block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, - context, data.getColumns().getAllPhysical())}, + block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)}, prepared_sets(query_info.sets), log{log} { diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 5bdd2cc7417..5d3860f449d 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -145,7 +145,7 @@ static ExpressionActionsPtr buildShardingKeyExpression(const ASTPtr & sharding_k { ASTPtr query = sharding_key; auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, columns); - return ExpressionAnalyzer(query, syntax_result, context, columns).getActions(project); + return ExpressionAnalyzer(query, syntax_result, context).getActions(project); } StorageDistributed::StorageDistributed( diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 3730ed92849..e8684478065 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -265,10 +265,8 @@ BlockInputStreams StorageMerge::createSourceStreams(const SelectQueryInfo & quer Context & modified_context, size_t streams_num, bool has_table_virtual_column, bool concat_streams) { - SelectQueryInfo modified_query_info; - modified_query_info.sets = query_info.sets; + SelectQueryInfo modified_query_info = query_info; modified_query_info.query = query_info.query->clone(); - modified_query_info.prewhere_info = query_info.prewhere_info; VirtualColumnUtils::rewriteEntityInAst(modified_query_info.query, "_table", storage ? storage->getTableName() : ""); @@ -460,7 +458,7 @@ void StorageMerge::convertingSourceStream(const Block & header, const Context & NameAndTypePair virtual_column = getColumn("_table"); source_columns.insert(source_columns.end(), virtual_column); auto syntax_result = SyntaxAnalyzer(context, {}).analyze(where_expression, source_columns); - ExpressionActionsPtr actions = ExpressionAnalyzer{where_expression, syntax_result, context, source_columns}.getActions(false, false); + ExpressionActionsPtr actions = ExpressionAnalyzer{where_expression, syntax_result, context}.getActions(false, false); Names required_columns = actions->getRequiredColumns(); for (const auto required_column : required_columns) diff --git a/dbms/src/Storages/VirtualColumnUtils.cpp b/dbms/src/Storages/VirtualColumnUtils.cpp index 5e32ca14fa2..ba7f7005d80 100644 --- a/dbms/src/Storages/VirtualColumnUtils.cpp +++ b/dbms/src/Storages/VirtualColumnUtils.cpp @@ -157,9 +157,8 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & c return; /// Let's analyze and calculate the expression. - NamesAndTypesList source_columns = block.getNamesAndTypesList(); - auto syntax_result = SyntaxAnalyzer(context, {}).analyze(expression_ast, source_columns); - ExpressionAnalyzer analyzer(expression_ast, syntax_result, context, source_columns); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(expression_ast, block.getNamesAndTypesList()); + ExpressionAnalyzer analyzer(expression_ast, syntax_result, context); ExpressionActionsPtr actions = analyzer.getActions(false); Block block_with_filter = block; diff --git a/dbms/src/Storages/transformQueryForExternalDatabase.cpp b/dbms/src/Storages/transformQueryForExternalDatabase.cpp index dabd6268aa7..f37e51b714e 100644 --- a/dbms/src/Storages/transformQueryForExternalDatabase.cpp +++ b/dbms/src/Storages/transformQueryForExternalDatabase.cpp @@ -29,7 +29,7 @@ static void replaceConstFunction(IAST & node, const Context & context, const Nam NamesAndTypesList source_columns = all_columns; ASTPtr query = function->ptr(); auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, source_columns); - auto result_block = KeyCondition::getBlockWithConstants(query, syntax_result, context, all_columns); + auto result_block = KeyCondition::getBlockWithConstants(query, syntax_result, context); if (!result_block.has(child->getColumnName())) return; @@ -92,9 +92,8 @@ String transformQueryForExternalDatabase( const Context & context) { auto clone_query = query.clone(); - NamesAndTypesList source_columns = available_columns; - auto syntax_result = SyntaxAnalyzer(context, {}).analyze(clone_query, source_columns); - ExpressionAnalyzer analyzer(clone_query, syntax_result, context, available_columns); + auto syntax_result = SyntaxAnalyzer(context, {}).analyze(clone_query, available_columns); + ExpressionAnalyzer analyzer(clone_query, syntax_result, context); const Names & used_columns = analyzer.getRequiredSourceColumns(); auto select = std::make_shared(); From 21c3b36d239fdcce5181ad57e9fe375ab7174f3e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Nov 2018 19:35:05 +0300 Subject: [PATCH 13/46] Updated test with mutations. --- dbms/tests/queries/0_stateless/00652_mergetree_mutations.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00652_mergetree_mutations.sh b/dbms/tests/queries/0_stateless/00652_mergetree_mutations.sh index d762d49a6ce..a11377fac81 100755 --- a/dbms/tests/queries/0_stateless/00652_mergetree_mutations.sh +++ b/dbms/tests/queries/0_stateless/00652_mergetree_mutations.sh @@ -23,7 +23,7 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO test.mutations(d, x, s) VALUES \ ${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.mutations DELETE WHERE nonexistent = 0" 2>/dev/null || echo "Query should fail 1" ${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.mutations DELETE WHERE d = '11'" 2>/dev/null || echo "Query should fail 2" # TODO: Queries involving alias columns are not supported yet and should fail on submission. -${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.mutations DELETE WHERE a = 0" 2>/dev/null || echo "Query involving aliases should fail on submission" +${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.mutations UPDATE s = s || '' WHERE a = 0" 2>/dev/null || echo "Query involving aliases should fail on submission" # Delete some values ${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.mutations DELETE WHERE x % 2 = 1" From 28be6d6443cc8031249a6892b58f4eb2ac631661 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Nov 2018 20:23:48 +0300 Subject: [PATCH 14/46] Review fixes. --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 26 ++++++++++---------- dbms/src/Interpreters/ExpressionAnalyzer.h | 2 +- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 10 ++++---- 3 files changed, 19 insertions(+), 19 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index d9b2163c0b7..55e04b7a888 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -97,7 +97,7 @@ ExpressionAnalyzer::ExpressionAnalyzer( : ExpressionAnalyzerData(syntax_analyzer_result_->source_columns, required_result_columns_, subqueries_for_sets_) , query(query_), context(context_), settings(context.getSettings()) , subquery_depth(subquery_depth_), do_global(do_global_) - , syntax(syntax_analyzer_result_), analyzed_join(syntax->analyzed_join) + , syntax(syntax_analyzer_result_) { storage = syntax->storage; rewrite_subqueries = syntax->rewrite_subqueries; @@ -163,7 +163,7 @@ void ExpressionAnalyzer::analyzeAggregation() if (table_join.using_expression_list) getRootActions(table_join.using_expression_list, true, temp_actions); if (table_join.on_expression) - for (const auto & key_ast : analyzed_join.key_asts_left) + for (const auto & key_ast : analyzedJoin().key_asts_left) getRootActions(key_ast, true, temp_actions); addJoinAction(temp_actions, true); @@ -417,7 +417,7 @@ void ExpressionAnalyzer::getActionsFromJoinKeys(const ASTTableJoin & table_join, actions_visitor.visit(table_join.using_expression_list); else if (table_join.on_expression) { - for (const auto & ast : analyzed_join.key_asts_left) + for (const auto & ast : analyzedJoin().key_asts_left) actions_visitor.visit(ast); } @@ -559,11 +559,11 @@ void ExpressionAnalyzer::addJoinAction(ExpressionActionsPtr & actions, bool only columns_added_by_join_list.push_back(joined_column.name_and_type); if (only_types) - actions->add(ExpressionAction::ordinaryJoin(nullptr, analyzed_join.key_names_left, columns_added_by_join_list)); + actions->add(ExpressionAction::ordinaryJoin(nullptr, analyzedJoin().key_names_left, columns_added_by_join_list)); else for (auto & subquery_for_set : subqueries_for_sets) if (subquery_for_set.second.join) - actions->add(ExpressionAction::ordinaryJoin(subquery_for_set.second.join, analyzed_join.key_names_left, + actions->add(ExpressionAction::ordinaryJoin(subquery_for_set.second.join, analyzedJoin().key_names_left, columns_added_by_join_list)); } @@ -625,7 +625,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty if (!subquery_for_set.join) { JoinPtr join = std::make_shared( - analyzed_join.key_names_left, analyzed_join.key_names_right, columns_added_by_join_from_right_keys, + analyzedJoin().key_names_left, analyzedJoin().key_names_right, columns_added_by_join_from_right_keys, settings.join_use_nulls, settings.size_limits_for_join, join_params.kind, join_params.strictness); @@ -646,7 +646,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty table = table_to_join.database_and_table_name; Names original_columns; - for (const auto & column : analyzed_join.columns_from_joined_table) + for (const auto & column : analyzedJoin().columns_from_joined_table) if (required_columns_from_joined_table.count(column.name_and_type.name)) original_columns.emplace_back(column.original_name); @@ -657,7 +657,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty } /// Alias duplicating columns as qualified. - for (const auto & column : analyzed_join.columns_from_joined_table) + for (const auto & column : analyzedJoin().columns_from_joined_table) if (required_columns_from_joined_table.count(column.name_and_type.name)) subquery_for_set.joined_block_aliases.emplace_back(column.original_name, column.name_and_type.name); @@ -1062,12 +1062,12 @@ void ExpressionAnalyzer::collectUsedColumns() * (Do not assume that they are required for reading from the "left" table). */ NameSet available_joined_columns; - for (const auto & joined_column : analyzed_join.available_joined_columns) + for (const auto & joined_column : analyzedJoin().available_joined_columns) available_joined_columns.insert(joined_column.name_and_type.name); NameSet required_joined_columns; - for (const auto & left_key_ast : analyzed_join.key_asts_left) + for (const auto & left_key_ast : analyzedJoin().key_asts_left) { NameSet empty; RequiredSourceColumnsVisitor columns_visitor(available_columns, required, ignored, empty, required_joined_columns); @@ -1077,7 +1077,7 @@ void ExpressionAnalyzer::collectUsedColumns() RequiredSourceColumnsVisitor columns_visitor(available_columns, required, ignored, available_joined_columns, required_joined_columns); columns_visitor.visit(query); - columns_added_by_join = analyzed_join.available_joined_columns; + columns_added_by_join = analyzedJoin().available_joined_columns; for (auto it = columns_added_by_join.begin(); it != columns_added_by_join.end();) { if (required_joined_columns.count(it->name_and_type.name)) @@ -1086,11 +1086,11 @@ void ExpressionAnalyzer::collectUsedColumns() columns_added_by_join.erase(it++); } - joined_block_actions = analyzed_join.createJoinedBlockActions( + joined_block_actions = analyzedJoin().createJoinedBlockActions( columns_added_by_join, select_query, context, required_columns_from_joined_table); /// Some columns from right join key may be used in query. This columns will be appended to block during join. - for (const auto & right_key_name : analyzed_join.key_names_right) + for (const auto & right_key_name : analyzedJoin().key_names_right) if (required_joined_columns.count(right_key_name)) columns_added_by_join_from_right_keys.insert(right_key_name); diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index c2eddcb820c..9ce717a0101 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -236,7 +236,7 @@ private: bool do_global; /// Do I need to prepare for execution global subqueries when analyzing the query. SyntaxAnalyzerResultPtr syntax; - const AnalyzedJoin & analyzed_join; + const AnalyzedJoin & analyzedJoin() const { return syntax->analyzed_join; } /** Remove all unnecessary columns from the list of all available columns of the table (`columns`). * At the same time, form a set of unknown columns (`unknown_required_source_columns`), diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 8724f632f8c..4f6a4b5befe 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -49,8 +49,8 @@ using LogAST = DebugASTLog; /// set to true to enable logs using Aliases = std::unordered_map; /// Add columns from storage to source_columns list. -void collectSourceColumns(NamesAndTypesList & source_columns, ASTSelectQuery * select_query, - const Context & context, StoragePtr & storage); +void collectSourceColumns(ASTSelectQuery * select_query, const Context & context, + StoragePtr & storage, NamesAndTypesList & source_columns); /// Translate qualified names such as db.table.column, table.column, table_alias.column to unqualified names. void translateQualifiedNames(ASTPtr & query, ASTSelectQuery * select_query, @@ -112,7 +112,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( result.storage = storage; result.source_columns = source_columns_; auto * select_query = typeid_cast(query.get()); - collectSourceColumns(result.source_columns, select_query, context, result.storage); + collectSourceColumns(select_query, context, result.storage, result.source_columns); const auto & settings = context.getSettingsRef(); @@ -192,8 +192,8 @@ void removeDuplicateColumns(NamesAndTypesList & columns) namespace { -void collectSourceColumns(NamesAndTypesList & source_columns, ASTSelectQuery * select_query, - const Context & context, StoragePtr & storage) +void collectSourceColumns(ASTSelectQuery * select_query, const Context & context, + StoragePtr & storage, NamesAndTypesList & source_columns) { if (!storage && select_query) { From 6a28d8ccb72e6557fb8f5eaa2baf9e949971b773 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 10 Nov 2018 15:37:54 +0300 Subject: [PATCH 15/46] Update build_osx.md --- docs/en/development/build_osx.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/development/build_osx.md b/docs/en/development/build_osx.md index 7de659e7ccd..749360c7a0e 100644 --- a/docs/en/development/build_osx.md +++ b/docs/en/development/build_osx.md @@ -18,8 +18,8 @@ brew install cmake ninja gcc icu4c mariadb-connector-c openssl libtool gettext r ## Checkout ClickHouse Sources ```bash -git clone --recursive --depth=10 git@github.com:yandex/ClickHouse.git -# or: git clone --recursive --depth=10 https://github.com/yandex/ClickHouse.git +git clone --recursive git@github.com:yandex/ClickHouse.git +# or: git clone --recursive https://github.com/yandex/ClickHouse.git cd ClickHouse ``` From 752cd7e4f81b2d020661075fcea0a90a07a63c43 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 11 Nov 2018 00:32:27 +0800 Subject: [PATCH 16/46] Apply upstream jemalloc patch for potential leak https://github.com/jemalloc/jemalloc/issues/1347 --- contrib/jemalloc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/jemalloc b/contrib/jemalloc index 41b7372eade..cd2931ad9bb 160000 --- a/contrib/jemalloc +++ b/contrib/jemalloc @@ -1 +1 @@ -Subproject commit 41b7372eadee941b9164751b8d4963f915d3ceae +Subproject commit cd2931ad9bbd78208565716ab102e86d858c2fff From 8f50c620334988b28018213ec0092fe6423847e2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 10 Nov 2018 20:52:25 +0300 Subject: [PATCH 17/46] Avoid busy loop in DDLWorker when ZooKeeper is not available [#CLICKHOUSE-4126] --- dbms/src/Interpreters/DDLWorker.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index 99fdf7dac3d..d13906d02cc 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -882,8 +882,12 @@ void DDLWorker::run() catch (const Coordination::Exception & e) { if (!Coordination::isHardwareError(e.code)) - throw; + throw; /// A logical error. + tryLogCurrentException(__PRETTY_FUNCTION__); + + /// Avoid busy loop when ZooKeeper is not available. + ::sleep(1); } } catch (...) From 1ba752e931f8131095db5f4e960a54c2f8b5f7fc Mon Sep 17 00:00:00 2001 From: BayoNet Date: Sat, 10 Nov 2018 23:00:22 +0300 Subject: [PATCH 18/46] Russian version of `hasAll` `hasAny` docs. (#3550) * Update of english version of descriprion of the table function `file`. * New syntax for ReplacingMergeTree. Some improvements in text. * Significantly change article about SummingMergeTree. Article is restructured, text is changed in many places of the document. New syntax for table creation is described. * Descriptions of AggregateFunction and AggregatingMergeTree are updated. Russian version. * New syntax for new syntax of CREATE TABLE * Added english docs on Aggregating, Replacing and SummingMergeTree. * CollapsingMergeTree docs. English version. * 1. Update of CollapsingMergeTree. 2. Minor changes in markup * Update aggregatefunction.md * Update aggregatefunction.md * Update aggregatefunction.md * Update aggregatingmergetree.md * GraphiteMergeTree docs update. New syntax for creation of Replicated* tables. Minor changes in *MergeTree tables creation syntax. * Markup fix * Markup and language fixes * Clarification in the CollapsingMergeTree article * DOCAPI-4821. Sync between ru and en versions of docs. * Fixed the ambiguity in geo functions description. * Example of JOIN in ru docs * Deleted misinforming example. * Descriptions for array functions 'hasAll' and 'hasAny' are translated into russian. * Update array_functions.md --- .../functions/array_functions.md | 6 +- .../functions/array_functions.md | 143 ++++++++++++++---- 2 files changed, 120 insertions(+), 29 deletions(-) diff --git a/docs/en/query_language/functions/array_functions.md b/docs/en/query_language/functions/array_functions.md index cd6528743d7..65c234935c3 100644 --- a/docs/en/query_language/functions/array_functions.md +++ b/docs/en/query_language/functions/array_functions.md @@ -55,7 +55,7 @@ arrayConcat(arrays) **Parameters** -- `arrays` – Arrays of comma-separated `[values]`. +- `arrays` – arbitrary number of arguments of type Array. **Example** @@ -71,9 +71,9 @@ SELECT arrayConcat([1, 2], [3, 4], [5, 6]) AS res ## arrayElement(arr, n), operator arr[n] -Get the element with the index 'n' from the array 'arr'.'n' must be any integer type. +Get the element with the index `n` from the array `arr`. `n` must be any integer type. Indexes in an array begin from one. -Negative indexes are supported. In this case, it selects the corresponding element numbered from the end. For example, 'arr\[-1\]' is the last item in the array. +Negative indexes are supported. In this case, it selects the corresponding element numbered from the end. For example, `arr[-1]` is the last item in the array. If the index falls outside of the bounds of an array, it returns some default value (0 for numbers, an empty string for strings, etc.). diff --git a/docs/ru/query_language/functions/array_functions.md b/docs/ru/query_language/functions/array_functions.md index 53bf3b94af6..1cbe686cefa 100644 --- a/docs/ru/query_language/functions/array_functions.md +++ b/docs/ru/query_language/functions/array_functions.md @@ -1,17 +1,19 @@ # Функции по работе с массивами - ## empty + Возвращает 1 для пустого массива, и 0 для непустого массива. Тип результата - UInt8. Функция также работает для строк. ## notEmpty + Возвращает 0 для пустого массива, и 1 для непустого массива. Тип результата - UInt8. Функция также работает для строк. ## length + Возвращает количество элементов в массиве. Тип результата - UInt64. Функция также работает для строк. @@ -25,16 +27,20 @@ ## emptyArrayDate, emptyArrayDateTime ## emptyArrayString + Принимает ноль аргументов и возвращает пустой массив соответствующего типа. ## emptyArrayToSingle + Принимает пустой массив и возвращает массив из одного элемента, равного значению по умолчанию. ## range(N) + Возвращает массив чисел от 0 до N-1. На всякий случай, если на блок данных, создаются массивы суммарной длины больше 100 000 000 элементов, то кидается исключение. ## array(x1, ...), оператор \[x1, ...\] + Создаёт массив из аргументов функции. Аргументы должны быть константами и иметь типы, для которых есть наименьший общий тип. Должен быть передан хотя бы один аргумент, так как иначе непонятно, какого типа создавать массив. То есть, с помощью этой функции невозможно создать пустой массив (для этого используйте функции emptyArray\*, описанные выше). Возвращает результат типа Array(T), где T - наименьший общий тип от переданных аргументов. @@ -47,31 +53,32 @@ arrayConcat(arrays) ``` -**Аргументы** +**Параметры** - `arrays` - Перечисленные через запятую массивы `[values]`. **Пример** -``` sql +```sql SELECT arrayConcat([1, 2], [3, 4], [5, 6]) AS res ``` + ``` ┌─res───────────┐ │ [1,2,3,4,5,6] │ └───────────────┘ ``` +## arrayElement(arr, n), operator arr[n] -## arrayElement(arr, n), оператор arr\[n\] -Достаёт элемент с индексом n из массива arr. -n должен быть любым целочисленным типом. +Достаёт элемент с индексом n из массива arr. n должен быть любым целочисленным типом. Индексы в массиве начинаются с единицы. -Поддерживаются отрицательные индексы - в этом случае, будет выбран соответствующий по номеру элемент с конца. Например, arr\[-1\] - последний элемент массива. +Поддерживаются отрицательные индексы. В этом случае, будет выбран соответствующий по номеру элемент с конца. Например, arr\[-1\] - последний элемент массива. Если индекс выходит за границы массива, то возвращается некоторое значение по умолчанию (0 для чисел, пустая строка для строк и т. п.). ## has(arr, elem) + Проверяет наличие элемента elem в массиве arr. Возвращает 0, если элемента в массиве нет, или 1, если есть. @@ -85,13 +92,87 @@ SELECT has([1, 2, NULL], NULL) └─────────────────────────┘ ``` +## hasAll + +Проверяет, является ли один массив подмножеством другого. + +``` +hasAll(set, subset) +``` + +**Параметры** + +- `set` – массив любого типа с набором элементов. +- `subset` – массив любого типа со значениями, которые проверяются на вхождение в `set`. + +**Возвращаемые значения** + +- `1`, если `set` содержит все элементы из `subset`. +- `0`, в противном случае. + +**Особенности** + +- Пустой массив является подмножеством любого массива. +- `NULL` обрабатывается как значение. +- Порядок значений в обоих массивах не имеет значения. + +**Примеры** + +`SELECT hasAll([], [])` возвращает 1. + +`SELECT hasAll([1, Null], [Null])` возвращает 1. + +`SELECT hasAll([1.0, 2, 3, 4], [1, 3])` возвращает 1. + +`SELECT hasAll(['a', 'b'], ['a'])` возвращает 1. + +`SELECT hasAll([1], ['a'])` возвращает 0. + +`SELECT hasAll([[1, 2], [3, 4]], [[1, 2], [3, 5]])` возвращает 0. + +## hasAny + +Проверяет, имеют ли два массива хотя бы один общий элемент. + +``` +hasAny(array1, array2) +``` + +**Параметры** + +- `array1` – массив любого типа с набором элементов. +- `array2` – массив любого типа с набором элементов. + +**Возвращаемые значения** + +- `1`, если `array1` и `array2` имеют хотя бы один одинаковый элемент. +- `0`, в противном случае. + +**Особенности** + +- `NULL` обрабатывается как значение. +- Порядок значений в обоих массивах не имеет значения. + +**Примеры** + +`SELECT hasAny([1], [])` возвращает `0`. + +`SELECT hasAny([Null], [Null, 1])` возвращает `1`. + +`SELECT hasAny([-128, 1., 512], [1])` возвращает `1`. + +`SELECT hasAny([[1, 2], [3, 4]], ['a', 'c'])` возвращает `0`. + +`SELECT hasAll([[1, 2], [3, 4]], [[1, 2], [1, 2]])` возвращает `1`. + ## indexOf(arr, x) + Возвращает индекс первого элемента x (начиная с 1), если он есть в массиве, или 0, если его нет. Пример: ``` -:) select indexOf([1,3,NULL,NULL],NULL) +:) SELECT indexOf([1,3,NULL,NULL],NULL) SELECT indexOf([1, 3, NULL, NULL], NULL) @@ -103,9 +184,10 @@ SELECT indexOf([1, 3, NULL, NULL], NULL) Элементы, равные `NULL`, обрабатываются как обычные значения. ## countEqual(arr, x) + Возвращает количество элементов массива, равных x. Эквивалентно arrayCount(elem -> elem = x, arr). -Элементы `NULL` обрабатываются как отдельные значения. +`NULL` обрабатывается как значение. Пример: @@ -117,13 +199,13 @@ SELECT countEqual([1, 2, NULL, NULL], NULL) └──────────────────────────────────────┘ ``` - ## arrayEnumerate(arr) + Возвращает массив \[1, 2, 3, ..., length(arr)\] Эта функция обычно используется совместно с ARRAY JOIN. Она позволяет, после применения ARRAY JOIN, посчитать что-либо только один раз для каждого массива. Пример: -``` sql +```sql SELECT count() AS Reaches, countIf(num = 1) AS Hits @@ -143,7 +225,7 @@ LIMIT 10 В этом примере, Reaches - число достижений целей (строк, получившихся после применения ARRAY JOIN), а Hits - число хитов (строк, которые были до ARRAY JOIN). В данном случае, тот же результат можно получить проще: -``` sql +```sql SELECT sum(length(GoalsReached)) AS Reaches, count() AS Hits @@ -160,13 +242,14 @@ WHERE (CounterID = 160656) AND notEmpty(GoalsReached) Также эта функция может быть использована в функциях высшего порядка. Например, с её помощью можно достать индексы массива для элементов, удовлетворяющих некоторому условию. ## arrayEnumerateUniq(arr, ...) + Возвращает массив, такого же размера, как исходный, где для каждого элемента указано, какой он по счету среди элементов с таким же значением. Например: arrayEnumerateUniq(\[10, 20, 10, 30\]) = \[1, 1, 2, 1\]. Эта функция полезна при использовании ARRAY JOIN и агрегации по элементам массива. Пример: -``` sql +```sql SELECT Goals.ID AS GoalID, sum(Sign) AS Reaches, @@ -200,7 +283,7 @@ LIMIT 10 Функция arrayEnumerateUniq может принимать несколько аргументов - массивов одинаковых размеров. В этом случае, уникальность считается для кортежей элементов на одинаковых позициях всех массивов. -``` sql +```sql SELECT arrayEnumerateUniq([1, 1, 1, 2, 2, 2], [1, 1, 2, 1, 1, 2]) AS res ``` @@ -220,15 +303,16 @@ SELECT arrayEnumerateUniq([1, 1, 1, 2, 2, 2], [1, 1, 2, 1, 1, 2]) AS res arrayPopBack(array) ``` -**Аргументы** +**Параметры** - `array` - Массив. **Пример** -``` sql +```sql SELECT arrayPopBack([1, 2, 3]) AS res ``` + ``` ┌─res───┐ │ [1,2] │ @@ -243,15 +327,16 @@ SELECT arrayPopBack([1, 2, 3]) AS res arrayPopFront(array) ``` -**Аргументы** +**Параметры** - `array` - Массив. **Пример** -``` sql +```sql SELECT arrayPopFront([1, 2, 3]) AS res ``` + ``` ┌─res───┐ │ [2,3] │ @@ -266,16 +351,17 @@ SELECT arrayPopFront([1, 2, 3]) AS res arrayPushBack(array, single_value) ``` -**Аргументы** +**Параметры** - `array` - Массив. - `single_value` - Одиночное значение. В массив с числам можно добавить только числа, в массив со строками только строки. При добавлении чисел ClickHouse автоматически приводит тип `single_value` к типу данных массива. Подробнее о типах данных в ClickHouse читайте в разделе "[Типы данных](../../data_types/index.md#data_types)". Может быть равно `NULL`. Функция добавит элемент `NULL` в массив, а тип элементов массива преобразует в `Nullable`. **Пример** -``` sql +```sql SELECT arrayPushBack(['a'], 'b') AS res ``` + ``` ┌─res───────┐ │ ['a','b'] │ @@ -290,16 +376,17 @@ SELECT arrayPushBack(['a'], 'b') AS res arrayPushFront(array, single_value) ``` -**Аргументы** +**Параметры** - `array` - Массив. - `single_value` - Одиночное значение. В массив с числам можно добавить только числа, в массив со строками только строки. При добавлении чисел ClickHouse автоматически приводит тип `single_value` к типу данных массива. Подробнее о типах данных в ClickHouse читайте в разделе "[Типы данных](../../data_types/index.md#data_types)". Может быть равно `NULL`. Функция добавит элемент `NULL` в массив, а тип элементов массива преобразует в `Nullable`. **Пример** -``` sql +```sql SELECT arrayPushBack(['b'], 'a') AS res ``` + ``` ┌─res───────┐ │ ['a','b'] │ @@ -314,12 +401,12 @@ SELECT arrayPushBack(['b'], 'a') AS res arrayResize(array, size[, extender]) ``` -**Параметры:** +**Параметры** - `array` — массив. - `size` — необходимая длина массива. - Если `size` меньше изначального размера массива, то массив обрезается справа. - - Если `size` больше изначального размера массива, массив дополняется справа значениями `extender` или значениями по умолчанию для типа данных элементов массива. + - Если `size` больше изначального размера массива, массив дополняется справа значениями `extender` или значениями по умолчанию для типа данных элементов массива. - `extender` — значение для дополнения массива. Может быть `NULL`. **Возвращаемое значение:** @@ -335,6 +422,7 @@ SELECT arrayResize([1], 3) │ [1,0,0] │ └─────────────────────┘ ``` + ``` SELECT arrayResize([1], 3, NULL) @@ -351,7 +439,7 @@ SELECT arrayResize([1], 3, NULL) arraySlice(array, offset[, length]) ``` -**Аргументы** +**Параметры** - `array` - Массив данных. - `offset` - Отступ от края массива. Положительное значение - отступ слева, отрицательное значение - отступ справа. Отсчет элементов массива начинается с 1. @@ -359,9 +447,10 @@ arraySlice(array, offset[, length]) **Пример** -``` sql +```sql SELECT arraySlice([1, 2, NULL, 4, 5], 2, 3) AS res ``` + ``` ┌─res────────┐ │ [2,NULL,4] │ @@ -371,12 +460,14 @@ SELECT arraySlice([1, 2, NULL, 4, 5], 2, 3) AS res Элементы массива равные `NULL` обрабатываются как обычные значения. ## arrayUniq(arr, ...) + Если передан один аргумент, считает количество разных элементов в массиве. Если передано несколько аргументов, считает количество разных кортежей из элементов на соответствующих позициях в нескольких массивах. Если необходимо получить список уникальных элементов массива, можно воспользоваться arrayReduce('groupUniqArray', arr). ## arrayJoin(arr) + Особенная функция. Смотрите раздел ["Функция arrayJoin"](array_join.md#functions_arrayjoin). [Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/array_functions/) From 5678d1ed98d0d2c6c438183216cb36dd34b4bef8 Mon Sep 17 00:00:00 2001 From: proller Date: Sat, 10 Nov 2018 23:09:07 +0300 Subject: [PATCH 19/46] Build fixes (#3545) * StorageSystemContributors clean * Fix * ARM fixes * Fix arm compile * fix * Fix macos? * Fix includes * fix * fix * Try fix apple build part 1 * Fix identation * Fix static libc++ in clang * fix arm build * better * fix * fix * better check-include --- CMakeLists.txt | 5 ++-- cmake/find_rdkafka.cmake | 4 ++- cmake/test_compiler.cmake | 4 +-- contrib/CMakeLists.txt | 4 ++- dbms/CMakeLists.txt | 2 -- dbms/programs/clang/Compiler-7.0.1 | 1 + dbms/src/Common/HashTable/Hash.h | 6 ++--- dbms/src/Common/XDBCBridgeHelper.h | 1 + dbms/src/Functions/DateTimeTransforms.h | 1 + dbms/src/Functions/EmptyImpl.h | 1 + .../FunctionDateOrDateTimeToSomething.h | 3 +-- dbms/src/Interpreters/ActionsVisitor.h | 1 - .../Interpreters/ArrayJoinedColumnsVisitor.h | 1 + .../Interpreters/DatabaseAndTableWithAlias.h | 1 + .../ExecuteScalarSubqueriesVisitor.h | 9 +++---- dbms/src/Interpreters/ExternalTablesVisitor.h | 5 ++++ .../Interpreters/GlobalSubqueriesVisitor.h | 16 ++++++++++++ dbms/src/Interpreters/QueryAliasesVisitor.h | 1 + .../RequiredSourceColumnsVisitor.h | 10 ++++++++ dbms/src/Parsers/ASTEnumElement.h | 2 ++ debian/.pbuilderrc | 10 +++++++- libs/libcommon/cmake/find_jemalloc.cmake | 2 +- libs/libcommon/include/common/StringRef.h | 2 +- utils/check-style/CMakeLists.txt | 2 +- utils/check-style/check-include | 13 +++++++--- utils/check-style/check-include-stat | 25 +++++++++++++++++++ 26 files changed, 104 insertions(+), 28 deletions(-) create mode 120000 dbms/programs/clang/Compiler-7.0.1 create mode 100755 utils/check-style/check-include-stat diff --git a/CMakeLists.txt b/CMakeLists.txt index 8c954ddad93..71c258eeaad 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -115,8 +115,8 @@ endif () include (cmake/test_cpu.cmake) -option (ARCHNATIVE "Enable -march=native compiler flag" OFF) -if (ARCHNATIVE) +option (ARCH_NATIVE "Enable -march=native compiler flag" OFF) +if (ARCH_NATIVE) set (COMPILER_FLAGS "${COMPILER_FLAGS} -march=native") endif () @@ -155,6 +155,7 @@ if (MAKE_STATIC_LIBRARIES AND NOT APPLE AND NOT (COMPILER_CLANG AND OS_FREEBSD)) endif () set(THREADS_PREFER_PTHREAD_FLAG ON) +find_package (Threads) include (cmake/test_compiler.cmake) diff --git a/cmake/find_rdkafka.cmake b/cmake/find_rdkafka.cmake index dc8e9913bc7..a978ff026f7 100644 --- a/cmake/find_rdkafka.cmake +++ b/cmake/find_rdkafka.cmake @@ -1,4 +1,6 @@ -option (ENABLE_RDKAFKA "Enable kafka" ON) +if (NOT ARCH_ARM) + option (ENABLE_RDKAFKA "Enable kafka" ON) +endif () if (ENABLE_RDKAFKA) diff --git a/cmake/test_compiler.cmake b/cmake/test_compiler.cmake index ba69155914a..cba7a04e5e3 100644 --- a/cmake/test_compiler.cmake +++ b/cmake/test_compiler.cmake @@ -14,6 +14,7 @@ if (CMAKE_CXX_COMPILER_ID STREQUAL "Clang") endif () set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG}") + set (CMAKE_REQUIRED_LIBRARIES ${CMAKE_REQUIRED_LIBRARIES} Threads::Threads) check_cxx_source_compiles(" #include @@ -25,8 +26,6 @@ if (CMAKE_CXX_COMPILER_ID STREQUAL "Clang") else () - cmake_push_check_state () - set (TEST_FLAG "-no-pie") set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG}") @@ -40,7 +39,6 @@ else () set (FLAG_NO_PIE ${TEST_FLAG}) endif () - endif () cmake_pop_check_state () diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 1c653f86808..2ade502966a 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -55,7 +55,7 @@ endif () if (USE_INTERNAL_ZLIB_LIBRARY) set (ZLIB_COMPAT 1 CACHE INTERNAL "") # also enables WITH_GZFILEOP - set (WITH_NATIVE_INSTRUCTIONS ${ARCHNATIVE} CACHE INTERNAL "") + set (WITH_NATIVE_INSTRUCTIONS ${ARCH_NATIVE} CACHE INTERNAL "") if (OS_FREEBSD OR ARCH_I386) set (WITH_OPTIM 0 CACHE INTERNAL "") # Bug in assembler endif () @@ -164,6 +164,8 @@ if (USE_INTERNAL_LLVM_LIBRARY) if (APPLE) set (LINKER_SUPPORTS_COLOR_DIAGNOSTICS 0 CACHE INTERNAL "") endif () + set (LLVM_ENABLE_EH 1 CACHE INTERNAL "") + set (LLVM_ENABLE_RTTI 1 CACHE INTERNAL "") add_subdirectory (llvm/llvm) endif () diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 60dfa5b8cf2..dd8437d1e52 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -48,8 +48,6 @@ if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") add_definitions ("-fno-tree-loop-distribute-patterns") endif () -find_package (Threads) - add_subdirectory (src) set(dbms_headers) diff --git a/dbms/programs/clang/Compiler-7.0.1 b/dbms/programs/clang/Compiler-7.0.1 new file mode 120000 index 00000000000..eeeb5bbc2c0 --- /dev/null +++ b/dbms/programs/clang/Compiler-7.0.1 @@ -0,0 +1 @@ +Compiler-7.0.0 \ No newline at end of file diff --git a/dbms/src/Common/HashTable/Hash.h b/dbms/src/Common/HashTable/Hash.h index 7e414dbe1af..cc3144e94f6 100644 --- a/dbms/src/Common/HashTable/Hash.h +++ b/dbms/src/Common/HashTable/Hash.h @@ -39,7 +39,7 @@ inline DB::UInt64 intHash64(DB::UInt64 x) #include #endif -#if __aarch64__ +#if __aarch64__ && __ARM_FEATURE_CRC32 #include #include #endif @@ -48,8 +48,8 @@ inline DB::UInt64 intHashCRC32(DB::UInt64 x) { #if __SSE4_2__ return _mm_crc32_u64(-1ULL, x); -#elif __aarch64__ - return __crc32cd(-1ULL, x); +#elif __aarch64__ && __ARM_FEATURE_CRC32 + return __crc32cd(-1U, x); #else /// On other platforms we do not have CRC32. NOTE This can be confusing. return intHash64(x); diff --git a/dbms/src/Common/XDBCBridgeHelper.h b/dbms/src/Common/XDBCBridgeHelper.h index 8cb025df7ab..efdf7b401dd 100644 --- a/dbms/src/Common/XDBCBridgeHelper.h +++ b/dbms/src/Common/XDBCBridgeHelper.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include diff --git a/dbms/src/Functions/DateTimeTransforms.h b/dbms/src/Functions/DateTimeTransforms.h index 9be0597698e..59d69d34abb 100644 --- a/dbms/src/Functions/DateTimeTransforms.h +++ b/dbms/src/Functions/DateTimeTransforms.h @@ -1,3 +1,4 @@ +#pragma once #include #include #include diff --git a/dbms/src/Functions/EmptyImpl.h b/dbms/src/Functions/EmptyImpl.h index f42e1331bd4..e66edf5f4ba 100644 --- a/dbms/src/Functions/EmptyImpl.h +++ b/dbms/src/Functions/EmptyImpl.h @@ -1,3 +1,4 @@ +#pragma once #include #include #include diff --git a/dbms/src/Functions/FunctionDateOrDateTimeToSomething.h b/dbms/src/Functions/FunctionDateOrDateTimeToSomething.h index b5345f197ee..19baf8569a2 100644 --- a/dbms/src/Functions/FunctionDateOrDateTimeToSomething.h +++ b/dbms/src/Functions/FunctionDateOrDateTimeToSomething.h @@ -1,9 +1,8 @@ #include #include - #include #include - +#include #include diff --git a/dbms/src/Interpreters/ActionsVisitor.h b/dbms/src/Interpreters/ActionsVisitor.h index b4ca04362c1..0344934455d 100644 --- a/dbms/src/Interpreters/ActionsVisitor.h +++ b/dbms/src/Interpreters/ActionsVisitor.h @@ -1,7 +1,6 @@ #pragma once #include - #include diff --git a/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h b/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h index 73e510a9b37..96e37cd6a42 100644 --- a/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h +++ b/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h @@ -4,6 +4,7 @@ #include #include +#include #include #include diff --git a/dbms/src/Interpreters/DatabaseAndTableWithAlias.h b/dbms/src/Interpreters/DatabaseAndTableWithAlias.h index 2444f037d8e..ad7c1f2f8d4 100644 --- a/dbms/src/Interpreters/DatabaseAndTableWithAlias.h +++ b/dbms/src/Interpreters/DatabaseAndTableWithAlias.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { diff --git a/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.h b/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.h index 7c3fc560665..d0e5c520a69 100644 --- a/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.h +++ b/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.h @@ -1,16 +1,15 @@ #pragma once #include +#include #include +#include +#include +#include namespace DB { -class Context; -class ASTSubquery; -class ASTFunction; -struct ASTTableExpression; - /// Visitors consist of functions with unified interface 'void visit(Casted & x, ASTPtr & y)', there x is y, successfully casted to Casted. /// Both types and fuction could have const specifiers. The second argument is used by visitor to replaces AST node (y) if needed. diff --git a/dbms/src/Interpreters/ExternalTablesVisitor.h b/dbms/src/Interpreters/ExternalTablesVisitor.h index d8abffd3d90..95b109987e0 100644 --- a/dbms/src/Interpreters/ExternalTablesVisitor.h +++ b/dbms/src/Interpreters/ExternalTablesVisitor.h @@ -1,5 +1,10 @@ #pragma once +#include +#include +#include +#include + namespace DB { diff --git a/dbms/src/Interpreters/GlobalSubqueriesVisitor.h b/dbms/src/Interpreters/GlobalSubqueriesVisitor.h index ce84e3b81fa..ec616b817b9 100644 --- a/dbms/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/dbms/src/Interpreters/GlobalSubqueriesVisitor.h @@ -1,5 +1,21 @@ #pragma once +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + namespace DB { diff --git a/dbms/src/Interpreters/QueryAliasesVisitor.h b/dbms/src/Interpreters/QueryAliasesVisitor.h index 87e5af688da..18d507821d2 100644 --- a/dbms/src/Interpreters/QueryAliasesVisitor.h +++ b/dbms/src/Interpreters/QueryAliasesVisitor.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { diff --git a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h index de50a5247c3..ada053a3657 100644 --- a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h +++ b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h @@ -1,11 +1,21 @@ #pragma once +#include +#include +#include +#include +#include +#include +#include +#include + namespace DB { namespace ErrorCodes { extern const int TYPE_MISMATCH; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } /// Visitors consist of functions with unified interface 'void visit(Casted & x, ASTPtr & y)', there x is y, successfully casted to Casted. diff --git a/dbms/src/Parsers/ASTEnumElement.h b/dbms/src/Parsers/ASTEnumElement.h index cd07db53d04..8e0a2ec1648 100644 --- a/dbms/src/Parsers/ASTEnumElement.h +++ b/dbms/src/Parsers/ASTEnumElement.h @@ -1,6 +1,8 @@ #pragma once #include +#include +#include namespace DB diff --git a/debian/.pbuilderrc b/debian/.pbuilderrc index cfdebbfc979..bc12479eb17 100644 --- a/debian/.pbuilderrc +++ b/debian/.pbuilderrc @@ -161,7 +161,15 @@ esac if [ "$ARCH" != arm64 ]; then case "$DIST" in - "cosmic" | "bionic" | "experimental" | "unstable" | "testing") + "unstable") + EXTRAPACKAGES+=" liblld-8-dev libclang-8-dev llvm-8-dev liblld-8 " + export CMAKE_FLAGS="-DLLVM_VERSION_POSTFIX=-8 $CMAKE_FLAGS" + ;; + "cosmic" | "experimental" | "testing") + EXTRAPACKAGES+=" liblld-7-dev libclang-7-dev llvm-7-dev liblld-7 " + export CMAKE_FLAGS="-DLLVM_VERSION_POSTFIX=-7 $CMAKE_FLAGS" + ;; + "bionic") EXTRAPACKAGES+=" liblld-6.0-dev libclang-6.0-dev liblld-6.0 " export CMAKE_FLAGS="-DLLVM_VERSION_POSTFIX=-6.0 $CMAKE_FLAGS" ;; diff --git a/libs/libcommon/cmake/find_jemalloc.cmake b/libs/libcommon/cmake/find_jemalloc.cmake index a8ef4289f5f..35e2a13358b 100644 --- a/libs/libcommon/cmake/find_jemalloc.cmake +++ b/libs/libcommon/cmake/find_jemalloc.cmake @@ -1,4 +1,4 @@ -if (OS_LINUX AND NOT SANITIZE) +if (OS_LINUX AND NOT SANITIZE AND NOT ARCH_ARM) set(ENABLE_JEMALLOC_DEFAULT 1) else () set(ENABLE_JEMALLOC_DEFAULT 0) diff --git a/libs/libcommon/include/common/StringRef.h b/libs/libcommon/include/common/StringRef.h index 21c23012024..05222902324 100644 --- a/libs/libcommon/include/common/StringRef.h +++ b/libs/libcommon/include/common/StringRef.h @@ -271,7 +271,7 @@ struct CRC32Hash size_t operator() (StringRef /* x */) const { throw std::logic_error{"Not implemented CRC32Hash without SSE"}; - }; + } }; struct StringRefHash : StringRefHash64 {}; diff --git a/utils/check-style/CMakeLists.txt b/utils/check-style/CMakeLists.txt index 47c9e1267b7..24b315019fe 100644 --- a/utils/check-style/CMakeLists.txt +++ b/utils/check-style/CMakeLists.txt @@ -1,2 +1,2 @@ add_test(NAME check-style COMMAND bash -c "${CMAKE_CURRENT_SOURCE_DIR}/check-style") -add_test(NAME check-include COMMAND bash -c "env RUN_DIR=${CMAKE_CURRENT_SOURCE_DIR} ROOT_DIR=${ClickHouse_SOURCE_DIR} BUILD_DIR=${ClickHouse_BINARY_DIR} ${CMAKE_CURRENT_SOURCE_DIR}/check-include") +add_test(NAME check-include COMMAND sh -c "env RUN_DIR=${CMAKE_CURRENT_SOURCE_DIR} ROOT_DIR=${ClickHouse_SOURCE_DIR} BUILD_DIR=${ClickHouse_BINARY_DIR} CXX=${CMAKE_CXX_COMPILER} ${CMAKE_CURRENT_SOURCE_DIR}/check-include-stat") diff --git a/utils/check-style/check-include b/utils/check-style/check-include index d4056e74428..cdc5a82d7ed 100755 --- a/utils/check-style/check-include +++ b/utils/check-style/check-include @@ -6,7 +6,9 @@ set -e # Finds missing #include <...> # prints compile time, number of includes, use with sort: ./check_include.sh 2>&1 | sort -rk3 # use with clang: CXX=`which clang++-7 clang++-7.0 clang++-6.0 clang++-5.0 | head -n1` ./check_include.sh -RUN_DIR=${RUN_DIR=`pwd`} +CUR_DIR=`dirname $0` +CUR_DIR=`readlink -f $CUR_DIR` +CUR_DIR="${CUR_DIR}/" BUILD_DIR=${BUILD_DIR:=./build} inc="-I. \ -I./contrib/libdivide \ @@ -37,19 +39,22 @@ inc="-I. \ -I./contrib/libbtrie/include \ -I./contrib/libpcg-random/include \ -I./contrib/capnproto/c++/src \ +-I./contrib/unixodbc/include \ +-I./contrib/unixodbc-cmake/linux_x86_64 \ -I./libs/libmysqlxx/include \ -I./libs/libcommon/include \ -I${BUILD_DIR}/libs/libcommon/include \ -I./libs/libpocoext/include \ -I./libs/libzkutil/include \ -I./libs/libdaemon/include \ --I./libs/libconsistent-hashing \ +-I./libs/consistent-hashing \ +-I./libs/consistent-hashing-sumbur \ -I./dbms/src \ -I${BUILD_DIR}/dbms/src" if [ -z $1 ]; then - cd ${ROOT_DIR=../..} - find dbms libs utils \( -name *.h -and -not -name *.inl.h \) -exec sh $RUN_DIR/`basename $0` {} \; ; + cd ${ROOT_DIR=${CUR_DIR}../..} + find dbms libs utils \( -name *.h -and -not -name *.inl.h \) -print0 | xargs -0 -n1 sh $CUR_DIR`basename $0` else echo -n "$1 " echo -n `grep "#include" $1| wc -l` " " diff --git a/utils/check-style/check-include-stat b/utils/check-style/check-include-stat new file mode 100755 index 00000000000..9dc9b7e7e9a --- /dev/null +++ b/utils/check-style/check-include-stat @@ -0,0 +1,25 @@ +#!/bin/sh +set -e + +CUR_DIR=`dirname $0` +CUR_DIR=`readlink -f $CUR_DIR` +CUR_DIR="${CUR_DIR}/" + +RESULT_FILE=${RESULT_FILE:=${CUR_DIR}check-include.log} +finish() { + echo include check failed: + cat $RESULT_FILE +} +trap finish 0 1 3 6 15 + +sh ${CUR_DIR}check-include > $RESULT_FILE 2>&1 + +echo Results: +echo Top by memory: +cat $RESULT_FILE | sort -rk4 | head -n20 + +echo Top by time: +cat $RESULT_FILE | sort -rk3 | head -n20 + +echo Top by includes: +cat $RESULT_FILE | sort -rk2 | head -n20 From 1b7b8e48bb3109ac0352f2bfeec58b42e76d4992 Mon Sep 17 00:00:00 2001 From: Alexander Kuranoff Date: Sun, 11 Nov 2018 13:06:18 +0300 Subject: [PATCH 20/46] Update formats.md (#3560) Fixed word --- docs/ru/interfaces/formats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 8d31b6707d7..1e8c8ac45aa 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -78,7 +78,7 @@ SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORD В качестве исключения, поддерживается также парсинг даты-с-временем в формате unix timestamp, если он состоит ровно из 10 десятичных цифр. Результат не зависит от часового пояса. Различение форматов YYYY-MM-DD hh:mm:ss и NNNNNNNNNN делается автоматически. -Строки выводятся с экранированием спец-символов с помощью обратного слеша. При выводе, используются следующие escape-последовательности: `\b`, `\f`, `\r`, `\n`, `\t`, `\0`, `\'`, `\\`. При парсинге, также поддерживаются последовательности `\a`, `\v`, а также `\xHH` (hex escape-последовательности) и любые последовательности вида `\c`, где `c` - любой символ - такие последовательности преобразуется в `c`. Таким образом, при чтении поддерживаются форматы, где перевод строки может быть записан как `\n` и как `\` и перевод строки. Например, строка `Hello world`, где между словами вместо пробела стоит перевод строки, может быть считана в любом из следующих вариантов: +Строки выводятся с экранированием спецсимволов с помощью обратного слеша. При выводе, используются следующие escape-последовательности: `\b`, `\f`, `\r`, `\n`, `\t`, `\0`, `\'`, `\\`. При парсинге, также поддерживаются последовательности `\a`, `\v`, а также `\xHH` (hex escape-последовательности) и любые последовательности вида `\c`, где `c` - любой символ - такие последовательности преобразуется в `c`. Таким образом, при чтении поддерживаются форматы, где перевод строки может быть записан как `\n` и как `\` и перевод строки. Например, строка `Hello world`, где между словами вместо пробела стоит перевод строки, может быть считана в любом из следующих вариантов: ``` Hello\nworld From 8098900efa1a76b582fc42fc6221eddeda332424 Mon Sep 17 00:00:00 2001 From: Alexander Kuranoff Date: Sun, 11 Nov 2018 13:06:50 +0300 Subject: [PATCH 21/46] Update http_interface.md (#3559) Fixed word --- docs/ru/interfaces/http_interface.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/interfaces/http_interface.md b/docs/ru/interfaces/http_interface.md index 6c8c0de1c11..2c3ed8e23df 100644 --- a/docs/ru/interfaces/http_interface.md +++ b/docs/ru/interfaces/http_interface.md @@ -172,7 +172,7 @@ echo 'SELECT 1' | curl 'http://localhost:8123/?user=user&password=password' -d @ ``` Если имя пользователя не указано, то используется имя пользователя default. Если пароль не указан, то используется пустой пароль. -Также в параметрах URL вы можете указать любые настроки, которые будут использованы для обработки одного запроса, или целые профили настроек. Пример: +Также в параметрах URL вы можете указать любые настройки, которые будут использованы для обработки одного запроса, или целые профили настроек. Пример: http://localhost:8123/?profile=web&max_rows_to_read=1000000000&query=SELECT+1 Подробнее см. раздел "Настройки". From 92bc884ccb6fd07a2c20a68121b41b83518f01ff Mon Sep 17 00:00:00 2001 From: Alexander Kuranoff Date: Sun, 11 Nov 2018 13:07:10 +0300 Subject: [PATCH 22/46] Update nyc_taxi.md (#3558) Delete the dot in a sentese --- docs/ru/getting_started/example_datasets/nyc_taxi.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting_started/example_datasets/nyc_taxi.md b/docs/ru/getting_started/example_datasets/nyc_taxi.md index d0402b9e2f4..3cfe7e439c5 100644 --- a/docs/ru/getting_started/example_datasets/nyc_taxi.md +++ b/docs/ru/getting_started/example_datasets/nyc_taxi.md @@ -356,7 +356,7 @@ Q2: 0.043 sec. Q3: 0.051 sec. Q4: 0.072 sec. -В этом случае, время выполнения запросов определяется в первую очередь сетевыми задержками.. +В этом случае, время выполнения запросов определяется в первую очередь сетевыми задержками. Мы выполняли запросы с помощью клиента, расположенного в датацентре Яндекса в Мянтсяля (Финляндия), на кластер в России, что добавляет порядка 20 мс задержки. ## Резюме From f76bfb00a3f1a10a575d93af4f4b0cf1e63da4ce Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 12 Nov 2018 00:07:28 +0800 Subject: [PATCH 23/46] Avoid materialization for dictGetTOrDefault --- .../Functions/FunctionsExternalDictionaries.h | 35 ++++++++++++------- 1 file changed, 22 insertions(+), 13 deletions(-) diff --git a/dbms/src/Functions/FunctionsExternalDictionaries.h b/dbms/src/Functions/FunctionsExternalDictionaries.h index 62f35ba8e9f..73d4375f88e 100644 --- a/dbms/src/Functions/FunctionsExternalDictionaries.h +++ b/dbms/src/Functions/FunctionsExternalDictionaries.h @@ -538,11 +538,17 @@ private: if (const auto default_col = checkAndGetColumn(default_col_untyped)) { /// const ids, vector defaults - /// @todo avoid materialization - const PaddedPODArray ids(id_col->size(), id_col->getValue()); - auto out = ColumnString::create(); - dictionary->getString(attr_name, ids, default_col, out.get()); - block.getByPosition(result).column = std::move(out); + const PaddedPODArray ids(1, id_col->getValue()); + PaddedPODArray flags(1); + dictionary->has(ids, flags); + if (flags.front()) + { + auto out = ColumnString::create(); + dictionary->getString(attr_name, ids, String(), out.get()); + block.getByPosition(result).column = DataTypeString().createColumnConst(id_col->size(), out->getDataAt(0).toString()); + } + else + block.getByPosition(result).column = block.getByPosition(arguments[3]).column; // reuse the default column } else if (const auto default_col = checkAndGetColumnConstStringOrFixedString(default_col_untyped)) { @@ -1033,14 +1039,17 @@ private: if (const auto default_col = checkAndGetColumn>(default_col_untyped)) { /// const ids, vector defaults - /// @todo avoid materialization - const PaddedPODArray ids(id_col->size(), id_col->getValue()); - - auto out = ColumnVector::create(id_col->size()); - auto & data = out->getData(); - const auto & defs = default_col->getData(); - DictGetTraits::getOrDefault(dictionary, attr_name, ids, defs, data); - block.getByPosition(result).column = std::move(out); + const PaddedPODArray ids(1, id_col->getValue()); + PaddedPODArray flags(1); + dictionary->has(ids, flags); + if (flags.front()) + { + PaddedPODArray data(1); + DictGetTraits::getOrDefault(dictionary, attr_name, ids, Type(), data); + block.getByPosition(result).column = DataTypeNumber().createColumnConst(id_col->size(), toField(data.front())); + } + else + block.getByPosition(result).column = block.getByPosition(arguments[3]).column; // reuse the default column } else if (const auto default_col = checkAndGetColumnConst>(default_col_untyped)) { From 6815b25d185db2f529efef39492481aa8025926c Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 12 Nov 2018 03:29:52 +0800 Subject: [PATCH 24/46] replace std::tuple with std::variant --- dbms/src/Common/tests/arena_with_free_lists.cpp | 5 +++-- dbms/src/Dictionaries/CacheDictionary.h | 6 +++--- dbms/src/Dictionaries/ComplexKeyCacheDictionary.h | 6 +++--- dbms/src/Dictionaries/ComplexKeyHashedDictionary.h | 6 +++--- dbms/src/Dictionaries/FlatDictionary.h | 6 +++--- dbms/src/Dictionaries/HashedDictionary.h | 6 +++--- dbms/src/Dictionaries/RangeHashedDictionary.h | 6 +++--- dbms/src/Dictionaries/TrieDictionary.h | 6 +++--- 8 files changed, 24 insertions(+), 23 deletions(-) diff --git a/dbms/src/Common/tests/arena_with_free_lists.cpp b/dbms/src/Common/tests/arena_with_free_lists.cpp index df1b47bbe02..0a16eff6c17 100644 --- a/dbms/src/Common/tests/arena_with_free_lists.cpp +++ b/dbms/src/Common/tests/arena_with_free_lists.cpp @@ -8,6 +8,7 @@ #include #endif +#include #include #include #include @@ -144,12 +145,12 @@ struct Dictionary struct Attribute final { AttributeUnderlyingType type; - std::tuple< + std::variant< UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64, String> null_values; - std::tuple< + std::variant< ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, diff --git a/dbms/src/Dictionaries/CacheDictionary.h b/dbms/src/Dictionaries/CacheDictionary.h index 373e3e70643..453e38246ec 100644 --- a/dbms/src/Dictionaries/CacheDictionary.h +++ b/dbms/src/Dictionaries/CacheDictionary.h @@ -13,7 +13,7 @@ #include #include #include -#include +#include #include #include @@ -182,14 +182,14 @@ private: struct Attribute final { AttributeUnderlyingType type; - std::tuple< + std::variant< UInt8, UInt16, UInt32, UInt64, UInt128, Int8, Int16, Int32, Int64, Decimal32, Decimal64, Decimal128, Float32, Float64, String> null_values; - std::tuple< + std::variant< ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h index bcfbb65411e..70b1302a86d 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include #include @@ -260,10 +260,10 @@ private: struct Attribute final { AttributeUnderlyingType type; - std::tuple null_values; - std::tuple, + std::variant, ContainerPtrType, ContainerPtrType, ContainerPtrType, diff --git a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h index 40a2c7174c8..a0392717d0c 100644 --- a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h +++ b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h @@ -11,7 +11,7 @@ #include #include #include -#include +#include namespace DB @@ -152,14 +152,14 @@ private: struct Attribute final { AttributeUnderlyingType type; - std::tuple< + std::variant< UInt8, UInt16, UInt32, UInt64, UInt128, Int8, Int16, Int32, Int64, Decimal32, Decimal64, Decimal128, Float32, Float64, String> null_values; - std::tuple< + std::variant< ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, diff --git a/dbms/src/Dictionaries/FlatDictionary.h b/dbms/src/Dictionaries/FlatDictionary.h index 96adf108745..7fa36fc9925 100644 --- a/dbms/src/Dictionaries/FlatDictionary.h +++ b/dbms/src/Dictionaries/FlatDictionary.h @@ -10,7 +10,7 @@ #include #include #include -#include +#include namespace DB @@ -152,14 +152,14 @@ private: struct Attribute final { AttributeUnderlyingType type; - std::tuple< + std::variant< UInt8, UInt16, UInt32, UInt64, UInt128, Int8, Int16, Int32, Int64, Decimal32, Decimal64, Decimal128, Float32, Float64, StringRef> null_values; - std::tuple< + std::variant< ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, diff --git a/dbms/src/Dictionaries/HashedDictionary.h b/dbms/src/Dictionaries/HashedDictionary.h index 8db0779d370..e54c01e3598 100644 --- a/dbms/src/Dictionaries/HashedDictionary.h +++ b/dbms/src/Dictionaries/HashedDictionary.h @@ -9,7 +9,7 @@ #include #include #include -#include +#include namespace DB @@ -150,14 +150,14 @@ private: struct Attribute final { AttributeUnderlyingType type; - std::tuple< + std::variant< UInt8, UInt16, UInt32, UInt64, UInt128, Int8, Int16, Int32, Int64, Decimal32, Decimal64, Decimal128, Float32, Float64, String> null_values; - std::tuple< + std::variant< CollectionPtrType, CollectionPtrType, CollectionPtrType, CollectionPtrType, CollectionPtrType, CollectionPtrType, CollectionPtrType, CollectionPtrType, CollectionPtrType, diff --git a/dbms/src/Dictionaries/RangeHashedDictionary.h b/dbms/src/Dictionaries/RangeHashedDictionary.h index 8ff7bf98c41..5e8adc4ceae 100644 --- a/dbms/src/Dictionaries/RangeHashedDictionary.h +++ b/dbms/src/Dictionaries/RangeHashedDictionary.h @@ -9,7 +9,7 @@ #include #include -#include +#include namespace DB @@ -118,13 +118,13 @@ private: { public: AttributeUnderlyingType type; - std::tuple null_values; - std::tuple, Ptr, Ptr, Ptr, + std::variant, Ptr, Ptr, Ptr, Ptr, Ptr, Ptr, Ptr, Ptr, Ptr, Ptr, Ptr, diff --git a/dbms/src/Dictionaries/TrieDictionary.h b/dbms/src/Dictionaries/TrieDictionary.h index 7b7173923fe..948e7329221 100644 --- a/dbms/src/Dictionaries/TrieDictionary.h +++ b/dbms/src/Dictionaries/TrieDictionary.h @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include struct btrie_s; @@ -155,14 +155,14 @@ private: struct Attribute final { AttributeUnderlyingType type; - std::tuple< + std::variant< UInt8, UInt16, UInt32, UInt64, UInt128, Int8, Int16, Int32, Int64, Decimal32, Decimal64, Decimal128, Float32, Float64, String> null_values; - std::tuple< + std::variant< ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, From 3429274de14a95ade6df28e940b1d78c56c346b3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 12 Nov 2018 11:40:23 +0300 Subject: [PATCH 25/46] Fix perf in IColumnUnique::size func. #3256 --- dbms/src/Columns/IColumnUnique.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Columns/IColumnUnique.h b/dbms/src/Columns/IColumnUnique.h index 62aef93eef1..7e5f3976dce 100644 --- a/dbms/src/Columns/IColumnUnique.h +++ b/dbms/src/Columns/IColumnUnique.h @@ -23,7 +23,7 @@ public: /// Uses thread-safe cache. virtual const UInt64 * tryGetSavedHash() const = 0; - size_t size() const override { return getNestedColumn()->size(); } + size_t size() const override { return getNestedNotNullableColumn()->size(); } /// Appends new value at the end of column (column's size is increased by 1). /// Is used to transform raw strings to Blocks (for example, inside input format parsers) From 03c35f2aa4cf425c6778db8d8c771c343d1bd18f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Nov 2018 18:40:33 +0300 Subject: [PATCH 26/46] Remove Analyzers because it was failed refactoring attempt [#CLICKHOUSE-2] --- dbms/src/Analyzers/AnalyzeArrayJoins.h | 46 -- dbms/src/Analyzers/AnalyzeColumns.cpp | 455 --------------- dbms/src/Analyzers/AnalyzeColumns.h | 48 -- dbms/src/Analyzers/AnalyzeLambdas.cpp | 201 ------- dbms/src/Analyzers/AnalyzeLambdas.h | 35 -- dbms/src/Analyzers/AnalyzeResultOfQuery.cpp | 70 --- dbms/src/Analyzers/AnalyzeResultOfQuery.h | 32 -- dbms/src/Analyzers/CMakeLists.txt | 3 - dbms/src/Analyzers/CollectAliases.cpp | 109 ---- dbms/src/Analyzers/CollectAliases.h | 54 -- dbms/src/Analyzers/CollectTables.cpp | 186 ------ dbms/src/Analyzers/CollectTables.h | 52 -- dbms/src/Analyzers/ExecuteTableFunctions.cpp | 80 --- dbms/src/Analyzers/ExecuteTableFunctions.h | 35 -- .../Analyzers/OptimizeGroupOrderLimitBy.cpp | 264 --------- .../src/Analyzers/OptimizeGroupOrderLimitBy.h | 36 -- .../TranslatePositionalArguments.cpp | 86 --- .../Analyzers/TranslatePositionalArguments.h | 21 - .../Analyzers/TypeAndConstantInference.cpp | 534 ------------------ dbms/src/Analyzers/TypeAndConstantInference.h | 61 -- dbms/src/Analyzers/tests/CMakeLists.txt | 24 - dbms/src/Analyzers/tests/analyze_columns.cpp | 71 --- .../Analyzers/tests/analyze_columns.reference | 18 - dbms/src/Analyzers/tests/analyze_columns.sh | 7 - dbms/src/Analyzers/tests/analyze_lambdas.cpp | 44 -- .../Analyzers/tests/analyze_lambdas.reference | 4 - dbms/src/Analyzers/tests/analyze_lambdas.sh | 3 - .../tests/analyze_result_of_query.cpp | 53 -- .../tests/analyze_result_of_query.reference | 1 - .../tests/analyze_result_of_query.sh | 3 - dbms/src/Analyzers/tests/collect_aliases.cpp | 36 -- .../Analyzers/tests/collect_aliases.reference | 8 - dbms/src/Analyzers/tests/collect_aliases.sh | 3 - dbms/src/Analyzers/tests/collect_tables.cpp | 56 -- .../tests/optimize_group_order_limit_by.cpp | 80 --- .../optimize_group_order_limit_by.reference | 19 - .../tests/optimize_group_order_limit_by.sh | 5 - .../tests/translate_positional_arguments.cpp | 37 -- .../translate_positional_arguments.reference | 8 - .../tests/translate_positional_arguments.sh | 3 - .../tests/type_and_constant_inference.cpp | 78 --- .../type_and_constant_inference.reference | 34 -- .../tests/type_and_constant_inference.sh | 26 - dbms/src/CMakeLists.txt | 1 - dbms/src/Interpreters/ExpressionAnalyzer.h | 2 +- .../tests/expression_analyzer.cpp | 29 +- 46 files changed, 2 insertions(+), 3059 deletions(-) delete mode 100644 dbms/src/Analyzers/AnalyzeArrayJoins.h delete mode 100644 dbms/src/Analyzers/AnalyzeColumns.cpp delete mode 100644 dbms/src/Analyzers/AnalyzeColumns.h delete mode 100644 dbms/src/Analyzers/AnalyzeLambdas.cpp delete mode 100644 dbms/src/Analyzers/AnalyzeLambdas.h delete mode 100644 dbms/src/Analyzers/AnalyzeResultOfQuery.cpp delete mode 100644 dbms/src/Analyzers/AnalyzeResultOfQuery.h delete mode 100644 dbms/src/Analyzers/CMakeLists.txt delete mode 100644 dbms/src/Analyzers/CollectAliases.cpp delete mode 100644 dbms/src/Analyzers/CollectAliases.h delete mode 100644 dbms/src/Analyzers/CollectTables.cpp delete mode 100644 dbms/src/Analyzers/CollectTables.h delete mode 100644 dbms/src/Analyzers/ExecuteTableFunctions.cpp delete mode 100644 dbms/src/Analyzers/ExecuteTableFunctions.h delete mode 100644 dbms/src/Analyzers/OptimizeGroupOrderLimitBy.cpp delete mode 100644 dbms/src/Analyzers/OptimizeGroupOrderLimitBy.h delete mode 100644 dbms/src/Analyzers/TranslatePositionalArguments.cpp delete mode 100644 dbms/src/Analyzers/TranslatePositionalArguments.h delete mode 100644 dbms/src/Analyzers/TypeAndConstantInference.cpp delete mode 100644 dbms/src/Analyzers/TypeAndConstantInference.h delete mode 100644 dbms/src/Analyzers/tests/CMakeLists.txt delete mode 100644 dbms/src/Analyzers/tests/analyze_columns.cpp delete mode 100644 dbms/src/Analyzers/tests/analyze_columns.reference delete mode 100755 dbms/src/Analyzers/tests/analyze_columns.sh delete mode 100644 dbms/src/Analyzers/tests/analyze_lambdas.cpp delete mode 100644 dbms/src/Analyzers/tests/analyze_lambdas.reference delete mode 100755 dbms/src/Analyzers/tests/analyze_lambdas.sh delete mode 100644 dbms/src/Analyzers/tests/analyze_result_of_query.cpp delete mode 100644 dbms/src/Analyzers/tests/analyze_result_of_query.reference delete mode 100755 dbms/src/Analyzers/tests/analyze_result_of_query.sh delete mode 100644 dbms/src/Analyzers/tests/collect_aliases.cpp delete mode 100644 dbms/src/Analyzers/tests/collect_aliases.reference delete mode 100755 dbms/src/Analyzers/tests/collect_aliases.sh delete mode 100644 dbms/src/Analyzers/tests/collect_tables.cpp delete mode 100644 dbms/src/Analyzers/tests/optimize_group_order_limit_by.cpp delete mode 100644 dbms/src/Analyzers/tests/optimize_group_order_limit_by.reference delete mode 100755 dbms/src/Analyzers/tests/optimize_group_order_limit_by.sh delete mode 100644 dbms/src/Analyzers/tests/translate_positional_arguments.cpp delete mode 100644 dbms/src/Analyzers/tests/translate_positional_arguments.reference delete mode 100755 dbms/src/Analyzers/tests/translate_positional_arguments.sh delete mode 100644 dbms/src/Analyzers/tests/type_and_constant_inference.cpp delete mode 100644 dbms/src/Analyzers/tests/type_and_constant_inference.reference delete mode 100755 dbms/src/Analyzers/tests/type_and_constant_inference.sh diff --git a/dbms/src/Analyzers/AnalyzeArrayJoins.h b/dbms/src/Analyzers/AnalyzeArrayJoins.h deleted file mode 100644 index 4bc9c3fb72b..00000000000 --- a/dbms/src/Analyzers/AnalyzeArrayJoins.h +++ /dev/null @@ -1,46 +0,0 @@ -#pragma once - -#include -#include -#include - - -namespace DB -{ - -class WriteBuffer; -struct CollectAliases; -struct CollectTables; - - -/** For every ARRAY JOIN, collect a map: - * result alias -> source - * - * There could be several variants: - * - * SELECT elem FROM t ARRAY JOIN array AS elem elem -> array - * SELECT n.elem FROM t ARRAY JOIN nested AS n n -> nested - * SELECT array FROM t ARRAY JOIN array array -> array - * SELECT nested.elem FROM t ARRAY JOIN nested nested -> nested - * SELECT elem FROM t ARRAY JOIN [1, 2, 3] AS elem elem -> [1, 2, 3] - * - * Does not analyze arrayJoin functions. - */ -struct AnalyzeArrayJoins -{ - void process(const ASTPtr & ast); - - struct SourceInfo - { - String column_name; - ASTPtr node; - }; - - using ResultToSource = std::unordered_map; - using ArrayJoins = std::vector; - - /// Debug output - void dump(WriteBuffer & out) const; -}; - -} diff --git a/dbms/src/Analyzers/AnalyzeColumns.cpp b/dbms/src/Analyzers/AnalyzeColumns.cpp deleted file mode 100644 index 93b43157664..00000000000 --- a/dbms/src/Analyzers/AnalyzeColumns.cpp +++ /dev/null @@ -1,455 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; - extern const int NOT_IMPLEMENTED; - extern const int AMBIGUOUS_TABLE_NAME; - extern const int AMBIGUOUS_COLUMN_NAME; - extern const int UNKNOWN_TABLE; - extern const int THERE_IS_NO_COLUMN; -} - - -namespace -{ - -/// Find by fully qualified name, like db.table.column -const CollectTables::TableInfo * findTableByDatabaseAndTableName( - const CollectTables & tables, const String & database_name, const String & table_name) -{ - for (const auto & table : tables.tables) - if (table.database_name == database_name && table.table_name == table_name) - return &table; - - return nullptr; -} - - -/** Find by single-qualified name, like table.column or alias.column. - * - * There are primary matches: - * when name is alias like - * SELECT name.column FROM (SELECT 1) AS name - * or name is table in current database like - * SELECT name.column FROM name - * - * And secondary matches: - * when name is name of table in explicitly specified database like - * SELECT name.column FROM db.name - * - * If there is only one primary match - return it. - * If there is many primary matches - ambiguity. - * If there is no primary matches and only one secondary match - return it. - * If there is no primary matches and many secondary matches - ambiguity. - * If there is no any matches - not found. - */ -const CollectTables::TableInfo * findTableByNameOrAlias( - const CollectTables & tables, const String & name) -{ - const CollectTables::TableInfo * primary_match = nullptr; - const CollectTables::TableInfo * secondary_match = nullptr; - - for (const auto & table : tables.tables) - { - if (table.alias == name - || (table.database_name.empty() && table.table_name == name)) - { - if (primary_match) - throw Exception("Table name " + backQuoteIfNeed(name) + " is ambiguous", ErrorCodes::AMBIGUOUS_TABLE_NAME); - primary_match = &table; - } - else if (!primary_match && table.table_name == name) - { - if (secondary_match) - throw Exception("Table name " + backQuoteIfNeed(name) + " is ambiguous", ErrorCodes::AMBIGUOUS_TABLE_NAME); - secondary_match = &table; - } - } - - if (primary_match) - return primary_match; - if (secondary_match) - return secondary_match; - return nullptr; -} - - -/** Find table in case when its name is not specified. Like just - * SELECT column FROM t1, t2 - * Select a table, where specified column exists. - * If more than one such table - ambiguity. - */ -const CollectTables::TableInfo * findTableWithUnqualifiedName(const CollectTables & tables, const String & column_name) -{ - const CollectTables::TableInfo * res = nullptr; - - for (const auto & table : tables.tables) - { - if (table.structure_of_subquery) - { - if (table.structure_of_subquery.has(column_name)) - { - if (res) - throw Exception("Ambiguous column name " + backQuoteIfNeed(column_name), ErrorCodes::AMBIGUOUS_COLUMN_NAME); - res = &table; - break; - } - } - else if (table.storage) - { - if (table.storage->hasColumn(column_name)) - { - if (res) - throw Exception("Ambiguous column name " + backQuoteIfNeed(column_name), ErrorCodes::AMBIGUOUS_COLUMN_NAME); - res = &table; - } - } - else - throw Exception("Logical error: no storage and no structure of subquery is specified for table", ErrorCodes::LOGICAL_ERROR); - } - - return res; -} - - -/// Create maximum-qualified identifier for column in table. -ASTPtr createASTIdentifierForColumnInTable(const String & column, const CollectTables::TableInfo & table) -{ - ASTPtr database_name_identifier_node; - if (!table.database_name.empty()) - database_name_identifier_node = std::make_shared(table.database_name); - - ASTPtr table_name_identifier_node; - String table_name_or_alias; - - if (!table.table_name.empty()) - table_name_or_alias = table.table_name; - else if (table.database_name.empty() && !table.alias.empty()) - table_name_or_alias = table.alias; - - if (!table_name_or_alias.empty()) - table_name_identifier_node = std::make_shared(table_name_or_alias); - - ASTPtr column_identifier_node = std::make_shared(column); - - String compound_name; - if (database_name_identifier_node) - compound_name += table.database_name + "."; - if (table_name_identifier_node) - compound_name += table_name_or_alias + "."; - compound_name += column; - - auto elem = std::make_shared(compound_name); - - if (database_name_identifier_node) - elem->children.emplace_back(std::move(database_name_identifier_node)); - if (table_name_identifier_node) - elem->children.emplace_back(std::move(table_name_identifier_node)); - if (!elem->children.empty()) - elem->children.emplace_back(std::move(column_identifier_node)); - - return elem; -} - - -void createASTsForAllColumnsInTable(const CollectTables::TableInfo & table, ASTs & res) -{ - if (table.storage) - for (const auto & name : table.storage->getColumns().getNamesOfPhysical()) - res.emplace_back(createASTIdentifierForColumnInTable(name, table)); - else - for (size_t i = 0, size = table.structure_of_subquery.columns(); i < size; ++i) - res.emplace_back(createASTIdentifierForColumnInTable(table.structure_of_subquery.getByPosition(i).name, table)); -} - - -ASTs expandUnqualifiedAsterisk(const CollectTables & tables) -{ - ASTs res; - for (const auto & table : tables.tables) - createASTsForAllColumnsInTable(table, res); - return res; -} - - -ASTs expandQualifiedAsterisk( - const IAST & ast, const CollectTables & tables) -{ - if (ast.children.size() != 1) - throw Exception("Logical error: AST node for qualified asterisk has number of children not equal to one", ErrorCodes::LOGICAL_ERROR); - - const ASTIdentifier & qualifier = static_cast(*ast.children[0]); - - const CollectTables::TableInfo * table = nullptr; - - if (qualifier.children.empty()) - table = findTableByNameOrAlias(tables, qualifier.name); - else if (qualifier.children.size() == 2) - table = findTableByDatabaseAndTableName(tables, - static_cast(*qualifier.children[0]).name, - static_cast(*qualifier.children[1]).name); - else - throw Exception("Unsupported number of components in asterisk qualifier", ErrorCodes::NOT_IMPLEMENTED); - - /// TODO Implement for case table.nested.* and database.table.nested.* - - if (!table) - throw Exception("There is no table " + qualifier.name + " in query", ErrorCodes::UNKNOWN_TABLE); - - ASTs res; - createASTsForAllColumnsInTable(*table, res); - return res; -} - - -void processIdentifier( - const ASTPtr & ast, AnalyzeColumns::Columns & columns, const CollectAliases & aliases, const CollectTables & tables) -{ - const ASTIdentifier & identifier = static_cast(*ast); - - if (aliases.aliases.count(identifier.name)) - return; - - if (columns.count(identifier.name)) - return; - - const CollectTables::TableInfo * table = nullptr; - String column_name; - - if (identifier.children.empty()) - { - /** Lambda parameters are not columns from table. Just skip them. - * This step requires AnalyzeLambdas to be done on AST. - */ - if (startsWith(identifier.name, "_lambda")) - return; - - table = findTableWithUnqualifiedName(tables, identifier.name); - if (table) - column_name = identifier.name; - } - else if (identifier.children.size() == 2) - { - const String & first = static_cast(*identifier.children[0]).name; - const String & second = static_cast(*identifier.children[1]).name; - - /// table.column - table = findTableByNameOrAlias(tables, first); - - if (table) - { - column_name = second; - } - else - { - /// column.nested - table = findTableWithUnqualifiedName(tables, identifier.name); - if (table) - column_name = identifier.name; - } - } - else if (identifier.children.size() == 3) - { - const String & first = static_cast(*identifier.children[0]).name; - const String & second = static_cast(*identifier.children[1]).name; - const String & third = static_cast(*identifier.children[2]).name; - - /// database.table.column - table = findTableByDatabaseAndTableName(tables, first, second); - - if (table) - { - column_name = third; - } - else - { - /// table.column.nested - table = findTableByNameOrAlias(tables, first); - - if (table) - { - column_name = second + "." + third; - } - else - { - /// column.nested.nested - table = findTableWithUnqualifiedName(tables, identifier.name); - if (table) - column_name = identifier.name; - } - } - } - - if (!table) - throw Exception("Cannot find column " + identifier.name, ErrorCodes::THERE_IS_NO_COLUMN); - - AnalyzeColumns::ColumnInfo info; - info.node = ast; - info.table = *table; - info.name_in_table = column_name; - - if (table->structure_of_subquery) - { - if (!table->structure_of_subquery.has(column_name)) - throw Exception("Cannot find column " + backQuoteIfNeed(column_name) + " in subquery", ErrorCodes::LOGICAL_ERROR); - - info.data_type = table->structure_of_subquery.getByName(column_name).type; - } - else if (table->storage) - { - info.data_type = table->storage->getColumn(column_name).type; - } - else - throw Exception("Logical error: no storage and no structure of subquery is specified for table", ErrorCodes::LOGICAL_ERROR); - - columns[identifier.name] = info; -} - - -void processImpl(ASTPtr & ast, AnalyzeColumns::Columns & columns, const CollectAliases & aliases, const CollectTables & tables) -{ - /// Don't go into subqueries and table-like expressions. - if (typeid_cast(ast.get()) - || typeid_cast(ast.get())) - { - return; - } - else if (const ASTFunction * func = typeid_cast(ast.get())) - { - String func_name_lowercase = Poco::toLower(func->name); - - /// As special case, treat count(*) as count(), not as count(list of all columns). - if (func_name_lowercase == "count" && func->arguments->children.size() == 1 - && typeid_cast(func->arguments->children[0].get())) - { - func->arguments->children.clear(); - } - } - else if (typeid_cast(ast.get())) - { - /// Replace asterisks to list of columns. - ASTs & asts = ast->children; - for (int i = static_cast(asts.size()) - 1; i >= 0; --i) - { - if (typeid_cast(asts[i].get())) - { - ASTs expanded = expandUnqualifiedAsterisk(tables); - asts.erase(asts.begin() + i); - asts.insert(asts.begin() + i, expanded.begin(), expanded.end()); - } - else if (ASTQualifiedAsterisk * asterisk = typeid_cast(asts[i].get())) - { - ASTs expanded = expandQualifiedAsterisk(*asterisk, tables); - asts.erase(asts.begin() + i); - asts.insert(asts.begin() + i, expanded.begin(), expanded.end()); - } - } - } - else if (typeid_cast(ast.get())) - { - processIdentifier(ast, columns, aliases, tables); - return; - } - - for (auto & child : ast->children) - processImpl(child, columns, aliases, tables); -} - -} - - -void AnalyzeColumns::process(ASTPtr & ast, const CollectAliases & aliases, const CollectTables & tables) -{ - /// If this is SELECT query, don't go into FORMAT and SETTINGS clauses - /// - they contain identifiers that are not columns. - const ASTSelectQuery * select = typeid_cast(ast.get()); - - for (auto & child : ast->children) - { - if (select && child.get() == select->settings.get()) - continue; - - processImpl(child, columns, aliases, tables); - } -} - - -void AnalyzeColumns::dump(WriteBuffer & out) const -{ - /// For need of tests, we need to dump result in some fixed order. - std::vector vec; - vec.reserve(columns.size()); - for (auto it = columns.begin(); it != columns.end(); ++it) - vec.emplace_back(it); - - std::sort(vec.begin(), vec.end(), [](const auto & a, const auto & b) { return a->first < b->first; }); - - for (const auto & it : vec) - { - writeString(it->first, out); - writeCString(" -> ", out); - - writeProbablyBackQuotedString(it->second.name_in_table, out); - writeCString(" ", out); - writeProbablyBackQuotedString(it->second.data_type->getName(), out); - - const auto & table = it->second.table; - - writeCString(". Database name: ", out); - if (table.database_name.empty()) - writeCString("(none)", out); - else - writeProbablyBackQuotedString(table.database_name, out); - - writeCString(". Table name: ", out); - if (table.table_name.empty()) - writeCString("(none)", out); - else - writeProbablyBackQuotedString(table.table_name, out); - - writeCString(". Alias: ", out); - if (table.alias.empty()) - writeCString("(none)", out); - else - writeProbablyBackQuotedString(table.alias, out); - - writeCString(". Storage: ", out); - if (!table.storage) - writeCString("(none)", out); - else - writeProbablyBackQuotedString(table.storage->getName(), out); - - writeCString(". AST: ", out); - if (it->second.node) - { - std::stringstream formatted_ast; - formatAST(*it->second.node, formatted_ast, false, true); - writeString(formatted_ast.str(), out); - } - else - writeCString("(none)", out); - - writeChar('\n', out); - } -} - - -} diff --git a/dbms/src/Analyzers/AnalyzeColumns.h b/dbms/src/Analyzers/AnalyzeColumns.h deleted file mode 100644 index cf93bbc5ed6..00000000000 --- a/dbms/src/Analyzers/AnalyzeColumns.h +++ /dev/null @@ -1,48 +0,0 @@ -#pragma once - -#include -#include -#include - - -namespace DB -{ - -class WriteBuffer; -struct CollectAliases; -struct CollectTables; - - -/** For every identifier, that is not an alias, - * determine from what table it comes, - * its original name in table, - * and its data type. - * - * Also: - * - expand asterisks (such as *, t.*, db.table.* and (TODO) even db.table.nested.*) to corresponding list of columns; - * - translate count(*) to count(); - * - TODO expand alias columns that come from table definition; - * - TODO replace column names to fully qualified names: identical columns will have same names. - * - * If column is not found or in case of ambiguity, throw an exception. - */ -struct AnalyzeColumns -{ - void process(ASTPtr & ast, const CollectAliases & aliases, const CollectTables & tables); - - struct ColumnInfo - { - ASTPtr node; - CollectTables::TableInfo table; - String name_in_table; - DataTypePtr data_type; - }; - - using Columns = std::unordered_map; - Columns columns; - - /// Debug output - void dump(WriteBuffer & out) const; -}; - -} diff --git a/dbms/src/Analyzers/AnalyzeLambdas.cpp b/dbms/src/Analyzers/AnalyzeLambdas.cpp deleted file mode 100644 index a060080c2a4..00000000000 --- a/dbms/src/Analyzers/AnalyzeLambdas.cpp +++ /dev/null @@ -1,201 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_LAMBDA; - extern const int RESERVED_IDENTIFIER_NAME; -} - - -AnalyzeLambdas::LambdaParameters AnalyzeLambdas::extractLambdaParameters(ASTPtr & ast) -{ - /// Lambda parameters could be specified in AST in two forms: - /// - just as single parameter: x -> x + 1 - /// - parameters in tuple: (x, y) -> x + 1 - -#define LAMBDA_ERROR_MESSAGE " There are two valid forms of lambda expressions: x -> ... and (x, y...) -> ..." - - if (!ast->tryGetAlias().empty()) - throw Exception("Lambda parameters cannot have aliases." - LAMBDA_ERROR_MESSAGE, ErrorCodes::BAD_LAMBDA); - - if (const ASTIdentifier * identifier = typeid_cast(ast.get())) - { - return { identifier->name }; - } - else if (const ASTFunction * function = typeid_cast(ast.get())) - { - if (function->name != "tuple") - throw Exception("Left hand side of '->' or first argument of 'lambda' is a function, but this function is not tuple." - LAMBDA_ERROR_MESSAGE " Found function '" + function->name + "' instead.", ErrorCodes::BAD_LAMBDA); - - if (!function->arguments || function->arguments->children.empty()) - throw Exception("Left hand side of '->' or first argument of 'lambda' is empty tuple." - LAMBDA_ERROR_MESSAGE, ErrorCodes::BAD_LAMBDA); - - LambdaParameters res; - res.reserve(function->arguments->children.size()); - - for (const ASTPtr & arg : function->arguments->children) - { - const ASTIdentifier * arg_identifier = typeid_cast(arg.get()); - - if (!arg_identifier) - throw Exception("Left hand side of '->' or first argument of 'lambda' contains something that is not just identifier." - LAMBDA_ERROR_MESSAGE, ErrorCodes::BAD_LAMBDA); - - if (!arg_identifier->children.empty()) - throw Exception("Left hand side of '->' or first argument of 'lambda' contains compound identifier." - LAMBDA_ERROR_MESSAGE, ErrorCodes::BAD_LAMBDA); - - if (!arg_identifier->alias.empty()) - throw Exception("Lambda parameters cannot have aliases." - LAMBDA_ERROR_MESSAGE, ErrorCodes::BAD_LAMBDA); - - res.emplace_back(arg_identifier->name); - } - - return res; - - } - else - throw Exception("Unexpected left hand side of '->' or first argument of 'lambda'." - LAMBDA_ERROR_MESSAGE, ErrorCodes::BAD_LAMBDA); - -#undef LAMBDA_ERROR_MESSAGE -} - - -namespace -{ - - -/// Currently visible parameters in all scopes of lambda expressions. -/// Lambda expressions could be nested: arrayMap(x -> arrayMap(y -> x[y], x), [[1], [2, 3]]) -using LambdaScopes = std::vector; - -void processIdentifier(ASTPtr & ast, LambdaScopes & lambda_scopes) -{ - ASTIdentifier & identifier = static_cast(*ast); - - if (identifier.children.empty()) - { - bool found = false; - - /// From most inner scope towards outer scopes. - for (ssize_t num_scopes = lambda_scopes.size(), scope_idx = num_scopes - 1; scope_idx >= 0; --scope_idx) - { - for (size_t arg_idx = 0, num_args = lambda_scopes[scope_idx].size(); arg_idx < num_args; ++arg_idx) - { - if (lambda_scopes[scope_idx][arg_idx] == identifier.name) - { - identifier.name = "_lambda" + toString(scope_idx) + "_arg" + toString(arg_idx); - - found = true; - break; - } - } - if (found) - break; - } - - if (!found && startsWith(identifier.name, "_lambda")) - throw Exception("Identifier names starting with '_lambda' are reserved for parameters of lambda expressions.", - ErrorCodes::RESERVED_IDENTIFIER_NAME); - } -} - - -void processImpl( - ASTPtr & ast, - LambdaScopes & lambda_scopes, - const ASTPtr & parent_function_for_this_argument, - AnalyzeLambdas::HigherOrderFunctions & higher_order_functions) -{ - /// Don't go into subqueries and table-like expressions. - if (typeid_cast(ast.get()) - || typeid_cast(ast.get())) - { - return; - } - else if (ASTFunction * func = typeid_cast(ast.get())) - { - /** We must memoize parameters from left hand side (x, y) and then analyze right hand side. - */ - if (func->name == "lambda") - { - auto num_arguments = func->arguments->children.size(); - if (num_arguments != 2) - throw Exception("Lambda expression ('->' or 'lambda' function) must have exactly two arguments." - " Found " + toString(num_arguments) + " instead.", ErrorCodes::BAD_LAMBDA); - - lambda_scopes.emplace_back(AnalyzeLambdas::extractLambdaParameters(func->arguments->children[0])); - for (size_t i = 0; i < num_arguments; ++i) - processImpl(func->arguments->children[i], lambda_scopes, nullptr, higher_order_functions); - lambda_scopes.pop_back(); - - if (!parent_function_for_this_argument) - throw Exception("Lambda expression ('->' or 'lambda' function) must be presented as an argument of higher-order function." - " Found standalone lambda expression instead.", ErrorCodes::BAD_LAMBDA); - - higher_order_functions.emplace_back(parent_function_for_this_argument); - } - else - { - /// When diving into function arguments, pass current ast node. - if (func->arguments) - for (auto & child : func->arguments->children) - processImpl(child, lambda_scopes, ast, higher_order_functions); - - if (func->parameters) - for (auto & child : func->parameters->children) - processImpl(child, lambda_scopes, nullptr, higher_order_functions); - } - - return; - } - else if (typeid_cast(ast.get())) - { - processIdentifier(ast, lambda_scopes); - return; - } - - for (auto & child : ast->children) - processImpl(child, lambda_scopes, nullptr, higher_order_functions); -} - -} - - -void AnalyzeLambdas::process(ASTPtr & ast) -{ - LambdaScopes lambda_scopes; - for (auto & child : ast->children) - processImpl(child, lambda_scopes, nullptr, higher_order_functions); -} - - -void AnalyzeLambdas::dump(WriteBuffer & out) const -{ - for (const auto & ast : higher_order_functions) - { - writeString(ast->getColumnName(), out); - writeChar('\n', out); - } -} - - -} diff --git a/dbms/src/Analyzers/AnalyzeLambdas.h b/dbms/src/Analyzers/AnalyzeLambdas.h deleted file mode 100644 index 0a693737295..00000000000 --- a/dbms/src/Analyzers/AnalyzeLambdas.h +++ /dev/null @@ -1,35 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -class WriteBuffer; - - -/** For every lambda expression, rename its parameters to '_lambda0_arg0' form. - * Check correctness of lambda expressions. - * Find functions, that have lambda expressions as arguments (they are called "higher order" functions). - * - * This should be done before CollectAliases. - */ -struct AnalyzeLambdas -{ - void process(ASTPtr & ast); - - /// Parameters of lambda expressions. - using LambdaParameters = std::vector; - static LambdaParameters extractLambdaParameters(ASTPtr & ast); - - - using HigherOrderFunctions = std::vector; - HigherOrderFunctions higher_order_functions; - - /// Debug output - void dump(WriteBuffer & out) const; -}; - -} diff --git a/dbms/src/Analyzers/AnalyzeResultOfQuery.cpp b/dbms/src/Analyzers/AnalyzeResultOfQuery.cpp deleted file mode 100644 index fa8f3b52dce..00000000000 --- a/dbms/src/Analyzers/AnalyzeResultOfQuery.cpp +++ /dev/null @@ -1,70 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; - extern const int UNEXPECTED_AST_STRUCTURE; -} - - -void AnalyzeResultOfQuery::process(ASTPtr & ast, const Context & context, ExecuteTableFunctions & table_functions) -{ - const ASTSelectQuery * select = typeid_cast(ast.get()); - if (!select) - throw Exception("AnalyzeResultOfQuery::process was called for not a SELECT query", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - if (!select->select_expression_list) - throw Exception("SELECT query doesn't have select_expression_list", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - - AnalyzeLambdas analyze_lambdas; - analyze_lambdas.process(ast); - - CollectAliases collect_aliases; - collect_aliases.process(ast); - - CollectTables collect_tables; - collect_tables.process(ast, context, collect_aliases, table_functions); - - AnalyzeColumns analyze_columns; - analyze_columns.process(ast, collect_aliases, collect_tables); - - TypeAndConstantInference inference; - inference.process(ast, context, collect_aliases, analyze_columns, analyze_lambdas, table_functions); - - for (const ASTPtr & child : select->select_expression_list->children) - { - auto it = inference.info.find(child->getColumnName()); - if (it == inference.info.end()) - throw Exception("Logical error: type information for result column of SELECT query was not inferred", ErrorCodes::LOGICAL_ERROR); - - String name = child->getAliasOrColumnName(); - const TypeAndConstantInference::ExpressionInfo & info = it->second; - - result.insert(ColumnWithTypeAndName( - info.is_constant_expression ? info.data_type->createColumnConst(1, info.value) : nullptr, - info.data_type, - std::move(name))); - } -} - - -void AnalyzeResultOfQuery::dump(WriteBuffer & out) const -{ - writeString(result.dumpStructure(), out); -} - - -} diff --git a/dbms/src/Analyzers/AnalyzeResultOfQuery.h b/dbms/src/Analyzers/AnalyzeResultOfQuery.h deleted file mode 100644 index 54a69c1758e..00000000000 --- a/dbms/src/Analyzers/AnalyzeResultOfQuery.h +++ /dev/null @@ -1,32 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -class WriteBuffer; -class Context; -struct ExecuteTableFunctions; - - -/** For SELECT query, determine names and types of columns of result, - * and if some columns are constant expressions, calculate their values. - * - * NOTE It's possible to memoize calculations, that happens under the hood - * and could be duplicated in subsequent analysis of subqueries. - */ -struct AnalyzeResultOfQuery -{ - void process(ASTPtr & ast, const Context & context, ExecuteTableFunctions & table_functions); - - /// Block will have non-nullptr columns for constant expressions. - Block result; - - /// Debug output - void dump(WriteBuffer & out) const; -}; - -} diff --git a/dbms/src/Analyzers/CMakeLists.txt b/dbms/src/Analyzers/CMakeLists.txt deleted file mode 100644 index 36cab0b3590..00000000000 --- a/dbms/src/Analyzers/CMakeLists.txt +++ /dev/null @@ -1,3 +0,0 @@ -if(ENABLE_TESTS) - add_subdirectory(tests) -endif() diff --git a/dbms/src/Analyzers/CollectAliases.cpp b/dbms/src/Analyzers/CollectAliases.cpp deleted file mode 100644 index 5f29595c72e..00000000000 --- a/dbms/src/Analyzers/CollectAliases.cpp +++ /dev/null @@ -1,109 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int MULTIPLE_EXPRESSIONS_FOR_ALIAS; -} - - -static void processImpl(const ASTPtr & ast, CollectAliases::Aliases & aliases, CollectAliases::Kind kind, size_t keep_kind_for_depth) -{ - String alias = ast->tryGetAlias(); - if (!alias.empty()) - { - auto it_inserted = aliases.emplace(alias, CollectAliases::AliasInfo(ast, kind)); - - if (!it_inserted.second && ast->getTreeHash() != it_inserted.first->second.node->getTreeHash()) - { - std::stringstream message; - message << "Different expressions with the same alias " << backQuoteIfNeed(alias) << ":\n"; - formatAST(*it_inserted.first->second.node, message, false, true); - message << "\nand\n"; - formatAST(*ast, message, false, true); - message << "\n"; - - throw Exception(message.str(), ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS); - } - } - - for (auto & child : ast->children) - { - if (typeid_cast(child.get())) - { - /// Don't go into subqueries. - } - else if (typeid_cast(child.get())) - { - processImpl(child, aliases, CollectAliases::Kind::Table, 1); - } - else if (typeid_cast(child.get())) - { - /// ASTArrayJoin -> ASTExpressionList -> element of expression AS alias - processImpl(child, aliases, CollectAliases::Kind::ArrayJoin, 3); - } - else if (keep_kind_for_depth > 0) - { - processImpl(child, aliases, kind, keep_kind_for_depth - 1); - } - else - { - processImpl(child, aliases, CollectAliases::Kind::Expression, 0); - } - } -} - - -void CollectAliases::process(const ASTPtr & ast) -{ - processImpl(ast, aliases, Kind::Expression, 0); -} - - -void CollectAliases::dump(WriteBuffer & out) const -{ - /// For need of tests, we need to dump result in some fixed order. - std::vector vec; - vec.reserve(aliases.size()); - for (auto it = aliases.begin(); it != aliases.end(); ++it) - vec.emplace_back(it); - - std::sort(vec.begin(), vec.end(), [](const auto & a, const auto & b) { return a->first < b->first; }); - - for (const auto & it : vec) - { - writeProbablyBackQuotedString(it->first, out); - writeCString(" -> ", out); - - switch (it->second.kind) - { - case Kind::Expression: - writeCString("(expression) ", out); - break; - case Kind::Table: - writeCString("(table) ", out); - break; - case Kind::ArrayJoin: - writeCString("(array join) ", out); - break; - } - - std::stringstream formatted_ast; - formatAST(*it->second.node, formatted_ast, false, true); - writeString(formatted_ast.str(), out); - - writeChar('\n', out); - } -} - - -} diff --git a/dbms/src/Analyzers/CollectAliases.h b/dbms/src/Analyzers/CollectAliases.h deleted file mode 100644 index 46ce74909f8..00000000000 --- a/dbms/src/Analyzers/CollectAliases.h +++ /dev/null @@ -1,54 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -class WriteBuffer; - - -/** Build a map: alias -> AST node. - * - * Also fill information about what kind each alias has: - * - expression alias; - * - table alias; - * - ARRAY JOIN alias. - * - * As extension to standard SQL, aliases could be specified and used in any part of query. - * Example: SELECT a, (1 AS a) + 1 AS b FROM t GROUP BY a, b - * Alias could be used in query before it is defined. - * - * Aliases could not be redefined. Example: 1 AS a, a + 1 AS a - is prohibited. - * - * Don't descend into subqueries (as aliases are local inside them). - */ -struct CollectAliases -{ - void process(const ASTPtr & ast); - - enum class Kind - { - Expression, /// Example: SELECT a AS b, f(x) AS y - Table, /// Example: SELECT t.* FROM (SELECT 1) AS t - ArrayJoin /// Example: SELECT t.x.a FROM t ARRAY JOIN arr AS x - }; - - struct AliasInfo - { - ASTPtr node; - Kind kind; - - AliasInfo(const ASTPtr & node, Kind kind) : node(node), kind(kind) {} - }; - - using Aliases = std::unordered_map; - Aliases aliases; - - /// Debug output - void dump(WriteBuffer & out) const; -}; - -} diff --git a/dbms/src/Analyzers/CollectTables.cpp b/dbms/src/Analyzers/CollectTables.cpp deleted file mode 100644 index d7dce3277c9..00000000000 --- a/dbms/src/Analyzers/CollectTables.cpp +++ /dev/null @@ -1,186 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int UNEXPECTED_AST_STRUCTURE; - extern const int LOGICAL_ERROR; -} - - -static CollectTables::TableInfo processOrdinaryTable(const ASTPtr & ast_database_and_table_name, const Context & context) -{ - const ASTIdentifier & identifier = static_cast(*ast_database_and_table_name); - - CollectTables::TableInfo res; - res.node = ast_database_and_table_name; - res.alias = identifier.tryGetAlias(); - - if (ast_database_and_table_name->children.empty()) - { - res.table_name = identifier.name; - } - else - { - if (ast_database_and_table_name->children.size() != 2) - throw Exception("Logical error: number of components in table expression not equal to two", ErrorCodes::LOGICAL_ERROR); - - res.database_name = static_cast(*identifier.children[0]).name; - res.table_name = static_cast(*identifier.children[1]).name; - } - - res.storage = context.getTable(res.database_name, res.table_name); - return res; -} - - -static CollectTables::TableInfo processTableFunction(const ASTPtr & ast_table_function, const ExecuteTableFunctions & table_functions) -{ - const ASTFunction & function = typeid_cast(*ast_table_function); - - IAST::Hash ast_hash = ast_table_function->getTreeHash(); - - auto it = table_functions.tables.find(ast_hash); - if (table_functions.tables.end() == it) - throw Exception("Table function " + function.name + " was not executed in advance.", ErrorCodes::LOGICAL_ERROR); - - CollectTables::TableInfo res; - res.node = ast_table_function; - res.alias = function.tryGetAlias(); - res.storage = it->second; - return res; -} - - -static CollectTables::TableInfo processNoTables(const Context & context) -{ - /// No FROM section. Interpret it as FROM system.one. - CollectTables::TableInfo res; - res.database_name = "system"; - res.table_name = "one"; - res.storage = context.getTable(res.database_name, res.table_name); - return res; -} - - -static CollectTables::TableInfo processSubquery(ASTPtr & ast_subquery, const Context & context, ExecuteTableFunctions & table_functions) -{ - AnalyzeResultOfQuery analyzer; - analyzer.process(typeid_cast(*ast_subquery).children.at(0), context, table_functions); - - CollectTables::TableInfo res; - res.node = ast_subquery; - res.alias = ast_subquery->tryGetAlias(); - res.structure_of_subquery = analyzer.result; - return res; -} - - -void CollectTables::process(ASTPtr & ast, const Context & context, const CollectAliases & /*aliases*/, ExecuteTableFunctions & table_functions) -{ - const ASTSelectQuery * select = typeid_cast(ast.get()); - if (!select) - throw Exception("CollectTables::process was called for not a SELECT query", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - - if (!select->tables) - { - tables.emplace_back(processNoTables(context)); - return; - } - - for (auto & child : select->tables->children) - { - ASTTablesInSelectQueryElement & element = static_cast(*child); - - if (!element.table_expression) /// This is ARRAY JOIN - continue; - - ASTTableExpression & table_expression = static_cast(*element.table_expression); - - if (table_expression.database_and_table_name) - { - tables.emplace_back(processOrdinaryTable(table_expression.database_and_table_name, context)); - - /// TODO It could be alias to another table expression. - } - else if (table_expression.table_function) - { - tables.emplace_back(processTableFunction(table_expression.table_function, table_functions)); - } - else if (table_expression.subquery) - { - tables.emplace_back(processSubquery(table_expression.subquery, context, table_functions)); - } - else - throw Exception("Logical error: no known elements in ASTTableExpression", ErrorCodes::LOGICAL_ERROR); - } - - /// TODO Control that tables don't have conflicting names. -} - - -void CollectTables::dump(WriteBuffer & out) const -{ - for (const auto & table : tables) - { - writeCString("Database name: ", out); - if (table.database_name.empty()) - writeCString("(none)", out); - else - writeProbablyBackQuotedString(table.database_name, out); - - writeCString(". Table name: ", out); - if (table.table_name.empty()) - writeCString("(none)", out); - else - writeProbablyBackQuotedString(table.table_name, out); - - writeCString(". Alias: ", out); - if (table.alias.empty()) - writeCString("(none)", out); - else - writeProbablyBackQuotedString(table.alias, out); - - writeCString(". Storage: ", out); - if (!table.storage) - writeCString("(none)", out); - else - writeProbablyBackQuotedString(table.storage->getName(), out); - - writeCString(". Structure of subquery: ", out); - if (!table.structure_of_subquery) - writeCString("(none)", out); - else - writeString(table.structure_of_subquery.dumpStructure(), out); - - writeCString(". AST: ", out); - if (!table.node) - writeCString("(none)", out); - else - { - std::stringstream formatted_ast; - formatAST(*table.node, formatted_ast, false, true); - writeString(formatted_ast.str(), out); - } - - writeChar('\n', out); - } -} - -} diff --git a/dbms/src/Analyzers/CollectTables.h b/dbms/src/Analyzers/CollectTables.h deleted file mode 100644 index dc31e296fe2..00000000000 --- a/dbms/src/Analyzers/CollectTables.h +++ /dev/null @@ -1,52 +0,0 @@ -#pragma once - -#include -#include -#include - - -namespace DB -{ - -class Context; -struct CollectAliases; -struct ExecuteTableFunctions; -class WriteBuffer; - - -/** Collect and analyze table-like expressions in section FROM in a query. - * For every expression, keep its alias. - * - * For ordinary tables, determine database and table name, obtain and keep StoragePtr. - * For subqueries, determine result structure. This requires analysis of subquery, such as type inference. - * For table functions, grab them from prepared ExecuteTableFunctions object. - */ -struct CollectTables -{ - void process(ASTPtr & ast, const Context & context, const CollectAliases & aliases, ExecuteTableFunctions & table_functions); - - enum class Kind - { - OrdinaryTable, - TableFunction, - Subquery - }; - - struct TableInfo - { - ASTPtr node; - String database_name; - String table_name; - String alias; - StoragePtr storage; - Block structure_of_subquery; - }; - - using Tables = std::vector; - Tables tables; - - /// Debug output - void dump(WriteBuffer & out) const; -}; - -} diff --git a/dbms/src/Analyzers/ExecuteTableFunctions.cpp b/dbms/src/Analyzers/ExecuteTableFunctions.cpp deleted file mode 100644 index 5e969c802e5..00000000000 --- a/dbms/src/Analyzers/ExecuteTableFunctions.cpp +++ /dev/null @@ -1,80 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include - -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int UNEXPECTED_AST_STRUCTURE; - extern const int LOGICAL_ERROR; -} - -/// Allows to execute exactly same table functions only once. -using ASTTreeToTable = std::map; - - -static void processTableFunction(const ASTPtr & ast_table_function, const Context & context, ExecuteTableFunctions::Tables & result_map) -{ - const ASTFunction & function = typeid_cast(*ast_table_function); - - /// If already executed. - IAST::Hash ast_hash = ast_table_function->getTreeHash(); - if (result_map.count(ast_hash)) - return; - - /// Obtain table function - TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(function.name, context); - /// Execute it and store result - StoragePtr table = table_function_ptr->execute(ast_table_function, context); - result_map[ast_hash] = table; -} - - -void ExecuteTableFunctions::process(ASTPtr & ast, const Context & context) -{ - const ASTSelectQuery * select = typeid_cast(ast.get()); - if (!select) - throw Exception("ExecuteTableFunctions::process was called for not a SELECT query", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - - if (!select->tables) - return; - - for (auto & child : select->tables->children) - { - ASTTablesInSelectQueryElement & element = static_cast(*child); - - if (!element.table_expression) /// This is ARRAY JOIN - continue; - - ASTTableExpression & table_expression = static_cast(*element.table_expression); - - if (!table_expression.table_function) - continue; - - processTableFunction(table_expression.table_function, context, tables); - } -} - - -void ExecuteTableFunctions::dump(WriteBuffer & out) const -{ - for (const auto & table : tables) - { - writeString(table.second->getName(), out); - writeCString("\n\n", out); - writeString(table.second->getColumns().getAllPhysical().toString(), out); - writeCString("\n", out); - } -} - -} diff --git a/dbms/src/Analyzers/ExecuteTableFunctions.h b/dbms/src/Analyzers/ExecuteTableFunctions.h deleted file mode 100644 index d5d70d6cacd..00000000000 --- a/dbms/src/Analyzers/ExecuteTableFunctions.h +++ /dev/null @@ -1,35 +0,0 @@ -#pragma once - -#include -#include -#include - - -namespace DB -{ - -class Context; -class WriteBuffer; - - -/** For every table function, found in first level of the query - * (don't go into subqueries) - * execute it and save corresponding StoragePtr. - * - * Execution of table functions must be done in a stage of query analysis, - * because otherwise we don't know table structure. So, it is assumed as cheap operation. - * - * Multiple occurences of table functions with same arguments will be executed only once. - */ -struct ExecuteTableFunctions -{ - void process(ASTPtr & ast, const Context & context); - - using Tables = std::map; - Tables tables; - - /// Debug output - void dump(WriteBuffer & out) const; -}; - -} diff --git a/dbms/src/Analyzers/OptimizeGroupOrderLimitBy.cpp b/dbms/src/Analyzers/OptimizeGroupOrderLimitBy.cpp deleted file mode 100644 index 5b34c6cede7..00000000000 --- a/dbms/src/Analyzers/OptimizeGroupOrderLimitBy.cpp +++ /dev/null @@ -1,264 +0,0 @@ -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; - extern const int UNEXPECTED_AST_STRUCTURE; -} - - -static bool isInjectiveFunction( - const ASTFunction * ast_function, - const TypeAndConstantInference::ExpressionInfo & function_info, - const TypeAndConstantInference::Info & all_info) -{ - if (!function_info.function) - return false; - - Block block_with_constants; - - const ASTs & children = ast_function->arguments->children; - for (const auto & child : children) - { - String child_name = child->getColumnName(); - const TypeAndConstantInference::ExpressionInfo & child_info = all_info.at(child_name); - - block_with_constants.insert(ColumnWithTypeAndName( - child_info.is_constant_expression ? child_info.data_type->createColumnConst(1, child_info.value) : nullptr, - child_info.data_type, - child_name)); - } - - return function_info.function->isInjective(block_with_constants); -} - - -static bool isDeterministicFunctionOfKeys( - const ASTFunction * ast_function, - const TypeAndConstantInference::ExpressionInfo & function_info, - const TypeAndConstantInference::Info & all_info, - const std::vector & keys) -{ - if (!function_info.function || !function_info.function->isDeterministicInScopeOfQuery()) - return false; - - for (const auto & child : ast_function->arguments->children) - { - String child_name = child->getColumnName(); - const TypeAndConstantInference::ExpressionInfo & child_info = all_info.at(child_name); - - /// Function argument is constant. - if (child_info.is_constant_expression) - continue; - - /// Function argument is one of keys. - if (keys.end() != std::find(keys.begin(), keys.end(), child_name)) - continue; - - /// Function argument is a function, that deterministically depend on keys. - if (const ASTFunction * child_function = typeid_cast(child.get())) - { - if (isDeterministicFunctionOfKeys(child_function, child_info, all_info, keys)) - continue; - } - - return false; - } - - return true; -} - - -static void processGroupByLikeList(ASTPtr & ast, TypeAndConstantInference & expression_info) -{ - if (!ast) - return; - - ASTs & elems = ast->children; - - std::unordered_set unique_keys; - size_t i = 0; - - auto restart = [&] - { - i = 0; - unique_keys.clear(); - }; - - /// Always leave last element in GROUP BY, even if it is constant. - while (i < elems.size() && elems.size() > 1) - { - ASTPtr & elem = elems[i]; - - String column_name = elem->getColumnName(); /// TODO canonicalization of names - auto it = expression_info.info.find(column_name); - if (it == expression_info.info.end()) - throw Exception("Type inference was not done for " + column_name, ErrorCodes::LOGICAL_ERROR); - const TypeAndConstantInference::ExpressionInfo & info = it->second; - - /// Removing constant expressions. - /// Removing duplicate keys. - if (info.is_constant_expression - || !unique_keys.emplace(column_name).second) - { - elems.erase(elems.begin() + i); - continue; - } - - if (info.function && !elem->children.empty()) - { - const ASTFunction * ast_function = typeid_cast(elem.get()); - if (!ast_function) - throw Exception("Column is marked as function during type inference, but corresponding AST node " - + column_name + " is not a function", ErrorCodes::LOGICAL_ERROR); - - /// Unwrap injective functions. - if (isInjectiveFunction(ast_function, info, expression_info.info)) - { - auto args = ast_function->arguments; - elems.erase(elems.begin() + i); - elems.insert(elems.begin() + i, args->children.begin(), args->children.end()); - - restart(); /// Previous keys may become deterministic function of newly added keys. - continue; - } - - /// Remove deterministic functions of another keys. - std::vector other_keys; - other_keys.reserve(elems.size() - 1); - for (size_t j = 0, size = elems.size(); j < size; ++j) - if (j != i) - other_keys.emplace_back(elems[j]->getColumnName()); - - if (isDeterministicFunctionOfKeys(ast_function, info, expression_info.info, other_keys)) - { - elems.erase(elems.begin() + i); - continue; - } - } - - ++i; - } -} - - -static void processOrderByList(ASTPtr & ast, TypeAndConstantInference & expression_info) -{ - if (!ast) - return; - - ASTs & elems = ast->children; - - /// sort column name and collation - std::set> unique_keys; - size_t i = 0; - while (i < elems.size()) - { - const ASTOrderByElement * order_by_elem = typeid_cast(elems[i].get()); - if (!order_by_elem) - throw Exception("Child of ORDER BY clause is not an ASTOrderByElement", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - - /// It has ASC|DESC and COLLATE inplace, and expression as its only child. - if (order_by_elem->children.empty()) - throw Exception("ORDER BY element has no children", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - - const ASTPtr & elem = order_by_elem->children[0]; - String collation; - if (order_by_elem->collation) - { - const ASTLiteral * lit = typeid_cast(order_by_elem->collation.get()); - if (!lit) - throw Exception("Collation in ORDER BY clause is not an ASTLiteral", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - - if (lit->value.getType() != Field::Types::String) - throw Exception("Collation in ORDER BY clause is not a string literal", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - - collation = lit->value.get(); - } - - String column_name = elem->getColumnName(); - auto it = expression_info.info.find(column_name); - if (it == expression_info.info.end()) - throw Exception("Type inference was not done for " + column_name, ErrorCodes::LOGICAL_ERROR); - const TypeAndConstantInference::ExpressionInfo & info = it->second; - - /// Removing constant expressions. - /// Removing duplicate keys. - if (info.is_constant_expression - /// Having same element but with empty collation. Empty collation is considered more "granular" than any special collation. - || unique_keys.count(std::make_pair(column_name, String())) - /// Having same element with same collation. - || !unique_keys.emplace(column_name, collation).second) - { - elems.erase(elems.begin() + i); - continue; - } - - if (i > 0 && collation.empty() && info.function && !elem->children.empty()) - { - const ASTFunction * ast_function = typeid_cast(elem.get()); - if (!ast_function) - throw Exception("Column is marked as function during type inference, but corresponding AST node " - + column_name + " is not a function", ErrorCodes::LOGICAL_ERROR); - - /// Remove deterministic functions of previous keys. Only consider keys without collation. - std::vector prev_keys; - prev_keys.reserve(i); - for (size_t j = 0; j < i; ++j) - if (!typeid_cast(*elems[j]).collation) - prev_keys.emplace_back(elems[j]->children.at(0)->getColumnName()); - - if (isDeterministicFunctionOfKeys(ast_function, info, expression_info.info, prev_keys)) - { - elems.erase(elems.begin() + i); - continue; - } - } - - ++i; - } -} - - -void OptimizeGroupOrderLimitBy::process(ASTPtr & ast, TypeAndConstantInference & expression_info) -{ - ASTSelectQuery * select = typeid_cast(ast.get()); - if (!select) - throw Exception("AnalyzeResultOfQuery::process was called for not a SELECT query", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - if (!select->select_expression_list) - throw Exception("SELECT query doesn't have select_expression_list", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - - processGroupByLikeList(select->group_expression_list, expression_info); - processGroupByLikeList(select->limit_by_expression_list, expression_info); - - if (select->order_expression_list) - { - processOrderByList(select->order_expression_list, expression_info); - - /// ORDER BY could be completely eliminated - if (select->order_expression_list->children.empty()) - { - select->children.erase(std::remove( - select->children.begin(), select->children.end(), select->order_expression_list), select->children.end()); - select->order_expression_list.reset(); - } - } -} - - -} diff --git a/dbms/src/Analyzers/OptimizeGroupOrderLimitBy.h b/dbms/src/Analyzers/OptimizeGroupOrderLimitBy.h deleted file mode 100644 index 44d83193df4..00000000000 --- a/dbms/src/Analyzers/OptimizeGroupOrderLimitBy.h +++ /dev/null @@ -1,36 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -class Context; -class WriteBuffer; -struct TypeAndConstantInference; - - -/** Transform GROUP BY, ORDER BY and LIMIT BY sections. - * (LIMIT BY is an extension to SQL language, do not be confused with ordinary LIMIT) - * - * Remove constant expressions (like ORDER BY concat('hello', 'world')). - * For GROUP BY, unwrap injective functions (like GROUP BY toString(x) -> GROUP BY x). - * For GROUP BY, remove deterministic functions of another keys (like GROUP BY x + 1, x -> GROUP BY x). - * TODO For ORDER BY, remove deterministic functions of previous keys (like ORDER BY num, toString(num) -> ORDER BY num), - * but only if no collation has specified. - * As a special case, remove duplicate keys. - * For LIMIT BY, apply all the same as for GROUP BY. - * - * TODO We should apply something similar for DISTINCT, - * but keys for DISTINCT are specified implicitly (as whole SELECT expression list). - * - * This should be run after CollectAliases, because some aliases will be lost from AST during this transformation. - * This should be run after TranslatePositionalArguments for positional arguments like ORDER BY 1, 2 not to be confused with constants. - */ -struct OptimizeGroupOrderLimitBy -{ - void process(ASTPtr & ast, TypeAndConstantInference & expression_info); -}; - -} diff --git a/dbms/src/Analyzers/TranslatePositionalArguments.cpp b/dbms/src/Analyzers/TranslatePositionalArguments.cpp deleted file mode 100644 index c4a0b8814ec..00000000000 --- a/dbms/src/Analyzers/TranslatePositionalArguments.cpp +++ /dev/null @@ -1,86 +0,0 @@ -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; - extern const int UNEXPECTED_AST_STRUCTURE; - extern const int INDEX_OF_POSITIONAL_ARGUMENT_IS_OUT_OF_RANGE; -} - - -static void processElement(ASTPtr & element, const ASTPtr & select_expression_list, const char * description) -{ - if (ASTLiteral * literal = typeid_cast(element.get())) - { - if (literal->value.getType() == Field::Types::UInt64) - { - UInt64 position = literal->value.get(); - - if (!literal->alias.empty()) - throw Exception("Unsigned numeric literal " + toString(position) + " in " + toString(description) - + " section is interpreted as positional argument, " - "but it has alias " + backQuoteIfNeed(literal->alias) + " that is not expected", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - - if (position == 0) - throw Exception("Unsigned numeric literal 0 in " + toString(description) + " section is interpreted as positional argument, " - "but positional arguments are 1-based", ErrorCodes::INDEX_OF_POSITIONAL_ARGUMENT_IS_OUT_OF_RANGE); - - if (position > select_expression_list->children.size()) - throw Exception("Unsigned numeric literal " + toString(position) + " in " + String(description) - + " section is interpreted as positional argument, " - "but it is greater than number of expressions in SELECT section (" - + toString(select_expression_list->children.size()) + ")", ErrorCodes::INDEX_OF_POSITIONAL_ARGUMENT_IS_OUT_OF_RANGE); - - element = select_expression_list->children[position - 1]->clone(); - } - } -} - - -static void processClause(ASTPtr & ast, const ASTPtr & select_expression_list, const char * description, bool is_order_by) -{ - if (!ast) - return; - - for (auto & child : ast->children) - { - if (is_order_by) - { - if (!typeid_cast(child.get())) - throw Exception("Child of ORDER BY clause is not an ASTOrderByElement", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - - /// It has ASC|DESC and COLLATE inplace, and expression as its only child. - if (child->children.empty()) - throw Exception("ORDER BY element has no children", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - - processElement(child->children[0], select_expression_list, description); - } - else - processElement(child, select_expression_list, description); - } -} - - -void TranslatePositionalArguments::process(ASTPtr & ast) -{ - ASTSelectQuery * select = typeid_cast(ast.get()); - if (!select) - throw Exception("TranslatePositionalArguments::process was called for not a SELECT query", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - if (!select->select_expression_list) - throw Exception("SELECT query doesn't have select_expression_list", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - - processClause(select->group_expression_list, select->select_expression_list, "GROUP BY", false); - processClause(select->order_expression_list, select->select_expression_list, "ORDER BY", true); - processClause(select->limit_by_expression_list, select->select_expression_list, "LIMIT BY", false); -} - -} diff --git a/dbms/src/Analyzers/TranslatePositionalArguments.h b/dbms/src/Analyzers/TranslatePositionalArguments.h deleted file mode 100644 index 6209685eaeb..00000000000 --- a/dbms/src/Analyzers/TranslatePositionalArguments.h +++ /dev/null @@ -1,21 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -class Context; -class WriteBuffer; - - -/** Transform GROUP BY, ORDER BY and LIMIT BY sections. - * Replace positional arguments (like ORDER BY 1, 2) to corresponding columns. - */ -struct TranslatePositionalArguments -{ - void process(ASTPtr & ast); -}; - -} diff --git a/dbms/src/Analyzers/TypeAndConstantInference.cpp b/dbms/src/Analyzers/TypeAndConstantInference.cpp deleted file mode 100644 index 71935d8ea9b..00000000000 --- a/dbms/src/Analyzers/TypeAndConstantInference.cpp +++ /dev/null @@ -1,534 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; - extern const int FUNCTION_CANNOT_HAVE_PARAMETERS; - extern const int UNEXPECTED_AST_STRUCTURE; - extern const int BAD_LAMBDA; -} - - -namespace -{ - -Field getValueFromConstantColumn(const ColumnPtr & column) -{ - if (!column->isColumnConst()) - throw Exception("Logical error: expected that column is constant", ErrorCodes::LOGICAL_ERROR); - if (column->size() != 1) - throw Exception("Logical error: expected that column with constant has single element", ErrorCodes::LOGICAL_ERROR); - return (*column)[0]; -} - - -/// Description of single parameter of lambda expression: name -> type. -/// Note, that after AnalyzeLambda step, names of lambda parameters are unique in single select query. -using LambdaParameters = std::unordered_map; - - -void processImpl( - ASTPtr & ast, const Context & context, - CollectAliases & aliases, const AnalyzeColumns & columns, - TypeAndConstantInference::Info & info, - const AnalyzeLambdas & lambdas, - ExecuteTableFunctions & table_functions); - - -void processLiteral(const String & column_name, const ASTPtr & ast, TypeAndConstantInference::Info & info) -{ - const ASTLiteral * literal = static_cast(ast.get()); - - TypeAndConstantInference::ExpressionInfo expression_info; - expression_info.node = ast; - expression_info.is_constant_expression = true; - expression_info.data_type = applyVisitor(FieldToDataType(), literal->value); - expression_info.value = convertFieldToType(literal->value, *expression_info.data_type); - info.emplace(column_name, std::move(expression_info)); -} - - -void processIdentifier(const String & column_name, const ASTPtr & ast, TypeAndConstantInference::Info & info, - const Context & context, CollectAliases & aliases, const AnalyzeColumns & columns, - const AnalyzeLambdas & lambdas, ExecuteTableFunctions & table_functions) -{ - /// Column from table - auto it = columns.columns.find(column_name); - if (it != columns.columns.end()) - { - TypeAndConstantInference::ExpressionInfo expression_info; - expression_info.node = ast; - expression_info.data_type = it->second.data_type; - - /// If it comes from subquery and we know, that it is constant expression. - const Block & structure_of_subquery = it->second.table.structure_of_subquery; - if (structure_of_subquery) - { - const ColumnWithTypeAndName & column_from_subquery = structure_of_subquery.getByName(it->second.name_in_table); - if (column_from_subquery.column) - { - expression_info.is_constant_expression = true; - expression_info.value = getValueFromConstantColumn(column_from_subquery.column); - } - } - - info.emplace(column_name, std::move(expression_info)); - } - else - { - /// Alias - auto jt = aliases.aliases.find(column_name); - if (jt != aliases.aliases.end()) - { - /// TODO Cyclic aliases. - - if (jt->second.kind != CollectAliases::Kind::Expression) - throw Exception("Logical error: unexpected kind of alias", ErrorCodes::LOGICAL_ERROR); - - processImpl(jt->second.node, context, aliases, columns, info, lambdas, table_functions); - info[column_name] = info[jt->second.node->getColumnName()]; - } - } -} - - -void processFunction(const String & column_name, ASTPtr & ast, TypeAndConstantInference::Info & info, - const Context & context) -{ - ASTFunction * function = static_cast(ast.get()); - - /// Special case for lambda functions. Lambda function has special return type "Function". - /// We first create info with Function of unspecified arguments, and will specify them later. - if (function->name == "lambda") - { - size_t number_of_lambda_parameters = AnalyzeLambdas::extractLambdaParameters(function->arguments->children.at(0)).size(); - - TypeAndConstantInference::ExpressionInfo expression_info; - expression_info.node = ast; - expression_info.data_type = std::make_unique(DataTypes(number_of_lambda_parameters)); - info.emplace(column_name, std::move(expression_info)); - return; - } - - DataTypes argument_types; - ColumnsWithTypeAndName argument_columns; - - if (function->arguments) - { - for (const auto & child : function->arguments->children) - { - auto it = info.find(child->getColumnName()); - if (it == info.end()) - throw Exception("Logical error: type of function argument was not inferred during depth-first search", ErrorCodes::LOGICAL_ERROR); - - argument_types.emplace_back(it->second.data_type); - argument_columns.emplace_back(ColumnWithTypeAndName(nullptr, it->second.data_type, "")); - if (it->second.is_constant_expression) - argument_columns.back().column = it->second.data_type->createColumnConst(1, it->second.value); - } - } - - /// Special cases for COUNT(DISTINCT ...) function. - bool column_name_changed = false; - String func_name_lowercase = Poco::toLower(function->name); - if (func_name_lowercase == "countdistinct") /// It comes in that form from parser. - { - /// Select implementation of countDistinct based on settings. - /// Important that it is done as query rewrite. It means rewritten query - /// will be sent to remote servers during distributed query execution, - /// and on all remote servers, function implementation will be same. - function->name = context.getSettingsRef().count_distinct_implementation; - column_name_changed = true; - } - - /// Aggregate function. - Array parameters = (function->parameters) ? getAggregateFunctionParametersArray(function->parameters) : Array(); - if (AggregateFunctionPtr aggregate_function_ptr = AggregateFunctionFactory::instance().tryGet(function->name, argument_types, parameters)) - { - /// Note that aggregate function could never be constant expression. - - /// (?) Replace function name to canonical one. Because same function could be referenced by different names. - // function->name = aggregate_function_ptr->getName(); - - TypeAndConstantInference::ExpressionInfo expression_info; - expression_info.node = ast; - expression_info.data_type = aggregate_function_ptr->getReturnType(); - expression_info.aggregate_function = aggregate_function_ptr; - info.emplace(column_name_changed ? ast->getColumnName() : column_name, std::move(expression_info)); - return; - } - - /// Ordinary function. - if (function->parameters) - throw Exception("The only parametric functions (functions with two separate parenthesis pairs) are aggregate functions" - ", and '" + function->name + "' is not an aggregate function.", ErrorCodes::FUNCTION_CANNOT_HAVE_PARAMETERS); - - /// IN operator. This is special case, because subqueries in right hand side are not scalar subqueries. - if (function->name == "in" - || function->name == "notIn" - || function->name == "globalIn" - || function->name == "globalNotIn") - { - /// For simplicity reasons, do not consider this as constant expression. We may change it in future. - TypeAndConstantInference::ExpressionInfo expression_info; - expression_info.node = ast; - expression_info.data_type = std::make_shared(); - info.emplace(column_name, std::move(expression_info)); - return; - } - - const auto & function_builder_ptr = FunctionFactory::instance().get(function->name, context); - - /// (?) Replace function name to canonical one. Because same function could be referenced by different names. - // function->name = function_ptr->getName(); - - ColumnsWithTypeAndName columns_for_analysis; - columns_for_analysis.reserve(argument_types.size()); - - bool all_consts = true; - if (function->arguments) - { - for (const auto & child : function->arguments->children) - { - String child_name = child->getColumnName(); - const TypeAndConstantInference::ExpressionInfo & child_info = info.at(child_name); - columns_for_analysis.emplace_back( - child_info.is_constant_expression ? child_info.data_type->createColumnConst(1, child_info.value) : nullptr, - child_info.data_type, - child_name); - - if (!child_info.is_constant_expression) - all_consts = false; - } - } - - auto function_ptr = function_builder_ptr->build(argument_columns); - - TypeAndConstantInference::ExpressionInfo expression_info; - expression_info.node = ast; - expression_info.function = function_ptr; - expression_info.data_type = function_ptr->getReturnType(); - - if (all_consts && function_ptr->isSuitableForConstantFolding()) - { - Block block_with_constants(columns_for_analysis); - - ColumnNumbers argument_numbers(columns_for_analysis.size()); - for (size_t i = 0, size = argument_numbers.size(); i < size; ++i) - argument_numbers[i] = i; - - size_t result_position = argument_numbers.size(); - block_with_constants.insert({nullptr, expression_info.data_type, column_name}); - - function_ptr->execute(block_with_constants, argument_numbers, result_position, 1); - - const auto & result_column = block_with_constants.getByPosition(result_position).column; - if (result_column->isColumnConst()) - { - expression_info.is_constant_expression = true; - expression_info.value = (*result_column)[0]; - } - } - - info.emplace(column_name, std::move(expression_info)); -} - - -void processScalarSubquery(const String & column_name, ASTPtr & ast, TypeAndConstantInference::Info & info, - const Context & context, ExecuteTableFunctions & table_functions) -{ - ASTSubquery * subquery = static_cast(ast.get()); - - AnalyzeResultOfQuery analyzer; - analyzer.process(subquery->children.at(0), context, table_functions); - - if (!analyzer.result) - throw Exception("Logical error: no columns returned from scalar subquery", ErrorCodes::LOGICAL_ERROR); - - TypeAndConstantInference::ExpressionInfo expression_info; - expression_info.node = ast; - - if (analyzer.result.columns() == 1) - { - const auto & elem = analyzer.result.getByPosition(0); - expression_info.data_type = elem.type; - - if (elem.column) - { - expression_info.is_constant_expression = true; - expression_info.value = getValueFromConstantColumn(elem.column); - } - } - else - { - /// Result of scalar subquery is interpreted as tuple. - size_t size = analyzer.result.columns(); - DataTypes types; - types.reserve(size); - bool all_consts = true; - for (size_t i = 0; i < size; ++i) - { - const auto & elem = analyzer.result.getByPosition(i); - types.emplace_back(elem.type); - if (!elem.column) - all_consts = false; - } - - expression_info.data_type = std::make_shared(types); - - if (all_consts) - { - TupleBackend value(size); - - for (size_t i = 0; i < size; ++i) - value[i] = getValueFromConstantColumn(analyzer.result.getByPosition(i).column); - - expression_info.is_constant_expression = true; - expression_info.value = Tuple(std::move(value)); - } - } - - info.emplace(column_name, std::move(expression_info)); -} - - -void processHigherOrderFunction( - ASTPtr & ast, const Context & context, - CollectAliases & aliases, const AnalyzeColumns & columns, - TypeAndConstantInference::Info & info, - const AnalyzeLambdas & lambdas, - ExecuteTableFunctions & table_functions) -{ - ASTFunction * function = static_cast(ast.get()); - - const auto & function_builder_ptr = FunctionFactory::instance().get(function->name, context); - - if (!function->arguments) - throw Exception("Unexpected AST for higher-order function", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - - DataTypes types; - types.reserve(function->arguments->children.size()); - for (const auto & child : function->arguments->children) - { - String child_name = child->getColumnName(); - const TypeAndConstantInference::ExpressionInfo & child_info = info.at(child_name); - types.emplace_back(child_info.data_type); - } - - function_builder_ptr->getLambdaArgumentTypes(types); - - /// For every lambda expression, dive into it. - - if (types.size() != function->arguments->children.size()) - throw Exception("Logical error: size of types was changed after call to IFunction::getLambdaArgumentTypes", - ErrorCodes::LOGICAL_ERROR); - - for (size_t i = 0, size = function->arguments->children.size(); i < size; ++i) - { - const auto & child = function->arguments->children[i]; - const ASTFunction * lambda = typeid_cast(child.get()); - if (lambda && lambda->name == "lambda") - { - const auto * lambda_type = typeid_cast(types[i].get()); - - if (!lambda_type) - throw Exception("Logical error: IFunction::getLambdaArgumentTypes returned data type for lambda expression," - " that is not DataTypeFunction", ErrorCodes::LOGICAL_ERROR); - - if (!lambda->arguments || lambda->arguments->children.size() != 2) - throw Exception("Lambda function must have exactly two arguments (sides of arrow)", ErrorCodes::BAD_LAMBDA); - - /// Set types for every lambda parameter. - - AnalyzeLambdas::LambdaParameters parameters = AnalyzeLambdas::extractLambdaParameters(lambda->arguments->children[0]); - - const DataTypes & lambda_argument_types = lambda_type->getArgumentTypes(); - - if (parameters.size() != lambda_argument_types.size()) - throw Exception("DataTypeExpression for lambda function has different number of argument types than number of lambda parameters", - ErrorCodes::LOGICAL_ERROR); - - for (size_t param_idx = 0, num_params = parameters.size(); param_idx < num_params; ++param_idx) - { - TypeAndConstantInference::ExpressionInfo expression_info; - expression_info.node = typeid_cast(lambda->arguments->children[0].get()) - ? lambda->arguments->children[0] - : typeid_cast(*lambda->arguments->children[0]).arguments->children.at(param_idx); - - expression_info.data_type = lambda_argument_types[param_idx]; - - info.emplace(parameters[param_idx], std::move(expression_info)); - } - - /// Now dive into. - - processImpl(lambda->arguments->children[1], context, aliases, columns, info, lambdas, table_functions); - - /// Update Expression type (expression signature). - - info.at(lambda->getColumnName()).data_type = std::make_shared( - lambda_argument_types, info.at(lambda->arguments->children[1]->getColumnName()).data_type); - } - } -} - - -void processImpl( - ASTPtr & ast, const Context & context, - CollectAliases & aliases, const AnalyzeColumns & columns, - TypeAndConstantInference::Info & info, - const AnalyzeLambdas & lambdas, - ExecuteTableFunctions & table_functions) -{ - const ASTFunction * function = typeid_cast(ast.get()); - - /// Bottom-up. - - /// Don't go into components of compound identifiers. - if (!typeid_cast(ast.get())) - { - for (auto & child : ast->children) - { - /** Don't go into subqueries and table-like expressions. - * Also don't go into components of compound identifiers. - */ - if (typeid_cast(child.get()) - || typeid_cast(child.get())) - continue; - - /** Postpone diving into lambda expressions. - * We must first infer types of other arguments of higher-order function, - * and then process lambda expression. - * Example: - * arrayMap((x, y) -> x + y, arr1, arr2) - * First, infer type of 'arr1' and 'arr2'. - * Then, ask function arrayMap, what types will have 'x' and 'y'. - * And then, infer type of 'x + y'. - */ - if (function && function->name == "lambda") - continue; - - processImpl(child, context, aliases, columns, info, lambdas, table_functions); - } - } - - const ASTLiteral * literal = nullptr; - const ASTIdentifier * identifier = nullptr; - const ASTSubquery * subquery = nullptr; - - function - || (literal = typeid_cast(ast.get())) - || (identifier = typeid_cast(ast.get())) - || (subquery = typeid_cast(ast.get())); - - if (!literal && !identifier && !function && !subquery) - return; - - /// Same expression is already processed. - String column_name = ast->getColumnName(); - if (info.count(column_name)) - return; - - if (function) - { - /// If this is higher-order function, determine types of lambda arguments and infer types of subexpressions inside lambdas. - if (lambdas.higher_order_functions.end() != std::find(lambdas.higher_order_functions.begin(), lambdas.higher_order_functions.end(), ast)) - processHigherOrderFunction(ast, context, aliases, columns, info, lambdas, table_functions); - - processFunction(column_name, ast, info, context); - } - else if (literal) - processLiteral(column_name, ast, info); - else if (identifier) - processIdentifier(column_name, ast, info, context, aliases, columns, lambdas, table_functions); - else if (subquery) - processScalarSubquery(column_name, ast, info, context, table_functions); -} - -} - - -void TypeAndConstantInference::process(ASTPtr & ast, const Context & context, - CollectAliases & aliases, - const AnalyzeColumns & columns, - const AnalyzeLambdas & lambdas, - ExecuteTableFunctions & table_functions) -{ - processImpl(ast, context, aliases, columns, info, lambdas, table_functions); -} - - -void TypeAndConstantInference::dump(WriteBuffer & out) const -{ - /// For need of tests, we need to dump result in some fixed order. - std::vector vec; - vec.reserve(info.size()); - for (auto it = info.begin(); it != info.end(); ++it) - vec.emplace_back(it); - - std::sort(vec.begin(), vec.end(), [](const auto & a, const auto & b) { return a->first < b->first; }); - - for (const auto & it : vec) - { - writeString(it->first, out); - writeCString(" -> ", out); - writeString(it->second.data_type->getName(), out); - - if (it->second.is_constant_expression) - { - writeCString(" = ", out); - String value = applyVisitor(FieldVisitorToString(), it->second.value); - writeString(value, out); - } - - writeCString(". AST: ", out); - if (!it->second.node) - writeCString("(none)", out); - else - { - std::stringstream formatted_ast; - formatAST(*it->second.node, formatted_ast, false, true); - writeString(formatted_ast.str(), out); - } - - writeChar('\n', out); - } -} - -} diff --git a/dbms/src/Analyzers/TypeAndConstantInference.h b/dbms/src/Analyzers/TypeAndConstantInference.h deleted file mode 100644 index 2a0da5ea937..00000000000 --- a/dbms/src/Analyzers/TypeAndConstantInference.h +++ /dev/null @@ -1,61 +0,0 @@ -#pragma once - -#include -#include -#include -#include - - -namespace DB -{ - -class Context; -class WriteBuffer; -struct CollectAliases; -struct AnalyzeColumns; -struct AnalyzeLambdas; -struct ExecuteTableFunctions; -class IFunctionBase; -class IAggregateFunction; - - -/** For every expression, deduce its type, - * and if it is a constant expression, calculate its value. - * - * Types and constants inference goes together, - * because sometimes resulting type of a function depend on value of constant expression. - * Notable examples: tupleElement(tuple, N) and toFixedString(s, N) functions. - * - * Also creates and stores function objects. - * TODO (?) Also calculate ids for expressions, that will identify common subexpressions. - */ -struct TypeAndConstantInference -{ - void process(ASTPtr & ast, const Context & context, - CollectAliases & aliases, - const AnalyzeColumns & columns, - const AnalyzeLambdas & analyze_lambdas, - ExecuteTableFunctions & table_functions); - - struct ExpressionInfo - { - /// Must identify identical expressions. - /// For example following three expressions in query are the same: SELECT sum(x) AS a, SUM(t.x) AS b, a FROM t - UInt128 id {}; - ASTPtr node; - DataTypePtr data_type; - bool is_constant_expression = false; - Field value; /// Has meaning if is_constant_expression == true. - std::shared_ptr function; - std::shared_ptr aggregate_function; - }; - - /// Key is getColumnName of AST node. - using Info = std::unordered_map; - Info info; - - /// Debug output - void dump(WriteBuffer & out) const; -}; - -} diff --git a/dbms/src/Analyzers/tests/CMakeLists.txt b/dbms/src/Analyzers/tests/CMakeLists.txt deleted file mode 100644 index a4f331dbd3a..00000000000 --- a/dbms/src/Analyzers/tests/CMakeLists.txt +++ /dev/null @@ -1,24 +0,0 @@ -add_executable(collect_aliases collect_aliases.cpp) -target_link_libraries(collect_aliases dbms) - -add_executable(collect_tables collect_tables.cpp) -target_link_libraries(collect_tables dbms clickhouse_storages_system) - -add_executable(analyze_columns analyze_columns.cpp) -target_link_libraries(analyze_columns dbms clickhouse_storages_system) - -add_executable(type_and_constant_inference type_and_constant_inference.cpp) -target_link_libraries(type_and_constant_inference - clickhouse_storages_system clickhouse_functions clickhouse_aggregate_functions clickhouse_table_functions) - -add_executable(analyze_result_of_query analyze_result_of_query.cpp) -target_link_libraries(analyze_result_of_query dbms clickhouse_storages_system) - -add_executable(translate_positional_arguments translate_positional_arguments.cpp) -target_link_libraries(translate_positional_arguments dbms) - -add_executable(optimize_group_order_limit_by optimize_group_order_limit_by.cpp) -target_link_libraries(optimize_group_order_limit_by dbms clickhouse_storages_system) - -add_executable(analyze_lambdas analyze_lambdas.cpp) -target_link_libraries(analyze_lambdas dbms) diff --git a/dbms/src/Analyzers/tests/analyze_columns.cpp b/dbms/src/Analyzers/tests/analyze_columns.cpp deleted file mode 100644 index af9cef55108..00000000000 --- a/dbms/src/Analyzers/tests/analyze_columns.cpp +++ /dev/null @@ -1,71 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -/// Parses query from stdin and print found columns and corresponding tables. - -int main(int, char **) -try -{ - using namespace DB; - - ReadBufferFromFileDescriptor in(STDIN_FILENO); - WriteBufferFromFileDescriptor out(STDOUT_FILENO); - - String query; - readStringUntilEOF(query, in); - - ParserSelectQuery parser; - ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0); - - Context context = Context::createGlobal(); - - auto system_database = std::make_shared("system"); - context.addDatabase("system", system_database); - system_database->attachTable("one", StorageSystemOne::create("one")); - system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false)); - context.setCurrentDatabase("system"); - - AnalyzeLambdas analyze_lambdas; - analyze_lambdas.process(ast); - - CollectAliases collect_aliases; - collect_aliases.process(ast); - - ExecuteTableFunctions execute_table_functions; - execute_table_functions.process(ast, context); - - CollectTables collect_tables; - collect_tables.process(ast, context, collect_aliases, execute_table_functions); - - AnalyzeColumns analyze_columns; - analyze_columns.process(ast, collect_aliases, collect_tables); - - analyze_columns.dump(out); - out.next(); - - std::cout << "\n"; - formatAST(*ast, std::cout, false, true); - std::cout << "\n"; - - return 0; -} -catch (...) -{ - std::cerr << DB::getCurrentExceptionMessage(true) << "\n"; - return 1; -} diff --git a/dbms/src/Analyzers/tests/analyze_columns.reference b/dbms/src/Analyzers/tests/analyze_columns.reference deleted file mode 100644 index 78370334334..00000000000 --- a/dbms/src/Analyzers/tests/analyze_columns.reference +++ /dev/null @@ -1,18 +0,0 @@ -dummy -> dummy UInt8. Database name: system. Table name: one. Alias: (none). Storage: SystemOne. AST: dummy -number -> number UInt64. Database name: system. Table name: numbers. Alias: t. Storage: SystemNumbers. AST: number -numbers.number -> number UInt64. Database name: system. Table name: numbers. Alias: t. Storage: SystemNumbers. AST: numbers.number -one.dummy -> dummy UInt8. Database name: system. Table name: one. Alias: (none). Storage: SystemOne. AST: one.dummy -system.numbers.number -> number UInt64. Database name: system. Table name: numbers. Alias: t. Storage: SystemNumbers. AST: system.numbers.number -system.one.dummy -> dummy UInt8. Database name: system. Table name: one. Alias: (none). Storage: SystemOne. AST: system.one.dummy -t.number -> number UInt64. Database name: system. Table name: numbers. Alias: t. Storage: SystemNumbers. AST: t.number - -SELECT dummy, number, one.dummy, numbers.number, system.one.dummy, system.numbers.number, system.one.dummy, system.numbers.number, system.one.dummy, system.numbers.number, system.one.dummy, system.numbers.number, system.numbers.number, t.number FROM system.one , system.numbers AS t - -c -> c UInt8. Database name: (none). Table name: (none). Alias: (none). Storage: (none). AST: c - -SELECT arrayMap((_lambda0_arg0, _lambda0_arg1) -> arrayMap((_lambda1_arg0, _lambda1_arg1) -> _lambda0_arg0[_lambda1_arg0], _lambda0_arg0, c), [[1], [2, 3]]) FROM (SELECT 1 AS c, 2 AS d) - -c -> c UInt8. Database name: (none). Table name: (none). Alias: (none). Storage: (none). AST: c -x -> x UInt8. Database name: (none). Table name: (none). Alias: (none). Storage: (none). AST: x - -SELECT x, arrayMap((_lambda0_arg0, _lambda0_arg1) -> (_lambda0_arg0 + _lambda0_arg1), x, c) FROM (SELECT 1 AS x, 2 AS c) diff --git a/dbms/src/Analyzers/tests/analyze_columns.sh b/dbms/src/Analyzers/tests/analyze_columns.sh deleted file mode 100755 index 4fb139ff37b..00000000000 --- a/dbms/src/Analyzers/tests/analyze_columns.sh +++ /dev/null @@ -1,7 +0,0 @@ -#!/bin/sh - -echo "SELECT dummy, number, one.dummy, numbers.number, system.one.dummy, system.numbers.number, one.*, numbers.*, system.one.*, system.numbers.*, *, t.*, t.number FROM system.one, system.numbers AS t" | ./analyze_columns -echo -echo "SELECT arrayMap((x, y) -> arrayMap((y, z) -> x[y], x, c), [[1], [2, 3]]) FROM (SELECT 1 AS c, 2 AS d)" | ./analyze_columns -echo -echo "SELECT x, arrayMap((x, y) -> x + y, x, c) FROM (SELECT 1 AS x, 2 AS c)" | ./analyze_columns diff --git a/dbms/src/Analyzers/tests/analyze_lambdas.cpp b/dbms/src/Analyzers/tests/analyze_lambdas.cpp deleted file mode 100644 index c82219e4722..00000000000 --- a/dbms/src/Analyzers/tests/analyze_lambdas.cpp +++ /dev/null @@ -1,44 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -/// Parses query from stdin and print found higher order functions and query with rewritten names of lambda parameters. - -int main(int, char **) -try -{ - using namespace DB; - - ReadBufferFromFileDescriptor in(STDIN_FILENO); - WriteBufferFromFileDescriptor out(STDOUT_FILENO); - - String query; - readStringUntilEOF(query, in); - - ParserSelectQuery parser; - ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0); - - AnalyzeLambdas analyzer; - analyzer.process(ast); - - analyzer.dump(out); - out.next(); - - std::cout << "\n"; - formatAST(*ast, std::cout, false, true); - std::cout << "\n"; - - return 0; -} -catch (...) -{ - std::cerr << DB::getCurrentExceptionMessage(true) << "\n"; - return 1; -} diff --git a/dbms/src/Analyzers/tests/analyze_lambdas.reference b/dbms/src/Analyzers/tests/analyze_lambdas.reference deleted file mode 100644 index 0dab2e44265..00000000000 --- a/dbms/src/Analyzers/tests/analyze_lambdas.reference +++ /dev/null @@ -1,4 +0,0 @@ -arrayMap(lambda(tuple(_lambda1_arg0), arrayElement(_lambda1_arg0, _lambda0_arg1)), arr3) -arrayMap(lambda(tuple(_lambda0_arg0, _lambda0_arg1), plus(_lambda0_arg0, arrayMap(lambda(tuple(_lambda1_arg0), arrayElement(_lambda1_arg0, _lambda0_arg1)), arr3))), arr1, arr2) - -SELECT arrayMap((_lambda0_arg0, _lambda0_arg1) -> (_lambda0_arg0 + arrayMap(_lambda1_arg0 -> _lambda1_arg0[_lambda0_arg1], arr3)), arr1, arr2) diff --git a/dbms/src/Analyzers/tests/analyze_lambdas.sh b/dbms/src/Analyzers/tests/analyze_lambdas.sh deleted file mode 100755 index 2ee03340bae..00000000000 --- a/dbms/src/Analyzers/tests/analyze_lambdas.sh +++ /dev/null @@ -1,3 +0,0 @@ -#!/bin/sh - -echo "SELECT arrayMap((x, y) -> x + arrayMap(x -> x[y], arr3), arr1, arr2)" | ./analyze_lambdas diff --git a/dbms/src/Analyzers/tests/analyze_result_of_query.cpp b/dbms/src/Analyzers/tests/analyze_result_of_query.cpp deleted file mode 100644 index 16d26bddf4b..00000000000 --- a/dbms/src/Analyzers/tests/analyze_result_of_query.cpp +++ /dev/null @@ -1,53 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -/// Parses query from stdin and print names and types of result columns. - -int main(int, char **) -try -{ - using namespace DB; - - ReadBufferFromFileDescriptor in(STDIN_FILENO); - WriteBufferFromFileDescriptor out(STDOUT_FILENO); - - String query; - readStringUntilEOF(query, in); - - ParserSelectQuery parser; - ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0); - - Context context = Context::createGlobal(); - - auto system_database = std::make_shared("system"); - context.addDatabase("system", system_database); - context.setCurrentDatabase("system"); - system_database->attachTable("one", StorageSystemOne::create("one")); - system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false)); - - ExecuteTableFunctions execute_table_functions; - execute_table_functions.process(ast, context); - - AnalyzeResultOfQuery analyzer; - analyzer.process(ast, context, execute_table_functions); - - analyzer.dump(out); - - return 0; -} -catch (...) -{ - std::cerr << DB::getCurrentExceptionMessage(true) << "\n"; - return 1; -} diff --git a/dbms/src/Analyzers/tests/analyze_result_of_query.reference b/dbms/src/Analyzers/tests/analyze_result_of_query.reference deleted file mode 100644 index 42eb3325db5..00000000000 --- a/dbms/src/Analyzers/tests/analyze_result_of_query.reference +++ /dev/null @@ -1 +0,0 @@ -1 UInt8 ColumnConst 1, plus(2, 3) UInt16 ColumnConst 1, x FixedString(20) ColumnConst 1, y String ColumnConst 1, z Int16 nullptr \ No newline at end of file diff --git a/dbms/src/Analyzers/tests/analyze_result_of_query.sh b/dbms/src/Analyzers/tests/analyze_result_of_query.sh deleted file mode 100755 index 03e4da15605..00000000000 --- a/dbms/src/Analyzers/tests/analyze_result_of_query.sh +++ /dev/null @@ -1,3 +0,0 @@ -#!/bin/sh - -echo "SELECT 1, 2 + 3, toFixedString('Hello, world', 20) AS x, ('Hello', 1).1 AS y, z FROM system.numbers WHERE (arrayJoin([-1, 1]) AS z) = 1" | ./analyze_result_of_query diff --git a/dbms/src/Analyzers/tests/collect_aliases.cpp b/dbms/src/Analyzers/tests/collect_aliases.cpp deleted file mode 100644 index 3666793ce61..00000000000 --- a/dbms/src/Analyzers/tests/collect_aliases.cpp +++ /dev/null @@ -1,36 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include - - -/// Parses query from stdin and print found aliases. - -int main(int, char **) -try -{ - using namespace DB; - - ReadBufferFromFileDescriptor in(STDIN_FILENO); - WriteBufferFromFileDescriptor out(STDOUT_FILENO); - - String query; - readStringUntilEOF(query, in); - - ParserSelectQuery parser; - ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0); - - CollectAliases collector; - collector.process(ast); - collector.dump(out); - - return 0; -} -catch (...) -{ - std::cerr << DB::getCurrentExceptionMessage(true) << "\n"; - return 1; -} diff --git a/dbms/src/Analyzers/tests/collect_aliases.reference b/dbms/src/Analyzers/tests/collect_aliases.reference deleted file mode 100644 index 014a598bc49..00000000000 --- a/dbms/src/Analyzers/tests/collect_aliases.reference +++ /dev/null @@ -1,8 +0,0 @@ -a -> (array join) arr AS a -b -> (array join) (arr1 AS z) + 1 AS b -ghi -> (expression) def AS ghi -t -> (table) d.t AS t -u -> (table) (SELECT 1 AS xxx) AS u -x -> (expression) 1 AS x -y -> (expression) x + 2 AS y -z -> (expression) arr1 AS z diff --git a/dbms/src/Analyzers/tests/collect_aliases.sh b/dbms/src/Analyzers/tests/collect_aliases.sh deleted file mode 100755 index 7473d22b8fc..00000000000 --- a/dbms/src/Analyzers/tests/collect_aliases.sh +++ /dev/null @@ -1,3 +0,0 @@ -#!/bin/sh - -echo "SELECT 1 AS x, x + 2 AS y FROM d.t AS t INNER JOIN (SELECT 1 AS xxx) AS u USING (abc, def AS ghi) ARRAY JOIN arr AS a, (arr1 AS z) + 1 AS b" | ./collect_aliases diff --git a/dbms/src/Analyzers/tests/collect_tables.cpp b/dbms/src/Analyzers/tests/collect_tables.cpp deleted file mode 100644 index 201880e7e54..00000000000 --- a/dbms/src/Analyzers/tests/collect_tables.cpp +++ /dev/null @@ -1,56 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -/// Parses query from stdin and print found tables. - -int main(int, char **) -try -{ - using namespace DB; - - ReadBufferFromFileDescriptor in(STDIN_FILENO); - WriteBufferFromFileDescriptor out(STDOUT_FILENO); - - String query; - readStringUntilEOF(query, in); - - ParserSelectQuery parser; - ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0); - - Context context = Context::createGlobal(); - - auto system_database = std::make_shared("system"); - context.addDatabase("system", system_database); - context.setCurrentDatabase("system"); - system_database->attachTable("one", StorageSystemOne::create("one")); - system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false)); - - CollectAliases collect_aliases; - collect_aliases.process(ast); - - ExecuteTableFunctions execute_table_functions; - execute_table_functions.process(ast, context); - - CollectTables collect_tables; - collect_tables.process(ast, context, collect_aliases, execute_table_functions); - collect_tables.dump(out); - - return 0; -} -catch (...) -{ - std::cerr << DB::getCurrentExceptionMessage(true) << "\n"; - return 1; -} diff --git a/dbms/src/Analyzers/tests/optimize_group_order_limit_by.cpp b/dbms/src/Analyzers/tests/optimize_group_order_limit_by.cpp deleted file mode 100644 index 5680f1de5c0..00000000000 --- a/dbms/src/Analyzers/tests/optimize_group_order_limit_by.cpp +++ /dev/null @@ -1,80 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -/// Parses query from stdin and print same query with optimized GROUP BY, ORDER BY, LIMIT BY. - -int main(int, char **) -try -{ - using namespace DB; - - ReadBufferFromFileDescriptor in(STDIN_FILENO); - WriteBufferFromFileDescriptor out(STDOUT_FILENO); - - String query; - readStringUntilEOF(query, in); - - ParserSelectQuery parser; - ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0); - - Context context = Context::createGlobal(); - - auto system_database = std::make_shared("system"); - context.addDatabase("system", system_database); - system_database->attachTable("one", StorageSystemOne::create("one")); - system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false)); - context.setCurrentDatabase("system"); - - AnalyzeLambdas analyze_lambdas; - analyze_lambdas.process(ast); - - CollectAliases collect_aliases; - collect_aliases.process(ast); - - ExecuteTableFunctions execute_table_functions; - execute_table_functions.process(ast, context); - - CollectTables collect_tables; - collect_tables.process(ast, context, collect_aliases, execute_table_functions); - - AnalyzeColumns analyze_columns; - analyze_columns.process(ast, collect_aliases, collect_tables); - - TypeAndConstantInference inference; - inference.process(ast, context, collect_aliases, analyze_columns, analyze_lambdas, execute_table_functions); - - TranslatePositionalArguments translation; - translation.process(ast); - - OptimizeGroupOrderLimitBy optimizer; - optimizer.process(ast, inference); - - formatAST(*ast, std::cout, false); - std::cout << "\n"; - return 0; - - return 0; -} -catch (...) -{ - std::cerr << DB::getCurrentExceptionMessage(true) << "\n"; - return 1; -} diff --git a/dbms/src/Analyzers/tests/optimize_group_order_limit_by.reference b/dbms/src/Analyzers/tests/optimize_group_order_limit_by.reference deleted file mode 100644 index 3dc91598468..00000000000 --- a/dbms/src/Analyzers/tests/optimize_group_order_limit_by.reference +++ /dev/null @@ -1,19 +0,0 @@ -SELECT - number, - materialize('abc') -FROM -( - SELECT - number, - 10 AS b - FROM system.numbers -) -GROUP BY number - -SELECT number -FROM system.numbers -GROUP BY number -ORDER BY - number ASC, - rand() ASC, - toString(rand()) ASC COLLATE 'tr' diff --git a/dbms/src/Analyzers/tests/optimize_group_order_limit_by.sh b/dbms/src/Analyzers/tests/optimize_group_order_limit_by.sh deleted file mode 100755 index 37219cd164f..00000000000 --- a/dbms/src/Analyzers/tests/optimize_group_order_limit_by.sh +++ /dev/null @@ -1,5 +0,0 @@ -#!/bin/sh - -echo "SELECT number, materialize('abc') FROM (SELECT number, 10 AS b FROM system.numbers) GROUP BY number, toString(number + 1), number + number, 1, 2, 'Hello', b" | ./optimize_group_order_limit_by -echo -echo "SELECT number FROM system.numbers GROUP BY 1 ORDER BY number, 'hello' DESC COLLATE 'tr', number + 1, rand(), identity(number * 2, rand()), toString(rand()) COLLATE 'tr'" | ./optimize_group_order_limit_by diff --git a/dbms/src/Analyzers/tests/translate_positional_arguments.cpp b/dbms/src/Analyzers/tests/translate_positional_arguments.cpp deleted file mode 100644 index 8d1b9ab6c60..00000000000 --- a/dbms/src/Analyzers/tests/translate_positional_arguments.cpp +++ /dev/null @@ -1,37 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include - - -/// Parses query from stdin and print same query with translated positional arguments. - -int main(int, char **) -try -{ - using namespace DB; - - ReadBufferFromFileDescriptor in(STDIN_FILENO); - - String query; - readStringUntilEOF(query, in); - - ParserSelectQuery parser; - ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0); - - TranslatePositionalArguments translator; - translator.process(ast); - - formatAST(*ast, std::cout, false); - std::cout << "\n"; - return 0; -} -catch (...) -{ - std::cerr << DB::getCurrentExceptionMessage(true) << "\n"; - return 1; -} diff --git a/dbms/src/Analyzers/tests/translate_positional_arguments.reference b/dbms/src/Analyzers/tests/translate_positional_arguments.reference deleted file mode 100644 index dc1e92a4f3d..00000000000 --- a/dbms/src/Analyzers/tests/translate_positional_arguments.reference +++ /dev/null @@ -1,8 +0,0 @@ -SELECT - abc, - def + 1, - count() -GROUP BY - abc, - def + 1 -ORDER BY abc DESC diff --git a/dbms/src/Analyzers/tests/translate_positional_arguments.sh b/dbms/src/Analyzers/tests/translate_positional_arguments.sh deleted file mode 100755 index f9f1d5a12bc..00000000000 --- a/dbms/src/Analyzers/tests/translate_positional_arguments.sh +++ /dev/null @@ -1,3 +0,0 @@ -#!/bin/sh - -echo "SELECT abc, def + 1, count() GROUP BY 1, 2 ORDER BY 1 DESC" | ./translate_positional_arguments diff --git a/dbms/src/Analyzers/tests/type_and_constant_inference.cpp b/dbms/src/Analyzers/tests/type_and_constant_inference.cpp deleted file mode 100644 index 372c7240ec3..00000000000 --- a/dbms/src/Analyzers/tests/type_and_constant_inference.cpp +++ /dev/null @@ -1,78 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -/// Parses query from stdin and print data types of expressions; and for constant expressions, print its values. - -int main(int, char **) -try -{ - using namespace DB; - - registerFunctions(); - registerAggregateFunctions(); - registerTableFunctions(); - - ReadBufferFromFileDescriptor in(STDIN_FILENO); - WriteBufferFromFileDescriptor out(STDOUT_FILENO); - - String query; - readStringUntilEOF(query, in); - - ParserSelectQuery parser; - ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0); - - Context context = Context::createGlobal(); - - auto system_database = std::make_shared("system"); - context.addDatabase("system", system_database); - system_database->attachTable("one", StorageSystemOne::create("one")); - system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false)); - context.setCurrentDatabase("system"); - - AnalyzeLambdas analyze_lambdas; - analyze_lambdas.process(ast); - - CollectAliases collect_aliases; - collect_aliases.process(ast); - - ExecuteTableFunctions execute_table_functions; - execute_table_functions.process(ast, context); - - CollectTables collect_tables; - collect_tables.process(ast, context, collect_aliases, execute_table_functions); - - AnalyzeColumns analyze_columns; - analyze_columns.process(ast, collect_aliases, collect_tables); - - TypeAndConstantInference inference; - inference.process(ast, context, collect_aliases, analyze_columns, analyze_lambdas, execute_table_functions); - - inference.dump(out); - out.next(); - - return 0; -} -catch (...) -{ - std::cerr << DB::getCurrentExceptionMessage(true) << "\n"; - return 1; -} diff --git a/dbms/src/Analyzers/tests/type_and_constant_inference.reference b/dbms/src/Analyzers/tests/type_and_constant_inference.reference deleted file mode 100644 index 882fbdc4e7e..00000000000 --- a/dbms/src/Analyzers/tests/type_and_constant_inference.reference +++ /dev/null @@ -1,34 +0,0 @@ -' ' -> String = ' '. AST: ' ' -'Hello' -> String = 'Hello'. AST: 'Hello' -'Hello, world' -> String = 'Hello, world'. AST: 'Hello, world' -'World' -> String = 'World'. AST: 'World' -0.5 -> Float64 = 0.5. AST: 0.5 -1 -> UInt8 = 1. AST: 1 -20 -> UInt8 = 20. AST: 20 -[1, -1, 0.1] -> Array(Float64) = [1., -1., 0.1]. AST: [1, -1, 0.1] -concat('Hello', ' ', 'World') -> String = 'Hello World'. AST: concat('Hello', ' ', 'World') AS world -count() -> UInt64. AST: count() -e -> Float64 = 2.718281828459045. AST: e() AS e -e() -> Float64 = 2.718281828459045. AST: e() AS e -pi() -> Float64 = 3.141592653589793. AST: pi() -plus(e, pi()) -> Float64 = 5.859874482048838. AST: e + pi() -quantileTiming(0.5)(1) -> Float32. AST: quantileTiming(0.5)(1) -rand() -> UInt32. AST: rand() -sleep(1) -> UInt8. AST: sleep(1) -sum(e) -> Float64. AST: sum(e) -system.numbers.number -> UInt64. AST: system.numbers.number -toFixedString('Hello, world', 20) -> FixedString(20) = 'Hello, world\0\0\0\0\0\0\0\0'. AST: toFixedString('Hello, world', 20) -tuple('Hello', 1) -> Tuple(String, UInt8) = ('Hello', 1). AST: ('Hello', 1) -tupleElement(tuple('Hello', 1), 1) -> String = 'Hello'. AST: ('Hello', 1).1 - -t.x -> UInt8 = 1. AST: t.x - -x -> UInt8 = 1. AST: x - -1 -> UInt8 = 1. AST: 1 -t.x -> UInt8 = 1. AST: t.x -x -> UInt8 = 1. AST: x - -x -> Tuple(UInt8, UInt8) = (1, 2). AST: x -y -> Tuple(UInt8, UInt8) = (3, 4). AST: y -z -> String = 'Hello, world'. AST: z diff --git a/dbms/src/Analyzers/tests/type_and_constant_inference.sh b/dbms/src/Analyzers/tests/type_and_constant_inference.sh deleted file mode 100755 index 58af97ebf45..00000000000 --- a/dbms/src/Analyzers/tests/type_and_constant_inference.sh +++ /dev/null @@ -1,26 +0,0 @@ -#!/bin/sh - -echo "SELECT - 1, - *, - sleep(1), - rand(), - 'Hello' || ' ' || 'World' AS world, - e() AS e, - e + pi(), - count(*), - sum(e), - quantileTiming(0.5)(1), - [1, -1, 0.1], - toFixedString('Hello, world', 20), - ('Hello', 1).1 -FROM system.numbers" | ./type_and_constant_inference - -echo -echo "SELECT t.x FROM (SELECT 1 AS x) AS t" | ./type_and_constant_inference -echo -echo "SELECT x FROM (SELECT 1 AS x)" | ./type_and_constant_inference -echo -echo "SELECT t.x, x, 1 FROM (SELECT 1 AS x) AS t" | ./type_and_constant_inference -echo -echo "SELECT *, z FROM (SELECT (1, 2) AS x, (SELECT 3, 4) AS y), (SELECT 'Hello, world' AS z)" | ./type_and_constant_inference diff --git a/dbms/src/CMakeLists.txt b/dbms/src/CMakeLists.txt index 30bd7c134cc..f6fa96e1d47 100644 --- a/dbms/src/CMakeLists.txt +++ b/dbms/src/CMakeLists.txt @@ -12,5 +12,4 @@ add_subdirectory (Interpreters) add_subdirectory (AggregateFunctions) add_subdirectory (Client) add_subdirectory (TableFunctions) -add_subdirectory (Analyzers) add_subdirectory (Formats) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index 9ce717a0101..0c714e78af0 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -34,7 +34,7 @@ class ASTSelectQuery; struct SyntaxAnalyzerResult; using SyntaxAnalyzerResultPtr = std::shared_ptr; -/// ExpressionAnalyzers sources, intermediates and results. It splits data and logic, allows to test them separately. +/// ExpressionAnalyzer sources, intermediates and results. It splits data and logic, allows to test them separately. /// If you are not writing a test you probably don't need it. Use ExpressionAnalyzer itself. struct ExpressionAnalyzerData { diff --git a/dbms/src/Interpreters/tests/expression_analyzer.cpp b/dbms/src/Interpreters/tests/expression_analyzer.cpp index 3079e1b8c21..8fb5fd88c19 100644 --- a/dbms/src/Interpreters/tests/expression_analyzer.cpp +++ b/dbms/src/Interpreters/tests/expression_analyzer.cpp @@ -10,10 +10,6 @@ #include #include -#include -#include -#include - #include #include @@ -36,14 +32,7 @@ struct TestEntry ASTPtr ast = parse(query); auto res = SyntaxAnalyzer(context, {}).analyze(ast, source_columns, required_result_columns); - - if (!checkAliases(*res)) - { - collectWithAnalysers(context, ast); - return false; - } - - return true; + return checkAliases(*res); } private: @@ -80,22 +69,6 @@ private: return ast; throw Exception(message); } - - void collectWithAnalysers(const Context & context, ASTPtr ast) const - { - ReadBufferFromFileDescriptor in(STDIN_FILENO); - WriteBufferFromFileDescriptor out(STDOUT_FILENO); - - CollectAliases collect_aliases; - collect_aliases.process(ast); - - ExecuteTableFunctions execute_table_functions; - execute_table_functions.process(ast, context); - - CollectTables collect_tables; - collect_tables.process(ast, context, collect_aliases, execute_table_functions); - collect_tables.dump(out); - } }; From 97d9ce5193e59fe64b40748696b87616d9326cc1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 12 Nov 2018 21:19:16 +0300 Subject: [PATCH 27/46] Fix WITH with single alias. #2465 --- dbms/src/Interpreters/ActionsVisitor.cpp | 5 ++++- dbms/src/Interpreters/ExpressionActions.cpp | 4 ++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index 7d4ee8e0bc4..fe5a71124b4 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -264,7 +264,7 @@ void ActionsVisitor::visit(const ASTPtr & ast) && projection_manipulator->tryToGetFromUpperProjection(getColumnName())) return; - if (typeid_cast(ast.get())) + if (auto * identifier = typeid_cast(ast.get())) { if (!only_consts && !projection_manipulator->tryToGetFromUpperProjection(getColumnName())) { @@ -279,6 +279,9 @@ void ActionsVisitor::visit(const ASTPtr & ast) if (found) throw Exception("Column " + getColumnName() + " is not under aggregate function and not in GROUP BY.", ErrorCodes::NOT_AN_AGGREGATE); + + if (identifier->prefer_alias_to_column_name && !identifier->alias.empty()) + actions_stack.addAction(ExpressionAction::addAliases({{identifier->name, identifier->alias}})); } } else if (ASTFunction * node = typeid_cast(ast.get())) diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index e06a8d11aeb..a2c0a9ceb46 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -691,6 +691,10 @@ void ExpressionActions::addImpl(ExpressionAction action, Names & new_names) action.result_type = action.function_base->getReturnType(); } + if (action.type == ExpressionAction::ADD_ALIASES) + for (const auto & name_with_alias : action.projection) + new_names.emplace_back(name_with_alias.second); + action.prepare(sample_block, settings); actions.push_back(action); } From b1886e1272c0c257399f26069c5c64015a1eb491 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 12 Nov 2018 21:26:48 +0300 Subject: [PATCH 28/46] Added test. #2465 --- .../queries/0_stateless/00753_with_with_single_alias.reference | 2 ++ dbms/tests/queries/0_stateless/00753_with_with_single_alias.sql | 2 ++ 2 files changed, 4 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00753_with_with_single_alias.reference create mode 100644 dbms/tests/queries/0_stateless/00753_with_with_single_alias.sql diff --git a/dbms/tests/queries/0_stateless/00753_with_with_single_alias.reference b/dbms/tests/queries/0_stateless/00753_with_with_single_alias.reference new file mode 100644 index 00000000000..0d66ea1aee9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00753_with_with_single_alias.reference @@ -0,0 +1,2 @@ +0 +1 diff --git a/dbms/tests/queries/0_stateless/00753_with_with_single_alias.sql b/dbms/tests/queries/0_stateless/00753_with_with_single_alias.sql new file mode 100644 index 00000000000..ff379140947 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00753_with_with_single_alias.sql @@ -0,0 +1,2 @@ +WITH dummy AS myName SELECT myName FROM system.one; +WITH dummy AS myName SELECT myName + 1 FROM system.one; From 986d490435ce2a9619b2c9b8bf794fe573a33918 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 12 Nov 2018 21:33:39 +0300 Subject: [PATCH 29/46] Comments. --- dbms/src/Interpreters/ActionsVisitor.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index fe5a71124b4..c781da56bcc 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -280,6 +280,7 @@ void ActionsVisitor::visit(const ASTPtr & ast) throw Exception("Column " + getColumnName() + " is not under aggregate function and not in GROUP BY.", ErrorCodes::NOT_AN_AGGREGATE); + /// Special check for WITH statement alias. Add alias action to be able to use this alias. if (identifier->prefer_alias_to_column_name && !identifier->alias.empty()) actions_stack.addAction(ExpressionAction::addAliases({{identifier->name, identifier->alias}})); } From c997f0b56a059f6dd6effb0e4bcdf7aeecf67d49 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 13 Nov 2018 20:18:38 +0300 Subject: [PATCH 30/46] Added a test for ATTACHing parts before ALTER (just in case) [#CLICKHOUSE-3] --- .../0_stateless/00753_alter_attach.reference | 12 +++++++++++ .../0_stateless/00753_alter_attach.sql | 20 +++++++++++++++++++ 2 files changed, 32 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00753_alter_attach.reference create mode 100644 dbms/tests/queries/0_stateless/00753_alter_attach.sql diff --git a/dbms/tests/queries/0_stateless/00753_alter_attach.reference b/dbms/tests/queries/0_stateless/00753_alter_attach.reference new file mode 100644 index 00000000000..007b99d4748 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00753_alter_attach.reference @@ -0,0 +1,12 @@ +1 1 +2 1 +3 1 +4 2 Hello +5 2 World +1 1 +2 1 +3 1 +4 2 +5 2 +6 3 +7 3 diff --git a/dbms/tests/queries/0_stateless/00753_alter_attach.sql b/dbms/tests/queries/0_stateless/00753_alter_attach.sql new file mode 100644 index 00000000000..10e53118c3b --- /dev/null +++ b/dbms/tests/queries/0_stateless/00753_alter_attach.sql @@ -0,0 +1,20 @@ +DROP TABLE IF EXISTS test.alter_attach; +CREATE TABLE test.alter_attach (x UInt64, p UInt8) ENGINE = MergeTree ORDER BY tuple() PARTITION BY p; +INSERT INTO test.alter_attach VALUES (1, 1), (2, 1), (3, 1); + +ALTER TABLE test.alter_attach DETACH PARTITION 1; + +ALTER TABLE test.alter_attach ADD COLUMN s String; +INSERT INTO test.alter_attach VALUES (4, 2, 'Hello'), (5, 2, 'World'); + +ALTER TABLE test.alter_attach ATTACH PARTITION 1; +SELECT * FROM test.alter_attach ORDER BY x; + +ALTER TABLE test.alter_attach DETACH PARTITION 2; +ALTER TABLE test.alter_attach DROP COLUMN s; +INSERT INTO test.alter_attach VALUES (6, 3), (7, 3); + +ALTER TABLE test.alter_attach ATTACH PARTITION 2; +SELECT * FROM test.alter_attach ORDER BY x; + +DROP TABLE test.alter_attach; From 062a56b9fc66c33a441be2cdea81444fe598dad9 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 13 Nov 2018 20:52:15 +0300 Subject: [PATCH 31/46] Update external_dicts_dict_sources.md --- docs/ru/query_language/dicts/external_dicts_dict_sources.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/query_language/dicts/external_dicts_dict_sources.md b/docs/ru/query_language/dicts/external_dicts_dict_sources.md index e1fa4578ac8..ca9a6d2a43c 100644 --- a/docs/ru/query_language/dicts/external_dicts_dict_sources.md +++ b/docs/ru/query_language/dicts/external_dicts_dict_sources.md @@ -28,8 +28,8 @@ - [Локальный файл](#dicts-external_dicts_dict_sources-local_file) - [Исполняемый файл](#dicts-external_dicts_dict_sources-executable) - [HTTP(s)](#dicts-external_dicts_dict_sources-http) -- [ODBC](#dicts-external_dicts_dict_sources-odbc) - СУБД: + - [ODBC](#dicts-external_dicts_dict_sources-odbc) - [MySQL](#dicts-external_dicts_dict_sources-mysql) - [ClickHouse](#dicts-external_dicts_dict_sources-clickhouse) - [MongoDB](#dicts-external_dicts_dict_sources-mongodb) From 55a576f762dfc9223b98e78a62e053e507879c2f Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 13 Nov 2018 20:52:36 +0300 Subject: [PATCH 32/46] Update external_dicts_dict_sources.md --- .../dicts/external_dicts_dict_sources.md | 864 +++++++++--------- 1 file changed, 432 insertions(+), 432 deletions(-) diff --git a/docs/en/query_language/dicts/external_dicts_dict_sources.md b/docs/en/query_language/dicts/external_dicts_dict_sources.md index 023f14f5a70..01952e18717 100644 --- a/docs/en/query_language/dicts/external_dicts_dict_sources.md +++ b/docs/en/query_language/dicts/external_dicts_dict_sources.md @@ -1,433 +1,433 @@ - - -# Sources of External Dictionaries - -An external dictionary can be connected from many different sources. - -The configuration looks like this: - -```xml - - - ... - - - - - - ... - - ... - -``` - -The source is configured in the `source` section. - -Types of sources (`source_type`): - -- [Local file](#dicts-external_dicts_dict_sources-local_file) -- [Executable file](#dicts-external_dicts_dict_sources-executable) -- [HTTP(s)](#dicts-external_dicts_dict_sources-http) -- [ODBC](#dicts-external_dicts_dict_sources-odbc) -- DBMS - - [MySQL](#dicts-external_dicts_dict_sources-mysql) - - [ClickHouse](#dicts-external_dicts_dict_sources-clickhouse) - - [MongoDB](#dicts-external_dicts_dict_sources-mongodb) - - - -## Local File - -Example of settings: - -```xml - - - /opt/dictionaries/os.tsv - TabSeparated - - -``` - -Setting fields: - -- `path` – The absolute path to the file. -- `format` – The file format. All the formats described in "[Formats](../../interfaces/formats.md#formats)" are supported. - - - -## Executable File - -Working with executable files depends on [how the dictionary is stored in memory](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request to the executable file's `STDIN`. - -Example of settings: - -```xml - - - cat /opt/dictionaries/os.tsv - TabSeparated - - -``` - -Setting fields: - -- `command` – The absolute path to the executable file, or the file name (if the program directory is written to `PATH`). -- `format` – The file format. All the formats described in "[Formats](../../interfaces/formats.md#formats)" are supported. - - - -## HTTP(s) - -Working with an HTTP(s) server depends on [how the dictionary is stored in memory](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request via the `POST` method. - -Example of settings: - -```xml - - - http://[::1]/os.tsv - TabSeparated - - -``` - -In order for ClickHouse to access an HTTPS resource, you must [configure openSSL](../../operations/server_settings/settings.md#server_settings-openSSL) in the server configuration. - -Setting fields: - -- `url` – The source URL. -- `format` – The file format. All the formats described in "[Formats](../../interfaces/formats.md#formats)" are supported. - - - -## ODBC - -You can use this method to connect any database that has an ODBC driver. - -Example of settings: - -```xml - - DatabaseName - ShemaName.TableName
- DSN=some_parameters - SQL_QUERY -
-``` - -Setting fields: - -- `db` – Name of the database. Omit it if the database name is set in the `` parameters. -- `table` – Name of the table and schema if exists. -- `connection_string` – Connection string. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](external_dicts_dict_lifetime.md#dicts-external_dicts_dict_lifetime). - -ClickHouse receives quoting symbols from ODBC-driver and quote all settings in queries to driver, so it's necessary to set table name accordingly to table name case in database. - -### Known vulnerability of the ODBC dictionary functionality - -!!! attention - When connecting to the database through the ODBC driver connection parameter `Servername` can be substituted. In this case values of `USERNAME` and `PASSWORD` from `odbc.ini` are sent to the remote server and can be compromised. - -**Example of insecure use** - -Let's configure unixODBC for PostgreSQL. Content of `/etc/odbc.ini`: - -``` -[gregtest] -Driver = /usr/lib/psqlodbca.so -Servername = localhost -PORT = 5432 -DATABASE = test_db -#OPTION = 3 -USERNAME = test -PASSWORD = test -``` - -If you then make a query such as - -``` -SELECT * FROM odbc('DSN=gregtest;Servername=some-server.com', 'test_db'); -``` - -ODBC driver will send values of `USERNAME` and `PASSWORD` from `odbc.ini` to `some-server.com`. - -### Example of Connecting PostgreSQL - -Ubuntu OS. - -Installing unixODBC and the ODBC driver for PostgreSQL: - - sudo apt-get install -y unixodbc odbcinst odbc-postgresql - -Configuring `/etc/odbc.ini` (or `~/.odbc.ini`): - -``` - [DEFAULT] - Driver = myconnection - - [myconnection] - Description = PostgreSQL connection to my_db - Driver = PostgreSQL Unicode - Database = my_db - Servername = 127.0.0.1 - UserName = username - Password = password - Port = 5432 - Protocol = 9.3 - ReadOnly = No - RowVersioning = No - ShowSystemTables = No - ConnSettings = -``` - -The dictionary configuration in ClickHouse: - -```xml - - - table_name - - + + +# Sources of External Dictionaries + +An external dictionary can be connected from many different sources. + +The configuration looks like this: + +```xml + + + ... + + + + + + ... + + ... + +``` + +The source is configured in the `source` section. + +Types of sources (`source_type`): + +- [Local file](#dicts-external_dicts_dict_sources-local_file) +- [Executable file](#dicts-external_dicts_dict_sources-executable) +- [HTTP(s)](#dicts-external_dicts_dict_sources-http) +- DBMS + - [MySQL](#dicts-external_dicts_dict_sources-mysql) + - [ClickHouse](#dicts-external_dicts_dict_sources-clickhouse) + - [MongoDB](#dicts-external_dicts_dict_sources-mongodb) + - [ODBC](#dicts-external_dicts_dict_sources-odbc) + + + +## Local File + +Example of settings: + +```xml + + + /opt/dictionaries/os.tsv + TabSeparated + + +``` + +Setting fields: + +- `path` – The absolute path to the file. +- `format` – The file format. All the formats described in "[Formats](../../interfaces/formats.md#formats)" are supported. + + + +## Executable File + +Working with executable files depends on [how the dictionary is stored in memory](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request to the executable file's `STDIN`. + +Example of settings: + +```xml + + + cat /opt/dictionaries/os.tsv + TabSeparated + + +``` + +Setting fields: + +- `command` – The absolute path to the executable file, or the file name (if the program directory is written to `PATH`). +- `format` – The file format. All the formats described in "[Formats](../../interfaces/formats.md#formats)" are supported. + + + +## HTTP(s) + +Working with an HTTP(s) server depends on [how the dictionary is stored in memory](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request via the `POST` method. + +Example of settings: + +```xml + + + http://[::1]/os.tsv + TabSeparated + + +``` + +In order for ClickHouse to access an HTTPS resource, you must [configure openSSL](../../operations/server_settings/settings.md#server_settings-openSSL) in the server configuration. + +Setting fields: + +- `url` – The source URL. +- `format` – The file format. All the formats described in "[Formats](../../interfaces/formats.md#formats)" are supported. + + + +## ODBC + +You can use this method to connect any database that has an ODBC driver. + +Example of settings: + +```xml + + DatabaseName + ShemaName.TableName
+ DSN=some_parameters + SQL_QUERY +
+``` + +Setting fields: + +- `db` – Name of the database. Omit it if the database name is set in the `` parameters. +- `table` – Name of the table and schema if exists. +- `connection_string` – Connection string. +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](external_dicts_dict_lifetime.md#dicts-external_dicts_dict_lifetime). + +ClickHouse receives quoting symbols from ODBC-driver and quote all settings in queries to driver, so it's necessary to set table name accordingly to table name case in database. + +### Known vulnerability of the ODBC dictionary functionality + +!!! attention + When connecting to the database through the ODBC driver connection parameter `Servername` can be substituted. In this case values of `USERNAME` and `PASSWORD` from `odbc.ini` are sent to the remote server and can be compromised. + +**Example of insecure use** + +Let's configure unixODBC for PostgreSQL. Content of `/etc/odbc.ini`: + +``` +[gregtest] +Driver = /usr/lib/psqlodbca.so +Servername = localhost +PORT = 5432 +DATABASE = test_db +#OPTION = 3 +USERNAME = test +PASSWORD = test +``` + +If you then make a query such as + +``` +SELECT * FROM odbc('DSN=gregtest;Servername=some-server.com', 'test_db'); +``` + +ODBC driver will send values of `USERNAME` and `PASSWORD` from `odbc.ini` to `some-server.com`. + +### Example of Connecting PostgreSQL + +Ubuntu OS. + +Installing unixODBC and the ODBC driver for PostgreSQL: + + sudo apt-get install -y unixodbc odbcinst odbc-postgresql + +Configuring `/etc/odbc.ini` (or `~/.odbc.ini`): + +``` + [DEFAULT] + Driver = myconnection + + [myconnection] + Description = PostgreSQL connection to my_db + Driver = PostgreSQL Unicode + Database = my_db + Servername = 127.0.0.1 + UserName = username + Password = password + Port = 5432 + Protocol = 9.3 + ReadOnly = No + RowVersioning = No + ShowSystemTables = No + ConnSettings = +``` + +The dictionary configuration in ClickHouse: + +```xml + + + table_name + + - - DSN=myconnection - postgresql_table
-
- - - 300 - 360 - - - - - - - id - - - some_column - UInt64 - 0 - - -
-
-``` - -You may need to edit `odbc.ini` to specify the full path to the library with the driver `DRIVER=/usr/local/lib/psqlodbcw.so`. - -### Example of Connecting MS SQL Server - -Ubuntu OS. - -Installing the driver: : - -``` - sudo apt-get install tdsodbc freetds-bin sqsh -``` - -Configuring the driver: : - -``` - $ cat /etc/freetds/freetds.conf - ... - - [MSSQL] - host = 192.168.56.101 - port = 1433 - tds version = 7.0 - client charset = UTF-8 - - $ cat /etc/odbcinst.ini - ... - - [FreeTDS] - Description = FreeTDS - Driver = /usr/lib/x86_64-linux-gnu/odbc/libtdsodbc.so - Setup = /usr/lib/x86_64-linux-gnu/odbc/libtdsS.so - FileUsage = 1 - UsageCount = 5 - - $ cat ~/.odbc.ini - ... - - [MSSQL] - Description = FreeTDS - Driver = FreeTDS - Servername = MSSQL - Database = test - UID = test - PWD = test - Port = 1433 -``` - -Configuring the dictionary in ClickHouse: - -```xml - - - test - - - dict
- DSN=MSSQL;UID=test;PWD=test -
- - - - 300 - 360 - - - - - - - - - k - - - s - String - - - -
-
-``` - -## DBMS - - - -### MySQL - -Example of settings: - -```xml - - - 3306 - clickhouse - qwerty - - example01-1 - 1 - - - example01-2 - 1 - - db_name - table_name
- id=10 - SQL_QUERY -
- -``` - -Setting fields: - -- `port` – The port on the MySQL server. You can specify it for all replicas, or for each one individually (inside ``). - -- `user` – Name of the MySQL user. You can specify it for all replicas, or for each one individually (inside ``). - -- `password` – Password of the MySQL user. You can specify it for all replicas, or for each one individually (inside ``). - -- `replica` – Section of replica configurations. There can be multiple sections. - - `replica/host` – The MySQL host. - - \* `replica/priority` – The replica priority. When attempting to connect, ClickHouse traverses the replicas in order of priority. The lower the number, the higher the priority. - -- `db` – Name of the database. - -- `table` – Name of the table. - -- `where ` – The selection criteria. Optional parameter. - -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](external_dicts_dict_lifetime.md#dicts-external_dicts_dict_lifetime). - -MySQL can be connected on a local host via sockets. To do this, set `host` and `socket`. - -Example of settings: - -```xml - - - localhost - /path/to/socket/file.sock - clickhouse - qwerty - db_name - table_name
- id=10 - SQL_QUERY -
- -``` - - - -### ClickHouse - -Example of settings: - -```xml - - - example01-01-1 - 9000 - default - - default - ids
- id=10 -
- -``` - -Setting fields: - -- `host` – The ClickHouse host. If it is a local host, the query is processed without any network activity. To improve fault tolerance, you can create a [Distributed](../../operations/table_engines/distributed.md#table_engines-distributed) table and enter it in subsequent configurations. -- `port` – The port on the ClickHouse server. -- `user` – Name of the ClickHouse user. -- `password` – Password of the ClickHouse user. -- `db` – Name of the database. -- `table` – Name of the table. -- `where ` – The selection criteria. May be omitted. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](external_dicts_dict_lifetime.md#dicts-external_dicts_dict_lifetime). - - - -### MongoDB - -Example of settings: - -```xml - - - localhost - 27017 - - - test - dictionary_source - - -``` - -Setting fields: - -- `host` – The MongoDB host. -- `port` – The port on the MongoDB server. -- `user` – Name of the MongoDB user. -- `password` – Password of the MongoDB user. -- `db` – Name of the database. -- `collection` – Name of the collection. - -[Original article](https://clickhouse.yandex/docs/en/query_language/dicts/external_dicts_dict_sources/) + + DSN=myconnection + postgresql_table
+
+ + + 300 + 360 + + + + + + + id + + + some_column + UInt64 + 0 + + +
+
+``` + +You may need to edit `odbc.ini` to specify the full path to the library with the driver `DRIVER=/usr/local/lib/psqlodbcw.so`. + +### Example of Connecting MS SQL Server + +Ubuntu OS. + +Installing the driver: : + +``` + sudo apt-get install tdsodbc freetds-bin sqsh +``` + +Configuring the driver: : + +``` + $ cat /etc/freetds/freetds.conf + ... + + [MSSQL] + host = 192.168.56.101 + port = 1433 + tds version = 7.0 + client charset = UTF-8 + + $ cat /etc/odbcinst.ini + ... + + [FreeTDS] + Description = FreeTDS + Driver = /usr/lib/x86_64-linux-gnu/odbc/libtdsodbc.so + Setup = /usr/lib/x86_64-linux-gnu/odbc/libtdsS.so + FileUsage = 1 + UsageCount = 5 + + $ cat ~/.odbc.ini + ... + + [MSSQL] + Description = FreeTDS + Driver = FreeTDS + Servername = MSSQL + Database = test + UID = test + PWD = test + Port = 1433 +``` + +Configuring the dictionary in ClickHouse: + +```xml + + + test + + + dict
+ DSN=MSSQL;UID=test;PWD=test +
+ + + + 300 + 360 + + + + + + + + + k + + + s + String + + + +
+
+``` + +## DBMS + + + +### MySQL + +Example of settings: + +```xml + + + 3306 + clickhouse + qwerty + + example01-1 + 1 + + + example01-2 + 1 + + db_name + table_name
+ id=10 + SQL_QUERY +
+ +``` + +Setting fields: + +- `port` – The port on the MySQL server. You can specify it for all replicas, or for each one individually (inside ``). + +- `user` – Name of the MySQL user. You can specify it for all replicas, or for each one individually (inside ``). + +- `password` – Password of the MySQL user. You can specify it for all replicas, or for each one individually (inside ``). + +- `replica` – Section of replica configurations. There can be multiple sections. + - `replica/host` – The MySQL host. + + \* `replica/priority` – The replica priority. When attempting to connect, ClickHouse traverses the replicas in order of priority. The lower the number, the higher the priority. + +- `db` – Name of the database. + +- `table` – Name of the table. + +- `where ` – The selection criteria. Optional parameter. + +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](external_dicts_dict_lifetime.md#dicts-external_dicts_dict_lifetime). + +MySQL can be connected on a local host via sockets. To do this, set `host` and `socket`. + +Example of settings: + +```xml + + + localhost + /path/to/socket/file.sock + clickhouse + qwerty + db_name + table_name
+ id=10 + SQL_QUERY +
+ +``` + + + +### ClickHouse + +Example of settings: + +```xml + + + example01-01-1 + 9000 + default + + default + ids
+ id=10 +
+ +``` + +Setting fields: + +- `host` – The ClickHouse host. If it is a local host, the query is processed without any network activity. To improve fault tolerance, you can create a [Distributed](../../operations/table_engines/distributed.md#table_engines-distributed) table and enter it in subsequent configurations. +- `port` – The port on the ClickHouse server. +- `user` – Name of the ClickHouse user. +- `password` – Password of the ClickHouse user. +- `db` – Name of the database. +- `table` – Name of the table. +- `where ` – The selection criteria. May be omitted. +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](external_dicts_dict_lifetime.md#dicts-external_dicts_dict_lifetime). + + + +### MongoDB + +Example of settings: + +```xml + + + localhost + 27017 + + + test + dictionary_source + + +``` + +Setting fields: + +- `host` – The MongoDB host. +- `port` – The port on the MongoDB server. +- `user` – Name of the MongoDB user. +- `password` – Password of the MongoDB user. +- `db` – Name of the database. +- `collection` – Name of the collection. + +[Original article](https://clickhouse.yandex/docs/en/query_language/dicts/external_dicts_dict_sources/) From 4b0645cb25eae9ace99f02ec53e41f0ff3da7e4e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 13 Nov 2018 21:41:07 +0300 Subject: [PATCH 33/46] Fixed error in #3565 --- dbms/src/Dictionaries/CacheDictionary.cpp | 60 +++++++++---------- ...acheDictionary_createAttributeWithType.cpp | 60 +++++++++---------- .../ComplexKeyHashedDictionary.cpp | 8 +-- dbms/src/Dictionaries/FlatDictionary.cpp | 10 ++-- dbms/src/Dictionaries/HashedDictionary.cpp | 8 +-- .../Dictionaries/RangeHashedDictionary.cpp | 8 +-- dbms/src/Dictionaries/TrieDictionary.cpp | 8 +-- 7 files changed, 80 insertions(+), 82 deletions(-) diff --git a/dbms/src/Dictionaries/CacheDictionary.cpp b/dbms/src/Dictionaries/CacheDictionary.cpp index 9063c19be01..21920cbfbba 100644 --- a/dbms/src/Dictionaries/CacheDictionary.cpp +++ b/dbms/src/Dictionaries/CacheDictionary.cpp @@ -455,78 +455,78 @@ CacheDictionary::Attribute CacheDictionary::createAttributeWithType(const Attrib switch (type) { case AttributeUnderlyingType::UInt8: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(UInt8); break; case AttributeUnderlyingType::UInt16: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(UInt16); break; case AttributeUnderlyingType::UInt32: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(UInt32); break; case AttributeUnderlyingType::UInt64: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(UInt64); break; case AttributeUnderlyingType::UInt128: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(UInt128); break; case AttributeUnderlyingType::Int8: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(Int8); break; case AttributeUnderlyingType::Int16: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(Int16); break; case AttributeUnderlyingType::Int32: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(Int32); break; case AttributeUnderlyingType::Int64: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(Int64); break; case AttributeUnderlyingType::Decimal32: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(Decimal32); break; case AttributeUnderlyingType::Decimal64: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(Decimal64); break; case AttributeUnderlyingType::Decimal128: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(Decimal128); break; case AttributeUnderlyingType::Float32: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(Float32); break; case AttributeUnderlyingType::Float64: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(Float64); break; case AttributeUnderlyingType::String: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(StringRef); if (!string_arena) string_arena = std::make_unique(); diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_createAttributeWithType.cpp b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_createAttributeWithType.cpp index 52770d37c6a..60f5139067f 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_createAttributeWithType.cpp +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_createAttributeWithType.cpp @@ -10,78 +10,78 @@ ComplexKeyCacheDictionary::Attribute ComplexKeyCacheDictionary::createAttributeW switch (type) { case AttributeUnderlyingType::UInt8: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(UInt8); break; case AttributeUnderlyingType::UInt16: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(UInt16); break; case AttributeUnderlyingType::UInt32: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(UInt32); break; case AttributeUnderlyingType::UInt64: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(UInt64); break; case AttributeUnderlyingType::UInt128: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(UInt128); break; case AttributeUnderlyingType::Int8: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(Int8); break; case AttributeUnderlyingType::Int16: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(Int16); break; case AttributeUnderlyingType::Int32: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(Int32); break; case AttributeUnderlyingType::Int64: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(Int64); break; case AttributeUnderlyingType::Float32: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(Float32); break; case AttributeUnderlyingType::Float64: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(Float64); break; case AttributeUnderlyingType::Decimal32: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(Decimal32); break; case AttributeUnderlyingType::Decimal64: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(Decimal64); break; case AttributeUnderlyingType::Decimal128: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(Decimal128); break; case AttributeUnderlyingType::String: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); bytes_allocated += size * sizeof(StringRef); if (!string_arena) string_arena = std::make_unique(); diff --git a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp index 96d50d2b904..ce6186817b0 100644 --- a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp +++ b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp @@ -421,8 +421,8 @@ void ComplexKeyHashedDictionary::calculateBytesAllocated() template void ComplexKeyHashedDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) { - std::get(attribute.null_values) = null_value.get::Type>(); - std::get>(attribute.maps) = std::make_unique>(); + attribute.null_values = null_value.get::Type>(); + attribute.maps = std::make_unique>(); } ComplexKeyHashedDictionary::Attribute ComplexKeyHashedDictionary::createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value) @@ -449,8 +449,8 @@ ComplexKeyHashedDictionary::Attribute ComplexKeyHashedDictionary::createAttribut case AttributeUnderlyingType::String: { - std::get(attr.null_values) = null_value.get(); - std::get>(attr.maps) = std::make_unique>(); + attr.null_values = null_value.get(); + attr.maps = std::make_unique>(); attr.string_arena = std::make_unique(); break; } diff --git a/dbms/src/Dictionaries/FlatDictionary.cpp b/dbms/src/Dictionaries/FlatDictionary.cpp index 78ebc43043e..9b3394fc2d6 100644 --- a/dbms/src/Dictionaries/FlatDictionary.cpp +++ b/dbms/src/Dictionaries/FlatDictionary.cpp @@ -440,10 +440,9 @@ void FlatDictionary::calculateBytesAllocated() template void FlatDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) { - const auto & null_value_ref = std::get(attribute.null_values) = - null_value.get::Type>(); - std::get>(attribute.arrays) = - std::make_unique>(initial_array_size, null_value_ref); + attribute.null_values = null_value.get::Type>(); + const auto & null_value_ref = std::get(attribute.null_values); + attribute.arrays = std::make_unique>(initial_array_size, null_value_ref); } template <> @@ -454,8 +453,7 @@ void FlatDictionary::createAttributeImpl(Attribute & attribute, const Fi const String & string = null_value.get::Type>(); const auto string_in_arena = attribute.string_arena->insert(string.data(), string.size()); null_value_ref = StringRef{string_in_arena, string.size()}; - std::get>(attribute.arrays) = - std::make_unique>(initial_array_size, null_value_ref); + attribute.arrays = std::make_unique>(initial_array_size, null_value_ref); } diff --git a/dbms/src/Dictionaries/HashedDictionary.cpp b/dbms/src/Dictionaries/HashedDictionary.cpp index e24bd91e6db..fa78d8c6755 100644 --- a/dbms/src/Dictionaries/HashedDictionary.cpp +++ b/dbms/src/Dictionaries/HashedDictionary.cpp @@ -430,8 +430,8 @@ void HashedDictionary::calculateBytesAllocated() template void HashedDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) { - std::get(attribute.null_values) = null_value.get::Type>(); - std::get>(attribute.maps) = std::make_unique>(); + attribute.null_values = null_value.get::Type>(); + attribute.maps = std::make_unique>(); } HashedDictionary::Attribute HashedDictionary::createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value) @@ -458,8 +458,8 @@ HashedDictionary::Attribute HashedDictionary::createAttributeWithType(const Attr case AttributeUnderlyingType::String: { - std::get(attr.null_values) = null_value.get(); - std::get>(attr.maps) = std::make_unique>(); + attr.null_values = null_value.get(); + attr.maps = std::make_unique>(); attr.string_arena = std::make_unique(); break; } diff --git a/dbms/src/Dictionaries/RangeHashedDictionary.cpp b/dbms/src/Dictionaries/RangeHashedDictionary.cpp index 91d8f48e825..b0752c0d877 100644 --- a/dbms/src/Dictionaries/RangeHashedDictionary.cpp +++ b/dbms/src/Dictionaries/RangeHashedDictionary.cpp @@ -261,8 +261,8 @@ void RangeHashedDictionary::calculateBytesAllocated() template void RangeHashedDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) { - std::get(attribute.null_values) = null_value.get::Type>(); - std::get>(attribute.maps) = std::make_unique>(); + attribute.null_values = null_value.get::Type>(); + attribute.maps = std::make_unique>(); } RangeHashedDictionary::Attribute RangeHashedDictionary::createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value) @@ -289,8 +289,8 @@ RangeHashedDictionary::Attribute RangeHashedDictionary::createAttributeWithType( case AttributeUnderlyingType::String: { - std::get(attr.null_values) = null_value.get(); - std::get>(attr.maps) = std::make_unique>(); + attr.null_values = null_value.get(); + attr.maps = std::make_unique>(); attr.string_arena = std::make_unique(); break; } diff --git a/dbms/src/Dictionaries/TrieDictionary.cpp b/dbms/src/Dictionaries/TrieDictionary.cpp index e461da27e2b..96399698be8 100644 --- a/dbms/src/Dictionaries/TrieDictionary.cpp +++ b/dbms/src/Dictionaries/TrieDictionary.cpp @@ -352,8 +352,8 @@ void TrieDictionary::validateKeyTypes(const DataTypes & key_types) const template void TrieDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) { - std::get(attribute.null_values) = null_value.get::Type>(); - std::get>(attribute.maps) = std::make_unique>(); + attribute.null_values = null_value.get::Type>(); + attribute.maps = std::make_unique>(); } TrieDictionary::Attribute TrieDictionary::createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value) @@ -380,8 +380,8 @@ TrieDictionary::Attribute TrieDictionary::createAttributeWithType(const Attribut case AttributeUnderlyingType::String: { - std::get(attr.null_values) = null_value.get(); - std::get>(attr.maps) = std::make_unique>(); + attr.null_values = null_value.get(); + attr.maps = std::make_unique>(); attr.string_arena = std::make_unique(); break; } From 5441b6c3d2f66488d622fc117c08479764041979 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 13 Nov 2018 21:54:38 +0300 Subject: [PATCH 34/46] Whitespace [#CLICKHOUSE-3] --- dbms/programs/client/Client.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index bf57d072f6b..516547de5d1 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -1527,7 +1527,7 @@ public: min_description_length = std::min(min_description_length, line_length - 2); } -#define DECLARE_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, po::value (), DESCRIPTION) +#define DECLARE_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, po::value(), DESCRIPTION) /// Main commandline options related to client functionality and all parameters from Settings. po::options_description main_description("Main options", line_length, min_description_length); From 7eb9fdef97d00c799462cf4ebfdac161b072ef3a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 13 Nov 2018 21:55:00 +0300 Subject: [PATCH 35/46] Removed Analyzers [#CLICKHOUSE-2] --- dbms/CMakeLists.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index dd8437d1e52..9bb852db923 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -59,7 +59,6 @@ add_headers_and_sources(clickhouse_common_io src/Common) add_headers_and_sources(clickhouse_common_io src/Common/HashTable) add_headers_and_sources(clickhouse_common_io src/IO) -add_headers_and_sources(dbms src/Analyzers) add_headers_and_sources(dbms src/Core) add_headers_and_sources(dbms src/DataStreams) add_headers_and_sources(dbms src/DataTypes) From 69e57187d263967dfe0216475db913606f69ea52 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 13 Nov 2018 22:43:17 +0300 Subject: [PATCH 36/46] Fixed error; removed unique_ptr #3565 --- dbms/src/Common/HashTable/HashTable.h | 24 +++++++++++++++++++ .../Dictionaries/ComplexKeyCacheDictionary.h | 9 +++---- .../ComplexKeyHashedDictionary.cpp | 22 ++++++++--------- .../Dictionaries/ComplexKeyHashedDictionary.h | 13 +++++----- dbms/src/Dictionaries/FlatDictionary.cpp | 24 +++++++++---------- dbms/src/Dictionaries/FlatDictionary.h | 13 +++++----- dbms/src/Dictionaries/HashedDictionary.cpp | 2 +- .../Dictionaries/RangeHashedDictionary.cpp | 2 +- dbms/src/Dictionaries/TrieDictionary.cpp | 12 +++++----- dbms/src/Dictionaries/TrieDictionary.h | 13 +++++----- 10 files changed, 76 insertions(+), 58 deletions(-) diff --git a/dbms/src/Common/HashTable/HashTable.h b/dbms/src/Common/HashTable/HashTable.h index 237d2955404..53b8412cf8d 100644 --- a/dbms/src/Common/HashTable/HashTable.h +++ b/dbms/src/Common/HashTable/HashTable.h @@ -494,12 +494,36 @@ public: alloc(grower); } + HashTable(HashTable && rhs) + : buf(nullptr) + { + *this = std::move(rhs); + } + ~HashTable() { destroyElements(); free(); } + HashTable & operator= (HashTable && rhs) + { + destroyElements(); + free(); + + std::swap(buf, rhs.buf); + std::swap(m_size, rhs.m_size); + std::swap(grower, rhs.grower); + + static_cast(*this) = std::move(static_cast(rhs)); + static_cast(*this) = std::move(static_cast(rhs)); + static_cast(*this) = std::move(static_cast(rhs)); + static_cast &>(*this) + = std::move(static_cast &>(rhs)); + + return *this; + } + class Reader final : private Cell::State { public: diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h index 70b1302a86d..a4a6ae4c16a 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h @@ -216,12 +216,9 @@ public: BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; private: - template - using MapType = HashMapWithSavedHash; - template - using ContainerType = Value[]; - template - using ContainerPtrType = std::unique_ptr>; + template using MapType = HashMapWithSavedHash; + template using ContainerType = Value[]; + template using ContainerPtrType = std::unique_ptr>; struct CellMetadata final { diff --git a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp index ce6186817b0..5d65213840e 100644 --- a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp +++ b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp @@ -376,9 +376,9 @@ void ComplexKeyHashedDictionary::loadData() template void ComplexKeyHashedDictionary::addAttributeSize(const Attribute & attribute) { - const auto & map_ref = std::get>(attribute.maps); - bytes_allocated += sizeof(ContainerType) + map_ref->getBufferSizeInBytes(); - bucket_count = map_ref->getBufferSizeInCells(); + const auto & map_ref = std::get>(attribute.maps); + bytes_allocated += sizeof(ContainerType) + map_ref.getBufferSizeInBytes(); + bucket_count = map_ref.getBufferSizeInCells(); } void ComplexKeyHashedDictionary::calculateBytesAllocated() @@ -421,8 +421,8 @@ void ComplexKeyHashedDictionary::calculateBytesAllocated() template void ComplexKeyHashedDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) { - attribute.null_values = null_value.get::Type>(); - attribute.maps = std::make_unique>(); + attribute.null_values = T(null_value.get::Type>()); + attribute.maps.emplace>(); } ComplexKeyHashedDictionary::Attribute ComplexKeyHashedDictionary::createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value) @@ -450,7 +450,7 @@ ComplexKeyHashedDictionary::Attribute ComplexKeyHashedDictionary::createAttribut case AttributeUnderlyingType::String: { attr.null_values = null_value.get(); - attr.maps = std::make_unique>(); + attr.maps.emplace>(); attr.string_arena = std::make_unique(); break; } @@ -497,7 +497,7 @@ void ComplexKeyHashedDictionary::getItemsImpl( ValueSetter && set_value, DefaultGetter && get_default) const { - const auto & attr = *std::get>(attribute.maps); + const auto & attr = std::get>(attribute.maps); const auto keys_size = key_columns.size(); StringRefs keys(keys_size); @@ -523,7 +523,7 @@ void ComplexKeyHashedDictionary::getItemsImpl( template bool ComplexKeyHashedDictionary::setAttributeValueImpl(Attribute & attribute, const StringRef key, const T value) { - auto & map = *std::get>(attribute.maps); + auto & map = std::get>(attribute.maps); const auto pair = map.insert({ key, value }); return pair.second; } @@ -550,7 +550,7 @@ bool ComplexKeyHashedDictionary::setAttributeValue(Attribute & attribute, const case AttributeUnderlyingType::String: { - auto & map = *std::get>(attribute.maps); + auto & map = std::get>(attribute.maps); const auto & string = value.get(); const auto string_in_arena = attribute.string_arena->insert(string.data(), string.size()); const auto pair = map.insert({ key, StringRef{string_in_arena, string.size()} }); @@ -596,7 +596,7 @@ StringRef ComplexKeyHashedDictionary::placeKeysInPool( template void ComplexKeyHashedDictionary::has(const Attribute & attribute, const Columns & key_columns, PaddedPODArray & out) const { - const auto & attr = *std::get>(attribute.maps); + const auto & attr = std::get>(attribute.maps); const auto keys_size = key_columns.size(); StringRefs keys(keys_size); Arena temporary_keys_pool; @@ -646,7 +646,7 @@ std::vector ComplexKeyHashedDictionary::getKeys() const template std::vector ComplexKeyHashedDictionary::getKeys(const Attribute & attribute) const { - const ContainerType & attr = *std::get>(attribute.maps); + const ContainerType & attr = std::get>(attribute.maps); std::vector keys; keys.reserve(attr.size()); for (const auto & key : attr) diff --git a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h index a0392717d0c..7dd5d5baff9 100644 --- a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h +++ b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h @@ -147,7 +147,6 @@ public: private: template using ContainerType = HashMapWithSavedHash; - template using ContainerPtrType = std::unique_ptr>; struct Attribute final { @@ -160,12 +159,12 @@ private: Float32, Float64, String> null_values; std::variant< - ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, - ContainerPtrType, - ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, - ContainerPtrType, ContainerPtrType, ContainerPtrType, - ContainerPtrType, ContainerPtrType, - ContainerPtrType> maps; + ContainerType, ContainerType, ContainerType, ContainerType, + ContainerType, + ContainerType, ContainerType, ContainerType, ContainerType, + ContainerType, ContainerType, ContainerType, + ContainerType, ContainerType, + ContainerType> maps; std::unique_ptr string_arena; }; diff --git a/dbms/src/Dictionaries/FlatDictionary.cpp b/dbms/src/Dictionaries/FlatDictionary.cpp index 9b3394fc2d6..f6f39a085cb 100644 --- a/dbms/src/Dictionaries/FlatDictionary.cpp +++ b/dbms/src/Dictionaries/FlatDictionary.cpp @@ -70,7 +70,7 @@ void FlatDictionary::isInImpl( PaddedPODArray & out) const { const auto null_value = std::get(hierarchical_attribute->null_values); - const auto & attr = *std::get>(hierarchical_attribute->arrays); + const auto & attr = std::get>(hierarchical_attribute->arrays); const auto rows = out.size(); size_t loaded_size = attr.size(); @@ -395,9 +395,9 @@ void FlatDictionary::loadData() template void FlatDictionary::addAttributeSize(const Attribute & attribute) { - const auto & array_ref = std::get>(attribute.arrays); - bytes_allocated += sizeof(PaddedPODArray) + array_ref->allocated_bytes(); - bucket_count = array_ref->capacity(); + const auto & array_ref = std::get>(attribute.arrays); + bytes_allocated += sizeof(PaddedPODArray) + array_ref.allocated_bytes(); + bucket_count = array_ref.capacity(); } @@ -440,9 +440,9 @@ void FlatDictionary::calculateBytesAllocated() template void FlatDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) { - attribute.null_values = null_value.get::Type>(); + attribute.null_values = T(null_value.get::Type>()); const auto & null_value_ref = std::get(attribute.null_values); - attribute.arrays = std::make_unique>(initial_array_size, null_value_ref); + attribute.arrays.emplace>(initial_array_size, null_value_ref); } template <> @@ -450,10 +450,10 @@ void FlatDictionary::createAttributeImpl(Attribute & attribute, const Fi { attribute.string_arena = std::make_unique(); auto & null_value_ref = std::get(attribute.null_values); - const String & string = null_value.get::Type>(); + const String & string = null_value.get(); const auto string_in_arena = attribute.string_arena->insert(string.data(), string.size()); null_value_ref = StringRef{string_in_arena, string.size()}; - attribute.arrays = std::make_unique>(initial_array_size, null_value_ref); + attribute.arrays.emplace>(initial_array_size, null_value_ref); } @@ -523,7 +523,7 @@ void FlatDictionary::getItemsImpl( ValueSetter && set_value, DefaultGetter && get_default) const { - const auto & attr = *std::get>(attribute.arrays); + const auto & attr = std::get>(attribute.arrays); const auto rows = ext::size(ids); for (const auto row : ext::range(0, rows)) @@ -541,7 +541,7 @@ void FlatDictionary::resize(Attribute & attribute, const Key id) if (id >= max_array_size) throw Exception{name + ": identifier should be less than " + toString(max_array_size), ErrorCodes::ARGUMENT_OUT_OF_BOUND}; - auto & array = *std::get>(attribute.arrays); + auto & array = std::get>(attribute.arrays); if (id >= array.size()) { const size_t elements_count = id + 1; //id=0 -> elements_count=1 @@ -554,7 +554,7 @@ template void FlatDictionary::setAttributeValueImpl(Attribute & attribute, const Key id, const T & value) { resize(attribute, id); - auto & array = *std::get>(attribute.arrays); + auto & array = std::get>(attribute.arrays); array[id] = value; loaded_ids[id] = true; } @@ -564,7 +564,7 @@ void FlatDictionary::setAttributeValueImpl(Attribute & attribute, const { resize(attribute, id); const auto string_in_arena = attribute.string_arena->insert(string.data(), string.size()); - auto & array = *std::get>(attribute.arrays); + auto & array = std::get>(attribute.arrays); array[id] = StringRef{string_in_arena, string.size()}; loaded_ids[id] = true; } diff --git a/dbms/src/Dictionaries/FlatDictionary.h b/dbms/src/Dictionaries/FlatDictionary.h index 7fa36fc9925..272b248d23e 100644 --- a/dbms/src/Dictionaries/FlatDictionary.h +++ b/dbms/src/Dictionaries/FlatDictionary.h @@ -147,7 +147,6 @@ public: private: template using ContainerType = PaddedPODArray; - template using ContainerPtrType = std::unique_ptr>; struct Attribute final { @@ -160,12 +159,12 @@ private: Float32, Float64, StringRef> null_values; std::variant< - ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, - ContainerPtrType, - ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, - ContainerPtrType, ContainerPtrType, ContainerPtrType, - ContainerPtrType, ContainerPtrType, - ContainerPtrType> arrays; + ContainerType, ContainerType, ContainerType, ContainerType, + ContainerType, + ContainerType, ContainerType, ContainerType, ContainerType, + ContainerType, ContainerType, ContainerType, + ContainerType, ContainerType, + ContainerType> arrays; std::unique_ptr string_arena; }; diff --git a/dbms/src/Dictionaries/HashedDictionary.cpp b/dbms/src/Dictionaries/HashedDictionary.cpp index fa78d8c6755..e453d907fe5 100644 --- a/dbms/src/Dictionaries/HashedDictionary.cpp +++ b/dbms/src/Dictionaries/HashedDictionary.cpp @@ -430,7 +430,7 @@ void HashedDictionary::calculateBytesAllocated() template void HashedDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) { - attribute.null_values = null_value.get::Type>(); + attribute.null_values = T(null_value.get::Type>()); attribute.maps = std::make_unique>(); } diff --git a/dbms/src/Dictionaries/RangeHashedDictionary.cpp b/dbms/src/Dictionaries/RangeHashedDictionary.cpp index b0752c0d877..54926e75f1a 100644 --- a/dbms/src/Dictionaries/RangeHashedDictionary.cpp +++ b/dbms/src/Dictionaries/RangeHashedDictionary.cpp @@ -261,7 +261,7 @@ void RangeHashedDictionary::calculateBytesAllocated() template void RangeHashedDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) { - attribute.null_values = null_value.get::Type>(); + attribute.null_values = T(null_value.get::Type>()); attribute.maps = std::make_unique>(); } diff --git a/dbms/src/Dictionaries/TrieDictionary.cpp b/dbms/src/Dictionaries/TrieDictionary.cpp index 96399698be8..1efbe8c2d77 100644 --- a/dbms/src/Dictionaries/TrieDictionary.cpp +++ b/dbms/src/Dictionaries/TrieDictionary.cpp @@ -295,7 +295,7 @@ void TrieDictionary::loadData() template void TrieDictionary::addAttributeSize(const Attribute & attribute) { - const auto & vec = *std::get>(attribute.maps); + const auto & vec = std::get>(attribute.maps); bytes_allocated += sizeof(ContainerType) + (vec.capacity() * sizeof(T)); bucket_count = vec.size(); } @@ -352,8 +352,8 @@ void TrieDictionary::validateKeyTypes(const DataTypes & key_types) const template void TrieDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) { - attribute.null_values = null_value.get::Type>(); - attribute.maps = std::make_unique>(); + attribute.null_values = T(null_value.get::Type>()); + attribute.maps.emplace>(); } TrieDictionary::Attribute TrieDictionary::createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value) @@ -381,7 +381,7 @@ TrieDictionary::Attribute TrieDictionary::createAttributeWithType(const Attribut case AttributeUnderlyingType::String: { attr.null_values = null_value.get(); - attr.maps = std::make_unique>(); + attr.maps.emplace>(); attr.string_arena = std::make_unique(); break; } @@ -428,7 +428,7 @@ void TrieDictionary::getItemsImpl( ValueSetter && set_value, DefaultGetter && get_default) const { - auto & vec = *std::get>(attribute.maps); + auto & vec = std::get>(attribute.maps); const auto first_column = key_columns.front(); const auto rows = first_column->size(); @@ -462,7 +462,7 @@ template bool TrieDictionary::setAttributeValueImpl(Attribute & attribute, const StringRef key, const T value) { // Insert value into appropriate vector type - auto & vec = *std::get>(attribute.maps); + auto & vec = std::get>(attribute.maps); size_t row = vec.size(); vec.push_back(value); diff --git a/dbms/src/Dictionaries/TrieDictionary.h b/dbms/src/Dictionaries/TrieDictionary.h index 948e7329221..af4879b5a04 100644 --- a/dbms/src/Dictionaries/TrieDictionary.h +++ b/dbms/src/Dictionaries/TrieDictionary.h @@ -150,7 +150,6 @@ public: private: template using ContainerType = std::vector; - template using ContainerPtrType = std::unique_ptr>; struct Attribute final { @@ -163,12 +162,12 @@ private: Float32, Float64, String> null_values; std::variant< - ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, - ContainerPtrType, - ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, - ContainerPtrType, ContainerPtrType, ContainerPtrType, - ContainerPtrType, ContainerPtrType, - ContainerPtrType> maps; + ContainerType, ContainerType, ContainerType, ContainerType, + ContainerType, + ContainerType, ContainerType, ContainerType, ContainerType, + ContainerType, ContainerType, ContainerType, + ContainerType, ContainerType, + ContainerType> maps; std::unique_ptr string_arena; }; From ead153e9e45a5b656e4c1875a28637e7b5320124 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 13 Nov 2018 22:48:47 +0300 Subject: [PATCH 37/46] Better code #3565 --- dbms/src/Common/HashTable/HashTable.h | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/dbms/src/Common/HashTable/HashTable.h b/dbms/src/Common/HashTable/HashTable.h index 53b8412cf8d..b35d3b41acc 100644 --- a/dbms/src/Common/HashTable/HashTable.h +++ b/dbms/src/Common/HashTable/HashTable.h @@ -515,11 +515,10 @@ public: std::swap(m_size, rhs.m_size); std::swap(grower, rhs.grower); - static_cast(*this) = std::move(static_cast(rhs)); - static_cast(*this) = std::move(static_cast(rhs)); - static_cast(*this) = std::move(static_cast(rhs)); - static_cast &>(*this) - = std::move(static_cast &>(rhs)); + Hash::operator=(std::move(rhs)); + Allocator::operator=(std::move(rhs)); + Cell::State::operator=(std::move(rhs)); + ZeroValueStorage::operator=(std::move(rhs)); return *this; } From d1b86225efa8c4ed05fb53833668b1ad8a47a262 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 14 Nov 2018 04:04:09 +0300 Subject: [PATCH 38/46] Fixed bad code #3565 --- dbms/src/Dictionaries/FlatDictionary.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/dbms/src/Dictionaries/FlatDictionary.cpp b/dbms/src/Dictionaries/FlatDictionary.cpp index f6f39a085cb..4c86da1ba8f 100644 --- a/dbms/src/Dictionaries/FlatDictionary.cpp +++ b/dbms/src/Dictionaries/FlatDictionary.cpp @@ -449,11 +449,10 @@ template <> void FlatDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) { attribute.string_arena = std::make_unique(); - auto & null_value_ref = std::get(attribute.null_values); const String & string = null_value.get(); - const auto string_in_arena = attribute.string_arena->insert(string.data(), string.size()); - null_value_ref = StringRef{string_in_arena, string.size()}; - attribute.arrays.emplace>(initial_array_size, null_value_ref); + const char * string_in_arena = attribute.string_arena->insert(string.data(), string.size()); + attribute.null_values.emplace(string_in_arena, string.size()); + attribute.arrays.emplace>(initial_array_size, StringRef(string_in_arena, string.size())); } From 37a9af5a68e64938df723112841404d1b381b2fd Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 14 Nov 2018 04:11:13 +0300 Subject: [PATCH 39/46] CLICKHOUSE-4109 mlock clickhouse (#3553) * CLICKHOUSE-4109 mlock clickhouse * wip * Fix * wip * fix * fix * better place * wip * clean * tidy * fix --- dbms/programs/server/Server.cpp | 42 ++++++++++++++++++++- dbms/src/Common/Exception.cpp | 11 ++++-- dbms/src/Common/Exception.h | 1 + dbms/src/Common/StatusFile.cpp | 8 +--- dbms/src/Common/TaskStatsInfoGetter.cpp | 17 ++------- dbms/src/Common/hasLinuxCapability.cpp | 49 +++++++++++++++++++++++++ dbms/src/Common/hasLinuxCapability.h | 10 +++++ dbms/tests/queries/bugs/fuzzy.sql | 5 +++ debian/clickhouse-server.postinst | 5 ++- debian/clickhouse-server.service | 2 +- 10 files changed, 124 insertions(+), 26 deletions(-) create mode 100644 dbms/src/Common/hasLinuxCapability.cpp create mode 100644 dbms/src/Common/hasLinuxCapability.h diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index ba603114783..c40962d5ebe 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -42,6 +42,11 @@ #include #include "TCPHandlerFactory.h" +#if defined(__linux__) +#include +#include +#endif + #if USE_POCO_NETSSL #include #include @@ -125,6 +130,41 @@ int Server::main(const std::vector & /*args*/) config().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false); } + const auto memory_amount = getMemoryAmount(); + { /// After full config loaded +#if defined(__linux__) + if (config().getBool("mlock_executable", + false // TODO: uncomment after tests: +/* +#if NDEBUG + memory_amount > 16000000000 + ? true // Change me to true in future + : false // Dont mlock if we have less than 16G ram +#else + false +#endif +*/ + )) + { + if (hasLinuxCapability(CAP_IPC_LOCK)) + { + if (0 != mlockall(MCL_CURRENT)) + LOG_WARNING(log, "Failed mlockall: " + errnoToString()); + else + LOG_TRACE(log, "Binary mlock'ed"); + } + else + { + LOG_INFO(log, "It looks like the process has no CAP_IPC_LOCK capability, binary mlock will be disabled." + " It could happen due to incorrect ClickHouse package installation." + " You could resolve the problem manually with 'sudo setcap cap_ipc_lock=+ep /usr/bin/clickhouse'." + " Note that it will not work on 'nosuid' mounted filesystems."); + + } + } +#endif + } + std::string path = getCanonicalPath(config().getString("path")); std::string default_database = config().getString("default_database", "default"); @@ -599,7 +639,7 @@ int Server::main(const std::vector & /*args*/) { std::stringstream message; - message << "Available RAM = " << formatReadableSizeWithBinarySuffix(getMemoryAmount()) << ";" + message << "Available RAM = " << formatReadableSizeWithBinarySuffix(memory_amount) << ";" << " physical cores = " << getNumberOfPhysicalCPUCores() << ";" // on ARM processors it can show only enabled at current moment cores << " threads = " << std::thread::hardware_concurrency() << "."; diff --git a/dbms/src/Common/Exception.cpp b/dbms/src/Common/Exception.cpp index ab59780c44d..606f180e9d7 100644 --- a/dbms/src/Common/Exception.cpp +++ b/dbms/src/Common/Exception.cpp @@ -25,7 +25,7 @@ namespace ErrorCodes } -void throwFromErrno(const std::string & s, int code, int e) +std::string errnoToString(int code, int e) { const size_t buf_size = 128; char buf[buf_size]; @@ -43,12 +43,17 @@ void throwFromErrno(const std::string & s, int code, int e) strcpy(buf, unknown_message); strcpy(buf + strlen(unknown_message), code); } - throw ErrnoException(s + ", errno: " + toString(e) + ", strerror: " + std::string(buf), code, e); + return "errno: " + toString(e) + ", strerror: " + std::string(buf); #else - throw ErrnoException(s + ", errno: " + toString(e) + ", strerror: " + std::string(strerror_r(e, buf, sizeof(buf))), code, e); + (void)code; + return "errno: " + toString(e) + ", strerror: " + std::string(strerror_r(e, buf, sizeof(buf))); #endif } +void throwFromErrno(const std::string & s, int code, int e) +{ + throw ErrnoException(s + ", " + errnoToString(code, e)); +} void tryLogCurrentException(const char * log_name, const std::string & start_of_message) { diff --git a/dbms/src/Common/Exception.h b/dbms/src/Common/Exception.h index aedef9fb976..4e3bdc8bafe 100644 --- a/dbms/src/Common/Exception.h +++ b/dbms/src/Common/Exception.h @@ -59,6 +59,7 @@ private: using Exceptions = std::vector; +std::string errnoToString(int code = 0, int the_errno = errno); [[noreturn]] void throwFromErrno(const std::string & s, int code = 0, int the_errno = errno); diff --git a/dbms/src/Common/StatusFile.cpp b/dbms/src/Common/StatusFile.cpp index c34ef553f89..84b1edc922b 100644 --- a/dbms/src/Common/StatusFile.cpp +++ b/dbms/src/Common/StatusFile.cpp @@ -81,15 +81,11 @@ StatusFile::StatusFile(const std::string & path_) StatusFile::~StatusFile() { - char buf[128]; - if (0 != close(fd)) - LOG_ERROR(&Logger::get("StatusFile"), "Cannot close file " << path << ", errno: " - << errno << ", strerror: " << strerror_r(errno, buf, sizeof(buf))); + LOG_ERROR(&Logger::get("StatusFile"), "Cannot close file " << path << ", " << errnoToString()); if (0 != unlink(path.c_str())) - LOG_ERROR(&Logger::get("StatusFile"), "Cannot unlink file " << path << ", errno: " - << errno << ", strerror: " << strerror_r(errno, buf, sizeof(buf))); + LOG_ERROR(&Logger::get("StatusFile"), "Cannot unlink file " << path << ", " << errnoToString()); } } diff --git a/dbms/src/Common/TaskStatsInfoGetter.cpp b/dbms/src/Common/TaskStatsInfoGetter.cpp index b57edbad160..7793f81940d 100644 --- a/dbms/src/Common/TaskStatsInfoGetter.cpp +++ b/dbms/src/Common/TaskStatsInfoGetter.cpp @@ -1,4 +1,4 @@ -#include +#include "TaskStatsInfoGetter.h" #include #include @@ -6,6 +6,7 @@ #if defined(__linux__) +#include "hasLinuxCapability.h" #include #include @@ -185,18 +186,8 @@ UInt16 getFamilyIdImpl(int fd) bool checkPermissionsImpl() { - /// See man getcap. - __user_cap_header_struct request{}; - request.version = _LINUX_CAPABILITY_VERSION_1; /// It's enough to check just single CAP_NET_ADMIN capability we are interested. - request.pid = getpid(); - - __user_cap_data_struct response{}; - - /// Avoid dependency on 'libcap'. - if (0 != syscall(SYS_capget, &request, &response)) - throwFromErrno("Cannot do 'capget' syscall", ErrorCodes::NETLINK_ERROR); - - if (!((1 << CAP_NET_ADMIN) & response.effective)) + static bool res = hasLinuxCapability(CAP_NET_ADMIN); + if (!res) return false; /// Check that we can successfully initialize TaskStatsInfoGetter. diff --git a/dbms/src/Common/hasLinuxCapability.cpp b/dbms/src/Common/hasLinuxCapability.cpp new file mode 100644 index 00000000000..ad0b112023c --- /dev/null +++ b/dbms/src/Common/hasLinuxCapability.cpp @@ -0,0 +1,49 @@ +#include "hasLinuxCapability.h" + +#if defined(__linux__) + +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NETLINK_ERROR; +} + + +namespace +{ + bool hasLinuxCapabilityImpl(decltype(CAP_NET_ADMIN) cap) + { + /// See man getcap. + __user_cap_header_struct request{}; + request.version = _LINUX_CAPABILITY_VERSION_1; /// It's enough to check just single CAP_NET_ADMIN capability we are interested. + request.pid = getpid(); + + __user_cap_data_struct response{}; + + /// Avoid dependency on 'libcap'. + if (0 != syscall(SYS_capget, &request, &response)) + throwFromErrno("Cannot do 'capget' syscall", ErrorCodes::NETLINK_ERROR); + + if (!((1 << cap) & response.effective)) + return false; + + return true; + } +} + +bool hasLinuxCapability(decltype(CAP_NET_ADMIN) cap) +{ + static bool res = hasLinuxCapabilityImpl(cap); + return res; +} +} + +#endif diff --git a/dbms/src/Common/hasLinuxCapability.h b/dbms/src/Common/hasLinuxCapability.h new file mode 100644 index 00000000000..133cdb8ccd1 --- /dev/null +++ b/dbms/src/Common/hasLinuxCapability.h @@ -0,0 +1,10 @@ +#if defined(__linux__) + +#include + +namespace DB +{ +bool hasLinuxCapability(decltype(CAP_NET_ADMIN) cap); +} + +#endif diff --git a/dbms/tests/queries/bugs/fuzzy.sql b/dbms/tests/queries/bugs/fuzzy.sql index a3c48df935c..80153803cfe 100644 --- a/dbms/tests/queries/bugs/fuzzy.sql +++ b/dbms/tests/queries/bugs/fuzzy.sql @@ -4,13 +4,18 @@ SELECT __inner_restore_projection__(067274, 'vb\s'); SELECT sequenceCount((CAST((( SELECT NULL ) AS rg, ( SELECT ( SELECT [], 'A') AS String))]]); SELECT truncate(895, -16); SELECT (CAST((lowerUTF8('a7\xwK>-')) AS String)), [6935]; SELECT upperUTF8(sipHash128('\0')), [], ['xD2jG']; +SELECT upperUTF8(SHA256('')); SELECT arrayEnumerateUniq(anyHeavy([]), []); SELECT notIn([['']], [[NULL]]); SELECT subtractDays((CAST((-5263074.47) AS DateTime)), -737895); +SELECT quantileDeterministic([], findClusterIndex(( SELECT subtractDays((CAST((566450.398706) AS DateTime)), 54) ) )), '\0', []; +SELECT addDays((CAST((96.338) AS DateTime)), -3); SELECT arrayEnumerateDense([], [sequenceCount(NULL)]); +SELECT '[RE7', ( SELECT '\0' ) AS riwwq, ( SELECT reverse([( SELECT bitTestAll(NULL) ) , ( SELECT '\0' ) AS ddfweeuy]) ) AS xuvv, '', ( SELECT * FROM mysql() ) AS wqgdswyc, ( SELECT * FROM mysql() ); diff --git a/debian/clickhouse-server.postinst b/debian/clickhouse-server.postinst index 269e7c2da0c..53e9248b71f 100644 --- a/debian/clickhouse-server.postinst +++ b/debian/clickhouse-server.postinst @@ -86,9 +86,10 @@ Please fix this and reinstall this package." >&2 fi # Set net_admin capabilities to support introspection of "taskstats" performance metrics from the kernel. + command -v setcap >/dev/null \ - && setcap cap_net_admin=+ep "${CLICKHOUSE_BINDIR}/${CLICKHOUSE_GENERIC_PROGRAM}" \ - || echo "Cannot set 'net_admin' capability for clickhouse binary. This is optional. Taskstats accounting will be disabled. To enable taskstats accounting you may add the required capability later manually." + && setcap "cap_net_admin=+ep cap_ipc_lock=+ep" "${CLICKHOUSE_BINDIR}/${CLICKHOUSE_GENERIC_PROGRAM}" \ + || echo "Cannot set 'net_admin' or 'ipc_lock' capability for clickhouse binary. This is optional. Taskstats accounting will be disabled. To enable taskstats accounting you may add the required capability later manually." # Clean old dynamic compilation results if [ -d "${CLICKHOUSE_DATADIR}/build" ]; then diff --git a/debian/clickhouse-server.service b/debian/clickhouse-server.service index 6f0690ec3bd..fa556f31951 100644 --- a/debian/clickhouse-server.service +++ b/debian/clickhouse-server.service @@ -13,7 +13,7 @@ ExecStartPre=-/bin/chown clickhouse:clickhouse -R /etc/clickhouse-server ExecStart=/usr/bin/clickhouse-server --config=/etc/clickhouse-server/config.xml LimitCORE=infinity LimitNOFILE=500000 -CapabilityBoundingSet=CAP_NET_ADMIN +CapabilityBoundingSet=CAP_NET_ADMIN CAP_IPC_LOCK [Install] WantedBy=multi-user.target From 1aa89ed63a35a7e7b898eefd3cc7bacbd40a89c3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 14 Nov 2018 04:20:46 +0300 Subject: [PATCH 40/46] Fixed code after review #3553 --- dbms/programs/server/Server.cpp | 31 +++++++++++-------------------- dbms/programs/server/config.xml | 6 ++++++ 2 files changed, 17 insertions(+), 20 deletions(-) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index c40962d5ebe..0937cf4b84a 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -131,39 +131,30 @@ int Server::main(const std::vector & /*args*/) } const auto memory_amount = getMemoryAmount(); - { /// After full config loaded + #if defined(__linux__) - if (config().getBool("mlock_executable", - false // TODO: uncomment after tests: -/* -#if NDEBUG - memory_amount > 16000000000 - ? true // Change me to true in future - : false // Dont mlock if we have less than 16G ram -#else - false -#endif -*/ - )) + /// After full config loaded + { + if (config().getBool("mlock_executable", false)) { if (hasLinuxCapability(CAP_IPC_LOCK)) { + LOG_TRACE(log, "Will mlockall to prevent executable memory from being paged out. It may take a few seconds."); if (0 != mlockall(MCL_CURRENT)) LOG_WARNING(log, "Failed mlockall: " + errnoToString()); else - LOG_TRACE(log, "Binary mlock'ed"); + LOG_TRACE(log, "The memory map of clickhouse executable has been mlock'ed"); } else { - LOG_INFO(log, "It looks like the process has no CAP_IPC_LOCK capability, binary mlock will be disabled." - " It could happen due to incorrect ClickHouse package installation." - " You could resolve the problem manually with 'sudo setcap cap_ipc_lock=+ep /usr/bin/clickhouse'." - " Note that it will not work on 'nosuid' mounted filesystems."); - + LOG_INFO(log, "It looks like the process has no CAP_IPC_LOCK capability, binary mlock will be disabled." + " It could happen due to incorrect ClickHouse package installation." + " You could resolve the problem manually with 'sudo setcap cap_ipc_lock=+ep /usr/bin/clickhouse'." + " Note that it will not work on 'nosuid' mounted filesystems."); } } -#endif } +#endif std::string path = getCanonicalPath(config().getString("path")); std::string default_database = config().getString("default_database", "default"); diff --git a/dbms/programs/server/config.xml b/dbms/programs/server/config.xml index e461d49d522..051f6f7fb2b 100644 --- a/dbms/programs/server/config.xml +++ b/dbms/programs/server/config.xml @@ -145,6 +145,12 @@ --> + + false + From 2daab83a2319644c07c3ad3d125dc3b79b5a9f71 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 14 Nov 2018 04:59:32 +0300 Subject: [PATCH 41/46] Fixed bad and wrong code #3553 --- dbms/src/Common/hasLinuxCapability.cpp | 39 +++++++++++--------------- dbms/src/Common/hasLinuxCapability.h | 5 +++- 2 files changed, 21 insertions(+), 23 deletions(-) diff --git a/dbms/src/Common/hasLinuxCapability.cpp b/dbms/src/Common/hasLinuxCapability.cpp index ad0b112023c..94793050dbe 100644 --- a/dbms/src/Common/hasLinuxCapability.cpp +++ b/dbms/src/Common/hasLinuxCapability.cpp @@ -1,7 +1,7 @@ -#include "hasLinuxCapability.h" - #if defined(__linux__) +#include "hasLinuxCapability.h" + #include #include #include @@ -11,39 +11,34 @@ namespace DB { + namespace ErrorCodes { extern const int NETLINK_ERROR; } - -namespace +static __user_cap_data_struct getCapabilities() { - bool hasLinuxCapabilityImpl(decltype(CAP_NET_ADMIN) cap) - { - /// See man getcap. - __user_cap_header_struct request{}; - request.version = _LINUX_CAPABILITY_VERSION_1; /// It's enough to check just single CAP_NET_ADMIN capability we are interested. - request.pid = getpid(); + /// See man getcap. + __user_cap_header_struct request{}; + request.version = _LINUX_CAPABILITY_VERSION_1; /// It's enough to check just single CAP_NET_ADMIN capability we are interested. + request.pid = getpid(); - __user_cap_data_struct response{}; + __user_cap_data_struct response{}; - /// Avoid dependency on 'libcap'. - if (0 != syscall(SYS_capget, &request, &response)) - throwFromErrno("Cannot do 'capget' syscall", ErrorCodes::NETLINK_ERROR); + /// Avoid dependency on 'libcap'. + if (0 != syscall(SYS_capget, &request, &response)) + throwFromErrno("Cannot do 'capget' syscall", ErrorCodes::NETLINK_ERROR); - if (!((1 << cap) & response.effective)) - return false; - - return true; - } + return response; } -bool hasLinuxCapability(decltype(CAP_NET_ADMIN) cap) +bool hasLinuxCapability(int cap) { - static bool res = hasLinuxCapabilityImpl(cap); - return res; + static __user_cap_data_struct capabilities = getCapabilities(); + return (1 << cap) & capabilities.effective; } + } #endif diff --git a/dbms/src/Common/hasLinuxCapability.h b/dbms/src/Common/hasLinuxCapability.h index 133cdb8ccd1..91bbca4c068 100644 --- a/dbms/src/Common/hasLinuxCapability.h +++ b/dbms/src/Common/hasLinuxCapability.h @@ -4,7 +4,10 @@ namespace DB { -bool hasLinuxCapability(decltype(CAP_NET_ADMIN) cap); + +/// Check that the current process has Linux capability. Examples: CAP_IPC_LOCK, CAP_NET_ADMIN. +bool hasLinuxCapability(int cap); + } #endif From b65250e672eb032563aa3b0c59d5ce5e7a50dee3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 14 Nov 2018 11:05:34 +0300 Subject: [PATCH 42/46] Better default value in packager script --- docker/packager/binary/Dockerfile | 3 --- docker/packager/deb/Dockerfile | 3 --- docker/packager/packager | 2 +- 3 files changed, 1 insertion(+), 7 deletions(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 2206bcc7e49..49b757e1b00 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -5,9 +5,6 @@ RUN apt-get update -y \ apt-get install --yes --no-install-recommends \ bash \ cmake \ - ccache \ - distcc \ - distcc-pump \ curl \ gcc-7 \ g++-7 \ diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index 08eeb3f3578..9e0abd3afd4 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -5,9 +5,6 @@ RUN apt-get update -y \ apt-get install --yes --no-install-recommends \ bash \ fakeroot \ - ccache \ - distcc \ - distcc-pump \ cmake \ curl \ gcc-7 \ diff --git a/docker/packager/packager b/docker/packager/packager index fed0c54fae5..c0006c7b0bb 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -86,7 +86,7 @@ if __name__ == "__main__": parser.add_argument("--build-type", choices=("debug", ""), default="") parser.add_argument("--compiler", choices=("clang-6.0", "gcc-7", "gcc-8"), default="gcc-7") parser.add_argument("--sanitizer", choices=("address", "thread", "memory", "undefined", ""), default="") - parser.add_argument("--cache", choices=("ccache", "distcc")) + parser.add_argument("--cache", choices=("", "ccache", "distcc"), default="") parser.add_argument("--distcc-hosts", nargs="+") parser.add_argument("--force-build-image", action="store_true") From 9021bfb2f1fd5d62f898c85ed2d52ac478349178 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 15 Nov 2018 17:29:53 +0800 Subject: [PATCH 43/46] Fix variant assignments --- dbms/src/Dictionaries/CacheDictionary.cpp | 89 +++++-------------- ...acheDictionary_createAttributeWithType.cpp | 89 +++++-------------- 2 files changed, 40 insertions(+), 138 deletions(-) diff --git a/dbms/src/Dictionaries/CacheDictionary.cpp b/dbms/src/Dictionaries/CacheDictionary.cpp index 21920cbfbba..154fb6759f7 100644 --- a/dbms/src/Dictionaries/CacheDictionary.cpp +++ b/dbms/src/Dictionaries/CacheDictionary.cpp @@ -454,76 +454,27 @@ CacheDictionary::Attribute CacheDictionary::createAttributeWithType(const Attrib switch (type) { - case AttributeUnderlyingType::UInt8: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(UInt8); - break; - case AttributeUnderlyingType::UInt16: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(UInt16); - break; - case AttributeUnderlyingType::UInt32: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(UInt32); - break; - case AttributeUnderlyingType::UInt64: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(UInt64); - break; - case AttributeUnderlyingType::UInt128: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(UInt128); - break; - case AttributeUnderlyingType::Int8: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(Int8); - break; - case AttributeUnderlyingType::Int16: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(Int16); - break; - case AttributeUnderlyingType::Int32: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(Int32); - break; - case AttributeUnderlyingType::Int64: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(Int64); - break; - case AttributeUnderlyingType::Decimal32: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(Decimal32); - break; - case AttributeUnderlyingType::Decimal64: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(Decimal64); - break; - case AttributeUnderlyingType::Decimal128: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(Decimal128); - break; - case AttributeUnderlyingType::Float32: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(Float32); - break; - case AttributeUnderlyingType::Float64: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(Float64); +#define DISPATCH(TYPE) \ + case AttributeUnderlyingType::TYPE: \ + attr.null_values = TYPE(null_value.get::Type>()); \ + attr.arrays = std::make_unique>(size); \ + bytes_allocated += size * sizeof(TYPE); \ break; + DISPATCH(UInt8) + DISPATCH(UInt16) + DISPATCH(UInt32) + DISPATCH(UInt64) + DISPATCH(UInt128) + DISPATCH(Int8) + DISPATCH(Int16) + DISPATCH(Int32) + DISPATCH(Int64) + DISPATCH(Decimal32) + DISPATCH(Decimal64) + DISPATCH(Decimal128) + DISPATCH(Float32) + DISPATCH(Float64) +#undef DISPATCH case AttributeUnderlyingType::String: attr.null_values = null_value.get(); attr.arrays = std::make_unique>(size); diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_createAttributeWithType.cpp b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_createAttributeWithType.cpp index 60f5139067f..91b74ef8340 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_createAttributeWithType.cpp +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_createAttributeWithType.cpp @@ -9,76 +9,27 @@ ComplexKeyCacheDictionary::Attribute ComplexKeyCacheDictionary::createAttributeW switch (type) { - case AttributeUnderlyingType::UInt8: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(UInt8); - break; - case AttributeUnderlyingType::UInt16: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(UInt16); - break; - case AttributeUnderlyingType::UInt32: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(UInt32); - break; - case AttributeUnderlyingType::UInt64: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(UInt64); - break; - case AttributeUnderlyingType::UInt128: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(UInt128); - break; - case AttributeUnderlyingType::Int8: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(Int8); - break; - case AttributeUnderlyingType::Int16: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(Int16); - break; - case AttributeUnderlyingType::Int32: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(Int32); - break; - case AttributeUnderlyingType::Int64: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(Int64); - break; - case AttributeUnderlyingType::Float32: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(Float32); - break; - case AttributeUnderlyingType::Float64: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(Float64); - break; - case AttributeUnderlyingType::Decimal32: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(Decimal32); - break; - case AttributeUnderlyingType::Decimal64: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(Decimal64); - break; - case AttributeUnderlyingType::Decimal128: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(Decimal128); +#define DISPATCH(TYPE) \ + case AttributeUnderlyingType::TYPE: \ + attr.null_values = TYPE(null_value.get::Type>()); \ + attr.arrays = std::make_unique>(size); \ + bytes_allocated += size * sizeof(TYPE); \ break; + DISPATCH(UInt8) + DISPATCH(UInt16) + DISPATCH(UInt32) + DISPATCH(UInt64) + DISPATCH(UInt128) + DISPATCH(Int8) + DISPATCH(Int16) + DISPATCH(Int32) + DISPATCH(Int64) + DISPATCH(Decimal32) + DISPATCH(Decimal64) + DISPATCH(Decimal128) + DISPATCH(Float32) + DISPATCH(Float64) +#undef DISPATCH case AttributeUnderlyingType::String: attr.null_values = null_value.get(); attr.arrays = std::make_unique>(size); From 160ffb22839692343c2d4ab67c07dd89620ba866 Mon Sep 17 00:00:00 2001 From: proller Date: Fri, 16 Nov 2018 15:22:51 +0300 Subject: [PATCH 44/46] Build fixes (clang 7.0.0+) (#3582) --- CMakeLists.txt | 2 +- cmake/find_llvm.cmake | 6 +- contrib/CMakeLists.txt | 35 +- .../clang/Compiler-5.0.0/CMakeLists.txt | 2 + .../clang/Compiler-6.0.0/CMakeLists.txt | 2 + .../clang/Compiler-6.0.0/cc1as_main.cpp | 2 +- .../clang/Compiler-7.0.0/CMakeLists.txt | 4 +- .../clang/Compiler-7.0.0/cc1_main.cpp | 24 +- .../clang/Compiler-7.0.0/cc1as_main.cpp | 57 +- .../Compiler-7.0.0/cc1gen_reproducer_main.cpp | 196 +++++++ dbms/programs/clang/Compiler-7.0.0/driver.cpp | 4 +- dbms/programs/clang/Compiler-7.0.0/lld.cpp | 156 ++++- dbms/programs/clang/Compiler-7.0.0bundled | 1 - .../Compiler-7.0.0bundled/CMakeLists.txt | 49 ++ .../clang/Compiler-7.0.0bundled/cc1_main.cpp | 243 ++++++++ .../Compiler-7.0.0bundled/cc1as_main.cpp | 555 ++++++++++++++++++ .../clang/Compiler-7.0.0bundled/driver.cpp | 512 ++++++++++++++++ .../clang/Compiler-7.0.0bundled/lld.cpp | 10 + dbms/programs/server/CMakeLists.txt | 4 +- dbms/src/Columns/ColumnVector.cpp | 9 +- dbms/src/Interpreters/ExpressionJIT.cpp | 17 + .../MergeTree/MergeTreeRangeReader.cpp | 2 +- 22 files changed, 1823 insertions(+), 69 deletions(-) create mode 100644 dbms/programs/clang/Compiler-7.0.0/cc1gen_reproducer_main.cpp delete mode 120000 dbms/programs/clang/Compiler-7.0.0bundled create mode 100644 dbms/programs/clang/Compiler-7.0.0bundled/CMakeLists.txt create mode 100644 dbms/programs/clang/Compiler-7.0.0bundled/cc1_main.cpp create mode 100644 dbms/programs/clang/Compiler-7.0.0bundled/cc1as_main.cpp create mode 100644 dbms/programs/clang/Compiler-7.0.0bundled/driver.cpp create mode 100644 dbms/programs/clang/Compiler-7.0.0bundled/lld.cpp diff --git a/CMakeLists.txt b/CMakeLists.txt index 71c258eeaad..552ec57c8fd 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -115,7 +115,7 @@ endif () include (cmake/test_cpu.cmake) -option (ARCH_NATIVE "Enable -march=native compiler flag" OFF) +option (ARCH_NATIVE "Enable -march=native compiler flag" ${ARCH_ARM}) if (ARCH_NATIVE) set (COMPILER_FLAGS "${COMPILER_FLAGS} -march=native") endif () diff --git a/cmake/find_llvm.cmake b/cmake/find_llvm.cmake index e540c3eafa9..81866b720c9 100644 --- a/cmake/find_llvm.cmake +++ b/cmake/find_llvm.cmake @@ -18,12 +18,12 @@ if (ENABLE_EMBEDDED_COMPILER) elseif (CMAKE_CXX_COMPILER_ID STREQUAL "Clang") find_package(LLVM ${CMAKE_CXX_COMPILER_VERSION} CONFIG PATHS ${LLVM_PATHS}) else () - find_package (LLVM 6 CONFIG PATHS ${LLVM_PATHS}) + find_package (LLVM 7 CONFIG PATHS ${LLVM_PATHS}) if (NOT LLVM_FOUND) - find_package (LLVM 5 CONFIG PATHS ${LLVM_PATHS}) + find_package (LLVM 6 CONFIG PATHS ${LLVM_PATHS}) endif () if (NOT LLVM_FOUND) - find_package (LLVM 7 CONFIG PATHS ${LLVM_PATHS}) + find_package (LLVM 5 CONFIG PATHS ${LLVM_PATHS}) endif () endif () diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 2ade502966a..928c48e9334 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -159,16 +159,6 @@ if (USE_INTERNAL_POCO_LIBRARY) endif () endif () -if (USE_INTERNAL_LLVM_LIBRARY) - # ld: unknown option: --color-diagnostics - if (APPLE) - set (LINKER_SUPPORTS_COLOR_DIAGNOSTICS 0 CACHE INTERNAL "") - endif () - set (LLVM_ENABLE_EH 1 CACHE INTERNAL "") - set (LLVM_ENABLE_RTTI 1 CACHE INTERNAL "") - add_subdirectory (llvm/llvm) -endif () - if (USE_INTERNAL_GTEST_LIBRARY) # Google Test from sources add_subdirectory(${ClickHouse_SOURCE_DIR}/contrib/googletest/googletest ${CMAKE_CURRENT_BINARY_DIR}/googletest) @@ -176,3 +166,28 @@ if (USE_INTERNAL_GTEST_LIBRARY) target_compile_definitions (gtest INTERFACE GTEST_HAS_POSIX_RE=0) target_include_directories (gtest SYSTEM INTERFACE ${ClickHouse_SOURCE_DIR}/contrib/googletest/include) endif () + +if (USE_INTERNAL_LLVM_LIBRARY) + file(GENERATE OUTPUT ${CMAKE_CURRENT_BINARY_DIR}/empty.cpp CONTENT " ") + add_library(LLVM0 ${CMAKE_CURRENT_BINARY_DIR}/empty.cpp) # silly cmake bug fix + # ld: unknown option: --color-diagnostics + if (APPLE) + set (LINKER_SUPPORTS_COLOR_DIAGNOSTICS 0 CACHE INTERNAL "") + endif () + set (LLVM_ENABLE_EH 1 CACHE INTERNAL "") + set (LLVM_ENABLE_RTTI 1 CACHE INTERNAL "") + set (LLVM_INCLUDE_TESTS 0 CACHE INTERNAL "") + set (LLVM_INCLUDE_EXAMPLES 0 CACHE INTERNAL "") + set (LLVM_INCLUDE_TOOLS 0 CACHE INTERNAL "") + set (LLVM_INSTALL_TOOLCHAIN_ONLY 0 CACHE INTERNAL "") + set (CLANG_BUILT_STANDALONE 0 CACHE INTERNAL "") + set (LLDB_BUILT_STANDALONE 0 CACHE INTERNAL "") + set (CLANG_ENABLE_STATIC_ANALYZER 0 CACHE INTERNAL "") + set (CLANG_ENABLE_ARCMT 0 CACHE INTERNAL "") + set (CLANG_BUILD_TOOLS 0 CACHE INTERNAL "") + set (BENCHMARK_ENABLE_GTEST_TESTS 0 CACHE INTERNAL "") + set (BENCHMARK_ENABLE_ASSEMBLY_TESTS 0 CACHE INTERNAL "") + set (LLVM_TARGETS_TO_BUILD "X86;AArch64" CACHE INTERNAL "") + add_subdirectory (llvm/llvm) +endif () + diff --git a/dbms/programs/clang/Compiler-5.0.0/CMakeLists.txt b/dbms/programs/clang/Compiler-5.0.0/CMakeLists.txt index 43c35e5eccc..7fe0cd92ef7 100644 --- a/dbms/programs/clang/Compiler-5.0.0/CMakeLists.txt +++ b/dbms/programs/clang/Compiler-5.0.0/CMakeLists.txt @@ -1,5 +1,7 @@ add_definitions(-Wno-error -Wno-unused-parameter -Wno-non-virtual-dtor -U_LIBCPP_DEBUG) +link_directories(${LLVM_LIBRARY_DIRS}) + add_library(clickhouse-compiler-lib ${LINK_MODE} driver.cpp cc1_main.cpp diff --git a/dbms/programs/clang/Compiler-6.0.0/CMakeLists.txt b/dbms/programs/clang/Compiler-6.0.0/CMakeLists.txt index bea78846705..b96bdb0647a 100644 --- a/dbms/programs/clang/Compiler-6.0.0/CMakeLists.txt +++ b/dbms/programs/clang/Compiler-6.0.0/CMakeLists.txt @@ -1,5 +1,7 @@ add_definitions(-Wno-error -Wno-unused-parameter -Wno-non-virtual-dtor -U_LIBCPP_DEBUG) +link_directories(${LLVM_LIBRARY_DIRS}) + add_library(clickhouse-compiler-lib ${LINK_MODE} driver.cpp cc1_main.cpp diff --git a/dbms/programs/clang/Compiler-6.0.0/cc1as_main.cpp b/dbms/programs/clang/Compiler-6.0.0/cc1as_main.cpp index 191c7df7ab1..caf8409054a 100644 --- a/dbms/programs/clang/Compiler-6.0.0/cc1as_main.cpp +++ b/dbms/programs/clang/Compiler-6.0.0/cc1as_main.cpp @@ -441,7 +441,7 @@ static bool ExecuteAssembler(AssemblerInvocation &Opts, auto Pair = StringRef(S).split('='); auto Sym = Pair.first; auto Val = Pair.second; - int64_t Value; + int64_t Value = 0; // We have already error checked this in the driver. Val.getAsInteger(0, Value); Ctx.setSymbolValue(Parser->getStreamer(), Sym, Value); diff --git a/dbms/programs/clang/Compiler-7.0.0/CMakeLists.txt b/dbms/programs/clang/Compiler-7.0.0/CMakeLists.txt index 009f09fe6aa..8b6ba6be994 100644 --- a/dbms/programs/clang/Compiler-7.0.0/CMakeLists.txt +++ b/dbms/programs/clang/Compiler-7.0.0/CMakeLists.txt @@ -1,8 +1,11 @@ add_definitions(-Wno-error -Wno-unused-parameter -Wno-non-virtual-dtor -U_LIBCPP_DEBUG) +link_directories(${LLVM_LIBRARY_DIRS}) + add_library(clickhouse-compiler-lib ${LINK_MODE} driver.cpp cc1_main.cpp + cc1gen_reproducer_main.cpp cc1as_main.cpp lld.cpp) @@ -19,7 +22,6 @@ target_include_directories(clickhouse-compiler-lib SYSTEM PRIVATE ${LLVM_INCLUDE # This is extracted almost directly from CMakeFiles/.../link.txt in LLVM build directory. target_link_libraries(clickhouse-compiler-lib PRIVATE - clangBasic clangCodeGen clangDriver clangFrontend clangFrontendTool diff --git a/dbms/programs/clang/Compiler-7.0.0/cc1_main.cpp b/dbms/programs/clang/Compiler-7.0.0/cc1_main.cpp index 3686475dd42..214bfa72476 100644 --- a/dbms/programs/clang/Compiler-7.0.0/cc1_main.cpp +++ b/dbms/programs/clang/Compiler-7.0.0/cc1_main.cpp @@ -16,6 +16,7 @@ #include "llvm/Option/Arg.h" #include "clang/CodeGen/ObjectFilePCHContainerOperations.h" #include "clang/Config/config.h" +#include "clang/Basic/Stack.h" #include "clang/Driver/DriverDiagnostic.h" #include "clang/Driver/Options.h" #include "clang/Frontend/CompilerInstance.h" @@ -76,13 +77,6 @@ void initializePollyPasses(llvm::PassRegistry &Registry); #endif #ifdef CLANG_HAVE_RLIMITS -// The amount of stack we think is "sufficient". If less than this much is -// available, we may be unable to reach our template instantiation depth -// limit and other similar limits. -// FIXME: Unify this with the stack we request when spawning a thread to build -// a module. -static const int kSufficientStack = 8 << 20; - #if defined(__linux__) && defined(__PIE__) static size_t getCurrentStackAllocation() { // If we can't compute the current stack usage, allow for 512K of command @@ -120,7 +114,7 @@ static size_t getCurrentStackAllocation() { #include LLVM_ATTRIBUTE_NOINLINE -static void ensureStackAddressSpace(int ExtraChunks = 0) { +static void ensureStackAddressSpace() { // Linux kernels prior to 4.1 will sometimes locate the heap of a PIE binary // relatively close to the stack (they are only guaranteed to be 128MiB // apart). This results in crashes if we happen to heap-allocate more than @@ -129,7 +123,7 @@ static void ensureStackAddressSpace(int ExtraChunks = 0) { // To avoid these crashes, ensure that we have sufficient virtual memory // pages allocated before we start running. size_t Curr = getCurrentStackAllocation(); - const int kTargetStack = kSufficientStack - 256 * 1024; + const int kTargetStack = DesiredStackSize - 256 * 1024; if (Curr < kTargetStack) { volatile char *volatile Alloc = static_cast(alloca(kTargetStack - Curr)); @@ -149,21 +143,23 @@ static void ensureSufficientStack() { // Increase the soft stack limit to our desired level, if necessary and // possible. - if (rlim.rlim_cur != RLIM_INFINITY && rlim.rlim_cur < kSufficientStack) { + if (rlim.rlim_cur != RLIM_INFINITY && + rlim.rlim_cur < rlim_t(DesiredStackSize)) { // Try to allocate sufficient stack. - if (rlim.rlim_max == RLIM_INFINITY || rlim.rlim_max >= kSufficientStack) - rlim.rlim_cur = kSufficientStack; + if (rlim.rlim_max == RLIM_INFINITY || + rlim.rlim_max >= rlim_t(DesiredStackSize)) + rlim.rlim_cur = DesiredStackSize; else if (rlim.rlim_cur == rlim.rlim_max) return; else rlim.rlim_cur = rlim.rlim_max; if (setrlimit(RLIMIT_STACK, &rlim) != 0 || - rlim.rlim_cur != kSufficientStack) + rlim.rlim_cur != DesiredStackSize) return; } - // We should now have a stack of size at least kSufficientStack. Ensure + // We should now have a stack of size at least DesiredStackSize. Ensure // that we can actually use that much, if necessary. ensureStackAddressSpace(); } diff --git a/dbms/programs/clang/Compiler-7.0.0/cc1as_main.cpp b/dbms/programs/clang/Compiler-7.0.0/cc1as_main.cpp index ce23422077f..d93b1f5cb1d 100644 --- a/dbms/programs/clang/Compiler-7.0.0/cc1as_main.cpp +++ b/dbms/programs/clang/Compiler-7.0.0/cc1as_main.cpp @@ -29,6 +29,7 @@ #include "llvm/MC/MCContext.h" #include "llvm/MC/MCInstrInfo.h" #include "llvm/MC/MCObjectFileInfo.h" +#include "llvm/MC/MCObjectWriter.h" #include "llvm/MC/MCParser/MCAsmParser.h" #include "llvm/MC/MCParser/MCTargetAsmParser.h" #include "llvm/MC/MCRegisterInfo.h" @@ -59,10 +60,9 @@ using namespace clang::driver::options; using namespace llvm; using namespace llvm::opt; - namespace { -/// \brief Helper class for representing a single invocation of the assembler. +/// Helper class for representing a single invocation of the assembler. struct AssemblerInvocation { /// @name Target Options /// @{ @@ -94,9 +94,11 @@ struct AssemblerInvocation { std::string DwarfDebugFlags; std::string DwarfDebugProducer; std::string DebugCompilationDir; + std::map DebugPrefixMap; llvm::DebugCompressionType CompressDebugSections = llvm::DebugCompressionType::None; std::string MainFileName; + std::string SplitDwarfFile; /// @} /// @name Frontend Options @@ -232,6 +234,9 @@ bool AssemblerInvocation::CreateFromArgs(AssemblerInvocation &Opts, Opts.DebugCompilationDir = Args.getLastArgValue(OPT_fdebug_compilation_dir); Opts.MainFileName = Args.getLastArgValue(OPT_main_file_name); + for (const auto &Arg : Args.getAllArgValues(OPT_fdebug_prefix_map_EQ)) + Opts.DebugPrefixMap.insert(StringRef(Arg).split('=')); + // Frontend Options if (Args.hasArg(OPT_INPUT)) { bool First = true; @@ -247,6 +252,7 @@ bool AssemblerInvocation::CreateFromArgs(AssemblerInvocation &Opts, } Opts.LLVMArgs = Args.getAllArgValues(OPT_mllvm); Opts.OutputPath = Args.getLastArgValue(OPT_o); + Opts.SplitDwarfFile = Args.getLastArgValue(OPT_split_dwarf_file); if (Arg *A = Args.getLastArg(OPT_filetype)) { StringRef Name = A->getValue(); unsigned OutputType = StringSwitch(Name) @@ -282,22 +288,17 @@ bool AssemblerInvocation::CreateFromArgs(AssemblerInvocation &Opts, } static std::unique_ptr -getOutputStream(AssemblerInvocation &Opts, DiagnosticsEngine &Diags, - bool Binary) { - if (Opts.OutputPath.empty()) - Opts.OutputPath = "-"; - +getOutputStream(StringRef Path, DiagnosticsEngine &Diags, bool Binary) { // Make sure that the Out file gets unlinked from the disk if we get a // SIGINT. - if (Opts.OutputPath != "-") - sys::RemoveFileOnSignal(Opts.OutputPath); + if (Path != "-") + sys::RemoveFileOnSignal(Path); std::error_code EC; auto Out = llvm::make_unique( - Opts.OutputPath, EC, (Binary ? sys::fs::F_None : sys::fs::F_Text)); + Path, EC, (Binary ? sys::fs::F_None : sys::fs::F_Text)); if (EC) { - Diags.Report(diag::err_fe_unable_to_open_output) << Opts.OutputPath - << EC.message(); + Diags.Report(diag::err_fe_unable_to_open_output) << Path << EC.message(); return nullptr; } @@ -342,9 +343,15 @@ static bool ExecuteAssembler(AssemblerInvocation &Opts, MAI->setRelaxELFRelocations(Opts.RelaxELFRelocations); bool IsBinary = Opts.OutputType == AssemblerInvocation::FT_Obj; - std::unique_ptr FDOS = getOutputStream(Opts, Diags, IsBinary); + if (Opts.OutputPath.empty()) + Opts.OutputPath = "-"; + std::unique_ptr FDOS = + getOutputStream(Opts.OutputPath, Diags, IsBinary); if (!FDOS) return true; + std::unique_ptr DwoOS; + if (!Opts.SplitDwarfFile.empty()) + DwoOS = getOutputStream(Opts.SplitDwarfFile, Diags, IsBinary); // FIXME: This is not pretty. MCContext has a ptr to MCObjectFileInfo and // MCObjectFileInfo needs a MCContext reference in order to initialize itself. @@ -374,6 +381,9 @@ static bool ExecuteAssembler(AssemblerInvocation &Opts, Ctx.setDwarfDebugProducer(StringRef(Opts.DwarfDebugProducer)); if (!Opts.DebugCompilationDir.empty()) Ctx.setCompilationDir(Opts.DebugCompilationDir); + if (!Opts.DebugPrefixMap.empty()) + for (const auto &KV : Opts.DebugPrefixMap) + Ctx.addDebugPrefixMapEntry(KV.first, KV.second); if (!Opts.MainFileName.empty()) Ctx.setMainFileName(StringRef(Opts.MainFileName)); Ctx.setDwarfVersion(Opts.DwarfVersion); @@ -427,11 +437,14 @@ static bool ExecuteAssembler(AssemblerInvocation &Opts, MCTargetOptions MCOptions; std::unique_ptr MAB( TheTarget->createMCAsmBackend(*STI, *MRI, MCOptions)); + std::unique_ptr OW = + DwoOS ? MAB->createDwoObjectWriter(*Out, *DwoOS) + : MAB->createObjectWriter(*Out); Triple T(Opts.Triple); Str.reset(TheTarget->createMCObjectStreamer( - T, Ctx, std::move(MAB), *Out, std::move(CE), *STI, Opts.RelaxAll, - Opts.IncrementalLinkerCompatible, + T, Ctx, std::move(MAB), std::move(OW), std::move(CE), *STI, + Opts.RelaxAll, Opts.IncrementalLinkerCompatible, /*DWARFMustBeAtTheEnd*/ true)); Str.get()->InitSections(Opts.NoExecStack); } @@ -456,7 +469,7 @@ static bool ExecuteAssembler(AssemblerInvocation &Opts, auto Pair = StringRef(S).split('='); auto Sym = Pair.first; auto Val = Pair.second; - int64_t Value; + int64_t Value = 1; // We have already error checked this in the driver. Val.getAsInteger(0, Value); Ctx.setSymbolValue(Parser->getStreamer(), Sym, Value); @@ -475,14 +488,18 @@ static bool ExecuteAssembler(AssemblerInvocation &Opts, FDOS.reset(); // Delete output file if there were errors. - if (Failed && Opts.OutputPath != "-") - sys::fs::remove(Opts.OutputPath); + if (Failed) { + if (Opts.OutputPath != "-") + sys::fs::remove(Opts.OutputPath); + if (!Opts.SplitDwarfFile.empty() && Opts.SplitDwarfFile != "-") + sys::fs::remove(Opts.SplitDwarfFile); + } return Failed; } static void LLVMErrorHandler(void *UserData, const std::string &Message, - bool /*GenCrashDiag*/) { + bool GenCrashDiag) { DiagnosticsEngine &Diags = *static_cast(UserData); Diags.Report(diag::err_fe_error_backend) << Message; @@ -491,7 +508,7 @@ static void LLVMErrorHandler(void *UserData, const std::string &Message, exit(1); } -int cc1as_main(ArrayRef Argv, const char */*Argv0*/, void */*MainAddr*/) { +int cc1as_main(ArrayRef Argv, const char *Argv0, void *MainAddr) { // Initialize targets and assembly printers/parsers. InitializeAllTargetInfos(); InitializeAllTargetMCs(); diff --git a/dbms/programs/clang/Compiler-7.0.0/cc1gen_reproducer_main.cpp b/dbms/programs/clang/Compiler-7.0.0/cc1gen_reproducer_main.cpp new file mode 100644 index 00000000000..a4c034d8d35 --- /dev/null +++ b/dbms/programs/clang/Compiler-7.0.0/cc1gen_reproducer_main.cpp @@ -0,0 +1,196 @@ +//===-- cc1gen_reproducer_main.cpp - Clang reproducer generator ----------===// +// +// The LLVM Compiler Infrastructure +// +// This file is distributed under the University of Illinois Open Source +// License. See LICENSE.TXT for details. +// +//===----------------------------------------------------------------------===// +// +// This is the entry point to the clang -cc1gen-reproducer functionality, which +// generates reproducers for invocations for clang-based tools. +// +//===----------------------------------------------------------------------===// + +#include "clang/Basic/Diagnostic.h" +#include "clang/Basic/LLVM.h" +#include "clang/Basic/VirtualFileSystem.h" +#include "clang/Driver/Compilation.h" +#include "clang/Driver/Driver.h" +#include "llvm/ADT/ArrayRef.h" +#include "llvm/ADT/STLExtras.h" +#include "llvm/Support/FileSystem.h" +#include "llvm/Support/TargetSelect.h" +#include "llvm/Support/YAMLTraits.h" +#include "llvm/Support/raw_ostream.h" + +using namespace clang; + +namespace { + +struct UnsavedFileHash { + std::string Name; + std::string MD5; +}; + +struct ClangInvocationInfo { + std::string Toolchain; + std::string LibclangOperation; + std::string LibclangOptions; + std::vector Arguments; + std::vector InvocationArguments; + std::vector UnsavedFileHashes; + bool Dump = false; +}; + +} // end anonymous namespace + +LLVM_YAML_IS_SEQUENCE_VECTOR(UnsavedFileHash) + +namespace llvm { +namespace yaml { + +template <> struct MappingTraits { + static void mapping(IO &IO, UnsavedFileHash &Info) { + IO.mapRequired("name", Info.Name); + IO.mapRequired("md5", Info.MD5); + } +}; + +template <> struct MappingTraits { + static void mapping(IO &IO, ClangInvocationInfo &Info) { + IO.mapRequired("toolchain", Info.Toolchain); + IO.mapOptional("libclang.operation", Info.LibclangOperation); + IO.mapOptional("libclang.opts", Info.LibclangOptions); + IO.mapRequired("args", Info.Arguments); + IO.mapOptional("invocation-args", Info.InvocationArguments); + IO.mapOptional("unsaved_file_hashes", Info.UnsavedFileHashes); + } +}; + +} // end namespace yaml +} // end namespace llvm + +static std::string generateReproducerMetaInfo(const ClangInvocationInfo &Info) { + std::string Result; + llvm::raw_string_ostream OS(Result); + OS << '{'; + bool NeedComma = false; + auto EmitKey = [&](StringRef Key) { + if (NeedComma) + OS << ", "; + NeedComma = true; + OS << '"' << Key << "\": "; + }; + auto EmitStringKey = [&](StringRef Key, StringRef Value) { + if (Value.empty()) + return; + EmitKey(Key); + OS << '"' << Value << '"'; + }; + EmitStringKey("libclang.operation", Info.LibclangOperation); + EmitStringKey("libclang.opts", Info.LibclangOptions); + if (!Info.InvocationArguments.empty()) { + EmitKey("invocation-args"); + OS << '['; + for (const auto &Arg : llvm::enumerate(Info.InvocationArguments)) { + if (Arg.index()) + OS << ','; + OS << '"' << Arg.value() << '"'; + } + OS << ']'; + } + OS << '}'; + // FIXME: Compare unsaved file hashes and report mismatch in the reproducer. + if (Info.Dump) + llvm::outs() << "REPRODUCER METAINFO: " << OS.str() << "\n"; + return std::move(OS.str()); +} + +/// Generates a reproducer for a set of arguments from a specific invocation. +static llvm::Optional +generateReproducerForInvocationArguments(ArrayRef Argv, + const ClangInvocationInfo &Info) { + using namespace driver; + auto TargetAndMode = ToolChain::getTargetAndModeFromProgramName(Argv[0]); + + IntrusiveRefCntPtr DiagOpts = new DiagnosticOptions; + + IntrusiveRefCntPtr DiagID(new DiagnosticIDs()); + DiagnosticsEngine Diags(DiagID, &*DiagOpts, new IgnoringDiagConsumer()); + ProcessWarningOptions(Diags, *DiagOpts, /*ReportDiags=*/false); + Driver TheDriver(Argv[0], llvm::sys::getDefaultTargetTriple(), Diags); + TheDriver.setTargetAndMode(TargetAndMode); + + std::unique_ptr C(TheDriver.BuildCompilation(Argv)); + if (C && !C->containsError()) { + for (const auto &J : C->getJobs()) { + if (const Command *Cmd = dyn_cast(&J)) { + Driver::CompilationDiagnosticReport Report; + TheDriver.generateCompilationDiagnostics( + *C, *Cmd, generateReproducerMetaInfo(Info), &Report); + return Report; + } + } + } + + return None; +} + +std::string GetExecutablePath(const char *Argv0, bool CanonicalPrefixes); + +static void printReproducerInformation( + llvm::raw_ostream &OS, const ClangInvocationInfo &Info, + const driver::Driver::CompilationDiagnosticReport &Report) { + OS << "REPRODUCER:\n"; + OS << "{\n"; + OS << R"("files":[)"; + for (const auto &File : llvm::enumerate(Report.TemporaryFiles)) { + if (File.index()) + OS << ','; + OS << '"' << File.value() << '"'; + } + OS << "]\n}\n"; +} + +int cc1gen_reproducer_main(ArrayRef Argv, const char *Argv0, + void *MainAddr) { + if (Argv.size() < 1) { + llvm::errs() << "error: missing invocation file\n"; + return 1; + } + // Parse the invocation descriptor. + StringRef Input = Argv[0]; + llvm::ErrorOr> Buffer = + llvm::MemoryBuffer::getFile(Input); + if (!Buffer) { + llvm::errs() << "error: failed to read " << Input << ": " + << Buffer.getError().message() << "\n"; + return 1; + } + llvm::yaml::Input YAML(Buffer.get()->getBuffer()); + ClangInvocationInfo InvocationInfo; + YAML >> InvocationInfo; + if (Argv.size() > 1 && Argv[1] == StringRef("-v")) + InvocationInfo.Dump = true; + + // Create an invocation that will produce the reproducer. + std::vector DriverArgs; + for (const auto &Arg : InvocationInfo.Arguments) + DriverArgs.push_back(Arg.c_str()); + std::string Path = GetExecutablePath(Argv0, /*CanonicalPrefixes=*/true); + DriverArgs[0] = Path.c_str(); + llvm::Optional Report = + generateReproducerForInvocationArguments(DriverArgs, InvocationInfo); + + // Emit the information about the reproduce files to stdout. + int Result = 1; + if (Report) { + printReproducerInformation(llvm::outs(), InvocationInfo, *Report); + Result = 0; + } + + // Remove the input file. + llvm::sys::fs::remove(Input); + return Result; +} diff --git a/dbms/programs/clang/Compiler-7.0.0/driver.cpp b/dbms/programs/clang/Compiler-7.0.0/driver.cpp index 9a061b9d137..79d71b08ba7 100644 --- a/dbms/programs/clang/Compiler-7.0.0/driver.cpp +++ b/dbms/programs/clang/Compiler-7.0.0/driver.cpp @@ -310,6 +310,8 @@ static int ExecuteCC1Tool(ArrayRef argv, StringRef Tool) { return cc1_main(argv.slice(2), argv[0], GetExecutablePathVP); if (Tool == "as") return cc1as_main(argv.slice(2), argv[0], GetExecutablePathVP); + if (Tool == "gen-reproducer") + return cc1gen_reproducer_main(argv.slice(2), argv[0], GetExecutablePathVP); // Reject unknown tools. llvm::errs() << "error: unknown integrated tool '" << Tool << "'. " @@ -317,7 +319,7 @@ static int ExecuteCC1Tool(ArrayRef argv, StringRef Tool) { return 1; } -int mainEntryClickHouseClang(int argc_, char **argv_) { +int mainEntryClickHouseClang(int argc_, /* const */ char **argv_) { llvm::InitLLVM X(argc_, argv_); SmallVector argv(argv_, argv_ + argc_); diff --git a/dbms/programs/clang/Compiler-7.0.0/lld.cpp b/dbms/programs/clang/Compiler-7.0.0/lld.cpp index 203e50d42a9..8e118b6e24b 100644 --- a/dbms/programs/clang/Compiler-7.0.0/lld.cpp +++ b/dbms/programs/clang/Compiler-7.0.0/lld.cpp @@ -1,10 +1,150 @@ -#include "lld/Common/Driver.h" -#include "llvm/Support/InitLLVM.h" -#include +//===- tools/lld/lld.cpp - Linker Driver Dispatcher -----------------------===// +// +// The LLVM Linker +// +// This file is distributed under the University of Illinois Open Source +// License. See LICENSE.TXT for details. +// +//===----------------------------------------------------------------------===// +// +// This file contains the main function of the lld executable. The main +// function is a thin wrapper which dispatches to the platform specific +// driver. +// +// lld is a single executable that contains four different linkers for ELF, +// COFF, WebAssembly and Mach-O. The main function dispatches according to +// argv[0] (i.e. command name). The most common name for each target is shown +// below: +// +// - ld.lld: ELF (Unix) +// - ld64: Mach-O (macOS) +// - lld-link: COFF (Windows) +// - ld-wasm: WebAssembly +// +// lld can be invoked as "lld" along with "-flavor" option. This is for +// backward compatibility and not recommended. +// +//===----------------------------------------------------------------------===// -int mainEntryClickHouseLLD(int argc, char ** argv) -{ - llvm::InitLLVM X(argc, argv); - std::vector args(argv, argv + argc); - return !lld::elf::link(args, false); +#include "lld/Common/Driver.h" +#include "llvm/ADT/STLExtras.h" +#include "llvm/ADT/StringSwitch.h" +#include "llvm/ADT/Twine.h" +#include "llvm/Support/InitLLVM.h" +#include "llvm/Support/Path.h" +#include +using namespace lld; +using namespace llvm; +using namespace llvm::sys; + +/* + +enum Flavor { + Invalid, + Gnu, // -flavor gnu + WinLink, // -flavor link + Darwin, // -flavor darwin + Wasm, // -flavor wasm +}; + +LLVM_ATTRIBUTE_NORETURN static void die(const Twine &S) { + errs() << S << "\n"; + exit(1); +} + +static Flavor getFlavor(StringRef S) { + return StringSwitch(S) + .CasesLower("ld", "ld.lld", "gnu", Gnu) + .CasesLower("wasm", "ld-wasm", Wasm) + .CaseLower("link", WinLink) + .CasesLower("ld64", "ld64.lld", "darwin", Darwin) + .Default(Invalid); +} + +static bool isPETarget(const std::vector &V) { + for (auto It = V.begin(); It + 1 != V.end(); ++It) { + if (StringRef(*It) != "-m") + continue; + StringRef S = *(It + 1); + return S == "i386pe" || S == "i386pep" || S == "thumb2pe" || S == "arm64pe"; + } + return false; +} + +static Flavor parseProgname(StringRef Progname) { +#if __APPLE__ + // Use Darwin driver for "ld" on Darwin. + if (Progname == "ld") + return Darwin; +#endif + +#if LLVM_ON_UNIX + // Use GNU driver for "ld" on other Unix-like system. + if (Progname == "ld") + return Gnu; +#endif + + // Progname may be something like "lld-gnu". Parse it. + SmallVector V; + Progname.split(V, "-"); + for (StringRef S : V) + if (Flavor F = getFlavor(S)) + return F; + return Invalid; +} + +static Flavor parseFlavor(std::vector &V) { + // Parse -flavor option. + if (V.size() > 1 && V[1] == StringRef("-flavor")) { + if (V.size() <= 2) + die("missing arg value for '-flavor'"); + Flavor F = getFlavor(V[2]); + if (F == Invalid) + die("Unknown flavor: " + StringRef(V[2])); + V.erase(V.begin() + 1, V.begin() + 3); + return F; + } + + // Deduct the flavor from argv[0]. + StringRef Arg0 = path::filename(V[0]); + if (Arg0.endswith_lower(".exe")) + Arg0 = Arg0.drop_back(4); + return parseProgname(Arg0); +} +*/ + +// If this function returns true, lld calls _exit() so that it quickly +// exits without invoking destructors of globally allocated objects. +// +// We don't want to do that if we are running tests though, because +// doing that breaks leak sanitizer. So, lit sets this environment variable, +// and we use it to detect whether we are running tests or not. +static bool canExitEarly() { return StringRef(getenv("LLD_IN_TEST")) != "1"; } + +/// Universal linker main(). This linker emulates the gnu, darwin, or +/// windows linker based on the argv[0] or -flavor option. +int mainEntryClickHouseLLD(int Argc, /* const */ char **Argv) { + InitLLVM X(Argc, Argv); + + std::vector Args(Argv, Argv + Argc); +/* + switch (parseFlavor(Args)) { + case Gnu: + if (isPETarget(Args)) + return !mingw::link(Args); +*/ + return !elf::link(Args, canExitEarly()); +/* + case WinLink: + return !coff::link(Args, canExitEarly()); + case Darwin: + return !mach_o::link(Args, canExitEarly()); + case Wasm: + return !wasm::link(Args, canExitEarly()); + default: + die("lld is a generic driver.\n" + "Invoke ld.lld (Unix), ld64.lld (macOS), lld-link (Windows), wasm-lld" + " (WebAssembly) instead"); + } +*/ } diff --git a/dbms/programs/clang/Compiler-7.0.0bundled b/dbms/programs/clang/Compiler-7.0.0bundled deleted file mode 120000 index eeeb5bbc2c0..00000000000 --- a/dbms/programs/clang/Compiler-7.0.0bundled +++ /dev/null @@ -1 +0,0 @@ -Compiler-7.0.0 \ No newline at end of file diff --git a/dbms/programs/clang/Compiler-7.0.0bundled/CMakeLists.txt b/dbms/programs/clang/Compiler-7.0.0bundled/CMakeLists.txt new file mode 100644 index 00000000000..d0ccc8d672c --- /dev/null +++ b/dbms/programs/clang/Compiler-7.0.0bundled/CMakeLists.txt @@ -0,0 +1,49 @@ +add_definitions(-Wno-error -Wno-unused-parameter -Wno-non-virtual-dtor -U_LIBCPP_DEBUG) + +link_directories(${LLVM_LIBRARY_DIRS}) + +add_library(clickhouse-compiler-lib ${LINK_MODE} + driver.cpp + cc1_main.cpp + cc1as_main.cpp + lld.cpp) + +target_compile_options(clickhouse-compiler-lib PRIVATE -fno-rtti -fno-exceptions -g0) + +string(REPLACE "${INCLUDE_DEBUG_HELPERS}" "" CMAKE_CXX_FLAGS ${CMAKE_CXX_FLAGS}) # cant compile with -fno-rtti + +llvm_libs_all(REQUIRED_LLVM_LIBRARIES) + +message(STATUS "Using LLVM ${LLVM_VERSION}: ${LLVM_INCLUDE_DIRS} : ${REQUIRED_LLVM_LIBRARIES}") + +target_include_directories(clickhouse-compiler-lib SYSTEM PRIVATE ${LLVM_INCLUDE_DIRS}) + +# This is extracted almost directly from CMakeFiles/.../link.txt in LLVM build directory. + +target_link_libraries(clickhouse-compiler-lib PRIVATE + +clangBasic clangCodeGen clangDriver +clangFrontend +clangFrontendTool +clangRewriteFrontend clangARCMigrate clangStaticAnalyzerFrontend +clangParse clangSerialization clangSema clangEdit clangStaticAnalyzerCheckers +clangASTMatchers clangStaticAnalyzerCore clangAnalysis clangAST clangRewrite clangLex clangBasic +clangCrossTU clangIndex + +lldCOFF +lldDriver +lldELF +lldMinGW +lldMachO +lldReaderWriter +lldYAML +lldCommon +lldCore + +${REQUIRED_LLVM_LIBRARIES} + +PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARY} Threads::Threads +${MALLOC_LIBRARIES} +${GLIBC_COMPATIBILITY_LIBRARIES} +${MEMCPY_LIBRARIES} +) diff --git a/dbms/programs/clang/Compiler-7.0.0bundled/cc1_main.cpp b/dbms/programs/clang/Compiler-7.0.0bundled/cc1_main.cpp new file mode 100644 index 00000000000..3686475dd42 --- /dev/null +++ b/dbms/programs/clang/Compiler-7.0.0bundled/cc1_main.cpp @@ -0,0 +1,243 @@ +//===-- cc1_main.cpp - Clang CC1 Compiler Frontend ------------------------===// +// +// The LLVM Compiler Infrastructure +// +// This file is distributed under the University of Illinois Open Source +// License. See LICENSE.TXT for details. +// +//===----------------------------------------------------------------------===// +// +// This is the entry point to the clang -cc1 functionality, which implements the +// core compiler functionality along with a number of additional tools for +// demonstration and testing purposes. +// +//===----------------------------------------------------------------------===// + +#include "llvm/Option/Arg.h" +#include "clang/CodeGen/ObjectFilePCHContainerOperations.h" +#include "clang/Config/config.h" +#include "clang/Driver/DriverDiagnostic.h" +#include "clang/Driver/Options.h" +#include "clang/Frontend/CompilerInstance.h" +#include "clang/Frontend/CompilerInvocation.h" +#include "clang/Frontend/FrontendDiagnostic.h" +#include "clang/Frontend/TextDiagnosticBuffer.h" +#include "clang/Frontend/TextDiagnosticPrinter.h" +#include "clang/Frontend/Utils.h" +#include "clang/FrontendTool/Utils.h" +#include "llvm/ADT/Statistic.h" +#include "llvm/Config/llvm-config.h" +#include "llvm/LinkAllPasses.h" +#include "llvm/Option/ArgList.h" +#include "llvm/Option/OptTable.h" +#include "llvm/Support/Compiler.h" +#include "llvm/Support/ErrorHandling.h" +#include "llvm/Support/ManagedStatic.h" +#include "llvm/Support/Signals.h" +#include "llvm/Support/TargetSelect.h" +#include "llvm/Support/Timer.h" +#include "llvm/Support/raw_ostream.h" +#include + +#ifdef CLANG_HAVE_RLIMITS +#include +#endif + +// have no .a version in packages +#undef LINK_POLLY_INTO_TOOLS + +using namespace clang; +using namespace llvm::opt; + +//===----------------------------------------------------------------------===// +// Main driver +//===----------------------------------------------------------------------===// + +static void LLVMErrorHandler(void *UserData, const std::string &Message, + bool GenCrashDiag) { + DiagnosticsEngine &Diags = *static_cast(UserData); + + Diags.Report(diag::err_fe_error_backend) << Message; + + // Run the interrupt handlers to make sure any special cleanups get done, in + // particular that we remove files registered with RemoveFileOnSignal. + llvm::sys::RunInterruptHandlers(); + + // We cannot recover from llvm errors. When reporting a fatal error, exit + // with status 70 to generate crash diagnostics. For BSD systems this is + // defined as an internal software error. Otherwise, exit with status 1. + exit(GenCrashDiag ? 70 : 1); +} + +#ifdef LINK_POLLY_INTO_TOOLS +namespace polly { +void initializePollyPasses(llvm::PassRegistry &Registry); +} +#endif + +#ifdef CLANG_HAVE_RLIMITS +// The amount of stack we think is "sufficient". If less than this much is +// available, we may be unable to reach our template instantiation depth +// limit and other similar limits. +// FIXME: Unify this with the stack we request when spawning a thread to build +// a module. +static const int kSufficientStack = 8 << 20; + +#if defined(__linux__) && defined(__PIE__) +static size_t getCurrentStackAllocation() { + // If we can't compute the current stack usage, allow for 512K of command + // line arguments and environment. + size_t Usage = 512 * 1024; + if (FILE *StatFile = fopen("/proc/self/stat", "r")) { + // We assume that the stack extends from its current address to the end of + // the environment space. In reality, there is another string literal (the + // program name) after the environment, but this is close enough (we only + // need to be within 100K or so). + unsigned long StackPtr, EnvEnd; + // Disable silly GCC -Wformat warning that complains about length + // modifiers on ignored format specifiers. We want to retain these + // for documentation purposes even though they have no effect. +#if defined(__GNUC__) && !defined(__clang__) +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wformat" +#endif + if (fscanf(StatFile, + "%*d %*s %*c %*d %*d %*d %*d %*d %*u %*lu %*lu %*lu %*lu %*lu " + "%*lu %*ld %*ld %*ld %*ld %*ld %*ld %*llu %*lu %*ld %*lu %*lu " + "%*lu %*lu %lu %*lu %*lu %*lu %*lu %*lu %*llu %*lu %*lu %*d %*d " + "%*u %*u %*llu %*lu %*ld %*lu %*lu %*lu %*lu %*lu %*lu %lu %*d", + &StackPtr, &EnvEnd) == 2) { +#if defined(__GNUC__) && !defined(__clang__) +#pragma GCC diagnostic pop +#endif + Usage = StackPtr < EnvEnd ? EnvEnd - StackPtr : StackPtr - EnvEnd; + } + fclose(StatFile); + } + return Usage; +} + +#include + +LLVM_ATTRIBUTE_NOINLINE +static void ensureStackAddressSpace(int ExtraChunks = 0) { + // Linux kernels prior to 4.1 will sometimes locate the heap of a PIE binary + // relatively close to the stack (they are only guaranteed to be 128MiB + // apart). This results in crashes if we happen to heap-allocate more than + // 128MiB before we reach our stack high-water mark. + // + // To avoid these crashes, ensure that we have sufficient virtual memory + // pages allocated before we start running. + size_t Curr = getCurrentStackAllocation(); + const int kTargetStack = kSufficientStack - 256 * 1024; + if (Curr < kTargetStack) { + volatile char *volatile Alloc = + static_cast(alloca(kTargetStack - Curr)); + Alloc[0] = 0; + Alloc[kTargetStack - Curr - 1] = 0; + } +} +#else +static void ensureStackAddressSpace() {} +#endif + +/// Attempt to ensure that we have at least 8MiB of usable stack space. +static void ensureSufficientStack() { + struct rlimit rlim; + if (getrlimit(RLIMIT_STACK, &rlim) != 0) + return; + + // Increase the soft stack limit to our desired level, if necessary and + // possible. + if (rlim.rlim_cur != RLIM_INFINITY && rlim.rlim_cur < kSufficientStack) { + // Try to allocate sufficient stack. + if (rlim.rlim_max == RLIM_INFINITY || rlim.rlim_max >= kSufficientStack) + rlim.rlim_cur = kSufficientStack; + else if (rlim.rlim_cur == rlim.rlim_max) + return; + else + rlim.rlim_cur = rlim.rlim_max; + + if (setrlimit(RLIMIT_STACK, &rlim) != 0 || + rlim.rlim_cur != kSufficientStack) + return; + } + + // We should now have a stack of size at least kSufficientStack. Ensure + // that we can actually use that much, if necessary. + ensureStackAddressSpace(); +} +#else +static void ensureSufficientStack() {} +#endif + +int cc1_main(ArrayRef Argv, const char *Argv0, void *MainAddr) { + ensureSufficientStack(); + + std::unique_ptr Clang(new CompilerInstance()); + IntrusiveRefCntPtr DiagID(new DiagnosticIDs()); + + // Register the support for object-file-wrapped Clang modules. + auto PCHOps = Clang->getPCHContainerOperations(); + PCHOps->registerWriter(llvm::make_unique()); + PCHOps->registerReader(llvm::make_unique()); + + // Initialize targets first, so that --version shows registered targets. + llvm::InitializeAllTargets(); + llvm::InitializeAllTargetMCs(); + llvm::InitializeAllAsmPrinters(); + llvm::InitializeAllAsmParsers(); + +#ifdef LINK_POLLY_INTO_TOOLS + llvm::PassRegistry &Registry = *llvm::PassRegistry::getPassRegistry(); + polly::initializePollyPasses(Registry); +#endif + + // Buffer diagnostics from argument parsing so that we can output them using a + // well formed diagnostic object. + IntrusiveRefCntPtr DiagOpts = new DiagnosticOptions(); + TextDiagnosticBuffer *DiagsBuffer = new TextDiagnosticBuffer; + DiagnosticsEngine Diags(DiagID, &*DiagOpts, DiagsBuffer); + bool Success = CompilerInvocation::CreateFromArgs( + Clang->getInvocation(), Argv.begin(), Argv.end(), Diags); + + // Infer the builtin include path if unspecified. + if (Clang->getHeaderSearchOpts().UseBuiltinIncludes && + Clang->getHeaderSearchOpts().ResourceDir.empty()) + Clang->getHeaderSearchOpts().ResourceDir = + CompilerInvocation::GetResourcesPath(Argv0, MainAddr); + + // Create the actual diagnostics engine. + Clang->createDiagnostics(); + if (!Clang->hasDiagnostics()) + return 1; + + // Set an error handler, so that any LLVM backend diagnostics go through our + // error handler. + llvm::install_fatal_error_handler(LLVMErrorHandler, + static_cast(&Clang->getDiagnostics())); + + DiagsBuffer->FlushDiagnostics(Clang->getDiagnostics()); + if (!Success) + return 1; + + // Execute the frontend actions. + Success = ExecuteCompilerInvocation(Clang.get()); + + // If any timers were active but haven't been destroyed yet, print their + // results now. This happens in -disable-free mode. + llvm::TimerGroup::printAll(llvm::errs()); + + // Our error handler depends on the Diagnostics object, which we're + // potentially about to delete. Uninstall the handler now so that any + // later errors use the default handling behavior instead. + llvm::remove_fatal_error_handler(); + + // When running with -disable-free, don't do any destruction or shutdown. + if (Clang->getFrontendOpts().DisableFree) { + BuryPointer(std::move(Clang)); + return !Success; + } + + return !Success; +} diff --git a/dbms/programs/clang/Compiler-7.0.0bundled/cc1as_main.cpp b/dbms/programs/clang/Compiler-7.0.0bundled/cc1as_main.cpp new file mode 100644 index 00000000000..ce23422077f --- /dev/null +++ b/dbms/programs/clang/Compiler-7.0.0bundled/cc1as_main.cpp @@ -0,0 +1,555 @@ +//===-- cc1as_main.cpp - Clang Assembler ---------------------------------===// +// +// The LLVM Compiler Infrastructure +// +// This file is distributed under the University of Illinois Open Source +// License. See LICENSE.TXT for details. +// +//===----------------------------------------------------------------------===// +// +// This is the entry point to the clang -cc1as functionality, which implements +// the direct interface to the LLVM MC based assembler. +// +//===----------------------------------------------------------------------===// + +#include "clang/Basic/Diagnostic.h" +#include "clang/Basic/DiagnosticOptions.h" +#include "clang/Driver/DriverDiagnostic.h" +#include "clang/Driver/Options.h" +#include "clang/Frontend/FrontendDiagnostic.h" +#include "clang/Frontend/TextDiagnosticPrinter.h" +#include "clang/Frontend/Utils.h" +#include "llvm/ADT/STLExtras.h" +#include "llvm/ADT/StringSwitch.h" +#include "llvm/ADT/Triple.h" +#include "llvm/IR/DataLayout.h" +#include "llvm/MC/MCAsmBackend.h" +#include "llvm/MC/MCAsmInfo.h" +#include "llvm/MC/MCCodeEmitter.h" +#include "llvm/MC/MCContext.h" +#include "llvm/MC/MCInstrInfo.h" +#include "llvm/MC/MCObjectFileInfo.h" +#include "llvm/MC/MCParser/MCAsmParser.h" +#include "llvm/MC/MCParser/MCTargetAsmParser.h" +#include "llvm/MC/MCRegisterInfo.h" +#include "llvm/MC/MCStreamer.h" +#include "llvm/MC/MCSubtargetInfo.h" +#include "llvm/MC/MCTargetOptions.h" +#include "llvm/Option/Arg.h" +#include "llvm/Option/ArgList.h" +#include "llvm/Option/OptTable.h" +#include "llvm/Support/CommandLine.h" +#include "llvm/Support/ErrorHandling.h" +#include "llvm/Support/FileSystem.h" +#include "llvm/Support/FormattedStream.h" +#include "llvm/Support/Host.h" +#include "llvm/Support/MemoryBuffer.h" +#include "llvm/Support/Path.h" +#include "llvm/Support/Signals.h" +#include "llvm/Support/SourceMgr.h" +#include "llvm/Support/TargetRegistry.h" +#include "llvm/Support/TargetSelect.h" +#include "llvm/Support/Timer.h" +#include "llvm/Support/raw_ostream.h" +#include +#include +using namespace clang; +using namespace clang::driver; +using namespace clang::driver::options; +using namespace llvm; +using namespace llvm::opt; + + +namespace { + +/// \brief Helper class for representing a single invocation of the assembler. +struct AssemblerInvocation { + /// @name Target Options + /// @{ + + /// The name of the target triple to assemble for. + std::string Triple; + + /// If given, the name of the target CPU to determine which instructions + /// are legal. + std::string CPU; + + /// The list of target specific features to enable or disable -- this should + /// be a list of strings starting with '+' or '-'. + std::vector Features; + + /// The list of symbol definitions. + std::vector SymbolDefs; + + /// @} + /// @name Language Options + /// @{ + + std::vector IncludePaths; + unsigned NoInitialTextSection : 1; + unsigned SaveTemporaryLabels : 1; + unsigned GenDwarfForAssembly : 1; + unsigned RelaxELFRelocations : 1; + unsigned DwarfVersion; + std::string DwarfDebugFlags; + std::string DwarfDebugProducer; + std::string DebugCompilationDir; + llvm::DebugCompressionType CompressDebugSections = + llvm::DebugCompressionType::None; + std::string MainFileName; + + /// @} + /// @name Frontend Options + /// @{ + + std::string InputFile; + std::vector LLVMArgs; + std::string OutputPath; + enum FileType { + FT_Asm, ///< Assembly (.s) output, transliterate mode. + FT_Null, ///< No output, for timing purposes. + FT_Obj ///< Object file output. + }; + FileType OutputType; + unsigned ShowHelp : 1; + unsigned ShowVersion : 1; + + /// @} + /// @name Transliterate Options + /// @{ + + unsigned OutputAsmVariant; + unsigned ShowEncoding : 1; + unsigned ShowInst : 1; + + /// @} + /// @name Assembler Options + /// @{ + + unsigned RelaxAll : 1; + unsigned NoExecStack : 1; + unsigned FatalWarnings : 1; + unsigned IncrementalLinkerCompatible : 1; + + /// The name of the relocation model to use. + std::string RelocationModel; + + /// @} + +public: + AssemblerInvocation() { + Triple = ""; + NoInitialTextSection = 0; + InputFile = "-"; + OutputPath = "-"; + OutputType = FT_Asm; + OutputAsmVariant = 0; + ShowInst = 0; + ShowEncoding = 0; + RelaxAll = 0; + NoExecStack = 0; + FatalWarnings = 0; + IncrementalLinkerCompatible = 0; + DwarfVersion = 0; + } + + static bool CreateFromArgs(AssemblerInvocation &Res, + ArrayRef Argv, + DiagnosticsEngine &Diags); +}; + +} + +bool AssemblerInvocation::CreateFromArgs(AssemblerInvocation &Opts, + ArrayRef Argv, + DiagnosticsEngine &Diags) { + bool Success = true; + + // Parse the arguments. + std::unique_ptr OptTbl(createDriverOptTable()); + + const unsigned IncludedFlagsBitmask = options::CC1AsOption; + unsigned MissingArgIndex, MissingArgCount; + InputArgList Args = OptTbl->ParseArgs(Argv, MissingArgIndex, MissingArgCount, + IncludedFlagsBitmask); + + // Check for missing argument error. + if (MissingArgCount) { + Diags.Report(diag::err_drv_missing_argument) + << Args.getArgString(MissingArgIndex) << MissingArgCount; + Success = false; + } + + // Issue errors on unknown arguments. + for (const Arg *A : Args.filtered(OPT_UNKNOWN)) { + auto ArgString = A->getAsString(Args); + std::string Nearest; + if (OptTbl->findNearest(ArgString, Nearest, IncludedFlagsBitmask) > 1) + Diags.Report(diag::err_drv_unknown_argument) << ArgString; + else + Diags.Report(diag::err_drv_unknown_argument_with_suggestion) + << ArgString << Nearest; + Success = false; + } + + // Construct the invocation. + + // Target Options + Opts.Triple = llvm::Triple::normalize(Args.getLastArgValue(OPT_triple)); + Opts.CPU = Args.getLastArgValue(OPT_target_cpu); + Opts.Features = Args.getAllArgValues(OPT_target_feature); + + // Use the default target triple if unspecified. + if (Opts.Triple.empty()) + Opts.Triple = llvm::sys::getDefaultTargetTriple(); + + // Language Options + Opts.IncludePaths = Args.getAllArgValues(OPT_I); + Opts.NoInitialTextSection = Args.hasArg(OPT_n); + Opts.SaveTemporaryLabels = Args.hasArg(OPT_msave_temp_labels); + // Any DebugInfoKind implies GenDwarfForAssembly. + Opts.GenDwarfForAssembly = Args.hasArg(OPT_debug_info_kind_EQ); + + if (const Arg *A = Args.getLastArg(OPT_compress_debug_sections, + OPT_compress_debug_sections_EQ)) { + if (A->getOption().getID() == OPT_compress_debug_sections) { + // TODO: be more clever about the compression type auto-detection + Opts.CompressDebugSections = llvm::DebugCompressionType::GNU; + } else { + Opts.CompressDebugSections = + llvm::StringSwitch(A->getValue()) + .Case("none", llvm::DebugCompressionType::None) + .Case("zlib", llvm::DebugCompressionType::Z) + .Case("zlib-gnu", llvm::DebugCompressionType::GNU) + .Default(llvm::DebugCompressionType::None); + } + } + + Opts.RelaxELFRelocations = Args.hasArg(OPT_mrelax_relocations); + Opts.DwarfVersion = getLastArgIntValue(Args, OPT_dwarf_version_EQ, 2, Diags); + Opts.DwarfDebugFlags = Args.getLastArgValue(OPT_dwarf_debug_flags); + Opts.DwarfDebugProducer = Args.getLastArgValue(OPT_dwarf_debug_producer); + Opts.DebugCompilationDir = Args.getLastArgValue(OPT_fdebug_compilation_dir); + Opts.MainFileName = Args.getLastArgValue(OPT_main_file_name); + + // Frontend Options + if (Args.hasArg(OPT_INPUT)) { + bool First = true; + for (const Arg *A : Args.filtered(OPT_INPUT)) { + if (First) { + Opts.InputFile = A->getValue(); + First = false; + } else { + Diags.Report(diag::err_drv_unknown_argument) << A->getAsString(Args); + Success = false; + } + } + } + Opts.LLVMArgs = Args.getAllArgValues(OPT_mllvm); + Opts.OutputPath = Args.getLastArgValue(OPT_o); + if (Arg *A = Args.getLastArg(OPT_filetype)) { + StringRef Name = A->getValue(); + unsigned OutputType = StringSwitch(Name) + .Case("asm", FT_Asm) + .Case("null", FT_Null) + .Case("obj", FT_Obj) + .Default(~0U); + if (OutputType == ~0U) { + Diags.Report(diag::err_drv_invalid_value) << A->getAsString(Args) << Name; + Success = false; + } else + Opts.OutputType = FileType(OutputType); + } + Opts.ShowHelp = Args.hasArg(OPT_help); + Opts.ShowVersion = Args.hasArg(OPT_version); + + // Transliterate Options + Opts.OutputAsmVariant = + getLastArgIntValue(Args, OPT_output_asm_variant, 0, Diags); + Opts.ShowEncoding = Args.hasArg(OPT_show_encoding); + Opts.ShowInst = Args.hasArg(OPT_show_inst); + + // Assemble Options + Opts.RelaxAll = Args.hasArg(OPT_mrelax_all); + Opts.NoExecStack = Args.hasArg(OPT_mno_exec_stack); + Opts.FatalWarnings = Args.hasArg(OPT_massembler_fatal_warnings); + Opts.RelocationModel = Args.getLastArgValue(OPT_mrelocation_model, "pic"); + Opts.IncrementalLinkerCompatible = + Args.hasArg(OPT_mincremental_linker_compatible); + Opts.SymbolDefs = Args.getAllArgValues(OPT_defsym); + + return Success; +} + +static std::unique_ptr +getOutputStream(AssemblerInvocation &Opts, DiagnosticsEngine &Diags, + bool Binary) { + if (Opts.OutputPath.empty()) + Opts.OutputPath = "-"; + + // Make sure that the Out file gets unlinked from the disk if we get a + // SIGINT. + if (Opts.OutputPath != "-") + sys::RemoveFileOnSignal(Opts.OutputPath); + + std::error_code EC; + auto Out = llvm::make_unique( + Opts.OutputPath, EC, (Binary ? sys::fs::F_None : sys::fs::F_Text)); + if (EC) { + Diags.Report(diag::err_fe_unable_to_open_output) << Opts.OutputPath + << EC.message(); + return nullptr; + } + + return Out; +} + +static bool ExecuteAssembler(AssemblerInvocation &Opts, + DiagnosticsEngine &Diags) { + // Get the target specific parser. + std::string Error; + const Target *TheTarget = TargetRegistry::lookupTarget(Opts.Triple, Error); + if (!TheTarget) + return Diags.Report(diag::err_target_unknown_triple) << Opts.Triple; + + ErrorOr> Buffer = + MemoryBuffer::getFileOrSTDIN(Opts.InputFile); + + if (std::error_code EC = Buffer.getError()) { + Error = EC.message(); + return Diags.Report(diag::err_fe_error_reading) << Opts.InputFile; + } + + SourceMgr SrcMgr; + + // Tell SrcMgr about this buffer, which is what the parser will pick up. + SrcMgr.AddNewSourceBuffer(std::move(*Buffer), SMLoc()); + + // Record the location of the include directories so that the lexer can find + // it later. + SrcMgr.setIncludeDirs(Opts.IncludePaths); + + std::unique_ptr MRI(TheTarget->createMCRegInfo(Opts.Triple)); + assert(MRI && "Unable to create target register info!"); + + std::unique_ptr MAI(TheTarget->createMCAsmInfo(*MRI, Opts.Triple)); + assert(MAI && "Unable to create target asm info!"); + + // Ensure MCAsmInfo initialization occurs before any use, otherwise sections + // may be created with a combination of default and explicit settings. + MAI->setCompressDebugSections(Opts.CompressDebugSections); + + MAI->setRelaxELFRelocations(Opts.RelaxELFRelocations); + + bool IsBinary = Opts.OutputType == AssemblerInvocation::FT_Obj; + std::unique_ptr FDOS = getOutputStream(Opts, Diags, IsBinary); + if (!FDOS) + return true; + + // FIXME: This is not pretty. MCContext has a ptr to MCObjectFileInfo and + // MCObjectFileInfo needs a MCContext reference in order to initialize itself. + std::unique_ptr MOFI(new MCObjectFileInfo()); + + MCContext Ctx(MAI.get(), MRI.get(), MOFI.get(), &SrcMgr); + + bool PIC = false; + if (Opts.RelocationModel == "static") { + PIC = false; + } else if (Opts.RelocationModel == "pic") { + PIC = true; + } else { + assert(Opts.RelocationModel == "dynamic-no-pic" && + "Invalid PIC model!"); + PIC = false; + } + + MOFI->InitMCObjectFileInfo(Triple(Opts.Triple), PIC, Ctx); + if (Opts.SaveTemporaryLabels) + Ctx.setAllowTemporaryLabels(false); + if (Opts.GenDwarfForAssembly) + Ctx.setGenDwarfForAssembly(true); + if (!Opts.DwarfDebugFlags.empty()) + Ctx.setDwarfDebugFlags(StringRef(Opts.DwarfDebugFlags)); + if (!Opts.DwarfDebugProducer.empty()) + Ctx.setDwarfDebugProducer(StringRef(Opts.DwarfDebugProducer)); + if (!Opts.DebugCompilationDir.empty()) + Ctx.setCompilationDir(Opts.DebugCompilationDir); + if (!Opts.MainFileName.empty()) + Ctx.setMainFileName(StringRef(Opts.MainFileName)); + Ctx.setDwarfVersion(Opts.DwarfVersion); + + // Build up the feature string from the target feature list. + std::string FS; + if (!Opts.Features.empty()) { + FS = Opts.Features[0]; + for (unsigned i = 1, e = Opts.Features.size(); i != e; ++i) + FS += "," + Opts.Features[i]; + } + + std::unique_ptr Str; + + std::unique_ptr MCII(TheTarget->createMCInstrInfo()); + std::unique_ptr STI( + TheTarget->createMCSubtargetInfo(Opts.Triple, Opts.CPU, FS)); + + raw_pwrite_stream *Out = FDOS.get(); + std::unique_ptr BOS; + + // FIXME: There is a bit of code duplication with addPassesToEmitFile. + if (Opts.OutputType == AssemblerInvocation::FT_Asm) { + MCInstPrinter *IP = TheTarget->createMCInstPrinter( + llvm::Triple(Opts.Triple), Opts.OutputAsmVariant, *MAI, *MCII, *MRI); + + std::unique_ptr CE; + if (Opts.ShowEncoding) + CE.reset(TheTarget->createMCCodeEmitter(*MCII, *MRI, Ctx)); + MCTargetOptions MCOptions; + std::unique_ptr MAB( + TheTarget->createMCAsmBackend(*STI, *MRI, MCOptions)); + + auto FOut = llvm::make_unique(*Out); + Str.reset(TheTarget->createAsmStreamer( + Ctx, std::move(FOut), /*asmverbose*/ true, + /*useDwarfDirectory*/ true, IP, std::move(CE), std::move(MAB), + Opts.ShowInst)); + } else if (Opts.OutputType == AssemblerInvocation::FT_Null) { + Str.reset(createNullStreamer(Ctx)); + } else { + assert(Opts.OutputType == AssemblerInvocation::FT_Obj && + "Invalid file type!"); + if (!FDOS->supportsSeeking()) { + BOS = make_unique(*FDOS); + Out = BOS.get(); + } + + std::unique_ptr CE( + TheTarget->createMCCodeEmitter(*MCII, *MRI, Ctx)); + MCTargetOptions MCOptions; + std::unique_ptr MAB( + TheTarget->createMCAsmBackend(*STI, *MRI, MCOptions)); + + Triple T(Opts.Triple); + Str.reset(TheTarget->createMCObjectStreamer( + T, Ctx, std::move(MAB), *Out, std::move(CE), *STI, Opts.RelaxAll, + Opts.IncrementalLinkerCompatible, + /*DWARFMustBeAtTheEnd*/ true)); + Str.get()->InitSections(Opts.NoExecStack); + } + + // Assembly to object compilation should leverage assembly info. + Str->setUseAssemblerInfoForParsing(true); + + bool Failed = false; + + std::unique_ptr Parser( + createMCAsmParser(SrcMgr, Ctx, *Str.get(), *MAI)); + + // FIXME: init MCTargetOptions from sanitizer flags here. + MCTargetOptions Options; + std::unique_ptr TAP( + TheTarget->createMCAsmParser(*STI, *Parser, *MCII, Options)); + if (!TAP) + Failed = Diags.Report(diag::err_target_unknown_triple) << Opts.Triple; + + // Set values for symbols, if any. + for (auto &S : Opts.SymbolDefs) { + auto Pair = StringRef(S).split('='); + auto Sym = Pair.first; + auto Val = Pair.second; + int64_t Value; + // We have already error checked this in the driver. + Val.getAsInteger(0, Value); + Ctx.setSymbolValue(Parser->getStreamer(), Sym, Value); + } + + if (!Failed) { + Parser->setTargetParser(*TAP.get()); + Failed = Parser->Run(Opts.NoInitialTextSection); + } + + // Close Streamer first. + // It might have a reference to the output stream. + Str.reset(); + // Close the output stream early. + BOS.reset(); + FDOS.reset(); + + // Delete output file if there were errors. + if (Failed && Opts.OutputPath != "-") + sys::fs::remove(Opts.OutputPath); + + return Failed; +} + +static void LLVMErrorHandler(void *UserData, const std::string &Message, + bool /*GenCrashDiag*/) { + DiagnosticsEngine &Diags = *static_cast(UserData); + + Diags.Report(diag::err_fe_error_backend) << Message; + + // We cannot recover from llvm errors. + exit(1); +} + +int cc1as_main(ArrayRef Argv, const char */*Argv0*/, void */*MainAddr*/) { + // Initialize targets and assembly printers/parsers. + InitializeAllTargetInfos(); + InitializeAllTargetMCs(); + InitializeAllAsmParsers(); + + // Construct our diagnostic client. + IntrusiveRefCntPtr DiagOpts = new DiagnosticOptions(); + TextDiagnosticPrinter *DiagClient + = new TextDiagnosticPrinter(errs(), &*DiagOpts); + DiagClient->setPrefix("clang -cc1as"); + IntrusiveRefCntPtr DiagID(new DiagnosticIDs()); + DiagnosticsEngine Diags(DiagID, &*DiagOpts, DiagClient); + + // Set an error handler, so that any LLVM backend diagnostics go through our + // error handler. + ScopedFatalErrorHandler FatalErrorHandler + (LLVMErrorHandler, static_cast(&Diags)); + + // Parse the arguments. + AssemblerInvocation Asm; + if (!AssemblerInvocation::CreateFromArgs(Asm, Argv, Diags)) + return 1; + + if (Asm.ShowHelp) { + std::unique_ptr Opts(driver::createDriverOptTable()); + Opts->PrintHelp(llvm::outs(), "clang -cc1as", "Clang Integrated Assembler", + /*Include=*/driver::options::CC1AsOption, /*Exclude=*/0, + /*ShowAllAliases=*/false); + return 0; + } + + // Honor -version. + // + // FIXME: Use a better -version message? + if (Asm.ShowVersion) { + llvm::cl::PrintVersionMessage(); + return 0; + } + + // Honor -mllvm. + // + // FIXME: Remove this, one day. + if (!Asm.LLVMArgs.empty()) { + unsigned NumArgs = Asm.LLVMArgs.size(); + auto Args = llvm::make_unique(NumArgs + 2); + Args[0] = "clang (LLVM option parsing)"; + for (unsigned i = 0; i != NumArgs; ++i) + Args[i + 1] = Asm.LLVMArgs[i].c_str(); + Args[NumArgs + 1] = nullptr; + llvm::cl::ParseCommandLineOptions(NumArgs + 1, Args.get()); + } + + // Execute the invocation, unless there were parsing errors. + bool Failed = Diags.hasErrorOccurred() || ExecuteAssembler(Asm, Diags); + + // If any timers were active but haven't been destroyed yet, print their + // results now. + TimerGroup::printAll(errs()); + + return !!Failed; +} diff --git a/dbms/programs/clang/Compiler-7.0.0bundled/driver.cpp b/dbms/programs/clang/Compiler-7.0.0bundled/driver.cpp new file mode 100644 index 00000000000..9a061b9d137 --- /dev/null +++ b/dbms/programs/clang/Compiler-7.0.0bundled/driver.cpp @@ -0,0 +1,512 @@ +//===-- driver.cpp - Clang GCC-Compatible Driver --------------------------===// +// +// The LLVM Compiler Infrastructure +// +// This file is distributed under the University of Illinois Open Source +// License. See LICENSE.TXT for details. +// +//===----------------------------------------------------------------------===// +// +// This is the entry point to the clang driver; it is a thin wrapper +// for functionality in the Driver clang library. +// +//===----------------------------------------------------------------------===// + +#include "clang/Driver/Driver.h" +#include "clang/Basic/DiagnosticOptions.h" +#include "clang/Driver/Compilation.h" +#include "clang/Driver/DriverDiagnostic.h" +#include "clang/Driver/Options.h" +#include "clang/Driver/ToolChain.h" +#include "clang/Frontend/ChainedDiagnosticConsumer.h" +#include "clang/Frontend/CompilerInvocation.h" +#include "clang/Frontend/SerializedDiagnosticPrinter.h" +#include "clang/Frontend/TextDiagnosticPrinter.h" +#include "clang/Frontend/Utils.h" +#include "llvm/ADT/ArrayRef.h" +#include "llvm/ADT/SmallString.h" +#include "llvm/ADT/SmallVector.h" +#include "llvm/Option/ArgList.h" +#include "llvm/Option/OptTable.h" +#include "llvm/Option/Option.h" +#include "llvm/Support/CommandLine.h" +#include "llvm/Support/ErrorHandling.h" +#include "llvm/Support/FileSystem.h" +#include "llvm/Support/Host.h" +#include "llvm/Support/InitLLVM.h" +#include "llvm/Support/Path.h" +#include "llvm/Support/Process.h" +#include "llvm/Support/Program.h" +#include "llvm/Support/Regex.h" +#include "llvm/Support/Signals.h" +#include "llvm/Support/StringSaver.h" +#include "llvm/Support/TargetSelect.h" +#include "llvm/Support/Timer.h" +#include "llvm/Support/raw_ostream.h" +#include +#include +#include +using namespace clang; +using namespace clang::driver; +using namespace llvm::opt; + +std::string GetExecutablePath(const char *Argv0, bool CanonicalPrefixes) { + if (!CanonicalPrefixes) { + SmallString<128> ExecutablePath(Argv0); + // Do a PATH lookup if Argv0 isn't a valid path. + if (!llvm::sys::fs::exists(ExecutablePath)) + if (llvm::ErrorOr P = + llvm::sys::findProgramByName(ExecutablePath)) + ExecutablePath = *P; + return ExecutablePath.str(); + } + + // This just needs to be some symbol in the binary; C++ doesn't + // allow taking the address of ::main however. + void *P = (void*) (intptr_t) GetExecutablePath; + return llvm::sys::fs::getMainExecutable(Argv0, P); +} + +static const char *GetStableCStr(std::set &SavedStrings, + StringRef S) { + return SavedStrings.insert(S).first->c_str(); +} + +/// ApplyQAOverride - Apply a list of edits to the input argument lists. +/// +/// The input string is a space separate list of edits to perform, +/// they are applied in order to the input argument lists. Edits +/// should be one of the following forms: +/// +/// '#': Silence information about the changes to the command line arguments. +/// +/// '^': Add FOO as a new argument at the beginning of the command line. +/// +/// '+': Add FOO as a new argument at the end of the command line. +/// +/// 's/XXX/YYY/': Substitute the regular expression XXX with YYY in the command +/// line. +/// +/// 'xOPTION': Removes all instances of the literal argument OPTION. +/// +/// 'XOPTION': Removes all instances of the literal argument OPTION, +/// and the following argument. +/// +/// 'Ox': Removes all flags matching 'O' or 'O[sz0-9]' and adds 'Ox' +/// at the end of the command line. +/// +/// \param OS - The stream to write edit information to. +/// \param Args - The vector of command line arguments. +/// \param Edit - The override command to perform. +/// \param SavedStrings - Set to use for storing string representations. +static void ApplyOneQAOverride(raw_ostream &OS, + SmallVectorImpl &Args, + StringRef Edit, + std::set &SavedStrings) { + // This does not need to be efficient. + + if (Edit[0] == '^') { + const char *Str = + GetStableCStr(SavedStrings, Edit.substr(1)); + OS << "### Adding argument " << Str << " at beginning\n"; + Args.insert(Args.begin() + 1, Str); + } else if (Edit[0] == '+') { + const char *Str = + GetStableCStr(SavedStrings, Edit.substr(1)); + OS << "### Adding argument " << Str << " at end\n"; + Args.push_back(Str); + } else if (Edit[0] == 's' && Edit[1] == '/' && Edit.endswith("/") && + Edit.slice(2, Edit.size()-1).find('/') != StringRef::npos) { + StringRef MatchPattern = Edit.substr(2).split('/').first; + StringRef ReplPattern = Edit.substr(2).split('/').second; + ReplPattern = ReplPattern.slice(0, ReplPattern.size()-1); + + for (unsigned i = 1, e = Args.size(); i != e; ++i) { + // Ignore end-of-line response file markers + if (Args[i] == nullptr) + continue; + std::string Repl = llvm::Regex(MatchPattern).sub(ReplPattern, Args[i]); + + if (Repl != Args[i]) { + OS << "### Replacing '" << Args[i] << "' with '" << Repl << "'\n"; + Args[i] = GetStableCStr(SavedStrings, Repl); + } + } + } else if (Edit[0] == 'x' || Edit[0] == 'X') { + auto Option = Edit.substr(1); + for (unsigned i = 1; i < Args.size();) { + if (Option == Args[i]) { + OS << "### Deleting argument " << Args[i] << '\n'; + Args.erase(Args.begin() + i); + if (Edit[0] == 'X') { + if (i < Args.size()) { + OS << "### Deleting argument " << Args[i] << '\n'; + Args.erase(Args.begin() + i); + } else + OS << "### Invalid X edit, end of command line!\n"; + } + } else + ++i; + } + } else if (Edit[0] == 'O') { + for (unsigned i = 1; i < Args.size();) { + const char *A = Args[i]; + // Ignore end-of-line response file markers + if (A == nullptr) + continue; + if (A[0] == '-' && A[1] == 'O' && + (A[2] == '\0' || + (A[3] == '\0' && (A[2] == 's' || A[2] == 'z' || + ('0' <= A[2] && A[2] <= '9'))))) { + OS << "### Deleting argument " << Args[i] << '\n'; + Args.erase(Args.begin() + i); + } else + ++i; + } + OS << "### Adding argument " << Edit << " at end\n"; + Args.push_back(GetStableCStr(SavedStrings, '-' + Edit.str())); + } else { + OS << "### Unrecognized edit: " << Edit << "\n"; + } +} + +/// ApplyQAOverride - Apply a comma separate list of edits to the +/// input argument lists. See ApplyOneQAOverride. +static void ApplyQAOverride(SmallVectorImpl &Args, + const char *OverrideStr, + std::set &SavedStrings) { + raw_ostream *OS = &llvm::errs(); + + if (OverrideStr[0] == '#') { + ++OverrideStr; + OS = &llvm::nulls(); + } + + *OS << "### CCC_OVERRIDE_OPTIONS: " << OverrideStr << "\n"; + + // This does not need to be efficient. + + const char *S = OverrideStr; + while (*S) { + const char *End = ::strchr(S, ' '); + if (!End) + End = S + strlen(S); + if (End != S) + ApplyOneQAOverride(*OS, Args, std::string(S, End), SavedStrings); + S = End; + if (*S != '\0') + ++S; + } +} + +extern int cc1_main(ArrayRef Argv, const char *Argv0, + void *MainAddr); +extern int cc1as_main(ArrayRef Argv, const char *Argv0, + void *MainAddr); +extern int cc1gen_reproducer_main(ArrayRef Argv, + const char *Argv0, void *MainAddr); + +static void insertTargetAndModeArgs(const ParsedClangName &NameParts, + SmallVectorImpl &ArgVector, + std::set &SavedStrings) { + // Put target and mode arguments at the start of argument list so that + // arguments specified in command line could override them. Avoid putting + // them at index 0, as an option like '-cc1' must remain the first. + int InsertionPoint = 0; + if (ArgVector.size() > 0) + ++InsertionPoint; + + if (NameParts.DriverMode) { + // Add the mode flag to the arguments. + ArgVector.insert(ArgVector.begin() + InsertionPoint, + GetStableCStr(SavedStrings, NameParts.DriverMode)); + } + + if (NameParts.TargetIsValid) { + const char *arr[] = {"-target", GetStableCStr(SavedStrings, + NameParts.TargetPrefix)}; + ArgVector.insert(ArgVector.begin() + InsertionPoint, + std::begin(arr), std::end(arr)); + } +} + +static void getCLEnvVarOptions(std::string &EnvValue, llvm::StringSaver &Saver, + SmallVectorImpl &Opts) { + llvm::cl::TokenizeWindowsCommandLine(EnvValue, Saver, Opts); + // The first instance of '#' should be replaced with '=' in each option. + for (const char *Opt : Opts) + if (char *NumberSignPtr = const_cast(::strchr(Opt, '#'))) + *NumberSignPtr = '='; +} + +static void SetBackdoorDriverOutputsFromEnvVars(Driver &TheDriver) { + // Handle CC_PRINT_OPTIONS and CC_PRINT_OPTIONS_FILE. + TheDriver.CCPrintOptions = !!::getenv("CC_PRINT_OPTIONS"); + if (TheDriver.CCPrintOptions) + TheDriver.CCPrintOptionsFilename = ::getenv("CC_PRINT_OPTIONS_FILE"); + + // Handle CC_PRINT_HEADERS and CC_PRINT_HEADERS_FILE. + TheDriver.CCPrintHeaders = !!::getenv("CC_PRINT_HEADERS"); + if (TheDriver.CCPrintHeaders) + TheDriver.CCPrintHeadersFilename = ::getenv("CC_PRINT_HEADERS_FILE"); + + // Handle CC_LOG_DIAGNOSTICS and CC_LOG_DIAGNOSTICS_FILE. + TheDriver.CCLogDiagnostics = !!::getenv("CC_LOG_DIAGNOSTICS"); + if (TheDriver.CCLogDiagnostics) + TheDriver.CCLogDiagnosticsFilename = ::getenv("CC_LOG_DIAGNOSTICS_FILE"); +} + +static void FixupDiagPrefixExeName(TextDiagnosticPrinter *DiagClient, + const std::string &Path) { + // If the clang binary happens to be named cl.exe for compatibility reasons, + // use clang-cl.exe as the prefix to avoid confusion between clang and MSVC. + StringRef ExeBasename(llvm::sys::path::filename(Path)); + if (ExeBasename.equals_lower("cl.exe")) + ExeBasename = "clang-cl.exe"; + DiagClient->setPrefix(ExeBasename); +} + +// This lets us create the DiagnosticsEngine with a properly-filled-out +// DiagnosticOptions instance. +static DiagnosticOptions * +CreateAndPopulateDiagOpts(ArrayRef argv) { + auto *DiagOpts = new DiagnosticOptions; + std::unique_ptr Opts(createDriverOptTable()); + unsigned MissingArgIndex, MissingArgCount; + InputArgList Args = + Opts->ParseArgs(argv.slice(1), MissingArgIndex, MissingArgCount); + // We ignore MissingArgCount and the return value of ParseDiagnosticArgs. + // Any errors that would be diagnosed here will also be diagnosed later, + // when the DiagnosticsEngine actually exists. + (void)ParseDiagnosticArgs(*DiagOpts, Args); + return DiagOpts; +} + +static void SetInstallDir(SmallVectorImpl &argv, + Driver &TheDriver, bool CanonicalPrefixes) { + // Attempt to find the original path used to invoke the driver, to determine + // the installed path. We do this manually, because we want to support that + // path being a symlink. + SmallString<128> InstalledPath(argv[0]); + + // Do a PATH lookup, if there are no directory components. + if (llvm::sys::path::filename(InstalledPath) == InstalledPath) + if (llvm::ErrorOr Tmp = llvm::sys::findProgramByName( + llvm::sys::path::filename(InstalledPath.str()))) + InstalledPath = *Tmp; + + // FIXME: We don't actually canonicalize this, we just make it absolute. + if (CanonicalPrefixes) + llvm::sys::fs::make_absolute(InstalledPath); + + StringRef InstalledPathParent(llvm::sys::path::parent_path(InstalledPath)); + if (llvm::sys::fs::exists(InstalledPathParent)) + TheDriver.setInstalledDir(InstalledPathParent); +} + +static int ExecuteCC1Tool(ArrayRef argv, StringRef Tool) { + void *GetExecutablePathVP = (void *)(intptr_t) GetExecutablePath; + if (Tool == "") + return cc1_main(argv.slice(2), argv[0], GetExecutablePathVP); + if (Tool == "as") + return cc1as_main(argv.slice(2), argv[0], GetExecutablePathVP); + + // Reject unknown tools. + llvm::errs() << "error: unknown integrated tool '" << Tool << "'. " + << "Valid tools include '-cc1' and '-cc1as'.\n"; + return 1; +} + +int mainEntryClickHouseClang(int argc_, char **argv_) { + llvm::InitLLVM X(argc_, argv_); + SmallVector argv(argv_, argv_ + argc_); + + if (llvm::sys::Process::FixupStandardFileDescriptors()) + return 1; + + llvm::InitializeAllTargets(); + auto TargetAndMode = ToolChain::getTargetAndModeFromProgramName(argv[0]); + + llvm::BumpPtrAllocator A; + llvm::StringSaver Saver(A); + + // Parse response files using the GNU syntax, unless we're in CL mode. There + // are two ways to put clang in CL compatibility mode: argv[0] is either + // clang-cl or cl, or --driver-mode=cl is on the command line. The normal + // command line parsing can't happen until after response file parsing, so we + // have to manually search for a --driver-mode=cl argument the hard way. + // Finally, our -cc1 tools don't care which tokenization mode we use because + // response files written by clang will tokenize the same way in either mode. + bool ClangCLMode = false; + if (StringRef(TargetAndMode.DriverMode).equals("--driver-mode=cl") || + std::find_if(argv.begin(), argv.end(), [](const char *F) { + return F && strcmp(F, "--driver-mode=cl") == 0; + }) != argv.end()) { + ClangCLMode = true; + } + enum { Default, POSIX, Windows } RSPQuoting = Default; + for (const char *F : argv) { + if (strcmp(F, "--rsp-quoting=posix") == 0) + RSPQuoting = POSIX; + else if (strcmp(F, "--rsp-quoting=windows") == 0) + RSPQuoting = Windows; + } + + // Determines whether we want nullptr markers in argv to indicate response + // files end-of-lines. We only use this for the /LINK driver argument with + // clang-cl.exe on Windows. + bool MarkEOLs = ClangCLMode; + + llvm::cl::TokenizerCallback Tokenizer; + if (RSPQuoting == Windows || (RSPQuoting == Default && ClangCLMode)) + Tokenizer = &llvm::cl::TokenizeWindowsCommandLine; + else + Tokenizer = &llvm::cl::TokenizeGNUCommandLine; + + if (MarkEOLs && argv.size() > 1 && StringRef(argv[1]).startswith("-cc1")) + MarkEOLs = false; + llvm::cl::ExpandResponseFiles(Saver, Tokenizer, argv, MarkEOLs); + + // Handle -cc1 integrated tools, even if -cc1 was expanded from a response + // file. + auto FirstArg = std::find_if(argv.begin() + 1, argv.end(), + [](const char *A) { return A != nullptr; }); + if (FirstArg != argv.end() && StringRef(*FirstArg).startswith("-cc1")) { + // If -cc1 came from a response file, remove the EOL sentinels. + if (MarkEOLs) { + auto newEnd = std::remove(argv.begin(), argv.end(), nullptr); + argv.resize(newEnd - argv.begin()); + } + return ExecuteCC1Tool(argv, argv[1] + 4); + } + + bool CanonicalPrefixes = true; + for (int i = 1, size = argv.size(); i < size; ++i) { + // Skip end-of-line response file markers + if (argv[i] == nullptr) + continue; + if (StringRef(argv[i]) == "-no-canonical-prefixes") { + CanonicalPrefixes = false; + break; + } + } + + // Handle CL and _CL_ which permits additional command line options to be + // prepended or appended. + if (ClangCLMode) { + // Arguments in "CL" are prepended. + llvm::Optional OptCL = llvm::sys::Process::GetEnv("CL"); + if (OptCL.hasValue()) { + SmallVector PrependedOpts; + getCLEnvVarOptions(OptCL.getValue(), Saver, PrependedOpts); + + // Insert right after the program name to prepend to the argument list. + argv.insert(argv.begin() + 1, PrependedOpts.begin(), PrependedOpts.end()); + } + // Arguments in "_CL_" are appended. + llvm::Optional Opt_CL_ = llvm::sys::Process::GetEnv("_CL_"); + if (Opt_CL_.hasValue()) { + SmallVector AppendedOpts; + getCLEnvVarOptions(Opt_CL_.getValue(), Saver, AppendedOpts); + + // Insert at the end of the argument list to append. + argv.append(AppendedOpts.begin(), AppendedOpts.end()); + } + } + + std::set SavedStrings; + // Handle CCC_OVERRIDE_OPTIONS, used for editing a command line behind the + // scenes. + if (const char *OverrideStr = ::getenv("CCC_OVERRIDE_OPTIONS")) { + // FIXME: Driver shouldn't take extra initial argument. + ApplyQAOverride(argv, OverrideStr, SavedStrings); + } + + std::string Path = GetExecutablePath(argv[0], CanonicalPrefixes); + + IntrusiveRefCntPtr DiagOpts = + CreateAndPopulateDiagOpts(argv); + + TextDiagnosticPrinter *DiagClient + = new TextDiagnosticPrinter(llvm::errs(), &*DiagOpts); + FixupDiagPrefixExeName(DiagClient, Path); + + IntrusiveRefCntPtr DiagID(new DiagnosticIDs()); + + DiagnosticsEngine Diags(DiagID, &*DiagOpts, DiagClient); + + if (!DiagOpts->DiagnosticSerializationFile.empty()) { + auto SerializedConsumer = + clang::serialized_diags::create(DiagOpts->DiagnosticSerializationFile, + &*DiagOpts, /*MergeChildRecords=*/true); + Diags.setClient(new ChainedDiagnosticConsumer( + Diags.takeClient(), std::move(SerializedConsumer))); + } + + ProcessWarningOptions(Diags, *DiagOpts, /*ReportDiags=*/false); + + Driver TheDriver(Path, llvm::sys::getDefaultTargetTriple(), Diags); + SetInstallDir(argv, TheDriver, CanonicalPrefixes); + TheDriver.setTargetAndMode(TargetAndMode); + + insertTargetAndModeArgs(TargetAndMode, argv, SavedStrings); + + SetBackdoorDriverOutputsFromEnvVars(TheDriver); + + std::unique_ptr C(TheDriver.BuildCompilation(argv)); + int Res = 1; + if (C && !C->containsError()) { + SmallVector, 4> FailingCommands; + Res = TheDriver.ExecuteCompilation(*C, FailingCommands); + + // Force a crash to test the diagnostics. + if (TheDriver.GenReproducer) { + Diags.Report(diag::err_drv_force_crash) + << !::getenv("FORCE_CLANG_DIAGNOSTICS_CRASH"); + + // Pretend that every command failed. + FailingCommands.clear(); + for (const auto &J : C->getJobs()) + if (const Command *C = dyn_cast(&J)) + FailingCommands.push_back(std::make_pair(-1, C)); + } + + for (const auto &P : FailingCommands) { + int CommandRes = P.first; + const Command *FailingCommand = P.second; + if (!Res) + Res = CommandRes; + + // If result status is < 0, then the driver command signalled an error. + // If result status is 70, then the driver command reported a fatal error. + // On Windows, abort will return an exit code of 3. In these cases, + // generate additional diagnostic information if possible. + bool DiagnoseCrash = CommandRes < 0 || CommandRes == 70; +#ifdef _WIN32 + DiagnoseCrash |= CommandRes == 3; +#endif + if (DiagnoseCrash) { + TheDriver.generateCompilationDiagnostics(*C, *FailingCommand); + break; + } + } + } + + Diags.getClient()->finish(); + + // If any timers were active but haven't been destroyed yet, print their + // results now. This happens in -disable-free mode. + llvm::TimerGroup::printAll(llvm::errs()); + +#ifdef _WIN32 + // Exit status should not be negative on Win32, unless abnormal termination. + // Once abnormal termiation was caught, negative status should not be + // propagated. + if (Res < 0) + Res = 1; +#endif + + // If we have multiple failing commands, we return the result of the first + // failing command. + return Res; +} diff --git a/dbms/programs/clang/Compiler-7.0.0bundled/lld.cpp b/dbms/programs/clang/Compiler-7.0.0bundled/lld.cpp new file mode 100644 index 00000000000..203e50d42a9 --- /dev/null +++ b/dbms/programs/clang/Compiler-7.0.0bundled/lld.cpp @@ -0,0 +1,10 @@ +#include "lld/Common/Driver.h" +#include "llvm/Support/InitLLVM.h" +#include + +int mainEntryClickHouseLLD(int argc, char ** argv) +{ + llvm::InitLLVM X(argc, argv); + std::vector args(argv, argv + argc); + return !lld::elf::link(args, false); +} diff --git a/dbms/programs/server/CMakeLists.txt b/dbms/programs/server/CMakeLists.txt index bab52f54092..bc6683d6e92 100644 --- a/dbms/programs/server/CMakeLists.txt +++ b/dbms/programs/server/CMakeLists.txt @@ -19,8 +19,8 @@ if (CLICKHOUSE_SPLIT_BINARY) install (TARGETS clickhouse-server ${CLICKHOUSE_ALL_TARGETS} RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) endif () -if (OS_LINUX) - set (GLIBC_MAX_REQUIRED 2.4) +if (OS_LINUX AND MAKE_STATIC_LIBRARIES) + set (GLIBC_MAX_REQUIRED 2.4 CACHE INTERNAL "") add_test(NAME GLIBC_required_version COMMAND bash -c "readelf -s ${CMAKE_CURRENT_BINARY_DIR}/../clickhouse-server | grep '@GLIBC' | grep -oP 'GLIBC_[\\d\\.]+' | sort | uniq | sort -r | perl -lnE 'exit 1 if $_ gt q{GLIBC_${GLIBC_MAX_REQUIRED}}'") endif () diff --git a/dbms/src/Columns/ColumnVector.cpp b/dbms/src/Columns/ColumnVector.cpp index 7d30759c844..d6fc07cd9d2 100644 --- a/dbms/src/Columns/ColumnVector.cpp +++ b/dbms/src/Columns/ColumnVector.cpp @@ -1,19 +1,16 @@ +#include "ColumnVector.h" + #include #include - #include #include #include #include #include - #include #include - -#include - +#include #include - #include #if __SSE2__ diff --git a/dbms/src/Interpreters/ExpressionJIT.cpp b/dbms/src/Interpreters/ExpressionJIT.cpp index a88faf13ca4..f12cfedc7c9 100644 --- a/dbms/src/Interpreters/ExpressionJIT.cpp +++ b/dbms/src/Interpreters/ExpressionJIT.cpp @@ -142,6 +142,8 @@ static llvm::TargetMachine * getNativeMachine() #if LLVM_VERSION_MAJOR >= 7 auto wrapJITSymbolResolver(llvm::JITSymbolResolver & jsr) { +#if USE_INTERNAL_LLVM_LIBRARY && LLVM_VERSION_PATCH == 0 + // REMOVE AFTER contrib/llvm upgrade auto flags = [&](llvm::orc::SymbolFlagsMap & flags, const llvm::orc::SymbolNameSet & symbols) { llvm::orc::SymbolNameSet missing; @@ -155,6 +157,21 @@ auto wrapJITSymbolResolver(llvm::JITSymbolResolver & jsr) } return missing; }; +#else + // Actually this should work for 7.0.0 but now we have OLDER 7.0.0svn in contrib + auto flags = [&](const llvm::orc::SymbolNameSet & symbols) + { + llvm::orc::SymbolFlagsMap flags; + for (const auto & symbol : symbols) + { + auto resolved = jsr.lookupFlags({*symbol}); + if (resolved && resolved->size()) + flags.emplace(symbol, resolved->begin()->second); + } + return flags; + }; +#endif + auto symbols = [&](std::shared_ptr query, llvm::orc::SymbolNameSet symbols) { llvm::orc::SymbolNameSet missing; diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 532118e5e06..21412b4f2c4 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -3,10 +3,10 @@ #include #include #include +#include #if __SSE2__ #include -#include #endif namespace DB From ddf231e57f68e4dae4b41896050e9579b94fdf6c Mon Sep 17 00:00:00 2001 From: proller Date: Sat, 17 Nov 2018 21:52:47 +0300 Subject: [PATCH 45/46] BI-154 Fix isLocal detection --- dbms/src/Interpreters/DDLWorker.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index c9e0d4f9622..77a4ec67b95 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -301,7 +301,11 @@ bool DDLWorker::initAndCheckTask(const String & entry_name, String & out_reason) for (const HostID & host : task->entry.hosts) { auto maybe_secure_port = context.getTCPPortSecure(); - bool is_local_port = maybe_secure_port ? host.isLocalAddress(*maybe_secure_port) : host.isLocalAddress(context.getTCPPort()); + bool is_local_port = false; + if (maybe_secure_port) + is_local_port = host.isLocalAddress(*maybe_secure_port); + if (!is_local_port) + is_local_port = host.isLocalAddress(context.getTCPPort()); if (!is_local_port) continue; From 786bf36aee0fa1d26823b847d1de88c5410bb8d0 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 19 Nov 2018 01:04:11 +0300 Subject: [PATCH 46/46] Update DDLWorker.cpp --- dbms/src/Interpreters/DDLWorker.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index 77a4ec67b95..f557ff0afa5 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -301,11 +301,11 @@ bool DDLWorker::initAndCheckTask(const String & entry_name, String & out_reason) for (const HostID & host : task->entry.hosts) { auto maybe_secure_port = context.getTCPPortSecure(); - bool is_local_port = false; - if (maybe_secure_port) - is_local_port = host.isLocalAddress(*maybe_secure_port); - if (!is_local_port) - is_local_port = host.isLocalAddress(context.getTCPPort()); + + /// The port is considered local if it matches TCP or TCP secure port that the server is listening. + bool is_local_port = (maybe_secure_port && host.isLocalAddress(*maybe_secure_port)) + || host.isLocalAddress(context.getTCPPort()); + if (!is_local_port) continue;