Revert "Fixed error [#CLICKHOUSE-3888]"

This reverts commit 1d8302a4d7.
This commit is contained in:
Alexey Milovidov 2018-09-10 00:17:29 +03:00
parent c6e31b47ad
commit 42c49da354
5 changed files with 53 additions and 48 deletions

View File

@ -901,8 +901,6 @@ void ExpressionAnalyzer::addASTAliases(ASTPtr & ast, int ignore_levels)
aliases[alias] = ast;
}
else if (auto subquery = typeid_cast<ASTSubquery *>(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.
@ -921,7 +919,6 @@ void ExpressionAnalyzer::addASTAliases(ASTPtr & ast, int ignore_levels)
aliases[alias] = ast;
}
}
}
void ExpressionAnalyzer::normalizeTree()
@ -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<Set>(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<const ASTIdentifier *>(arg.get());
if (typeid_cast<const ASTSubquery *>(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)
{
@ -2094,12 +2081,32 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
}
if (functionIsInOrGlobalInOperator(node->name))
{
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);
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);
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<DataTypeUInt8>();
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

View File

@ -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();

View File

@ -467,7 +467,8 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
if (expressions.prewhere_info)
pipeline.streams.back() = std::make_shared<FilterBlockInputStream>(
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
{

View File

@ -8,18 +8,14 @@ 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);
}
}
void ASTSubquery::formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{

View File

@ -22,8 +22,6 @@
#include <Parsers/parseQuery.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTDropQuery.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/InterpreterAlterQuery.h>
@ -41,7 +39,10 @@
#include <Poco/DirectoryIterator.h>
#include <memory>
#include <boost/filesystem.hpp>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTDropQuery.h>
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());