mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 17:41:59 +00:00
JoinToSubqueryTransform with short names
This commit is contained in:
parent
1fc6c60668
commit
48ee3e83fa
38
dbms/src/Interpreters/AsteriskSemantic.h
Normal file
38
dbms/src/Interpreters/AsteriskSemantic.h
Normal file
@ -0,0 +1,38 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <unordered_map>
|
||||||
|
|
||||||
|
#include <Parsers/ASTAsterisk.h>
|
||||||
|
#include <Parsers/ASTQualifiedAsterisk.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct AsteriskSemanticImpl
|
||||||
|
{
|
||||||
|
using RevertedAliases = std::unordered_map<String, std::vector<String>>;
|
||||||
|
using RevertedAliasesPtr = std::shared_ptr<RevertedAliases>;
|
||||||
|
|
||||||
|
RevertedAliasesPtr aliases; /// map of aliases that should be set in phase of * expanding.
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
struct AsteriskSemantic
|
||||||
|
{
|
||||||
|
using RevertedAliases = AsteriskSemanticImpl::RevertedAliases;
|
||||||
|
using RevertedAliasesPtr = AsteriskSemanticImpl::RevertedAliasesPtr;
|
||||||
|
|
||||||
|
static void setAliases(ASTAsterisk & node, const RevertedAliasesPtr & aliases) { node.semantic = makeSemantic(aliases); }
|
||||||
|
static void setAliases(ASTQualifiedAsterisk & node, const RevertedAliasesPtr & aliases) { node.semantic = makeSemantic(aliases); }
|
||||||
|
|
||||||
|
static RevertedAliasesPtr getAliases(const ASTAsterisk & node) { return node.semantic ? node.semantic->aliases : nullptr; }
|
||||||
|
static RevertedAliasesPtr getAliases(const ASTQualifiedAsterisk & node) { return node.semantic ? node.semantic->aliases : nullptr; }
|
||||||
|
|
||||||
|
private:
|
||||||
|
static std::shared_ptr<AsteriskSemanticImpl> makeSemantic(const RevertedAliasesPtr & aliases)
|
||||||
|
{
|
||||||
|
return std::make_shared<AsteriskSemanticImpl>(AsteriskSemanticImpl{aliases});
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
@ -126,15 +126,18 @@ void IdentifierSemantic::setColumnNormalName(ASTIdentifier & identifier, const D
|
|||||||
identifier.semantic->can_be_alias = false;
|
identifier.semantic->can_be_alias = false;
|
||||||
|
|
||||||
if (identifier.semantic->need_long_name)
|
if (identifier.semantic->need_long_name)
|
||||||
|
setColumnLongName(identifier, db_and_table);
|
||||||
|
}
|
||||||
|
|
||||||
|
void IdentifierSemantic::setColumnLongName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
|
||||||
|
{
|
||||||
|
String prefix = db_and_table.getQualifiedNamePrefix();
|
||||||
|
if (!prefix.empty())
|
||||||
{
|
{
|
||||||
String prefix = db_and_table.getQualifiedNamePrefix();
|
String short_name = identifier.shortName();
|
||||||
if (!prefix.empty())
|
identifier.name = prefix + short_name;
|
||||||
{
|
prefix.resize(prefix.size() - 1); /// crop dot
|
||||||
String short_name = identifier.shortName();
|
identifier.name_parts = {prefix, short_name};
|
||||||
identifier.name = prefix + short_name;
|
|
||||||
prefix.resize(prefix.size() - 1); /// crop dot
|
|
||||||
identifier.name_parts = {prefix, short_name};
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -145,4 +148,9 @@ String IdentifierSemantic::columnNormalName(const ASTIdentifier & identifier, co
|
|||||||
return copy->getAliasOrColumnName();
|
return copy->getAliasOrColumnName();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
String IdentifierSemantic::columnLongName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
|
||||||
|
{
|
||||||
|
return db_and_table.getQualifiedNamePrefix() + identifier.shortName();
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -28,7 +28,9 @@ struct IdentifierSemantic
|
|||||||
|
|
||||||
static size_t canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
|
static size_t canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
|
||||||
static String columnNormalName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
|
static String columnNormalName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
|
||||||
|
static String columnLongName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
|
||||||
static void setColumnNormalName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
|
static void setColumnNormalName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
|
||||||
|
static void setColumnLongName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
|
||||||
static void setNeedLongName(ASTIdentifier & identifier, bool); /// if set setColumnNormalName makes qualified name
|
static void setNeedLongName(ASTIdentifier & identifier, bool); /// if set setColumnNormalName makes qualified name
|
||||||
static bool canBeAlias(const ASTIdentifier & identifier);
|
static bool canBeAlias(const ASTIdentifier & identifier);
|
||||||
static void setMembership(ASTIdentifier & identifier, size_t table_no);
|
static void setMembership(ASTIdentifier & identifier, size_t table_no);
|
||||||
|
@ -1,5 +1,8 @@
|
|||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include <Interpreters/JoinToSubqueryTransformVisitor.h>
|
#include <Interpreters/JoinToSubqueryTransformVisitor.h>
|
||||||
|
#include <Interpreters/IdentifierSemantic.h>
|
||||||
|
#include <Interpreters/AsteriskSemantic.h>
|
||||||
|
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||||
#include <Parsers/ASTSelectQuery.h>
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/ASTIdentifier.h>
|
||||||
@ -9,6 +12,7 @@
|
|||||||
#include <Parsers/parseQuery.h>
|
#include <Parsers/parseQuery.h>
|
||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -16,48 +20,143 @@ namespace ErrorCodes
|
|||||||
{
|
{
|
||||||
extern const int LOGICAL_ERROR;
|
extern const int LOGICAL_ERROR;
|
||||||
extern const int TOO_DEEP_AST;
|
extern const int TOO_DEEP_AST;
|
||||||
|
extern const int AMBIGUOUS_COLUMN_NAME;
|
||||||
}
|
}
|
||||||
|
|
||||||
#if 0
|
namespace
|
||||||
/// Attach additional semantic info to generated select.
|
{
|
||||||
|
|
||||||
|
/// 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)
|
||||||
|
struct ColumnAliasesVisitorData
|
||||||
|
{
|
||||||
|
using TypeToVisit = ASTIdentifier;
|
||||||
|
|
||||||
|
const std::vector<DatabaseAndTableWithAlias> tables;
|
||||||
|
AsteriskSemantic::RevertedAliases rev_aliases;
|
||||||
|
std::unordered_map<String, String> aliases;
|
||||||
|
std::vector<ASTIdentifier *> short_identifiers;
|
||||||
|
std::vector<ASTIdentifier *> compound_identifiers;
|
||||||
|
|
||||||
|
ColumnAliasesVisitorData(std::vector<DatabaseAndTableWithAlias> && tables_)
|
||||||
|
: tables(tables_)
|
||||||
|
{}
|
||||||
|
|
||||||
|
void visit(ASTIdentifier & node, ASTPtr &)
|
||||||
|
{
|
||||||
|
if (node.isShort())
|
||||||
|
{
|
||||||
|
short_identifiers.push_back(&node);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
bool last_table = false;
|
||||||
|
String long_name;
|
||||||
|
for (auto & table : tables)
|
||||||
|
{
|
||||||
|
if (IdentifierSemantic::canReferColumnToTable(node, table))
|
||||||
|
{
|
||||||
|
if (!long_name.empty())
|
||||||
|
throw Exception("Cannot refer column '" + node.name + "' to one table", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
|
||||||
|
IdentifierSemantic::setColumnLongName(node, table); /// table_name.column_name -> table_alias.column_name
|
||||||
|
long_name = node.name;
|
||||||
|
if (&table == &tables.back())
|
||||||
|
last_table = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (long_name.empty())
|
||||||
|
throw Exception("Cannot refer column '" + node.name + "' to table", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
|
||||||
|
|
||||||
|
String alias = node.tryGetAlias();
|
||||||
|
if (!alias.empty())
|
||||||
|
{
|
||||||
|
aliases[alias] = long_name;
|
||||||
|
rev_aliases[long_name].push_back(alias);
|
||||||
|
|
||||||
|
if (!last_table)
|
||||||
|
{
|
||||||
|
node.setShortName(alias);
|
||||||
|
node.setAlias("");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else
|
||||||
|
compound_identifiers.push_back(&node);
|
||||||
|
}
|
||||||
|
|
||||||
|
void replaceIdentifiersWithAliases()
|
||||||
|
{
|
||||||
|
for (auto * identifier : short_identifiers)
|
||||||
|
if (!aliases.count(identifier->name))
|
||||||
|
throw Exception("Short column name '" + identifier->name + "' is not an alias", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
|
||||||
|
|
||||||
|
for (auto * identifier : compound_identifiers)
|
||||||
|
{
|
||||||
|
auto it = rev_aliases.find(identifier->name);
|
||||||
|
if (it == rev_aliases.end())
|
||||||
|
{
|
||||||
|
bool last_table = IdentifierSemantic::canReferColumnToTable(*identifier, tables.back());
|
||||||
|
if (!last_table)
|
||||||
|
throw Exception("Column name without alias '" + identifier->name + "'", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
if (it->second.empty())
|
||||||
|
throw Exception("No alias for '" + identifier->name + "'", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
identifier->setShortName(it->second[0]);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
/// Attach additional semantic info to generated selects.
|
||||||
struct AppendSemanticVisitorData
|
struct AppendSemanticVisitorData
|
||||||
{
|
{
|
||||||
using TypeToVisit = ASTSelectQuery;
|
using TypeToVisit = ASTSelectQuery;
|
||||||
|
|
||||||
const SemanticPtr & semantic;
|
AsteriskSemantic::RevertedAliasesPtr rev_aliases = {};
|
||||||
bool done = false;
|
bool done = false;
|
||||||
|
|
||||||
void visit(ASTSelectQuery & select, ASTPtr &)
|
void visit(ASTSelectQuery & select, ASTPtr &)
|
||||||
{
|
{
|
||||||
if (done || !semantic)
|
if (done || !rev_aliases || !select.select_expression_list)
|
||||||
return;
|
return;
|
||||||
select.semantic = semantic->clone();
|
|
||||||
|
for (auto & child : select.select_expression_list->children)
|
||||||
|
{
|
||||||
|
if (auto * node = typeid_cast<ASTAsterisk *>(child.get()))
|
||||||
|
AsteriskSemantic::setAliases(*node, rev_aliases);
|
||||||
|
if (auto * node = typeid_cast<ASTQualifiedAsterisk *>(child.get()))
|
||||||
|
AsteriskSemantic::setAliases(*node, rev_aliases);
|
||||||
|
}
|
||||||
|
|
||||||
done = true;
|
done = true;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
#endif
|
|
||||||
|
|
||||||
/// Replaces one table element with pair.
|
|
||||||
|
/// Replaces table elements with pair.
|
||||||
struct RewriteTablesVisitorData
|
struct RewriteTablesVisitorData
|
||||||
{
|
{
|
||||||
using TypeToVisit = ASTTablesInSelectQuery;
|
using TypeToVisit = ASTTablesInSelectQuery;
|
||||||
|
|
||||||
const ASTPtr & left;
|
ASTPtr left;
|
||||||
const ASTPtr & right;
|
ASTPtr right;
|
||||||
bool done = false;
|
bool done = false;
|
||||||
|
|
||||||
|
/// @note Do not change ASTTablesInSelectQuery itself. No need to change select.tables.
|
||||||
void visit(ASTTablesInSelectQuery &, ASTPtr & ast)
|
void visit(ASTTablesInSelectQuery &, ASTPtr & ast)
|
||||||
{
|
{
|
||||||
if (done)
|
if (done)
|
||||||
return;
|
return;
|
||||||
ast->children.clear();
|
std::vector<ASTPtr> new_tables{left, right};
|
||||||
ast->children.push_back(left);
|
ast->children.swap(new_tables);
|
||||||
ast->children.push_back(right);
|
|
||||||
done = true;
|
done = true;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
static bool needRewrite(ASTSelectQuery & select)
|
bool needRewrite(ASTSelectQuery & select)
|
||||||
{
|
{
|
||||||
if (!select.tables)
|
if (!select.tables)
|
||||||
return false;
|
return false;
|
||||||
@ -70,28 +169,32 @@ static bool needRewrite(ASTSelectQuery & select)
|
|||||||
if (num_tables <= 2)
|
if (num_tables <= 2)
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
|
for (size_t i = 1; i < tables->children.size(); ++i)
|
||||||
|
{
|
||||||
|
auto table = typeid_cast<const ASTTablesInSelectQueryElement *>(tables->children[i].get());
|
||||||
|
if (!table || !table->table_join)
|
||||||
|
throw Exception("Multiple JOIN expects joined tables", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
auto join = typeid_cast<const ASTTableJoin *>(table->table_join.get());
|
||||||
|
if (join->kind == ASTTableJoin::Kind::Comma)
|
||||||
|
throw Exception("Multiple COMMA JOIN is not supported", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
/// it's not trivial to support mix of JOIN ON & JOIN USING cause of short names
|
||||||
|
if (!join || !join->on_expression)
|
||||||
|
throw Exception("Multiple JOIN expects JOIN with ON section", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
static void appendTableNameAndAlias(std::vector<String> & hidden, const ASTPtr & table_element)
|
using RewriteMatcher = OneTypeMatcher<RewriteTablesVisitorData>;
|
||||||
{
|
using RewriteVisitor = InDepthNodeVisitor<RewriteMatcher, true>;
|
||||||
auto element = typeid_cast<const ASTTablesInSelectQueryElement *>(table_element.get());
|
using ColumnAliasesMatcher = OneTypeMatcher<ColumnAliasesVisitorData>;
|
||||||
if (!element || element->children.empty())
|
using ColumnAliasesVisitor = InDepthNodeVisitor<ColumnAliasesMatcher, true>;
|
||||||
throw Exception("Expected TablesInSelectQueryElement with at least one child", ErrorCodes::LOGICAL_ERROR);
|
using AppendSemanticMatcher = OneTypeMatcher<AppendSemanticVisitorData>;
|
||||||
|
using AppendSemanticVisitor = InDepthNodeVisitor<AppendSemanticMatcher, true>;
|
||||||
|
|
||||||
auto table_expression = typeid_cast<const ASTTableExpression *>(element->children[0].get());
|
} /// namelesspace
|
||||||
if (!table_expression || table_expression->children.empty())
|
|
||||||
throw Exception("Expected TableExpression with at least one child", ErrorCodes::LOGICAL_ERROR);
|
|
||||||
|
|
||||||
String alias = table_expression->children[0]->tryGetAlias();
|
|
||||||
if (!alias.empty())
|
|
||||||
hidden.push_back(alias);
|
|
||||||
|
|
||||||
if (auto opt_name = getIdentifierName(table_expression->children[0]))
|
|
||||||
hidden.push_back(*opt_name);
|
|
||||||
else if (alias.empty())
|
|
||||||
throw Exception("Expected Identifier or subquery with alias", ErrorCodes::LOGICAL_ERROR);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
std::vector<ASTPtr *> JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data & data)
|
std::vector<ASTPtr *> JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data & data)
|
||||||
@ -101,79 +204,84 @@ std::vector<ASTPtr *> JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data &
|
|||||||
return {};
|
return {};
|
||||||
}
|
}
|
||||||
|
|
||||||
void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast, Data & data)
|
void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr &, Data & data)
|
||||||
{
|
{
|
||||||
static String alias_prefix = "__join"; /// FIXME
|
using RevertedAliases = AsteriskSemantic::RevertedAliases;
|
||||||
|
|
||||||
if (!needRewrite(select))
|
if (!needRewrite(select))
|
||||||
return;
|
return;
|
||||||
|
|
||||||
auto tables = typeid_cast<const ASTTablesInSelectQuery *>(select.tables.get());
|
ColumnAliasesVisitor::Data aliases_data(getDatabaseAndTables(select, ""));
|
||||||
if (!tables)
|
if (select.select_expression_list)
|
||||||
throw Exception("TablesInSelectQuery expected", ErrorCodes::LOGICAL_ERROR);
|
ColumnAliasesVisitor(aliases_data).visit(select.select_expression_list);
|
||||||
|
if (select.where_expression)
|
||||||
|
ColumnAliasesVisitor(aliases_data).visit(select.where_expression);
|
||||||
|
if (select.prewhere_expression)
|
||||||
|
ColumnAliasesVisitor(aliases_data).visit(select.prewhere_expression);
|
||||||
|
if (select.having_expression)
|
||||||
|
ColumnAliasesVisitor(aliases_data).visit(select.having_expression);
|
||||||
|
|
||||||
size_t num_tables = tables->children.size();
|
/// JOIN sections
|
||||||
ASTPtr left = tables->children[0];
|
for (auto & child : select.tables->children)
|
||||||
|
|
||||||
for (size_t i = 1; i < num_tables - 1; ++i)
|
|
||||||
{
|
{
|
||||||
ASTPtr right = tables->children[i];
|
auto table = typeid_cast<ASTTablesInSelectQueryElement *>(child.get());
|
||||||
std::vector<String> hidden_names;
|
if (table->table_join)
|
||||||
appendTableNameAndAlias(hidden_names, left);
|
{
|
||||||
appendTableNameAndAlias(hidden_names, right);
|
auto * join = typeid_cast<ASTTableJoin *>(table->table_join.get());
|
||||||
|
ColumnAliasesVisitor(aliases_data).visit(join->on_expression);
|
||||||
String subquery_name = alias_prefix + toString(i);
|
}
|
||||||
|
|
||||||
left = replaceJoin(select, left, right, subquery_name);
|
|
||||||
if (!left)
|
|
||||||
return;
|
|
||||||
|
|
||||||
//SemanticSelectQuery::hideNames(select, hidden_names, subquery_name);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
select.tables = std::make_shared<ASTTablesInSelectQuery>();
|
aliases_data.replaceIdentifiersWithAliases();
|
||||||
select.tables->children.push_back(left);
|
|
||||||
select.tables->children.push_back(tables->children.back());
|
auto rev_aliases = std::make_shared<RevertedAliases>();
|
||||||
|
rev_aliases->swap(aliases_data.rev_aliases);
|
||||||
|
|
||||||
|
auto & src_tables = select.tables->children;
|
||||||
|
ASTPtr left_table = src_tables[0];
|
||||||
|
|
||||||
|
for (size_t i = 1; i < src_tables.size() - 1; ++i)
|
||||||
|
{
|
||||||
|
left_table = replaceJoin(left_table, src_tables[i]);
|
||||||
|
if (!left_table)
|
||||||
|
throw Exception("Cannot replace tables with subselect", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
/// attach data to generated asterisk
|
||||||
|
AppendSemanticVisitor::Data semantic_data{rev_aliases, false};
|
||||||
|
AppendSemanticVisitor(semantic_data).visit(left_table);
|
||||||
|
}
|
||||||
|
|
||||||
|
/// replace tables in select with generated two-table join
|
||||||
|
RewriteVisitor::Data visitor_data{left_table, src_tables.back()};
|
||||||
|
RewriteVisitor(visitor_data).visit(select.tables);
|
||||||
|
|
||||||
ast = ast->clone(); /// rewrite AST in right manner
|
|
||||||
data.done = true;
|
data.done = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
ASTPtr JoinToSubqueryTransformMatcher::replaceJoin(ASTSelectQuery &, ASTPtr ast_left, ASTPtr ast_right, const String & subquery_alias)
|
static ASTPtr makeSubqueryTemplate()
|
||||||
{
|
{
|
||||||
#if 0
|
ParserTablesInSelectQueryElement parser(true);
|
||||||
using RewriteMatcher = LinkedMatcher<
|
ASTPtr subquery_template = parseQuery(parser, "(select * from _t)", 0);
|
||||||
OneTypeMatcher<RewriteTablesVisitorData>,
|
if (!subquery_template)
|
||||||
OneTypeMatcher<AppendSemanticVisitorData>>;
|
throw Exception("Cannot parse subquery template", ErrorCodes::LOGICAL_ERROR);
|
||||||
#else
|
return subquery_template;
|
||||||
using RewriteMatcher = OneTypeMatcher<RewriteTablesVisitorData>;
|
}
|
||||||
#endif
|
|
||||||
using RewriteVisitor = InDepthNodeVisitor<RewriteMatcher, true>;
|
|
||||||
|
|
||||||
|
ASTPtr JoinToSubqueryTransformMatcher::replaceJoin(ASTPtr ast_left, ASTPtr ast_right)
|
||||||
|
{
|
||||||
auto left = typeid_cast<const ASTTablesInSelectQueryElement *>(ast_left.get());
|
auto left = typeid_cast<const ASTTablesInSelectQueryElement *>(ast_left.get());
|
||||||
auto right = typeid_cast<const ASTTablesInSelectQueryElement *>(ast_right.get());
|
auto right = typeid_cast<const ASTTablesInSelectQueryElement *>(ast_right.get());
|
||||||
if (!left || !right)
|
if (!left || !right)
|
||||||
throw Exception("Two TablesInSelectQueryElements expected", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Two TablesInSelectQueryElements expected", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
if (!right->table_join || right->array_join)
|
if (!right->table_join)
|
||||||
return {};
|
throw Exception("Table join expected", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
auto table_join = typeid_cast<const ASTTableJoin *>(right->table_join.get());
|
static ASTPtr subquery_template = makeSubqueryTemplate();
|
||||||
if (table_join->kind != ASTTableJoin::Kind::Inner)
|
|
||||||
return {};
|
|
||||||
|
|
||||||
ParserTablesInSelectQueryElement parser(true);
|
/// replace '_t' with pair of joined tables
|
||||||
String subquery = "(select * from _t) as " + subquery_alias;
|
ASTPtr res = subquery_template->clone();
|
||||||
ASTPtr res = parseQuery(parser, subquery, 0);
|
|
||||||
if (!res)
|
|
||||||
throw Exception("Cannot parse rewrite query", ErrorCodes::LOGICAL_ERROR);
|
|
||||||
|
|
||||||
#if 0
|
|
||||||
RewriteVisitor::Data visitor_data =
|
|
||||||
std::make_pair<RewriteTablesVisitorData, AppendSemanticVisitorData>({ast_left, ast_right}, {select.semantic});
|
|
||||||
#else
|
|
||||||
RewriteVisitor::Data visitor_data{ast_left, ast_right};
|
RewriteVisitor::Data visitor_data{ast_left, ast_right};
|
||||||
#endif
|
|
||||||
RewriteVisitor(visitor_data).visit(res);
|
RewriteVisitor(visitor_data).visit(res);
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
@ -42,7 +42,7 @@ private:
|
|||||||
static void visit(ASTSelectQuery & select, ASTPtr & ast, Data & data);
|
static void visit(ASTSelectQuery & select, ASTPtr & ast, Data & data);
|
||||||
|
|
||||||
/// @return combined TablesInSelectQueryElement or nullptr if cannot rewrite
|
/// @return combined TablesInSelectQueryElement or nullptr if cannot rewrite
|
||||||
static ASTPtr replaceJoin(ASTSelectQuery & select, ASTPtr left, ASTPtr right, const String & subquery_alias);
|
static ASTPtr replaceJoin(ASTPtr left, ASTPtr right);
|
||||||
};
|
};
|
||||||
|
|
||||||
using JoinToSubqueryTransformVisitor = InDepthNodeVisitor<JoinToSubqueryTransformMatcher, true>;
|
using JoinToSubqueryTransformVisitor = InDepthNodeVisitor<JoinToSubqueryTransformMatcher, true>;
|
||||||
|
@ -2,6 +2,7 @@
|
|||||||
|
|
||||||
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
|
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
|
||||||
#include <Interpreters/IdentifierSemantic.h>
|
#include <Interpreters/IdentifierSemantic.h>
|
||||||
|
#include <Interpreters/AsteriskSemantic.h>
|
||||||
|
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include <Core/Names.h>
|
#include <Core/Names.h>
|
||||||
@ -141,6 +142,32 @@ std::vector<ASTPtr *> TranslateQualifiedNamesMatcher::visit(ASTSelectQuery & sel
|
|||||||
return out;
|
return out;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// qualifed names for duplicates
|
||||||
|
static std::shared_ptr<ASTIdentifier> makeIdentifier(const String & short_name, const String & long_name, bool need_long_name)
|
||||||
|
{
|
||||||
|
if (need_long_name)
|
||||||
|
return std::make_shared<ASTIdentifier>(long_name);
|
||||||
|
return std::make_shared<ASTIdentifier>(short_name);
|
||||||
|
}
|
||||||
|
|
||||||
|
static void addIdentifier(ASTs & nodes, std::shared_ptr<ASTIdentifier> identifier, const String & long_name,
|
||||||
|
AsteriskSemantic::RevertedAliasesPtr aliases)
|
||||||
|
{
|
||||||
|
bool added = false;
|
||||||
|
if (aliases && aliases->count(long_name))
|
||||||
|
{
|
||||||
|
for (const String & alias : (*aliases)[long_name])
|
||||||
|
{
|
||||||
|
nodes.push_back(identifier->clone());
|
||||||
|
nodes.back()->setAlias(alias);
|
||||||
|
added = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!added)
|
||||||
|
nodes.emplace_back(identifier);
|
||||||
|
}
|
||||||
|
|
||||||
/// Replace *, alias.*, database.table.* with a list of columns.
|
/// Replace *, alias.*, database.table.* with a list of columns.
|
||||||
void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPtr &, Data & data)
|
void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPtr &, Data & data)
|
||||||
{
|
{
|
||||||
@ -177,20 +204,21 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
|
|||||||
|
|
||||||
for (const auto & child : old_children)
|
for (const auto & child : old_children)
|
||||||
{
|
{
|
||||||
if (typeid_cast<const ASTAsterisk *>(child.get()))
|
if (const auto * asterisk = typeid_cast<const ASTAsterisk *>(child.get()))
|
||||||
{
|
{
|
||||||
bool first_table = true;
|
bool first_table = true;
|
||||||
for (const auto & [table_name, table_columns] : tables_with_columns)
|
for (const auto & [table_name, table_columns] : tables_with_columns)
|
||||||
{
|
{
|
||||||
for (const auto & column_name : table_columns)
|
for (const auto & column_name : table_columns)
|
||||||
|
{
|
||||||
if (first_table || !data.join_using_columns.count(column_name))
|
if (first_table || !data.join_using_columns.count(column_name))
|
||||||
{
|
{
|
||||||
/// qualifed names for duplicates
|
bool need_prefix = !first_table && source_columns.count(column_name);
|
||||||
if (!first_table && source_columns.count(column_name))
|
String long_name = table_name.getQualifiedNamePrefix() + column_name;
|
||||||
node.children.emplace_back(std::make_shared<ASTIdentifier>(table_name.getQualifiedNamePrefix() + column_name));
|
auto identifier = makeIdentifier(column_name, long_name, need_prefix);
|
||||||
else
|
addIdentifier(node.children, identifier, long_name, AsteriskSemantic::getAliases(*asterisk));
|
||||||
node.children.emplace_back(std::make_shared<ASTIdentifier>(column_name));
|
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
first_table = false;
|
first_table = false;
|
||||||
}
|
}
|
||||||
@ -206,11 +234,10 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
|
|||||||
{
|
{
|
||||||
for (const auto & column_name : table_columns)
|
for (const auto & column_name : table_columns)
|
||||||
{
|
{
|
||||||
/// qualifed names for duplicates
|
bool need_prefix = !first_table && source_columns.count(column_name);
|
||||||
if (!first_table && source_columns.count(column_name))
|
String long_name = table_name.getQualifiedNamePrefix() + column_name;
|
||||||
node.children.emplace_back(std::make_shared<ASTIdentifier>(table_name.getQualifiedNamePrefix() + column_name));
|
auto identifier = makeIdentifier(column_name, long_name, need_prefix);
|
||||||
else
|
addIdentifier(node.children, identifier, long_name, AsteriskSemantic::getAliases(*qualified_asterisk));
|
||||||
node.children.emplace_back(std::make_shared<ASTIdentifier>(column_name));
|
|
||||||
}
|
}
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
@ -6,6 +6,9 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
struct AsteriskSemantic;
|
||||||
|
struct AsteriskSemanticImpl;
|
||||||
|
|
||||||
class ASTAsterisk : public IAST
|
class ASTAsterisk : public IAST
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
@ -15,6 +18,11 @@ public:
|
|||||||
|
|
||||||
protected:
|
protected:
|
||||||
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
|
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
|
||||||
|
|
||||||
|
private:
|
||||||
|
std::shared_ptr<AsteriskSemanticImpl> semantic; /// pimpl
|
||||||
|
|
||||||
|
friend struct AsteriskSemantic;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -6,6 +6,9 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
struct AsteriskSemantic;
|
||||||
|
struct AsteriskSemanticImpl;
|
||||||
|
|
||||||
/** Something like t.*
|
/** Something like t.*
|
||||||
* It will have qualifier as its child ASTIdentifier.
|
* It will have qualifier as its child ASTIdentifier.
|
||||||
*/
|
*/
|
||||||
@ -23,6 +26,11 @@ public:
|
|||||||
|
|
||||||
protected:
|
protected:
|
||||||
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||||
|
|
||||||
|
private:
|
||||||
|
std::shared_ptr<AsteriskSemanticImpl> semantic; /// pimpl
|
||||||
|
|
||||||
|
friend struct AsteriskSemantic;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user