mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 00:22:29 +00:00
add IdentifierSemantic and PredicateExpressionsOptimizer refactoring
This commit is contained in:
parent
02f236e457
commit
091efcda95
@ -11,6 +11,7 @@
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/DumpASTNode.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -99,7 +100,7 @@ private:
|
||||
/// @note It expects that only table (not column) identifiers are visited.
|
||||
void visit(const ASTIdentifier & identifier, ASTPtr & ast) const
|
||||
{
|
||||
if (identifier.name_parts.empty())
|
||||
if (!identifier.compound())
|
||||
ast = createTableIdentifier(database_name, identifier.name);
|
||||
}
|
||||
|
||||
|
@ -10,6 +10,7 @@
|
||||
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Interpreters/Aliases.h>
|
||||
|
||||
|
||||
@ -95,7 +96,7 @@ private:
|
||||
NameToNameMap & array_join_alias_to_name = data.array_join_alias_to_name;
|
||||
NameToNameMap & array_join_result_to_source = data.array_join_result_to_source;
|
||||
|
||||
if (!getColumnIdentifierName(node))
|
||||
if (!IdentifierSemantic::getColumnName(node))
|
||||
return;
|
||||
|
||||
auto splitted = Nested::splitName(node.name); /// ParsedParams, Key1
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Interpreters/ColumnNamesContext.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
|
||||
namespace DB
|
||||
@ -31,7 +32,7 @@ bool ColumnNamesContext::addColumnAliasIfAny(const IAST & ast, bool is_public)
|
||||
|
||||
void ColumnNamesContext::addColumnIdentifier(const ASTIdentifier & node, bool is_public)
|
||||
{
|
||||
if (!getColumnIdentifierName(node))
|
||||
if (!IdentifierSemantic::getColumnName(node))
|
||||
return;
|
||||
|
||||
required_names.insert(node.name);
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
@ -11,60 +12,13 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Checks that ast is ASTIdentifier and remove num_qualifiers_to_strip components from left.
|
||||
/// Example: 'database.table.name' -> (num_qualifiers_to_strip = 2) -> 'name'.
|
||||
void stripIdentifier(const DB::ASTPtr & ast, size_t num_qualifiers_to_strip)
|
||||
{
|
||||
ASTIdentifier * identifier = typeid_cast<ASTIdentifier *>(ast.get());
|
||||
|
||||
if (!identifier)
|
||||
throw DB::Exception("ASTIdentifier expected for stripIdentifier", DB::ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (num_qualifiers_to_strip)
|
||||
{
|
||||
identifier->name_parts.erase(identifier->name_parts.begin(), identifier->name_parts.begin() + num_qualifiers_to_strip);
|
||||
DB::String new_name;
|
||||
for (const auto & part : identifier->name_parts)
|
||||
{
|
||||
if (!new_name.empty())
|
||||
new_name += '.';
|
||||
new_name += part;
|
||||
}
|
||||
identifier->name.swap(new_name);
|
||||
}
|
||||
}
|
||||
|
||||
/// Get the number of components of identifier which are correspond to 'alias.', 'table.' or 'databas.table.' from names.
|
||||
size_t getNumComponentsToStripInOrderToTranslateQualifiedName(const ASTIdentifier & identifier,
|
||||
const DatabaseAndTableWithAlias & names)
|
||||
{
|
||||
/// database.table.column
|
||||
if (doesIdentifierBelongTo(identifier, names.database, names.table))
|
||||
return 2;
|
||||
|
||||
/// table.column or alias.column.
|
||||
if (doesIdentifierBelongTo(identifier, names.table) ||
|
||||
doesIdentifierBelongTo(identifier, names.alias))
|
||||
return 1;
|
||||
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
||||
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database)
|
||||
{
|
||||
database = current_database;
|
||||
table = identifier.name;
|
||||
alias = identifier.tryGetAlias();
|
||||
|
||||
if (!identifier.name_parts.empty())
|
||||
{
|
||||
if (identifier.name_parts.size() != 2)
|
||||
throw Exception("Logical error: 2 components expected in table expression '" + identifier.name + "'", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
database = identifier.name_parts[0];
|
||||
table = identifier.name_parts[1];
|
||||
}
|
||||
std::tie(database, table) = IdentifierSemantic::extractDatabaseAndTable(identifier);
|
||||
if (database.empty())
|
||||
database = current_database;
|
||||
}
|
||||
|
||||
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTPtr & node, const String & current_database)
|
||||
@ -108,19 +62,7 @@ String DatabaseAndTableWithAlias::getQualifiedNamePrefix() const
|
||||
{
|
||||
if (alias.empty() && table.empty())
|
||||
return "";
|
||||
|
||||
return (!alias.empty() ? alias : (database + '.' + table)) + '.';
|
||||
}
|
||||
|
||||
void DatabaseAndTableWithAlias::makeQualifiedName(const ASTPtr & ast) const
|
||||
{
|
||||
if (auto identifier = typeid_cast<ASTIdentifier *>(ast.get()))
|
||||
{
|
||||
String prefix = getQualifiedNamePrefix();
|
||||
identifier->name.insert(identifier->name.begin(), prefix.begin(), prefix.end());
|
||||
|
||||
addIdentifierQualifier(*identifier, database, table, alias);
|
||||
}
|
||||
return (!alias.empty() ? alias : table) + '.';
|
||||
}
|
||||
|
||||
std::vector<const ASTTableExpression *> getSelectTablesExpression(const ASTSelectQuery & select_query)
|
||||
|
@ -29,21 +29,13 @@ struct DatabaseAndTableWithAlias
|
||||
DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database = "");
|
||||
DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, const String & current_database);
|
||||
|
||||
/// "alias." or "database.table." if alias is empty
|
||||
/// "alias." or "table." if alias is empty
|
||||
String getQualifiedNamePrefix() const;
|
||||
|
||||
/// If ast is ASTIdentifier, prepend getQualifiedNamePrefix() to it's name.
|
||||
void makeQualifiedName(const ASTPtr & ast) 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);
|
||||
};
|
||||
|
||||
void stripIdentifier(const DB::ASTPtr & ast, size_t num_qualifiers_to_strip);
|
||||
|
||||
size_t getNumComponentsToStripInOrderToTranslateQualifiedName(const ASTIdentifier & identifier,
|
||||
const DatabaseAndTableWithAlias & names);
|
||||
|
||||
std::vector<DatabaseAndTableWithAlias> getDatabaseAndTables(const ASTSelectQuery & select_query, const String & current_database);
|
||||
std::optional<DatabaseAndTableWithAlias> getDatabaseAndTable(const ASTSelectQuery & select, size_t table_number);
|
||||
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -33,7 +34,7 @@ public:
|
||||
private:
|
||||
static std::vector<ASTPtr *> visit(const ASTIdentifier & node, ASTPtr &, Data & data)
|
||||
{
|
||||
if (auto opt_name = getTableIdentifierName(node))
|
||||
if (auto opt_name = IdentifierSemantic::getTableName(node))
|
||||
if (StoragePtr external_storage = data.context.tryGetExternalTable(*opt_name))
|
||||
data.external_tables[*opt_name] = external_storage;
|
||||
return {};
|
||||
|
@ -16,6 +16,7 @@
|
||||
#include <Storages/StorageMemory.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
106
dbms/src/Interpreters/IdentifierSemantic.cpp
Normal file
106
dbms/src/Interpreters/IdentifierSemantic.cpp
Normal file
@ -0,0 +1,106 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
std::optional<String> IdentifierSemantic::getColumnName(const ASTIdentifier & node)
|
||||
{
|
||||
if (!node.semantic->special)
|
||||
return node.name;
|
||||
return {};
|
||||
}
|
||||
|
||||
std::optional<String> IdentifierSemantic::getColumnName(const ASTPtr & ast)
|
||||
{
|
||||
if (ast)
|
||||
if (auto id = typeid_cast<const ASTIdentifier *>(ast.get()))
|
||||
if (!id->semantic->special)
|
||||
return id->name;
|
||||
return {};
|
||||
}
|
||||
|
||||
std::optional<String> IdentifierSemantic::getTableName(const ASTIdentifier & node)
|
||||
{
|
||||
if (node.semantic->special)
|
||||
return node.name;
|
||||
return {};
|
||||
}
|
||||
|
||||
std::optional<String> IdentifierSemantic::getTableName(const ASTPtr & ast)
|
||||
{
|
||||
if (ast)
|
||||
if (auto id = typeid_cast<const ASTIdentifier *>(ast.get()))
|
||||
if (id->semantic->special)
|
||||
return id->name;
|
||||
return {};
|
||||
}
|
||||
|
||||
std::pair<String, String> IdentifierSemantic::extractDatabaseAndTable(const ASTIdentifier & identifier)
|
||||
{
|
||||
if (identifier.name_parts.size() > 2)
|
||||
throw Exception("Logical error: more than two components in table expression", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (identifier.name_parts.size() == 2)
|
||||
return { identifier.name_parts[0], identifier.name_parts[1] };
|
||||
return { "", identifier.name };
|
||||
}
|
||||
|
||||
bool IdentifierSemantic::doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & database, const String & table)
|
||||
{
|
||||
size_t num_components = identifier.name_parts.size();
|
||||
if (num_components >= 3)
|
||||
return identifier.name_parts[0] == database &&
|
||||
identifier.name_parts[1] == table;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool IdentifierSemantic::doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & table)
|
||||
{
|
||||
size_t num_components = identifier.name_parts.size();
|
||||
if (num_components >= 2)
|
||||
return identifier.name_parts[0] == table;
|
||||
return false;
|
||||
}
|
||||
|
||||
size_t IdentifierSemantic::canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
|
||||
{
|
||||
/// database.table.column
|
||||
if (doesIdentifierBelongTo(identifier, db_and_table.database, db_and_table.table))
|
||||
return 2;
|
||||
|
||||
/// table.column or alias.column.
|
||||
if (doesIdentifierBelongTo(identifier, db_and_table.table) ||
|
||||
doesIdentifierBelongTo(identifier, db_and_table.alias))
|
||||
return 1;
|
||||
|
||||
return 0;
|
||||
}
|
||||
|
||||
/// Checks that ast is ASTIdentifier and remove num_qualifiers_to_strip components from left.
|
||||
/// Example: 'database.table.name' -> (num_qualifiers_to_strip = 2) -> 'name'.
|
||||
void IdentifierSemantic::setColumnShortName(ASTIdentifier & identifier, size_t to_strip)
|
||||
{
|
||||
if (!to_strip)
|
||||
return;
|
||||
|
||||
std::vector<String> stripped(identifier.name_parts.begin() + to_strip, identifier.name_parts.end());
|
||||
|
||||
DB::String new_name;
|
||||
for (const auto & part : stripped)
|
||||
{
|
||||
if (!new_name.empty())
|
||||
new_name += '.';
|
||||
new_name += part;
|
||||
}
|
||||
identifier.name.swap(new_name);
|
||||
}
|
||||
|
||||
void IdentifierSemantic::setColumnQualifiedName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
|
||||
{
|
||||
String prefix = db_and_table.getQualifiedNamePrefix();
|
||||
identifier.name.insert(identifier.name.begin(), prefix.begin(), prefix.end());
|
||||
}
|
||||
|
||||
}
|
35
dbms/src/Interpreters/IdentifierSemantic.h
Normal file
35
dbms/src/Interpreters/IdentifierSemantic.h
Normal file
@ -0,0 +1,35 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct IdentifierSemanticImpl
|
||||
{
|
||||
bool special = false;
|
||||
};
|
||||
|
||||
/// Static calss to manipulate IdentifierSemanticImpl via ASTIdentifier
|
||||
struct IdentifierSemantic
|
||||
{
|
||||
/// @returns name for column identifiers
|
||||
static std::optional<String> getColumnName(const ASTIdentifier & node);
|
||||
static std::optional<String> getColumnName(const ASTPtr & ast);
|
||||
|
||||
/// @returns name for 'not a column' identifiers
|
||||
static std::optional<String> getTableName(const ASTIdentifier & node);
|
||||
static std::optional<String> getTableName(const ASTPtr & ast);
|
||||
static std::pair<String, String> extractDatabaseAndTable(const ASTIdentifier & identifier);
|
||||
|
||||
static size_t canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
|
||||
static void setColumnShortName(ASTIdentifier & identifier, size_t match);
|
||||
static void setColumnQualifiedName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
|
||||
|
||||
private:
|
||||
static bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & database, const String & table);
|
||||
static bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & table);
|
||||
};
|
||||
|
||||
}
|
@ -1,6 +1,7 @@
|
||||
#include <Interpreters/InJoinSubqueriesPreprocessor.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Storages/StorageDistributed.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterDescribeQuery.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
@ -83,20 +84,11 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl()
|
||||
}
|
||||
else
|
||||
{
|
||||
auto identifier = typeid_cast<const ASTIdentifier *>(table_expression->database_and_table_name.get());
|
||||
|
||||
String database_name;
|
||||
String table_name;
|
||||
|
||||
auto identifier = typeid_cast<const ASTIdentifier *>(table_expression->database_and_table_name.get());
|
||||
if (identifier->name_parts.size() > 2)
|
||||
throw Exception("Logical error: more than two components in table expression", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (identifier->name_parts.size() > 1)
|
||||
{
|
||||
database_name = identifier->name_parts[0];
|
||||
table_name = identifier->name_parts[1];
|
||||
}
|
||||
else
|
||||
table_name = identifier->name;
|
||||
std::tie(database_name, table_name) = IdentifierSemantic::extractDatabaseAndTable(*identifier);
|
||||
|
||||
table = context.getTable(database_name, table_name);
|
||||
}
|
||||
|
@ -1,12 +1,23 @@
|
||||
#include <iostream>
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Interpreters/PredicateExpressionsOptimizer.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <iostream>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTAsterisk.h>
|
||||
#include <Parsers/ASTQualifiedAsterisk.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/QueryNormalizer.h>
|
||||
#include <Interpreters/QueryAliasesVisitor.h>
|
||||
#include "TranslateQualifiedNamesVisitor.h"
|
||||
@ -22,6 +33,59 @@ namespace ErrorCodes
|
||||
|
||||
static constexpr auto and_function_name = "and";
|
||||
|
||||
|
||||
struct FindIdentifierBestTableData
|
||||
{
|
||||
using TypeToVisit = ASTIdentifier;
|
||||
|
||||
const std::vector<DatabaseAndTableWithAlias> & tables;
|
||||
std::vector<std::pair<ASTIdentifier *, const DatabaseAndTableWithAlias *>> identifier_table;
|
||||
|
||||
FindIdentifierBestTableData(const std::vector<DatabaseAndTableWithAlias> & tables_)
|
||||
: tables(tables_)
|
||||
{}
|
||||
|
||||
void visit(ASTIdentifier & identifier, ASTPtr &)
|
||||
{
|
||||
const DatabaseAndTableWithAlias * best_table = nullptr;
|
||||
|
||||
if (!identifier.compound())
|
||||
{
|
||||
if (!tables.empty())
|
||||
best_table = &tables[0];
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t best_match = 0;
|
||||
for (const DatabaseAndTableWithAlias & table : tables)
|
||||
{
|
||||
if (size_t match = IdentifierSemantic::canReferColumnToTable(identifier, table))
|
||||
if (match > best_match)
|
||||
{
|
||||
best_match = match;
|
||||
best_table = &table;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
identifier_table.emplace_back(&identifier, best_table);
|
||||
}
|
||||
};
|
||||
|
||||
using FindIdentifierBestTableMatcher = OneTypeMatcher<FindIdentifierBestTableData>;
|
||||
using FindIdentifierBestTableVisitor = InDepthNodeVisitor<FindIdentifierBestTableMatcher, true>;
|
||||
|
||||
|
||||
static bool allowPushDown(const ASTSelectQuery * subquery)
|
||||
{
|
||||
return subquery &&
|
||||
!subquery->final() &&
|
||||
!subquery->limit_by_expression_list &&
|
||||
!subquery->limit_length &&
|
||||
!subquery->with_expression_list;
|
||||
}
|
||||
|
||||
|
||||
PredicateExpressionsOptimizer::PredicateExpressionsOptimizer(
|
||||
ASTSelectQuery * ast_select_, ExtractedSettings && settings_, const Context & context_)
|
||||
: ast_select(ast_select_), settings(settings_), context(context_)
|
||||
@ -36,47 +100,50 @@ bool PredicateExpressionsOptimizer::optimize()
|
||||
if (!ast_select->where_expression && !ast_select->prewhere_expression)
|
||||
return false;
|
||||
|
||||
SubqueriesProjectionColumns all_subquery_projection_columns;
|
||||
getAllSubqueryProjectionColumns(all_subquery_projection_columns);
|
||||
SubqueriesProjectionColumns all_subquery_projection_columns = getAllSubqueryProjectionColumns();
|
||||
|
||||
bool is_rewrite_subqueries = false;
|
||||
if (!all_subquery_projection_columns.empty())
|
||||
{
|
||||
is_rewrite_subqueries |= optimizeImpl(ast_select->where_expression, all_subquery_projection_columns, false);
|
||||
is_rewrite_subqueries |= optimizeImpl(ast_select->prewhere_expression, all_subquery_projection_columns, true);
|
||||
is_rewrite_subqueries |= optimizeImpl(ast_select->where_expression, all_subquery_projection_columns, OptimizeKind::PUSH_TO_WHERE);
|
||||
is_rewrite_subqueries |= optimizeImpl(ast_select->prewhere_expression, all_subquery_projection_columns, OptimizeKind::PUSH_TO_PREWHERE);
|
||||
}
|
||||
return is_rewrite_subqueries;
|
||||
}
|
||||
|
||||
bool PredicateExpressionsOptimizer::optimizeImpl(
|
||||
ASTPtr & outer_expression, SubqueriesProjectionColumns & subqueries_projection_columns, bool is_prewhere)
|
||||
ASTPtr & outer_expression, SubqueriesProjectionColumns & subqueries_projection_columns, OptimizeKind expression_kind)
|
||||
{
|
||||
/// split predicate with `and`
|
||||
PredicateExpressions outer_predicate_expressions = splitConjunctionPredicate(outer_expression);
|
||||
std::vector<ASTPtr> outer_predicate_expressions = splitConjunctionPredicate(outer_expression);
|
||||
|
||||
std::vector<DatabaseAndTableWithAlias> database_and_table_with_aliases =
|
||||
getDatabaseAndTables(*ast_select, context.getCurrentDatabase());
|
||||
|
||||
bool is_rewrite_subquery = false;
|
||||
for (const auto & outer_predicate : outer_predicate_expressions)
|
||||
for (auto & outer_predicate : outer_predicate_expressions)
|
||||
{
|
||||
if (isArrayJoinFunction(outer_predicate))
|
||||
continue;
|
||||
|
||||
IdentifiersWithQualifiedNameSet outer_predicate_dependencies;
|
||||
getDependenciesAndQualifiedOfExpression(outer_predicate, outer_predicate_dependencies, database_and_table_with_aliases);
|
||||
auto outer_predicate_dependencies = getDependenciesAndQualifiers(outer_predicate, database_and_table_with_aliases);
|
||||
|
||||
/// TODO: remove origin expression
|
||||
for (const auto & subquery_projection_columns : subqueries_projection_columns)
|
||||
for (const auto & [subquery, projection_columns] : subqueries_projection_columns)
|
||||
{
|
||||
auto subquery = static_cast<ASTSelectQuery *>(subquery_projection_columns.first);
|
||||
const ProjectionsWithAliases projection_columns = subquery_projection_columns.second;
|
||||
|
||||
OptimizeKind optimize_kind = OptimizeKind::NONE;
|
||||
if (!cannotPushDownOuterPredicate(projection_columns, subquery, outer_predicate_dependencies, is_prewhere, optimize_kind))
|
||||
if (allowPushDown(subquery) && canPushDownOuterPredicate(projection_columns, outer_predicate_dependencies, optimize_kind))
|
||||
{
|
||||
ASTPtr inner_predicate;
|
||||
cloneOuterPredicateForInnerPredicate(outer_predicate, projection_columns, database_and_table_with_aliases, inner_predicate);
|
||||
if (optimize_kind == OptimizeKind::NONE)
|
||||
optimize_kind = expression_kind;
|
||||
|
||||
ASTPtr inner_predicate = outer_predicate->clone();
|
||||
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);
|
||||
|
||||
setNewAliasesForInnerPredicate(projection_columns, inner_predicate_dependencies);
|
||||
|
||||
switch (optimize_kind)
|
||||
{
|
||||
@ -91,9 +158,9 @@ bool PredicateExpressionsOptimizer::optimizeImpl(
|
||||
return is_rewrite_subquery;
|
||||
}
|
||||
|
||||
PredicateExpressions PredicateExpressionsOptimizer::splitConjunctionPredicate(ASTPtr & predicate_expression)
|
||||
std::vector<ASTPtr> PredicateExpressionsOptimizer::splitConjunctionPredicate(ASTPtr & predicate_expression)
|
||||
{
|
||||
PredicateExpressions predicate_expressions;
|
||||
std::vector<ASTPtr> predicate_expressions;
|
||||
|
||||
if (predicate_expression)
|
||||
{
|
||||
@ -127,77 +194,79 @@ PredicateExpressions PredicateExpressionsOptimizer::splitConjunctionPredicate(AS
|
||||
return predicate_expressions;
|
||||
}
|
||||
|
||||
void PredicateExpressionsOptimizer::getDependenciesAndQualifiedOfExpression(const ASTPtr & expression,
|
||||
IdentifiersWithQualifiedNameSet & dependencies_and_qualified,
|
||||
std::vector<DatabaseAndTableWithAlias> & tables_with_aliases)
|
||||
std::vector<PredicateExpressionsOptimizer::IdentifierWithQualifier>
|
||||
PredicateExpressionsOptimizer::getDependenciesAndQualifiers(ASTPtr & expression, std::vector<DatabaseAndTableWithAlias> & tables)
|
||||
{
|
||||
if (const auto identifier = typeid_cast<ASTIdentifier *>(expression.get()))
|
||||
FindIdentifierBestTableVisitor::Data find_data(tables);
|
||||
FindIdentifierBestTableVisitor(find_data).visit(expression);
|
||||
|
||||
std::vector<IdentifierWithQualifier> dependencies;
|
||||
|
||||
for (const auto & [identifier, table] : find_data.identifier_table)
|
||||
{
|
||||
String table_alias;
|
||||
if (!identifier->name_parts.empty())
|
||||
{
|
||||
if (!tables_with_aliases.empty())
|
||||
table_alias = tables_with_aliases[0].getQualifiedNamePrefix();
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t best_table_pos = 0;
|
||||
size_t max_num_qualifiers_to_strip = 0;
|
||||
if (table)
|
||||
table_alias = table->getQualifiedNamePrefix();
|
||||
|
||||
/// translate qualifiers for dependent columns
|
||||
for (size_t table_pos = 0; table_pos < tables_with_aliases.size(); ++table_pos)
|
||||
{
|
||||
const auto & table = tables_with_aliases[table_pos];
|
||||
auto num_qualifiers_to_strip = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, table);
|
||||
|
||||
if (num_qualifiers_to_strip > max_num_qualifiers_to_strip)
|
||||
{
|
||||
max_num_qualifiers_to_strip = num_qualifiers_to_strip;
|
||||
best_table_pos = table_pos;
|
||||
}
|
||||
}
|
||||
|
||||
table_alias = tables_with_aliases[best_table_pos].getQualifiedNamePrefix();
|
||||
}
|
||||
|
||||
String qualified_name = table_alias + expression->getAliasOrColumnName();
|
||||
dependencies_and_qualified.emplace_back(std::pair(identifier, qualified_name));
|
||||
}
|
||||
else
|
||||
{
|
||||
for (const auto & child : expression->children)
|
||||
getDependenciesAndQualifiedOfExpression(child, dependencies_and_qualified, tables_with_aliases);
|
||||
dependencies.emplace_back(identifier, table_alias);
|
||||
}
|
||||
|
||||
return dependencies;
|
||||
}
|
||||
|
||||
bool PredicateExpressionsOptimizer::cannotPushDownOuterPredicate(
|
||||
const ProjectionsWithAliases & subquery_projection_columns, ASTSelectQuery * subquery,
|
||||
IdentifiersWithQualifiedNameSet & outer_predicate_dependencies, bool & is_prewhere, OptimizeKind & optimize_kind)
|
||||
static String qualifiedName(ASTIdentifier * identifier, const String & prefix)
|
||||
{
|
||||
if (subquery->final() || subquery->limit_by_expression_list || subquery->limit_length || subquery->with_expression_list)
|
||||
return true;
|
||||
if (identifier->isShort())
|
||||
return prefix + identifier->getAliasOrColumnName();
|
||||
return identifier->getAliasOrColumnName();
|
||||
}
|
||||
|
||||
for (auto & predicate_dependency : outer_predicate_dependencies)
|
||||
bool PredicateExpressionsOptimizer::canPushDownOuterPredicate(
|
||||
const std::vector<ProjectionWithAlias> & projection_columns,
|
||||
const std::vector<IdentifierWithQualifier> & dependencies,
|
||||
OptimizeKind & optimize_kind)
|
||||
{
|
||||
for (const auto & [identifier, prefix] : dependencies)
|
||||
{
|
||||
bool is_found = false;
|
||||
String qualified_name = qualifiedName(identifier, prefix);
|
||||
|
||||
for (auto projection_column : subquery_projection_columns)
|
||||
for (const auto & [ast, alias] : projection_columns)
|
||||
{
|
||||
if (projection_column.second == predicate_dependency.second)
|
||||
if (alias == qualified_name)
|
||||
{
|
||||
is_found = true;
|
||||
optimize_kind = isAggregateFunction(projection_column.first) ? OptimizeKind::PUSH_TO_HAVING : optimize_kind;
|
||||
if (isAggregateFunction(ast))
|
||||
optimize_kind = OptimizeKind::PUSH_TO_HAVING;
|
||||
}
|
||||
}
|
||||
|
||||
if (!is_found)
|
||||
return true;
|
||||
return false;
|
||||
}
|
||||
|
||||
if (optimize_kind == OptimizeKind::NONE)
|
||||
optimize_kind = is_prewhere ? OptimizeKind::PUSH_TO_PREWHERE : OptimizeKind::PUSH_TO_WHERE;
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
void PredicateExpressionsOptimizer::setNewAliasesForInnerPredicate(
|
||||
const std::vector<ProjectionWithAlias> & projection_columns,
|
||||
const std::vector<IdentifierWithQualifier> & dependencies)
|
||||
{
|
||||
for (auto & [identifier, prefix] : dependencies)
|
||||
{
|
||||
String qualified_name = qualifiedName(identifier, prefix);
|
||||
|
||||
for (auto & [ast, alias] : projection_columns)
|
||||
{
|
||||
if (alias == qualified_name)
|
||||
{
|
||||
if (!isIdentifier(ast) && ast->tryGetAlias().empty())
|
||||
ast->setAlias(ast->getColumnName());
|
||||
|
||||
identifier->resetWithAlias(ast->getAliasOrColumnName());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
bool PredicateExpressionsOptimizer::isArrayJoinFunction(const ASTPtr & node)
|
||||
@ -215,47 +284,21 @@ bool PredicateExpressionsOptimizer::isArrayJoinFunction(const ASTPtr & node)
|
||||
return false;
|
||||
}
|
||||
|
||||
bool PredicateExpressionsOptimizer::isAggregateFunction(ASTPtr & node)
|
||||
bool PredicateExpressionsOptimizer::isAggregateFunction(const ASTPtr & node)
|
||||
{
|
||||
if (auto function = typeid_cast<ASTFunction *>(node.get()))
|
||||
if (auto function = typeid_cast<const ASTFunction *>(node.get()))
|
||||
{
|
||||
if (AggregateFunctionFactory::instance().isAggregateFunctionName(function->name))
|
||||
return true;
|
||||
}
|
||||
|
||||
for (auto & child : node->children)
|
||||
for (const auto & child : node->children)
|
||||
if (isAggregateFunction(child))
|
||||
return true;
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
void PredicateExpressionsOptimizer::cloneOuterPredicateForInnerPredicate(
|
||||
const ASTPtr & outer_predicate, const ProjectionsWithAliases & projection_columns,
|
||||
std::vector<DatabaseAndTableWithAlias> & tables, ASTPtr & inner_predicate)
|
||||
{
|
||||
inner_predicate = outer_predicate->clone();
|
||||
|
||||
/// clears the alias name contained in the outer predicate
|
||||
cleanExpressionAlias(inner_predicate);
|
||||
IdentifiersWithQualifiedNameSet new_expression_requires;
|
||||
getDependenciesAndQualifiedOfExpression(inner_predicate, new_expression_requires, tables);
|
||||
|
||||
for (auto & require : new_expression_requires)
|
||||
{
|
||||
for (auto projection : projection_columns)
|
||||
{
|
||||
if (require.second == projection.second)
|
||||
{
|
||||
ASTPtr & ast = projection.first;
|
||||
if (!typeid_cast<ASTIdentifier *>(ast.get()) && ast->tryGetAlias().empty())
|
||||
ast->setAlias(ast->getColumnName());
|
||||
require.first->name = ast->getAliasOrColumnName();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
bool PredicateExpressionsOptimizer::optimizeExpression(const ASTPtr & outer_expression, ASTPtr & subquery_expression, ASTSelectQuery * subquery)
|
||||
{
|
||||
ASTPtr new_subquery_expression = subquery_expression;
|
||||
@ -272,33 +315,32 @@ bool PredicateExpressionsOptimizer::optimizeExpression(const ASTPtr & outer_expr
|
||||
return true;
|
||||
}
|
||||
|
||||
void PredicateExpressionsOptimizer::getAllSubqueryProjectionColumns(SubqueriesProjectionColumns & all_subquery_projection_columns)
|
||||
PredicateExpressionsOptimizer::SubqueriesProjectionColumns PredicateExpressionsOptimizer::getAllSubqueryProjectionColumns()
|
||||
{
|
||||
const auto tables_expression = getSelectTablesExpression(*ast_select);
|
||||
SubqueriesProjectionColumns projection_columns;
|
||||
|
||||
for (const auto & table_expression : tables_expression)
|
||||
{
|
||||
for (const auto & table_expression : getSelectTablesExpression(*ast_select))
|
||||
if (table_expression->subquery)
|
||||
{
|
||||
/// Use qualifiers to translate the columns of subqueries
|
||||
DatabaseAndTableWithAlias database_and_table_with_alias(*table_expression, context.getCurrentDatabase());
|
||||
String qualified_name_prefix = database_and_table_with_alias.getQualifiedNamePrefix();
|
||||
getSubqueryProjectionColumns(all_subquery_projection_columns, qualified_name_prefix,
|
||||
static_cast<const ASTSubquery *>(table_expression->subquery.get())->children[0]);
|
||||
}
|
||||
}
|
||||
getSubqueryProjectionColumns(table_expression->subquery, projection_columns);
|
||||
|
||||
return projection_columns;
|
||||
}
|
||||
|
||||
void PredicateExpressionsOptimizer::getSubqueryProjectionColumns(SubqueriesProjectionColumns & all_subquery_projection_columns,
|
||||
String & qualified_name_prefix, const ASTPtr & subquery)
|
||||
void PredicateExpressionsOptimizer::getSubqueryProjectionColumns(const ASTPtr & subquery, SubqueriesProjectionColumns & projection_columns)
|
||||
{
|
||||
ASTs select_with_union_projections;
|
||||
auto select_with_union_query = static_cast<ASTSelectWithUnionQuery *>(subquery.get());
|
||||
String qualified_name_prefix = subquery->tryGetAlias();
|
||||
if (!qualified_name_prefix.empty())
|
||||
qualified_name_prefix += '.';
|
||||
|
||||
for (auto & select_without_union_query : select_with_union_query->list_of_selects->children)
|
||||
const ASTPtr & subselect = subquery->children[0];
|
||||
|
||||
ASTs select_with_union_projections;
|
||||
auto select_with_union_query = static_cast<ASTSelectWithUnionQuery *>(subselect.get());
|
||||
|
||||
for (auto & select : select_with_union_query->list_of_selects->children)
|
||||
{
|
||||
ProjectionsWithAliases subquery_projections;
|
||||
auto select_projection_columns = getSelectQueryProjectionColumns(select_without_union_query);
|
||||
std::vector<ProjectionWithAlias> subquery_projections;
|
||||
auto select_projection_columns = getSelectQueryProjectionColumns(select);
|
||||
|
||||
if (!select_projection_columns.empty())
|
||||
{
|
||||
@ -309,7 +351,7 @@ void PredicateExpressionsOptimizer::getSubqueryProjectionColumns(SubqueriesProje
|
||||
subquery_projections.emplace_back(std::pair(select_projection_columns[i],
|
||||
qualified_name_prefix + select_with_union_projections[i]->getAliasOrColumnName()));
|
||||
|
||||
all_subquery_projection_columns.insert(std::pair(select_without_union_query.get(), subquery_projections));
|
||||
projection_columns.insert(std::pair(static_cast<ASTSelectQuery *>(select.get()), subquery_projections));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -1,26 +1,14 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using PredicateExpressions = std::vector<ASTPtr>;
|
||||
using ProjectionWithAlias = std::pair<ASTPtr, String>;
|
||||
using ProjectionsWithAliases = std::vector<ProjectionWithAlias>;
|
||||
using SubqueriesProjectionColumns = std::map<IAST *, ProjectionsWithAliases>;
|
||||
using IdentifierWithQualifiedName = std::pair<ASTIdentifier *, String>;
|
||||
using IdentifiersWithQualifiedNameSet = std::vector<IdentifierWithQualifiedName>;
|
||||
|
||||
class ASTIdentifier;
|
||||
class ASTSelectQuery;
|
||||
class ASTSubquery;
|
||||
class Context;
|
||||
|
||||
/** This class provides functions for Push-Down predicate expressions
|
||||
*
|
||||
@ -35,6 +23,10 @@ using IdentifiersWithQualifiedNameSet = std::vector<IdentifierWithQualifiedName>
|
||||
*/
|
||||
class PredicateExpressionsOptimizer
|
||||
{
|
||||
using ProjectionWithAlias = std::pair<ASTPtr, String>;
|
||||
using SubqueriesProjectionColumns = std::map<ASTSelectQuery *, std::vector<ProjectionWithAlias>>;
|
||||
using IdentifierWithQualifier = std::pair<ASTIdentifier *, String>;
|
||||
|
||||
/// Extracts settings, mostly to show which are used and which are not.
|
||||
struct ExtractedSettings
|
||||
{
|
||||
@ -73,29 +65,29 @@ private:
|
||||
PUSH_TO_HAVING,
|
||||
};
|
||||
|
||||
bool isAggregateFunction(ASTPtr & node);
|
||||
bool isAggregateFunction(const ASTPtr & node);
|
||||
|
||||
bool isArrayJoinFunction(const ASTPtr & node);
|
||||
|
||||
PredicateExpressions splitConjunctionPredicate(ASTPtr & predicate_expression);
|
||||
std::vector<ASTPtr> splitConjunctionPredicate(ASTPtr & predicate_expression);
|
||||
|
||||
void getDependenciesAndQualifiedOfExpression(const ASTPtr & expression, IdentifiersWithQualifiedNameSet & dependencies_and_qualified,
|
||||
std::vector<DatabaseAndTableWithAlias> & tables_with_aliases);
|
||||
std::vector<IdentifierWithQualifier> getDependenciesAndQualifiers(ASTPtr & expression,
|
||||
std::vector<DatabaseAndTableWithAlias> & tables_with_aliases);
|
||||
|
||||
bool optimizeExpression(const ASTPtr & outer_expression, ASTPtr & subquery_expression, ASTSelectQuery * subquery);
|
||||
|
||||
bool optimizeImpl(ASTPtr & outer_expression, SubqueriesProjectionColumns & subqueries_projection_columns, bool is_prewhere);
|
||||
bool optimizeImpl(ASTPtr & outer_expression, SubqueriesProjectionColumns & subqueries_projection_columns, OptimizeKind optimize_kind);
|
||||
|
||||
bool cannotPushDownOuterPredicate(const ProjectionsWithAliases & subquery_projection_columns, ASTSelectQuery * subquery,
|
||||
IdentifiersWithQualifiedNameSet & outer_predicate_dependencies, bool & is_prewhere, OptimizeKind & optimize_kind);
|
||||
bool canPushDownOuterPredicate(const std::vector<ProjectionWithAlias> & subquery_projection_columns,
|
||||
const std::vector<IdentifierWithQualifier> & outer_predicate_dependencies,
|
||||
OptimizeKind & optimize_kind);
|
||||
|
||||
void cloneOuterPredicateForInnerPredicate(const ASTPtr & outer_predicate, const ProjectionsWithAliases & projection_columns,
|
||||
std::vector<DatabaseAndTableWithAlias> & tables, ASTPtr & inner_predicate);
|
||||
void setNewAliasesForInnerPredicate(const std::vector<ProjectionWithAlias> & projection_columns,
|
||||
const std::vector<IdentifierWithQualifier> & inner_predicate_dependencies);
|
||||
|
||||
void getAllSubqueryProjectionColumns(SubqueriesProjectionColumns & all_subquery_projection_columns);
|
||||
SubqueriesProjectionColumns getAllSubqueryProjectionColumns();
|
||||
|
||||
void getSubqueryProjectionColumns(SubqueriesProjectionColumns & all_subquery_projection_columns,
|
||||
String & qualified_name_prefix, const ASTPtr & subquery);
|
||||
void getSubqueryProjectionColumns(const ASTPtr & subquery, SubqueriesProjectionColumns & all_subquery_projection_columns);
|
||||
|
||||
ASTs getSelectQueryProjectionColumns(ASTPtr & ast);
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Poco/String.h>
|
||||
#include <Core/Names.h>
|
||||
#include <Interpreters/QueryNormalizer.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/AnalyzedJoin.h>
|
||||
#include <Parsers/ASTAsterisk.h>
|
||||
@ -107,7 +108,7 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data)
|
||||
auto & current_asts = data.current_asts;
|
||||
String & current_alias = data.current_alias;
|
||||
|
||||
if (!getColumnIdentifierName(node))
|
||||
if (!IdentifierSemantic::getColumnName(node))
|
||||
return;
|
||||
|
||||
/// If it is an alias, but not a parent alias (for constructs like "SELECT column + 1 AS column").
|
||||
@ -124,7 +125,7 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data)
|
||||
if (!my_alias.empty() && my_alias != alias_node->getAliasOrColumnName())
|
||||
{
|
||||
/// Avoid infinite recursion here
|
||||
auto opt_name = getColumnIdentifierName(alias_node);
|
||||
auto opt_name = IdentifierSemantic::getColumnName(alias_node);
|
||||
bool is_cycle = opt_name && *opt_name == node.name;
|
||||
|
||||
if (!is_cycle)
|
||||
@ -273,8 +274,7 @@ void QueryNormalizer::visitChildren(const ASTPtr & node, Data & data)
|
||||
visit(child, data);
|
||||
}
|
||||
}
|
||||
else if (!typeid_cast<ASTIdentifier *>(node.get()) &&
|
||||
!typeid_cast<ASTSelectQuery *>(node.get()))
|
||||
else if (!typeid_cast<ASTSelectQuery *>(node.get()))
|
||||
{
|
||||
for (auto & child : node->children)
|
||||
{
|
||||
|
@ -46,8 +46,7 @@ bool RequiredSourceColumnsMatcher::needChildVisit(ASTPtr & node, const ASTPtr &
|
||||
return false;
|
||||
|
||||
/// Processed. Do not need children.
|
||||
if (typeid_cast<ASTIdentifier *>(node.get()) ||
|
||||
typeid_cast<ASTTableExpression *>(node.get()) ||
|
||||
if (typeid_cast<ASTTableExpression *>(node.get()) ||
|
||||
typeid_cast<ASTArrayJoin *>(node.get()) ||
|
||||
typeid_cast<ASTSelectQuery *>(node.get()))
|
||||
return false;
|
||||
|
@ -18,6 +18,7 @@
|
||||
#include <Parsers/ASTOrderByElement.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
@ -477,20 +478,18 @@ void collectJoinedColumnsFromJoinOnExpr(AnalyzedJoin & analyzed_join, const ASTS
|
||||
std::function<TableBelonging(const ASTPtr &)> get_table_belonging;
|
||||
get_table_belonging = [&](const ASTPtr & ast) -> TableBelonging
|
||||
{
|
||||
if (getColumnIdentifierName(ast))
|
||||
if (IdentifierSemantic::getColumnName(ast))
|
||||
{
|
||||
auto * identifier = typeid_cast<const ASTIdentifier *>(ast.get());
|
||||
|
||||
{
|
||||
auto left_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, left_source_names);
|
||||
auto right_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, right_source_names);
|
||||
size_t left_match_degree = IdentifierSemantic::canReferColumnToTable(*identifier, left_source_names);
|
||||
size_t right_match_degree = IdentifierSemantic::canReferColumnToTable(*identifier, right_source_names);
|
||||
|
||||
if (left_match_degree > right_match_degree)
|
||||
return {identifier, nullptr};
|
||||
if (left_match_degree < right_match_degree)
|
||||
return {nullptr, identifier};
|
||||
|
||||
/// Assume that component from definite table if num_components is greater than for the other table.
|
||||
if (left_num_components > right_num_components)
|
||||
return {identifier, nullptr};
|
||||
if (left_num_components < right_num_components)
|
||||
return {nullptr, identifier};
|
||||
}
|
||||
return {};
|
||||
}
|
||||
|
||||
@ -516,19 +515,15 @@ void collectJoinedColumnsFromJoinOnExpr(AnalyzedJoin & analyzed_join, const ASTS
|
||||
std::function<void(ASTPtr &, const DatabaseAndTableWithAlias &, bool)> translate_qualified_names;
|
||||
translate_qualified_names = [&](ASTPtr & ast, const DatabaseAndTableWithAlias & source_names, bool right_table)
|
||||
{
|
||||
if (getColumnIdentifierName(ast))
|
||||
if (IdentifierSemantic::getColumnName(ast))
|
||||
{
|
||||
auto * identifier = typeid_cast<const ASTIdentifier *>(ast.get());
|
||||
auto * identifier = typeid_cast<ASTIdentifier *>(ast.get());
|
||||
|
||||
{
|
||||
auto num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, source_names);
|
||||
stripIdentifier(ast, num_components);
|
||||
size_t match = IdentifierSemantic::canReferColumnToTable(*identifier, source_names);
|
||||
IdentifierSemantic::setColumnShortName(*identifier, match);
|
||||
|
||||
if (right_table && source_columns.count(ast->getColumnName()))
|
||||
source_names.makeQualifiedName(ast);
|
||||
|
||||
}
|
||||
return;
|
||||
if (right_table && source_columns.count(ast->getColumnName()))
|
||||
IdentifierSemantic::setColumnQualifiedName(*identifier, source_names);
|
||||
}
|
||||
|
||||
for (auto & child : ast->children)
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Core/Names.h>
|
||||
@ -28,8 +29,7 @@ bool TranslateQualifiedNamesMatcher::needChildVisit(ASTPtr & node, const ASTPtr
|
||||
return false;
|
||||
|
||||
/// Processed nodes. Do not go into children.
|
||||
if (typeid_cast<ASTIdentifier *>(node.get()) ||
|
||||
typeid_cast<ASTQualifiedAsterisk *>(node.get()) ||
|
||||
if (typeid_cast<ASTQualifiedAsterisk *>(node.get()) ||
|
||||
typeid_cast<ASTTableJoin *>(node.get()))
|
||||
return false;
|
||||
|
||||
@ -50,38 +50,25 @@ std::vector<ASTPtr *> TranslateQualifiedNamesMatcher::visit(ASTPtr & ast, Data &
|
||||
return {};
|
||||
}
|
||||
|
||||
std::vector<ASTPtr *> TranslateQualifiedNamesMatcher::visit(const ASTIdentifier & identifier, ASTPtr & ast, Data & data)
|
||||
std::vector<ASTPtr *> TranslateQualifiedNamesMatcher::visit(ASTIdentifier & identifier, ASTPtr & ast, Data & data)
|
||||
{
|
||||
const NameSet & source_columns = data.source_columns;
|
||||
const std::vector<DatabaseAndTableWithAlias> & tables = data.tables;
|
||||
|
||||
if (getColumnIdentifierName(identifier))
|
||||
if (IdentifierSemantic::getColumnName(identifier))
|
||||
{
|
||||
/// Select first table name with max number of qualifiers which can be stripped.
|
||||
size_t max_num_qualifiers_to_strip = 0;
|
||||
size_t best_table_pos = 0;
|
||||
bool best_table_pos = 0;
|
||||
size_t best_match = 0;
|
||||
for (size_t i = 0; i < data.tables.size(); ++i)
|
||||
if (size_t match = IdentifierSemantic::canReferColumnToTable(identifier, data.tables[i]))
|
||||
if (match > best_match)
|
||||
{
|
||||
best_match = match;
|
||||
best_table_pos = i;
|
||||
}
|
||||
|
||||
for (size_t table_pos = 0; table_pos < tables.size(); ++table_pos)
|
||||
{
|
||||
const auto & table = tables[table_pos];
|
||||
auto num_qualifiers_to_strip = getNumComponentsToStripInOrderToTranslateQualifiedName(identifier, table);
|
||||
|
||||
if (num_qualifiers_to_strip > max_num_qualifiers_to_strip)
|
||||
{
|
||||
max_num_qualifiers_to_strip = num_qualifiers_to_strip;
|
||||
best_table_pos = table_pos;
|
||||
}
|
||||
}
|
||||
|
||||
if (max_num_qualifiers_to_strip)
|
||||
stripIdentifier(ast, max_num_qualifiers_to_strip);
|
||||
IdentifierSemantic::setColumnShortName(identifier, best_match);
|
||||
|
||||
/// In case if column from the joined table are in source columns, change it's name to qualified.
|
||||
if (best_table_pos && source_columns.count(ast->getColumnName()))
|
||||
{
|
||||
const DatabaseAndTableWithAlias & table = tables[best_table_pos];
|
||||
table.makeQualifiedName(ast);
|
||||
}
|
||||
if (best_table_pos && data.source_columns.count(ast->getColumnName()))
|
||||
IdentifierSemantic::setColumnQualifiedName(identifier, data.tables[best_table_pos]);
|
||||
}
|
||||
|
||||
return {};
|
||||
|
@ -30,7 +30,7 @@ public:
|
||||
static bool needChildVisit(ASTPtr & node, const ASTPtr & child);
|
||||
|
||||
private:
|
||||
static std::vector<ASTPtr *> visit(const ASTIdentifier & node, ASTPtr & ast, Data &);
|
||||
static std::vector<ASTPtr *> visit(ASTIdentifier & node, ASTPtr & ast, Data &);
|
||||
static std::vector<ASTPtr *> visit(const ASTQualifiedAsterisk & node, const ASTPtr & ast, Data &);
|
||||
static std::vector<ASTPtr *> visit(ASTTableJoin & node, const ASTPtr & ast, Data &);
|
||||
static std::vector<ASTPtr *> visit(ASTSelectQuery & node, const ASTPtr & ast, Data &);
|
||||
|
@ -1,12 +1,27 @@
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
std::shared_ptr<ASTIdentifier> ASTIdentifier::createSpecial(const String & name, std::vector<String> && name_parts)
|
||||
{
|
||||
auto ret = std::make_shared<ASTIdentifier>(name, std::move(name_parts));
|
||||
ret->semantic->special = true;
|
||||
return ret;
|
||||
}
|
||||
|
||||
ASTIdentifier::ASTIdentifier(const String & name_, std::vector<String> && name_parts_)
|
||||
: name(name_)
|
||||
, name_parts(name_parts_)
|
||||
, semantic(std::make_shared<IdentifierSemanticImpl>())
|
||||
{
|
||||
}
|
||||
|
||||
void ASTIdentifier::formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const
|
||||
{
|
||||
auto format_element = [&](const String & elem_name)
|
||||
@ -74,74 +89,11 @@ bool getIdentifierName(const ASTPtr & ast, String & name)
|
||||
return false;
|
||||
}
|
||||
|
||||
std::optional<String> getColumnIdentifierName(const ASTIdentifier & node)
|
||||
{
|
||||
if (!node.special)
|
||||
return node.name;
|
||||
return {};
|
||||
}
|
||||
|
||||
std::optional<String> getColumnIdentifierName(const ASTPtr & ast)
|
||||
{
|
||||
if (ast)
|
||||
if (auto id = typeid_cast<const ASTIdentifier *>(ast.get()))
|
||||
if (!id->special)
|
||||
return id->name;
|
||||
return {};
|
||||
}
|
||||
|
||||
std::optional<String> getTableIdentifierName(const ASTIdentifier & node)
|
||||
{
|
||||
if (node.special)
|
||||
return node.name;
|
||||
return {};
|
||||
}
|
||||
|
||||
std::optional<String> getTableIdentifierName(const ASTPtr & ast)
|
||||
{
|
||||
if (ast)
|
||||
if (auto id = typeid_cast<const ASTIdentifier *>(ast.get()))
|
||||
if (id->special)
|
||||
return id->name;
|
||||
return {};
|
||||
}
|
||||
|
||||
void setIdentifierSpecial(ASTPtr & ast)
|
||||
{
|
||||
if (ast)
|
||||
if (ASTIdentifier * id = typeid_cast<ASTIdentifier *>(ast.get()))
|
||||
id->setSpecial();
|
||||
}
|
||||
|
||||
void addIdentifierQualifier(ASTIdentifier & identifier, const String & database, const String & table, const String & alias)
|
||||
{
|
||||
if (!alias.empty())
|
||||
{
|
||||
identifier.name_parts.emplace_back(alias);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!database.empty())
|
||||
identifier.name_parts.emplace_back(database);
|
||||
identifier.name_parts.emplace_back(table);
|
||||
}
|
||||
}
|
||||
|
||||
bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & database, const String & table)
|
||||
{
|
||||
size_t num_components = identifier.name_parts.size();
|
||||
if (num_components >= 3)
|
||||
return identifier.name_parts[0] == database &&
|
||||
identifier.name_parts[1] == table;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & table)
|
||||
{
|
||||
size_t num_components = identifier.name_parts.size();
|
||||
if (num_components >= 2)
|
||||
return identifier.name_parts[0] == table;
|
||||
return false;
|
||||
id->semantic->special = true;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -8,6 +8,11 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct IdentifierSemantic;
|
||||
struct IdentifierSemanticImpl;
|
||||
struct DatabaseAndTableWithAlias;
|
||||
|
||||
|
||||
/// Identifier (column, table or alias)
|
||||
class ASTIdentifier : public ASTWithAlias
|
||||
{
|
||||
@ -15,14 +20,8 @@ public:
|
||||
/// The composite identifier will have a concatenated name (of the form a.b.c),
|
||||
/// and individual components will be available inside the name_parts.
|
||||
String name;
|
||||
std::vector<String> name_parts;
|
||||
|
||||
ASTIdentifier(const String & name_, std::vector<String> && name_parts_ = {})
|
||||
: name(name_)
|
||||
, name_parts(name_parts_)
|
||||
, special(false)
|
||||
{
|
||||
}
|
||||
ASTIdentifier(const String & name_, std::vector<String> && name_parts_ = {});
|
||||
|
||||
/** Get the text that identifies this element. */
|
||||
String getID(char delim) const override { return "Identifier" + (delim + name); }
|
||||
@ -34,6 +33,15 @@ public:
|
||||
set.insert(name);
|
||||
}
|
||||
|
||||
bool compound() const { return !name_parts.empty(); }
|
||||
bool isShort() const { return name_parts.empty() || name == name_parts.back(); }
|
||||
|
||||
void resetWithAlias(const String & new_name)
|
||||
{
|
||||
name = new_name;
|
||||
name_parts.clear();
|
||||
}
|
||||
|
||||
protected:
|
||||
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||
void appendColumnNameImpl(WriteBuffer & ostr) const override;
|
||||
@ -41,29 +49,21 @@ protected:
|
||||
private:
|
||||
using ASTWithAlias::children; /// ASTIdentifier is child free
|
||||
|
||||
bool special; /// TODO: it would be ptr to semantic here
|
||||
std::vector<String> name_parts;
|
||||
std::shared_ptr<IdentifierSemanticImpl> semantic; /// pimpl
|
||||
|
||||
static std::shared_ptr<ASTIdentifier> createSpecial(const String & name, std::vector<String> && name_parts = {})
|
||||
{
|
||||
auto ret = std::make_shared<ASTIdentifier>(name, std::move(name_parts));
|
||||
ret->special = true;
|
||||
return ret;
|
||||
}
|
||||
static std::shared_ptr<ASTIdentifier> createSpecial(const String & name, std::vector<String> && name_parts = {});
|
||||
|
||||
void setSpecial() { special = true; }
|
||||
|
||||
friend void setIdentifierSpecial(ASTPtr &);
|
||||
friend std::optional<String> getColumnIdentifierName(const ASTIdentifier & node);
|
||||
friend std::optional<String> getColumnIdentifierName(const ASTPtr & ast);
|
||||
friend std::optional<String> getTableIdentifierName(const ASTIdentifier & node);
|
||||
friend std::optional<String> getTableIdentifierName(const ASTPtr & ast);
|
||||
friend struct IdentifierSemantic;
|
||||
friend ASTPtr createTableIdentifier(const String & database_name, const String & table_name);
|
||||
friend void setIdentifierSpecial(ASTPtr & ast);
|
||||
};
|
||||
|
||||
|
||||
/// ASTIdentifier Helpers: hide casts and semantic.
|
||||
|
||||
ASTPtr createTableIdentifier(const String & database_name, const String & table_name);
|
||||
void setIdentifierSpecial(ASTPtr & ast);
|
||||
|
||||
bool isIdentifier(const IAST * const ast);
|
||||
inline bool isIdentifier(const ASTPtr & ast) { return isIdentifier(ast.get()); }
|
||||
@ -72,17 +72,5 @@ std::optional<String> getIdentifierName(const IAST * const ast);
|
||||
inline std::optional<String> getIdentifierName(const ASTPtr & ast) { return getIdentifierName(ast.get()); }
|
||||
bool getIdentifierName(const ASTPtr & ast, String & name);
|
||||
|
||||
/// @returns name for column identifiers
|
||||
std::optional<String> getColumnIdentifierName(const ASTIdentifier & node);
|
||||
std::optional<String> getColumnIdentifierName(const ASTPtr & ast);
|
||||
|
||||
/// @returns name for 'not a column' identifiers
|
||||
std::optional<String> getTableIdentifierName(const ASTIdentifier & node);
|
||||
std::optional<String> getTableIdentifierName(const ASTPtr & ast);
|
||||
|
||||
void setIdentifierSpecial(ASTPtr & ast);
|
||||
void addIdentifierQualifier(ASTIdentifier & identifier, const String & database, const String & table, const String & alias);
|
||||
bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & table_or_alias);
|
||||
bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & database, const String & table);
|
||||
|
||||
}
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Storages/MergeTree/MergeTreeWhereOptimizer.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/MergeTree/KeyCondition.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
@ -380,7 +381,7 @@ bool MergeTreeWhereOptimizer::cannotBeMoved(const ASTPtr & ptr) const
|
||||
if ("indexHint" == function_ptr->name)
|
||||
return true;
|
||||
}
|
||||
else if (auto opt_name = getColumnIdentifierName(ptr))
|
||||
else if (auto opt_name = IdentifierSemantic::getColumnName(ptr))
|
||||
{
|
||||
/// disallow moving result of ARRAY JOIN to PREWHERE
|
||||
if (array_joined_names.count(*opt_name) ||
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
|
||||
#include <Storages/StorageView.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Interpreters/SyntaxAnalyzer.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
@ -96,7 +97,7 @@ static bool isValidFunction(const ASTPtr & expression, const NameSet & columns)
|
||||
if (!isValidFunction(expression->children[i], columns))
|
||||
return false;
|
||||
|
||||
if (auto opt_name = getColumnIdentifierName(expression))
|
||||
if (auto opt_name = IdentifierSemantic::getColumnName(expression))
|
||||
return columns.count(*opt_name);
|
||||
|
||||
return true;
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/Cluster.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/parseRemoteDescription.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
@ -13,6 +13,7 @@
|
||||
3 3
|
||||
2000-01-01 1 test string 1 1
|
||||
3 3
|
||||
-------Force push down-------
|
||||
2000-01-01 1 test string 1 1
|
||||
2000-01-01 1 test string 1 1
|
||||
2000-01-01 1 test string 1 1
|
||||
|
@ -33,6 +33,7 @@ SELECT * FROM (SELECT toUInt64(b), sum(id) AS b FROM test.test) WHERE `toUInt64(
|
||||
SELECT date, id, name, value FROM (SELECT date, name, value, min(id) AS id FROM test.test GROUP BY date, name, value) WHERE id = 1;
|
||||
SELECT * FROM (SELECT toUInt64(table_alias.b) AS a, sum(id) AS b FROM test.test AS table_alias) AS outer_table_alias WHERE outer_table_alias.b = 3;
|
||||
|
||||
SELECT '-------Force push down-------';
|
||||
SET force_primary_key = 1;
|
||||
|
||||
-- Optimize predicate expression with asterisk
|
||||
@ -65,7 +66,7 @@ SELECT * FROM (SELECT 1 AS id, toDate('2000-01-01') AS date FROM system.numbers
|
||||
SELECT * FROM test.test_view WHERE id = 1;
|
||||
SELECT * FROM test.test_view WHERE id = 2;
|
||||
SELECT id FROM test.test_view WHERE id = 1;
|
||||
SELECT s.id FROM test.test_view AS s WHERE id = 1;
|
||||
SELECT s.id FROM test.test_view AS s WHERE s.id = 1;
|
||||
|
||||
SELECT '-------Push to having expression, need check.-------';
|
||||
SELECT id FROM (SELECT min(id) AS id FROM test.test) WHERE id = 1; -- { serverError 277 }
|
||||
|
@ -46,8 +46,8 @@ join on complex expression
|
||||
2 3
|
||||
2 3
|
||||
duplicate column names
|
||||
{"a1":1,"test.tab1_copy.a1":2}
|
||||
{"a1":1,"test.tab1_copy.a1":2}
|
||||
{"a1":1,"tab1_copy.a1":2}
|
||||
{"a1":1,"tab1_copy.a1":2}
|
||||
{"a1":1,"copy.a1":2}
|
||||
{"a1":1,"copy.a1":2}
|
||||
{"a1":1,"copy.a1":2}
|
||||
|
@ -76,11 +76,11 @@ select a2, b2 from test.tab2 second any left join test.tab3 third on third.a3 +
|
||||
select a2, b2 from test.tab2 second any left join test.tab3 third on third.a3 + test.tab3.b3 = test.tab2.a2 + second.b2;
|
||||
|
||||
select 'duplicate column names';
|
||||
select a1, tab1_copy.a1 from test.tab1 any left join test.tab1_copy on tab1.b1 + 3 = b1 + 2 FORMAT JSONEachRow;
|
||||
select a1, test.tab1_copy.a1 from test.tab1 any left join test.tab1_copy on tab1.b1 + 3 = b1 + 2 FORMAT JSONEachRow;
|
||||
select a1, copy.a1 from test.tab1 any left join test.tab1_copy copy on tab1.b1 + 3 = b1 + 2 FORMAT JSONEachRow;
|
||||
select a1, tab1_copy.a1 from test.tab1 any left join test.tab1_copy copy on tab1.b1 + 3 = b1 + 2 FORMAT JSONEachRow;
|
||||
select a1, test.tab1_copy.a1 from test.tab1 any left join test.tab1_copy copy on tab1.b1 + 3 = b1 + 2 FORMAT JSONEachRow;
|
||||
select a1, tab1_copy.a1 from test.tab1 any left join test.tab1_copy on tab1.b1 + 3 = tab1_copy.b1 + 2 FORMAT JSONEachRow;
|
||||
select a1, test.tab1_copy.a1 from test.tab1 any left join test.tab1_copy on tab1.b1 + 3 = tab1_copy.b1 + 2 FORMAT JSONEachRow;
|
||||
select a1, copy.a1 from test.tab1 any left join test.tab1_copy copy on tab1.b1 + 3 = tab1_copy.b1 + 2 FORMAT JSONEachRow;
|
||||
select a1, tab1_copy.a1 from test.tab1 any left join test.tab1_copy copy on tab1.b1 + 3 = tab1_copy.b1 + 2 FORMAT JSONEachRow;
|
||||
select a1, test.tab1_copy.a1 from test.tab1 any left join test.tab1_copy copy on tab1.b1 + 3 = tab1_copy.b1 + 2 FORMAT JSONEachRow;
|
||||
|
||||
select 'subquery';
|
||||
select a1 from test.tab1 any left join (select * from test.tab2) on b1 = a2;
|
||||
@ -104,4 +104,4 @@ select a1, a2, b1, b2 from test.tab1 first any left join (select * from test.tab
|
||||
select a1, a2, b1, b2 from test.tab1 first any left join (select *, a2 as z from test.tab2) second on first.b1 = second.z;
|
||||
select a1, a2, b1, b2 from test.tab1 first any left join (select *, a2 + 1 as z from test.tab2) second on first.b1 + 1 = second.z;
|
||||
select tab1.a1, a2, test.tab1.b1, second.b2 from test.tab1 first any left join (select * from test.tab2) second on first.b1 = second.a2;
|
||||
select a1, s.a1 from test.tab1 any left join (select * from test.tab1_copy) s on tab1.b1 + 3 = b1 + 2 FORMAT JSONEachRow;
|
||||
select a1, s.a1 from test.tab1 any left join (select * from test.tab1_copy) s on tab1.b1 + 3 = s.b1 + 2 FORMAT JSONEachRow;
|
||||
|
@ -7,7 +7,7 @@ create table test.tab1_copy (a1 Int32, b1 Int32) engine = MergeTree order by a1;
|
||||
insert into test.tab1 values (1, 2);
|
||||
insert into test.tab1_copy values (2, 3);
|
||||
|
||||
select tab1.a1, tab1_copy.a1, tab1.b1 from test.tab1 any left join test.tab1_copy on tab1.b1 + 3 = b1 + 2;
|
||||
select tab1.a1, tab1_copy.a1, tab1.b1 from test.tab1 any left join test.tab1_copy on tab1.b1 + 3 = tab1_copy.b1 + 2;
|
||||
|
||||
drop table test.tab1;
|
||||
drop table test.tab1_copy;
|
||||
|
Loading…
Reference in New Issue
Block a user