This commit is contained in:
Nikita Vasilev 2021-04-03 15:12:45 +03:00
parent 02090bf59a
commit 08206ab20b
4 changed files with 65 additions and 68 deletions

View File

@ -34,11 +34,11 @@ public:
std::vector<ASTPtr> getEqual(const ASTPtr & ast) const; std::vector<ASTPtr> getEqual(const ASTPtr & ast) const;
std::optional<ASTPtr> getEqualConst(const ASTPtr & ast) const; std::optional<ASTPtr> getEqualConst(const ASTPtr & ast) const;
/// Find constants less and greater. /// Find constants lessOrEqual and greaterOrEqual.
/// For int and double linear programming can be applied here. /// For int and double linear programming can be applied here.
// TODO: implement // TODO: implement
//ASTPtr getMax(const ASTPtr &) const { return nullptr; } // sup //ASTPtr getUpperBound(const ASTPtr &) const { return nullptr; } // sup
//ASTPtr getMin(const ASTPtr &) const { return nullptr; } // inf //ASTPtr getLowerBound(const ASTPtr &) const { return nullptr; } // inf
private: private:
/// strongly connected component /// strongly connected component

View File

@ -17,62 +17,6 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
} }
std::vector<std::vector<CNFQuery::AtomicFormula>> getConstraintData(const StorageMetadataPtr & metadata_snapshot)
{
std::vector<std::vector<CNFQuery::AtomicFormula>> constraint_data;
for (const auto & constraint :
metadata_snapshot->getConstraints().filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE))
{
const auto cnf = TreeCNFConverter::toCNF(constraint->as<ASTConstraintDeclaration>()->expr->ptr())
.pullNotOutFunctions(); /// TODO: move prepare stage to ConstraintsDescription
for (const auto & group : cnf.getStatements())
constraint_data.emplace_back(std::begin(group), std::end(group));
}
return constraint_data;
}
std::vector<CNFQuery::AtomicFormula> getAtomicConstraintData(const StorageMetadataPtr & metadata_snapshot)
{
std::vector<CNFQuery::AtomicFormula> constraint_data;
for (const auto & constraint :
metadata_snapshot->getConstraints().filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE))
{
const auto cnf = TreeCNFConverter::toCNF(constraint->as<ASTConstraintDeclaration>()->expr->ptr())
.pullNotOutFunctions();
for (const auto & group : cnf.getStatements()) {
if (group.size() == 1)
constraint_data.push_back(*group.begin());
}
}
return constraint_data;
}
ComparisonGraph getComparisonGraph(const StorageMetadataPtr & metadata_snapshot)
{
static const std::set<std::string> relations = {
"equals", "less", "lessOrEquals", "greaterOrEquals", "greater"};
std::vector<ASTPtr> constraints_for_graph;
auto atomic_formulas = getAtomicConstraintData(metadata_snapshot);
const std::vector<CNFQuery::AtomicFormula> atomic_constraints = getAtomicConstraintData(metadata_snapshot);
for (auto & atomic_formula : atomic_formulas)
{
pushNotIn(atomic_formula);
auto * func = atomic_formula.ast->as<ASTFunction>();
if (func && relations.count(func->name))
{
if (atomic_formula.negative)
throw Exception(": ", ErrorCodes::LOGICAL_ERROR);
constraints_for_graph.push_back(atomic_formula.ast);
}
}
return ComparisonGraph(constraints_for_graph);
}
WhereConstraintsOptimizer::WhereConstraintsOptimizer( WhereConstraintsOptimizer::WhereConstraintsOptimizer(
ASTSelectQuery * select_query_, ASTSelectQuery * select_query_,
Aliases & /*aliases_*/, Aliases & /*aliases_*/,
@ -111,10 +55,7 @@ MatchState match(CNFQuery::AtomicFormula a, CNFQuery::AtomicFormula b)
bool checkIfGroupAlwaysTrueFullMatch(const CNFQuery::OrGroup & group, const std::vector<std::vector<CNFQuery::AtomicFormula>> & constraints) bool checkIfGroupAlwaysTrueFullMatch(const CNFQuery::OrGroup & group, const std::vector<std::vector<CNFQuery::AtomicFormula>> & constraints)
{ {
/// TODO: this is temporary; need to write more effective search for (const auto & constraint : constraints)
/// TODO: go deeper into asts (a < b, a = b,...) with z3 or some visitor
for (const auto & constraint : constraints) /// one constraint in group is enough,
/// otherwise it's difficult to make judgements without using constraint solving (z3..)
{ {
bool group_always_true = true; bool group_always_true = true;
for (const auto & constraint_ast : constraint) for (const auto & constraint_ast : constraint)
@ -211,8 +152,6 @@ bool checkIfGroupAlwaysTrueGraph(const CNFQuery::OrGroup & group, const Comparis
bool checkIfAtomAlwaysFalseFullMatch(const CNFQuery::AtomicFormula & atom, const std::vector<std::vector<CNFQuery::AtomicFormula>> & constraints) bool checkIfAtomAlwaysFalseFullMatch(const CNFQuery::AtomicFormula & atom, const std::vector<std::vector<CNFQuery::AtomicFormula>> & constraints)
{ {
/// TODO: more efficient matching
for (const auto & constraint : constraints) for (const auto & constraint : constraints)
{ {
if (constraint.size() > 1) if (constraint.size() > 1)
@ -290,8 +229,8 @@ void WhereConstraintsOptimizer::perform()
{ {
if (select_query->where() && metadata_snapshot) if (select_query->where() && metadata_snapshot)
{ {
const auto constraint_data = getConstraintData(metadata_snapshot); const auto constraint_data = metadata_snapshot->getConstraints().getConstraintData();
const auto compare_graph = getComparisonGraph(metadata_snapshot); const auto compare_graph = metadata_snapshot->getConstraints().getGraph();
Poco::Logger::get("BEFORE CNF ").information(select_query->where()->dumpTree()); Poco::Logger::get("BEFORE CNF ").information(select_query->where()->dumpTree());
auto cnf = TreeCNFConverter::toCNF(select_query->where()); auto cnf = TreeCNFConverter::toCNF(select_query->where());
Poco::Logger::get("BEFORE OPT").information(cnf.dump()); Poco::Logger::get("BEFORE OPT").information(cnf.dump());

View File

@ -7,6 +7,7 @@
#include <Parsers/ParserCreateQuery.h> #include <Parsers/ParserCreateQuery.h>
#include <Parsers/parseQuery.h> #include <Parsers/parseQuery.h>
#include <Parsers/ASTExpressionList.h> #include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Core/Defines.h> #include <Core/Defines.h>
@ -65,6 +66,58 @@ ASTs ConstraintsDescription::filterConstraints(ConstraintType selection) const
return res; return res;
} }
std::vector<std::vector<CNFQuery::AtomicFormula>> ConstraintsDescription::getConstraintData() const
{
std::vector<std::vector<CNFQuery::AtomicFormula>> constraint_data;
for (const auto & constraint : filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE))
{
const auto cnf = TreeCNFConverter::toCNF(constraint->as<ASTConstraintDeclaration>()->expr->ptr())
.pullNotOutFunctions(); /// TODO: move prepare stage to ConstraintsDescription
for (const auto & group : cnf.getStatements())
constraint_data.emplace_back(std::begin(group), std::end(group));
}
return constraint_data;
}
std::vector<CNFQuery::AtomicFormula> ConstraintsDescription::getAtomicConstraintData() const
{
std::vector<CNFQuery::AtomicFormula> constraint_data;
for (const auto & constraint : filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE))
{
const auto cnf = TreeCNFConverter::toCNF(constraint->as<ASTConstraintDeclaration>()->expr->ptr())
.pullNotOutFunctions();
for (const auto & group : cnf.getStatements()) {
if (group.size() == 1)
constraint_data.push_back(*group.begin());
}
}
return constraint_data;
}
ComparisonGraph ConstraintsDescription::getGraph() const
{
static const std::set<std::string> relations = {
"equals", "less", "lessOrEquals", "greaterOrEquals", "greater"};
std::vector<ASTPtr> constraints_for_graph;
auto atomic_formulas = getAtomicConstraintData();
for (auto & atomic_formula : atomic_formulas)
{
pushNotIn(atomic_formula);
auto * func = atomic_formula.ast->as<ASTFunction>();
if (func && relations.count(func->name))
{
if (atomic_formula.negative)
throw Exception(": ", ErrorCodes::LOGICAL_ERROR);
constraints_for_graph.push_back(atomic_formula.ast);
}
}
return ComparisonGraph(constraints_for_graph);
}
ConstraintsExpressions ConstraintsDescription::getExpressionsToCheck(const DB::Context & context, ConstraintsExpressions ConstraintsDescription::getExpressionsToCheck(const DB::Context & context,
const DB::NamesAndTypesList & source_columns_) const const DB::NamesAndTypesList & source_columns_) const
{ {

View File

@ -3,6 +3,7 @@
#include <Parsers/ASTConstraintDeclaration.h> #include <Parsers/ASTConstraintDeclaration.h>
#include <Interpreters/ExpressionActions.h> #include <Interpreters/ExpressionActions.h>
#include <Interpreters/TreeCNFConverter.h> #include <Interpreters/TreeCNFConverter.h>
#include <Interpreters/ComparisonGraph.h>
namespace DB namespace DB
{ {
@ -31,7 +32,11 @@ struct ConstraintsDescription
ASTs filterConstraints(ConstraintType selection) const; ASTs filterConstraints(ConstraintType selection) const;
// TODO: перенести преобразование в КНФ + get constraitns // TODO: перенести преобразование в КНФ + get constraitns
//ASTs filterAtomicConstraints(ConstraintType selection) const; //ASTs filterAtomicConstraints(ConstraintType selection) const;
//ASTs filterEqualConstraints(ConstraintType selection) const;
std::vector<std::vector<CNFQuery::AtomicFormula>> getConstraintData() const;
std::vector<CNFQuery::AtomicFormula> getAtomicConstraintData() const;
ComparisonGraph getGraph() const;
ConstraintsExpressions getExpressionsToCheck(const Context & context, const NamesAndTypesList & source_columns_) const; ConstraintsExpressions getExpressionsToCheck(const Context & context, const NamesAndTypesList & source_columns_) const;