mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Implement push-down inside join-statement
This commit is contained in:
parent
0e2316e0ac
commit
09af5512b8
@ -5,7 +5,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
FindIdentifierBestTableData::FindIdentifierBestTableData(const std::vector<DatabaseAndTableWithAlias> & tables_)
|
||||
FindIdentifierBestTableData::FindIdentifierBestTableData(const std::vector<TableWithColumnNames> & tables_)
|
||||
: tables(tables_)
|
||||
{
|
||||
}
|
||||
@ -16,13 +16,21 @@ void FindIdentifierBestTableData::visit(ASTIdentifier & identifier, ASTPtr &)
|
||||
|
||||
if (!identifier.compound())
|
||||
{
|
||||
if (!tables.empty())
|
||||
best_table = &tables[0];
|
||||
for (const auto & [table, names] : tables)
|
||||
{
|
||||
if (std::find(names.begin(), names.end(), identifier.name) != names.end())
|
||||
{
|
||||
// TODO: make sure no collision ever happens
|
||||
if (!best_table)
|
||||
best_table = &table;
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
// FIXME: make a better matcher using `names`?
|
||||
size_t best_match = 0;
|
||||
for (const DatabaseAndTableWithAlias & table : tables)
|
||||
for (const auto & [table, names] : tables)
|
||||
{
|
||||
if (size_t match = IdentifierSemantic::canReferColumnToTable(identifier, table))
|
||||
if (match > best_match)
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -12,10 +13,10 @@ struct FindIdentifierBestTableData
|
||||
using TypeToVisit = ASTIdentifier;
|
||||
using IdentifierWithTable = std::pair<ASTIdentifier *, const DatabaseAndTableWithAlias *>;
|
||||
|
||||
const std::vector<DatabaseAndTableWithAlias> & tables;
|
||||
const std::vector<TableWithColumnNames> & tables;
|
||||
std::vector<IdentifierWithTable> identifier_table;
|
||||
|
||||
FindIdentifierBestTableData(const std::vector<DatabaseAndTableWithAlias> & tables_);
|
||||
FindIdentifierBestTableData(const std::vector<TableWithColumnNames> & tables_);
|
||||
|
||||
void visit(ASTIdentifier & identifier, ASTPtr &);
|
||||
};
|
||||
|
@ -64,13 +64,12 @@ bool PredicateExpressionsOptimizer::optimize()
|
||||
}
|
||||
|
||||
bool PredicateExpressionsOptimizer::optimizeImpl(
|
||||
ASTPtr & outer_expression, SubqueriesProjectionColumns & subqueries_projection_columns, OptimizeKind expression_kind)
|
||||
ASTPtr & outer_expression, const SubqueriesProjectionColumns & subqueries_projection_columns, OptimizeKind expression_kind)
|
||||
{
|
||||
/// split predicate with `and`
|
||||
std::vector<ASTPtr> outer_predicate_expressions = splitConjunctionPredicate(outer_expression);
|
||||
|
||||
std::vector<DatabaseAndTableWithAlias> database_and_table_with_aliases =
|
||||
getDatabaseAndTables(*ast_select, context.getCurrentDatabase());
|
||||
std::vector<TableWithColumnNames> tables_with_columns = getDatabaseAndTablesWithColumnNames(*ast_select, context);
|
||||
|
||||
bool is_rewrite_subquery = false;
|
||||
for (auto & outer_predicate : outer_predicate_expressions)
|
||||
@ -78,7 +77,7 @@ bool PredicateExpressionsOptimizer::optimizeImpl(
|
||||
if (isArrayJoinFunction(outer_predicate))
|
||||
continue;
|
||||
|
||||
auto outer_predicate_dependencies = getDependenciesAndQualifiers(outer_predicate, database_and_table_with_aliases);
|
||||
auto outer_predicate_dependencies = getDependenciesAndQualifiers(outer_predicate, tables_with_columns);
|
||||
|
||||
/// TODO: remove origin expression
|
||||
for (const auto & [subquery, projection_columns] : subqueries_projection_columns)
|
||||
@ -93,7 +92,7 @@ bool PredicateExpressionsOptimizer::optimizeImpl(
|
||||
cleanExpressionAlias(inner_predicate); /// clears the alias name contained in the outer predicate
|
||||
|
||||
std::vector<IdentifierWithQualifier> inner_predicate_dependencies =
|
||||
getDependenciesAndQualifiers(inner_predicate, database_and_table_with_aliases);
|
||||
getDependenciesAndQualifiers(inner_predicate, tables_with_columns);
|
||||
|
||||
setNewAliasesForInnerPredicate(projection_columns, inner_predicate_dependencies);
|
||||
|
||||
@ -170,7 +169,7 @@ std::vector<ASTPtr> PredicateExpressionsOptimizer::splitConjunctionPredicate(AST
|
||||
}
|
||||
|
||||
std::vector<PredicateExpressionsOptimizer::IdentifierWithQualifier>
|
||||
PredicateExpressionsOptimizer::getDependenciesAndQualifiers(ASTPtr & expression, std::vector<DatabaseAndTableWithAlias> & tables)
|
||||
PredicateExpressionsOptimizer::getDependenciesAndQualifiers(ASTPtr & expression, std::vector<TableWithColumnNames> & tables)
|
||||
{
|
||||
FindIdentifierBestTableVisitor::Data find_data(tables);
|
||||
FindIdentifierBestTableVisitor(find_data).visit(expression);
|
||||
|
@ -72,11 +72,11 @@ private:
|
||||
std::vector<ASTPtr> splitConjunctionPredicate(ASTPtr & predicate_expression);
|
||||
|
||||
std::vector<IdentifierWithQualifier> getDependenciesAndQualifiers(ASTPtr & expression,
|
||||
std::vector<DatabaseAndTableWithAlias> & tables_with_aliases);
|
||||
std::vector<TableWithColumnNames> & tables_with_aliases);
|
||||
|
||||
bool optimizeExpression(const ASTPtr & outer_expression, ASTPtr & subquery_expression, ASTSelectQuery * subquery);
|
||||
|
||||
bool optimizeImpl(ASTPtr & outer_expression, SubqueriesProjectionColumns & subqueries_projection_columns, OptimizeKind optimize_kind);
|
||||
bool optimizeImpl(ASTPtr & outer_expression, const SubqueriesProjectionColumns & subqueries_projection_columns, OptimizeKind optimize_kind);
|
||||
|
||||
bool allowPushDown(const ASTSelectQuery * subquery);
|
||||
|
||||
|
@ -16,12 +16,14 @@
|
||||
#include <Interpreters/Settings.h>
|
||||
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
|
||||
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTOrderByElement.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTOrderByElement.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ParserTablesInSelectQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
@ -594,8 +596,28 @@ Names qualifyOccupiedNames(NamesAndTypesList & columns, const NameSet & source_c
|
||||
return originals;
|
||||
}
|
||||
|
||||
void replaceJoinedTable(const ASTTablesInSelectQueryElement* join)
|
||||
{
|
||||
if (!join || !join->table_expression)
|
||||
return;
|
||||
|
||||
auto & table_expr = static_cast<ASTTableExpression &>(*join->table_expression.get());
|
||||
if (table_expr.database_and_table_name)
|
||||
{
|
||||
auto & table_id = typeid_cast<ASTIdentifier &>(*table_expr.database_and_table_name.get());
|
||||
String expr = "(select * from " + table_id.name + ")";
|
||||
if (!table_id.alias.empty())
|
||||
{
|
||||
expr += " as " + table_id.alias;
|
||||
}
|
||||
|
||||
ParserTableExpression parser;
|
||||
table_expr = static_cast<ASTTableExpression &>(*parseQuery(parser, expr, 0));
|
||||
}
|
||||
}
|
||||
|
||||
} // namespace
|
||||
|
||||
|
||||
SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
|
||||
ASTPtr & query,
|
||||
@ -643,6 +665,8 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
|
||||
{
|
||||
if (const ASTTablesInSelectQueryElement * node = select_query->join())
|
||||
{
|
||||
replaceJoinedTable(node);
|
||||
|
||||
const auto & joined_expression = static_cast<const ASTTableExpression &>(*node->table_expression);
|
||||
DatabaseAndTableWithAlias table(joined_expression, context.getCurrentDatabase());
|
||||
|
||||
|
@ -23,7 +23,7 @@ public:
|
||||
bool distinct = false;
|
||||
ASTPtr with_expression_list;
|
||||
ASTPtr select_expression_list;
|
||||
ASTPtr tables;
|
||||
ASTPtr tables; // pointer to TablesInSelectQuery
|
||||
ASTPtr prewhere_expression;
|
||||
ASTPtr where_expression;
|
||||
ASTPtr group_expression_list;
|
||||
|
@ -43,6 +43,10 @@ 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, \n date, \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) \nANY LEFT JOIN \n(\n SELECT *\n FROM test.test \n WHERE id = 1\n) USING (id)\nWHERE 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 toInt8(1) AS id\n) \nANY LEFT JOIN \n(\n SELECT *\n FROM test.test \n WHERE value = 1\n) 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 \n(\n SELECT *\n FROM test.test \n WHERE value = 1\n) 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
|
||||
|
@ -89,8 +89,13 @@ SELECT * FROM (SELECT * FROM test.test UNION ALL SELECT * FROM test.test) WHERE
|
||||
ANALYZE SELECT * FROM (SELECT * FROM test.test) ANY LEFT JOIN (SELECT * FROM test.test) USING id WHERE id = 1;
|
||||
SELECT * FROM (SELECT * FROM test.test) ANY LEFT JOIN (SELECT * FROM test.test) USING id WHERE id = 1;
|
||||
|
||||
ANALYZE SELECT * FROM (SELECT toInt8(1) AS id) ANY LEFT JOIN test.test USING id WHERE value = 1;
|
||||
SELECT * FROM (SELECT toInt8(1) AS id) ANY LEFT JOIN test.test USING id WHERE value = 1;
|
||||
|
||||
ANALYZE SELECT b.value FROM (SELECT toInt8(1) AS id) ANY LEFT JOIN test.test AS b USING id WHERE value = 1;
|
||||
SELECT b.value FROM (SELECT toInt8(1) AS id) ANY LEFT JOIN test.test AS b USING id WHERE value = 1;
|
||||
|
||||
-- Optimize predicate expression with join and nested subquery
|
||||
-- FIXME: should be pushed down to the innermost subqueries in both parts
|
||||
ANALYZE SELECT * FROM (SELECT * FROM (SELECT * FROM test.test) ANY LEFT JOIN (SELECT * FROM test.test) USING id) WHERE id = 1;
|
||||
SELECT * FROM (SELECT * FROM (SELECT * FROM test.test) ANY LEFT JOIN (SELECT * FROM test.test) USING id) WHERE id = 1;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user