improve hypothesis

This commit is contained in:
Nikita Vasilev 2021-05-03 22:08:26 +03:00
parent 564a484642
commit cdb0d86e2b
7 changed files with 146 additions and 37 deletions

View File

@ -191,11 +191,11 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con
if (start == finish)
return CompareResult::EQUAL;
auto [has_path, is_strict] = findPath(start, finish);
const auto [has_path, is_strict] = findPath(start, finish);
if (has_path)
return is_strict ? CompareResult::GREATER : CompareResult::GREATER_OR_EQUAL;
auto [has_path_reverse, is_strict_reverse] = findPath(finish, start);
const auto [has_path_reverse, is_strict_reverse] = findPath(finish, start);
if (has_path_reverse)
return is_strict_reverse ? CompareResult::LESS : CompareResult::LESS_OR_EQUAL;
@ -279,6 +279,11 @@ std::optional<std::size_t> ComparisonGraph::getComponentId(const ASTPtr & ast) c
}
}
bool ComparisonGraph::hasPath(const size_t left, const size_t right) const
{
return findPath(left, right).first || findPath(right, left).first;
}
std::vector<ASTPtr> ComparisonGraph::getComponent(const std::size_t id) const
{
return graph.vertexes[id].asts;

View File

@ -41,6 +41,7 @@ public:
std::optional<std::size_t> getComponentId(const ASTPtr & ast) const;
std::vector<ASTPtr> getComponent(const std::size_t id) const;
bool hasPath(const size_t left, const size_t right) const;
/// Find constants lessOrEqual and greaterOrEqual.
/// For int and double linear programming can be applied here.

View File

@ -55,10 +55,11 @@ void MergeTreeIndexMergedCondition::addIndex(const MergeTreeIndexPtr & index)
std::vector<ASTPtr> compare_hypotheses_data;
std::vector<CNFQuery::OrGroup> hypotheses_data;
const auto cnf = TreeCNFConverter::toCNF(hypothesis_index->index.expression_list_ast->children.front()).pullNotOutFunctions();
for (const auto & group : cnf.getStatements()) {
hypotheses_data.push_back(group);
for (const auto & group : cnf.getStatements())
{
if (group.size() == 1)
{
hypotheses_data.push_back(group);
CNFQuery::AtomicFormula atom = *group.begin();
pushNotIn(atom);
if (atom.negative)
@ -120,6 +121,53 @@ ComparisonGraph::CompareResult getExpectedCompare(const CNFQuery::AtomicFormula
}
/// Replaces < -> <=, > -> >= and assumes that all hypotheses are true then checks if path exists
bool MergeTreeIndexMergedCondition::alwaysUnknownOrTrue() const {
std::vector<ASTPtr> active_atomic_formulas(atomic_constraints);
for (size_t i = 0; i < index_to_compare_atomic_hypotheses.size(); ++i)
{
active_atomic_formulas.insert(
std::end(active_atomic_formulas),
std::begin(index_to_compare_atomic_hypotheses[i]),
std::end(index_to_compare_atomic_hypotheses[i]));
}
/// transform active formulas
for (auto & formula : active_atomic_formulas)
{
formula = formula->clone(); /// do all operations with copy
auto * func = formula->as<ASTFunction>();
if (func && func->name == "less")
func->name = "lessOrEquals";
if (func && func->name == "greater")
func->name = "greaterOrEquals";
}
const auto weak_graph = std::make_unique<ComparisonGraph>(active_atomic_formulas);
bool useless = true;
expression_cnf->iterateGroups(
[&](const CNFQuery::OrGroup & or_group)
{
for (auto atom : or_group)
{
pushNotIn(atom);
const auto * func = atom.ast->as<ASTFunction>();
if (func && func->arguments->children.size() == 2)
{
const auto left = weak_graph->getComponentId(func->arguments->children[0]);
const auto right = weak_graph->getComponentId(func->arguments->children[1]);
if (left && right && weak_graph->hasPath(left.value(), right.value()))
{
useless = false;
return;
}
}
}
});
return useless;
}
bool MergeTreeIndexMergedCondition::mayBeTrueOnGranule(const MergeTreeIndexGranules & granules) const
{
std::vector<bool> values;
@ -141,20 +189,36 @@ bool MergeTreeIndexMergedCondition::mayBeTrueOnGranule(const MergeTreeIndexGranu
for (auto atom : or_group)
{
pushNotIn(atom);
Poco::Logger::get("KEK").information(atom.ast->dumpTree());
const auto * func = atom.ast->as<ASTFunction>();
if (func && func->arguments->children.size() == 2)
{
const auto expected = getExpectedCompare(atom);
if (graph.isPossibleCompare(
expected,
func->arguments->children[0],
func->arguments->children[1]))
{
return;
}
}
pushNotIn(atom);
Poco::Logger::get("KEK").information(atom.ast->dumpTree());
const auto * func = atom.ast->as<ASTFunction>();
if (func && func->arguments->children.size() == 2)
{
const auto expected = getExpectedCompare(atom);
if (graph.isPossibleCompare(expected, func->arguments->children[0], func->arguments->children[1]))
{
/// If graph failed use matching.
/// We don't need to check constraints.
return;
}
}
}
for (auto atom : or_group)
{
pushNotIn(atom);
for (size_t i = 0; i < values.size(); ++i)
if (values[i])
for (const auto & hypothesis_or_group : index_to_atomic_hypotheses[i])
{
if (hypothesis_or_group.size() == 1)
{
const auto & hypothesis_atom = *std::begin(hypothesis_or_group);
if (atom.ast->getTreeHash() == hypothesis_atom.ast->getTreeHash())
{
return;
}
}
}
}
always_false = true;
});

View File

@ -22,7 +22,7 @@ public:
void addIndex(const MergeTreeIndexPtr & index);
void addConstraints(const ConstraintsDescription & constraints_description);
bool alwaysUnknownOrTrue() const { return false; } // TODO: replace < -> <=, > -> >= and assume all hypotheses are true + check path exists
bool alwaysUnknownOrTrue() const;
bool mayBeTrueOnGranule(const MergeTreeIndexGranules & granules) const;
//TODO: add constraints

View File

@ -1 +1,8 @@
2
"rows_read": 4,
"rows_read": 2,
"rows_read": 4,
"rows_read": 2,
"rows_read": 2,
"rows_read": 2,
"rows_read": 4,
"rows_read": 2,

View File

@ -0,0 +1,49 @@
#!/usr/bin/env bash
set -e
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
SETTINGS="SET convert_query_to_cnf = 1; SET optimize_using_constraints = 1; SET optimize_move_to_prewhere = 1;"
$CLICKHOUSE_CLIENT -n --query="
DROP DATABASE IF EXISTS constraint_test;
DROP TABLE IF EXISTS constraint_test.test;
CREATE DATABASE constraint_test;
CREATE TABLE constraint_test.test (
i UInt64,
a UInt64,
b UInt64,
c Float64,
INDEX t (a < b) TYPE hypothesis GRANULARITY 1,
INDEX t2 (b <= c) TYPE hypothesis GRANULARITY 1
) ENGINE = MergeTree() ORDER BY i SETTINGS index_granularity=1;
"
$CLICKHOUSE_CLIENT --query="INSERT INTO constraint_test.test VALUES
(0, 1, 2, 2),
(1, 2, 1, 2),
(2, 2, 2, 1),
(3, 1, 2, 3)"
$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE b > a FORMAT JSON" | grep "rows_read" # 4
$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE b <= a FORMAT JSON" | grep "rows_read"
$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE b >= a FORMAT JSON" | grep "rows_read" # 4
$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE b = a FORMAT JSON" | grep "rows_read"
$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE c < a FORMAT JSON" | grep "rows_read"
$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE c = a FORMAT JSON" | grep "rows_read"
$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE c > a FORMAT JSON" | grep "rows_read" # 4
$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE c < a FORMAT JSON" | grep "rows_read"
$CLICKHOUSE_CLIENT -n --query="
DROP TABLE constraint_test.test;
DROP DATABASE constraint_test;"

View File

@ -1,17 +0,0 @@
SET convert_query_to_cnf = 1;
SET optimize_using_constraints = 1;
SET optimize_move_to_prewhere = 1;
DROP DATABASE IF EXISTS constraint_test;
DROP TABLE IF EXISTS constraint_test.test;
CREATE DATABASE constraint_test;
CREATE TABLE constraint_test.test (i UInt64, a UInt64, b UInt64, INDEX t (a = b) TYPE hypothesis GRANULARITY 1) ENGINE = MergeTree() ORDER BY i SETTINGS index_granularity=1;
INSERT INTO constraint_test.test VALUES (1, 1, 1), (2, 1, 2), (3, 2, 1), (4, 2, 2);
SELECT count() FROM constraint_test.test WHERE a = b;
DROP TABLE constraint_test.test;
DROP DATABASE constraint_test;