2019-12-18 03:56:03 +00:00
|
|
|
#include <Interpreters/PredicateExpressionsOptimizer.h>
|
2020-01-04 04:31:45 +00:00
|
|
|
|
2019-01-25 15:42:24 +00:00
|
|
|
#include <Parsers/IAST.h>
|
|
|
|
#include <Parsers/ASTFunction.h>
|
|
|
|
#include <Parsers/ASTSelectQuery.h>
|
|
|
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
|
|
|
#include <Interpreters/Context.h>
|
2019-12-12 15:28:24 +00:00
|
|
|
#include <Interpreters/getTableExpressions.h>
|
2020-01-04 04:31:45 +00:00
|
|
|
#include <Interpreters/PredicateRewriteVisitor.h>
|
2019-12-18 03:56:03 +00:00
|
|
|
#include <Interpreters/ExtractExpressionInfoVisitor.h>
|
2019-07-16 20:05:00 +00:00
|
|
|
|
2018-03-04 16:15:31 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2019-01-09 15:44:20 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2018-03-04 16:15:31 +00:00
|
|
|
PredicateExpressionsOptimizer::PredicateExpressionsOptimizer(
|
2020-06-05 21:17:00 +00:00
|
|
|
const Context & context_, const TablesWithColumns & tables_with_columns_, const Settings & settings)
|
|
|
|
: enable_optimize_predicate_expression(settings.enable_optimize_predicate_expression)
|
|
|
|
, enable_optimize_predicate_expression_to_final_subquery(settings.enable_optimize_predicate_expression_to_final_subquery)
|
|
|
|
, context(context_)
|
|
|
|
, tables_with_columns(tables_with_columns_)
|
2018-03-04 16:15:31 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2019-12-18 03:56:03 +00:00
|
|
|
bool PredicateExpressionsOptimizer::optimize(ASTSelectQuery & select_query)
|
2018-03-04 16:15:31 +00:00
|
|
|
{
|
2020-06-05 21:17:00 +00:00
|
|
|
if (!enable_optimize_predicate_expression)
|
2020-01-04 04:31:45 +00:00
|
|
|
return false;
|
|
|
|
|
|
|
|
if (select_query.having() && (!select_query.group_by_with_cube && !select_query.group_by_with_rollup && !select_query.group_by_with_totals))
|
|
|
|
tryMovePredicatesFromHavingToWhere(select_query);
|
|
|
|
|
|
|
|
if (!select_query.tables() || select_query.tables()->children.empty())
|
2018-03-04 16:15:31 +00:00
|
|
|
return false;
|
|
|
|
|
2020-03-23 02:12:31 +00:00
|
|
|
if ((!select_query.where() && !select_query.prewhere()) || select_query.arrayJoinExpressionList())
|
2018-11-15 15:23:44 +00:00
|
|
|
return false;
|
|
|
|
|
2019-12-18 03:56:03 +00:00
|
|
|
const auto & tables_predicates = extractTablesPredicates(select_query.where(), select_query.prewhere());
|
2019-04-18 10:39:25 +00:00
|
|
|
|
2019-12-18 03:56:03 +00:00
|
|
|
if (!tables_predicates.empty())
|
|
|
|
return tryRewritePredicatesToTables(select_query.refTables()->children, tables_predicates);
|
2018-03-04 16:15:31 +00:00
|
|
|
|
2019-12-18 03:56:03 +00:00
|
|
|
return false;
|
2018-03-04 16:15:31 +00:00
|
|
|
}
|
|
|
|
|
2019-12-18 03:56:03 +00:00
|
|
|
static ASTs splitConjunctionPredicate(const std::initializer_list<const ASTPtr> & predicates)
|
2018-03-04 16:15:31 +00:00
|
|
|
{
|
2019-12-18 03:56:03 +00:00
|
|
|
std::vector<ASTPtr> res;
|
2018-03-04 16:15:31 +00:00
|
|
|
|
2019-12-18 03:56:03 +00:00
|
|
|
for (const auto & predicate : predicates)
|
2018-03-04 16:15:31 +00:00
|
|
|
{
|
2019-12-18 03:56:03 +00:00
|
|
|
if (!predicate)
|
2018-09-12 05:41:09 +00:00
|
|
|
continue;
|
|
|
|
|
2019-12-18 03:56:03 +00:00
|
|
|
res.emplace_back(predicate);
|
2018-03-04 16:15:31 +00:00
|
|
|
|
2019-12-18 03:56:03 +00:00
|
|
|
for (size_t idx = 0; idx < res.size();)
|
2018-03-04 16:15:31 +00:00
|
|
|
{
|
2020-05-01 22:21:09 +00:00
|
|
|
ASTPtr expression = res.at(idx);
|
2019-12-18 03:56:03 +00:00
|
|
|
|
|
|
|
if (const auto * function = expression->as<ASTFunction>(); function && function->name == "and")
|
2018-03-04 16:15:31 +00:00
|
|
|
{
|
2020-05-01 22:21:09 +00:00
|
|
|
res.erase(res.begin() + idx);
|
|
|
|
|
2019-12-18 03:56:03 +00:00
|
|
|
for (auto & child : function->arguments->children)
|
|
|
|
res.emplace_back(child);
|
|
|
|
|
|
|
|
continue;
|
2018-03-04 16:15:31 +00:00
|
|
|
}
|
2019-12-18 03:56:03 +00:00
|
|
|
++idx;
|
2018-03-04 16:15:31 +00:00
|
|
|
}
|
|
|
|
}
|
2019-12-18 03:56:03 +00:00
|
|
|
|
|
|
|
return res;
|
2018-03-04 16:15:31 +00:00
|
|
|
}
|
|
|
|
|
2019-12-18 03:56:03 +00:00
|
|
|
std::vector<ASTs> PredicateExpressionsOptimizer::extractTablesPredicates(const ASTPtr & where, const ASTPtr & prewhere)
|
2019-01-30 02:47:26 +00:00
|
|
|
{
|
2019-12-18 03:56:03 +00:00
|
|
|
std::vector<ASTs> tables_predicates(tables_with_columns.size());
|
|
|
|
|
|
|
|
for (const auto & predicate_expression : splitConjunctionPredicate({where, prewhere}))
|
2019-01-30 02:47:26 +00:00
|
|
|
{
|
2019-12-18 03:56:03 +00:00
|
|
|
ExpressionInfoVisitor::Data expression_info{.context = context, .tables = tables_with_columns};
|
|
|
|
ExpressionInfoVisitor(expression_info).visit(predicate_expression);
|
2019-01-30 02:47:26 +00:00
|
|
|
|
2019-12-18 03:56:03 +00:00
|
|
|
if (expression_info.is_stateful_function)
|
|
|
|
return {}; /// give up the optimization when the predicate contains stateful function
|
2019-02-15 08:03:18 +00:00
|
|
|
|
2019-12-18 03:56:03 +00:00
|
|
|
if (!expression_info.is_array_join)
|
|
|
|
{
|
|
|
|
if (expression_info.unique_reference_tables_pos.size() == 1)
|
|
|
|
tables_predicates[*expression_info.unique_reference_tables_pos.begin()].emplace_back(predicate_expression);
|
2020-03-08 21:40:00 +00:00
|
|
|
else if (expression_info.unique_reference_tables_pos.empty())
|
2019-12-18 03:56:03 +00:00
|
|
|
{
|
2020-03-09 00:28:05 +00:00
|
|
|
for (auto & predicate : tables_predicates)
|
|
|
|
predicate.emplace_back(predicate_expression);
|
2019-12-18 03:56:03 +00:00
|
|
|
}
|
2019-01-30 02:47:26 +00:00
|
|
|
}
|
2019-04-18 10:39:25 +00:00
|
|
|
}
|
|
|
|
|
2019-12-18 03:56:03 +00:00
|
|
|
return tables_predicates; /// everything is OK, it can be optimized
|
|
|
|
}
|
|
|
|
|
|
|
|
bool PredicateExpressionsOptimizer::tryRewritePredicatesToTables(ASTs & tables_element, const std::vector<ASTs> & tables_predicates)
|
|
|
|
{
|
|
|
|
bool is_rewrite_tables = false;
|
2019-01-30 02:47:26 +00:00
|
|
|
|
2019-12-18 03:56:03 +00:00
|
|
|
for (size_t index = tables_element.size(); index > 0; --index)
|
2019-04-18 10:39:25 +00:00
|
|
|
{
|
2019-12-18 03:56:03 +00:00
|
|
|
size_t table_pos = index - 1;
|
2019-04-18 10:39:25 +00:00
|
|
|
|
|
|
|
/// 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.
|
|
|
|
|
2019-12-18 03:56:03 +00:00
|
|
|
if (const auto & table_element = tables_element[table_pos]->as<ASTTablesInSelectQueryElement>())
|
2019-04-18 10:39:25 +00:00
|
|
|
{
|
2019-12-18 03:56:03 +00:00
|
|
|
if (table_element->table_join && isLeft(table_element->table_join->as<ASTTableJoin>()->kind))
|
|
|
|
continue; /// Skip right table optimization
|
2019-01-30 02:47:26 +00:00
|
|
|
|
2019-12-18 03:56:03 +00:00
|
|
|
if (table_element->table_join && isFull(table_element->table_join->as<ASTTableJoin>()->kind))
|
|
|
|
break; /// Skip left and right table optimization
|
2018-03-04 16:15:31 +00:00
|
|
|
|
2019-12-18 03:56:03 +00:00
|
|
|
is_rewrite_tables |= tryRewritePredicatesToTable(tables_element[table_pos], tables_predicates[table_pos],
|
2020-06-05 21:17:00 +00:00
|
|
|
tables_with_columns[table_pos].columns.getNames());
|
2018-03-04 16:15:31 +00:00
|
|
|
|
2019-12-18 03:56:03 +00:00
|
|
|
if (table_element->table_join && isRight(table_element->table_join->as<ASTTableJoin>()->kind))
|
|
|
|
break; /// Skip left table optimization
|
2018-03-04 16:15:31 +00:00
|
|
|
}
|
|
|
|
}
|
2019-01-25 15:42:24 +00:00
|
|
|
|
2019-12-18 03:56:03 +00:00
|
|
|
return is_rewrite_tables;
|
2018-03-04 16:15:31 +00:00
|
|
|
}
|
|
|
|
|
2020-06-05 21:28:44 +00:00
|
|
|
bool PredicateExpressionsOptimizer::tryRewritePredicatesToTable(ASTPtr & table_element, const ASTs & table_predicates, Names && table_columns) const
|
2019-01-25 15:42:24 +00:00
|
|
|
{
|
2019-12-18 03:56:03 +00:00
|
|
|
if (!table_predicates.empty())
|
2019-01-25 15:42:24 +00:00
|
|
|
{
|
2020-06-05 21:17:00 +00:00
|
|
|
auto optimize_final = enable_optimize_predicate_expression_to_final_subquery;
|
2020-06-05 21:28:44 +00:00
|
|
|
PredicateRewriteVisitor::Data data(context, table_predicates, std::move(table_columns), optimize_final);
|
2019-01-25 15:42:24 +00:00
|
|
|
|
2019-12-18 03:56:03 +00:00
|
|
|
PredicateRewriteVisitor(data).visit(table_element);
|
2020-01-04 04:31:45 +00:00
|
|
|
return data.is_rewrite;
|
2018-09-12 05:41:09 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
2020-01-04 04:31:45 +00:00
|
|
|
bool PredicateExpressionsOptimizer::tryMovePredicatesFromHavingToWhere(ASTSelectQuery & select_query)
|
|
|
|
{
|
|
|
|
ASTs where_predicates;
|
|
|
|
ASTs having_predicates;
|
|
|
|
|
|
|
|
const auto & reduce_predicates = [&](const ASTs & predicates)
|
|
|
|
{
|
|
|
|
ASTPtr res = predicates[0];
|
|
|
|
for (size_t index = 1; index < predicates.size(); ++index)
|
|
|
|
res = makeASTFunction("and", res, predicates[index]);
|
|
|
|
|
|
|
|
return res;
|
|
|
|
};
|
|
|
|
|
|
|
|
for (const auto & moving_predicate: splitConjunctionPredicate({select_query.having()}))
|
|
|
|
{
|
|
|
|
ExpressionInfoVisitor::Data expression_info{.context = context, .tables = {}};
|
|
|
|
ExpressionInfoVisitor(expression_info).visit(moving_predicate);
|
|
|
|
|
|
|
|
/// TODO: If there is no group by, where, and prewhere expression, we can push down the stateful function
|
|
|
|
if (expression_info.is_stateful_function)
|
|
|
|
return false;
|
|
|
|
|
|
|
|
if (expression_info.is_aggregate_function)
|
|
|
|
having_predicates.emplace_back(moving_predicate);
|
|
|
|
else
|
|
|
|
where_predicates.emplace_back(moving_predicate);
|
|
|
|
}
|
|
|
|
|
|
|
|
if (having_predicates.empty())
|
|
|
|
select_query.setExpression(ASTSelectQuery::Expression::HAVING, {});
|
|
|
|
else
|
|
|
|
{
|
|
|
|
auto having_predicate = reduce_predicates(having_predicates);
|
|
|
|
select_query.setExpression(ASTSelectQuery::Expression::HAVING, std::move(having_predicate));
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!where_predicates.empty())
|
|
|
|
{
|
|
|
|
auto moved_predicate = reduce_predicates(where_predicates);
|
|
|
|
moved_predicate = select_query.where() ? makeASTFunction("and", select_query.where(), moved_predicate) : moved_predicate;
|
|
|
|
select_query.setExpression(ASTSelectQuery::Expression::WHERE, std::move(moved_predicate));
|
|
|
|
}
|
|
|
|
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2018-03-04 16:15:31 +00:00
|
|
|
}
|