mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
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:
commit
c557a71199
@ -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,9 +87,10 @@ ExpressionAction ExpressionAction::applyFunction(const FunctionBuilderPtr & func
|
||||
return a;
|
||||
}
|
||||
|
||||
ExpressionAction ExpressionAction::addColumn(const ColumnWithTypeAndName & added_column_,
|
||||
const std::string & row_projection_column,
|
||||
bool is_row_projection_complementary)
|
||||
ExpressionAction ExpressionAction::addColumn(
|
||||
const ColumnWithTypeAndName & added_column_,
|
||||
const std::string & row_projection_column,
|
||||
bool is_row_projection_complementary)
|
||||
{
|
||||
ExpressionAction a;
|
||||
a.type = ADD_COLUMN;
|
||||
|
@ -905,18 +905,21 @@ 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.
|
||||
|
||||
size_t subquery_index = 1;
|
||||
while (true)
|
||||
if (subquery->alias.empty())
|
||||
{
|
||||
alias = "_subquery" + toString(subquery_index);
|
||||
if (!aliases.count("_subquery" + toString(subquery_index)))
|
||||
break;
|
||||
++subquery_index;
|
||||
}
|
||||
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;
|
||||
subquery->setAlias(alias);
|
||||
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;
|
||||
}
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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);
|
||||
|
||||
Hash hash = getTreeHash();
|
||||
writeCString("__subquery_", ostr);
|
||||
writeText(hash.first, ostr);
|
||||
ostr.write('_');
|
||||
writeText(hash.second, 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
|
||||
|
@ -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());
|
||||
|
@ -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
|
25
dbms/tests/queries/0_stateless/00697_in_subquery_shard.sql
Normal file
25
dbms/tests/queries/0_stateless/00697_in_subquery_shard.sql
Normal 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));
|
Loading…
Reference in New Issue
Block a user