mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Asterisks for multiple joins CLICKHOUSE-4372
This commit is contained in:
parent
97dd0e2aa0
commit
6b49900e1a
@ -203,7 +203,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
|
||||
if (settings.allow_experimental_multiple_joins_emulation)
|
||||
{
|
||||
JoinToSubqueryTransformVisitor::Data join_to_subs_data;
|
||||
JoinToSubqueryTransformVisitor::Data join_to_subs_data{context};
|
||||
JoinToSubqueryTransformVisitor(join_to_subs_data).visit(query_ptr);
|
||||
}
|
||||
|
||||
|
@ -1,8 +1,10 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Interpreters/JoinToSubqueryTransformVisitor.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Interpreters/AsteriskSemantic.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
@ -22,11 +24,113 @@ namespace ErrorCodes
|
||||
extern const int TOO_DEEP_AST;
|
||||
extern const int AMBIGUOUS_COLUMN_NAME;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int UNKNOWN_IDENTIFIER;
|
||||
}
|
||||
|
||||
NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpression & table_expression, const Context & context);
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/// Replace asterisks in select_expression_list with column identifiers
|
||||
struct ExtractAsterisksMatcher
|
||||
{
|
||||
using Visitor = InDepthNodeVisitor<ExtractAsterisksMatcher, true>;
|
||||
|
||||
struct Data
|
||||
{
|
||||
std::unordered_map<String, NamesAndTypesList> table_columns;
|
||||
std::shared_ptr<ASTExpressionList> new_select_expression_list;
|
||||
|
||||
Data(const Context & context, const std::vector<const ASTTableExpression *> & table_expressions)
|
||||
{
|
||||
for (const auto & expr : table_expressions)
|
||||
{
|
||||
if (expr->subquery)
|
||||
{
|
||||
table_columns.clear();
|
||||
break;
|
||||
}
|
||||
|
||||
String table_name = DatabaseAndTableWithAlias(*expr, context.getCurrentDatabase()).getQualifiedNamePrefix(false);
|
||||
NamesAndTypesList columns = getNamesAndTypeListFromTableExpression(*expr, context);
|
||||
table_columns.emplace(std::move(table_name), std::move(columns));
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
static bool needChildVisit(ASTPtr &, const ASTPtr &) { return false; }
|
||||
|
||||
static void visit(ASTPtr & ast, Data & data)
|
||||
{
|
||||
if (auto * t = ast->as<ASTSelectQuery>())
|
||||
visit(*t, ast, data);
|
||||
if (auto * t = ast->as<ASTExpressionList>())
|
||||
visit(*t, ast, data);
|
||||
}
|
||||
|
||||
static void visit(ASTSelectQuery & node, ASTPtr &, Data & data)
|
||||
{
|
||||
if (data.table_columns.empty())
|
||||
return;
|
||||
|
||||
Visitor(data).visit(node.select_expression_list);
|
||||
if (!data.new_select_expression_list)
|
||||
return;
|
||||
|
||||
size_t pos = 0;
|
||||
for (; pos < node.children.size(); ++pos)
|
||||
if (node.children[pos].get() == node.select_expression_list.get())
|
||||
break;
|
||||
if (pos == node.children.size())
|
||||
throw Exception("No select expressions list in select", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
node.select_expression_list = data.new_select_expression_list;
|
||||
node.children[pos] = node.select_expression_list;
|
||||
}
|
||||
|
||||
static void visit(ASTExpressionList & node, ASTPtr &, Data & data)
|
||||
{
|
||||
bool has_asterisks = false;
|
||||
data.new_select_expression_list = std::make_shared<ASTExpressionList>();
|
||||
data.new_select_expression_list->children.reserve(node.children.size());
|
||||
|
||||
for (auto & child : node.children)
|
||||
{
|
||||
if (child->as<ASTAsterisk>())
|
||||
{
|
||||
has_asterisks = true;
|
||||
|
||||
for (auto & pr : data.table_columns)
|
||||
for (const auto & column : pr.second)
|
||||
data.new_select_expression_list->children.push_back(
|
||||
std::make_shared<ASTIdentifier>(std::vector<String>{pr.first, column.name}));
|
||||
}
|
||||
else if (child->as<ASTQualifiedAsterisk>())
|
||||
{
|
||||
has_asterisks = true;
|
||||
|
||||
if (child->children.size() != 1)
|
||||
throw Exception("Logical error: qualified asterisk must have exactly one child", ErrorCodes::LOGICAL_ERROR);
|
||||
ASTIdentifier & identifier = child->children[0]->as<ASTIdentifier &>();
|
||||
|
||||
auto it = data.table_columns.find(identifier.name);
|
||||
if (it == data.table_columns.end())
|
||||
throw Exception("Unknown qualified identifier: " + identifier.name, ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||
|
||||
for (const auto & column : it->second)
|
||||
data.new_select_expression_list->children.push_back(
|
||||
std::make_shared<ASTIdentifier>(std::vector<String>{it->first, column.name}));
|
||||
}
|
||||
else
|
||||
data.new_select_expression_list->children.push_back(child);
|
||||
}
|
||||
|
||||
if (!has_asterisks)
|
||||
data.new_select_expression_list.reset();
|
||||
}
|
||||
};
|
||||
|
||||
/// Find columns with aliases to push them into rewritten subselects.
|
||||
/// Normalize table aliases: table_name.column_name -> table_alias.column_name
|
||||
/// Make aliases maps (alias -> column_name, column_name -> alias)
|
||||
@ -41,7 +145,7 @@ struct ColumnAliasesMatcher
|
||||
std::vector<std::pair<ASTIdentifier *, bool>> compound_identifiers;
|
||||
std::set<String> allowed_long_names; /// original names allowed as aliases '--t.x as t.x' (select expressions only).
|
||||
|
||||
Data(std::vector<DatabaseAndTableWithAlias> && tables_)
|
||||
Data(const std::vector<DatabaseAndTableWithAlias> && tables_)
|
||||
: tables(tables_)
|
||||
, public_names(false)
|
||||
{}
|
||||
@ -101,7 +205,7 @@ struct ColumnAliasesMatcher
|
||||
visit(*t, ast, data);
|
||||
|
||||
if (ast->as<ASTAsterisk>() || ast->as<ASTQualifiedAsterisk>())
|
||||
throw Exception("Multiple JOIN do not support asterisks yet", ErrorCodes::NOT_IMPLEMENTED);
|
||||
throw Exception("Multiple JOIN do not support asterisks for complex queries yet", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
static void visit(ASTIdentifier & node, ASTPtr &, Data & data)
|
||||
@ -190,7 +294,7 @@ struct RewriteTablesVisitorData
|
||||
}
|
||||
};
|
||||
|
||||
bool needRewrite(ASTSelectQuery & select)
|
||||
bool needRewrite(ASTSelectQuery & select, std::vector<const ASTTableExpression *> & table_expressions)
|
||||
{
|
||||
if (!select.tables)
|
||||
return false;
|
||||
@ -203,9 +307,16 @@ bool needRewrite(ASTSelectQuery & select)
|
||||
if (num_tables <= 2)
|
||||
return false;
|
||||
|
||||
for (size_t i = 1; i < tables->children.size(); ++i)
|
||||
table_expressions.reserve(num_tables);
|
||||
for (size_t i = 0; i < num_tables; ++i)
|
||||
{
|
||||
const auto * table = tables->children[i]->as<ASTTablesInSelectQueryElement>();
|
||||
if (table && table->table_expression)
|
||||
if (const auto * expression = table->table_expression->as<ASTTableExpression>())
|
||||
table_expressions.push_back(expression);
|
||||
if (!i)
|
||||
continue;
|
||||
|
||||
if (!table || !table->table_join)
|
||||
throw Exception("Multiple JOIN expects joined tables", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
@ -223,6 +334,7 @@ bool needRewrite(ASTSelectQuery & select)
|
||||
|
||||
using RewriteMatcher = OneTypeMatcher<RewriteTablesVisitorData>;
|
||||
using RewriteVisitor = InDepthNodeVisitor<RewriteMatcher, true>;
|
||||
using ExtractAsterisksVisitor = ExtractAsterisksMatcher::Visitor;
|
||||
using ColumnAliasesVisitor = InDepthNodeVisitor<ColumnAliasesMatcher, true>;
|
||||
using AppendSemanticMatcher = OneTypeMatcher<AppendSemanticVisitorData>;
|
||||
using AppendSemanticVisitor = InDepthNodeVisitor<AppendSemanticMatcher, true>;
|
||||
@ -236,13 +348,17 @@ void JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data & data)
|
||||
visit(*t, ast, data);
|
||||
}
|
||||
|
||||
void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr &, Data & data)
|
||||
void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast, Data & data)
|
||||
{
|
||||
using RevertedAliases = AsteriskSemantic::RevertedAliases;
|
||||
|
||||
if (!needRewrite(select))
|
||||
std::vector<const ASTTableExpression *> table_expressions;
|
||||
if (!needRewrite(select, table_expressions))
|
||||
return;
|
||||
|
||||
ExtractAsterisksVisitor::Data asterisks_data(data.context, table_expressions);
|
||||
ExtractAsterisksVisitor(asterisks_data).visit(ast);
|
||||
|
||||
ColumnAliasesVisitor::Data aliases_data(getDatabaseAndTables(select, ""));
|
||||
if (select.select_expression_list)
|
||||
{
|
||||
|
@ -6,6 +6,7 @@ namespace DB
|
||||
{
|
||||
|
||||
class ASTSelectQuery;
|
||||
class Context;
|
||||
|
||||
/// AST transformer. It replaces multiple joins to (subselect + join) track.
|
||||
/// 'select * from t1 join t2 on ... join t3 on ... join t4 on ...' would be rewriten with
|
||||
@ -15,6 +16,7 @@ class JoinToSubqueryTransformMatcher
|
||||
public:
|
||||
struct Data
|
||||
{
|
||||
const Context & context;
|
||||
bool done = false;
|
||||
};
|
||||
|
||||
|
@ -0,0 +1,5 @@
|
||||
0 0 0
|
||||
0 0 0
|
||||
0
|
||||
0 0
|
||||
0 0 0
|
@ -0,0 +1,9 @@
|
||||
select t1.dummy, t2.dummy, t3.dummy from system.one t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy;
|
||||
select * from system.one t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy;
|
||||
select t1.* from system.one t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy;
|
||||
select t2.*, t3.* from system.one t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy;
|
||||
select t1.dummy, t2.*, t3.dummy from system.one t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy;
|
||||
|
||||
select t1.dummy, t2.*, t3.dummy from (select * from system.one) t1
|
||||
join system.one t2 on t1.dummy = t2.dummy
|
||||
join system.one t3 ON t1.dummy = t3.dummy; -- { serverError 48 }
|
Loading…
Reference in New Issue
Block a user