2019-01-25 15:42:24 +00:00
|
|
|
#include <iostream>
|
|
|
|
|
2018-03-04 16:15:31 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
|
|
|
#include <Storages/IStorage.h>
|
|
|
|
#include <Interpreters/PredicateExpressionsOptimizer.h>
|
|
|
|
#include <Interpreters/InterpreterSelectQuery.h>
|
2019-01-25 15:42:24 +00:00
|
|
|
#include <Interpreters/IdentifierSemantic.h>
|
2018-03-04 16:15:31 +00:00
|
|
|
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
2019-01-25 15:42:24 +00:00
|
|
|
#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>
|
2018-08-22 06:42:37 +00:00
|
|
|
#include <Parsers/ASTAsterisk.h>
|
|
|
|
#include <Parsers/ASTQualifiedAsterisk.h>
|
|
|
|
#include <Parsers/queryToString.h>
|
2019-01-25 15:42:24 +00:00
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
#include <Interpreters/ExpressionActions.h>
|
2018-09-12 05:41:09 +00:00
|
|
|
#include <Interpreters/QueryNormalizer.h>
|
2018-09-28 10:52:08 +00:00
|
|
|
#include <Interpreters/QueryAliasesVisitor.h>
|
2019-01-30 02:47:26 +00:00
|
|
|
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
|
|
|
|
#include <Interpreters/FindIdentifierBestTableVisitor.h>
|
2019-01-30 04:37:24 +00:00
|
|
|
#include <Interpreters/ExtractFunctionDataVisitor.h>
|
2019-01-30 02:47:26 +00:00
|
|
|
#include <Functions/FunctionFactory.h>
|
2018-03-04 16:15:31 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2019-01-09 15:44:20 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
extern const int UNKNOWN_ELEMENT_IN_AST;
|
|
|
|
}
|
|
|
|
|
2019-04-18 10:39:25 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
|
|
|
constexpr auto and_function_name = "and";
|
|
|
|
|
|
|
|
String qualifiedName(ASTIdentifier * identifier, const String & prefix)
|
|
|
|
{
|
|
|
|
if (identifier->isShort())
|
|
|
|
return prefix + identifier->getAliasOrColumnName();
|
|
|
|
return identifier->getAliasOrColumnName();
|
|
|
|
}
|
|
|
|
|
2019-06-13 10:43:37 +00:00
|
|
|
}
|
2018-03-04 16:15:31 +00:00
|
|
|
|
|
|
|
PredicateExpressionsOptimizer::PredicateExpressionsOptimizer(
|
2018-10-18 15:03:14 +00:00
|
|
|
ASTSelectQuery * ast_select_, ExtractedSettings && settings_, const Context & context_)
|
2019-04-18 10:39:25 +00:00
|
|
|
: ast_select(ast_select_), settings(settings_), context(context_)
|
2018-03-04 16:15:31 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
bool PredicateExpressionsOptimizer::optimize()
|
|
|
|
{
|
2019-04-09 14:22:35 +00:00
|
|
|
if (!settings.enable_optimize_predicate_expression || !ast_select || !ast_select->tables() || ast_select->tables()->children.empty())
|
2018-03-04 16:15:31 +00:00
|
|
|
return false;
|
|
|
|
|
2019-04-09 14:22:35 +00:00
|
|
|
if (!ast_select->where() && !ast_select->prewhere())
|
2018-11-15 15:23:44 +00:00
|
|
|
return false;
|
|
|
|
|
2019-04-18 10:39:25 +00:00
|
|
|
if (ast_select->array_join_expression_list())
|
|
|
|
return false;
|
|
|
|
|
2019-01-25 15:42:24 +00:00
|
|
|
SubqueriesProjectionColumns all_subquery_projection_columns = getAllSubqueryProjectionColumns();
|
2018-03-04 16:15:31 +00:00
|
|
|
|
2018-08-20 15:49:39 +00:00
|
|
|
bool is_rewrite_subqueries = false;
|
2018-03-04 16:15:31 +00:00
|
|
|
if (!all_subquery_projection_columns.empty())
|
|
|
|
{
|
2019-04-09 16:49:52 +00:00
|
|
|
is_rewrite_subqueries |= optimizeImpl(ast_select->where(), all_subquery_projection_columns, OptimizeKind::PUSH_TO_WHERE);
|
|
|
|
is_rewrite_subqueries |= optimizeImpl(ast_select->prewhere(), all_subquery_projection_columns, OptimizeKind::PUSH_TO_PREWHERE);
|
2018-03-04 16:15:31 +00:00
|
|
|
}
|
2019-02-11 19:53:55 +00:00
|
|
|
|
2018-08-20 15:49:39 +00:00
|
|
|
return is_rewrite_subqueries;
|
2018-03-04 16:15:31 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
bool PredicateExpressionsOptimizer::optimizeImpl(
|
2019-04-09 16:49:52 +00:00
|
|
|
const ASTPtr & outer_expression, const SubqueriesProjectionColumns & subqueries_projection_columns, OptimizeKind expression_kind)
|
2018-03-04 16:15:31 +00:00
|
|
|
{
|
|
|
|
/// split predicate with `and`
|
2019-01-25 15:42:24 +00:00
|
|
|
std::vector<ASTPtr> outer_predicate_expressions = splitConjunctionPredicate(outer_expression);
|
2018-03-04 16:15:31 +00:00
|
|
|
|
2019-02-26 10:07:58 +00:00
|
|
|
std::vector<TableWithColumnNames> tables_with_columns = getDatabaseAndTablesWithColumnNames(*ast_select, context);
|
2018-08-22 06:42:37 +00:00
|
|
|
|
2018-03-04 16:15:31 +00:00
|
|
|
bool is_rewrite_subquery = false;
|
2019-01-25 15:42:24 +00:00
|
|
|
for (auto & outer_predicate : outer_predicate_expressions)
|
2018-03-04 16:15:31 +00:00
|
|
|
{
|
2018-09-12 05:41:09 +00:00
|
|
|
if (isArrayJoinFunction(outer_predicate))
|
|
|
|
continue;
|
|
|
|
|
2019-02-26 10:07:58 +00:00
|
|
|
auto outer_predicate_dependencies = getDependenciesAndQualifiers(outer_predicate, tables_with_columns);
|
2018-03-04 16:15:31 +00:00
|
|
|
|
|
|
|
/// TODO: remove origin expression
|
2019-01-25 15:42:24 +00:00
|
|
|
for (const auto & [subquery, projection_columns] : subqueries_projection_columns)
|
2018-03-04 16:15:31 +00:00
|
|
|
{
|
|
|
|
OptimizeKind optimize_kind = OptimizeKind::NONE;
|
2019-04-18 10:39:25 +00:00
|
|
|
if (allowPushDown(subquery, outer_predicate, projection_columns, outer_predicate_dependencies, optimize_kind))
|
2018-03-04 16:15:31 +00:00
|
|
|
{
|
2019-01-25 15:42:24 +00:00
|
|
|
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 =
|
2019-02-26 10:07:58 +00:00
|
|
|
getDependenciesAndQualifiers(inner_predicate, tables_with_columns);
|
2019-01-25 15:42:24 +00:00
|
|
|
|
|
|
|
setNewAliasesForInnerPredicate(projection_columns, inner_predicate_dependencies);
|
2018-03-04 16:15:31 +00:00
|
|
|
|
2018-11-23 18:52:00 +00:00
|
|
|
switch (optimize_kind)
|
2018-03-04 16:15:31 +00:00
|
|
|
{
|
|
|
|
case OptimizeKind::NONE: continue;
|
2019-04-09 16:49:52 +00:00
|
|
|
case OptimizeKind::PUSH_TO_WHERE:
|
|
|
|
is_rewrite_subquery |= optimizeExpression(inner_predicate, subquery, ASTSelectQuery::Expression::WHERE);
|
|
|
|
continue;
|
|
|
|
case OptimizeKind::PUSH_TO_HAVING:
|
|
|
|
is_rewrite_subquery |= optimizeExpression(inner_predicate, subquery, ASTSelectQuery::Expression::HAVING);
|
|
|
|
continue;
|
|
|
|
case OptimizeKind::PUSH_TO_PREWHERE:
|
|
|
|
is_rewrite_subquery |= optimizeExpression(inner_predicate, subquery, ASTSelectQuery::Expression::PREWHERE);
|
|
|
|
continue;
|
2018-03-04 16:15:31 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return is_rewrite_subquery;
|
|
|
|
}
|
|
|
|
|
2019-04-18 10:39:25 +00:00
|
|
|
bool PredicateExpressionsOptimizer::allowPushDown(
|
|
|
|
const ASTSelectQuery * subquery,
|
|
|
|
const ASTPtr &,
|
|
|
|
const std::vector<ProjectionWithAlias> & projection_columns,
|
|
|
|
const std::vector<IdentifierWithQualifier> & dependencies,
|
|
|
|
OptimizeKind & optimize_kind)
|
2019-01-30 02:47:26 +00:00
|
|
|
{
|
2019-04-18 10:39:25 +00:00
|
|
|
if (!subquery || subquery->final() || subquery->limitBy() || subquery->limitLength() || subquery->with())
|
|
|
|
return false;
|
|
|
|
else
|
2019-01-30 02:47:26 +00:00
|
|
|
{
|
2019-04-09 14:22:35 +00:00
|
|
|
ASTPtr expr_list = ast_select->select();
|
2019-01-30 02:47:26 +00:00
|
|
|
ExtractFunctionVisitor::Data extract_data;
|
|
|
|
ExtractFunctionVisitor(extract_data).visit(expr_list);
|
|
|
|
|
|
|
|
for (const auto & subquery_function : extract_data.functions)
|
|
|
|
{
|
2019-02-15 08:03:18 +00:00
|
|
|
const auto & function = FunctionFactory::instance().tryGet(subquery_function->name, context);
|
|
|
|
|
2019-04-18 10:39:25 +00:00
|
|
|
/// Skip lambda, tuple and other special functions
|
2019-02-15 08:03:18 +00:00
|
|
|
if (function && function->isStateful())
|
2019-01-30 02:47:26 +00:00
|
|
|
return false;
|
|
|
|
}
|
2019-04-18 10:39:25 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
const auto * ast_join = ast_select->join();
|
|
|
|
const ASTTableExpression * left_table_expr = nullptr;
|
|
|
|
const ASTTableExpression * right_table_expr = nullptr;
|
|
|
|
const ASTSelectQuery * left_subquery = nullptr;
|
|
|
|
const ASTSelectQuery * right_subquery = nullptr;
|
2019-01-30 02:47:26 +00:00
|
|
|
|
2019-04-18 10:39:25 +00:00
|
|
|
if (ast_join)
|
|
|
|
{
|
|
|
|
left_table_expr = ast_select
|
|
|
|
->tables()->as<ASTTablesInSelectQuery>()
|
|
|
|
->children[0]->as<ASTTablesInSelectQueryElement>()
|
|
|
|
->table_expression->as<ASTTableExpression>();
|
|
|
|
right_table_expr = ast_select
|
|
|
|
->tables()->as<ASTTablesInSelectQuery>()
|
|
|
|
->children[1]->as<ASTTablesInSelectQueryElement>()
|
|
|
|
->table_expression->as<ASTTableExpression>();
|
|
|
|
|
|
|
|
if (left_table_expr && left_table_expr->subquery)
|
|
|
|
left_subquery = left_table_expr->subquery
|
|
|
|
->children[0]->as<ASTSelectWithUnionQuery>()
|
|
|
|
->list_of_selects->children[0]->as<ASTSelectQuery>();
|
|
|
|
if (right_table_expr && right_table_expr->subquery)
|
|
|
|
right_subquery = right_table_expr->subquery
|
|
|
|
->children[0]->as<ASTSelectWithUnionQuery>()
|
|
|
|
->list_of_selects->children[0]->as<ASTSelectQuery>();
|
|
|
|
|
|
|
|
/// NOTE: the syntactic way of pushdown has limitations and should be partially disabled in case of JOINs.
|
|
|
|
/// Let's take a look at the query:
|
|
|
|
///
|
|
|
|
/// SELECT a, b FROM (SELECT 1 AS a) ANY LEFT JOIN (SELECT 1 AS a, 1 AS b) USING (a) WHERE b = 0
|
|
|
|
///
|
|
|
|
/// The result is empty - without pushdown. But the pushdown tends to modify it in this way:
|
|
|
|
///
|
|
|
|
/// SELECT a, b FROM (SELECT 1 AS a) ANY LEFT JOIN (SELECT 1 AS a, 1 AS b WHERE b = 0) USING (a) WHERE b = 0
|
|
|
|
///
|
|
|
|
/// That leads to the empty result in the right subquery and changes the whole outcome to (1, 0) or (1, NULL).
|
|
|
|
/// It happens because the not-matching columns are replaced with a global default values on JOIN.
|
|
|
|
/// Same is true for RIGHT JOIN and FULL JOIN.
|
|
|
|
|
|
|
|
/// Check right side for LEFT'o'FULL JOIN
|
|
|
|
if (isLeftOrFull(ast_join->table_join->as<ASTTableJoin>()->kind) && right_subquery == subquery)
|
|
|
|
return false;
|
|
|
|
|
|
|
|
/// Check left side for RIGHT'o'FULL JOIN
|
|
|
|
if (isRightOrFull(ast_join->table_join->as<ASTTableJoin>()->kind) && left_subquery == subquery)
|
|
|
|
return false;
|
2019-01-30 02:47:26 +00:00
|
|
|
}
|
|
|
|
|
2019-04-18 10:39:25 +00:00
|
|
|
return checkDependencies(projection_columns, dependencies, optimize_kind);
|
|
|
|
}
|
|
|
|
|
|
|
|
bool PredicateExpressionsOptimizer::checkDependencies(
|
|
|
|
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 (const auto & [ast, alias] : projection_columns)
|
|
|
|
{
|
|
|
|
if (alias == qualified_name)
|
|
|
|
{
|
|
|
|
is_found = true;
|
|
|
|
ASTPtr projection_column = ast;
|
|
|
|
ExtractFunctionVisitor::Data extract_data;
|
|
|
|
ExtractFunctionVisitor(extract_data).visit(projection_column);
|
|
|
|
|
|
|
|
if (!extract_data.aggregate_functions.empty())
|
|
|
|
optimize_kind = OptimizeKind::PUSH_TO_HAVING;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!is_found)
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
return true;
|
2019-01-30 02:47:26 +00:00
|
|
|
}
|
|
|
|
|
2019-04-09 16:49:52 +00:00
|
|
|
std::vector<ASTPtr> PredicateExpressionsOptimizer::splitConjunctionPredicate(const ASTPtr & predicate_expression)
|
2018-03-04 16:15:31 +00:00
|
|
|
{
|
2019-01-25 15:42:24 +00:00
|
|
|
std::vector<ASTPtr> predicate_expressions;
|
2018-03-04 16:15:31 +00:00
|
|
|
|
|
|
|
if (predicate_expression)
|
|
|
|
{
|
|
|
|
predicate_expressions.emplace_back(predicate_expression);
|
|
|
|
|
|
|
|
auto remove_expression_at_index = [&predicate_expressions] (const size_t index)
|
|
|
|
{
|
|
|
|
if (index < predicate_expressions.size() - 1)
|
|
|
|
std::swap(predicate_expressions[index], predicate_expressions.back());
|
|
|
|
predicate_expressions.pop_back();
|
|
|
|
};
|
|
|
|
|
|
|
|
for (size_t idx = 0; idx < predicate_expressions.size();)
|
|
|
|
{
|
|
|
|
const auto expression = predicate_expressions.at(idx);
|
|
|
|
|
2019-03-11 13:22:51 +00:00
|
|
|
if (const auto * function = expression->as<ASTFunction>())
|
2018-03-04 16:15:31 +00:00
|
|
|
{
|
|
|
|
if (function->name == and_function_name)
|
|
|
|
{
|
|
|
|
for (auto & child : function->arguments->children)
|
|
|
|
predicate_expressions.emplace_back(child);
|
|
|
|
|
|
|
|
remove_expression_at_index(idx);
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
idx++;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return predicate_expressions;
|
|
|
|
}
|
|
|
|
|
2019-01-25 15:42:24 +00:00
|
|
|
std::vector<PredicateExpressionsOptimizer::IdentifierWithQualifier>
|
2019-02-26 10:07:58 +00:00
|
|
|
PredicateExpressionsOptimizer::getDependenciesAndQualifiers(ASTPtr & expression, std::vector<TableWithColumnNames> & tables)
|
2018-03-04 16:15:31 +00:00
|
|
|
{
|
2019-01-25 15:42:24 +00:00
|
|
|
FindIdentifierBestTableVisitor::Data find_data(tables);
|
|
|
|
FindIdentifierBestTableVisitor(find_data).visit(expression);
|
2018-03-04 16:15:31 +00:00
|
|
|
|
2019-01-25 15:42:24 +00:00
|
|
|
std::vector<IdentifierWithQualifier> dependencies;
|
2019-01-17 17:01:48 +00:00
|
|
|
|
2019-01-25 15:42:24 +00:00
|
|
|
for (const auto & [identifier, table] : find_data.identifier_table)
|
2018-08-22 06:42:37 +00:00
|
|
|
{
|
2019-01-25 15:42:24 +00:00
|
|
|
String table_alias;
|
|
|
|
if (table)
|
|
|
|
table_alias = table->getQualifiedNamePrefix();
|
|
|
|
|
|
|
|
dependencies.emplace_back(identifier, table_alias);
|
2018-08-22 06:42:37 +00:00
|
|
|
}
|
2019-01-25 15:42:24 +00:00
|
|
|
|
|
|
|
return dependencies;
|
2018-03-04 16:15:31 +00:00
|
|
|
}
|
|
|
|
|
2019-01-25 15:42:24 +00:00
|
|
|
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)
|
|
|
|
{
|
2019-02-08 15:37:43 +00:00
|
|
|
String name;
|
2019-03-11 13:22:51 +00:00
|
|
|
if (auto * id = ast->as<ASTIdentifier>())
|
2019-02-08 15:37:43 +00:00
|
|
|
{
|
|
|
|
name = id->tryGetAlias();
|
|
|
|
if (name.empty())
|
|
|
|
name = id->shortName();
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
if (ast->tryGetAlias().empty())
|
|
|
|
ast->setAlias(ast->getColumnName());
|
|
|
|
name = ast->getAliasOrColumnName();
|
|
|
|
}
|
2019-01-25 15:42:24 +00:00
|
|
|
|
2019-02-08 15:37:43 +00:00
|
|
|
identifier->setShortName(name);
|
2019-01-25 15:42:24 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2018-03-04 16:15:31 +00:00
|
|
|
}
|
|
|
|
|
2018-09-12 05:41:09 +00:00
|
|
|
bool PredicateExpressionsOptimizer::isArrayJoinFunction(const ASTPtr & node)
|
|
|
|
{
|
2019-03-11 13:22:51 +00:00
|
|
|
if (const auto * function = node->as<ASTFunction>())
|
2018-09-12 05:41:09 +00:00
|
|
|
{
|
|
|
|
if (function->name == "arrayJoin")
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
for (auto & child : node->children)
|
|
|
|
if (isArrayJoinFunction(child))
|
|
|
|
return true;
|
|
|
|
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
2019-04-09 16:49:52 +00:00
|
|
|
bool PredicateExpressionsOptimizer::optimizeExpression(const ASTPtr & outer_expression, ASTSelectQuery * subquery,
|
|
|
|
ASTSelectQuery::Expression expr)
|
2018-03-04 16:15:31 +00:00
|
|
|
{
|
2019-04-09 16:49:52 +00:00
|
|
|
ASTPtr subquery_expression = subquery->getExpression(expr, false);
|
|
|
|
subquery_expression = subquery_expression ? makeASTFunction(and_function_name, outer_expression, subquery_expression) : outer_expression;
|
2018-03-04 16:15:31 +00:00
|
|
|
|
2019-04-09 16:49:52 +00:00
|
|
|
subquery->setExpression(expr, std::move(subquery_expression));
|
2018-03-04 16:15:31 +00:00
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2019-01-25 15:42:24 +00:00
|
|
|
PredicateExpressionsOptimizer::SubqueriesProjectionColumns PredicateExpressionsOptimizer::getAllSubqueryProjectionColumns()
|
2018-03-04 16:15:31 +00:00
|
|
|
{
|
2019-01-25 15:42:24 +00:00
|
|
|
SubqueriesProjectionColumns projection_columns;
|
2018-03-04 16:15:31 +00:00
|
|
|
|
2019-01-25 15:42:24 +00:00
|
|
|
for (const auto & table_expression : getSelectTablesExpression(*ast_select))
|
2018-08-22 06:42:37 +00:00
|
|
|
if (table_expression->subquery)
|
2019-01-25 15:42:24 +00:00
|
|
|
getSubqueryProjectionColumns(table_expression->subquery, projection_columns);
|
|
|
|
|
|
|
|
return projection_columns;
|
2018-08-22 06:42:37 +00:00
|
|
|
}
|
2018-03-04 16:15:31 +00:00
|
|
|
|
2019-01-25 15:42:24 +00:00
|
|
|
void PredicateExpressionsOptimizer::getSubqueryProjectionColumns(const ASTPtr & subquery, SubqueriesProjectionColumns & projection_columns)
|
2018-08-22 06:42:37 +00:00
|
|
|
{
|
2019-01-25 15:42:24 +00:00
|
|
|
String qualified_name_prefix = subquery->tryGetAlias();
|
|
|
|
if (!qualified_name_prefix.empty())
|
|
|
|
qualified_name_prefix += '.';
|
|
|
|
|
|
|
|
const ASTPtr & subselect = subquery->children[0];
|
|
|
|
|
2018-08-22 06:42:37 +00:00
|
|
|
ASTs select_with_union_projections;
|
2019-03-11 13:22:51 +00:00
|
|
|
const auto * select_with_union_query = subselect->as<ASTSelectWithUnionQuery>();
|
2018-08-22 06:42:37 +00:00
|
|
|
|
2019-01-25 15:42:24 +00:00
|
|
|
for (auto & select : select_with_union_query->list_of_selects->children)
|
2018-03-04 16:15:31 +00:00
|
|
|
{
|
2019-01-25 15:42:24 +00:00
|
|
|
std::vector<ProjectionWithAlias> subquery_projections;
|
|
|
|
auto select_projection_columns = getSelectQueryProjectionColumns(select);
|
2018-03-04 16:15:31 +00:00
|
|
|
|
2018-08-24 03:32:20 +00:00
|
|
|
if (!select_projection_columns.empty())
|
|
|
|
{
|
|
|
|
if (select_with_union_projections.empty())
|
|
|
|
select_with_union_projections = select_projection_columns;
|
2018-03-04 16:15:31 +00:00
|
|
|
|
2018-08-24 03:32:20 +00:00
|
|
|
for (size_t i = 0; i < select_projection_columns.size(); i++)
|
|
|
|
subquery_projections.emplace_back(std::pair(select_projection_columns[i],
|
|
|
|
qualified_name_prefix + select_with_union_projections[i]->getAliasOrColumnName()));
|
2018-03-04 16:15:31 +00:00
|
|
|
|
2019-03-11 13:22:51 +00:00
|
|
|
projection_columns.insert(std::pair(select->as<ASTSelectQuery>(), subquery_projections));
|
2018-08-24 03:32:20 +00:00
|
|
|
}
|
2018-08-22 06:42:37 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
ASTs PredicateExpressionsOptimizer::getSelectQueryProjectionColumns(ASTPtr & ast)
|
|
|
|
{
|
2018-11-15 15:23:44 +00:00
|
|
|
ASTs projection_columns;
|
2019-03-11 13:22:51 +00:00
|
|
|
auto * select_query = ast->as<ASTSelectQuery>();
|
2018-11-15 15:23:44 +00:00
|
|
|
|
2018-09-12 05:41:09 +00:00
|
|
|
/// first should normalize query tree.
|
|
|
|
std::unordered_map<String, ASTPtr> aliases;
|
2018-11-15 15:23:44 +00:00
|
|
|
std::vector<DatabaseAndTableWithAlias> tables = getDatabaseAndTables(*select_query, context.getCurrentDatabase());
|
|
|
|
|
2019-02-11 14:19:09 +00:00
|
|
|
/// TODO: get tables from evaluateAsterisk instead of tablesOnly() to extract asterisks in general way
|
|
|
|
std::vector<TableWithColumnNames> tables_with_columns = TranslateQualifiedNamesVisitor::Data::tablesOnly(tables);
|
2019-04-18 10:39:25 +00:00
|
|
|
TranslateQualifiedNamesVisitor::Data qn_visitor_data({}, tables_with_columns, false);
|
2018-12-06 15:29:55 +00:00
|
|
|
TranslateQualifiedNamesVisitor(qn_visitor_data).visit(ast);
|
2019-01-11 14:09:23 +00:00
|
|
|
|
2018-12-10 13:02:45 +00:00
|
|
|
QueryAliasesVisitor::Data query_aliases_data{aliases};
|
2018-12-06 19:02:42 +00:00
|
|
|
QueryAliasesVisitor(query_aliases_data).visit(ast);
|
2019-01-11 14:09:23 +00:00
|
|
|
|
|
|
|
QueryNormalizer::Data normalizer_data(aliases, settings);
|
|
|
|
QueryNormalizer(normalizer_data).visit(ast);
|
2018-09-12 05:41:09 +00:00
|
|
|
|
2019-04-09 14:22:35 +00:00
|
|
|
for (const auto & projection_column : select_query->select()->children)
|
2018-08-22 06:42:37 +00:00
|
|
|
{
|
2019-03-11 13:22:51 +00:00
|
|
|
if (projection_column->as<ASTAsterisk>() || projection_column->as<ASTQualifiedAsterisk>())
|
2018-08-22 06:42:37 +00:00
|
|
|
{
|
|
|
|
ASTs evaluated_columns = evaluateAsterisk(select_query, projection_column);
|
|
|
|
|
|
|
|
for (const auto & column : evaluated_columns)
|
|
|
|
projection_columns.emplace_back(column);
|
|
|
|
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
|
|
|
|
projection_columns.emplace_back(projection_column);
|
|
|
|
}
|
|
|
|
return projection_columns;
|
|
|
|
}
|
|
|
|
|
2018-08-24 03:32:20 +00:00
|
|
|
ASTs PredicateExpressionsOptimizer::evaluateAsterisk(ASTSelectQuery * select_query, const ASTPtr & asterisk)
|
2018-08-22 06:42:37 +00:00
|
|
|
{
|
2018-08-24 05:30:49 +00:00
|
|
|
/// SELECT *, SELECT dummy, SELECT 1 AS id
|
2019-04-09 14:22:35 +00:00
|
|
|
if (!select_query->tables() || select_query->tables()->children.empty())
|
2018-08-24 03:32:20 +00:00
|
|
|
return {};
|
2018-08-22 06:42:37 +00:00
|
|
|
|
2018-10-30 16:31:21 +00:00
|
|
|
std::vector<const ASTTableExpression *> tables_expression = getSelectTablesExpression(*select_query);
|
2018-08-22 06:42:37 +00:00
|
|
|
|
2019-03-11 13:22:51 +00:00
|
|
|
if (const auto * qualified_asterisk = asterisk->as<ASTQualifiedAsterisk>())
|
2018-08-22 06:42:37 +00:00
|
|
|
{
|
|
|
|
if (qualified_asterisk->children.size() != 1)
|
|
|
|
throw Exception("Logical error: qualified asterisk must have exactly one child", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
2019-01-17 17:01:48 +00:00
|
|
|
DatabaseAndTableWithAlias ident_db_and_name(qualified_asterisk->children[0]);
|
2018-08-22 06:42:37 +00:00
|
|
|
|
2019-01-17 17:01:48 +00:00
|
|
|
for (auto it = tables_expression.begin(); it != tables_expression.end();)
|
2018-08-22 06:42:37 +00:00
|
|
|
{
|
|
|
|
const ASTTableExpression * table_expression = *it;
|
2018-10-30 16:31:21 +00:00
|
|
|
DatabaseAndTableWithAlias database_and_table_with_alias(*table_expression, context.getCurrentDatabase());
|
2019-01-17 17:01:48 +00:00
|
|
|
|
|
|
|
if (ident_db_and_name.satisfies(database_and_table_with_alias, true))
|
|
|
|
++it;
|
2018-08-22 06:42:37 +00:00
|
|
|
else
|
2019-01-17 17:01:48 +00:00
|
|
|
it = tables_expression.erase(it); /// It's not a required table
|
2018-08-22 06:42:37 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
ASTs projection_columns;
|
|
|
|
for (auto & table_expression : tables_expression)
|
|
|
|
{
|
|
|
|
if (table_expression->subquery)
|
|
|
|
{
|
2019-03-11 13:22:51 +00:00
|
|
|
const auto * subquery = table_expression->subquery->as<ASTSubquery>();
|
|
|
|
const auto * select_with_union_query = subquery->children[0]->as<ASTSelectWithUnionQuery>();
|
2018-08-22 06:42:37 +00:00
|
|
|
const auto subquery_projections = getSelectQueryProjectionColumns(select_with_union_query->list_of_selects->children[0]);
|
|
|
|
projection_columns.insert(projection_columns.end(), subquery_projections.begin(), subquery_projections.end());
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
StoragePtr storage;
|
2018-03-04 16:15:31 +00:00
|
|
|
|
2018-08-22 06:42:37 +00:00
|
|
|
if (table_expression->table_function)
|
2018-08-24 07:40:09 +00:00
|
|
|
{
|
|
|
|
auto query_context = const_cast<Context *>(&context.getQueryContext());
|
|
|
|
storage = query_context->executeTableFunction(table_expression->table_function);
|
|
|
|
}
|
2018-08-22 06:42:37 +00:00
|
|
|
else if (table_expression->database_and_table_name)
|
|
|
|
{
|
2019-03-11 13:22:51 +00:00
|
|
|
const auto * database_and_table_ast = table_expression->database_and_table_name->as<ASTIdentifier>();
|
2018-10-30 16:31:21 +00:00
|
|
|
DatabaseAndTableWithAlias database_and_table_name(*database_and_table_ast);
|
|
|
|
storage = context.getTable(database_and_table_name.database, database_and_table_name.table);
|
2018-08-22 06:42:37 +00:00
|
|
|
}
|
2019-01-09 15:44:20 +00:00
|
|
|
else
|
|
|
|
throw Exception("Logical error: unexpected table expression", ErrorCodes::LOGICAL_ERROR);
|
2018-03-04 16:15:31 +00:00
|
|
|
|
2018-08-22 06:42:37 +00:00
|
|
|
const auto block = storage->getSampleBlock();
|
|
|
|
for (size_t idx = 0; idx < block.columns(); idx++)
|
|
|
|
projection_columns.emplace_back(std::make_shared<ASTIdentifier>(block.getByPosition(idx).name));
|
|
|
|
}
|
2018-03-04 16:15:31 +00:00
|
|
|
}
|
2018-08-22 06:42:37 +00:00
|
|
|
return projection_columns;
|
|
|
|
}
|
|
|
|
|
2018-09-27 05:27:45 +00:00
|
|
|
void PredicateExpressionsOptimizer::cleanExpressionAlias(ASTPtr & expression)
|
|
|
|
{
|
|
|
|
const auto my_alias = expression->tryGetAlias();
|
|
|
|
if (!my_alias.empty())
|
|
|
|
expression->setAlias("");
|
|
|
|
|
|
|
|
for (auto & child : expression->children)
|
|
|
|
cleanExpressionAlias(child);
|
|
|
|
}
|
|
|
|
|
2018-03-04 16:15:31 +00:00
|
|
|
}
|