Merge pull request #3087 from yandex/CLICKHOUSE-3888-2

Fixed error in case when the only column returned by remote server is a column with result of an IN expression with a subquery.
This commit is contained in:
alexey-milovidov 2018-09-10 07:39:03 +03:00 committed by GitHub
commit c557a71199
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 129 additions and 39 deletions

View File

@ -60,7 +60,8 @@ Names ExpressionAction::getNeededColumns() const
}
ExpressionAction ExpressionAction::applyFunction(const FunctionBuilderPtr & function_,
ExpressionAction ExpressionAction::applyFunction(
const FunctionBuilderPtr & function_,
const std::vector<std::string> & argument_names_,
std::string result_name_,
const std::string & row_projection_column)
@ -86,7 +87,8 @@ ExpressionAction ExpressionAction::applyFunction(const FunctionBuilderPtr & func
return a;
}
ExpressionAction ExpressionAction::addColumn(const ColumnWithTypeAndName & added_column_,
ExpressionAction ExpressionAction::addColumn(
const ColumnWithTypeAndName & added_column_,
const std::string & row_projection_column,
bool is_row_projection_complementary)
{

View File

@ -905,6 +905,8 @@ void ExpressionAnalyzer::addASTAliases(ASTPtr & ast, int ignore_levels)
/// 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.
if (subquery->alias.empty())
{
size_t subquery_index = 1;
while (true)
{
@ -918,6 +920,7 @@ void ExpressionAnalyzer::addASTAliases(ASTPtr & ast, int ignore_levels)
subquery->prefer_alias_to_column_name = true;
aliases[alias] = ast;
}
}
}
@ -2082,12 +2085,11 @@ 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());
}
@ -2096,14 +2098,13 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
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);
/// Do not subquery and create sets. We treat "IN" as "ignore" function.
actions_stack.addAction(ExpressionAction::applyFunction(
FunctionFactory::instance().get("ignore", context),
{ node->arguments->children.at(0)->getColumnName() },
projection_manipulator->getColumnName(getColumnName()),
projection_manipulator->getProjectionSourceColumn()));
}
return;
}

View File

@ -467,8 +467,7 @@ 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,13 +8,17 @@ 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,6 +22,8 @@
#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>
@ -39,10 +41,7 @@
#include <Poco/DirectoryIterator.h>
#include <memory>
#include <boost/filesystem.hpp>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTDropQuery.h>
namespace DB
@ -259,8 +258,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());

View File

@ -0,0 +1,60 @@
1
0
1
0
1
0
1
1
0
0
0
1
0
0
0
0
0
0
1
0
0
1
0
0
0
0
0
0
1
0
1
1
90
1
1
0
0
0
1
0
0
0
0
0
0
1
0
0
1
0
0
0
0
0
0
1
0
1
1
90

View File

@ -0,0 +1,25 @@
SELECT 0 IN 0;
SELECT 0 IN 1;
SELECT 0 IN (SELECT 0);
SELECT 0 IN (SELECT 1);
SELECT dummy IN (SELECT 0) FROM remote('127.0.0.1', system.one);
SELECT dummy IN (SELECT 1) FROM remote('127.0.0.1', system.one);
SELECT dummy IN (SELECT 0) FROM remote('127.0.0.{1,2}', system.one);
SELECT dummy IN (SELECT 1) FROM remote('127.0.0.{1,2}', system.one);
SELECT number IN (SELECT toUInt64(arrayJoin([1, 8]))) FROM remote('127.0.0.{1,2}', numbers(10));
SELECT arrayExists(x -> (x IN (SELECT 1)), [1]) FROM remote('127.0.0.{1,2}', system.one);
SELECT sumIf(number, arrayExists(x -> (x IN (SELECT 1)), [1])) FROM remote('127.0.0.{1,2}', numbers(10));
SET prefer_localhost_replica = 0;
SELECT dummy IN (SELECT 0) FROM remote('127.0.0.{1,2}', system.one);
SELECT dummy IN (SELECT 1) FROM remote('127.0.0.{1,2}', system.one);
SELECT number IN (SELECT toUInt64(arrayJoin([1, 8]))) FROM remote('127.0.0.{1,2}', numbers(10));
SELECT arrayExists(x -> (x IN (SELECT 1)), [1]) FROM remote('127.0.0.{1,2}', system.one);
SELECT sumIf(number, arrayExists(x -> (x IN (SELECT 1)), [1])) FROM remote('127.0.0.{1,2}', numbers(10));