This commit is contained in:
Nikita Vasilev 2021-05-19 22:07:27 +03:00
parent 6102ea643a
commit af55e75e8d
3 changed files with 65 additions and 47 deletions

View File

@ -28,7 +28,7 @@ void splitMultiLogic(ASTPtr & node)
{
res = makeASTFunction(func->name, res, func->arguments->children[i]->clone());
}
node = std::move(res);
node = res;
}
auto * new_func = node->as<ASTFunction>();
@ -36,6 +36,11 @@ void splitMultiLogic(ASTPtr & node)
for (auto & child : new_func->arguments->children)
splitMultiLogic(child);
}
else if (func->name == "not")
{
for (auto & child : func->arguments->children)
splitMultiLogic(child);
}
}
/// Push NOT to leafs, remove NOT NOT ...
@ -76,72 +81,44 @@ void traversePushNot(ASTPtr & node, bool add_negation)
}
}
void findOrs(ASTPtr & node, std::vector<std::reference_wrapper<ASTPtr>> & ors)
{
/// Push Or inside And (actually pull AND to top)
void traversePushOr(ASTPtr & node) {
auto * func = node->as<ASTFunction>();
if (func && func->name == "or")
ors.push_back(node);
if (func && (func->name == "and" || func->name == "not"))
if (func && (func->name == "or" || func->name == "and"))
{
for (auto & child : func->arguments->children)
findOrs(child, ors);
traversePushOr(child);
}
}
/// Push Or inside And (actually pull AND to top)
void pushOr(ASTPtr & query)
{
std::vector<std::reference_wrapper<ASTPtr>> ors;
findOrs(query, ors);
while (!ors.empty())
if (func && func->name == "or")
{
std::reference_wrapper<ASTPtr> or_node = ors.back();
ors.pop_back();
auto * or_func = or_node.get()->as<ASTFunction>();
if (or_func->arguments->children.size() != 2)
throw Exception("Bad OR function", ErrorCodes::LOGICAL_ERROR);
/// find or upper than and
size_t and_node_id = or_func->arguments->children.size();
for (size_t i = 0; i < or_func->arguments->children.size(); ++i)
size_t and_node_id = func->arguments->children.size();
for (size_t i = 0; i < func->arguments->children.size(); ++i)
{
auto & child = or_func->arguments->children[i];
auto & child = func->arguments->children[i];
auto * and_func = child->as<ASTFunction>();
if (and_func && and_func->name == "and")
{
and_node_id = i;
}
}
if (and_node_id == or_func->arguments->children.size())
continue;
if (and_node_id == func->arguments->children.size())
return;
const size_t other_node_id = 1 - and_node_id;
const auto * and_func = or_func->arguments->children[and_node_id]->as<ASTFunction>();
if (and_func->arguments->children.size() != 2)
{
Poco::Logger::get("$$$$$").information(and_func->name);
Poco::Logger::get("CHILDREN: ").information(std::to_string(and_func->arguments->children.size()));
throw Exception("Bad AND function", ErrorCodes::LOGICAL_ERROR);
}
auto a = or_func->arguments->children[other_node_id];
const auto * and_func = func->arguments->children[and_node_id]->as<ASTFunction>();
auto a = func->arguments->children[other_node_id];
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) )
or_node.get() = makeASTFunction(
"and",
makeASTFunction("or", a->clone(), b->clone()),
makeASTFunction("or", a->clone(), c->clone()));
node = makeASTFunction(
"and",
makeASTFunction("or", a->clone(), b),
makeASTFunction("or", a, c));
/// add new ors to stack
auto * new_func = or_node.get()->as<ASTFunction>();
for (auto & new_or : new_func->arguments->children)
ors.push_back(new_or);
traversePushOr(node);
}
}
@ -191,8 +168,17 @@ CNFQuery TreeCNFConverter::toCNF(const ASTPtr & query)
auto cnf = query->clone();
splitMultiLogic(cnf);
Poco::Logger::get("CNF CONVERSION").information("SPLIT:" + cnf->dumpTree());
Poco::Logger::get("CNF CONVERSION").information("SPLIT:" + cnf->getColumnName());
traversePushNot(cnf, false);
pushOr(cnf);
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());
CNFQuery::AndGroup and_group;
traverseCNF(cnf, and_group);

