This commit is contained in:
Nikita Vasilev 2021-03-04 15:11:43 +03:00
parent 7b2d8e1094
commit 1fb947b70b
7 changed files with 247 additions and 90 deletions

View File

@ -63,7 +63,7 @@ add_subdirectory (Server)
set(dbms_headers) set(dbms_headers)
set(dbms_sources Interpreters/ConstraintMatcherVisitor.cpp Interpreters/ConstraintMatcherVisitor.h Interpreters/WhereConstraintsOptimizer.cpp Interpreters/WhereConstraintsOptimizer.h Interpreters/TreeCNFConverter.cpp Interpreters/TreeCNFConverter.h) set(dbms_sources Interpreters/ConstraintMatcherVisitor.cpp Interpreters/ConstraintMatcherVisitor.h Interpreters/WhereConstraintsOptimizer.cpp Interpreters/WhereConstraintsOptimizer.h Interpreters/TreeCNFConverter.cpp Interpreters/TreeCNFConverter.h Interpreters/ComparisonGraph.cpp Interpreters/ComparisonGraph.h)
add_headers_and_sources(clickhouse_common_io Common) add_headers_and_sources(clickhouse_common_io Common)
add_headers_and_sources(clickhouse_common_io Common/HashTable) add_headers_and_sources(clickhouse_common_io Common/HashTable)

View File

@ -0,0 +1,38 @@
#include <Interpreters/ComparisonGraph.h>
#include <algorithm>
namespace DB
{
ComparisonGraph::ComparisonGraph(const std::vector<ASTPtr> & /*atomic_formulas*/)
{
}
std::vector<ASTPtr> ComparisonGraph::getEqual(const ASTPtr & ast) const
{
const auto hash_it = graph.ast_hash_to_component.find(ast->getTreeHash().second);
if (hash_it != std::end(graph.ast_hash_to_component))
return {};
const size_t index = hash_it->second;
//const auto vertex_it = std::find(std::begin(graph.vertexes[index].asts), std::end(graph.vertexes[index].asts), ast, );
if (std::any_of(
std::cbegin(graph.vertexes[index].asts),
std::cend(graph.vertexes[index].asts),
[ast](const ASTPtr & constraint_ast)
{
return constraint_ast->getTreeHash() == ast->getTreeHash() &&
constraint_ast->getColumnName() == ast->getColumnName();
})) {
return graph.vertexes[index].asts;
} else {
return {};
}
}
ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAsts(const Graph & /*asts_graph*/)
{
return {};
}
}

View File

@ -0,0 +1,76 @@
#pragma once
#include <Parsers/IAST_fwd.h>
#include <Parsers/ASTLiteral.h>
#include <unordered_map>
#include <vector>
namespace DB
{
class ComparisonGraph
{
public:
ComparisonGraph(const std::vector<ASTPtr> & atomic_formulas);
/// Works for string and num.
/// For other -- only eq.
enum class CompareResult
{
LESS,
LESS_OR_EQUAL,
EQUAL,
GREATER_OR_EQUAL,
GREATER,
//NOT_EQUAL,
UNKNOWN,
};
// TODO: implement
CompareResult compare(const ASTPtr & /*left*/, const ASTPtr & /*right*/) const { return CompareResult::UNKNOWN; }
std::vector<ASTPtr> getEqual(const ASTPtr & ast) const;
/// Find constants less and greater.
/// For int and double linear programming can be applied here.
// TODO: implement
ASTPtr getMax(const ASTPtr &) const { return nullptr; } // sup
ASTPtr getMin(const ASTPtr &) const { return nullptr; } // inf
private:
/// strongly connected component
struct EqualComponent
{
std::vector<ASTPtr> asts;
};
/// TODO: move to diff for int and double:
/// LESS and LESS_OR_EQUAL with +const or 0 --- ok
/// with -const --- not ok
/// EQUAL is ok only for 0
struct Edge
{
enum Type
{
LESS,
LESS_OR_EQUAL,
EQUAL,
};
Type type;
EqualComponent to;
};
struct Graph
{
std::unordered_map<UInt64, size_t> ast_hash_to_component;
std::vector<EqualComponent> vertexes;
std::vector<std::vector<Edge>> edges;
};
Graph BuildGraphFromAsts(const Graph & asts_graph);
Graph graph;
};
}

