Merge pull request #4595 from 4ertus2/joins

Multiple JOINs multiple fixes
This commit is contained in:
alexey-milovidov 2019-03-06 02:55:05 +03:00 committed by GitHub
commit 74f7ef390a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 138 additions and 46 deletions

View File

@ -62,11 +62,11 @@ bool DatabaseAndTableWithAlias::satisfies(const DatabaseAndTableWithAlias & db_t
return database == db_table.database && table == db_table.table;
}
String DatabaseAndTableWithAlias::getQualifiedNamePrefix() const
String DatabaseAndTableWithAlias::getQualifiedNamePrefix(bool with_dot) const
{
if (alias.empty() && table.empty())
return "";
return (!alias.empty() ? alias : table) + '.';
return (!alias.empty() ? alias : table) + (with_dot ? "." : "");
}
std::vector<const ASTTableExpression *> getSelectTablesExpression(const ASTSelectQuery & select_query)

View File

@ -32,7 +32,7 @@ struct DatabaseAndTableWithAlias
DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, const String & current_database = "");
/// "alias." or "table." if alias is empty
String getQualifiedNamePrefix() const;
String getQualifiedNamePrefix(bool with_dot = true) const;
/// Check if it satisfies another db_table name. @note opterion is not symmetric.
bool satisfies(const DatabaseAndTableWithAlias & table, bool table_may_be_an_alias);

View File

@ -36,9 +36,10 @@ struct ColumnAliasesMatcher
{
const std::vector<DatabaseAndTableWithAlias> tables;
bool public_names;
AsteriskSemantic::RevertedAliases rev_aliases;
std::unordered_map<String, String> aliases;
AsteriskSemantic::RevertedAliases rev_aliases; /// long_name -> aliases
std::unordered_map<String, String> aliases; /// alias -> long_name
std::vector<std::pair<ASTIdentifier *, bool>> compound_identifiers;
std::set<String> allowed_long_names; /// original names allowed as aliases '--t.x as t.x' (select expressions only).
Data(std::vector<DatabaseAndTableWithAlias> && tables_)
: tables(tables_)
@ -51,29 +52,37 @@ struct ColumnAliasesMatcher
for (auto & [identifier, is_public] : compound_identifiers)
{
auto it = rev_aliases.find(identifier->name);
String long_name = identifier->name;
auto it = rev_aliases.find(long_name);
if (it == rev_aliases.end())
{
bool last_table = IdentifierSemantic::canReferColumnToTable(*identifier, tables.back());
if (!last_table)
{
String long_name = identifier->name;
String alias = hide_prefix + long_name;
aliases[alias] = long_name;
rev_aliases[long_name].push_back(alias);
identifier->setShortName(alias);
if (is_public)
{
identifier->setAlias(long_name);
allowed_long_names.insert(long_name);
}
}
else if (is_public)
identifier->setAlias(identifier->name); /// prevent crop long to short name
identifier->setAlias(long_name); /// prevent crop long to short name
}
else
{
if (it->second.empty())
throw Exception("No alias for '" + identifier->name + "'", ErrorCodes::LOGICAL_ERROR);
identifier->setShortName(it->second[0]);
throw Exception("No alias for '" + long_name + "'", ErrorCodes::LOGICAL_ERROR);
if (is_public && allowed_long_names.count(long_name))
; /// leave original name unchanged for correct output
else
identifier->setShortName(it->second[0]);
}
}
}
@ -131,7 +140,7 @@ struct ColumnAliasesMatcher
node.setAlias("");
}
}
else
else if (node.compound())
data.compound_identifiers.emplace_back(&node, data.public_names);
}
};

View File

