ClickHouse/src/Interpreters/TreeCNFConverter.cpp

416 lines
12 KiB
C++
Raw Normal View History

2021-01-04 20:55:32 +00:00
#include <Interpreters/TreeCNFConverter.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTFunction.h>
#include <Poco/Logger.h>
namespace DB
{
2021-05-06 08:29:24 +00:00
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
2021-05-07 10:49:05 +00:00
extern const int INCORRECT_QUERY;
2021-05-06 08:29:24 +00:00
}
2021-01-04 20:55:32 +00:00
/// Splits AND(a, b, c) to AND(a, AND(b, c)) for AND/OR
void splitMultiLogic(ASTPtr & node)
{
auto * func = node->as<ASTFunction>();
if (func && (func->name == "and" || func->name == "or"))
{
2021-05-07 10:49:05 +00:00
if (func->arguments->children.size() < 2)
throw Exception("Bad logical function", ErrorCodes::INCORRECT_QUERY);
2021-01-04 20:55:32 +00:00
if (func->arguments->children.size() > 2)
{
2021-05-08 09:19:18 +00:00
ASTPtr res = func->arguments->children[0]->clone();
2021-01-04 20:55:32 +00:00
for (size_t i = 1; i < func->arguments->children.size(); ++i)
{
2021-02-14 14:47:15 +00:00
res = makeASTFunction(func->name, res, func->arguments->children[i]->clone());
2021-01-04 20:55:32 +00:00
}
2021-05-19 19:07:27 +00:00
node = res;
2021-01-04 20:55:32 +00:00
}
auto * new_func = node->as<ASTFunction>();
2021-05-08 09:19:18 +00:00
Poco::Logger::get("new_func_children").information(std::to_string(new_func->arguments->children.size()));
2021-01-04 20:55:32 +00:00
for (auto & child : new_func->arguments->children)
splitMultiLogic(child);
}
2021-05-19 19:07:27 +00:00
else if (func->name == "not")
{
for (auto & child : func->arguments->children)
splitMultiLogic(child);
}
2021-01-04 20:55:32 +00:00
}
2021-01-05 20:51:19 +00:00
/// Push NOT to leafs, remove NOT NOT ...
2021-01-04 20:55:32 +00:00
void traversePushNot(ASTPtr & node, bool add_negation)
{
auto * func = node->as<ASTFunction>();
if (func && (func->name == "and" || func->name == "or"))
{
2021-05-04 18:43:58 +00:00
if (add_negation)
{
2021-05-05 13:29:28 +00:00
if (func->arguments->children.size() != 2)
throw Exception("Bad AND or OR function.", ErrorCodes::LOGICAL_ERROR);
2021-01-04 20:55:32 +00:00
/// apply De Morgan's Law
node = makeASTFunction(
(func->name == "and" ? "or" : "and"),
2021-02-14 14:47:15 +00:00
func->arguments->children[0]->clone(),
func->arguments->children[1]->clone());
2021-01-04 20:55:32 +00:00
}
auto * new_func = node->as<ASTFunction>();
for (auto & child : new_func->arguments->children)
traversePushNot(child, add_negation);
}
else if (func && func->name == "not")
{
2021-05-05 13:29:28 +00:00
if (func->arguments->children.size() != 1)
2021-05-07 10:49:05 +00:00
throw Exception("Bad NOT function.", ErrorCodes::INCORRECT_QUERY);
2021-01-04 20:55:32 +00:00
/// delete NOT
2021-02-14 14:47:15 +00:00
node = func->arguments->children[0]->clone();
2021-01-04 20:55:32 +00:00
traversePushNot(node, !add_negation);
}
else
{
if (add_negation)
2021-02-14 14:47:15 +00:00
node = makeASTFunction("not", node->clone());
2021-01-04 20:55:32 +00:00
}
}
2021-05-19 19:07:27 +00:00
/// Push Or inside And (actually pull AND to top)
void traversePushOr(ASTPtr & node) {
2021-01-04 20:55:32 +00:00
auto * func = node->as<ASTFunction>();
2021-05-19 19:07:27 +00:00
if (func && (func->name == "or" || func->name == "and"))
2021-01-04 20:55:32 +00:00
{
for (auto & child : func->arguments->children)
2021-05-19 19:07:27 +00:00
traversePushOr(child);
2021-01-04 20:55:32 +00:00
}
2021-05-19 19:07:27 +00:00
if (func && func->name == "or")
2021-01-04 20:55:32 +00:00
{
2021-05-19 19:07:27 +00:00
size_t and_node_id = func->arguments->children.size();
for (size_t i = 0; i < func->arguments->children.size(); ++i)
2021-01-04 20:55:32 +00:00
{
2021-05-19 19:07:27 +00:00
auto & child = func->arguments->children[i];
2021-01-04 20:55:32 +00:00
auto * and_func = child->as<ASTFunction>();
2021-05-04 21:36:30 +00:00
if (and_func && and_func->name == "and")
2021-01-04 20:55:32 +00:00
{
and_node_id = i;
}
}
2021-05-19 19:07:27 +00:00
if (and_node_id == func->arguments->children.size())
return;
2021-01-04 20:55:32 +00:00
const size_t other_node_id = 1 - and_node_id;
2021-05-19 19:07:27 +00:00
const auto * and_func = func->arguments->children[and_node_id]->as<ASTFunction>();
auto a = func->arguments->children[other_node_id];
2021-01-04 20:55:32 +00:00
auto b = and_func->arguments->children[0];
auto c = and_func->arguments->children[1];
/// apply the distributive law ( a or (b and c) -> (a or b) and (a or c) )
2021-05-19 19:07:27 +00:00
node = makeASTFunction(
"and",
makeASTFunction("or", a->clone(), b),
makeASTFunction("or", a, c));
traversePushOr(node);
2021-01-04 20:55:32 +00:00
}
}
/// transform ast into cnf groups
void traverseCNF(const ASTPtr & node, CNFQuery::AndGroup & and_group, CNFQuery::OrGroup & or_group)
{
auto * func = node->as<ASTFunction>();
if (func && func->name == "and")
{
for (auto & child : func->arguments->children)
{
CNFQuery::OrGroup group;
traverseCNF(child, and_group, group);
if (!group.empty())
and_group.insert(std::move(group));
}
}
else if (func && func->name == "or")
{
for (auto & child : func->arguments->children)
{
traverseCNF(child, and_group, or_group);
}
}
2021-03-04 12:11:43 +00:00
else if (func && func->name == "not")
{
2021-05-07 10:49:05 +00:00
if (func->arguments->children.size() != 1)
throw Exception("Bad NOT function", ErrorCodes::INCORRECT_QUERY);
2021-03-04 12:11:43 +00:00
or_group.insert(CNFQuery::AtomicFormula{true, func->arguments->children.front()});
}
2021-01-04 20:55:32 +00:00
else
{
2021-03-04 12:11:43 +00:00
or_group.insert(CNFQuery::AtomicFormula{false, node});
2021-01-04 20:55:32 +00:00
}
}
void traverseCNF(const ASTPtr & node, CNFQuery::AndGroup & result)
{
CNFQuery::OrGroup or_group;
traverseCNF(node, result, or_group);
if (!or_group.empty())
result.insert(or_group);
}
CNFQuery TreeCNFConverter::toCNF(const ASTPtr & query)
{
auto cnf = query->clone();
splitMultiLogic(cnf);
2021-05-19 19:07:27 +00:00
Poco::Logger::get("CNF CONVERSION").information("SPLIT:" + cnf->dumpTree());
Poco::Logger::get("CNF CONVERSION").information("SPLIT:" + cnf->getColumnName());
2021-01-04 20:55:32 +00:00
traversePushNot(cnf, false);
2021-05-19 19:07:27 +00:00
Poco::Logger::get("PUSH NOT").information("SPLIT:" + cnf->dumpTree());
Poco::Logger::get("PUSH NOT").information("SPLIT:" + cnf->getColumnName());
traversePushOr(cnf);
Poco::Logger::get("PUSH OR").information("SPLIT:" + cnf->dumpTree());
Poco::Logger::get("PUSH OR").information("SPLIT:" + cnf->getColumnName());
2021-01-04 20:55:32 +00:00
CNFQuery::AndGroup and_group;
traverseCNF(cnf, and_group);
CNFQuery result{std::move(and_group)};
Poco::Logger::get("CNF CONVERSION").information("DONE: " + result.dump());
return result;
}
ASTPtr TreeCNFConverter::fromCNF(const CNFQuery & cnf)
{
const auto & groups = cnf.getStatements();
if (groups.empty())
return nullptr;
ASTs or_groups;
for (const auto & group : groups)
{
if (group.size() == 1)
2021-03-04 12:11:43 +00:00
{
if ((*group.begin()).negative)
or_groups.push_back(makeASTFunction("not", (*group.begin()).ast->clone()));
else
or_groups.push_back((*group.begin()).ast->clone());
}
2021-01-04 20:55:32 +00:00
else if (group.size() > 1)
{
or_groups.push_back(makeASTFunction("or"));
auto * func = or_groups.back()->as<ASTFunction>();
2021-03-04 12:11:43 +00:00
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());
}
2021-01-04 20:55:32 +00:00
}
}
if (or_groups.size() == 1)
return or_groups.front();
ASTPtr res = makeASTFunction("and");
auto * func = res->as<ASTFunction>();
for (const auto & group : or_groups)
func->arguments->children.push_back(group);
return res;
}
2021-03-04 12:11:43 +00:00
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)
2021-01-05 20:51:19 +00:00
{
static const std::map<std::string, std::string> inverse_relations = {
{"notEquals", "equals"},
{"greaterOrEquals", "less"},
{"greater", "lessOrEquals"},
{"notIn", "in"},
{"notLike", "like"},
{"notEmpty", "empty"},
};
2021-03-04 12:11:43 +00:00
pushPullNotInAtom(atom, inverse_relations);
2021-01-05 20:51:19 +00:00
}
2021-03-04 12:11:43 +00:00
void pushNotIn(CNFQuery::AtomicFormula & atom)
2021-01-05 20:51:19 +00:00
{
2021-03-04 12:11:43 +00:00
if (!atom.negative)
return;
2021-01-05 20:51:19 +00:00
static const std::map<std::string, std::string> inverse_relations = {
{"equals", "notEquals"},
{"less", "greaterOrEquals"},
{"lessOrEquals", "greater"},
{"in", "notIn"},
{"like", "notLike"},
{"empty", "notEmpty"},
2021-03-04 12:11:43 +00:00
{"notEquals", "equals"},
{"greaterOrEquals", "less"},
{"greater", "lessOrEquals"},
{"notIn", "in"},
{"notLike", "like"},
{"notEmpty", "empty"},
2021-01-05 20:51:19 +00:00
};
2021-03-04 12:11:43 +00:00
pushPullNotInAtom(atom, inverse_relations);
2021-01-05 20:51:19 +00:00
}
CNFQuery & CNFQuery::pullNotOutFunctions()
{
2021-03-04 12:11:43 +00:00
transformAtoms([](const AtomicFormula & atom) -> AtomicFormula
{
AtomicFormula result{atom.negative, atom.ast->clone()};
pullNotOut(result);
return result;
});
2021-01-05 20:51:19 +00:00
return *this;
}
CNFQuery & CNFQuery::pushNotInFuntions()
{
2021-03-04 12:11:43 +00:00
transformAtoms([](const AtomicFormula & atom) -> AtomicFormula
2021-01-05 20:51:19 +00:00
{
2021-03-04 12:11:43 +00:00
AtomicFormula result{atom.negative, atom.ast->clone()};
pushNotIn(result);
2021-01-05 20:51:19 +00:00
return result;
});
return *this;
}
2021-05-05 08:51:25 +00:00
namespace
{
CNFQuery::AndGroup reduceOnce(const CNFQuery::AndGroup & groups)
{
CNFQuery::AndGroup result;
for (const CNFQuery::OrGroup & group : groups)
{
CNFQuery::OrGroup copy(group);
bool inserted = false;
for (const CNFQuery::AtomicFormula & atom : group)
{
copy.erase(atom);
CNFQuery::AtomicFormula negative_atom(atom);
negative_atom.negative = !atom.negative;
copy.insert(negative_atom);
if (groups.contains(copy))
{
copy.erase(negative_atom);
result.insert(copy);
inserted = true;
break;
}
copy.erase(negative_atom);
copy.insert(atom);
}
if (!inserted)
result.insert(group);
}
return result;
}
2021-05-05 11:28:02 +00:00
bool isSubset(const CNFQuery::OrGroup & left, const CNFQuery::OrGroup & right)
{
if (left.size() > right.size())
return false;
for (const auto & elem : left)
if (!right.contains(elem))
return false;
return true;
}
CNFQuery::AndGroup filterSubsets(const CNFQuery::AndGroup & groups)
{
CNFQuery::AndGroup result;
for (const CNFQuery::OrGroup & group : groups)
{
bool insert = true;
for (const CNFQuery::OrGroup & other_group : groups)
{
if (isSubset(other_group, group) && group != other_group)
{
insert = false;
break;
}
}
if (insert)
result.insert(group);
}
return result;
}
2021-05-05 08:51:25 +00:00
}
CNFQuery & CNFQuery::reduce()
{
while (true)
{
AndGroup new_statements = reduceOnce(statements);
if (statements == new_statements)
2021-05-05 11:28:02 +00:00
{
statements = filterSubsets(statements);
2021-05-05 08:51:25 +00:00
return *this;
2021-05-05 11:28:02 +00:00
}
2021-05-05 08:51:25 +00:00
else
statements = new_statements;
}
}
2021-01-04 20:55:32 +00:00
std::string CNFQuery::dump() const
{
2021-05-04 18:43:58 +00:00
WriteBufferFromOwnString res;
2021-01-04 20:55:32 +00:00
bool first = true;
for (const auto & group : statements)
{
if (!first)
res << " AND ";
first = false;
res << "(";
bool first_in_group = true;
2021-03-04 12:11:43 +00:00
for (const auto & atom : group)
2021-01-04 20:55:32 +00:00
{
if (!first_in_group)
res << " OR ";
first_in_group = false;
2021-03-04 12:11:43 +00:00
if (atom.negative)
res << " NOT ";
res << atom.ast->getColumnName();
2021-01-04 20:55:32 +00:00
}
res << ")";
}
return res.str();
}
}