diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index 8a4d9f0e0be..37f9b99a15e 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -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 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 ComparisonGraph::getComponent(const std::size_t id) const { return graph.vertexes[id].asts; diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index 9fee991c399..5450a69bd4b 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -41,6 +41,7 @@ public: std::optional getComponentId(const ASTPtr & ast) const; std::vector 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. diff --git a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp index fd4640000aa..5f9c2807469 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp @@ -55,10 +55,11 @@ void MergeTreeIndexMergedCondition::addIndex(const MergeTreeIndexPtr & index) std::vector compare_hypotheses_data; std::vector 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 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(); + if (func && func->name == "less") + func->name = "lessOrEquals"; + if (func && func->name == "greater") + func->name = "greaterOrEquals"; + } + + const auto weak_graph = std::make_unique(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(); + 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 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(); - 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(); + 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; }); diff --git a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.h b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.h index 3048a9d6bdc..204e6f01ea2 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.h +++ b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.h @@ -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 diff --git a/tests/queries/0_stateless/01624_soft_constraints.reference b/tests/queries/0_stateless/01624_soft_constraints.reference index 0cfbf08886f..f0a866816d6 100644 --- a/tests/queries/0_stateless/01624_soft_constraints.reference +++ b/tests/queries/0_stateless/01624_soft_constraints.reference @@ -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, diff --git a/tests/queries/0_stateless/01624_soft_constraints.sh b/tests/queries/0_stateless/01624_soft_constraints.sh new file mode 100755 index 00000000000..30f77446b49 --- /dev/null +++ b/tests/queries/0_stateless/01624_soft_constraints.sh @@ -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;" diff --git a/tests/queries/0_stateless/01624_soft_constraints.sql b/tests/queries/0_stateless/01624_soft_constraints.sql deleted file mode 100644 index 53a2d4ac9f6..00000000000 --- a/tests/queries/0_stateless/01624_soft_constraints.sql +++ /dev/null @@ -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; \ No newline at end of file