2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeWhereOptimizer.h>
|
|
|
|
#include <Storages/MergeTree/MergeTreeData.h>
|
2018-04-20 00:20:36 +00:00
|
|
|
#include <Storages/MergeTree/KeyCondition.h>
|
2019-01-25 15:42:24 +00:00
|
|
|
#include <Interpreters/IdentifierSemantic.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Parsers/ASTSelectQuery.h>
|
|
|
|
#include <Parsers/ASTFunction.h>
|
|
|
|
#include <Parsers/ASTIdentifier.h>
|
|
|
|
#include <Parsers/ASTLiteral.h>
|
|
|
|
#include <Parsers/ASTExpressionList.h>
|
|
|
|
#include <Parsers/ASTSubquery.h>
|
|
|
|
#include <Parsers/formatAST.h>
|
2019-10-23 13:59:03 +00:00
|
|
|
#include <Interpreters/misc.h>
|
2017-07-13 20:58:19 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2017-12-25 18:58:39 +00:00
|
|
|
#include <DataTypes/NestedUtils.h>
|
2017-06-06 17:18:32 +00:00
|
|
|
#include <ext/map.h>
|
2017-01-14 09:00:19 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2019-02-14 23:51:17 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Conditions like "x = N" are considered good if abs(N) > threshold.
|
|
|
|
/// This is used to assume that condition is likely to have good selectivity.
|
2019-02-14 19:59:51 +00:00
|
|
|
static constexpr auto threshold = 2;
|
2017-01-14 09:00:19 +00:00
|
|
|
|
|
|
|
|
|
|
|
MergeTreeWhereOptimizer::MergeTreeWhereOptimizer(
|
2017-07-15 03:48:36 +00:00
|
|
|
SelectQueryInfo & query_info,
|
2017-07-14 00:33:37 +00:00
|
|
|
const Context & context,
|
2021-02-06 15:08:42 +00:00
|
|
|
std::unordered_map<std::string, UInt64> column_sizes_,
|
2020-06-17 12:39:20 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
2019-08-03 11:02:40 +00:00
|
|
|
const Names & queried_columns_,
|
2020-05-30 21:57:37 +00:00
|
|
|
Poco::Logger * log_)
|
2020-06-17 12:39:20 +00:00
|
|
|
: table_columns{ext::map<std::unordered_set>(
|
|
|
|
metadata_snapshot->getColumns().getAllPhysical(), [](const NameAndTypePair & col) { return col.name; })}
|
2021-03-17 14:19:10 +00:00
|
|
|
, queried_columns{queried_columns_}
|
2021-03-24 08:35:20 +00:00
|
|
|
, sorting_key_names{NameSet(
|
|
|
|
metadata_snapshot->getSortingKey().column_names.begin(), metadata_snapshot->getSortingKey().column_names.end())}
|
2021-03-29 08:12:02 +00:00
|
|
|
, block_with_constants{KeyCondition::getBlockWithConstants(query_info.query->clone(), query_info.syntax_analyzer_result, context)}
|
2020-06-17 12:39:20 +00:00
|
|
|
, log{log_}
|
2021-02-06 15:08:42 +00:00
|
|
|
, column_sizes{std::move(column_sizes_)}
|
2017-01-14 09:00:19 +00:00
|
|
|
{
|
2020-06-17 12:39:20 +00:00
|
|
|
const auto & primary_key = metadata_snapshot->getPrimaryKey();
|
2020-05-21 19:46:03 +00:00
|
|
|
if (!primary_key.column_names.empty())
|
|
|
|
first_primary_key_column = primary_key.column_names[0];
|
2019-02-14 16:48:41 +00:00
|
|
|
|
2021-02-06 15:08:42 +00:00
|
|
|
for (const auto & [_, size] : column_sizes)
|
|
|
|
total_size_of_queried_columns += size;
|
|
|
|
|
2019-03-15 16:14:13 +00:00
|
|
|
determineArrayJoinedNames(query_info.query->as<ASTSelectQuery &>());
|
|
|
|
optimize(query_info.query->as<ASTSelectQuery &>());
|
2017-01-14 09:00:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-02-14 23:51:17 +00:00
|
|
|
static void collectIdentifiersNoSubqueries(const ASTPtr & ast, NameSet & set)
|
2019-02-14 19:59:51 +00:00
|
|
|
{
|
2019-08-08 20:10:58 +00:00
|
|
|
if (auto opt_name = tryGetIdentifierName(ast))
|
2019-02-14 23:51:17 +00:00
|
|
|
return (void)set.insert(*opt_name);
|
2019-02-14 19:59:51 +00:00
|
|
|
|
2019-03-11 13:22:51 +00:00
|
|
|
if (ast->as<ASTSubquery>())
|
2019-02-14 23:51:17 +00:00
|
|
|
return;
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2019-02-14 23:51:17 +00:00
|
|
|
for (const auto & child : ast->children)
|
|
|
|
collectIdentifiersNoSubqueries(child, set);
|
2017-01-14 09:00:19 +00:00
|
|
|
}
|
|
|
|
|
2020-03-18 00:57:00 +00:00
|
|
|
static bool isConditionGood(const ASTPtr & condition)
|
|
|
|
{
|
|
|
|
const auto * function = condition->as<ASTFunction>();
|
|
|
|
if (!function)
|
|
|
|
return false;
|
|
|
|
|
|
|
|
/** we are only considering conditions of form `equals(one, another)` or `one = another`,
|
|
|
|
* especially if either `one` or `another` is ASTIdentifier */
|
|
|
|
if (function->name != "equals")
|
|
|
|
return false;
|
|
|
|
|
2020-04-22 06:34:20 +00:00
|
|
|
auto * left_arg = function->arguments->children.front().get();
|
|
|
|
auto * right_arg = function->arguments->children.back().get();
|
2020-03-18 00:57:00 +00:00
|
|
|
|
|
|
|
/// try to ensure left_arg points to ASTIdentifier
|
|
|
|
if (!left_arg->as<ASTIdentifier>() && right_arg->as<ASTIdentifier>())
|
|
|
|
std::swap(left_arg, right_arg);
|
|
|
|
|
|
|
|
if (left_arg->as<ASTIdentifier>())
|
|
|
|
{
|
|
|
|
/// condition may be "good" if only right_arg is a constant and its value is outside the threshold
|
|
|
|
if (const auto * literal = right_arg->as<ASTLiteral>())
|
|
|
|
{
|
|
|
|
const auto & field = literal->value;
|
|
|
|
const auto type = field.getType();
|
|
|
|
|
|
|
|
/// check the value with respect to threshold
|
|
|
|
if (type == Field::Types::UInt64)
|
|
|
|
{
|
|
|
|
const auto value = field.get<UInt64>();
|
|
|
|
return value > threshold;
|
|
|
|
}
|
|
|
|
else if (type == Field::Types::Int64)
|
|
|
|
{
|
|
|
|
const auto value = field.get<Int64>();
|
|
|
|
return value < -threshold || threshold < value;
|
|
|
|
}
|
|
|
|
else if (type == Field::Types::Float64)
|
|
|
|
{
|
|
|
|
const auto value = field.get<Float64>();
|
|
|
|
return value < threshold || threshold < value;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-03-17 15:56:55 +00:00
|
|
|
void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const ASTPtr & node, bool is_final) const
|
2017-01-14 09:00:19 +00:00
|
|
|
{
|
2019-03-11 13:22:51 +00:00
|
|
|
if (const auto * func_and = node->as<ASTFunction>(); func_and && func_and->name == "and")
|
2018-01-10 00:04:08 +00:00
|
|
|
{
|
2019-02-14 23:51:17 +00:00
|
|
|
for (const auto & elem : func_and->arguments->children)
|
2021-03-17 15:56:55 +00:00
|
|
|
analyzeImpl(res, elem, is_final);
|
2019-02-14 23:51:17 +00:00
|
|
|
}
|
|
|
|
else
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-02-14 23:51:17 +00:00
|
|
|
Condition cond;
|
|
|
|
cond.node = node;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-02-14 23:51:17 +00:00
|
|
|
collectIdentifiersNoSubqueries(node, cond.identifiers);
|
2019-02-15 19:00:19 +00:00
|
|
|
|
2020-01-13 18:12:53 +00:00
|
|
|
cond.columns_size = getIdentifiersColumnSize(cond.identifiers);
|
|
|
|
|
2019-02-15 19:00:19 +00:00
|
|
|
cond.viable =
|
|
|
|
/// Condition depend on some column. Constant expressions are not moved.
|
|
|
|
!cond.identifiers.empty()
|
2021-03-17 15:56:55 +00:00
|
|
|
&& !cannotBeMoved(node, is_final)
|
2019-02-15 19:00:19 +00:00
|
|
|
/// Do not take into consideration the conditions consisting only of the first primary key column
|
|
|
|
&& !hasPrimaryKeyAtoms(node)
|
2019-02-15 19:25:29 +00:00
|
|
|
/// Only table columns are considered. Not array joined columns. NOTE We're assuming that aliases was expanded.
|
2019-02-15 19:00:19 +00:00
|
|
|
&& isSubsetOfTableColumns(cond.identifiers)
|
|
|
|
/// Do not move conditions involving all queried columns.
|
2020-11-23 18:27:59 +00:00
|
|
|
&& cond.identifiers.size() < queried_columns.size();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-02-14 23:51:17 +00:00
|
|
|
if (cond.viable)
|
|
|
|
cond.good = isConditionGood(node);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-02-14 23:51:17 +00:00
|
|
|
res.emplace_back(std::move(cond));
|
|
|
|
}
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-02-14 23:51:17 +00:00
|
|
|
/// Transform conjunctions chain in WHERE expression to Conditions list.
|
2021-03-17 15:56:55 +00:00
|
|
|
MergeTreeWhereOptimizer::Conditions MergeTreeWhereOptimizer::analyze(const ASTPtr & expression, bool is_final) const
|
2019-02-14 23:51:17 +00:00
|
|
|
{
|
|
|
|
Conditions res;
|
2021-03-17 15:56:55 +00:00
|
|
|
analyzeImpl(res, expression, is_final);
|
2019-02-14 23:51:17 +00:00
|
|
|
return res;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-02-14 23:51:17 +00:00
|
|
|
/// Transform Conditions list to WHERE or PREWHERE expression.
|
2020-03-18 00:57:00 +00:00
|
|
|
ASTPtr MergeTreeWhereOptimizer::reconstruct(const Conditions & conditions)
|
2019-02-14 23:51:17 +00:00
|
|
|
{
|
|
|
|
if (conditions.empty())
|
|
|
|
return {};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-02-14 23:51:17 +00:00
|
|
|
if (conditions.size() == 1)
|
|
|
|
return conditions.front().node;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-02-14 23:51:17 +00:00
|
|
|
const auto function = std::make_shared<ASTFunction>();
|
2019-02-14 19:59:51 +00:00
|
|
|
|
2019-02-14 23:51:17 +00:00
|
|
|
function->name = "and";
|
|
|
|
function->arguments = std::make_shared<ASTExpressionList>();
|
|
|
|
function->children.push_back(function->arguments);
|
2019-02-14 19:59:51 +00:00
|
|
|
|
2019-02-14 23:51:17 +00:00
|
|
|
for (const auto & elem : conditions)
|
|
|
|
function->arguments->children.push_back(elem.node);
|
2019-02-14 19:59:51 +00:00
|
|
|
|
2019-02-14 23:51:17 +00:00
|
|
|
return function;
|
|
|
|
}
|
2019-02-14 19:59:51 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-02-14 23:51:17 +00:00
|
|
|
void MergeTreeWhereOptimizer::optimize(ASTSelectQuery & select) const
|
|
|
|
{
|
2019-04-09 14:22:35 +00:00
|
|
|
if (!select.where() || select.prewhere())
|
2019-02-14 23:51:17 +00:00
|
|
|
return;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-03-17 10:49:24 +00:00
|
|
|
Conditions where_conditions = analyze(select.where(), select.final());
|
2019-02-14 23:51:17 +00:00
|
|
|
Conditions prewhere_conditions;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-02-15 19:25:29 +00:00
|
|
|
UInt64 total_size_of_moved_conditions = 0;
|
2020-11-23 18:27:59 +00:00
|
|
|
UInt64 total_number_of_moved_columns = 0;
|
2019-02-14 19:59:51 +00:00
|
|
|
|
2019-02-15 19:25:29 +00:00
|
|
|
/// Move condition and all other conditions depend on the same set of columns.
|
|
|
|
auto move_condition = [&](Conditions::iterator cond_it)
|
|
|
|
{
|
|
|
|
prewhere_conditions.splice(prewhere_conditions.end(), where_conditions, cond_it);
|
|
|
|
total_size_of_moved_conditions += cond_it->columns_size;
|
2020-11-23 18:27:59 +00:00
|
|
|
total_number_of_moved_columns += cond_it->identifiers.size();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-04-11 22:13:06 +00:00
|
|
|
/// Move all other viable conditions that depend on the same set of columns.
|
2019-02-15 19:25:29 +00:00
|
|
|
for (auto jt = where_conditions.begin(); jt != where_conditions.end();)
|
|
|
|
{
|
2020-04-11 22:13:06 +00:00
|
|
|
if (jt->viable && jt->columns_size == cond_it->columns_size && jt->identifiers == cond_it->identifiers)
|
2019-02-15 19:25:29 +00:00
|
|
|
prewhere_conditions.splice(prewhere_conditions.end(), where_conditions, jt++);
|
|
|
|
else
|
|
|
|
++jt;
|
|
|
|
}
|
|
|
|
};
|
2019-02-14 19:59:51 +00:00
|
|
|
|
2019-02-15 19:25:29 +00:00
|
|
|
/// Move conditions unless the ratio of total_size_of_moved_conditions to the total_size_of_queried_columns is less than some threshold.
|
|
|
|
while (!where_conditions.empty())
|
2019-02-14 20:09:44 +00:00
|
|
|
{
|
2019-02-15 19:25:29 +00:00
|
|
|
/// Move the best condition to PREWHERE if it is viable.
|
|
|
|
|
|
|
|
auto it = std::min_element(where_conditions.begin(), where_conditions.end());
|
|
|
|
|
|
|
|
if (!it->viable)
|
|
|
|
break;
|
|
|
|
|
|
|
|
/// 10% ratio is just a guess.
|
2020-11-23 18:27:59 +00:00
|
|
|
/// If sizes of compressed columns cannot be calculated, e.g. for compact parts,
|
|
|
|
/// use number of moved columns as a fallback.
|
|
|
|
bool moved_enough =
|
|
|
|
(total_size_of_queried_columns > 0 && total_size_of_moved_conditions > 0
|
|
|
|
&& (total_size_of_moved_conditions + it->columns_size) * 10 > total_size_of_queried_columns)
|
|
|
|
|| (total_number_of_moved_columns > 0
|
|
|
|
&& (total_number_of_moved_columns + it->identifiers.size()) * 10 > queried_columns.size());
|
|
|
|
|
|
|
|
if (moved_enough)
|
2019-02-15 19:25:29 +00:00
|
|
|
break;
|
|
|
|
|
|
|
|
move_condition(it);
|
2019-02-14 20:09:44 +00:00
|
|
|
}
|
|
|
|
|
2019-02-15 19:25:29 +00:00
|
|
|
/// Nothing was moved.
|
|
|
|
if (prewhere_conditions.empty())
|
|
|
|
return;
|
|
|
|
|
2019-02-14 23:51:17 +00:00
|
|
|
/// Rewrite the SELECT query.
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2019-04-09 14:22:35 +00:00
|
|
|
select.setExpression(ASTSelectQuery::Expression::WHERE, reconstruct(where_conditions));
|
|
|
|
select.setExpression(ASTSelectQuery::Expression::PREWHERE, reconstruct(prewhere_conditions));
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2020-05-23 22:24:01 +00:00
|
|
|
LOG_DEBUG(log, "MergeTreeWhereOptimizer: condition \"{}\" moved to PREWHERE", select.prewhere());
|
2017-01-14 09:00:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-02-15 19:25:29 +00:00
|
|
|
UInt64 MergeTreeWhereOptimizer::getIdentifiersColumnSize(const NameSet & identifiers) const
|
2017-01-14 09:00:19 +00:00
|
|
|
{
|
2019-02-15 19:25:29 +00:00
|
|
|
UInt64 size = 0;
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
for (const auto & identifier : identifiers)
|
|
|
|
if (column_sizes.count(identifier))
|
2019-02-15 19:25:29 +00:00
|
|
|
size += column_sizes.at(identifier);
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return size;
|
2017-01-14 09:00:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-02-14 23:51:17 +00:00
|
|
|
bool MergeTreeWhereOptimizer::hasPrimaryKeyAtoms(const ASTPtr & ast) const
|
2017-01-14 09:00:19 +00:00
|
|
|
{
|
2019-03-11 13:22:51 +00:00
|
|
|
if (const auto * func = ast->as<ASTFunction>())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
const auto & args = func->arguments->children;
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if ((func->name == "not" && 1 == args.size()) || func->name == "and" || func->name == "or")
|
|
|
|
{
|
|
|
|
for (const auto & arg : args)
|
2019-02-14 23:51:17 +00:00
|
|
|
if (hasPrimaryKeyAtoms(arg))
|
2017-04-01 07:20:54 +00:00
|
|
|
return true;
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return false;
|
|
|
|
}
|
|
|
|
}
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return isPrimaryKeyAtom(ast);
|
2017-01-14 09:00:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-02-14 23:51:17 +00:00
|
|
|
bool MergeTreeWhereOptimizer::isPrimaryKeyAtom(const ASTPtr & ast) const
|
2017-01-14 09:00:19 +00:00
|
|
|
{
|
2019-03-11 13:22:51 +00:00
|
|
|
if (const auto * func = ast->as<ASTFunction>())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-04-20 00:20:36 +00:00
|
|
|
if (!KeyCondition::atom_map.count(func->name))
|
2017-04-01 07:20:54 +00:00
|
|
|
return false;
|
|
|
|
|
|
|
|
const auto & args = func->arguments->children;
|
|
|
|
if (args.size() != 2)
|
|
|
|
return false;
|
|
|
|
|
|
|
|
const auto & first_arg_name = args.front()->getColumnName();
|
|
|
|
const auto & second_arg_name = args.back()->getColumnName();
|
|
|
|
|
2019-02-14 16:48:41 +00:00
|
|
|
if ((first_primary_key_column == first_arg_name && isConstant(args[1]))
|
|
|
|
|| (first_primary_key_column == second_arg_name && isConstant(args[0]))
|
|
|
|
|| (first_primary_key_column == first_arg_name && functionIsInOrGlobalInOperator(func->name)))
|
2017-04-01 07:20:54 +00:00
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
return false;
|
2017-01-14 09:00:19 +00:00
|
|
|
}
|
|
|
|
|
2021-03-17 16:25:43 +00:00
|
|
|
|
2021-03-24 08:35:20 +00:00
|
|
|
bool MergeTreeWhereOptimizer::isSortingKey(const String & column_name) const
|
2021-03-17 16:25:43 +00:00
|
|
|
{
|
2021-03-24 08:35:20 +00:00
|
|
|
return sorting_key_names.count(column_name);
|
2021-03-17 10:49:24 +00:00
|
|
|
}
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2021-03-17 16:25:43 +00:00
|
|
|
|
2017-01-14 09:00:19 +00:00
|
|
|
bool MergeTreeWhereOptimizer::isConstant(const ASTPtr & expr) const
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
const auto column_name = expr->getColumnName();
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2020-03-09 00:28:05 +00:00
|
|
|
return expr->as<ASTLiteral>()
|
|
|
|
|| (block_with_constants.has(column_name) && isColumnConst(*block_with_constants.getByName(column_name).column));
|
2017-01-14 09:00:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-02-14 23:51:17 +00:00
|
|
|
bool MergeTreeWhereOptimizer::isSubsetOfTableColumns(const NameSet & identifiers) const
|
2017-01-14 09:00:19 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
for (const auto & identifier : identifiers)
|
|
|
|
if (table_columns.count(identifier) == 0)
|
|
|
|
return false;
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return true;
|
2017-01-14 09:00:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-03-17 15:56:55 +00:00
|
|
|
bool MergeTreeWhereOptimizer::cannotBeMoved(const ASTPtr & ptr, bool is_final) const
|
2017-01-14 09:00:19 +00:00
|
|
|
{
|
2019-03-11 13:22:51 +00:00
|
|
|
if (const auto * function_ptr = ptr->as<ASTFunction>())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
/// disallow arrayJoin expressions to be moved to PREWHERE for now
|
2019-02-14 19:59:51 +00:00
|
|
|
if ("arrayJoin" == function_ptr->name)
|
2017-04-01 07:20:54 +00:00
|
|
|
return true;
|
|
|
|
|
|
|
|
/// disallow GLOBAL IN, GLOBAL NOT IN
|
2019-02-14 19:59:51 +00:00
|
|
|
if ("globalIn" == function_ptr->name
|
|
|
|
|| "globalNotIn" == function_ptr->name)
|
2017-04-01 07:20:54 +00:00
|
|
|
return true;
|
|
|
|
}
|
2019-01-25 15:42:24 +00:00
|
|
|
else if (auto opt_name = IdentifierSemantic::getColumnName(ptr))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
/// disallow moving result of ARRAY JOIN to PREWHERE
|
2019-01-14 18:15:04 +00:00
|
|
|
if (array_joined_names.count(*opt_name) ||
|
2021-03-17 10:49:24 +00:00
|
|
|
array_joined_names.count(Nested::extractTableName(*opt_name)) ||
|
2021-03-24 08:35:20 +00:00
|
|
|
(is_final && !isSortingKey(*opt_name)))
|
2019-01-14 18:15:04 +00:00
|
|
|
return true;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
for (const auto & child : ptr->children)
|
2021-03-17 15:56:55 +00:00
|
|
|
if (cannotBeMoved(child, is_final))
|
2017-04-01 07:20:54 +00:00
|
|
|
return true;
|
|
|
|
|
|
|
|
return false;
|
2017-01-14 09:00:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void MergeTreeWhereOptimizer::determineArrayJoinedNames(ASTSelectQuery & select)
|
|
|
|
{
|
2020-03-23 02:12:31 +00:00
|
|
|
auto array_join_expression_list = select.arrayJoinExpressionList();
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// much simplified code from ExpressionAnalyzer::getArrayJoinedColumns()
|
|
|
|
if (!array_join_expression_list)
|
|
|
|
return;
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
for (const auto & ast : array_join_expression_list->children)
|
|
|
|
array_joined_names.emplace(ast->getAliasOrColumnName());
|
2017-01-14 09:00:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|