@ -652,7 +652,8 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
{
if (const ASTTablesInSelectQueryElement * node = select_query->join())
{
replaceJoinedTable(node);
if (settings.enable_optimize_predicate_expression)
replaceJoinedTable(node);
const auto & joined_expression = static_cast<const ASTTableExpression &>(*node->table_expression);
DatabaseAndTableWithAlias table(joined_expression, context.getCurrentDatabase());

View File

@ -143,21 +143,14 @@ void TranslateQualifiedNamesMatcher::visit(ASTSelectQuery & select, const ASTPtr
Visitor(data).visit(*add_node);
}
/// qualifed names for duplicates
static std::shared_ptr<ASTIdentifier> makeIdentifier(const String & short_name, const String & long_name, bool need_long_name)
static void addIdentifier(ASTs & nodes, const String & table_name, const String & column_name, AsteriskSemantic::RevertedAliasesPtr aliases)
{
if (need_long_name)
return std::make_shared<ASTIdentifier>(long_name);
return std::make_shared<ASTIdentifier>(short_name);
}
auto identifier = std::make_shared<ASTIdentifier>(std::vector<String>{table_name, column_name});
static void addIdentifier(ASTs & nodes, std::shared_ptr<ASTIdentifier> identifier, const String & long_name,
AsteriskSemantic::RevertedAliasesPtr aliases)
{
bool added = false;
if (aliases && aliases->count(long_name))
if (aliases && aliases->count(identifier->name))
{
for (const String & alias : (*aliases)[long_name])
for (const String & alias : (*aliases)[identifier->name])
{
nodes.push_back(identifier->clone());
nodes.back()->setAlias(alias);
@ -173,7 +166,6 @@ static void addIdentifier(ASTs & nodes, std::shared_ptr<ASTIdentifier> identifie
void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPtr &, Data & data)
{
const auto & tables_with_columns = data.tables;
const auto & source_columns = data.source_columns;
ASTs old_children;
if (data.processAsterisks())
@ -208,16 +200,14 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
if (const auto * asterisk = typeid_cast<const ASTAsterisk *>(child.get()))
{
bool first_table = true;
for (const auto & [table_name, table_columns] : tables_with_columns)
for (const auto & [table, table_columns] : tables_with_columns)
{
for (const auto & column_name : table_columns)
{
if (first_table || !data.join_using_columns.count(column_name))
{
bool need_prefix = !first_table && source_columns.count(column_name);
String long_name = table_name.getQualifiedNamePrefix() + column_name;
auto identifier = makeIdentifier(column_name, long_name, need_prefix);
addIdentifier(node.children, identifier, long_name, AsteriskSemantic::getAliases(*asterisk));
String table_name = table.getQualifiedNamePrefix(false);
addIdentifier(node.children, table_name, column_name, AsteriskSemantic::getAliases(*asterisk));
}
}
@ -228,22 +218,17 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
{
DatabaseAndTableWithAlias ident_db_and_name(qualified_asterisk->children[0]);
bool first_table = true;
for (const auto & [table_name, table_columns] : tables_with_columns)
for (const auto & [table, table_columns] : tables_with_columns)
{
if (ident_db_and_name.satisfies(table_name, true))
if (ident_db_and_name.satisfies(table, true))
{
for (const auto & column_name : table_columns)
{
bool need_prefix = !first_table && source_columns.count(column_name);
String long_name = table_name.getQualifiedNamePrefix() + column_name;
auto identifier = makeIdentifier(column_name, long_name, need_prefix);
addIdentifier(node.children, identifier, long_name, AsteriskSemantic::getAliases(*qualified_asterisk));
String table_name = table.getQualifiedNamePrefix(false);
addIdentifier(node.children, table_name, column_name, AsteriskSemantic::getAliases(*qualified_asterisk));
}
break;
}
first_table = false;
}
}
else

View File

@ -29,6 +29,10 @@ ASTIdentifier::ASTIdentifier(const String & name_, std::vector<String> && name_p
{
}
ASTIdentifier::ASTIdentifier(std::vector<String> && name_parts_)
: ASTIdentifier(name_parts_.at(0) + '.' + name_parts_.at(1), std::move(name_parts_))
{}
void ASTIdentifier::setShortName(const String & new_name)
{
name = new_name;
@ -48,9 +52,8 @@ void ASTIdentifier::formatImplWithoutAlias(const FormatSettings & settings, Form
settings.ostr << (settings.hilite ? hilite_none : "");
};
/// A simple or compound identifier?
if (name_parts.size() > 1)
/// It could be compound but short
if (!isShort())
{
for (size_t i = 0, size = name_parts.size(); i < size; ++i)
{

View File

@ -22,6 +22,7 @@ public:
String name;
ASTIdentifier(const String & name_, std::vector<String> && name_parts_ = {});
ASTIdentifier(std::vector<String> && name_parts_);
/** Get the text that identifies this element. */
String getID(char delim) const override { return "Identifier" + (delim + name); }

View File

@ -20,7 +20,7 @@ SELECT \n a, \n b\nFROM \n(\n SELECT \n toUInt64(sum(id) AS b) A
3 3
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n name, \n value, \n min(id) AS id\n FROM test.test \n GROUP BY \n date, \n name, \n value\n HAVING id = 1\n) \nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n a, \n b\nFROM \n(\n SELECT \n toUInt64(sum(id) AS b) AS a, \n b\n FROM test.test AS table_alias \n HAVING b = 3\n) AS outer_table_alias \nWHERE outer_table_alias.b = 3
SELECT \n a, \n b\nFROM \n(\n SELECT \n toUInt64(sum(id) AS b) AS a, \n b\n FROM test.test AS table_alias \n HAVING b = 3\n) AS outer_table_alias \nWHERE b = 3
3 3
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test.test \n WHERE id = 1\n) \nWHERE id = 1
2000-01-01 1 test string 1 1
@ -32,9 +32,9 @@ SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n
2000-01-01 1 test string 1 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test.test \n WHERE id = 1\n ) \n WHERE id = 1\n) \nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test.test \n WHERE id = 1\n) AS b \nWHERE b.id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test.test \n WHERE id = 1\n) AS b \nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test.test \n WHERE id = 1\n ) AS a \n WHERE id = 1\n) AS b \nWHERE b.id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test.test \n WHERE id = 1\n ) AS a \n WHERE id = 1\n) AS b \nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n id, \n date, \n value\nFROM \n(\n SELECT \n id, \n date, \n min(value) AS value\n FROM test.test \n WHERE id = 1\n GROUP BY \n id, \n date\n) \nWHERE id = 1
1 2000-01-01 1
@ -45,11 +45,11 @@ SELECT \n date, \n id, \n name, \n value, \n date, \n name, \n
2000-01-01 1 test string 1 1 2000-01-01 test string 1 1
SELECT \n id, \n date, \n name, \n value\nFROM \n(\n SELECT toInt8(1) AS id\n) \nANY LEFT JOIN test.test USING (id)\nWHERE value = 1
1 2000-01-01 test string 1 1
SELECT b.value\nFROM \n(\n SELECT toInt8(1) AS id\n) \nANY LEFT JOIN test.test AS b USING (id)\nWHERE value = 1
SELECT value\nFROM \n(\n SELECT toInt8(1) AS id\n) \nANY LEFT JOIN test.test AS b USING (id)\nWHERE value = 1
1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value, \n date, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test.test \n WHERE id = 1\n ) \n ANY LEFT JOIN \n (\n SELECT *\n FROM test.test \n WHERE id = 1\n ) USING (id)\n WHERE id = 1\n) \nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n date, \n id, \n name, \n value, \n `b.date`, \n `b.name`, \n `b.value`\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test.test \n) \nANY LEFT JOIN \n(\n SELECT *\n FROM test.test \n WHERE id = 1\n) AS b USING (id)\nWHERE b.id = 1
SELECT \n date, \n id, \n name, \n value, \n b.date, \n b.name, \n b.value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test.test \n) \nANY LEFT JOIN \n(\n SELECT *\n FROM test.test \n WHERE id = 1\n) AS b USING (id)\nWHERE b.id = 1
2000-01-01 1 test string 1 1 2000-01-01 test string 1 1
SELECT \n id, \n date, \n name, \n value\nFROM \n(\n SELECT \n toInt8(1) AS id, \n toDate(\'2000-01-01\') AS date\n FROM system.numbers \n LIMIT 1\n) \nANY LEFT JOIN \n(\n SELECT *\n FROM test.test \n WHERE date = toDate(\'2000-01-01\')\n) AS b USING (date, id)\nWHERE b.date = toDate(\'2000-01-01\')
1 2000-01-01 test string 1 1

View File

@ -0,0 +1,45 @@
Row 1:
──────
t.a: 1
s.b: 1
s.a: 1
s.b: 1
y.a: 1
y.b: 1
Row 2:
──────
t.a: 2
s.b: 0
s.a: 0
s.b: 0
y.a: 0
y.b: 0
┌─t.a─┬─s.b─┬─s.a─┬─s.b─┬─y.a─┬─y.b─┐
│ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │
│ 2 │ 0 │ 0 │ 0 │ 0 │ 0 │
└─────┴─────┴─────┴─────┴─────┴─────┘
┌─t_a─┐
│ 1 │
│ 2 │
└─────┘
┌─t.a─┬─s_a─┐
│ 1 │ 1 │
│ 2 │ 0 │
└─────┴─────┘
┌─t.a─┬─t.a─┬─t_b─┐
│ 1 │ 1 │ 1 │
│ 2 │ 2 │ 2 │
└─────┴─────┴─────┘
┌─s.a─┬─s.a─┬─s_b─┬─s_b─┐
│ 1 │ 1 │ 1 │ 1 │
│ 0 │ 0 │ 0 │ 0 │
└─────┴─────┴─────┴─────┘
┌─y.a─┬─y.a─┬─y_b─┬─y_b─┐
│ 1 │ 1 │ 1 │ 1 │
│ 0 │ 0 │ 0 │ 0 │
└─────┴─────┴─────┴─────┘
┌─t_a─┬─t_a─┬─s_a─┬─s_a─┬─y_a─┬─y_a─┐
│ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │
│ 2 │ 2 │ 0 │ 0 │ 0 │ 0 │
└─────┴─────┴─────┴─────┴─────┴─────┘

View File

@ -0,0 +1,48 @@
use test;
drop table if exists t;
drop table if exists s;
drop table if exists y;
create table t(a Int64, b Int64) engine = TinyLog;
create table s(a Int64, b Int64) engine = TinyLog;
create table y(a Int64, b Int64) engine = TinyLog;
insert into t values (1,1), (2,2);
insert into s values (1,1);
insert into y values (1,1);
select t.a, s.b, s.a, s.b, y.a, y.b from t
left join s on (t.a = s.a and t.b = s.b)
left join y on (y.a = s.a and y.b = s.b) format Vertical;
select t.a, s.b, s.a, s.b, y.a, y.b from t
left join s on (t.a = s.a and s.b = t.b)
left join y on (y.a = s.a and y.b = s.b) format PrettyCompactNoEscapes;
select t.a as t_a from t
left join s on s.a = t_a format PrettyCompactNoEscapes;
select t.a, s.a as s_a from t
left join s on s.a = t.a
left join y on y.b = s.b format PrettyCompactNoEscapes;
select t.a, t.a, t.b as t_b from t
left join s on t.a = s.a
left join y on y.b = s.b format PrettyCompactNoEscapes;
select s.a, s.a, s.b as s_b, s.b from t
left join s on s.a = t.a
left join y on s.b = y.b format PrettyCompactNoEscapes;
select y.a, y.a, y.b as y_b, y.b from t
left join s on s.a = t.a
left join y on y.b = s.b format PrettyCompactNoEscapes;
select t.a, t.a as t_a, s.a, s.a as s_a, y.a, y.a as y_a from t
left join s on t.a = s.a
left join y on y.b = s.b format PrettyCompactNoEscapes;
drop table t;
drop table s;
drop table y;