View File

@ -0,0 +1,18 @@
SELECT i
FROM cnf_test
WHERE (i <= 2) AND (i <= 1)
SELECT i
FROM cnf_test
WHERE (i <= 2) OR (i <= 1)
SELECT i
FROM cnf_test
WHERE ((i > 4) OR (i > 1) OR (i > 6)) AND ((i > 4) OR (i > 1) OR (i > 5)) AND ((i > 4) OR (i > 6) OR (i > 2)) AND ((i > 4) OR (i > 2) OR (i > 5)) AND ((i > 1) OR (i > 6) OR (i > 3)) AND ((i > 1) OR (i > 3) OR (i > 5)) AND ((i > 6) OR (i > 2) OR (i > 3)) AND ((i > 2) OR (i > 3) OR (i > 5))
SELECT i
FROM cnf_test
WHERE ((i <= 5) OR (i <= 2) OR (i <= 3)) AND ((i <= 5) OR (i <= 2) OR (i <= 4)) AND ((i <= 5) OR (i <= 3) OR (i <= 1)) AND ((i <= 5) OR (i <= 4) OR (i <= 1)) AND ((i <= 2) OR (i <= 3) OR (i <= 6)) AND ((i <= 2) OR (i <= 4) OR (i <= 6)) AND ((i <= 3) OR (i <= 1) OR (i <= 6)) AND ((i <= 4) OR (i <= 1) OR (i <= 6))
SELECT i
FROM cnf_test
WHERE ((i > 4) OR (i > 1) OR (i > 6)) AND ((i > 4) OR (i > 1) OR (i > 5)) AND ((i > 4) OR (i > 6) OR (i > 2)) AND ((i > 4) OR (i > 6) OR (i > 7)) AND ((i > 4) OR (i > 2) OR (i > 5)) AND ((i > 4) OR (i > 7) OR (i > 5)) AND ((i > 1) OR (i > 8) OR (i > 6)) AND ((i > 1) OR (i > 8) OR (i > 5)) AND ((i > 1) OR (i > 6) OR (i > 3)) AND ((i > 1) OR (i > 3) OR (i > 5)) AND ((i > 8) OR (i > 6) OR (i > 2)) AND ((i > 8) OR (i > 6) OR (i > 7)) AND ((i > 8) OR (i > 2) OR (i > 5)) AND ((i > 8) OR (i > 7) OR (i > 5)) AND ((i > 6) OR (i > 2) OR (i > 3)) AND ((i > 6) OR (i > 3) OR (i > 7)) AND ((i > 2) OR (i > 3) OR (i > 5)) AND ((i > 3) OR (i > 7) OR (i > 5))
SELECT i
FROM cnf_test
WHERE ((i > 4) OR (i > 8) OR (i > 3)) AND (i <= 5) AND ((i > 1) OR (i > 2) OR (i > 7)) AND (i <= 6)

View File

@ -0,0 +1,14 @@
SET convert_query_to_cnf = 1;
CREATE TABLE cnf_test (i Int64) ENGINE = MergeTree() ORDER BY i;
EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE NOT ((i > 1) OR (i > 2));
EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE NOT ((i > 1) AND (i > 2));
EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE ((i > 1) AND (i > 2)) OR ((i > 3) AND (i > 4)) OR ((i > 5) AND (i > 6));
EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE NOT (((i > 1) OR (i > 2)) AND ((i > 3) OR (i > 4)) AND ((i > 5) OR (i > 6)));
EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE ((i > 1) AND (i > 2) AND (i > 7)) OR ((i > 3) AND (i > 4) AND (i > 8)) OR ((i > 5) AND (i > 6));
EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE ((i > 1) OR (i > 2) OR (i > 7)) AND ((i > 3) OR (i > 4) OR (i > 8)) AND NOT ((i > 5) OR (i > 6));