mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-02 12:32:04 +00:00
impr
This commit is contained in:
parent
7b2d8e1094
commit
1fb947b70b
@ -63,7 +63,7 @@ add_subdirectory (Server)
|
||||
|
||||
|
||||
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/HashTable)
|
||||
|
38
src/Interpreters/ComparisonGraph.cpp
Normal file
38
src/Interpreters/ComparisonGraph.cpp
Normal 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 {};
|
||||
}
|
||||
|
||||
}
|
76
src/Interpreters/ComparisonGraph.h
Normal file
76
src/Interpreters/ComparisonGraph.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
@ -149,9 +149,13 @@ void traverseCNF(const ASTPtr & node, CNFQuery::AndGroup & and_group, CNFQuery::
|
||||
traverseCNF(child, and_group, or_group);
|
||||
}
|
||||
}
|
||||
else if (func && func->name == "not")
|
||||
{
|
||||
or_group.insert(CNFQuery::AtomicFormula{true, func->arguments->children.front()});
|
||||
}
|
||||
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)
|
||||
{
|
||||
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)
|
||||
{
|
||||
or_groups.push_back(makeASTFunction("or"));
|
||||
auto * func = or_groups.back()->as<ASTFunction>();
|
||||
for (const auto & ast : group)
|
||||
func->arguments->children.push_back(ast->clone());
|
||||
for (const auto & atom : group)
|
||||
{
|
||||
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;
|
||||
}
|
||||
|
||||
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 = {
|
||||
{"notEquals", "equals"},
|
||||
@ -222,22 +252,14 @@ void pullNotOut(ASTPtr & node)
|
||||
{"notEmpty", "empty"},
|
||||
};
|
||||
|
||||
auto * func = node->as<ASTFunction>();
|
||||
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);
|
||||
}
|
||||
pushPullNotInAtom(atom, inverse_relations);
|
||||
}
|
||||
|
||||
void pushNotIn(ASTPtr & node)
|
||||
void pushNotIn(CNFQuery::AtomicFormula & atom)
|
||||
{
|
||||
if (!atom.negative)
|
||||
return;
|
||||
|
||||
static const std::map<std::string, std::string> inverse_relations = {
|
||||
{"equals", "notEquals"},
|
||||
{"less", "greaterOrEquals"},
|
||||
@ -245,51 +267,34 @@ void pushNotIn(ASTPtr & node)
|
||||
{"in", "notIn"},
|
||||
{"like", "notLike"},
|
||||
{"empty", "notEmpty"},
|
||||
{"notEquals", "equals"},
|
||||
{"greaterOrEquals", "less"},
|
||||
{"greater", "lessOrEquals"},
|
||||
{"notIn", "in"},
|
||||
{"notLike", "like"},
|
||||
{"notEmpty", "empty"},
|
||||
};
|
||||
|
||||
auto * func = node->as<ASTFunction>();
|
||||
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);
|
||||
}
|
||||
pushPullNotInAtom(atom, inverse_relations);
|
||||
}
|
||||
|
||||
CNFQuery & CNFQuery::pullNotOutFunctions()
|
||||
{
|
||||
transformAtoms([](const ASTPtr & node) -> ASTPtr
|
||||
{
|
||||
auto * func = node->as<ASTFunction>();
|
||||
if (!func)
|
||||
return node;
|
||||
ASTPtr result = node->clone();
|
||||
if (func->name == "not")
|
||||
pullNotOut(func->arguments->children.front());
|
||||
else
|
||||
pullNotOut(result);
|
||||
traversePushNot(result, false);
|
||||
return result;
|
||||
});
|
||||
transformAtoms([](const AtomicFormula & atom) -> AtomicFormula
|
||||
{
|
||||
AtomicFormula result{atom.negative, atom.ast->clone()};
|
||||
pullNotOut(result);
|
||||
return result;
|
||||
});
|
||||
return *this;
|
||||
}
|
||||
|
||||
CNFQuery & CNFQuery::pushNotInFuntions()
|
||||
{
|
||||
transformAtoms([](const ASTPtr & node) -> ASTPtr
|
||||
transformAtoms([](const AtomicFormula & atom) -> AtomicFormula
|
||||
{
|
||||
auto * func = node->as<ASTFunction>();
|
||||
if (!func)
|
||||
return node;
|
||||
ASTPtr result = node->clone();
|
||||
if (func->name == "not")
|
||||
pushNotIn(func->arguments->children.front());
|
||||
traversePushNot(result, false);
|
||||
AtomicFormula result{atom.negative, atom.ast->clone()};
|
||||
pushNotIn(result);
|
||||
return result;
|
||||
});
|
||||
return *this;
|
||||
@ -306,12 +311,14 @@ std::string CNFQuery::dump() const
|
||||
first = false;
|
||||
res << "(";
|
||||
bool first_in_group = true;
|
||||
for (const auto & ast : group)
|
||||
for (const auto & atom : group)
|
||||
{
|
||||
if (!first_in_group)
|
||||
res << " OR ";
|
||||
first_in_group = false;
|
||||
res << ast->getColumnName();
|
||||
if (atom.negative)
|
||||
res << " NOT ";
|
||||
res << atom.ast->getColumnName();
|
||||
}
|
||||
res << ")";
|
||||
}
|
||||
|
@ -12,7 +12,19 @@ namespace DB
|
||||
class CNFQuery
|
||||
{
|
||||
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>;
|
||||
|
||||
CNFQuery(AndGroup && statements_) : statements(std::move(statements_)) { }
|
||||
@ -46,10 +58,10 @@ public:
|
||||
filtered.insert(filtered_group);
|
||||
else
|
||||
{
|
||||
/// all atoms false -> group false -> CNF false
|
||||
/// all atoms false -> group false -> CNF false
|
||||
filtered.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);
|
||||
std::swap(statements, filtered);
|
||||
return *this;
|
||||
@ -79,11 +91,11 @@ public:
|
||||
transformGroups([func](const OrGroup & group) -> OrGroup
|
||||
{
|
||||
OrGroup result;
|
||||
for (const auto & ast : group)
|
||||
for (const auto & atom : group)
|
||||
{
|
||||
auto new_ast = func(ast);
|
||||
if (new_ast)
|
||||
result.insert(std::move(new_ast));
|
||||
auto new_atom = func(atom);
|
||||
if (new_atom.ast)
|
||||
result.insert(std::move(new_atom));
|
||||
}
|
||||
return result;
|
||||
});
|
||||
|
@ -9,9 +9,9 @@
|
||||
|
||||
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 :
|
||||
metadata_snapshot->getConstraints().filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE))
|
||||
{
|
||||
@ -24,6 +24,39 @@ std::vector<std::vector<ASTPtr>> getConstraintData(const StorageMetadataPtr & me
|
||||
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(
|
||||
ASTSelectQuery * select_query_,
|
||||
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;
|
||||
|
||||
{
|
||||
auto * func_a = a->as<ASTFunction>();
|
||||
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())
|
||||
if (a.ast->getTreeHash() == b.ast->getTreeHash() &&
|
||||
a.ast->getColumnName() == b.ast->getColumnName())
|
||||
{
|
||||
return match_means_ok ? MatchState::FULL_MATCH : MatchState::NOT_MATCH;
|
||||
}
|
||||
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: go deeper into asts (a < b, a = b,...) with z3 or some visitor
|
||||
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;
|
||||
}
|
||||
|
||||
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
|
||||
|
||||
@ -120,9 +138,9 @@ bool checkIfAtomAlwaysFalse(const ASTPtr & atom, const std::vector<std::vector<A
|
||||
if (constraint.size() > 1)
|
||||
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)
|
||||
return match_result == MatchState::NOT_MATCH;
|
||||
@ -137,14 +155,15 @@ void WhereConstraintsOptimizer::perform()
|
||||
if (select_query->where() && 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());
|
||||
Poco::Logger::get("BEFORE OPT").information(cnf.dump());
|
||||
cnf.pullNotOutFunctions()
|
||||
.filterAlwaysTrueGroups([&constraint_data](const auto & group) { /// remove always true groups from CNF
|
||||
return !checkIfGroupAlwaysTrue(group, constraint_data);
|
||||
})
|
||||
.filterAlwaysFalseAtoms([&constraint_data](const auto & ast) { /// remove always false atoms from CNF
|
||||
return !checkIfAtomAlwaysFalse(ast, constraint_data);
|
||||
.filterAlwaysFalseAtoms([&constraint_data](const auto & atom) { /// remove always false atoms from CNF
|
||||
return !checkIfAtomAlwaysFalse(atom, constraint_data);
|
||||
})
|
||||
.pushNotInFuntions();
|
||||
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Parsers/ASTConstraintDeclaration.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/TreeCNFConverter.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -11,6 +12,7 @@ using ConstraintsExpressions = std::vector<ExpressionActionsPtr>;
|
||||
struct ConstraintsDescription
|
||||
{
|
||||
std::vector<ASTPtr> constraints;
|
||||
std::vector<CNFQuery> cnf_constraints;
|
||||
|
||||
ConstraintsDescription() = default;
|
||||
|
||||
@ -27,6 +29,9 @@ struct ConstraintsDescription
|
||||
};
|
||||
|
||||
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;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user