ClickHouse/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp

215 lines
7.4 KiB
C++
Raw Normal View History

2021-05-02 19:16:40 +00:00
#include <Storages/MergeTree/MergeTreeIndexMergedCondition.h>
#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>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
MergeTreeIndexMergedCondition::MergeTreeIndexMergedCondition(
const SelectQueryInfo & query_,
ContextPtr /*context_*/,
const size_t granularity_)
: granularity(granularity_)
{
const auto & select = query_.query->as<ASTSelectQuery &>();
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>(TreeCNFConverter::toCNF(expression_ast));
}
void MergeTreeIndexMergedCondition::addIndex(const MergeTreeIndexPtr & index)
{
if (!index->isMergeable() || index->getGranularity() != granularity)
throw Exception("Index can not be merged",
ErrorCodes::LOGICAL_ERROR);
const auto hypothesis_index = std::dynamic_pointer_cast<const MergeTreeIndexHypothesis>(index);
if (!hypothesis_index)
throw Exception(
"Only hypothesis index is supported here.", ErrorCodes::LOGICAL_ERROR);
static const std::set<std::string> relations = {
2021-05-04 10:47:23 +00:00
"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-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-02 19:16:40 +00:00
CNFQuery::AtomicFormula atom = *group.begin();
pushNotIn(atom);
if (atom.negative)
throw Exception("negative atom", ErrorCodes::LOGICAL_ERROR);
auto * func = atom.ast->as<ASTFunction>();
if (func && relations.count(func->name))
compare_hypotheses_data.push_back(atom.ast);
}
}
index_to_compare_atomic_hypotheses.push_back(compare_hypotheses_data);
index_to_atomic_hypotheses.push_back(hypotheses_data);
}
void MergeTreeIndexMergedCondition::addConstraints(const ConstraintsDescription & constraints_description)
{
auto atomic_constraints_data = constraints_description.getAtomicConstraintData();
for (auto & atom : atomic_constraints_data)
{
pushNotIn(atom);
atomic_constraints.push_back(atom.ast);
}
}
namespace
{
ComparisonGraph::CompareResult getExpectedCompare(const CNFQuery::AtomicFormula & atom)
{
2021-05-04 10:47:23 +00:00
const auto * func = atom.ast->as<ASTFunction>();
if (func)
{
auto expected = ComparisonGraph::getCompareResult(func->name);
if (atom.negative)
expected = ComparisonGraph::inverseCompareResult(expected);
return expected;
}
2021-05-02 19:16:40 +00:00
return ComparisonGraph::CompareResult::UNKNOWN;
}
}
2021-05-03 19:08:26 +00:00
/// Replaces < -> <=, > -> >= and assumes that all hypotheses are true then checks if path exists
2021-05-04 18:43:58 +00:00
bool MergeTreeIndexMergedCondition::alwaysUnknownOrTrue() const
{
2021-05-03 19:08:26 +00:00
std::vector<ASTPtr> active_atomic_formulas(atomic_constraints);
for (size_t i = 0; i < index_to_compare_atomic_hypotheses.size(); ++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]));
}
/// 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)
{
for (auto atom : or_group)
{
pushNotIn(atom);
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-05-02 19:16:40 +00:00
bool MergeTreeIndexMergedCondition::mayBeTrueOnGranule(const MergeTreeIndexGranules & granules) const
{
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("Only hypothesis index is supported here.", ErrorCodes::LOGICAL_ERROR);
values.push_back(granule->met);
}
const auto & graph = getGraph(values);
bool always_false = false;
expression_cnf->iterateGroups(
[&](const CNFQuery::OrGroup & or_group)
{
if (always_false)
return;
for (auto atom : or_group)
{
2021-05-03 19:08:26 +00:00
pushNotIn(atom);
Poco::Logger::get("KEK").information(atom.ast->dumpTree());
const auto * func = atom.ast->as<ASTFunction>();
if (func && func->arguments->children.size() == 2)
{
const auto expected = getExpectedCompare(atom);
if (graph.isPossibleCompare(expected, func->arguments->children[0], func->arguments->children[1]))
{
/// If graph failed use matching.
/// We don't need to check constraints.
return;
}
}
}
2021-05-02 19:16:40 +00:00
always_false = true;
});
return !always_false;
}
std::unique_ptr<ComparisonGraph> MergeTreeIndexMergedCondition::buildGraph(const std::vector<bool> & values) const
{
Poco::Logger::get("MergeTreeIndexMergedCondition").information("New graph");
std::vector<ASTPtr> active_atomic_formulas(atomic_constraints);
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);
}
const ComparisonGraph & MergeTreeIndexMergedCondition::getGraph(const std::vector<bool> & values) const
{
if (!graphCache.contains(values))
graphCache[values] = buildGraph(values);
return *graphCache.at(values);
}
}