mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 21:24:28 +00:00
fix
This commit is contained in:
parent
02090bf59a
commit
08206ab20b
@ -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
|
||||||
|
@ -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());
|
||||||
|
@ -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
|
||||||
{
|
{
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user