mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #7358 from 4ertus2/ast
Better ambiguous column detection
This commit is contained in:
commit
df32d92ee5
@ -164,7 +164,8 @@ size_t CollectJoinOnKeysMatcher::getTableForIdentifiers(std::vector<const ASTIde
|
||||
|
||||
/// Column name could be cropped to a short form in TranslateQualifiedNamesVisitor.
|
||||
/// In this case it saves membership in IdentifierSemantic.
|
||||
size_t membership = IdentifierSemantic::getMembership(*identifier);
|
||||
auto opt = IdentifierSemantic::getMembership(*identifier);
|
||||
size_t membership = opt ? (*opt + 1) : 0;
|
||||
|
||||
if (!membership)
|
||||
{
|
||||
|
@ -90,9 +90,12 @@ public:
|
||||
using TypeToVisit = const ASTFunction;
|
||||
|
||||
CheckExpressionVisitorData(const std::vector<JoinedTable> & tables_)
|
||||
: tables(tables_)
|
||||
: joined_tables(tables_)
|
||||
, ands_only(true)
|
||||
{}
|
||||
{
|
||||
for (auto & joined : joined_tables)
|
||||
tables.push_back(joined.table);
|
||||
}
|
||||
|
||||
void visit(const ASTFunction & node, const ASTPtr & ast)
|
||||
{
|
||||
@ -156,7 +159,8 @@ public:
|
||||
}
|
||||
|
||||
private:
|
||||
const std::vector<JoinedTable> & tables;
|
||||
const std::vector<JoinedTable> & joined_tables;
|
||||
std::vector<DatabaseAndTableWithAlias> tables;
|
||||
std::map<size_t, std::vector<ASTPtr>> asts_to_join_on;
|
||||
bool ands_only;
|
||||
|
||||
@ -180,31 +184,16 @@ private:
|
||||
/// @return table position to attach expression to or 0.
|
||||
size_t checkIdentifiers(const ASTIdentifier & left, const ASTIdentifier & right)
|
||||
{
|
||||
/// {best_match, best_table_pos}
|
||||
std::pair<size_t, size_t> left_best{0, 0};
|
||||
std::pair<size_t, size_t> right_best{0, 0};
|
||||
size_t left_table_pos = 0;
|
||||
bool left_match = IdentifierSemantic::chooseTable(left, tables, left_table_pos);
|
||||
|
||||
for (size_t i = 0; i < tables.size(); ++i)
|
||||
size_t right_table_pos = 0;
|
||||
bool right_match = IdentifierSemantic::chooseTable(right, tables, right_table_pos);
|
||||
|
||||
if (left_match && right_match && (left_table_pos != right_table_pos))
|
||||
{
|
||||
size_t match = IdentifierSemantic::canReferColumnToTable(left, tables[i].table);
|
||||
if (match > left_best.first)
|
||||
{
|
||||
left_best.first = match;
|
||||
left_best.second = i;
|
||||
}
|
||||
|
||||
match = IdentifierSemantic::canReferColumnToTable(right, tables[i].table);
|
||||
if (match > right_best.first)
|
||||
{
|
||||
right_best.first = match;
|
||||
right_best.second = i;
|
||||
}
|
||||
}
|
||||
|
||||
if (left_best.first && right_best.first && (left_best.second != right_best.second))
|
||||
{
|
||||
size_t table_pos = std::max(left_best.second, right_best.second);
|
||||
if (tables[table_pos].canAttachOnExpression())
|
||||
size_t table_pos = std::max(left_table_pos, right_table_pos);
|
||||
if (joined_tables[table_pos].canAttachOnExpression())
|
||||
return table_pos;
|
||||
}
|
||||
return 0;
|
||||
@ -212,20 +201,10 @@ private:
|
||||
|
||||
size_t checkIdentifier(const ASTIdentifier & identifier)
|
||||
{
|
||||
size_t best_match = 0;
|
||||
size_t best_table_pos = 0;
|
||||
bool match = IdentifierSemantic::chooseTable(identifier, tables, best_table_pos);
|
||||
|
||||
for (size_t i = 0; i < tables.size(); ++i)
|
||||
{
|
||||
size_t match = IdentifierSemantic::canReferColumnToTable(identifier, tables[i].table);
|
||||
if (match > best_match)
|
||||
{
|
||||
best_match = match;
|
||||
best_table_pos = i;
|
||||
}
|
||||
}
|
||||
|
||||
if (best_match && tables[best_table_pos].canAttachOnExpression())
|
||||
if (match && joined_tables[best_table_pos].canAttachOnExpression())
|
||||
return best_table_pos;
|
||||
return 0;
|
||||
}
|
||||
|
@ -28,17 +28,9 @@ void FindIdentifierBestTableData::visit(ASTIdentifier & identifier, ASTPtr &)
|
||||
}
|
||||
else
|
||||
{
|
||||
// FIXME: make a better matcher using `names`?
|
||||
size_t best_match = 0;
|
||||
for (const auto & table_names : tables)
|
||||
{
|
||||
if (size_t match = IdentifierSemantic::canReferColumnToTable(identifier, table_names.first))
|
||||
if (match > best_match)
|
||||
{
|
||||
best_match = match;
|
||||
best_table = &table_names.first;
|
||||
}
|
||||
}
|
||||
size_t best_table_pos = 0;
|
||||
if (IdentifierSemantic::chooseTable(identifier, tables, best_table_pos))
|
||||
best_table = &tables[best_table_pos].first;
|
||||
}
|
||||
|
||||
identifier_table.emplace_back(&identifier, best_table);
|
||||
|
@ -5,6 +5,61 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int AMBIGUOUS_COLUMN_NAME;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
const DatabaseAndTableWithAlias & extractTable(const DatabaseAndTableWithAlias & table)
|
||||
{
|
||||
return table;
|
||||
}
|
||||
|
||||
const DatabaseAndTableWithAlias & extractTable(const TableWithColumnNames & table)
|
||||
{
|
||||
return table.first;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
IdentifierSemantic::ColumnMatch tryChooseTable(const ASTIdentifier & identifier, const std::vector<T> & tables,
|
||||
size_t & best_table_pos, bool allow_ambiguous)
|
||||
{
|
||||
using ColumnMatch = IdentifierSemantic::ColumnMatch;
|
||||
|
||||
best_table_pos = 0;
|
||||
auto best_match = ColumnMatch::NoMatch;
|
||||
size_t same_match = 0;
|
||||
|
||||
for (size_t i = 0; i < tables.size(); ++i)
|
||||
{
|
||||
auto match = IdentifierSemantic::canReferColumnToTable(identifier, extractTable(tables[i]));
|
||||
if (match != ColumnMatch::NoMatch)
|
||||
{
|
||||
if (match > best_match)
|
||||
{
|
||||
best_match = match;
|
||||
best_table_pos = i;
|
||||
same_match = 0;
|
||||
}
|
||||
else if (match == best_match)
|
||||
++same_match;
|
||||
}
|
||||
}
|
||||
|
||||
if ((best_match != ColumnMatch::NoMatch) && same_match)
|
||||
{
|
||||
if (!allow_ambiguous)
|
||||
throw Exception("Ambiguous column '" + identifier.name + "'", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
|
||||
return ColumnMatch::Ambiguous;
|
||||
}
|
||||
return best_match;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
std::optional<String> IdentifierSemantic::getColumnName(const ASTIdentifier & node)
|
||||
{
|
||||
if (!node.semantic->special)
|
||||
@ -37,26 +92,36 @@ std::optional<String> IdentifierSemantic::getTableName(const ASTPtr & ast)
|
||||
return {};
|
||||
}
|
||||
|
||||
void IdentifierSemantic::setNeedLongName(ASTIdentifier & identifier, bool value)
|
||||
{
|
||||
identifier.semantic->need_long_name = value;
|
||||
}
|
||||
|
||||
bool IdentifierSemantic::canBeAlias(const ASTIdentifier & identifier)
|
||||
{
|
||||
return identifier.semantic->can_be_alias;
|
||||
}
|
||||
|
||||
void IdentifierSemantic::setMembership(ASTIdentifier & identifier, size_t table_no)
|
||||
void IdentifierSemantic::setMembership(ASTIdentifier & identifier, size_t table_pos)
|
||||
{
|
||||
identifier.semantic->membership = table_no;
|
||||
identifier.semantic->membership = table_pos;
|
||||
identifier.semantic->can_be_alias = false;
|
||||
}
|
||||
|
||||
size_t IdentifierSemantic::getMembership(const ASTIdentifier & identifier)
|
||||
std::optional<size_t> IdentifierSemantic::getMembership(const ASTIdentifier & identifier)
|
||||
{
|
||||
return identifier.semantic->membership;
|
||||
}
|
||||
|
||||
bool IdentifierSemantic::chooseTable(const ASTIdentifier & identifier, const std::vector<DatabaseAndTableWithAlias> & tables,
|
||||
size_t & best_table_pos, bool ambiguous)
|
||||
{
|
||||
static constexpr auto no_match = IdentifierSemantic::ColumnMatch::NoMatch;
|
||||
return tryChooseTable<DatabaseAndTableWithAlias>(identifier, tables, best_table_pos, ambiguous) != no_match;
|
||||
}
|
||||
|
||||
bool IdentifierSemantic::chooseTable(const ASTIdentifier & identifier, const std::vector<TableWithColumnNames> & tables,
|
||||
size_t & best_table_pos, bool ambiguous)
|
||||
{
|
||||
static constexpr auto no_match = IdentifierSemantic::ColumnMatch::NoMatch;
|
||||
return tryChooseTable<TableWithColumnNames>(identifier, tables, best_table_pos, ambiguous) != no_match;
|
||||
}
|
||||
|
||||
std::pair<String, String> IdentifierSemantic::extractDatabaseAndTable(const ASTIdentifier & identifier)
|
||||
{
|
||||
if (identifier.name_parts.size() > 2)
|
||||
@ -84,24 +149,49 @@ bool IdentifierSemantic::doesIdentifierBelongTo(const ASTIdentifier & identifier
|
||||
return false;
|
||||
}
|
||||
|
||||
size_t IdentifierSemantic::canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
|
||||
IdentifierSemantic::ColumnMatch IdentifierSemantic::canReferColumnToTable(const ASTIdentifier & identifier,
|
||||
const DatabaseAndTableWithAlias & db_and_table)
|
||||
{
|
||||
/// database.table.column
|
||||
if (doesIdentifierBelongTo(identifier, db_and_table.database, db_and_table.table))
|
||||
return 2;
|
||||
return ColumnMatch::DbAndTable;
|
||||
|
||||
/// table.column or alias.column.
|
||||
if (doesIdentifierBelongTo(identifier, db_and_table.table) ||
|
||||
doesIdentifierBelongTo(identifier, db_and_table.alias))
|
||||
return 1;
|
||||
/// alias.column
|
||||
if (doesIdentifierBelongTo(identifier, db_and_table.alias))
|
||||
return ColumnMatch::TableAlias;
|
||||
|
||||
return 0;
|
||||
/// table.column
|
||||
if (doesIdentifierBelongTo(identifier, db_and_table.table))
|
||||
{
|
||||
if (!db_and_table.alias.empty())
|
||||
return ColumnMatch::AliasedTableName;
|
||||
else
|
||||
return ColumnMatch::TableName;
|
||||
}
|
||||
|
||||
return ColumnMatch::NoMatch;
|
||||
}
|
||||
|
||||
/// Checks that ast is ASTIdentifier and remove num_qualifiers_to_strip components from left.
|
||||
/// Example: 'database.table.name' -> (num_qualifiers_to_strip = 2) -> 'name'.
|
||||
void IdentifierSemantic::setColumnShortName(ASTIdentifier & identifier, size_t to_strip)
|
||||
/// Strip qualificators from left side of column name.
|
||||
/// Example: 'database.table.name' -> 'name'.
|
||||
void IdentifierSemantic::setColumnShortName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
|
||||
{
|
||||
auto match = IdentifierSemantic::canReferColumnToTable(identifier, db_and_table);
|
||||
size_t to_strip = 0;
|
||||
switch (match)
|
||||
{
|
||||
case ColumnMatch::TableName:
|
||||
case ColumnMatch::AliasedTableName:
|
||||
case ColumnMatch::TableAlias:
|
||||
to_strip = 1;
|
||||
break;
|
||||
case ColumnMatch::DbAndTable:
|
||||
to_strip = 2;
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
|
||||
if (!to_strip)
|
||||
return;
|
||||
|
||||
@ -117,18 +207,6 @@ void IdentifierSemantic::setColumnShortName(ASTIdentifier & identifier, size_t t
|
||||
identifier.name.swap(new_name);
|
||||
}
|
||||
|
||||
void IdentifierSemantic::setColumnNormalName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
|
||||
{
|
||||
size_t match = IdentifierSemantic::canReferColumnToTable(identifier, db_and_table);
|
||||
|
||||
setColumnShortName(identifier, match);
|
||||
if (match)
|
||||
identifier.semantic->can_be_alias = false;
|
||||
|
||||
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();
|
||||
@ -141,16 +219,4 @@ void IdentifierSemantic::setColumnLongName(ASTIdentifier & identifier, const Dat
|
||||
}
|
||||
}
|
||||
|
||||
String IdentifierSemantic::columnNormalName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
|
||||
{
|
||||
ASTPtr copy = identifier.clone();
|
||||
setColumnNormalName(copy->as<ASTIdentifier &>(), db_and_table);
|
||||
return copy->getAliasOrColumnName();
|
||||
}
|
||||
|
||||
String IdentifierSemantic::columnLongName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
|
||||
{
|
||||
return db_and_table.getQualifiedNamePrefix() + identifier.shortName();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <optional>
|
||||
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
|
||||
@ -9,14 +11,23 @@ namespace DB
|
||||
struct IdentifierSemanticImpl
|
||||
{
|
||||
bool special = false; /// for now it's 'not a column': tables, subselects and some special stuff like FORMAT
|
||||
bool need_long_name = false;/// if column presents in multiple tables we need qualified names
|
||||
bool can_be_alias = true; /// if it's a cropped name it could not be an alias
|
||||
size_t membership = 0; /// table position in join (starting from 1) detected by qualifier or 0 if not detected.
|
||||
std::optional<size_t> membership; /// table position in join
|
||||
};
|
||||
|
||||
/// Static calss to manipulate IdentifierSemanticImpl via ASTIdentifier
|
||||
struct IdentifierSemantic
|
||||
{
|
||||
enum class ColumnMatch
|
||||
{
|
||||
NoMatch,
|
||||
AliasedTableName, /// column qualified with table name (but table has an alias so its priority is lower than TableName)
|
||||
TableName, /// column qualified with table name
|
||||
DbAndTable, /// column qualified with database and table name
|
||||
TableAlias, /// column qualified with table alias
|
||||
Ambiguous,
|
||||
};
|
||||
|
||||
/// @returns name for column identifiers
|
||||
static std::optional<String> getColumnName(const ASTIdentifier & node);
|
||||
static std::optional<String> getColumnName(const ASTPtr & ast);
|
||||
@ -26,20 +37,20 @@ struct IdentifierSemantic
|
||||
static std::optional<String> getTableName(const ASTPtr & ast);
|
||||
static std::pair<String, String> extractDatabaseAndTable(const ASTIdentifier & identifier);
|
||||
|
||||
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 columnLongName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
|
||||
static void setColumnNormalName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
|
||||
static ColumnMatch canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
|
||||
static void setColumnShortName(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 bool canBeAlias(const ASTIdentifier & identifier);
|
||||
static void setMembership(ASTIdentifier & identifier, size_t table_no);
|
||||
static size_t getMembership(const ASTIdentifier & identifier);
|
||||
static void setMembership(ASTIdentifier &, size_t table_no);
|
||||
static std::optional<size_t> getMembership(const ASTIdentifier & identifier);
|
||||
static bool chooseTable(const ASTIdentifier &, const std::vector<DatabaseAndTableWithAlias> & tables, size_t & best_table_pos,
|
||||
bool ambiguous = false);
|
||||
static bool chooseTable(const ASTIdentifier &, const std::vector<TableWithColumnNames> & tables, size_t & best_table_pos,
|
||||
bool ambiguous = false);
|
||||
|
||||
private:
|
||||
static bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & database, const String & table);
|
||||
static bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & table);
|
||||
static void setColumnShortName(ASTIdentifier & identifier, size_t match);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -147,7 +147,13 @@ struct ColumnAliasesMatcher
|
||||
auto it = rev_aliases.find(long_name);
|
||||
if (it == rev_aliases.end())
|
||||
{
|
||||
bool last_table = IdentifierSemantic::canReferColumnToTable(*identifier, tables.back());
|
||||
bool last_table = false;
|
||||
{
|
||||
size_t best_table_pos = 0;
|
||||
if (IdentifierSemantic::chooseTable(*identifier, tables, best_table_pos))
|
||||
last_table = (best_table_pos + 1 == tables.size());
|
||||
}
|
||||
|
||||
if (!last_table)
|
||||
{
|
||||
String alias = hide_prefix + long_name;
|
||||
@ -202,17 +208,15 @@ struct ColumnAliasesMatcher
|
||||
|
||||
bool last_table = false;
|
||||
String long_name;
|
||||
for (auto & table : data.tables)
|
||||
|
||||
size_t table_pos = 0;
|
||||
if (IdentifierSemantic::chooseTable(node, data.tables, table_pos))
|
||||
{
|
||||
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 == &data.tables.back())
|
||||
last_table = true;
|
||||
}
|
||||
auto & table = data.tables[table_pos];
|
||||
IdentifierSemantic::setColumnLongName(node, table); /// table_name.column_name -> table_alias.column_name
|
||||
long_name = node.name;
|
||||
if (&table == &data.tables.back())
|
||||
last_table = true;
|
||||
}
|
||||
|
||||
if (long_name.empty())
|
||||
|
@ -112,7 +112,10 @@ void collectSourceColumns(const ColumnsDescription & columns, NamesAndTypesList
|
||||
}
|
||||
}
|
||||
|
||||
std::vector<TableWithColumnNames> getTablesWithColumns(const ASTSelectQuery & select_query, const Context & context)
|
||||
std::vector<TableWithColumnNames> getTablesWithColumns(const ASTSelectQuery & select_query, const Context & context,
|
||||
const ASTTablesInSelectQueryElement * table_join_node,
|
||||
NamesAndTypesList & columns_from_joined_table,
|
||||
std::function<Names()> get_column_names)
|
||||
{
|
||||
std::vector<TableWithColumnNames> tables_with_columns = getDatabaseAndTablesWithColumnNames(select_query, context);
|
||||
|
||||
@ -125,6 +128,27 @@ std::vector<TableWithColumnNames> getTablesWithColumns(const ASTSelectQuery & se
|
||||
ErrorCodes::ALIAS_REQUIRED);
|
||||
}
|
||||
|
||||
TableWithColumnNames joined_table;
|
||||
|
||||
if (table_join_node)
|
||||
{
|
||||
const auto & joined_expression = table_join_node->table_expression->as<ASTTableExpression &>();
|
||||
|
||||
columns_from_joined_table = getNamesAndTypeListFromTableExpression(joined_expression, context);
|
||||
|
||||
joined_table.first = DatabaseAndTableWithAlias(joined_expression, context.getCurrentDatabase());
|
||||
for (const auto & column : columns_from_joined_table)
|
||||
joined_table.second.push_back(column.name);
|
||||
}
|
||||
|
||||
/// If empty make table(s) with list of source and joined columns
|
||||
if (tables_with_columns.empty())
|
||||
{
|
||||
tables_with_columns.emplace_back(DatabaseAndTableWithAlias{}, get_column_names());
|
||||
if (!joined_table.second.empty())
|
||||
tables_with_columns.emplace_back(std::move(joined_table));
|
||||
}
|
||||
|
||||
return tables_with_columns;
|
||||
}
|
||||
|
||||
@ -562,11 +586,16 @@ void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery & s
|
||||
}
|
||||
}
|
||||
|
||||
void replaceJoinedTable(const ASTTablesInSelectQueryElement* join)
|
||||
void replaceJoinedTable(const ASTTablesInSelectQueryElement * join)
|
||||
{
|
||||
if (!join || !join->table_expression)
|
||||
return;
|
||||
|
||||
/// TODO: Push down for CROSS JOIN is not OK [disabled]
|
||||
const auto & table_join = join->table_join->as<ASTTableJoin &>();
|
||||
if (table_join.kind == ASTTableJoin::Kind::Cross)
|
||||
return;
|
||||
|
||||
auto & table_expr = join->table_expression->as<ASTTableExpression &>();
|
||||
if (table_expr.database_and_table_name)
|
||||
{
|
||||
@ -826,41 +855,34 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
|
||||
if (remove_duplicates)
|
||||
renameDuplicatedColumns(select_query);
|
||||
|
||||
if (const ASTTablesInSelectQueryElement * node = select_query->join())
|
||||
const ASTTablesInSelectQueryElement * table_join_node = select_query->join();
|
||||
if (table_join_node)
|
||||
{
|
||||
if (!settings.any_join_distinct_right_table_keys)
|
||||
checkJoin(node);
|
||||
checkJoin(table_join_node);
|
||||
|
||||
if (settings.enable_optimize_predicate_expression)
|
||||
replaceJoinedTable(node);
|
||||
|
||||
const auto & joined_expression = node->table_expression->as<ASTTableExpression &>();
|
||||
DatabaseAndTableWithAlias table(joined_expression, context.getCurrentDatabase());
|
||||
|
||||
result.analyzed_join->columns_from_joined_table = getNamesAndTypeListFromTableExpression(joined_expression, context);
|
||||
result.analyzed_join->deduplicateAndQualifyColumnNames(source_columns_set, table.getQualifiedNamePrefix());
|
||||
replaceJoinedTable(table_join_node);
|
||||
}
|
||||
|
||||
auto tables_with_columns = getTablesWithColumns(*select_query, context);
|
||||
|
||||
/// If empty make fake table with list of source and joined columns
|
||||
if (tables_with_columns.empty())
|
||||
auto get_column_names = [&]() -> Names
|
||||
{
|
||||
Names columns_list;
|
||||
if (storage)
|
||||
columns_list = storage->getColumns().getOrdinary().getNames();
|
||||
else
|
||||
{
|
||||
columns_list.reserve(result.source_columns.size());
|
||||
for (const auto & column : result.source_columns)
|
||||
columns_list.emplace_back(column.name);
|
||||
}
|
||||
return storage->getColumns().getOrdinary().getNames();
|
||||
|
||||
for (auto & column : result.analyzed_join->getQualifiedColumnsSet())
|
||||
columns_list.emplace_back(column);
|
||||
Names columns;
|
||||
columns.reserve(result.source_columns.size());
|
||||
for (const auto & column : result.source_columns)
|
||||
columns.push_back(column.name);
|
||||
return columns;
|
||||
};
|
||||
|
||||
tables_with_columns.emplace_back(DatabaseAndTableWithAlias{}, std::move(columns_list));
|
||||
}
|
||||
auto tables_with_columns = getTablesWithColumns(*select_query, context, table_join_node,
|
||||
result.analyzed_join->columns_from_joined_table, get_column_names);
|
||||
|
||||
if (tables_with_columns.size() > 1)
|
||||
result.analyzed_join->deduplicateAndQualifyColumnNames(
|
||||
source_columns_set, tables_with_columns[1].first.getQualifiedNamePrefix());
|
||||
|
||||
translateQualifiedNames(query, *select_query, source_columns_set, std::move(tables_with_columns));
|
||||
|
||||
|
@ -61,24 +61,20 @@ void TranslateQualifiedNamesMatcher::visit(ASTIdentifier & identifier, ASTPtr &,
|
||||
{
|
||||
if (IdentifierSemantic::getColumnName(identifier))
|
||||
{
|
||||
size_t best_table_pos = 0;
|
||||
size_t best_match = 0;
|
||||
for (size_t i = 0; i < data.tables.size(); ++i)
|
||||
if (size_t match = IdentifierSemantic::canReferColumnToTable(identifier, data.tables[i].first))
|
||||
if (match > best_match)
|
||||
{
|
||||
best_match = match;
|
||||
best_table_pos = i;
|
||||
}
|
||||
String short_name = identifier.shortName();
|
||||
size_t table_pos = 0;
|
||||
bool allow_ambiguous = data.join_using_columns.count(short_name);
|
||||
if (IdentifierSemantic::chooseTable(identifier, data.tables, table_pos, allow_ambiguous))
|
||||
{
|
||||
IdentifierSemantic::setMembership(identifier, table_pos);
|
||||
|
||||
if (best_match)
|
||||
IdentifierSemantic::setMembership(identifier, best_table_pos + 1);
|
||||
|
||||
/// In case if column from the joined table are in source columns, change it's name to qualified.
|
||||
if (best_table_pos && data.source_columns.count(identifier.shortName()))
|
||||
IdentifierSemantic::setNeedLongName(identifier, true);
|
||||
if (!data.tables.empty())
|
||||
IdentifierSemantic::setColumnNormalName(identifier, data.tables[best_table_pos].first);
|
||||
/// In case if column from the joined table are in source columns, change it's name to qualified.
|
||||
auto & table = data.tables[table_pos].first;
|
||||
if (table_pos && data.hasColumn(short_name))
|
||||
IdentifierSemantic::setColumnLongName(identifier, table);
|
||||
else
|
||||
IdentifierSemantic::setColumnShortName(identifier, table);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -134,8 +130,10 @@ void TranslateQualifiedNamesMatcher::visit(ASTSelectQuery & select, const ASTPtr
|
||||
Visitor(data).visit(select.refHaving());
|
||||
}
|
||||
|
||||
static void addIdentifier(ASTs & nodes, const String & table_name, const String & column_name, AsteriskSemantic::RevertedAliasesPtr aliases)
|
||||
static void addIdentifier(ASTs & nodes, const DatabaseAndTableWithAlias & table, const String & column_name,
|
||||
AsteriskSemantic::RevertedAliasesPtr aliases)
|
||||
{
|
||||
String table_name = table.getQualifiedNamePrefix(false);
|
||||
auto identifier = std::make_shared<ASTIdentifier>(std::vector<String>{table_name, column_name});
|
||||
|
||||
bool added = false;
|
||||
@ -197,8 +195,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
|
||||
{
|
||||
if (first_table || !data.join_using_columns.count(column_name))
|
||||
{
|
||||
String table_name = table.getQualifiedNamePrefix(false);
|
||||
addIdentifier(node.children, table_name, column_name, AsteriskSemantic::getAliases(*asterisk));
|
||||
addIdentifier(node.children, table, column_name, AsteriskSemantic::getAliases(*asterisk));
|
||||
}
|
||||
}
|
||||
|
||||
@ -214,8 +211,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
|
||||
{
|
||||
if (asterisk_pattern->isColumnMatching(column_name) && (first_table || !data.join_using_columns.count(column_name)))
|
||||
{
|
||||
String table_name = table.getQualifiedNamePrefix(false);
|
||||
addIdentifier(node.children, table_name, column_name, AsteriskSemantic::getAliases(*asterisk_pattern));
|
||||
addIdentifier(node.children, table, column_name, AsteriskSemantic::getAliases(*asterisk_pattern));
|
||||
}
|
||||
}
|
||||
|
||||
@ -232,8 +228,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
|
||||
{
|
||||
for (const auto & column_name : table_columns)
|
||||
{
|
||||
String table_name = table.getQualifiedNamePrefix(false);
|
||||
addIdentifier(node.children, table_name, column_name, AsteriskSemantic::getAliases(*qualified_asterisk));
|
||||
addIdentifier(node.children, table, column_name, AsteriskSemantic::getAliases(*qualified_asterisk));
|
||||
}
|
||||
break;
|
||||
}
|
||||
@ -269,11 +264,13 @@ void TranslateQualifiedNamesMatcher::extractJoinUsingColumns(const ASTPtr ast, D
|
||||
|
||||
void RestoreQualifiedNamesData::visit(ASTIdentifier & identifier, ASTPtr & ast)
|
||||
{
|
||||
if (IdentifierSemantic::getColumnName(identifier) &&
|
||||
IdentifierSemantic::getMembership(identifier))
|
||||
if (IdentifierSemantic::getColumnName(identifier))
|
||||
{
|
||||
ast = identifier.clone();
|
||||
ast->as<ASTIdentifier>()->restoreCompoundName();
|
||||
if (IdentifierSemantic::getMembership(identifier))
|
||||
{
|
||||
ast = identifier.clone();
|
||||
ast->as<ASTIdentifier>()->restoreCompoundName();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -35,6 +35,10 @@ public:
|
||||
, has_columns(has_columns_)
|
||||
{}
|
||||
|
||||
bool hasColumn(const String & name) const { return source_columns.count(name); }
|
||||
bool hasTable() const { return !tables.empty(); }
|
||||
bool processAsterisks() const { return hasTable() && has_columns; }
|
||||
|
||||
static std::vector<TableWithColumnNames> tablesOnly(const std::vector<DatabaseAndTableWithAlias> & tables)
|
||||
{
|
||||
std::vector<TableWithColumnNames> tables_with_columns;
|
||||
@ -44,8 +48,6 @@ public:
|
||||
tables_with_columns.emplace_back(TableWithColumnNames{table, {}});
|
||||
return tables_with_columns;
|
||||
}
|
||||
|
||||
bool processAsterisks() const { return !tables.empty() && has_columns; }
|
||||
};
|
||||
|
||||
static void visit(ASTPtr & ast, Data & data);
|
||||
|
@ -34,10 +34,20 @@ ASTIdentifier::ASTIdentifier(const String & name_, std::vector<String> && name_p
|
||||
, name_parts(name_parts_)
|
||||
, semantic(std::make_shared<IdentifierSemanticImpl>())
|
||||
{
|
||||
if (name_parts.size() && name_parts[0] == "")
|
||||
name_parts.erase(name_parts.begin());
|
||||
|
||||
if (name == "")
|
||||
{
|
||||
if (name_parts.size() == 2)
|
||||
name = name_parts[0] + '.' + name_parts[1];
|
||||
else if (name_parts.size() == 1)
|
||||
name = name_parts[0];
|
||||
}
|
||||
}
|
||||
|
||||
ASTIdentifier::ASTIdentifier(std::vector<String> && name_parts_)
|
||||
: ASTIdentifier(name_parts_.at(0) + '.' + name_parts_.at(1), std::move(name_parts_))
|
||||
: ASTIdentifier("", std::move(name_parts_))
|
||||
{}
|
||||
|
||||
void ASTIdentifier::setShortName(const String & new_name)
|
||||
|
@ -56,26 +56,26 @@ comma nullable
|
||||
1 1 1 1
|
||||
2 2 1 2
|
||||
cross
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE a = t2_00826.a
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826 ON a = t2_00826.a\nWHERE a = t2_00826.a
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE a = t2_00826.a
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON a = t2_00826.a\nWHERE a = t2_00826.a
|
||||
cross nullable
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\n, \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE a = t2_00826.a
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826 ON a = t2_00826.a\nWHERE a = t2_00826.a
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\n, t2_00826\nWHERE a = t2_00826.a
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON a = t2_00826.a\nWHERE a = t2_00826.a
|
||||
cross nullable vs not nullable
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE a = t2_00826.b
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826 ON a = t2_00826.b\nWHERE a = t2_00826.b
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE a = t2_00826.b
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON a = t2_00826.b\nWHERE a = t2_00826.b
|
||||
cross self
|
||||
SELECT \n a, \n b, \n y.a, \n y.b\nFROM t1_00826 AS x\nCROSS JOIN t1_00826 AS y\nWHERE (a = y.a) AND (b = y.b)
|
||||
SELECT \n a, \n b, \n y.a, \n y.b\nFROM t1_00826 AS x\nALL INNER JOIN t1_00826 AS y ON (a = y.a) AND (b = y.b)\nWHERE (a = y.a) AND (b = y.b)
|
||||
cross one table expr
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE a = b
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE a = b
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE a = b
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE a = b
|
||||
cross multiple ands
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE (a = t2_00826.a) AND (b = t2_00826.b)
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND (b = t2_00826.b)
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE (a = t2_00826.a) AND (b = t2_00826.b)
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND (b = t2_00826.b)
|
||||
cross and inside and
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b)))
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826 ON (a = t2_00826.a) AND (a = t2_00826.a) AND (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b)))
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b)))
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (a = t2_00826.a) AND (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b)))
|
||||
cross split conjunction
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n WHERE b > 0\n) AS t2_00826\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) AND (a >= 1) AND (t2_00826.b > 0)
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n WHERE b > 0\n) AS t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) AND (a >= 1) AND (t2_00826.b > 0)
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) AND (a >= 1) AND (t2_00826.b > 0)
|
||||
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) AND (a >= 1) AND (t2_00826.b > 0)
|
||||
|
@ -1,9 +1,10 @@
|
||||
SET enable_debug_queries = 1;
|
||||
SET enable_optimize_predicate_expression = 0;
|
||||
|
||||
set allow_experimental_cross_to_join_conversion = 0;
|
||||
select * from system.one cross join system.one;
|
||||
select * from system.one l cross join system.one r;
|
||||
set allow_experimental_cross_to_join_conversion = 1;
|
||||
select * from system.one cross join system.one;
|
||||
select * from system.one l cross join system.one r;
|
||||
|
||||
DROP TABLE IF EXISTS t1_00826;
|
||||
DROP TABLE IF EXISTS t2_00826;
|
||||
|
@ -1,17 +1,17 @@
|
||||
SELECT a\nFROM t1_00849\nCROSS JOIN \n(\n SELECT *\n FROM t2_00849\n) AS t2_00849
|
||||
SELECT a\nFROM t1_00849\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00849\n) AS t2_00849 ON a = t2_00849.a\nWHERE a = t2_00849.a
|
||||
SELECT a\nFROM t1_00849\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00849\n) AS t2_00849 ON b = t2_00849.b\nWHERE b = t2_00849.b
|
||||
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n WHERE `--t1_00849.a` = `--t2_00849.a`\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t3_00849\n) AS t3_00849 ON `--t1_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = a)
|
||||
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b AS `--t1_00849.b`, \n t2_00849.a, \n t2_00849.b AS `--t2_00849.b`\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.b` = `--t2_00849.b`\n WHERE `--t1_00849.b` = `--t2_00849.b`\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t3_00849\n) AS t3_00849 ON `--t1_00849.b` = b\nWHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = b)
|
||||
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n WHERE `--t1_00849.a` = `--t2_00849.a`\n )\n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849 ON `--t1_00849.a` = `--t3_00849.a`\n WHERE (`--t1_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = `--t2_00849.a`)\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON `--t1_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = a)
|
||||
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n `--t1_00849.b`, \n `t2_00849.a`, \n `--t2_00849.b`, \n a, \n b AS `--t3_00849.b`\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b AS `--t1_00849.b`, \n t2_00849.a, \n t2_00849.b AS `--t2_00849.b`\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.b` = `--t2_00849.b`\n WHERE `--t1_00849.b` = `--t2_00849.b`\n )\n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849 ON `--t1_00849.b` = `--t3_00849.b`\n WHERE (`--t1_00849.b` = `--t3_00849.b`) AND (`--t1_00849.b` = `--t2_00849.b`)\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON `--t1_00849.b` = b\nWHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = `--t3_00849.b`) AND (`--t1_00849.b` = b)
|
||||
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t2_00849.a` = `--t1_00849.a`\n WHERE `--t2_00849.a` = `--t1_00849.a`\n )\n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849 ON `--t2_00849.a` = `--t3_00849.a`\n WHERE (`--t2_00849.a` = `--t3_00849.a`) AND (`--t2_00849.a` = `--t1_00849.a`)\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON `--t2_00849.a` = a\nWHERE (`--t2_00849.a` = `--t1_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t2_00849.a` = a)
|
||||
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849\n )\n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849 ON (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`)\n WHERE (`--t3_00849.a` = `--t2_00849.a`) AND (`--t3_00849.a` = `--t1_00849.a`)\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON `--t3_00849.a` = a\nWHERE (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`) AND (`--t3_00849.a` = a)
|
||||
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849\n )\n CROSS JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`)\nWHERE (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`)
|
||||
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n WHERE `--t1_00849.a` = `--t2_00849.a`\n )\n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849 ON `--t2_00849.a` = `--t3_00849.a`\n WHERE (`--t2_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = `--t2_00849.a`)\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON `--t3_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t3_00849.a` = a)
|
||||
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `t2_00849.a`, \n `t2_00849.b`, \n a, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849\n )\n CROSS JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849\n)\nCROSS JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849
|
||||
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `t2_00849.a`, \n `t2_00849.b`, \n a, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849\n )\n CROSS JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849\n)\nCROSS JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849
|
||||
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n)\nCROSS JOIN \n(\n SELECT *\n FROM t3_00849\n) AS t3_00849
|
||||
SELECT a\nFROM t1_00849\nCROSS JOIN t2_00849
|
||||
SELECT a\nFROM t1_00849\nALL INNER JOIN t2_00849 ON a = t2_00849.a\nWHERE a = t2_00849.a
|
||||
SELECT a\nFROM t1_00849\nALL INNER JOIN t2_00849 ON b = t2_00849.b\nWHERE b = t2_00849.b
|
||||
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n)\nALL INNER JOIN t3_00849 ON `--t1_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = a)
|
||||
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b AS `--t1_00849.b`, \n t2_00849.a, \n t2_00849.b AS `--t2_00849.b`\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.b` = `--t2_00849.b`\n)\nALL INNER JOIN t3_00849 ON `--t1_00849.b` = b\nWHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = b)
|
||||
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n )\n ALL INNER JOIN t3_00849 ON `--t1_00849.a` = `--t3_00849.a`\n)\nALL INNER JOIN t4_00849 ON `--t1_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = a)
|
||||
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n `--t1_00849.b`, \n `t2_00849.a`, \n `--t2_00849.b`, \n a, \n b AS `--t3_00849.b`\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b AS `--t1_00849.b`, \n t2_00849.a, \n t2_00849.b AS `--t2_00849.b`\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.b` = `--t2_00849.b`\n )\n ALL INNER JOIN t3_00849 ON `--t1_00849.b` = `--t3_00849.b`\n)\nALL INNER JOIN t4_00849 ON `--t1_00849.b` = b\nWHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = `--t3_00849.b`) AND (`--t1_00849.b` = b)
|
||||
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t2_00849.a` = `--t1_00849.a`\n )\n ALL INNER JOIN t3_00849 ON `--t2_00849.a` = `--t3_00849.a`\n)\nALL INNER JOIN t4_00849 ON `--t2_00849.a` = a\nWHERE (`--t2_00849.a` = `--t1_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t2_00849.a` = a)
|
||||
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n )\n ALL INNER JOIN t3_00849 ON (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`)\n)\nALL INNER JOIN t4_00849 ON `--t3_00849.a` = a\nWHERE (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`) AND (`--t3_00849.a` = a)
|
||||
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n )\n CROSS JOIN t3_00849\n)\nALL INNER JOIN t4_00849 ON (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`)\nWHERE (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`)
|
||||
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n )\n ALL INNER JOIN t3_00849 ON `--t2_00849.a` = `--t3_00849.a`\n)\nALL INNER JOIN t4_00849 ON `--t3_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t3_00849.a` = a)
|
||||
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `t2_00849.a`, \n `t2_00849.b`, \n a, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n )\n CROSS JOIN t3_00849\n)\nCROSS JOIN t4_00849
|
||||
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `t2_00849.a`, \n `t2_00849.b`, \n a, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n )\n CROSS JOIN t3_00849\n)\nCROSS JOIN t4_00849
|
||||
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n)\nCROSS JOIN t3_00849
|
||||
SELECT * FROM t1, t2
|
||||
1 1 1 1
|
||||
1 1 1 \N
|
||||
|
@ -1,4 +1,5 @@
|
||||
SET enable_debug_queries = 1;
|
||||
SET enable_optimize_predicate_expression = 0;
|
||||
SET joined_subquery_requires_alias = 0;
|
||||
|
||||
DROP TABLE IF EXISTS t1_00849;
|
||||
|
@ -0,0 +1,13 @@
|
||||
0 0
|
||||
0 0
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
┌─[1mone.dummy[0m─┬─[1mA.dummy[0m─┬─[1mB.dummy[0m─┐
|
||||
│ 0 │ 0 │ 0 │
|
||||
└───────────┴─────────┴─────────┘
|
||||
┌─[1mA.dummy[0m─┬─[1mone.dummy[0m─┬─[1mtwo.dummy[0m─┐
|
||||
│ 0 │ 0 │ 0 │
|
||||
└─────────┴───────────┴───────────┘
|
||||
0
|
27
dbms/tests/queries/0_stateless/01018_ambiguous_column.sql
Normal file
27
dbms/tests/queries/0_stateless/01018_ambiguous_column.sql
Normal file
@ -0,0 +1,27 @@
|
||||
select * from system.one cross join system.one; -- { serverError 352 }
|
||||
select * from system.one cross join system.one r;
|
||||
select * from system.one l cross join system.one;
|
||||
select * from system.one left join system.one using dummy;
|
||||
select dummy from system.one left join system.one using dummy;
|
||||
|
||||
USE system;
|
||||
|
||||
SELECT dummy FROM one AS A JOIN one ON A.dummy = one.dummy;
|
||||
SELECT dummy FROM one JOIN one AS A ON A.dummy = one.dummy;
|
||||
SELECT dummy FROM one l JOIN one r ON dummy = r.dummy; -- { serverError 352 }
|
||||
SELECT dummy FROM one l JOIN one r ON l.dummy = dummy; -- { serverError 352 }
|
||||
SELECT dummy FROM one l JOIN one r ON one.dummy = r.dummy; -- { serverError 352 }
|
||||
SELECT dummy FROM one l JOIN one r ON l.dummy = one.dummy; -- { serverError 352 }
|
||||
|
||||
SELECT * from one
|
||||
JOIN one A ON one.dummy = A.dummy
|
||||
JOIN one B ON one.dummy = B.dummy
|
||||
FORMAT PrettyCompact;
|
||||
|
||||
SELECT * from one A
|
||||
JOIN system.one one ON A.dummy = one.dummy
|
||||
JOIN system.one two ON A.dummy = two.dummy
|
||||
FORMAT PrettyCompact;
|
||||
|
||||
-- SELECT one.dummy FROM one AS A FULL JOIN (SELECT 0 AS dymmy) AS one USING dummy;
|
||||
SELECT one.dummy FROM one AS A JOIN (SELECT 0 AS dummy) B USING dummy;
|
Loading…
Reference in New Issue
Block a user