View File

@ -149,9 +149,13 @@ void traverseCNF(const ASTPtr & node, CNFQuery::AndGroup & and_group, CNFQuery::
traverseCNF(child, and_group, or_group); traverseCNF(child, and_group, or_group);
} }
} }
else if (func && func->name == "not")
{
or_group.insert(CNFQuery::AtomicFormula{true, func->arguments->children.front()});
}
else else
{ {
or_group.insert(node); or_group.insert(CNFQuery::AtomicFormula{false, node});
} }
} }
@ -190,13 +194,23 @@ ASTPtr TreeCNFConverter::fromCNF(const CNFQuery & cnf)
for (const auto & group : groups) for (const auto & group : groups)
{ {
if (group.size() == 1) if (group.size() == 1)
or_groups.push_back((*group.begin())->clone()); {
if ((*group.begin()).negative)
or_groups.push_back(makeASTFunction("not", (*group.begin()).ast->clone()));
else
or_groups.push_back((*group.begin()).ast->clone());
}
else if (group.size() > 1) else if (group.size() > 1)
{ {
or_groups.push_back(makeASTFunction("or")); or_groups.push_back(makeASTFunction("or"));
auto * func = or_groups.back()->as<ASTFunction>(); auto * func = or_groups.back()->as<ASTFunction>();
for (const auto & ast : group) for (const auto & atom : group)
func->arguments->children.push_back(ast->clone()); {
if ((*group.begin()).negative)
func->arguments->children.push_back(makeASTFunction("not", atom.ast->clone()));
else
func->arguments->children.push_back(atom.ast->clone());
}
} }
} }
@ -211,7 +225,23 @@ ASTPtr TreeCNFConverter::fromCNF(const CNFQuery & cnf)
return res; return res;
} }
void pullNotOut(ASTPtr & node) void pushPullNotInAtom(CNFQuery::AtomicFormula & atom, const std::map<std::string, std::string> & inverse_relations)
{
auto * func = atom.ast->as<ASTFunction>();
if (!func)
return;
if (auto it = inverse_relations.find(func->name); it != std::end(inverse_relations))
{
/// inverse func
atom.ast = atom.ast->clone();
auto * new_func = atom.ast->as<ASTFunction>();
new_func->name = it->second;
/// add not
atom.negative = !atom.negative;
}
}
void pullNotOut(CNFQuery::AtomicFormula & atom)
{ {
static const std::map<std::string, std::string> inverse_relations = { static const std::map<std::string, std::string> inverse_relations = {
{"notEquals", "equals"}, {"notEquals", "equals"},
@ -222,22 +252,14 @@ void pullNotOut(ASTPtr & node)
{"notEmpty", "empty"}, {"notEmpty", "empty"},
}; };
auto * func = node->as<ASTFunction>(); pushPullNotInAtom(atom, inverse_relations);
if (!func)
return;
if (auto it = inverse_relations.find(func->name); it != std::end(inverse_relations))
{
/// inverse func
node = node->clone();
auto * new_func = node->as<ASTFunction>();
new_func->name = it->second;
/// add not
node = makeASTFunction("not", node);
}
} }
void pushNotIn(ASTPtr & node) void pushNotIn(CNFQuery::AtomicFormula & atom)
{ {
if (!atom.negative)
return;
static const std::map<std::string, std::string> inverse_relations = { static const std::map<std::string, std::string> inverse_relations = {
{"equals", "notEquals"}, {"equals", "notEquals"},
{"less", "greaterOrEquals"}, {"less", "greaterOrEquals"},
@ -245,35 +267,23 @@ void pushNotIn(ASTPtr & node)
{"in", "notIn"}, {"in", "notIn"},
{"like", "notLike"}, {"like", "notLike"},
{"empty", "notEmpty"}, {"empty", "notEmpty"},
{"notEquals", "equals"},
{"greaterOrEquals", "less"},
{"greater", "lessOrEquals"},
{"notIn", "in"},
{"notLike", "like"},
{"notEmpty", "empty"},
}; };
auto * func = node->as<ASTFunction>(); pushPullNotInAtom(atom, inverse_relations);
if (!func)
return;
if (auto it = inverse_relations.find(func->name); it != std::end(inverse_relations))
{
/// inverse func
node = node->clone();
auto * new_func = node->as<ASTFunction>();
new_func->name = it->second;
/// add not
node = makeASTFunction("not", node);
}
} }
CNFQuery & CNFQuery::pullNotOutFunctions() CNFQuery & CNFQuery::pullNotOutFunctions()
{ {
transformAtoms([](const ASTPtr & node) -> ASTPtr transformAtoms([](const AtomicFormula & atom) -> AtomicFormula
{ {
auto * func = node->as<ASTFunction>(); AtomicFormula result{atom.negative, atom.ast->clone()};
if (!func)
return node;
ASTPtr result = node->clone();
if (func->name == "not")
pullNotOut(func->arguments->children.front());
else
pullNotOut(result); pullNotOut(result);
traversePushNot(result, false);
return result; return result;
}); });
return *this; return *this;
@ -281,15 +291,10 @@ CNFQuery & CNFQuery::pullNotOutFunctions()
CNFQuery & CNFQuery::pushNotInFuntions() CNFQuery & CNFQuery::pushNotInFuntions()
{ {
transformAtoms([](const ASTPtr & node) -> ASTPtr transformAtoms([](const AtomicFormula & atom) -> AtomicFormula
{ {
auto * func = node->as<ASTFunction>(); AtomicFormula result{atom.negative, atom.ast->clone()};
if (!func) pushNotIn(result);
return node;
ASTPtr result = node->clone();
if (func->name == "not")
pushNotIn(func->arguments->children.front());
traversePushNot(result, false);
return result; return result;
}); });
return *this; return *this;
@ -306,12 +311,14 @@ std::string CNFQuery::dump() const
first = false; first = false;
res << "("; res << "(";
bool first_in_group = true; bool first_in_group = true;
for (const auto & ast : group) for (const auto & atom : group)
{ {
if (!first_in_group) if (!first_in_group)
res << " OR "; res << " OR ";
first_in_group = false; first_in_group = false;
res << ast->getColumnName(); if (atom.negative)
res << " NOT ";
res << atom.ast->getColumnName();
} }
res << ")"; res << ")";
} }

View File

@ -12,7 +12,19 @@ namespace DB
class CNFQuery class CNFQuery
{ {
public: public:
using OrGroup = std::set<ASTPtr>; // Add NOT container??? struct AtomicFormula
{
bool negative = false;
ASTPtr ast;
/// for set
bool operator<(const AtomicFormula & rhs) const
{
return ast == rhs.ast ? negative < rhs.negative : ast < rhs.ast;
}
};
using OrGroup = std::set<AtomicFormula>;
using AndGroup = std::set<OrGroup>; using AndGroup = std::set<OrGroup>;
CNFQuery(AndGroup && statements_) : statements(std::move(statements_)) { } CNFQuery(AndGroup && statements_) : statements(std::move(statements_)) { }
@ -49,7 +61,7 @@ public:
/// all atoms false -> group false -> CNF false /// all atoms false -> group false -> CNF false
filtered.clear(); filtered.clear();
filtered_group.clear(); filtered_group.clear();
filtered_group.insert(std::make_shared<ASTLiteral>(static_cast<UInt8>(0))); filtered_group.insert(AtomicFormula{false, std::make_shared<ASTLiteral>(static_cast<UInt8>(0))});
filtered.insert(filtered_group); filtered.insert(filtered_group);
std::swap(statements, filtered); std::swap(statements, filtered);
return *this; return *this;
@ -79,11 +91,11 @@ public:
transformGroups([func](const OrGroup & group) -> OrGroup transformGroups([func](const OrGroup & group) -> OrGroup
{ {
OrGroup result; OrGroup result;
for (const auto & ast : group) for (const auto & atom : group)
{ {
auto new_ast = func(ast); auto new_atom = func(atom);
if (new_ast) if (new_atom.ast)
result.insert(std::move(new_ast)); result.insert(std::move(new_atom));
} }
return result; return result;
}); });

View File

@ -9,9 +9,9 @@
namespace DB namespace DB
{ {
std::vector<std::vector<ASTPtr>> getConstraintData(const StorageMetadataPtr & metadata_snapshot) std::vector<std::vector<CNFQuery::AtomicFormula>> getConstraintData(const StorageMetadataPtr & metadata_snapshot)
{ {
std::vector<std::vector<ASTPtr>> constraint_data; std::vector<std::vector<CNFQuery::AtomicFormula>> constraint_data;
for (const auto & constraint : for (const auto & constraint :
metadata_snapshot->getConstraints().filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE)) metadata_snapshot->getConstraints().filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE))
{ {
@ -24,6 +24,39 @@ std::vector<std::vector<ASTPtr>> getConstraintData(const StorageMetadataPtr & me
return constraint_data; 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(); /// TODO: move prepare stage to ConstraintsDescription
for (const auto & group : cnf.getStatements()) {
if (group.size() == 1)
constraint_data.push_back(*group.begin());
}
}
return constraint_data;
}
std::vector<std::pair<ASTPtr, ASTPtr>> getEqualConstraintData(const StorageMetadataPtr & metadata_snapshot)
{
std::vector<std::pair<ASTPtr, ASTPtr>> equal_constraints;
const std::vector<CNFQuery::AtomicFormula> atomic_constraints = getAtomicConstraintData(metadata_snapshot);
for (const auto & constraint : atomic_constraints) {
auto * func = constraint.ast->as<ASTFunction>();
if (func && (func->name == "equal" && !constraint.negative))
{
equal_constraints.emplace_back(
func->arguments->children[0],
func->arguments->children[1]);
}
}
return equal_constraints;
}
WhereConstraintsOptimizer::WhereConstraintsOptimizer( WhereConstraintsOptimizer::WhereConstraintsOptimizer(
ASTSelectQuery * select_query_, ASTSelectQuery * select_query_,
Aliases & /*aliases_*/, Aliases & /*aliases_*/,
@ -48,37 +81,22 @@ namespace
}; };
} }
MatchState match(ASTPtr a, ASTPtr b) MatchState match(CNFQuery::AtomicFormula a, CNFQuery::AtomicFormula b)
{ {
bool match_means_ok = true; bool match_means_ok = true ^ a.negative ^ b.negative;
{ if (a.ast->getTreeHash() == b.ast->getTreeHash() &&
auto * func_a = a->as<ASTFunction>(); a.ast->getColumnName() == b.ast->getColumnName())
if (func_a && func_a->name == "not")
{
a = func_a->arguments->children.front();
match_means_ok ^= true;
}
}
{
auto * func_b = b->as<ASTFunction>();
if (func_b && func_b->name == "not")
{
b = func_b->arguments->children.front();
match_means_ok ^= true;
}
}
if (a->getTreeHash() == b->getTreeHash() &&
a->getColumnName() == b->getColumnName())
{ {
return match_means_ok ? MatchState::FULL_MATCH : MatchState::NOT_MATCH; return match_means_ok ? MatchState::FULL_MATCH : MatchState::NOT_MATCH;
} }
return MatchState::NONE; return MatchState::NONE;
} }
bool checkIfGroupAlwaysTrue(const CNFQuery::OrGroup & group, const std::vector<std::vector<ASTPtr>> & constraints) bool checkIfGroupAlwaysTrue(const CNFQuery::OrGroup & group, const std::vector<std::vector<CNFQuery::AtomicFormula>> & constraints)
{ {
/// TODO: constraints graph
/// TODO: this is temporary; need to write more effective search /// TODO: this is temporary; need to write more effective search
/// TODO: go deeper into asts (a < b, a = b,...) with z3 or some visitor /// 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, for (const auto & constraint : constraints) /// one constraint in group is enough,
@ -111,7 +129,7 @@ bool checkIfGroupAlwaysTrue(const CNFQuery::OrGroup & group, const std::vector<s
return false; return false;
} }
bool checkIfAtomAlwaysFalse(const ASTPtr & atom, const std::vector<std::vector<ASTPtr>> & constraints) bool checkIfAtomAlwaysFalse(const CNFQuery::AtomicFormula & atom, const std::vector<std::vector<CNFQuery::AtomicFormula>> & constraints)
{ {
/// TODO: more efficient matching /// TODO: more efficient matching
@ -120,9 +138,9 @@ bool checkIfAtomAlwaysFalse(const ASTPtr & atom, const std::vector<std::vector<A
if (constraint.size() > 1) if (constraint.size() > 1)
continue; /// TMP continue; /// TMP
for (const auto & constraint_ast : constraint) for (const auto & constraint_atoms : constraint)
{ {
const auto match_result = match(constraint_ast, atom); const auto match_result = match(constraint_atoms, atom);
if (match_result != MatchState::NONE) if (match_result != MatchState::NONE)
return match_result == MatchState::NOT_MATCH; return match_result == MatchState::NOT_MATCH;
@ -137,14 +155,15 @@ 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 = getConstraintData(metadata_snapshot);
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());
cnf.pullNotOutFunctions() cnf.pullNotOutFunctions()
.filterAlwaysTrueGroups([&constraint_data](const auto & group) { /// remove always true groups from CNF .filterAlwaysTrueGroups([&constraint_data](const auto & group) { /// remove always true groups from CNF
return !checkIfGroupAlwaysTrue(group, constraint_data); return !checkIfGroupAlwaysTrue(group, constraint_data);
}) })
.filterAlwaysFalseAtoms([&constraint_data](const auto & ast) { /// remove always false atoms from CNF .filterAlwaysFalseAtoms([&constraint_data](const auto & atom) { /// remove always false atoms from CNF
return !checkIfAtomAlwaysFalse(ast, constraint_data); return !checkIfAtomAlwaysFalse(atom, constraint_data);
}) })
.pushNotInFuntions(); .pushNotInFuntions();

View File

@ -2,6 +2,7 @@
#include <Parsers/ASTConstraintDeclaration.h> #include <Parsers/ASTConstraintDeclaration.h>
#include <Interpreters/ExpressionActions.h> #include <Interpreters/ExpressionActions.h>
#include <Interpreters/TreeCNFConverter.h>
namespace DB namespace DB
{ {
@ -11,6 +12,7 @@ using ConstraintsExpressions = std::vector<ExpressionActionsPtr>;
struct ConstraintsDescription struct ConstraintsDescription
{ {
std::vector<ASTPtr> constraints; std::vector<ASTPtr> constraints;
std::vector<CNFQuery> cnf_constraints;
ConstraintsDescription() = default; ConstraintsDescription() = default;
@ -27,6 +29,9 @@ struct ConstraintsDescription
}; };
ASTs filterConstraints(ConstraintType selection) const; ASTs filterConstraints(ConstraintType selection) const;
// TODO: перенести преобразование в КНФ + get constraitns
//ASTs filterAtomicConstraints(ConstraintType selection) const;
//ASTs filterEqualConstraints(ConstraintType selection) const;
ConstraintsExpressions getExpressionsToCheck(const Context & context, const NamesAndTypesList & source_columns_) const; ConstraintsExpressions getExpressionsToCheck(const Context & context, const NamesAndTypesList & source_columns_) const;