Better "optimize_move_to_prewhere" heuristic; simpler code

This commit is contained in:
Alexey Milovidov 2019-02-15 02:51:17 +03:00
parent 793b92987f
commit e758caa7a9
2 changed files with 142 additions and 200 deletions

View File

@ -10,21 +10,21 @@
#include <Parsers/ASTSubquery.h>
#include <Parsers/formatAST.h>
#include <Interpreters/QueryNormalizer.h>
#include <Common/escapeForFileName.h>
#include <Common/typeid_cast.h>
#include <DataTypes/NestedUtils.h>
#include <ext/scope_guard.h>
#include <ext/collection_cast.h>
#include <ext/map.h>
#include <memory>
#include <unordered_map>
#include <tuple>
#include <cstddef>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
/// Conditions like "x = N" are considered good if abs(N) > threshold.
/// This is used to assume that condition is likely to have good selectivity.
static constexpr auto threshold = 2;
@ -49,19 +49,6 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer(
}
void MergeTreeWhereOptimizer::optimize(ASTSelectQuery & select) const
{
if (!select.where_expression || select.prewhere_expression)
return;
const auto function = typeid_cast<ASTFunction *>(select.where_expression.get());
if (function && function->name == "and")
optimizeConjunction(select, function);
else
optimizeArbitrary(select);
}
void MergeTreeWhereOptimizer::calculateColumnSizes(const MergeTreeData & data, const Names & column_names)
{
for (const auto & column_name : column_names)
@ -69,177 +56,120 @@ void MergeTreeWhereOptimizer::calculateColumnSizes(const MergeTreeData & data, c
}
namespace
static void collectIdentifiersNoSubqueries(const ASTPtr & ast, NameSet & set)
{
struct ConditionCandidate
{
size_t columns_size;
int64_t position;
IdentifierNameSet identifiers;
bool is_good;
if (auto opt_name = getIdentifierName(ast))
return (void)set.insert(*opt_name);
auto tuple() const
{
/// We'll move conditions from back to keep "position".
return std::forward_as_tuple(!is_good, columns_size, -position);
}
if (typeid_cast<const ASTSubquery *>(ast.get()))
return;
bool operator< (const ConditionCandidate & rhs) const
{
return tuple() < rhs.tuple();
}
};
for (const auto & child : ast->children)
collectIdentifiersNoSubqueries(child, set);
}
void MergeTreeWhereOptimizer::optimizeConjunction(ASTSelectQuery & select, ASTFunction * const fun) const
void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const ASTPtr & node) const
{
std::vector<ConditionCandidate> condition_candidates;
auto & conditions = fun->arguments->children;
/// remove condition by swapping it with the last one and calling ::pop_back()
const auto remove_condition_at_index = [&conditions] (const size_t idx)
if (const auto func_and = typeid_cast<ASTFunction *>(node.get()); func_and && func_and->name == "and")
{
if (idx < conditions.size() - 1)
std::swap(conditions[idx], conditions.back());
conditions.pop_back();
};
/// linearize conjunction and extract possible conditions to move
for (size_t idx = 0; idx < conditions.size();)
for (const auto & elem : func_and->arguments->children)
analyzeImpl(res, elem);
}
else
{
const auto condition = conditions[idx].get();
/// linearize sub-conjunctions
if (const auto function = typeid_cast<ASTFunction *>(condition))
{
if (function->name == "and")
{
for (auto & child : function->arguments->children)
conditions.emplace_back(std::move(child));
/// remove the condition corresponding to conjunction
remove_condition_at_index(idx);
/// continue iterating without increment to ensure the just added conditions are processed
continue;
}
}
SCOPE_EXIT(++idx);
if (cannotBeMoved(conditions[idx]))
continue;
IdentifierNameSet identifiers;
collectIdentifiersNoSubqueries(condition, identifiers);
Condition cond;
cond.node = node;
/// do not take into consideration the conditions consisting only of the first primary key column
if (!hasPrimaryKeyAtoms(condition) && isSubsetOfTableColumns(identifiers))
collectIdentifiersNoSubqueries(node, cond.identifiers);
cond.viable = !cannotBeMoved(node) && !hasPrimaryKeyAtoms(node) && isSubsetOfTableColumns(cond.identifiers);
if (cond.viable)
{
ConditionCandidate candidate;
candidate.position = idx;
candidate.columns_size = getIdentifiersColumnSize(identifiers);
candidate.is_good = isConditionGood(condition);
candidate.identifiers = identifiers;
condition_candidates.emplace_back(std::move(candidate));
}
}
if (condition_candidates.empty())
return;
const auto move_condition_to_prewhere = [&] (const size_t idx)
{
if (!select.prewhere_expression)
{
select.prewhere_expression = conditions[idx];
select.children.push_back(select.prewhere_expression);
}
else if (auto func_and = typeid_cast<ASTFunction *>(select.prewhere_expression.get()); func_and && func_and->name == "and")
{
/// Add argument to AND chain
func_and->arguments->children.emplace_back(conditions[idx]);
}
else
{
/// Make old_cond AND new_cond
auto func = std::make_shared<ASTFunction>();
func->name = "and";
func->arguments->children = {select.prewhere_expression, conditions[idx]};
select.children.clear();
select.prewhere_expression = std::move(func);
select.children.push_back(select.prewhere_expression);
cond.columns_size = getIdentifiersColumnSize(cond.identifiers);
cond.good = isConditionGood(node);
}
remove_condition_at_index(idx);
};
/// Lightest conditions first. NOTE The algorithm is suboptimal, replace with priority_queue if you want.
std::sort(condition_candidates.begin(), condition_candidates.end());
/// Pick the best condition and also all other conditions with the same set of columns.
/// For example, if we take "EventTime >= '2014-03-20 00:00:00'", we will also take "EventTime < '2014-03-21 00:00:00'".
IdentifierNameSet identifiers_of_moved_condition = condition_candidates[0].identifiers;
move_condition_to_prewhere(condition_candidates[0].position);
for (size_t i = 1, size = condition_candidates.size(); i < size; ++i)
{
if (identifiers_of_moved_condition == condition_candidates[i].identifiers)
move_condition_to_prewhere(condition_candidates[i].position);
else
break;
res.emplace_back(std::move(cond));
}
/** Replace conjunction with the only remaining argument if only two conditions were present,
* remove selected condition from conjunction otherwise.
*/
if (conditions.size() == 1)
{
/// find old where_expression in children of select
const auto it = std::find(std::begin(select.children), std::end(select.children), select.where_expression);
/// replace where_expression with the remaining argument
select.where_expression = std::move(conditions.front());
/// overwrite child entry with the new where_expression
*it = select.where_expression;
}
else if (conditions.empty())
{
select.children.erase(std::find(std::begin(select.children), std::end(select.children), select.where_expression));
select.where_expression.reset();
}
if (select.prewhere_expression)
LOG_DEBUG(log, "MergeTreeWhereOptimizer: condition \"" << select.prewhere_expression << "\" moved to PREWHERE");
}
void MergeTreeWhereOptimizer::optimizeArbitrary(ASTSelectQuery & select) const
/// Transform conjunctions chain in WHERE expression to Conditions list.
MergeTreeWhereOptimizer::Conditions MergeTreeWhereOptimizer::analyze(const ASTPtr & expression) const
{
auto & condition = select.where_expression;
Conditions res;
analyzeImpl(res, expression);
return res;
}
/// do not optimize restricted expressions
if (cannotBeMoved(select.where_expression))
return;
/// Transform Conditions list to WHERE or PREWHERE expression.
ASTPtr MergeTreeWhereOptimizer::reconstruct(const Conditions & conditions) const
{
if (conditions.empty())
return {};
IdentifierNameSet identifiers{};
collectIdentifiersNoSubqueries(condition.get(), identifiers);
if (conditions.size() == 1)
return conditions.front().node;
if (hasPrimaryKeyAtoms(condition.get()) || !isSubsetOfTableColumns(identifiers))
return;
const auto function = std::make_shared<ASTFunction>();
/// add the condition to PREWHERE, remove it from WHERE
std::swap(select.prewhere_expression, condition);
LOG_DEBUG(log, "MergeTreeWhereOptimizer: condition `" << select.prewhere_expression << "` moved to PREWHERE");
function->name = "and";
function->arguments = std::make_shared<ASTExpressionList>();
function->children.push_back(function->arguments);
for (const auto & elem : conditions)
function->arguments->children.push_back(elem.node);
return function;
}
size_t MergeTreeWhereOptimizer::getIdentifiersColumnSize(const IdentifierNameSet & identifiers) const
void MergeTreeWhereOptimizer::optimize(ASTSelectQuery & select) const
{
if (!select.where_expression || select.prewhere_expression)
return;
Conditions where_conditions = analyze(select.where_expression);
Conditions prewhere_conditions;
auto it = std::min_element(where_conditions.begin(), where_conditions.end());
if (!it->viable)
return;
/// Move the best condition to PREWHERE if it is viable.
prewhere_conditions.splice(prewhere_conditions.end(), where_conditions, it);
/// Move all other conditions that depend on the same set of columns.
for (auto jt = where_conditions.begin(); jt != where_conditions.end();)
{
if (jt->columns_size == it->columns_size && jt->identifiers == it->identifiers)
prewhere_conditions.splice(prewhere_conditions.end(), where_conditions, jt++);
else
++jt;
}
/// Rewrite the SELECT query.
auto old_where = std::find(std::begin(select.children), std::end(select.children), select.where_expression);
if (old_where == select.children.end())
throw Exception("Logical error: cannot find WHERE expression in the list of children of SELECT query", ErrorCodes::LOGICAL_ERROR);
select.where_expression = reconstruct(where_conditions);
select.prewhere_expression = reconstruct(prewhere_conditions);
if (select.where_expression)
*old_where = select.where_expression;
else
select.children.erase(old_where);
select.children.push_back(select.prewhere_expression);
LOG_DEBUG(log, "MergeTreeWhereOptimizer: condition \"" << select.prewhere_expression << "\" moved to PREWHERE");
}
size_t MergeTreeWhereOptimizer::getIdentifiersColumnSize(const NameSet & identifiers) const
{
/** for expressions containing no columns (or where columns could not be determined otherwise) assume maximum
* possible size so they do not have priority in eligibility over other expressions. */
@ -256,9 +186,9 @@ size_t MergeTreeWhereOptimizer::getIdentifiersColumnSize(const IdentifierNameSet
}
bool MergeTreeWhereOptimizer::isConditionGood(const IAST * condition) const
bool MergeTreeWhereOptimizer::isConditionGood(const ASTPtr & condition) const
{
const auto function = typeid_cast<const ASTFunction *>(condition);
const auto function = typeid_cast<const ASTFunction *>(condition.get());
if (!function)
return false;
@ -305,29 +235,16 @@ bool MergeTreeWhereOptimizer::isConditionGood(const IAST * condition) const
}
void MergeTreeWhereOptimizer::collectIdentifiersNoSubqueries(const IAST * const ast, IdentifierNameSet & set)
bool MergeTreeWhereOptimizer::hasPrimaryKeyAtoms(const ASTPtr & ast) const
{
if (auto opt_name = getIdentifierName(ast))
return (void) set.insert(*opt_name);
if (typeid_cast<const ASTSubquery *>(ast))
return;
for (const auto & child : ast->children)
collectIdentifiersNoSubqueries(child.get(), set);
}
bool MergeTreeWhereOptimizer::hasPrimaryKeyAtoms(const IAST * ast) const
{
if (const auto func = typeid_cast<const ASTFunction *>(ast))
if (const auto func = typeid_cast<const ASTFunction *>(ast.get()))
{
const auto & args = func->arguments->children;
if ((func->name == "not" && 1 == args.size()) || func->name == "and" || func->name == "or")
{
for (const auto & arg : args)
if (hasPrimaryKeyAtoms(arg.get()))
if (hasPrimaryKeyAtoms(arg))
return true;
return false;
@ -338,9 +255,9 @@ bool MergeTreeWhereOptimizer::hasPrimaryKeyAtoms(const IAST * ast) const
}
bool MergeTreeWhereOptimizer::isPrimaryKeyAtom(const IAST * const ast) const
bool MergeTreeWhereOptimizer::isPrimaryKeyAtom(const ASTPtr & ast) const
{
if (const auto func = typeid_cast<const ASTFunction *>(ast))
if (const auto func = typeid_cast<const ASTFunction *>(ast.get()))
{
if (!KeyCondition::atom_map.count(func->name))
return false;
@ -366,15 +283,15 @@ bool MergeTreeWhereOptimizer::isConstant(const ASTPtr & expr) const
{
const auto column_name = expr->getColumnName();
if (typeid_cast<const ASTLiteral *>(expr.get()) ||
(block_with_constants.has(column_name) && block_with_constants.getByName(column_name).column->isColumnConst()))
if (typeid_cast<const ASTLiteral *>(expr.get())
|| (block_with_constants.has(column_name) && block_with_constants.getByName(column_name).column->isColumnConst()))
return true;
return false;
}
bool MergeTreeWhereOptimizer::isSubsetOfTableColumns(const IdentifierNameSet & identifiers) const
bool MergeTreeWhereOptimizer::isSubsetOfTableColumns(const NameSet & identifiers) const
{
for (const auto & identifier : identifiers)
if (table_columns.count(identifier) == 0)

View File

@ -17,9 +17,6 @@ class ASTSelectQuery;
class ASTFunction;
class MergeTreeData;
using IdentifierNameSet = std::set<std::string>;
/** Identifies WHERE expressions that can be placed in PREWHERE by calculating respective
* sizes of columns used in particular expression and identifying "good" conditions of
* form "column_name = constant", where "constant" is outside some `threshold` specified in advance.
@ -40,25 +37,53 @@ public:
private:
void optimize(ASTSelectQuery & select) const;
struct Condition
{
ASTPtr node;
size_t columns_size = 0;
NameSet identifiers;
bool viable = false;
bool good = false;
auto tuple() const
{
return std::make_tuple(!viable, !good, columns_size);
}
/// Is condition a better candidate for moving to PREWHERE?
bool operator< (const Condition & rhs) const
{
return tuple() < rhs.tuple();
}
};
using Conditions = std::list<Condition>;
void analyzeImpl(Conditions & res, const ASTPtr & node) const;
/// Transform conjunctions chain in WHERE expression to Conditions list.
Conditions analyze(const ASTPtr & expression) const;
/// Transform Conditions list to WHERE or PREWHERE expression.
ASTPtr reconstruct(const Conditions & conditions) const;
void calculateColumnSizes(const MergeTreeData & data, const Names & column_names);
void optimizeConjunction(ASTSelectQuery & select, ASTFunction * const fun) const;
void optimizeArbitrary(ASTSelectQuery & select) const;
size_t getIdentifiersColumnSize(const IdentifierNameSet & identifiers) const;
size_t getIdentifiersColumnSize(const NameSet & identifiers) const;
bool isConditionGood(const IAST * condition) const;
bool isConditionGood(const ASTPtr & condition) const;
static void collectIdentifiersNoSubqueries(const IAST * const ast, IdentifierNameSet & set);
bool hasPrimaryKeyAtoms(const ASTPtr & ast) const;
bool hasPrimaryKeyAtoms(const IAST * ast) const;
bool isPrimaryKeyAtom(const IAST * const ast) const;
bool isPrimaryKeyAtom(const ASTPtr & ast) const;
bool isConstant(const ASTPtr & expr) const;
bool isSubsetOfTableColumns(const IdentifierNameSet & identifiers) const;
bool isSubsetOfTableColumns(const NameSet & identifiers) const;
/** ARRAY JOIN'ed columns as well as arrayJoin() result cannot be used in PREWHERE, therefore expressions
* containing said columns should not be moved to PREWHERE at all.