Merge pull request #7358 from 4ertus2/ast

Better ambiguous column detection
This commit is contained in:
Artem Zuikov 2019-10-24 20:56:23 +03:00 committed by GitHub
commit df32d92ee5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
16 changed files with 327 additions and 201 deletions

View File

@ -164,7 +164,8 @@ size_t CollectJoinOnKeysMatcher::getTableForIdentifiers(std::vector<const ASTIde
/// Column name could be cropped to a short form in TranslateQualifiedNamesVisitor. /// Column name could be cropped to a short form in TranslateQualifiedNamesVisitor.
/// In this case it saves membership in IdentifierSemantic. /// 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) if (!membership)
{ {

View File

@ -90,9 +90,12 @@ public:
using TypeToVisit = const ASTFunction; using TypeToVisit = const ASTFunction;
CheckExpressionVisitorData(const std::vector<JoinedTable> & tables_) CheckExpressionVisitorData(const std::vector<JoinedTable> & tables_)
: tables(tables_) : joined_tables(tables_)
, ands_only(true) , ands_only(true)
{} {
for (auto & joined : joined_tables)
tables.push_back(joined.table);
}
void visit(const ASTFunction & node, const ASTPtr & ast) void visit(const ASTFunction & node, const ASTPtr & ast)
{ {
@ -156,7 +159,8 @@ public:
} }
private: 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; std::map<size_t, std::vector<ASTPtr>> asts_to_join_on;
bool ands_only; bool ands_only;
@ -180,31 +184,16 @@ private:
/// @return table position to attach expression to or 0. /// @return table position to attach expression to or 0.
size_t checkIdentifiers(const ASTIdentifier & left, const ASTIdentifier & right) size_t checkIdentifiers(const ASTIdentifier & left, const ASTIdentifier & right)
{ {
/// {best_match, best_table_pos} size_t left_table_pos = 0;
std::pair<size_t, size_t> left_best{0, 0}; bool left_match = IdentifierSemantic::chooseTable(left, tables, left_table_pos);
std::pair<size_t, size_t> right_best{0, 0};
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); size_t table_pos = std::max(left_table_pos, right_table_pos);
if (match > left_best.first) if (joined_tables[table_pos].canAttachOnExpression())
{
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())
return table_pos; return table_pos;
} }
return 0; return 0;
@ -212,20 +201,10 @@ private:
size_t checkIdentifier(const ASTIdentifier & identifier) size_t checkIdentifier(const ASTIdentifier & identifier)
{ {
size_t best_match = 0;
size_t best_table_pos = 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) if (match && joined_tables[best_table_pos].canAttachOnExpression())
{
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())
return best_table_pos; return best_table_pos;
return 0; return 0;
} }

View File

@ -28,17 +28,9 @@ void FindIdentifierBestTableData::visit(ASTIdentifier & identifier, ASTPtr &)
} }
else else
{ {
// FIXME: make a better matcher using `names`? size_t best_table_pos = 0;
size_t best_match = 0; if (IdentifierSemantic::chooseTable(identifier, tables, best_table_pos))
for (const auto & table_names : tables) best_table = &tables[best_table_pos].first;
{
if (size_t match = IdentifierSemantic::canReferColumnToTable(identifier, table_names.first))
if (match > best_match)
{
best_match = match;
best_table = &table_names.first;
}
}
} }
identifier_table.emplace_back(&identifier, best_table); identifier_table.emplace_back(&identifier, best_table);

View File

