ClickHouse/src/Interpreters/WhereConstraintsOptimizer.cpp

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

183 lines
6.0 KiB
C++
Raw Normal View History

2021-01-04 20:55:32 +00:00
#include <Interpreters/WhereConstraintsOptimizer.h>
2021-01-05 20:51:19 +00:00
#include <Interpreters/TreeCNFConverter.h>
2021-03-05 09:54:13 +00:00
#include <Interpreters/ComparisonGraph.h>
#include <Parsers/IAST_fwd.h>
2021-01-06 10:41:06 +00:00
#include <Parsers/ASTFunction.h>
2021-01-04 20:55:32 +00:00
#include <Storages/StorageInMemoryMetadata.h>
2021-04-28 17:35:51 +00:00
#include <Interpreters/AddIndexConstraintsOptimizer.h>
2021-01-04 20:55:32 +00:00
#include <Parsers/ASTSelectQuery.h>
2021-01-06 20:04:49 +00:00
#include <Poco/Logger.h>
2021-01-04 20:55:32 +00:00
2021-11-10 17:57:59 +00:00
#include <Parsers/queryToString.h>
2021-01-04 20:55:32 +00:00
namespace DB
{
2021-03-05 09:54:13 +00:00
2021-01-04 20:55:32 +00:00
WhereConstraintsOptimizer::WhereConstraintsOptimizer(
ASTSelectQuery * select_query_,
2021-05-04 19:18:37 +00:00
const StorageMetadataPtr & metadata_snapshot_,
2021-11-10 17:57:59 +00:00
bool optimize_append_index_)
2021-01-04 20:55:32 +00:00
: select_query(select_query_)
, metadata_snapshot(metadata_snapshot_)
2021-05-04 19:18:37 +00:00
, optimize_append_index(optimize_append_index_)
2021-01-04 20:55:32 +00:00
{
}
2021-01-06 10:41:06 +00:00
namespace
{
2024-05-08 23:08:33 +00:00
enum class MatchState : uint8_t
2021-01-06 10:41:06 +00:00
{
2021-11-10 17:57:59 +00:00
FULL_MATCH, /// a = b
NOT_MATCH, /// a = not b
NONE, /// other
};
2021-01-06 10:41:06 +00:00
2021-11-10 17:57:59 +00:00
MatchState match(CNFQuery::AtomicFormula a, CNFQuery::AtomicFormula b)
{
bool match_means_ok = (a.negative == b.negative);
if (a.ast->getTreeHash(/*ignore_aliases=*/ true) == b.ast->getTreeHash(/*ignore_aliases=*/ true))
2021-01-06 10:41:06 +00:00
return match_means_ok ? MatchState::FULL_MATCH : MatchState::NOT_MATCH;
2021-11-10 17:57:59 +00:00
2021-01-06 10:41:06 +00:00
return MatchState::NONE;
}
2021-05-05 11:17:49 +00:00
bool checkIfGroupAlwaysTrueFullMatch(const CNFQuery::OrGroup & group, const ConstraintsDescription & constraints_description)
2021-01-05 20:51:19 +00:00
{
2021-11-10 17:57:59 +00:00
/// We have constraints in CNF.
/// CNF is always true => Each OR group in CNF is always true.
/// So, we try to check whether we have al least one OR group from CNF as subset in our group.
/// If we've found one then our group is always true too.
2021-05-08 09:19:18 +00:00
const auto & constraints_data = constraints_description.getConstraintData();
2021-11-10 17:57:59 +00:00
std::vector<size_t> found(constraints_data.size());
2021-05-08 09:19:18 +00:00
for (size_t i = 0; i < constraints_data.size(); ++i)
found[i] = constraints_data[i].size();
2021-05-05 11:17:49 +00:00
for (const auto & atom : group)
2021-01-05 20:51:19 +00:00
{
2021-05-05 11:17:49 +00:00
const auto constraint_atom_ids = constraints_description.getAtomIds(atom.ast);
if (constraint_atom_ids)
2021-01-05 20:51:19 +00:00
{
2021-11-10 17:57:59 +00:00
const auto constraint_atoms = constraints_description.getAtomsById(*constraint_atom_ids);
2021-05-08 09:19:18 +00:00
for (size_t i = 0; i < constraint_atoms.size(); ++i)
2021-01-05 20:51:19 +00:00
{
2021-05-08 09:19:18 +00:00
if (match(constraint_atoms[i], atom) == MatchState::FULL_MATCH)
{
2021-11-10 17:57:59 +00:00
if ((--found[(*constraint_atom_ids)[i].group_id]) == 0)
2021-05-08 09:19:18 +00:00
return true;
}
2021-01-05 20:51:19 +00:00
}
}
}
return false;
}
2023-03-17 13:38:01 +00:00
bool checkIfGroupAlwaysTrueGraph(const CNFQuery::OrGroup & group, const ComparisonGraph<ASTPtr> & graph)
2021-03-05 09:54:13 +00:00
{
2021-11-10 17:57:59 +00:00
/// We try to find at least one atom that is always true by using comparison graph.
2021-03-05 09:54:13 +00:00
for (const auto & atom : group)
{
const auto * func = atom.ast->as<ASTFunction>();
if (func && func->arguments->children.size() == 2)
{
2023-03-17 13:38:01 +00:00
const auto expected = ComparisonGraph<ASTPtr>::atomToCompareResult(atom);
2021-11-10 17:57:59 +00:00
if (graph.isAlwaysCompare(expected, func->arguments->children[0], func->arguments->children[1]))
return true;
2021-03-05 09:54:13 +00:00
}
}
2021-11-10 17:57:59 +00:00
2021-03-05 09:54:13 +00:00
return false;
}
2021-05-05 11:17:49 +00:00
bool checkIfAtomAlwaysFalseFullMatch(const CNFQuery::AtomicFormula & atom, const ConstraintsDescription & constraints_description)
2021-01-05 20:51:19 +00:00
{
2021-05-05 11:17:49 +00:00
const auto constraint_atom_ids = constraints_description.getAtomIds(atom.ast);
if (constraint_atom_ids)
2021-01-05 20:51:19 +00:00
{
2021-11-10 17:57:59 +00:00
for (const auto & constraint_atom : constraints_description.getAtomsById(*constraint_atom_ids))
2021-01-05 20:51:19 +00:00
{
2021-05-05 11:17:49 +00:00
const auto match_result = match(constraint_atom, atom);
if (match_result == MatchState::NOT_MATCH)
return true;
2021-01-05 20:51:19 +00:00
}
}
return false;
}
2023-03-17 13:38:01 +00:00
bool checkIfAtomAlwaysFalseGraph(const CNFQuery::AtomicFormula & atom, const ComparisonGraph<ASTPtr> & graph)
2021-03-05 12:13:00 +00:00
{
const auto * func = atom.ast->as<ASTFunction>();
if (func && func->arguments->children.size() == 2)
{
/// TODO: special support for !=
2023-03-17 13:38:01 +00:00
const auto expected = ComparisonGraph<ASTPtr>::atomToCompareResult(atom);
2021-05-02 19:16:40 +00:00
return !graph.isPossibleCompare(expected, func->arguments->children[0], func->arguments->children[1]);
2021-03-05 12:13:00 +00:00
}
return false;
}
2023-03-17 13:38:01 +00:00
void replaceToConstants(ASTPtr & term, const ComparisonGraph<ASTPtr> & graph)
2021-03-05 12:46:42 +00:00
{
const auto equal_constant = graph.getEqualConst(term);
if (equal_constant)
{
term = (*equal_constant)->clone();
}
else
{
for (auto & child : term->children)
replaceToConstants(child, graph);
}
}
2023-03-17 13:38:01 +00:00
CNFQuery::AtomicFormula replaceTermsToConstants(const CNFQuery::AtomicFormula & atom, const ComparisonGraph<ASTPtr> & graph)
2021-03-05 12:46:42 +00:00
{
CNFQuery::AtomicFormula result;
result.negative = atom.negative;
result.ast = atom.ast->clone();
replaceToConstants(result.ast, graph);
return result;
}
2021-03-05 12:13:00 +00:00
2021-11-10 17:57:59 +00:00
}
2021-01-04 20:55:32 +00:00
void WhereConstraintsOptimizer::perform()
{
2021-01-06 20:04:49 +00:00
if (select_query->where() && metadata_snapshot)
2021-01-05 20:51:19 +00:00
{
2021-04-28 17:35:51 +00:00
const auto & compare_graph = metadata_snapshot->getConstraints().getGraph();
2021-01-05 20:51:19 +00:00
auto cnf = TreeCNFConverter::toCNF(select_query->where());
cnf.pullNotOutFunctions()
2021-05-05 11:17:49 +00:00
.filterAlwaysTrueGroups([&compare_graph, this](const auto & group)
2021-05-04 18:43:58 +00:00
{
/// remove always true groups from CNF
2021-05-05 11:17:49 +00:00
return !checkIfGroupAlwaysTrueFullMatch(group, metadata_snapshot->getConstraints()) && !checkIfGroupAlwaysTrueGraph(group, compare_graph);
2021-01-06 10:41:06 +00:00
})
2021-05-05 11:17:49 +00:00
.filterAlwaysFalseAtoms([&compare_graph, this](const auto & atom)
2021-05-04 18:43:58 +00:00
{
/// remove always false atoms from CNF
2021-05-05 11:17:49 +00:00
return !checkIfAtomAlwaysFalseFullMatch(atom, metadata_snapshot->getConstraints()) && !checkIfAtomAlwaysFalseGraph(atom, compare_graph);
2021-01-06 10:41:06 +00:00
})
2021-05-04 18:43:58 +00:00
.transformAtoms([&compare_graph](const auto & atom)
{
2021-03-05 12:46:42 +00:00
return replaceTermsToConstants(atom, compare_graph);
})
2021-05-05 08:51:25 +00:00
.reduce()
2022-09-05 01:50:24 +00:00
.pushNotInFunctions();
2021-05-04 19:18:37 +00:00
if (optimize_append_index)
AddIndexConstraintsOptimizer(metadata_snapshot).perform(cnf);
2021-01-05 20:51:19 +00:00
select_query->setExpression(ASTSelectQuery::Expression::WHERE, TreeCNFConverter::fromCNF(cnf));
}
2021-01-04 20:55:32 +00:00
}
}