Remove joined table qualifiers from column names. [#CLICKHOUSE-3761]

This commit is contained in:
Nikolai Kochetov 2018-07-26 18:11:53 +03:00
parent 65dcca22a2
commit 5123e0f86f
2 changed files with 150 additions and 68 deletions

View File

@ -163,6 +163,34 @@ void removeDuplicateColumns(NamesAndTypesList & columns)
}
String DatabaseAndTableWithAlias::getQualifiedNamePrefix() const
{
return (!alias.empty() ? alias : (database + '.' + table)) + '.';
}
void DatabaseAndTableWithAlias::makeQualifiedName(const ASTPtr & ast) const
{
if (auto identifier = typeid_cast<ASTIdentifier *>(ast.get()))
{
String prefix = getQualifiedNamePrefix();
identifier->name.insert(identifier->name.begin(), prefix.begin(), prefix.end());
Names qualifiers;
if (!alias.empty())
qualifiers.push_back(alias);
else
{
qualifiers.push_back(database);
qualifiers.push_back(table);
}
for (const auto & qualifier : qualifiers)
identifier->children.emplace_back(std::make_shared<ASTIdentifier>(qualifier));
}
}
ExpressionAnalyzer::ExpressionAnalyzer(
const ASTPtr & ast_,
const Context & context_,
@ -276,16 +304,16 @@ static DatabaseAndTableWithAlias getTableNameWithAliasFromTableExpression(const
if (table_expression.database_and_table_name->children.empty())
{
database_and_table_with_alias.database_name = context.getCurrentDatabase();
database_and_table_with_alias.table_name = identifier.name;
database_and_table_with_alias.database = context.getCurrentDatabase();
database_and_table_with_alias.table = identifier.name;
}
else
{
if (table_expression.database_and_table_name->children.size() != 2)
throw Exception("Logical error: number of components in table expression not equal to two", ErrorCodes::LOGICAL_ERROR);
database_and_table_with_alias.database_name = static_cast<const ASTIdentifier &>(*identifier.children[0]).name;
database_and_table_with_alias.table_name = static_cast<const ASTIdentifier &>(*identifier.children[1]).name;
database_and_table_with_alias.database = static_cast<const ASTIdentifier &>(*identifier.children[0]).name;
database_and_table_with_alias.table = static_cast<const ASTIdentifier &>(*identifier.children[1]).name;
}
}
else if (table_expression.table_function)
@ -313,11 +341,18 @@ void ExpressionAnalyzer::translateQualifiedNames()
if (!element.table_expression) /// This is ARRAY JOIN without a table at the left side.
return;
ASTTableExpression & table_expression = static_cast<ASTTableExpression &>(*element.table_expression);
auto & table_expression = static_cast<ASTTableExpression &>(*element.table_expression);
auto * join = select_query->join();
auto database_and_table_with_alias = getTableNameWithAliasFromTableExpression(table_expression, context);
std::vector<DatabaseAndTableWithAlias> tables = {getTableNameWithAliasFromTableExpression(table_expression, context)};
translateQualifiedNamesImpl(ast, database_and_table_with_alias);
if (join)
{
const auto & join_table_expression = static_cast<const ASTTableExpression &>(*join->table_expression);
tables.emplace_back(getTableNameWithAliasFromTableExpression(join_table_expression, context));
}
translateQualifiedNamesImpl(ast, tables);
}
@ -335,16 +370,16 @@ static size_t getNumComponentsToStripInOrderToTranslateQualifiedName(const ASTId
/// database.table.column
if (num_components >= 3
&& !names.database_name.empty()
&& get_identifier_name(identifier.children[0]) == names.database_name
&& get_identifier_name(identifier.children[1]) == names.table_name)
&& !names.database.empty()
&& get_identifier_name(identifier.children[0]) == names.database
&& get_identifier_name(identifier.children[1]) == names.table)
{
num_qualifiers_to_strip = 2;
}
/// table.column or alias.column. If num_components > 2, it is like table.nested.column.
if (num_components >= 2
&& ((!names.table_name.empty() && get_identifier_name(identifier.children[0]) == names.table_name)
&& ((!names.table.empty() && get_identifier_name(identifier.children[0]) == names.table)
|| (!names.alias.empty() && get_identifier_name(identifier.children[0]) == names.alias)))
{
num_qualifiers_to_strip = 1;
@ -391,16 +426,36 @@ static void stripIdentifier(ASTPtr & ast, size_t num_qualifiers_to_strip)
}
void ExpressionAnalyzer::translateQualifiedNamesImpl(ASTPtr & ast, const DatabaseAndTableWithAlias & table_names)
void ExpressionAnalyzer::translateQualifiedNamesImpl(ASTPtr & ast, const std::vector<DatabaseAndTableWithAlias> & tables)
{
if (auto * identifier = typeid_cast<ASTIdentifier *>(ast.get()))
{
if (identifier->kind == ASTIdentifier::Column)
{
size_t num_qualifiers_to_strip
= getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, table_names);
/// Select first table name with max number of qualifiers which can be stripped.
size_t max_num_qualifiers_to_strip = 0;
size_t best_table_pos = 0;
stripIdentifier(ast, num_qualifiers_to_strip);
for (size_t table_pos = 0; table_pos < tables.size(); ++table_pos)
{
const auto & table = tables[table_pos];
auto num_qualifiers_to_strip = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, table);
if (num_qualifiers_to_strip > max_num_qualifiers_to_strip)
{
max_num_qualifiers_to_strip = num_qualifiers_to_strip;
best_table_pos = table_pos;
}
}
stripIdentifier(ast, max_num_qualifiers_to_strip);
/// In case if column from the joined table are in source columns, change it's name to qualified.
if (best_table_pos && source_columns.contains(ast->getColumnName()))
{
duplicate_columns_from_joined_table.insert(ast->getColumnName());
tables[best_table_pos].makeQualifiedName(ast);
}
}
}
else if (typeid_cast<ASTQualifiedAsterisk *>(ast.get()))
@ -416,19 +471,28 @@ void ExpressionAnalyzer::translateQualifiedNamesImpl(ASTPtr & ast, const Databas
if (num_components > 2)
throw Exception("Qualified asterisk cannot have more than two qualifiers", ErrorCodes::UNKNOWN_ELEMENT_IN_AST);
/// database.table.*, table.* or alias.*
if ( (num_components == 2
&& !table_names.database_name.empty()
&& static_cast<const ASTIdentifier &>(*ident->children[0]).name == table_names.database_name
&& static_cast<const ASTIdentifier &>(*ident->children[1]).name == table_names.table_name)
|| (num_components == 0
&& ((!table_names.table_name.empty() && ident->name == table_names.table_name)
|| (!table_names.alias.empty() && ident->name == table_names.alias))))
for (const auto & table_names : tables)
{
/// Replace to plain asterisk.
ast = std::make_shared<ASTAsterisk>();
/// database.table.*, table.* or alias.*
if ((num_components == 2
&& !table_names.database.empty()
&& static_cast<const ASTIdentifier &>(*ident->children[0]).name == table_names.database
&& static_cast<const ASTIdentifier &>(*ident->children[1]).name == table_names.table)
|| (num_components == 0
&& ((!table_names.table.empty() && ident->name == table_names.table)
|| (!table_names.alias.empty() && ident->name == table_names.alias))))
{
/// Replace to plain asterisk.
ast = std::make_shared<ASTAsterisk>();
}
}
}
else if (auto * join = typeid_cast<ASTTableJoin *>(ast.get()))
{
/// Don't translate on_expression hear in order to resolve equasions parts later.
if (join->using_expression_list)
translateQualifiedNamesImpl(join->using_expression_list, tables);
}
else
{
for (auto & child : ast->children)
@ -437,7 +501,7 @@ void ExpressionAnalyzer::translateQualifiedNamesImpl(ASTPtr & ast, const Databas
if (!typeid_cast<const ASTTableExpression *>(child.get())
&& !typeid_cast<const ASTSelectWithUnionQuery *>(child.get()))
{
translateQualifiedNamesImpl(child, table_names);
translateQualifiedNamesImpl(child, tables);
}
}
}
@ -2057,28 +2121,8 @@ void ExpressionAnalyzer::getActionsFromJoinKeys(const ASTTableJoin & table_join,
getActionsImpl(table_join.using_expression_list, no_subqueries, only_consts, scopes, projection_manipulator);
else if (table_join.on_expression)
{
std::function<void(const ASTPtr &)> get_actions;
get_actions = [&](const ASTPtr & ast)
{
bool key_expr = false;
if (const auto * ast_function = typeid_cast<const ASTFunction *>(ast.get()))
{
auto column_name = ast_function->getColumnName();
key_expr = join_key_names_left.end()
!= std::find(join_key_names_left.begin(), join_key_names_left.end(), column_name);
}
if (key_expr)
getActionsImpl(ast, no_subqueries, only_consts, scopes, projection_manipulator);
else
{
for (auto & child : ast->children)
get_actions(child);
}
};
get_actions(table_join.on_expression);
for (const auto & ast : join_key_asts_left)
getActionsImpl(ast, no_subqueries, only_consts, scopes, projection_manipulator);
}
actions = scopes.popLevel();
@ -2531,9 +2575,10 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
initChain(chain, source_columns);
ExpressionActionsChain::Step & step = chain.steps.back();
const ASTTablesInSelectQueryElement & join_element = static_cast<const ASTTablesInSelectQueryElement &>(*select_query->join());
const ASTTableJoin & join_params = static_cast<const ASTTableJoin &>(*join_element.table_join);
const ASTTableExpression & table_to_join = static_cast<const ASTTableExpression &>(*join_element.table_expression);
const auto & join_element = static_cast<const ASTTablesInSelectQueryElement &>(*select_query->join());
const auto & join_params = static_cast<const ASTTableJoin &>(*join_element.table_join);
const auto & table_to_join = static_cast<const ASTTableExpression &>(*join_element.table_expression);
const auto joined_table_name = getTableNameWithAliasFromTableExpression(table_to_join, context);
getActionsFromJoinKeys(join_params, only_types, false, step.actions);
@ -2546,8 +2591,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
/// TODO This syntax does not support specifying a database name.
if (table_to_join.database_and_table_name)
{
auto database_table = getDatabaseAndTableNameFromIdentifier(static_cast<const ASTIdentifier &>(*table_to_join.database_and_table_name));
StoragePtr table = context.tryGetTable(database_table.first, database_table.second);
StoragePtr table = context.tryGetTable(joined_table_name.database, joined_table_name.table);
if (table)
{
@ -2584,6 +2628,16 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
{
ASTPtr select_expression_list;
auto add_aliases_to_duplicating_joined_columns = [&](const ASTPtr & expression_list)
{
for (const auto & name : duplicate_columns_from_joined_table)
{
auto identifier = std::make_shared<ASTIdentifier>(name);
identifier->setAlias(joined_table_name.getQualifiedNamePrefix() + name);
expression_list->children.emplace_back(std::move(identifier));
}
};
ASTPtr table;
if (table_to_join.database_and_table_name)
{
@ -2593,28 +2647,37 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
select_expression_list = std::make_shared<ASTExpressionList>();
for (const auto & join_right_key : join_key_asts_right)
select_expression_list->children.emplace_back(join_right_key);
for (const auto & column : columns_added_by_join)
if (join_key_names_right.end() == std::find(join_key_names_right.begin(),
join_key_names_right.end(), column.name))
select_expression_list->children.emplace_back(std::make_shared<ASTIdentifier>(column.name));
add_aliases_to_duplicating_joined_columns(select_expression_list);
}
else
{
table = table_to_join.subquery->clone();
ASTSubquery * subquery = typeid_cast<ASTSubquery *>(table.get());
auto * subquery = typeid_cast<ASTSubquery *>(table.get());
if (!subquery)
throw Exception("Expected ASTSubquery as joined subquery.", ErrorCodes::LOGICAL_ERROR);
const auto & query_ptr = subquery->children.at(0);
ASTSelectWithUnionQuery * select_with_union = typeid_cast<ASTSelectWithUnionQuery *>(query_ptr.get());
auto * select_with_union = typeid_cast<ASTSelectWithUnionQuery *>(query_ptr.get());
if (!select_with_union)
throw Exception("Expected ASTSelectWithUnionQuery as joined subquery.", ErrorCodes::LOGICAL_ERROR);
const auto & first_select = select_with_union->list_of_selects->children.at(0);
ASTSelectQuery * select_query = typeid_cast<ASTSelectQuery *>(first_select.get());
auto * select_query = typeid_cast<ASTSelectQuery *>(first_select.get());
if (!select_query)
throw Exception("Expected ASTSelectQuery as joined subquery.", ErrorCodes::LOGICAL_ERROR);
/// Append right join keys to subquery expression list. Duplicate expressions will be removed further.
for (auto & expr : join_key_asts_right)
select_query->select_expression_list->children.emplace_back(expr);
add_aliases_to_duplicating_joined_columns(select_query->select_expression_list);
}
auto interpreter = interpretSubquery(table, context, subquery_depth, required_joined_columns, select_expression_list);
@ -3141,14 +3204,14 @@ void ExpressionAnalyzer::collectJoinedColumns(NameSet & joined_columns, NamesAnd
if (!node)
return;
const ASTTableJoin & table_join = static_cast<const ASTTableJoin &>(*node->table_join);
const ASTTableExpression & table_expression = static_cast<const ASTTableExpression &>(*node->table_expression);
const auto & table_join = static_cast<const ASTTableJoin &>(*node->table_join);
const auto & table_expression = static_cast<const ASTTableExpression &>(*node->table_expression);
auto joined_table_name = getTableNameWithAliasFromTableExpression(table_expression, context);
Block nested_result_sample;
if (table_expression.database_and_table_name)
{
auto database_table = getDatabaseAndTableNameFromIdentifier(static_cast<const ASTIdentifier &>(*table_expression.database_and_table_name));
const auto & table = context.getTable(database_table.first, database_table.second);
const auto & table = context.getTable(joined_table_name.database, joined_table_name.table);
nested_result_sample = table->getSampleBlockNonMaterialized();
}
else if (table_expression.subquery)
@ -3180,16 +3243,28 @@ void ExpressionAnalyzer::collectJoinedColumns(NameSet & joined_columns, NamesAnd
else if (table_join.on_expression)
collectJoinedColumnsFromJoinOnExpr();
for (const auto i : ext::range(0, nested_result_sample.columns()))
{
const auto & col = nested_result_sample.safeGetByPosition(i);
if (join_key_names_left.end() == std::find(join_key_names_left.begin(), join_key_names_left.end(), col.name)
&& !joined_columns.count(col.name)) /// Duplicate columns in the subquery for JOIN do not make sense.
if (join_key_names_left.end() == std::find(join_key_names_left.begin(), join_key_names_left.end(), col.name))
{
joined_columns.insert(col.name);
auto name = col.name;
/// Change name for duplicate column form joined table.
if (source_columns.contains(name))
{
duplicate_columns_from_joined_table.insert(name);
name = joined_table_name.getQualifiedNamePrefix() + name;
}
bool make_nullable = settings.join_use_nulls && (table_join.kind == ASTTableJoin::Kind::Left || table_join.kind == ASTTableJoin::Kind::Full);
joined_columns_name_type.emplace_back(col.name, make_nullable ? makeNullable(col.type) : col.type);
if (joined_columns.count(name)) /// Duplicate columns in the subquery for JOIN do not make sense.
continue;
joined_columns.insert(name);
bool make_nullable = settings.join_use_nulls && (table_join.kind == ASTTableJoin::Kind::Left ||
table_join.kind == ASTTableJoin::Kind::Full);
joined_columns_name_type.emplace_back(name, make_nullable ? makeNullable(col.type) : col.type);
}
}
}

View File

@ -89,9 +89,15 @@ struct ScopeStack
struct DatabaseAndTableWithAlias
{
String database_name;
String table_name;
String database;
String table;
String alias;
/// "alias." or "database.table." if alias is empty
String getQualifiedNamePrefix() const;
/// If ast is ASTIdentifier, prepend getQualifiedNamePrefix() to it's name.
void makeQualifiedName(const ASTPtr & ast) const;
};
/** Transforms an expression from a syntax tree into a sequence of actions to execute it.
@ -228,6 +234,7 @@ private:
Names join_key_names_right;
ASTs join_key_asts_left;
ASTs join_key_asts_right;
NameSet duplicate_columns_from_joined_table;
NamesAndTypesList columns_added_by_join;
@ -372,7 +379,7 @@ private:
* only one ("main") table is supported. Ambiguity is not detected or resolved.
*/
void translateQualifiedNames();
void translateQualifiedNamesImpl(ASTPtr & node, const DatabaseAndTableWithAlias & table_names);
void translateQualifiedNamesImpl(ASTPtr & node, const std::vector<DatabaseAndTableWithAlias> & tables);
/** Sometimes we have to calculate more columns in SELECT clause than will be returned from query.
* This is the case when we have DISTINCT or arrayJoin: we require more columns in SELECT even if we need less columns in result.