mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
a AND not a
This commit is contained in:
parent
d274d4a579
commit
1d26957029
@ -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));
|
||||
}
|
||||
|
||||
}
|
@ -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>;
|
||||
|
||||
}
|
@ -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;
|
||||
|
@ -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;
|
||||
};
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
Loading…
Reference in New Issue
Block a user