ClickHouse/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

215 lines
7.7 KiB
C++
Raw Normal View History

2021-11-21 19:14:20 +00:00
#include <Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.h>
2021-05-02 19:16:40 +00:00
#include <Storages/MergeTree/MergeTreeIndexHypothesis.h>
#include <Interpreters/TreeCNFConverter.h>
#include <Interpreters/ComparisonGraph.h>
#include <Parsers/IAST_fwd.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSelectQuery.h>
2021-05-02 19:16:40 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
2021-11-21 19:14:20 +00:00
MergeTreeIndexhypothesisMergedCondition::MergeTreeIndexhypothesisMergedCondition(
const SelectQueryInfo & query, const ConstraintsDescription & constraints, size_t granularity_)
: IMergeTreeIndexMergedCondition(granularity_)
2021-05-02 19:16:40 +00:00
{
2021-11-21 19:14:20 +00:00
const auto & select = query.query->as<ASTSelectQuery &>();
2021-05-02 19:16:40 +00:00
if (select.where() && select.prewhere())
expression_ast = makeASTFunction(
"and",
select.where()->clone(),
select.prewhere()->clone());
else if (select.where())
expression_ast = select.where()->clone();
else if (select.prewhere())
expression_ast = select.prewhere()->clone();
expression_cnf = std::make_unique<CNFQuery>(
expression_ast ? TreeCNFConverter::toCNF(expression_ast) : CNFQuery::AndGroup{});
2021-11-21 19:14:20 +00:00
addConstraints(constraints);
2021-05-02 19:16:40 +00:00
}
2021-11-21 19:14:20 +00:00
void MergeTreeIndexhypothesisMergedCondition::addIndex(const MergeTreeIndexPtr & index)
2021-05-02 19:16:40 +00:00
{
if (!index->isMergeable() || index->getGranularity() != granularity)
2021-11-21 19:14:20 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR, "Index {} can not be merged", index->index.type);
2021-05-02 19:16:40 +00:00
const auto hypothesis_index = std::dynamic_pointer_cast<const MergeTreeIndexHypothesis>(index);
if (!hypothesis_index)
2021-11-21 19:14:20 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR, "Only hypothesis index is supported here");
2021-05-02 19:16:40 +00:00
2021-11-21 19:14:20 +00:00
static const NameSet relations = { "equals", "notEquals", "less", "lessOrEquals", "greaterOrEquals", "greater"};
2021-05-02 19:16:40 +00:00
// TODO: move to index hypothesis
std::vector<ASTPtr> compare_hypotheses_data;
std::vector<CNFQuery::OrGroup> hypotheses_data;
const auto cnf = TreeCNFConverter::toCNF(hypothesis_index->index.expression_list_ast->children.front()).pullNotOutFunctions();
2021-11-21 19:14:20 +00:00
2021-05-03 19:08:26 +00:00
for (const auto & group : cnf.getStatements())
{
2021-05-02 19:16:40 +00:00
if (group.size() == 1)
{
2021-05-03 19:08:26 +00:00
hypotheses_data.push_back(group);
2021-05-06 08:29:24 +00:00
CNFQuery::AtomicFormula atomic_formula = *group.begin();
CNFQuery::AtomicFormula atom{atomic_formula.negative, atomic_formula.ast->clone()};
2021-05-02 19:16:40 +00:00
pushNotIn(atom);
2021-11-21 19:14:20 +00:00
assert(!atom.negative);
2021-05-02 19:16:40 +00:00
2021-11-21 19:14:20 +00:00
const auto * func = atom.ast->as<ASTFunction>();
if (func && relations.contains(func->name))
2021-05-02 19:16:40 +00:00
compare_hypotheses_data.push_back(atom.ast);
}
}
2021-11-21 19:14:20 +00:00
2021-05-02 19:16:40 +00:00
index_to_compare_atomic_hypotheses.push_back(compare_hypotheses_data);
index_to_atomic_hypotheses.push_back(hypotheses_data);
}
2021-11-21 19:14:20 +00:00
void MergeTreeIndexhypothesisMergedCondition::addConstraints(const ConstraintsDescription & constraints_description)
2021-05-02 19:16:40 +00:00
{
auto atomic_constraints_data = constraints_description.getAtomicConstraintData();
2021-05-06 08:29:24 +00:00
for (const auto & atomic_formula : atomic_constraints_data)
2021-05-02 19:16:40 +00:00
{
2021-05-06 08:29:24 +00:00
CNFQuery::AtomicFormula atom{atomic_formula.negative, atomic_formula.ast->clone()};
2021-05-02 19:16:40 +00:00
pushNotIn(atom);
atomic_constraints.push_back(atom.ast);
}
}
2021-05-03 19:08:26 +00:00
/// Replaces < -> <=, > -> >= and assumes that all hypotheses are true then checks if path exists
2021-11-21 19:14:20 +00:00
bool MergeTreeIndexhypothesisMergedCondition::alwaysUnknownOrTrue() const
2021-05-04 18:43:58 +00:00
{
ASTs active_atomic_formulas(atomic_constraints);
2021-05-06 08:29:24 +00:00
for (const auto & hypothesis : index_to_compare_atomic_hypotheses)
2021-05-03 19:08:26 +00:00
{
active_atomic_formulas.insert(
std::end(active_atomic_formulas),
2021-05-06 08:29:24 +00:00
std::begin(hypothesis),
std::end(hypothesis));
2021-05-03 19:08:26 +00:00
}
/// transform active formulas
for (auto & formula : active_atomic_formulas)
{
formula = formula->clone(); /// do all operations with copy
auto * func = formula->as<ASTFunction>();
if (func && func->name == "less")
func->name = "lessOrEquals";
if (func && func->name == "greater")
func->name = "greaterOrEquals";
}
const auto weak_graph = std::make_unique<ComparisonGraph>(active_atomic_formulas);
bool useless = true;
expression_cnf->iterateGroups(
[&](const CNFQuery::OrGroup & or_group)
{
2021-05-06 08:29:24 +00:00
for (const auto & atomic_formula : or_group)
2021-05-03 19:08:26 +00:00
{
2021-05-06 08:29:24 +00:00
CNFQuery::AtomicFormula atom{atomic_formula.negative, atomic_formula.ast->clone()};
2021-05-03 19:08:26 +00:00
pushNotIn(atom);
2021-11-21 19:14:20 +00:00
2021-05-03 19:08:26 +00:00
const auto * func = atom.ast->as<ASTFunction>();
if (func && func->arguments->children.size() == 2)
{
const auto left = weak_graph->getComponentId(func->arguments->children[0]);
const auto right = weak_graph->getComponentId(func->arguments->children[1]);
if (left && right && weak_graph->hasPath(left.value(), right.value()))
{
useless = false;
return;
}
}
}
});
return useless;
}
2021-11-21 19:14:20 +00:00
bool MergeTreeIndexhypothesisMergedCondition::mayBeTrueOnGranule(const MergeTreeIndexGranules & granules) const
2021-05-02 19:16:40 +00:00
{
std::vector<bool> values;
for (const auto & index_granule : granules)
{
const auto granule = std::dynamic_pointer_cast<const MergeTreeIndexGranuleHypothesis>(index_granule);
if (!granule)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Only hypothesis index is supported here.");
2021-05-02 19:16:40 +00:00
values.push_back(granule->met);
}
2021-05-19 19:40:38 +00:00
const ComparisonGraph * graph = nullptr;
2021-05-19 19:40:38 +00:00
{
std::lock_guard lock(cache_mutex);
if (const auto it = answer_cache.find(values); it != std::end(answer_cache))
return it->second;
graph = getGraph(values);
}
2021-05-02 19:16:40 +00:00
bool always_false = false;
expression_cnf->iterateGroups(
[&](const CNFQuery::OrGroup & or_group)
{
if (always_false)
return;
2021-05-06 08:29:24 +00:00
for (const auto & atomic_formula : or_group)
2021-05-02 19:16:40 +00:00
{
2021-05-06 08:29:24 +00:00
CNFQuery::AtomicFormula atom{atomic_formula.negative, atomic_formula.ast->clone()};
2021-05-03 19:08:26 +00:00
pushNotIn(atom);
const auto * func = atom.ast->as<ASTFunction>();
if (func && func->arguments->children.size() == 2)
{
2021-11-10 17:57:59 +00:00
const auto expected = ComparisonGraph::atomToCompareResult(atom);
if (graph->isPossibleCompare(expected, func->arguments->children[0], func->arguments->children[1]))
2021-05-03 19:08:26 +00:00
{
/// If graph failed use matching.
/// We don't need to check constraints.
return;
}
}
}
2021-05-02 19:16:40 +00:00
always_false = true;
});
2021-11-10 17:57:59 +00:00
std::lock_guard lock(cache_mutex);
2021-11-21 19:14:20 +00:00
answer_cache[values] = !always_false;
2021-05-02 19:16:40 +00:00
return !always_false;
}
2021-11-21 19:14:20 +00:00
std::unique_ptr<ComparisonGraph> MergeTreeIndexhypothesisMergedCondition::buildGraph(const std::vector<bool> & values) const
2021-05-02 19:16:40 +00:00
{
ASTs active_atomic_formulas(atomic_constraints);
2021-05-02 19:16:40 +00:00
for (size_t i = 0; i < values.size(); ++i)
{
if (values[i])
active_atomic_formulas.insert(
std::end(active_atomic_formulas),
std::begin(index_to_compare_atomic_hypotheses[i]),
std::end(index_to_compare_atomic_hypotheses[i]));
}
return std::make_unique<ComparisonGraph>(active_atomic_formulas);
}
2021-11-21 19:14:20 +00:00
const ComparisonGraph * MergeTreeIndexhypothesisMergedCondition::getGraph(const std::vector<bool> & values) const
2021-05-02 19:16:40 +00:00
{
auto [it, inserted] = graph_cache.try_emplace(values);
if (inserted)
it->second = buildGraph(values);
return it->second.get();
2021-05-02 19:16:40 +00:00
}
}