From 42c49da35434cdcde7fbf753ae122af2c6ed7fca Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Sep 2018 00:17:29 +0300 Subject: [PATCH] Revert "Fixed error [#CLICKHOUSE-3888]" This reverts commit 1d8302a4d7a66869c93241df82a30e18df50867c. --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 71 ++++++++++--------- dbms/src/Interpreters/ExpressionAnalyzer.h | 2 +- .../Interpreters/InterpreterSelectQuery.cpp | 3 +- dbms/src/Parsers/ASTSubquery.cpp | 16 ++--- dbms/src/Storages/StorageDistributed.cpp | 9 +-- 5 files changed, 53 insertions(+), 48 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 1bb808cc1ae..4fad806bef0 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -902,24 +902,21 @@ void ExpressionAnalyzer::addASTAliases(ASTPtr & ast, int ignore_levels) } else if (auto subquery = typeid_cast(ast.get())) { - if (subquery->alias.empty()) + /// Set unique aliases for all subqueries. This is needed, because content of subqueries could change after recursive analysis, + /// and auto-generated column names could become incorrect. + + size_t subquery_index = 1; + while (true) { - /// Set unique aliases for all subqueries. This is needed, because content of subqueries could change after recursive analysis, - /// and auto-generated column names could become incorrect. - - size_t subquery_index = 1; - while (true) - { - alias = "_subquery" + toString(subquery_index); - if (!aliases.count("_subquery" + toString(subquery_index))) - break; - ++subquery_index; - } - - subquery->setAlias(alias); - subquery->prefer_alias_to_column_name = true; - aliases[alias] = ast; + alias = "_subquery" + toString(subquery_index); + if (!aliases.count("_subquery" + toString(subquery_index))) + break; + ++subquery_index; } + + subquery->setAlias(alias); + subquery->prefer_alias_to_column_name = true; + aliases[alias] = ast; } } @@ -1564,7 +1561,7 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block & } -void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_block, bool no_subqueries) +void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_block) { /** You need to convert the right argument to a set. * This can be a table name, a value, a value enumeration, or a subquery. @@ -1577,16 +1574,6 @@ void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_ if (prepared_sets.count(arg->range)) return; - if (no_subqueries) - { - String set_id = arg->getColumnName(); - SubqueryForSet & subquery_for_set = subqueries_for_sets[set_id]; - SetPtr set = std::make_shared(SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode), false); - subquery_for_set.set = set; - prepared_sets[arg->range] = set; - return; - } - /// If the subquery or table name for SELECT. const ASTIdentifier * identifier = typeid_cast(arg.get()); if (typeid_cast(arg.get()) || identifier) @@ -1594,7 +1581,7 @@ void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_ /// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery. String set_id = arg->getColumnName(); - /// A special case if the name of the table is specified on the right side of the IN statement, + /// A special case is if the name of the table is specified on the right side of the IN statement, /// and the table has the type Set (a previously prepared set). if (identifier) { @@ -2095,11 +2082,31 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries, if (functionIsInOrGlobalInOperator(node->name)) { - /// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything). - getActionsImpl(node->arguments->children.at(0), no_subqueries, only_consts, actions_stack, projection_manipulator); + if (!no_subqueries) + { + /// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything). + getActionsImpl(node->arguments->children.at(0), no_subqueries, only_consts, actions_stack, + projection_manipulator); - /// Transform tuple or subquery into a set. - makeSet(node, actions_stack.getSampleBlock(), no_subqueries); + /// Transform tuple or subquery into a set. + makeSet(node, actions_stack.getSampleBlock()); + } + else + { + if (!only_consts) + { + /// We are in the part of the tree that we are not going to compute. You just need to define types. + /// Do not subquery and create sets. We insert an arbitrary column of the correct type. + ColumnWithTypeAndName fake_column; + fake_column.name = projection_manipulator->getColumnName(getColumnName()); + fake_column.type = std::make_shared(); + fake_column.column = fake_column.type->createColumn(); + actions_stack.addAction(ExpressionAction::addColumn(fake_column, projection_manipulator->getProjectionSourceColumn(), false)); + getActionsImpl(node->arguments->children.at(0), no_subqueries, only_consts, actions_stack, + projection_manipulator); + } + return; + } } /// A special function `indexHint`. Everything that is inside it is not calculated diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index 16aad4b64d9..8b11a8225a2 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -336,7 +336,7 @@ private: void optimizeIfWithConstantConditionImpl(ASTPtr & current_ast); bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & value) const; - void makeSet(const ASTFunction * node, const Block & sample_block, bool no_subqueries); + void makeSet(const ASTFunction * node, const Block & sample_block); /// Adds a list of ALIAS columns from the table. void addAliasColumns(); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 3049c77523e..d8dd75b23f1 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -467,7 +467,8 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt if (expressions.prewhere_info) pipeline.streams.back() = std::make_shared( pipeline.streams.back(), expressions.prewhere_info->prewhere_actions, - expressions.prewhere_info->prewhere_column_name, expressions.prewhere_info->remove_prewhere_column); + expressions.prewhere_info->prewhere_column_name, expressions.prewhere_info->remove_prewhere_column + ); } else { diff --git a/dbms/src/Parsers/ASTSubquery.cpp b/dbms/src/Parsers/ASTSubquery.cpp index 55ea89e3f07..b0d45cd26f1 100644 --- a/dbms/src/Parsers/ASTSubquery.cpp +++ b/dbms/src/Parsers/ASTSubquery.cpp @@ -8,17 +8,13 @@ void ASTSubquery::appendColumnNameImpl(WriteBuffer & ostr) const { /// This is a hack. We use alias, if available, because otherwise tree could change during analysis. if (!alias.empty()) - { writeString(alias, ostr); - } - else - { - Hash hash = getTreeHash(); - writeCString("__subquery_", ostr); - writeText(hash.first, ostr); - ostr.write('_'); - writeText(hash.second, ostr); - } + + Hash hash = getTreeHash(); + writeCString("__subquery_", ostr); + writeText(hash.first, ostr); + ostr.write('_'); + writeText(hash.second, ostr); } void ASTSubquery::formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index bd33f393d24..f561727566d 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -22,8 +22,6 @@ #include #include #include -#include -#include #include #include @@ -41,7 +39,10 @@ #include #include + #include +#include +#include namespace DB @@ -258,8 +259,8 @@ BlockInputStreams StorageDistributed::read( Block header = materializeBlock(InterpreterSelectQuery(query_info.query, context, Names{}, processed_stage).getSampleBlock()); - ClusterProxy::SelectStreamFactory select_stream_factory = remote_table_function_ptr - ? ClusterProxy::SelectStreamFactory( + ClusterProxy::SelectStreamFactory select_stream_factory = remote_table_function_ptr ? + ClusterProxy::SelectStreamFactory( header, processed_stage, remote_table_function_ptr, context.getExternalTables()) : ClusterProxy::SelectStreamFactory( header, processed_stage, QualifiedTableName{remote_database, remote_table}, context.getExternalTables());