a AND not a

This commit is contained in:
Nikita Vasilev 2021-05-05 11:51:25 +03:00
parent d274d4a579
commit 1d26957029
7 changed files with 66 additions and 65 deletions

View File

@ -1,38 +0,0 @@
#include "ConstraintMatcherVisitor.h"
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/IAST.h>
#include <Poco/Logger.h>
namespace DB
{
bool ConstraintMatcher::needChildVisit(const ASTPtr & node, const ASTPtr &)
{
return node->as<ASTFunction>() || node->as<ASTExpressionList>();
}
std::optional<bool> ConstraintMatcher::getASTValue(const ASTPtr & node, Data & data)
{
const auto it = data.constraints.find(node->getTreeHash().second);
if (it != std::end(data.constraints))
{
for (const auto & ast : it->second)
{
if (node->getColumnName() == ast->getColumnName())
{
return true;
}
}
}
return std::nullopt;
}
void ConstraintMatcher::visit(ASTPtr & ast, Data & data)
{
if (const auto always_value = getASTValue(ast, data); always_value)
ast = std::make_shared<ASTLiteral>(static_cast<UInt8>(*always_value));
}
}

View File

@ -1,24 +0,0 @@
#pragma once
#include <Interpreters/InDepthNodeVisitor.h>
namespace DB
{
struct ConstraintMatcher
{
struct Data
{
std::unordered_map<UInt64, std::vector<ASTPtr>> constraints;
};
using Visitor = InDepthNodeVisitor<ConstraintMatcher, true>;
static bool needChildVisit(const ASTPtr & node, const ASTPtr &);
static std::optional<bool> getASTValue(const ASTPtr & node, Data & data);
static void visit(ASTPtr & ast, Data & data);
};
using ConstraintMatcherVisitor = InDepthNodeVisitor<ConstraintMatcher, true>;
}

View File

@ -38,6 +38,7 @@ void traversePushNot(ASTPtr & node, bool add_negation)
{
if (add_negation)
{
ASSERT(func->arguments->size() == 2)
/// apply De Morgan's Law
node = makeASTFunction(
(func->name == "and" ? "or" : "and"),
@ -51,6 +52,7 @@ void traversePushNot(ASTPtr & node, bool add_negation)
}
else if (func && func->name == "not")
{
ASSERT(func->arguments->size() == 1)
/// delete NOT
node = func->arguments->children[0]->clone();
@ -91,6 +93,7 @@ void pushOr(ASTPtr & query)
auto * or_func = or_node.get()->as<ASTFunction>();
ASSERT(or_func)
ASSERT(or_func->name == "or")
ASSERT(or_func->arguments->children.size() == 2)
/// find or upper than and
size_t and_node_id = or_func->arguments->children.size();
@ -110,6 +113,7 @@ void pushOr(ASTPtr & query)
auto and_func = or_func->arguments->children[and_node_id]->as<ASTFunction>();
ASSERT(and_func)
ASSERT(and_func->name == "and")
ASSERT(and_func->arguments->children.size() == 2)
auto a = or_func->arguments->children[other_node_id];
auto b = and_func->arguments->children[0];
@ -300,6 +304,52 @@ CNFQuery & CNFQuery::pushNotInFuntions()
return *this;
}
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;
}
}
CNFQuery & CNFQuery::reduce()
{
while (true)
{
AndGroup new_statements = reduceOnce(statements);
if (statements == new_statements)
return *this;
else
statements = new_statements;
}
}
std::string CNFQuery::dump() const
{
WriteBufferFromOwnString res;

View File

@ -135,6 +135,9 @@ public:
/// Revert pullNotOutFunctions actions
CNFQuery & pushNotInFuntions();
/// (a OR b OR ...) AND (NOT a OR b OR ...) -> (b OR ...)
CNFQuery & reduce();
private:
AndGroup statements;
};

View File

@ -36,8 +36,7 @@ MatchState match(CNFQuery::AtomicFormula a, CNFQuery::AtomicFormula b)
{
bool match_means_ok = true ^ a.negative ^ b.negative;
if (a.ast->getTreeHash() == b.ast->getTreeHash() &&
a.ast->getColumnName() == b.ast->getColumnName())
if (a.ast->getTreeHash() == b.ast->getTreeHash())
{
return match_means_ok ? MatchState::FULL_MATCH : MatchState::NOT_MATCH;
}
@ -109,7 +108,7 @@ bool checkIfAtomAlwaysFalseFullMatch(const CNFQuery::AtomicFormula & atom, const
for (const auto & constraint : constraints)
{
if (constraint.size() > 1)
continue; /// TMP
continue;
for (const auto & constraint_atoms : constraint)
{
@ -185,6 +184,7 @@ void WhereConstraintsOptimizer::perform()
{
return replaceTermsToConstants(atom, compare_graph);
})
.reduce()
.pushNotInFuntions();
if (optimize_append_index)

View File

@ -31,3 +31,9 @@
0
1
0
SELECT count()
FROM constraint_test_constants
WHERE (c > 100) OR (b > 100)
SELECT count()
FROM constraint_test_constants
WHERE c > 100

View File

@ -95,4 +95,8 @@ SELECT count() FROM constraint_test_constants WHERE 10 < a; ---> assumption -> 0
SELECT count() FROM constraint_test_constants WHERE 9 <= a; ---> assumption -> 1
SELECT count() FROM constraint_test_constants WHERE 11 <= a; ---> assumption -> 0
-- A AND NOT A
EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100);
EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100);
DROP TABLE constraint_test_constants;