@ -5,6 +5,61 @@
namespace DB 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) std::optional<String> IdentifierSemantic::getColumnName(const ASTIdentifier & node)
{ {
if (!node.semantic->special) if (!node.semantic->special)
@ -37,26 +92,36 @@ std::optional<String> IdentifierSemantic::getTableName(const ASTPtr & ast)
return {}; return {};
} }
void IdentifierSemantic::setNeedLongName(ASTIdentifier & identifier, bool value)
{
identifier.semantic->need_long_name = value;
}
bool IdentifierSemantic::canBeAlias(const ASTIdentifier & identifier) bool IdentifierSemantic::canBeAlias(const ASTIdentifier & identifier)
{ {
return identifier.semantic->can_be_alias; 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; 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) std::pair<String, String> IdentifierSemantic::extractDatabaseAndTable(const ASTIdentifier & identifier)
{ {
if (identifier.name_parts.size() > 2) if (identifier.name_parts.size() > 2)
@ -84,24 +149,49 @@ bool IdentifierSemantic::doesIdentifierBelongTo(const ASTIdentifier & identifier
return false; 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 /// database.table.column
if (doesIdentifierBelongTo(identifier, db_and_table.database, db_and_table.table)) if (doesIdentifierBelongTo(identifier, db_and_table.database, db_and_table.table))
return 2; return ColumnMatch::DbAndTable;
/// table.column or alias.column. /// alias.column
if (doesIdentifierBelongTo(identifier, db_and_table.table) || if (doesIdentifierBelongTo(identifier, db_and_table.alias))
doesIdentifierBelongTo(identifier, db_and_table.alias)) return ColumnMatch::TableAlias;
return 1;
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. /// Strip qualificators from left side of column name.
/// Example: 'database.table.name' -> (num_qualifiers_to_strip = 2) -> 'name'. /// Example: 'database.table.name' -> 'name'.
void IdentifierSemantic::setColumnShortName(ASTIdentifier & identifier, size_t to_strip) 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) if (!to_strip)
return; return;
@ -117,18 +207,6 @@ void IdentifierSemantic::setColumnShortName(ASTIdentifier & identifier, size_t t
identifier.name.swap(new_name); 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) void IdentifierSemantic::setColumnLongName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
{ {
String prefix = db_and_table.getQualifiedNamePrefix(); 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();
}
} }

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <optional>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Interpreters/DatabaseAndTableWithAlias.h> #include <Interpreters/DatabaseAndTableWithAlias.h>
@ -9,14 +11,23 @@ namespace DB
struct IdentifierSemanticImpl struct IdentifierSemanticImpl
{ {
bool special = false; /// for now it's 'not a column': tables, subselects and some special stuff like FORMAT 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 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 /// Static calss to manipulate IdentifierSemanticImpl via ASTIdentifier
struct IdentifierSemantic 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 /// @returns name for column identifiers
static std::optional<String> getColumnName(const ASTIdentifier & node); static std::optional<String> getColumnName(const ASTIdentifier & node);
static std::optional<String> getColumnName(const ASTPtr & ast); static std::optional<String> getColumnName(const ASTPtr & ast);
@ -26,20 +37,20 @@ struct IdentifierSemantic
static std::optional<String> getTableName(const ASTPtr & ast); static std::optional<String> getTableName(const ASTPtr & ast);
static std::pair<String, String> extractDatabaseAndTable(const ASTIdentifier & identifier); static std::pair<String, String> extractDatabaseAndTable(const ASTIdentifier & identifier);
static size_t canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table); static ColumnMatch canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static String columnNormalName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table); static void setColumnShortName(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 setColumnLongName(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 bool canBeAlias(const ASTIdentifier & identifier);
static void setMembership(ASTIdentifier & identifier, size_t table_no); static void setMembership(ASTIdentifier &, size_t table_no);
static size_t getMembership(const ASTIdentifier & identifier); 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: private:
static bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & database, const String & table); static bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & database, const String & table);
static bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & table); static bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & table);
static void setColumnShortName(ASTIdentifier & identifier, size_t match);
}; };
} }

View File

@ -147,7 +147,13 @@ struct ColumnAliasesMatcher
auto it = rev_aliases.find(long_name); auto it = rev_aliases.find(long_name);
if (it == rev_aliases.end()) 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) if (!last_table)
{ {
String alias = hide_prefix + long_name; String alias = hide_prefix + long_name;
@ -202,17 +208,15 @@ struct ColumnAliasesMatcher
bool last_table = false; bool last_table = false;
String long_name; 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)) auto & table = data.tables[table_pos];
{ IdentifierSemantic::setColumnLongName(node, table); /// table_name.column_name -> table_alias.column_name
if (!long_name.empty()) long_name = node.name;
throw Exception("Cannot refer column '" + node.name + "' to one table", ErrorCodes::AMBIGUOUS_COLUMN_NAME); if (&table == &data.tables.back())
IdentifierSemantic::setColumnLongName(node, table); /// table_name.column_name -> table_alias.column_name last_table = true;
long_name = node.name;
if (&table == &data.tables.back())
last_table = true;
}
} }
if (long_name.empty()) if (long_name.empty())

