mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 13:13:36 +00:00
graph
This commit is contained in:
parent
06a774e179
commit
31caff2113
@ -1,21 +1,148 @@
|
||||
#include <Interpreters/ComparisonGraph.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
#include <Poco/Logger.h>
|
||||
#include <algorithm>
|
||||
#include <deque>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ComparisonGraph::ComparisonGraph(const std::vector<ASTPtr> & /*atomic_formulas*/)
|
||||
/// make function a < b or a <= b
|
||||
ASTPtr ComparisonGraph::normalizeAtom(const ASTPtr & atom) const
|
||||
{
|
||||
static const std::map<std::string, std::string> inverse_relations = {
|
||||
{"greaterOrEquals", "less"},
|
||||
{"greater", "lessOrEquals"},
|
||||
};
|
||||
|
||||
ASTPtr res = atom->clone();
|
||||
{
|
||||
auto * func = res->as<ASTFunction>();
|
||||
if (func)
|
||||
{
|
||||
if (const auto it = inverse_relations.find(func->name); it != std::end(inverse_relations))
|
||||
{
|
||||
res = makeASTFunction(it->second, func->arguments->children[1]->clone(), func->arguments->children[0]->clone());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
ComparisonGraph::ComparisonGraph(const std::vector<ASTPtr> & atomic_formulas)
|
||||
{
|
||||
static const std::map<std::string, Edge::Type> relation_to_enum = {
|
||||
{"equals", Edge::Type::EQUAL},
|
||||
{"less", Edge::Type::LESS},
|
||||
{"lessOrEquals", Edge::Type::LESS_OR_EQUAL},
|
||||
};
|
||||
|
||||
Graph g;
|
||||
for (const auto & atom_raw : atomic_formulas) {
|
||||
const auto atom = normalizeAtom(atom_raw);
|
||||
|
||||
const auto bad_term = std::numeric_limits<std::size_t>::max();
|
||||
auto get_index = [](const ASTPtr & ast, Graph & asts_graph) -> std::size_t {
|
||||
const auto it = asts_graph.ast_hash_to_component.find(ast->getTreeHash());
|
||||
if (it != std::end(asts_graph.ast_hash_to_component))
|
||||
{
|
||||
if (!std::any_of(
|
||||
std::cbegin(asts_graph.vertexes[it->second].asts),
|
||||
std::cend(asts_graph.vertexes[it->second].asts),
|
||||
[ast](const ASTPtr & constraint_ast) {
|
||||
return constraint_ast->getTreeHash() == ast->getTreeHash()
|
||||
&& constraint_ast->getColumnName() == ast->getColumnName();
|
||||
}))
|
||||
{
|
||||
return bad_term;
|
||||
}
|
||||
|
||||
return it->second;
|
||||
}
|
||||
else
|
||||
{
|
||||
asts_graph.ast_hash_to_component[ast->getTreeHash()] = asts_graph.vertexes.size();
|
||||
asts_graph.vertexes.push_back(EqualComponent{{ast}});
|
||||
asts_graph.edges.emplace_back();
|
||||
return asts_graph.vertexes.size() - 1;
|
||||
}
|
||||
};
|
||||
|
||||
const auto * func = atom->as<ASTFunction>();
|
||||
if (func)
|
||||
{
|
||||
if (const auto it = relation_to_enum.find(func->name); it != std::end(relation_to_enum) && func->arguments->children.size() == 2)
|
||||
{
|
||||
const size_t index_left = get_index(func->arguments->children[0], g);
|
||||
const size_t index_right = get_index(func->arguments->children[1], g);
|
||||
|
||||
if (index_left != bad_term && index_right != bad_term)
|
||||
{
|
||||
Poco::Logger::get("Edges").information("GOOD: " + atom->dumpTree());
|
||||
Poco::Logger::get("Edges").information("left=" + std::to_string(index_left) + " right=" + std::to_string(index_right));
|
||||
Poco::Logger::get("Edges").information("sz=" + std::to_string(g.edges.size()));
|
||||
g.edges[index_right].push_back(Edge{it->second, index_left});
|
||||
if (func->name == "equals")
|
||||
{
|
||||
Poco::Logger::get("Edges").information("right=" + std::to_string(index_left) + " left=" + std::to_string(index_right));
|
||||
g.edges[index_left].push_back(Edge{it->second, index_right});
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
Poco::Logger::get("Edges").information("BAD: " + atom->dumpTree());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
graph = BuildGraphFromAstsGraph(g);
|
||||
}
|
||||
|
||||
ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, const ASTPtr & right) const
|
||||
{
|
||||
size_t start = 0;
|
||||
size_t finish = 0;
|
||||
{
|
||||
/// TODO: check full ast
|
||||
const auto it_left = graph.ast_hash_to_component.find(left->getTreeHash());
|
||||
const auto it_right = graph.ast_hash_to_component.find(right->getTreeHash());
|
||||
if (it_left == std::end(graph.ast_hash_to_component) || it_right == std::end(graph.ast_hash_to_component))
|
||||
{
|
||||
Poco::Logger::get("Graph").information("not found");
|
||||
Poco::Logger::get("Graph").information(std::to_string(left->getTreeHash().second));
|
||||
Poco::Logger::get("Graph").information(std::to_string(right->getTreeHash().second));
|
||||
for (const auto & [hash, id] : graph.ast_hash_to_component)
|
||||
{
|
||||
Poco::Logger::get("Graph MAP").information(std::to_string(hash.second) + " " + std::to_string(id));
|
||||
}
|
||||
return CompareResult::UNKNOWN;
|
||||
}
|
||||
else
|
||||
{
|
||||
start = it_left->second;
|
||||
finish = it_right->second;
|
||||
Poco::Logger::get("Graph").information("found:" + std::to_string(start) + " " + std::to_string(finish));
|
||||
}
|
||||
}
|
||||
|
||||
if (start == finish)
|
||||
return CompareResult::EQUAL;
|
||||
|
||||
/// TODO: precalculate in O(n^3) using Floyd–Warshall algorithm where < = -1 and =< = 0.
|
||||
/// TODO: use it for less, greater and so on
|
||||
return CompareResult::UNKNOWN;
|
||||
}
|
||||
|
||||
std::vector<ASTPtr> ComparisonGraph::getEqual(const ASTPtr & ast) const
|
||||
{
|
||||
const auto hash_it = graph.ast_hash_to_component.find(ast->getTreeHash().second);
|
||||
const auto hash_it = graph.ast_hash_to_component.find(ast->getTreeHash());
|
||||
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),
|
||||
@ -30,9 +157,125 @@ std::vector<ASTPtr> ComparisonGraph::getEqual(const ASTPtr & ast) const
|
||||
}
|
||||
}
|
||||
|
||||
ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAsts(const Graph & /*asts_graph*/)
|
||||
void ComparisonGraph::dfsOrder(const Graph & asts_graph, size_t v, std::vector<bool> & visited, std::vector<size_t> & order) const
|
||||
{
|
||||
return {};
|
||||
visited[v] = true;
|
||||
for (const auto & edge : asts_graph.edges[v])
|
||||
{
|
||||
if (!visited[edge.to])
|
||||
{
|
||||
dfsOrder(asts_graph, edge.to, visited, order);
|
||||
}
|
||||
}
|
||||
order.push_back(v);
|
||||
}
|
||||
|
||||
ComparisonGraph::Graph ComparisonGraph::reverseGraph(const Graph & asts_graph) const
|
||||
{
|
||||
Graph g;
|
||||
g.ast_hash_to_component = asts_graph.ast_hash_to_component;
|
||||
g.vertexes = asts_graph.vertexes;
|
||||
g.edges.resize(g.vertexes.size());
|
||||
for (size_t v = 0; v < asts_graph.vertexes.size(); ++v)
|
||||
{
|
||||
for (const auto & edge : asts_graph.edges[v])
|
||||
{
|
||||
g.edges[edge.to].push_back(Edge{edge.type, v});
|
||||
}
|
||||
}
|
||||
return asts_graph;
|
||||
}
|
||||
|
||||
void ComparisonGraph::dfsComponents(
|
||||
const Graph & reversed_graph, size_t v, std::vector<size_t> & components, const size_t not_visited, const size_t component) const
|
||||
{
|
||||
components[v] = component;
|
||||
for (const auto & edge : reversed_graph.edges[v])
|
||||
{
|
||||
if (components[edge.to] == not_visited)
|
||||
{
|
||||
dfsComponents(reversed_graph, edge.to, components, not_visited, component);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & asts_graph) const
|
||||
{
|
||||
Poco::Logger::get("Graph").information("building");
|
||||
/// Find strongly connected component
|
||||
const auto n = asts_graph.vertexes.size();
|
||||
|
||||
std::vector<size_t> order;
|
||||
{
|
||||
std::vector<bool> visited(n, false);
|
||||
for (size_t v = 0; v < n; ++v)
|
||||
{
|
||||
if (!visited[v])
|
||||
dfsOrder(asts_graph, v, visited, order);
|
||||
}
|
||||
}
|
||||
|
||||
Poco::Logger::get("Graph").information("dfs1");
|
||||
|
||||
const auto not_visited = std::numeric_limits<size_t>::max();
|
||||
std::vector<size_t> components(n, not_visited);
|
||||
size_t component = 0;
|
||||
{
|
||||
const Graph reversed_graph = reverseGraph(asts_graph);
|
||||
for (const size_t v : order)
|
||||
{
|
||||
if (components[v] == not_visited)
|
||||
{
|
||||
dfsComponents(reversed_graph, v, components, not_visited, component);
|
||||
++component;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Poco::Logger::get("Graph").information("dfs2");
|
||||
|
||||
Graph result;
|
||||
result.vertexes.resize(component);
|
||||
result.edges.resize(component);
|
||||
for (const auto & [hash, index] : asts_graph.ast_hash_to_component)
|
||||
{
|
||||
result.ast_hash_to_component[hash] = components[index];
|
||||
result.vertexes[components[index]].asts.insert(
|
||||
std::end(result.vertexes[components[index]].asts),
|
||||
std::begin(asts_graph.vertexes[index].asts),
|
||||
std::end(asts_graph.vertexes[index].asts)); // asts_graph has only one ast per vertex
|
||||
}
|
||||
|
||||
Poco::Logger::get("Graph").information("components: " + std::to_string(component));
|
||||
|
||||
for (size_t v = 0; v < n; ++v)
|
||||
{
|
||||
for (const auto & edge : asts_graph.edges[v])
|
||||
{
|
||||
result.edges[components[v]].push_back(Edge{edge.type, components[edge.to]});
|
||||
}
|
||||
// TODO: make edges unique (most strict)
|
||||
}
|
||||
|
||||
Poco::Logger::get("Graph").information("finish");
|
||||
|
||||
for (size_t v = 0; v < result.vertexes.size(); ++v)
|
||||
{
|
||||
std::stringstream s;
|
||||
for (const auto & atom : result.vertexes[v].asts)
|
||||
{
|
||||
s << atom->getTreeHash().second << " ";
|
||||
}
|
||||
s << "|";
|
||||
for (const auto & atom : result.ast_hash_to_component)
|
||||
{
|
||||
s << atom.first.second << " -" << atom.second << " ";
|
||||
}
|
||||
|
||||
Poco::Logger::get("Graph").information(s.str());
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -22,20 +22,20 @@ public:
|
||||
EQUAL,
|
||||
GREATER_OR_EQUAL,
|
||||
GREATER,
|
||||
//NOT_EQUAL,
|
||||
UNKNOWN,
|
||||
};
|
||||
|
||||
// TODO: implement
|
||||
CompareResult compare(const ASTPtr & /*left*/, const ASTPtr & /*right*/) const { return CompareResult::UNKNOWN; }
|
||||
CompareResult compare(const ASTPtr & left, const ASTPtr & right) const;
|
||||
CompareResult getExpectedCompare(const ASTPtr & ast) const;
|
||||
|
||||
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
|
||||
//ASTPtr getMax(const ASTPtr &) const { return nullptr; } // sup
|
||||
//ASTPtr getMin(const ASTPtr &) const { return nullptr; } // inf
|
||||
//ASTPtr getEqualConst(const ASTPtr &) const { return nullptr; } // inf
|
||||
|
||||
private:
|
||||
/// strongly connected component
|
||||
@ -58,17 +58,29 @@ private:
|
||||
};
|
||||
|
||||
Type type;
|
||||
EqualComponent to;
|
||||
size_t to;
|
||||
};
|
||||
|
||||
struct Graph
|
||||
{
|
||||
std::unordered_map<UInt64, size_t> ast_hash_to_component;
|
||||
struct ASTHash {
|
||||
size_t operator() (const IAST::Hash & hash) const {
|
||||
return hash.first;
|
||||
}
|
||||
};
|
||||
|
||||
std::unordered_map<IAST::Hash, size_t, ASTHash> ast_hash_to_component;
|
||||
std::vector<EqualComponent> vertexes;
|
||||
std::vector<std::vector<Edge>> edges;
|
||||
};
|
||||
|
||||
Graph BuildGraphFromAsts(const Graph & asts_graph);
|
||||
ASTPtr normalizeAtom(const ASTPtr & atom) const;
|
||||
Graph BuildGraphFromAstsGraph(const Graph & asts_graph) const;
|
||||
|
||||
Graph reverseGraph(const Graph & asts_graph) const;
|
||||
void dfsOrder(const Graph & asts_graph, size_t v, std::vector<bool> & visited, std::vector<size_t> & order) const;
|
||||
void dfsComponents(
|
||||
const Graph & reversed_graph, size_t v, std::vector<size_t> & components, const size_t not_visited, const size_t component) const;
|
||||
|
||||
Graph graph;
|
||||
};
|
||||
|
@ -124,4 +124,6 @@ public:
|
||||
static ASTPtr fromCNF(const CNFQuery & cnf);
|
||||
};
|
||||
|
||||
void pushNotIn(CNFQuery::AtomicFormula & atom);
|
||||
|
||||
}
|
||||
|
@ -1,6 +1,8 @@
|
||||
#include <Interpreters/WhereConstraintsOptimizer.h>
|
||||
|
||||
#include <Interpreters/TreeCNFConverter.h>
|
||||
#include <Interpreters/ComparisonGraph.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTConstraintDeclaration.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
@ -9,6 +11,13 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
std::vector<std::vector<CNFQuery::AtomicFormula>> getConstraintData(const StorageMetadataPtr & metadata_snapshot)
|
||||
{
|
||||
std::vector<std::vector<CNFQuery::AtomicFormula>> constraint_data;
|
||||
@ -31,7 +40,7 @@ std::vector<CNFQuery::AtomicFormula> getAtomicConstraintData(const StorageMetada
|
||||
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
|
||||
.pullNotOutFunctions();
|
||||
for (const auto & group : cnf.getStatements()) {
|
||||
if (group.size() == 1)
|
||||
constraint_data.push_back(*group.begin());
|
||||
@ -41,20 +50,27 @@ std::vector<CNFQuery::AtomicFormula> getAtomicConstraintData(const StorageMetada
|
||||
return constraint_data;
|
||||
}
|
||||
|
||||
std::vector<std::pair<ASTPtr, ASTPtr>> getEqualConstraintData(const StorageMetadataPtr & metadata_snapshot)
|
||||
ComparisonGraph getComparisonGraph(const StorageMetadataPtr & metadata_snapshot)
|
||||
{
|
||||
std::vector<std::pair<ASTPtr, ASTPtr>> equal_constraints;
|
||||
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 (const auto & constraint : atomic_constraints) {
|
||||
auto * func = constraint.ast->as<ASTFunction>();
|
||||
if (func && (func->name == "equal" && !constraint.negative))
|
||||
for (auto & atomic_formula : atomic_formulas)
|
||||
{
|
||||
pushNotIn(atomic_formula);
|
||||
auto * func = atomic_formula.ast->as<ASTFunction>();
|
||||
if (func && relations.count(func->name))
|
||||
{
|
||||
equal_constraints.emplace_back(
|
||||
func->arguments->children[0],
|
||||
func->arguments->children[1]);
|
||||
if (atomic_formula.negative)
|
||||
throw Exception(": ", ErrorCodes::LOGICAL_ERROR);
|
||||
constraints_for_graph.push_back(atomic_formula.ast);
|
||||
}
|
||||
}
|
||||
return equal_constraints;
|
||||
|
||||
return ComparisonGraph(constraints_for_graph);
|
||||
}
|
||||
|
||||
WhereConstraintsOptimizer::WhereConstraintsOptimizer(
|
||||
@ -93,7 +109,7 @@ MatchState match(CNFQuery::AtomicFormula a, CNFQuery::AtomicFormula b)
|
||||
return MatchState::NONE;
|
||||
}
|
||||
|
||||
bool checkIfGroupAlwaysTrue(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: constraints graph
|
||||
|
||||
@ -106,9 +122,9 @@ bool checkIfGroupAlwaysTrue(const CNFQuery::OrGroup & group, const std::vector<s
|
||||
for (const auto & constraint_ast : constraint)
|
||||
{
|
||||
bool found_match = false;
|
||||
for (const auto & group_ast : group)
|
||||
for (const auto & atom_ast : group)
|
||||
{
|
||||
const auto match_result = match(constraint_ast, group_ast);
|
||||
const auto match_result = match(constraint_ast, atom_ast);
|
||||
|
||||
if (match_result == MatchState::FULL_MATCH)
|
||||
{
|
||||
@ -129,7 +145,61 @@ bool checkIfGroupAlwaysTrue(const CNFQuery::OrGroup & group, const std::vector<s
|
||||
return false;
|
||||
}
|
||||
|
||||
bool checkIfAtomAlwaysFalse(const CNFQuery::AtomicFormula & atom, const std::vector<std::vector<CNFQuery::AtomicFormula>> & constraints)
|
||||
ComparisonGraph::CompareResult getExpectedCompare(const CNFQuery::AtomicFormula & atom)
|
||||
{
|
||||
static const std::map<std::string, std::string> inverse_relations = {
|
||||
{"equals", "notEquals"},
|
||||
{"less", "greaterOrEquals"},
|
||||
{"lessOrEquals", "greater"},
|
||||
{"notEquals", "equals"},
|
||||
{"greaterOrEquals", "less"},
|
||||
{"greater", "lessOrEquals"},
|
||||
};
|
||||
|
||||
static const std::map<std::string, ComparisonGraph::CompareResult> relation_to_compare = {
|
||||
{"equals", ComparisonGraph::CompareResult::EQUAL},
|
||||
{"less", ComparisonGraph::CompareResult::LESS},
|
||||
{"lessOrEquals", ComparisonGraph::CompareResult::LESS_OR_EQUAL},
|
||||
{"notEquals", ComparisonGraph::CompareResult::UNKNOWN},
|
||||
{"greaterOrEquals", ComparisonGraph::CompareResult::GREATER_OR_EQUAL},
|
||||
{"greater", ComparisonGraph::CompareResult::GREATER},
|
||||
};
|
||||
|
||||
|
||||
const auto * func = atom.ast->as<ASTFunction>();
|
||||
if (func && inverse_relations.count(func->name))
|
||||
{
|
||||
std::string function_name = func->name;
|
||||
if (atom.negative)
|
||||
{
|
||||
function_name = inverse_relations.at(func->name);
|
||||
}
|
||||
return relation_to_compare.at(function_name);
|
||||
}
|
||||
return ComparisonGraph::CompareResult::UNKNOWN;
|
||||
}
|
||||
|
||||
|
||||
bool checkIfGroupAlwaysTrueGraph(const CNFQuery::OrGroup & group, const ComparisonGraph & graph)
|
||||
{
|
||||
for (const auto & atom : group)
|
||||
{
|
||||
const auto * func = atom.ast->as<ASTFunction>();
|
||||
if (func && func->arguments->children.size() == 2)
|
||||
{
|
||||
const auto expected = getExpectedCompare(atom);
|
||||
const auto result = graph.compare(func->arguments->children[0], func->arguments->children[1]);
|
||||
Poco::Logger::get("GRAPH REASON").information("neg: " + std::to_string(atom.negative));
|
||||
Poco::Logger::get("GRAPH REASON").information(atom.ast->dumpTree());
|
||||
Poco::Logger::get("GRAPH REASON").information(std::to_string(static_cast<int>(expected)) + " " + std::to_string(static_cast<int>(result)));
|
||||
return expected != ComparisonGraph::CompareResult::UNKNOWN && expected == result;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
bool checkIfAtomAlwaysFalseFullMatch(const CNFQuery::AtomicFormula & atom, const std::vector<std::vector<CNFQuery::AtomicFormula>> & constraints)
|
||||
{
|
||||
/// TODO: more efficient matching
|
||||
|
||||
@ -155,15 +225,16 @@ void WhereConstraintsOptimizer::perform()
|
||||
if (select_query->where() && metadata_snapshot)
|
||||
{
|
||||
const auto constraint_data = getConstraintData(metadata_snapshot);
|
||||
const auto compare_graph = getComparisonGraph(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);
|
||||
.filterAlwaysTrueGroups([&constraint_data, &compare_graph](const auto & group) { /// remove always true groups from CNF
|
||||
return !checkIfGroupAlwaysTrueFullMatch(group, constraint_data) && !checkIfGroupAlwaysTrueGraph(group, compare_graph);
|
||||
})
|
||||
.filterAlwaysFalseAtoms([&constraint_data](const auto & atom) { /// remove always false atoms from CNF
|
||||
return !checkIfAtomAlwaysFalse(atom, constraint_data);
|
||||
return !checkIfAtomAlwaysFalseFullMatch(atom, constraint_data);
|
||||
})
|
||||
.pushNotInFuntions();
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user