未验证 提交 44b878c8 编写于 作者: A alexey-milovidov 提交者: GitHub

Merge pull request #4462 from 4ertus2/joins

Multiple JOINS: resolve column names via flat aliases
#pragma once
#include <unordered_map>
#include <Parsers/ASTAsterisk.h>
#include <Parsers/ASTQualifiedAsterisk.h>
namespace DB
{
struct AsteriskSemanticImpl
{
using RevertedAliases = std::unordered_map<String, std::vector<String>>;
using RevertedAliasesPtr = std::shared_ptr<RevertedAliases>;
RevertedAliasesPtr aliases; /// map of aliases that should be set in phase of * expanding.
};
struct AsteriskSemantic
{
using RevertedAliases = AsteriskSemanticImpl::RevertedAliases;
using RevertedAliasesPtr = AsteriskSemanticImpl::RevertedAliasesPtr;
static void setAliases(ASTAsterisk & node, const RevertedAliasesPtr & aliases) { node.semantic = makeSemantic(aliases); }
static void setAliases(ASTQualifiedAsterisk & node, const RevertedAliasesPtr & aliases) { node.semantic = makeSemantic(aliases); }
static RevertedAliasesPtr getAliases(const ASTAsterisk & node) { return node.semantic ? node.semantic->aliases : nullptr; }
static RevertedAliasesPtr getAliases(const ASTQualifiedAsterisk & node) { return node.semantic ? node.semantic->aliases : nullptr; }
private:
static std::shared_ptr<AsteriskSemanticImpl> makeSemantic(const RevertedAliasesPtr & aliases)
{
return std::make_shared<AsteriskSemanticImpl>(AsteriskSemanticImpl{aliases});
}
};
}
......@@ -126,15 +126,18 @@ void IdentifierSemantic::setColumnNormalName(ASTIdentifier & identifier, const D
identifier.semantic->can_be_alias = false;
if (identifier.semantic->need_long_name)
setColumnLongName(identifier, db_and_table);
}
void IdentifierSemantic::setColumnLongName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
{
String prefix = db_and_table.getQualifiedNamePrefix();
if (!prefix.empty())
{
String prefix = db_and_table.getQualifiedNamePrefix();
if (!prefix.empty())
{
String short_name = identifier.shortName();
identifier.name = prefix + short_name;
prefix.resize(prefix.size() - 1); /// crop dot
identifier.name_parts = {prefix, short_name};
}
String short_name = identifier.shortName();
identifier.name = prefix + short_name;
prefix.resize(prefix.size() - 1); /// crop dot
identifier.name_parts = {prefix, short_name};
}
}
......@@ -145,4 +148,9 @@ String IdentifierSemantic::columnNormalName(const ASTIdentifier & identifier, co
return copy->getAliasOrColumnName();
}
String IdentifierSemantic::columnLongName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
{
return db_and_table.getQualifiedNamePrefix() + identifier.shortName();
}
}
......@@ -28,7 +28,9 @@ struct IdentifierSemantic
static size_t canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static String columnNormalName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static String columnLongName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static void setColumnNormalName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static void 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);
......
#include <Common/typeid_cast.h>
#include <Interpreters/JoinToSubqueryTransformVisitor.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/AsteriskSemantic.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTIdentifier.h>
......@@ -9,6 +12,7 @@
#include <Parsers/parseQuery.h>
#include <IO/WriteHelpers.h>
namespace DB
{
......@@ -16,48 +20,167 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int TOO_DEEP_AST;
extern const int AMBIGUOUS_COLUMN_NAME;
extern const int NOT_IMPLEMENTED;
}
#if 0
/// Attach additional semantic info to generated select.
namespace
{
/// Find columns with aliases to push them into rewritten subselects.
/// Normalize table aliases: table_name.column_name -> table_alias.column_name
/// Make aliases maps (alias -> column_name, column_name -> alias)
struct ColumnAliasesMatcher
{
struct Data
{
const std::vector<DatabaseAndTableWithAlias> tables;
AsteriskSemantic::RevertedAliases rev_aliases;
std::unordered_map<String, String> aliases;
std::vector<ASTIdentifier *> compound_identifiers;
Data(std::vector<DatabaseAndTableWithAlias> && tables_)
: tables(tables_)
{}
void replaceIdentifiersWithAliases()
{
String hide_prefix = "--"; /// @note restriction: user should not use alises like `--table.column`
for (auto * identifier : compound_identifiers)
{
auto it = rev_aliases.find(identifier->name);
if (it == rev_aliases.end())
{
bool last_table = IdentifierSemantic::canReferColumnToTable(*identifier, tables.back());
if (!last_table)
{
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);
//identifier->setAlias(long_name);
}
}
else
{
if (it->second.empty())
throw Exception("No alias for '" + identifier->name + "'", ErrorCodes::LOGICAL_ERROR);
identifier->setShortName(it->second[0]);
}
}
}
};
static constexpr const char * label = "ColumnAliases";
static bool needChildVisit(ASTPtr & node, const ASTPtr &)
{
if (typeid_cast<const ASTQualifiedAsterisk *>(node.get()))
return false;
return true;
}
static std::vector<ASTPtr *> visit(ASTPtr & ast, Data & data)
{
if (auto * t = typeid_cast<ASTIdentifier *>(ast.get()))
visit(*t, ast, data);
if (typeid_cast<ASTAsterisk *>(ast.get()) ||
typeid_cast<ASTQualifiedAsterisk *>(ast.get()))
throw Exception("Multiple JOIN do not support asterisks yet", ErrorCodes::NOT_IMPLEMENTED);
return {};
}
static void visit(ASTIdentifier & node, ASTPtr &, Data & data)
{
if (node.isShort())
return;
bool last_table = false;
String long_name;
for (auto & table : data.tables)
{
if (IdentifierSemantic::canReferColumnToTable(node, table))
{
if (!long_name.empty())
throw Exception("Cannot refer column '" + node.name + "' to one table", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
IdentifierSemantic::setColumnLongName(node, table); /// table_name.column_name -> table_alias.column_name
long_name = node.name;
if (&table == &data.tables.back())
last_table = true;
}
}
if (long_name.empty())
throw Exception("Cannot refer column '" + node.name + "' to table", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
String alias = node.tryGetAlias();
if (!alias.empty())
{
data.aliases[alias] = long_name;
data.rev_aliases[long_name].push_back(alias);
if (!last_table)
{
node.setShortName(alias);
node.setAlias("");
}
}
else
data.compound_identifiers.push_back(&node);
}
};
/// Attach additional semantic info to generated selects.
struct AppendSemanticVisitorData
{
using TypeToVisit = ASTSelectQuery;
const SemanticPtr & semantic;
AsteriskSemantic::RevertedAliasesPtr rev_aliases = {};
bool done = false;
void visit(ASTSelectQuery & select, ASTPtr &)
{
if (done || !semantic)
if (done || !rev_aliases || !select.select_expression_list)
return;
select.semantic = semantic->clone();
for (auto & child : select.select_expression_list->children)
{
if (auto * node = typeid_cast<ASTAsterisk *>(child.get()))
AsteriskSemantic::setAliases(*node, rev_aliases);
if (auto * node = typeid_cast<ASTQualifiedAsterisk *>(child.get()))
AsteriskSemantic::setAliases(*node, rev_aliases);
}
done = true;
}
};
#endif
/// Replaces one table element with pair.
/// Replaces table elements with pair.
struct RewriteTablesVisitorData
{
using TypeToVisit = ASTTablesInSelectQuery;
const ASTPtr & left;
const ASTPtr & right;
ASTPtr left;
ASTPtr right;
bool done = false;
/// @note Do not change ASTTablesInSelectQuery itself. No need to change select.tables.
void visit(ASTTablesInSelectQuery &, ASTPtr & ast)
{
if (done)
return;
ast->children.clear();
ast->children.push_back(left);
ast->children.push_back(right);
std::vector<ASTPtr> new_tables{left, right};
ast->children.swap(new_tables);
done = true;
}
};
static bool needRewrite(ASTSelectQuery & select)
bool needRewrite(ASTSelectQuery & select)
{
if (!select.tables)
return false;
......@@ -70,28 +193,31 @@ static bool needRewrite(ASTSelectQuery & select)
if (num_tables <= 2)
return false;
for (size_t i = 1; i < tables->children.size(); ++i)
{
auto table = typeid_cast<const ASTTablesInSelectQueryElement *>(tables->children[i].get());
if (!table || !table->table_join)
throw Exception("Multiple JOIN expects joined tables", ErrorCodes::LOGICAL_ERROR);
auto join = typeid_cast<const ASTTableJoin *>(table->table_join.get());
if (join->kind == ASTTableJoin::Kind::Comma)
throw Exception("Multiple COMMA JOIN is not supported", ErrorCodes::NOT_IMPLEMENTED);
/// it's not trivial to support mix of JOIN ON & JOIN USING cause of short names
if (!join || !join->on_expression)
throw Exception("Multiple JOIN expects JOIN with ON section", ErrorCodes::NOT_IMPLEMENTED);
}
return true;
}
static void appendTableNameAndAlias(std::vector<String> & hidden, const ASTPtr & table_element)
{
auto element = typeid_cast<const ASTTablesInSelectQueryElement *>(table_element.get());
if (!element || element->children.empty())
throw Exception("Expected TablesInSelectQueryElement with at least one child", ErrorCodes::LOGICAL_ERROR);
auto table_expression = typeid_cast<const ASTTableExpression *>(element->children[0].get());
if (!table_expression || table_expression->children.empty())
throw Exception("Expected TableExpression with at least one child", ErrorCodes::LOGICAL_ERROR);
String alias = table_expression->children[0]->tryGetAlias();
if (!alias.empty())
hidden.push_back(alias);
if (auto opt_name = getIdentifierName(table_expression->children[0]))
hidden.push_back(*opt_name);
else if (alias.empty())
throw Exception("Expected Identifier or subquery with alias", ErrorCodes::LOGICAL_ERROR);
}
using RewriteMatcher = OneTypeMatcher<RewriteTablesVisitorData>;
using RewriteVisitor = InDepthNodeVisitor<RewriteMatcher, true>;
using ColumnAliasesVisitor = InDepthNodeVisitor<ColumnAliasesMatcher, true>;
using AppendSemanticMatcher = OneTypeMatcher<AppendSemanticVisitorData>;
using AppendSemanticVisitor = InDepthNodeVisitor<AppendSemanticMatcher, true>;
} /// namelesspace
std::vector<ASTPtr *> JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data & data)
......@@ -101,79 +227,84 @@ std::vector<ASTPtr *> JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data &
return {};
}
void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast, Data & data)
void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr &, Data & data)
{
static String alias_prefix = "__join"; /// FIXME
using RevertedAliases = AsteriskSemantic::RevertedAliases;
if (!needRewrite(select))
return;
auto tables = typeid_cast<const ASTTablesInSelectQuery *>(select.tables.get());
if (!tables)
throw Exception("TablesInSelectQuery expected", ErrorCodes::LOGICAL_ERROR);
ColumnAliasesVisitor::Data aliases_data(getDatabaseAndTables(select, ""));
if (select.select_expression_list)
ColumnAliasesVisitor(aliases_data).visit(select.select_expression_list);
if (select.where_expression)
ColumnAliasesVisitor(aliases_data).visit(select.where_expression);
if (select.prewhere_expression)
ColumnAliasesVisitor(aliases_data).visit(select.prewhere_expression);
if (select.having_expression)
ColumnAliasesVisitor(aliases_data).visit(select.having_expression);
/// JOIN sections
for (auto & child : select.tables->children)
{
auto table = typeid_cast<ASTTablesInSelectQueryElement *>(child.get());
if (table->table_join)
{
auto * join = typeid_cast<ASTTableJoin *>(table->table_join.get());
ColumnAliasesVisitor(aliases_data).visit(join->on_expression);
}
}
size_t num_tables = tables->children.size();
ASTPtr left = tables->children[0];
aliases_data.replaceIdentifiersWithAliases();
for (size_t i = 1; i < num_tables - 1; ++i)
{
ASTPtr right = tables->children[i];
std::vector<String> hidden_names;
appendTableNameAndAlias(hidden_names, left);
appendTableNameAndAlias(hidden_names, right);
auto rev_aliases = std::make_shared<RevertedAliases>();
rev_aliases->swap(aliases_data.rev_aliases);
String subquery_name = alias_prefix + toString(i);
auto & src_tables = select.tables->children;
ASTPtr left_table = src_tables[0];
left = replaceJoin(select, left, right, subquery_name);
if (!left)
return;
for (size_t i = 1; i < src_tables.size() - 1; ++i)
{
left_table = replaceJoin(left_table, src_tables[i]);
if (!left_table)
throw Exception("Cannot replace tables with subselect", ErrorCodes::LOGICAL_ERROR);
//SemanticSelectQuery::hideNames(select, hidden_names, subquery_name);
/// attach data to generated asterisk
AppendSemanticVisitor::Data semantic_data{rev_aliases, false};
AppendSemanticVisitor(semantic_data).visit(left_table);
}
select.tables = std::make_shared<ASTTablesInSelectQuery>();
select.tables->children.push_back(left);
select.tables->children.push_back(tables->children.back());
/// replace tables in select with generated two-table join
RewriteVisitor::Data visitor_data{left_table, src_tables.back()};
RewriteVisitor(visitor_data).visit(select.tables);
ast = ast->clone(); /// rewrite AST in right manner
data.done = true;
}
ASTPtr JoinToSubqueryTransformMatcher::replaceJoin(ASTSelectQuery &, ASTPtr ast_left, ASTPtr ast_right, const String & subquery_alias)
static ASTPtr makeSubqueryTemplate()
{
#if 0
using RewriteMatcher = LinkedMatcher<
OneTypeMatcher<RewriteTablesVisitorData>,
OneTypeMatcher<AppendSemanticVisitorData>>;
#else
using RewriteMatcher = OneTypeMatcher<RewriteTablesVisitorData>;
#endif
using RewriteVisitor = InDepthNodeVisitor<RewriteMatcher, true>;
ParserTablesInSelectQueryElement parser(true);
ASTPtr subquery_template = parseQuery(parser, "(select * from _t)", 0);
if (!subquery_template)
throw Exception("Cannot parse subquery template", ErrorCodes::LOGICAL_ERROR);
return subquery_template;
}
ASTPtr JoinToSubqueryTransformMatcher::replaceJoin(ASTPtr ast_left, ASTPtr ast_right)
{
auto left = typeid_cast<const ASTTablesInSelectQueryElement *>(ast_left.get());
auto right = typeid_cast<const ASTTablesInSelectQueryElement *>(ast_right.get());
if (!left || !right)
throw Exception("Two TablesInSelectQueryElements expected", ErrorCodes::LOGICAL_ERROR);
if (!right->table_join || right->array_join)
return {};
if (!right->table_join)
throw Exception("Table join expected", ErrorCodes::LOGICAL_ERROR);
auto table_join = typeid_cast<const ASTTableJoin *>(right->table_join.get());
if (table_join->kind != ASTTableJoin::Kind::Inner)
return {};
static ASTPtr subquery_template = makeSubqueryTemplate();
ParserTablesInSelectQueryElement parser(true);
String subquery = "(select * from _t) as " + subquery_alias;
ASTPtr res = parseQuery(parser, subquery, 0);
if (!res)
throw Exception("Cannot parse rewrite query", ErrorCodes::LOGICAL_ERROR);
#if 0
RewriteVisitor::Data visitor_data =
std::make_pair<RewriteTablesVisitorData, AppendSemanticVisitorData>({ast_left, ast_right}, {select.semantic});
#else
/// replace '_t' with pair of joined tables
ASTPtr res = subquery_template->clone();
RewriteVisitor::Data visitor_data{ast_left, ast_right};
#endif
RewriteVisitor(visitor_data).visit(res);
return res;
}
......
......@@ -42,7 +42,7 @@ private:
static void visit(ASTSelectQuery & select, ASTPtr & ast, Data & data);
/// @return combined TablesInSelectQueryElement or nullptr if cannot rewrite
static ASTPtr replaceJoin(ASTSelectQuery & select, ASTPtr left, ASTPtr right, const String & subquery_alias);
static ASTPtr replaceJoin(ASTPtr left, ASTPtr right);
};
using JoinToSubqueryTransformVisitor = InDepthNodeVisitor<JoinToSubqueryTransformMatcher, true>;
......
......@@ -2,6 +2,7 @@
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/AsteriskSemantic.h>
#include <Common/typeid_cast.h>
#include <Core/Names.h>
......@@ -141,6 +142,32 @@ std::vector<ASTPtr *> TranslateQualifiedNamesMatcher::visit(ASTSelectQuery & sel
return out;
}
/// qualifed names for duplicates
static std::shared_ptr<ASTIdentifier> makeIdentifier(const String & short_name, const String & long_name, bool need_long_name)
{
if (need_long_name)
return std::make_shared<ASTIdentifier>(long_name);
return std::make_shared<ASTIdentifier>(short_name);
}
static void addIdentifier(ASTs & nodes, std::shared_ptr<ASTIdentifier> identifier, const String & long_name,
AsteriskSemantic::RevertedAliasesPtr aliases)
{
bool added = false;
if (aliases && aliases->count(long_name))
{
for (const String & alias : (*aliases)[long_name])
{
nodes.push_back(identifier->clone());
nodes.back()->setAlias(alias);
added = true;
}
}
if (!added)
nodes.emplace_back(identifier);
}
/// Replace *, alias.*, database.table.* with a list of columns.
void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPtr &, Data & data)
{
......@@ -177,20 +204,21 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
for (const auto & child : old_children)
{
if (typeid_cast<const ASTAsterisk *>(child.get()))
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 & column_name : table_columns)
{
if (first_table || !data.join_using_columns.count(column_name))
{
/// qualifed names for duplicates
if (!first_table && source_columns.count(column_name))
node.children.emplace_back(std::make_shared<ASTIdentifier>(table_name.getQualifiedNamePrefix() + column_name));
else
node.children.emplace_back(std::make_shared<ASTIdentifier>(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));
}
}
first_table = false;
}
......@@ -206,11 +234,10 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
{
for (const auto & column_name : table_columns)
{
/// qualifed names for duplicates
if (!first_table && source_columns.count(column_name))
node.children.emplace_back(std::make_shared<ASTIdentifier>(table_name.getQualifiedNamePrefix() + column_name));
else
node.children.emplace_back(std::make_shared<ASTIdentifier>(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(*qualified_asterisk));
}
break;
}
......
......@@ -6,6 +6,9 @@
namespace DB
{
struct AsteriskSemantic;
struct AsteriskSemanticImpl;
class ASTAsterisk : public IAST
{
public:
......@@ -15,6 +18,11 @@ public:
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
private:
std::shared_ptr<AsteriskSemanticImpl> semantic; /// pimpl
friend struct AsteriskSemantic;
};
}
......@@ -6,6 +6,9 @@
namespace DB
{
struct AsteriskSemantic;
struct AsteriskSemanticImpl;
/** Something like t.*
* It will have qualifier as its child ASTIdentifier.
*/
......@@ -23,6 +26,11 @@ public:
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
private:
std::shared_ptr<AsteriskSemanticImpl> semantic; /// pimpl
friend struct AsteriskSemantic;
};
}
......@@ -231,7 +231,8 @@ static const ASTTablesInSelectQueryElement * getFirstTableJoin(const ASTSelectQu
if (!joined_table)
joined_table = &tables_element;
else
throw Exception("Support for more than one JOIN in query is not implemented", ErrorCodes::NOT_IMPLEMENTED);
throw Exception("Multiple JOIN disabled or does not support the query. "
"'set allow_experimental_multiple_joins_emulation' to enable.", ErrorCodes::NOT_IMPLEMENTED);
}
}
......
......@@ -5,3 +5,34 @@
0 0 0
10 100 1000
20 200 2000
┌─--t1.a─┬─--t2.a─┬─--t2.b─┬─--t3.b─┬─--t3.c─┬─a─┬─b─┬─c─┐
│ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │
└────────┴────────┴────────┴────────┴────────┴───┴───┴───┘
0 0 0 0
6 6 60 60
12 12 120 120
18 18 180 180
0 0 0 0
6 6 60 60
12 12 120 120
18 18 180 180
0 0 0 0
6 6 60 60
12 12 120 120
18 18 180 180
0 0 0 0
6 6 60 60
12 12 120 120
18 18 180 180
0 0 0 0
6 6 60 60
12 12 120 120
18 18 180 180
0 0 0 0
6 6 60 60
12 12 120 120
18 18 180 180
0 0 0 0 0 0 0
6 6 60 60 66 66 120
12 12 120 120 132 132 240
18 18 180 180 198 198 360
......@@ -17,12 +17,68 @@ INSERT INTO table5 SELECT number * 5, number * 50, number * 500 FROM numbers(10)
SET allow_experimental_multiple_joins_emulation = 1;
SELECT 1 LIMIT 0;
select t1.a, t2.b, t3.c from table1 as t1 join table2 as t2 on t1.a = t2.a join table3 as t3 on t2.b = t3.b;
select t1.a, t2.b, t5.c from table1 as t1 join table2 as t2 on t1.a = t2.a join table5 as t5 on t1.a = t5.a AND t2.b = t5.b;
-- FIXME: wrong names qualification
select a, b, c from table1 as t1 join table2 as t2 on t1.a = t2.a join table3 as t3 on b = t3.b;
select a, b, c from table1 as t1 join table2 as t2 on t1.a = t2.a join table5 as t5 on a = t5.a AND b = t5.b;
--select a, b, c from table1 as t1 join table2 as t2 on t1.a = t2.a join table3 as t3 on b = t3.b join table5 as t5 on c = t5.c;
select t1.a, t2.a, t2.b, t3.b, t3.c, t5.a, t5.b, t5.c
from table1 as t1
join table2 as t2 on t1.a = t2.a
join table3 as t3 on t2.b = t3.b
join table5 as t5 on t3.c = t5.c
FORMAT PrettyCompactNoEscapes;
select t1.a as t1_a, t2.a as t2_a, t2.b as t2_b, t3.b as t3_b
from table1 as t1
join table2 as t2 on t1_a = t2_a
join table3 as t3 on t2_b = t3_b;
select t1.a as t1_a, t2.a as t2_a, t2.b as t2_b, t3.b as t3_b
from table1 as t1
join table2 as t2 on t1.a = t2.a
join table3 as t3 on t2.b = t3.b;
select t1.a as t1_a, t2.a as t2_a, t2.b as t2_b, t3.b as t3_b
from table1 as t1
join table2 as t2 on table1.a = table2.a
join table3 as t3 on table2.b = table3.b;
select t1.a, t2.a, t2.b, t3.b
from table1 as t1
join table2 as t2 on table1.a = table2.a
join table3 as t3 on table2.b = table3.b;
select t1.a, t2.a, t2.b, t3.b
from table1 as t1
join table2 as t2 on t1.a = t2.a
join table3 as t3 on t2.b = t3.b;
select table1.a, table2.a, table2.b, table3.b
from table1 as t1
join table2 as t2 on table1.a = table2.a
join table3 as t3 on table2.b = table3.b;
-- TODO
select t1.*, t2.*, t3.*
from table1 as t1
join table2 as t2 on table1.a = table2.a
join table3 as t3 on table2.b = table3.b; -- { serverError 48 }
-- TODO
select *
from table1 as t1
join table2 as t2 on t1.a = t2.a
join table3 as t3 on t2.b = t3.b; -- { serverError 48 }
select t1.a as t1_a, t2.a as t2_a, t2.b as t2_b, t3.b as t3_b,
(t1.a + table2.b) as t1_t2_x, (table1.a + table3.b) as t1_t3_x, (t2.b + t3.b) as t2_t3_x
from table1 as t1
join table2 as t2 on t1_a = t2_a
join table3 as t3 on t2_b = t3_b;
--select (t1.a + table2.b) as t1_t2_x, (table1.a + table3.b) as t1_t3_x, (t2.b + t3.b) as t2_t3_x
--from table1 as t1
--join table2 as t2 on t1_t2_x = t2.a
--join table3 as t3 on t1_t3_x = t2_t3_x;
DROP TABLE table1;
DROP TABLE table2;
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册