View File

@ -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); std::vector<TableWithColumnNames> tables_with_columns = getDatabaseAndTablesWithColumnNames(select_query, context);
@ -125,6 +128,27 @@ std::vector<TableWithColumnNames> getTablesWithColumns(const ASTSelectQuery & se
ErrorCodes::ALIAS_REQUIRED); 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; 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) if (!join || !join->table_expression)
return; 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 &>(); auto & table_expr = join->table_expression->as<ASTTableExpression &>();
if (table_expr.database_and_table_name) if (table_expr.database_and_table_name)
{ {
@ -826,41 +855,34 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
if (remove_duplicates) if (remove_duplicates)
renameDuplicatedColumns(select_query); 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) if (!settings.any_join_distinct_right_table_keys)
checkJoin(node); checkJoin(table_join_node);
if (settings.enable_optimize_predicate_expression) if (settings.enable_optimize_predicate_expression)
replaceJoinedTable(node); replaceJoinedTable(table_join_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());
} }
auto tables_with_columns = getTablesWithColumns(*select_query, context); auto get_column_names = [&]() -> Names
/// If empty make fake table with list of source and joined columns
if (tables_with_columns.empty())
{ {
Names columns_list;
if (storage) if (storage)
columns_list = storage->getColumns().getOrdinary().getNames(); return 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);
}
for (auto & column : result.analyzed_join->getQualifiedColumnsSet()) Names columns;
columns_list.emplace_back(column); 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)); translateQualifiedNames(query, *select_query, source_columns_set, std::move(tables_with_columns));

View File

@ -61,24 +61,20 @@ void TranslateQualifiedNamesMatcher::visit(ASTIdentifier & identifier, ASTPtr &,
{ {
if (IdentifierSemantic::getColumnName(identifier)) if (IdentifierSemantic::getColumnName(identifier))
{ {
size_t best_table_pos = 0; String short_name = identifier.shortName();
size_t best_match = 0; size_t table_pos = 0;
for (size_t i = 0; i < data.tables.size(); ++i) bool allow_ambiguous = data.join_using_columns.count(short_name);
if (size_t match = IdentifierSemantic::canReferColumnToTable(identifier, data.tables[i].first)) if (IdentifierSemantic::chooseTable(identifier, data.tables, table_pos, allow_ambiguous))
if (match > best_match) {
{ IdentifierSemantic::setMembership(identifier, table_pos);
best_match = match;
best_table_pos = i;
}
if (best_match) /// In case if column from the joined table are in source columns, change it's name to qualified.
IdentifierSemantic::setMembership(identifier, best_table_pos + 1); auto & table = data.tables[table_pos].first;
if (table_pos && data.hasColumn(short_name))
/// In case if column from the joined table are in source columns, change it's name to qualified. IdentifierSemantic::setColumnLongName(identifier, table);
if (best_table_pos && data.source_columns.count(identifier.shortName())) else
IdentifierSemantic::setNeedLongName(identifier, true); IdentifierSemantic::setColumnShortName(identifier, table);
if (!data.tables.empty()) }
IdentifierSemantic::setColumnNormalName(identifier, data.tables[best_table_pos].first);
} }
} }
@ -134,8 +130,10 @@ void TranslateQualifiedNamesMatcher::visit(ASTSelectQuery & select, const ASTPtr
Visitor(data).visit(select.refHaving()); 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}); auto identifier = std::make_shared<ASTIdentifier>(std::vector<String>{table_name, column_name});
bool added = false; bool added = false;
@ -197,8 +195,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
{ {
if (first_table || !data.join_using_columns.count(column_name)) if (first_table || !data.join_using_columns.count(column_name))
{ {
String table_name = table.getQualifiedNamePrefix(false); addIdentifier(node.children, table, column_name, AsteriskSemantic::getAliases(*asterisk));
addIdentifier(node.children, table_name, 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))) 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, column_name, AsteriskSemantic::getAliases(*asterisk_pattern));
addIdentifier(node.children, table_name, column_name, AsteriskSemantic::getAliases(*asterisk_pattern));
} }
} }
@ -232,8 +228,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
{ {
for (const auto & column_name : table_columns) for (const auto & column_name : table_columns)
{ {
String table_name = table.getQualifiedNamePrefix(false); addIdentifier(node.children, table, column_name, AsteriskSemantic::getAliases(*qualified_asterisk));
addIdentifier(node.children, table_name, column_name, AsteriskSemantic::getAliases(*qualified_asterisk));
} }
break; break;
} }
@ -269,11 +264,13 @@ void TranslateQualifiedNamesMatcher::extractJoinUsingColumns(const ASTPtr ast, D
void RestoreQualifiedNamesData::visit(ASTIdentifier & identifier, ASTPtr & ast) void RestoreQualifiedNamesData::visit(ASTIdentifier & identifier, ASTPtr & ast)
{ {
if (IdentifierSemantic::getColumnName(identifier) && if (IdentifierSemantic::getColumnName(identifier))
IdentifierSemantic::getMembership(identifier))
{ {
ast = identifier.clone(); if (IdentifierSemantic::getMembership(identifier))
ast->as<ASTIdentifier>()->restoreCompoundName(); {
ast = identifier.clone();
ast->as<ASTIdentifier>()->restoreCompoundName();
}
} }
} }

View File

@ -35,6 +35,10 @@ public:
, has_columns(has_columns_) , 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) static std::vector<TableWithColumnNames> tablesOnly(const std::vector<DatabaseAndTableWithAlias> & tables)
{ {
std::vector<TableWithColumnNames> tables_with_columns; std::vector<TableWithColumnNames> tables_with_columns;
@ -44,8 +48,6 @@ public:
tables_with_columns.emplace_back(TableWithColumnNames{table, {}}); tables_with_columns.emplace_back(TableWithColumnNames{table, {}});
return tables_with_columns; return tables_with_columns;
} }
bool processAsterisks() const { return !tables.empty() && has_columns; }
}; };
static void visit(ASTPtr & ast, Data & data); static void visit(ASTPtr & ast, Data & data);

View File

@ -34,10 +34,20 @@ ASTIdentifier::ASTIdentifier(const String & name_, std::vector<String> && name_p
, name_parts(name_parts_) , name_parts(name_parts_)
, semantic(std::make_shared<IdentifierSemanticImpl>()) , 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::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) void ASTIdentifier::setShortName(const String & new_name)

View File

@ -56,26 +56,26 @@ comma nullable
1 1 1 1 1 1 1 1
2 2 1 2 2 2 1 2
cross 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\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 \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\nALL INNER JOIN t2_00826 ON a = t2_00826.a\nWHERE a = t2_00826.a
cross nullable 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\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 \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\nALL INNER JOIN t2_00826 ON a = t2_00826.a\nWHERE a = t2_00826.a
cross nullable vs not nullable 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\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 \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\nALL INNER JOIN t2_00826 ON a = t2_00826.b\nWHERE a = t2_00826.b
cross self 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\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) 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 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 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
cross multiple ands 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\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 \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\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 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\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 \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\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 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\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 \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\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)

View File

@ -1,9 +1,10 @@
SET enable_debug_queries = 1; SET enable_debug_queries = 1;
SET enable_optimize_predicate_expression = 0;
set allow_experimental_cross_to_join_conversion = 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; 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 t1_00826;
DROP TABLE IF EXISTS t2_00826; DROP TABLE IF EXISTS t2_00826;

View File

@ -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\nCROSS JOIN 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 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 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 \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, \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 \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 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 \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 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 \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 `--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 \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 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 \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 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 \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 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 \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 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 \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 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 \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 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 \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 `--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 SELECT * FROM t1, t2
1 1 1 1 1 1 1 1
1 1 1 \N 1 1 1 \N

View File

@ -1,4 +1,5 @@
SET enable_debug_queries = 1; SET enable_debug_queries = 1;
SET enable_optimize_predicate_expression = 0;
SET joined_subquery_requires_alias = 0; SET joined_subquery_requires_alias = 0;
DROP TABLE IF EXISTS t1_00849; DROP TABLE IF EXISTS t1_00849;

View File

@ -0,0 +1,13 @@
0 0
0 0
0
0
0
0
┌─one.dummy─┬─A.dummy─┬─B.dummy─┐
│ 0 │ 0 │ 0 │
└───────────┴─────────┴─────────┘
┌─A.dummy─┬─one.dummy─┬─two.dummy─┐
│ 0 │ 0 │ 0 │
└─────────┴───────────┴───────────┘
0

View 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;