mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge branch 'master' into fix-write-buffers
This commit is contained in:
commit
a9c1fd1647
1384
.github/workflows/master.yml
vendored
Normal file
1384
.github/workflows/master.yml
vendored
Normal file
File diff suppressed because it is too large
Load Diff
@ -423,6 +423,7 @@ class IColumn;
|
||||
M(Bool, optimize_move_functions_out_of_any, false, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \
|
||||
M(Bool, optimize_normalize_count_variants, true, "Rewrite aggregate functions that semantically equals to count() as count().", 0) \
|
||||
M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \
|
||||
M(Bool, convert_query_to_cnf, false, "Convert SELECT query to CNF", 0) \
|
||||
M(Bool, optimize_arithmetic_operations_in_aggregate_functions, true, "Move arithmetic operations out of aggregation functions", 0) \
|
||||
M(Bool, optimize_duplicate_order_by_and_distinct, true, "Remove duplicate ORDER BY and DISTINCT if it's possible", 0) \
|
||||
M(Bool, optimize_redundant_functions_in_order_by, true, "Remove functions from ORDER BY if its argument is also in ORDER BY", 0) \
|
||||
@ -430,6 +431,9 @@ class IColumn;
|
||||
M(Bool, optimize_if_transform_strings_to_enum, false, "Replaces string-type arguments in If and Transform to enum. Disabled by default cause it could make inconsistent change in distributed query that would lead to its fail.", 0) \
|
||||
M(Bool, optimize_monotonous_functions_in_order_by, true, "Replace monotonous function with its argument in ORDER BY", 0) \
|
||||
M(Bool, optimize_functions_to_subcolumns, false, "Transform functions to subcolumns, if possible, to reduce amount of read data. E.g. 'length(arr)' -> 'arr.size0', 'col IS NULL' -> 'col.null' ", 0) \
|
||||
M(Bool, optimize_using_constraints, false, "Use constraints for query optimization", 0) \
|
||||
M(Bool, optimize_substitute_columns, false, "Use constraints for column substitution", 0) \
|
||||
M(Bool, optimize_append_index, false, "Use constraints in order to append index condition (indexHint)", 0) \
|
||||
M(Bool, normalize_function_names, true, "Normalize function names to their canonical names", 0) \
|
||||
M(Bool, allow_experimental_alter_materialized_view_structure, false, "Allow atomic alter on Materialized views. Work in progress.", 0) \
|
||||
M(Bool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there", 0) \
|
||||
|
181
src/Interpreters/AddIndexConstraintsOptimizer.cpp
Normal file
181
src/Interpreters/AddIndexConstraintsOptimizer.cpp
Normal file
@ -0,0 +1,181 @@
|
||||
#include <Interpreters/AddIndexConstraintsOptimizer.h>
|
||||
|
||||
#include <Interpreters/TreeCNFConverter.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTConstraintDeclaration.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
AddIndexConstraintsOptimizer::AddIndexConstraintsOptimizer(
|
||||
const StorageMetadataPtr & metadata_snapshot_)
|
||||
: metadata_snapshot(metadata_snapshot_)
|
||||
{
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
bool onlyIndexColumns(const ASTPtr & ast, const std::unordered_set<std::string_view> & primary_key_set)
|
||||
{
|
||||
const auto * identifier = ast->as<ASTIdentifier>();
|
||||
if (identifier && !primary_key_set.contains(identifier->name()))
|
||||
return false;
|
||||
for (auto & child : ast->children)
|
||||
if (!onlyIndexColumns(child, primary_key_set))
|
||||
return false;
|
||||
return true;
|
||||
}
|
||||
|
||||
bool onlyConstants(const ASTPtr & ast)
|
||||
{
|
||||
const auto * identifier = ast->as<ASTIdentifier>();
|
||||
if (identifier)
|
||||
return false;
|
||||
for (auto & child : ast->children)
|
||||
if (!onlyConstants(child))
|
||||
return false;
|
||||
return true;
|
||||
}
|
||||
|
||||
const std::unordered_map<std::string, ComparisonGraph::CompareResult> & getRelationMap()
|
||||
{
|
||||
const static std::unordered_map<std::string, ComparisonGraph::CompareResult> relations =
|
||||
{
|
||||
{"equals", ComparisonGraph::CompareResult::EQUAL},
|
||||
{"less", ComparisonGraph::CompareResult::LESS},
|
||||
{"lessOrEquals", ComparisonGraph::CompareResult::LESS_OR_EQUAL},
|
||||
{"greaterOrEquals", ComparisonGraph::CompareResult::GREATER_OR_EQUAL},
|
||||
{"greater", ComparisonGraph::CompareResult::GREATER},
|
||||
};
|
||||
return relations;
|
||||
}
|
||||
|
||||
const std::unordered_map<ComparisonGraph::CompareResult, std::string> & getReverseRelationMap()
|
||||
{
|
||||
const static std::unordered_map<ComparisonGraph::CompareResult, std::string> relations =
|
||||
{
|
||||
{ComparisonGraph::CompareResult::EQUAL, "equals"},
|
||||
{ComparisonGraph::CompareResult::LESS, "less"},
|
||||
{ComparisonGraph::CompareResult::LESS_OR_EQUAL, "lessOrEquals"},
|
||||
{ComparisonGraph::CompareResult::GREATER_OR_EQUAL, "greaterOrEquals"},
|
||||
{ComparisonGraph::CompareResult::GREATER, "greater"},
|
||||
};
|
||||
return relations;
|
||||
}
|
||||
|
||||
bool canBeSequence(const ComparisonGraph::CompareResult left, const ComparisonGraph::CompareResult right)
|
||||
{
|
||||
using CR = ComparisonGraph::CompareResult;
|
||||
if (left == CR::UNKNOWN || right == CR::UNKNOWN || left == CR::NOT_EQUAL || right == CR::NOT_EQUAL)
|
||||
return false;
|
||||
if ((left == CR::GREATER || left == CR::GREATER_OR_EQUAL) && (right == CR::LESS || right == CR::LESS_OR_EQUAL))
|
||||
return false;
|
||||
if ((right == CR::GREATER || right == CR::GREATER_OR_EQUAL) && (left == CR::LESS || left == CR::LESS_OR_EQUAL))
|
||||
return false;
|
||||
return true;
|
||||
}
|
||||
|
||||
ComparisonGraph::CompareResult mostStrict(const ComparisonGraph::CompareResult left, const ComparisonGraph::CompareResult right)
|
||||
{
|
||||
using CR = ComparisonGraph::CompareResult;
|
||||
if (left == CR::LESS || left == CR::GREATER)
|
||||
return left;
|
||||
if (right == CR::LESS || right == CR::GREATER)
|
||||
return right;
|
||||
if (left == CR::LESS_OR_EQUAL || left == CR::GREATER_OR_EQUAL)
|
||||
return left;
|
||||
if (right == CR::LESS_OR_EQUAL || right == CR::GREATER_OR_EQUAL)
|
||||
return right;
|
||||
if (left == CR::EQUAL)
|
||||
return left;
|
||||
if (right == CR::EQUAL)
|
||||
return right;
|
||||
return CR::UNKNOWN;
|
||||
}
|
||||
|
||||
/// Create OR-group for 'indexHint'.
|
||||
/// Consider we have expression like A <op1> C, where C is constant.
|
||||
/// Consider we have a constraint I <op2> A, where I depends only on columns from primary key.
|
||||
/// Then if op1 and op2 forms a sequence of comparisons (e.g. A < C and I < A),
|
||||
/// we can add to expression 'indexHint(I < A)' condition.
|
||||
CNFQuery::OrGroup createIndexHintGroup(
|
||||
const CNFQuery::OrGroup & group,
|
||||
const ComparisonGraph & graph,
|
||||
const ASTs & primary_key_only_asts)
|
||||
{
|
||||
CNFQuery::OrGroup result;
|
||||
for (const auto & atom : group)
|
||||
{
|
||||
const auto * func = atom.ast->as<ASTFunction>();
|
||||
if (func && func->arguments->children.size() == 2 && getRelationMap().contains(func->name))
|
||||
{
|
||||
auto check_and_insert = [&](const size_t index, const ComparisonGraph::CompareResult need_result)
|
||||
{
|
||||
if (!onlyConstants(func->arguments->children[1 - index]))
|
||||
return false;
|
||||
|
||||
for (const auto & primary_key_ast : primary_key_only_asts)
|
||||
{
|
||||
ComparisonGraph::CompareResult actual_result;
|
||||
if (index == 0)
|
||||
actual_result = graph.compare(primary_key_ast, func->arguments->children[index]);
|
||||
else
|
||||
actual_result = graph.compare(func->arguments->children[index], primary_key_ast);
|
||||
|
||||
if (canBeSequence(need_result, actual_result))
|
||||
{
|
||||
ASTPtr helper_ast = func->clone();
|
||||
auto * helper_func = helper_ast->as<ASTFunction>();
|
||||
helper_func->name = getReverseRelationMap().at(mostStrict(need_result, actual_result));
|
||||
helper_func->arguments->children[index] = primary_key_ast->clone();
|
||||
result.insert(CNFQuery::AtomicFormula{atom.negative, helper_ast});
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
};
|
||||
|
||||
auto expected = getRelationMap().at(func->name);
|
||||
if (!check_and_insert(0, expected) && !check_and_insert(1, expected))
|
||||
return {};
|
||||
}
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
||||
void AddIndexConstraintsOptimizer::perform(CNFQuery & cnf_query)
|
||||
{
|
||||
const auto primary_key = metadata_snapshot->getColumnsRequiredForPrimaryKey();
|
||||
const auto & graph = metadata_snapshot->getConstraints().getGraph();
|
||||
const std::unordered_set<std::string_view> primary_key_set(std::begin(primary_key), std::end(primary_key));
|
||||
|
||||
ASTs primary_key_only_asts;
|
||||
for (const auto & vertex : graph.getVertices())
|
||||
for (const auto & ast : vertex)
|
||||
if (onlyIndexColumns(ast, primary_key_set))
|
||||
primary_key_only_asts.push_back(ast);
|
||||
|
||||
CNFQuery::AndGroup and_group;
|
||||
cnf_query.iterateGroups([&](const auto & or_group)
|
||||
{
|
||||
auto add_group = createIndexHintGroup(or_group, graph, primary_key_only_asts);
|
||||
if (!add_group.empty())
|
||||
and_group.emplace(std::move(add_group));
|
||||
});
|
||||
|
||||
if (!and_group.empty())
|
||||
{
|
||||
CNFQuery::OrGroup new_or_group;
|
||||
new_or_group.insert(CNFQuery::AtomicFormula{false, makeASTFunction("indexHint", TreeCNFConverter::fromCNF(CNFQuery(std::move(and_group))))});
|
||||
cnf_query.appendGroup(CNFQuery::AndGroup{new_or_group});
|
||||
}
|
||||
}
|
||||
|
||||
}
|
35
src/Interpreters/AddIndexConstraintsOptimizer.h
Normal file
35
src/Interpreters/AddIndexConstraintsOptimizer.h
Normal file
@ -0,0 +1,35 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Interpreters/Aliases.h>
|
||||
#include <Interpreters/SelectQueryOptions.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Interpreters/TreeCNFConverter.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct StorageInMemoryMetadata;
|
||||
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
|
||||
|
||||
/// Optimizer that extracts constraints that
|
||||
/// depends only on columns of primary key
|
||||
/// and tries to add function 'indexHint' to
|
||||
/// WHERE clause, which reduces amount of read data.
|
||||
class AddIndexConstraintsOptimizer final
|
||||
{
|
||||
public:
|
||||
AddIndexConstraintsOptimizer(
|
||||
const StorageMetadataPtr & metadata_snapshot);
|
||||
|
||||
void perform(CNFQuery & cnf_query);
|
||||
|
||||
private:
|
||||
const StorageMetadataPtr & metadata_snapshot;
|
||||
};
|
||||
|
||||
}
|
640
src/Interpreters/ComparisonGraph.cpp
Normal file
640
src/Interpreters/ComparisonGraph.cpp
Normal file
@ -0,0 +1,640 @@
|
||||
#include <Interpreters/ComparisonGraph.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int VIOLATED_CONSTRAINT;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/// Make function a > b or a >= b
|
||||
ASTPtr normalizeAtom(const ASTPtr & atom)
|
||||
{
|
||||
static const std::map<std::string, std::string> inverse_relations =
|
||||
{
|
||||
{"lessOrEquals", "greaterOrEquals"},
|
||||
{"less", "greater"},
|
||||
};
|
||||
|
||||
ASTPtr res = atom->clone();
|
||||
if (const auto * func = res->as<ASTFunction>())
|
||||
{
|
||||
if (const auto it = inverse_relations.find(func->name); it != std::end(inverse_relations))
|
||||
{
|
||||
res = makeASTFunction(it->second, func->arguments->children[1]->clone(), func->arguments->children[0]->clone());
|
||||
}
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
bool less(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateLess{}, lhs, rhs); }
|
||||
bool greater(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateLess{}, rhs, lhs); }
|
||||
bool equals(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateEquals{}, lhs, rhs); }
|
||||
|
||||
}
|
||||
|
||||
ComparisonGraph::ComparisonGraph(const ASTs & atomic_formulas)
|
||||
{
|
||||
if (atomic_formulas.empty())
|
||||
return;
|
||||
|
||||
static const std::unordered_map<std::string, Edge::Type> relation_to_enum =
|
||||
{
|
||||
{"equals", Edge::EQUAL},
|
||||
{"greater", Edge::GREATER},
|
||||
{"greaterOrEquals", Edge::GREATER_OR_EQUAL},
|
||||
};
|
||||
|
||||
/// Firstly build an intermediate graph,
|
||||
/// in which each vertex corresponds to one expression.
|
||||
/// That means that if we have edge (A, B) with type GREATER, then always A > B.
|
||||
/// If we have EQUAL relation, then we add both edges (A, B) and (B, A).
|
||||
|
||||
Graph g;
|
||||
for (const auto & atom_raw : atomic_formulas)
|
||||
{
|
||||
const auto atom = normalizeAtom(atom_raw);
|
||||
|
||||
auto get_index = [](const ASTPtr & ast, Graph & asts_graph) -> std::optional<size_t>
|
||||
{
|
||||
const auto it = asts_graph.ast_hash_to_component.find(ast->getTreeHash());
|
||||
if (it != std::end(asts_graph.ast_hash_to_component))
|
||||
{
|
||||
if (!std::any_of(
|
||||
std::cbegin(asts_graph.vertices[it->second].asts),
|
||||
std::cend(asts_graph.vertices[it->second].asts),
|
||||
[ast](const ASTPtr & constraint_ast)
|
||||
{
|
||||
return constraint_ast->getTreeHash() == ast->getTreeHash()
|
||||
&& constraint_ast->getColumnName() == ast->getColumnName();
|
||||
}))
|
||||
{
|
||||
return {};
|
||||
}
|
||||
|
||||
return it->second;
|
||||
}
|
||||
else
|
||||
{
|
||||
asts_graph.ast_hash_to_component[ast->getTreeHash()] = asts_graph.vertices.size();
|
||||
asts_graph.vertices.push_back(EqualComponent{{ast}, std::nullopt});
|
||||
asts_graph.edges.emplace_back();
|
||||
return asts_graph.vertices.size() - 1;
|
||||
}
|
||||
};
|
||||
|
||||
const auto * func = atom->as<ASTFunction>();
|
||||
if (func && func->arguments->children.size() == 2)
|
||||
{
|
||||
auto index_left = get_index(func->arguments->children[0], g);
|
||||
auto index_right = get_index(func->arguments->children[1], g);
|
||||
|
||||
if (index_left && index_right)
|
||||
{
|
||||
if (const auto it = relation_to_enum.find(func->name); it != std::end(relation_to_enum))
|
||||
{
|
||||
g.edges[*index_left].push_back(Edge{it->second, *index_right});
|
||||
if (it->second == Edge::EQUAL)
|
||||
g.edges[*index_right].push_back(Edge{it->second, *index_left});
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Now expressions A and B are equal, if and only if
|
||||
/// we have both paths from A to B and from B to A in graph.
|
||||
/// That means that equivalence classes of expressions
|
||||
/// are the same as strongly connected components in graph.
|
||||
/// So, we find such components and build graph on them.
|
||||
/// All expressions from one equivalence class will be stored
|
||||
/// in the corresponding vertex of new graph.
|
||||
|
||||
graph = buildGraphFromAstsGraph(g);
|
||||
dists = buildDistsFromGraph(graph);
|
||||
std::tie(ast_const_lower_bound, ast_const_upper_bound) = buildConstBounds();
|
||||
|
||||
/// Find expressions that are known to be unequal.
|
||||
static const std::unordered_set<String> not_equals_functions = {"notEquals", "greater"};
|
||||
|
||||
/// Explicitly save unequal components.
|
||||
/// TODO: Build a graph for unequal components.
|
||||
for (const auto & atom_raw : atomic_formulas)
|
||||
{
|
||||
const auto atom = normalizeAtom(atom_raw);
|
||||
const auto * func = atom->as<ASTFunction>();
|
||||
|
||||
if (func && not_equals_functions.contains(func->name))
|
||||
{
|
||||
auto index_left = graph.ast_hash_to_component.at(func->arguments->children[0]->getTreeHash());
|
||||
auto index_right = graph.ast_hash_to_component.at(func->arguments->children[1]->getTreeHash());
|
||||
|
||||
if (index_left == index_right)
|
||||
throw Exception(ErrorCodes::VIOLATED_CONSTRAINT,
|
||||
"Found expression '{}', but its arguments considered equal according to constraints",
|
||||
queryToString(atom));
|
||||
|
||||
not_equal.emplace(index_left, index_right);
|
||||
not_equal.emplace(index_right, index_left);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ComparisonGraph::CompareResult ComparisonGraph::pathToCompareResult(Path path, bool inverse)
|
||||
{
|
||||
switch (path)
|
||||
{
|
||||
case Path::GREATER: return inverse ? CompareResult::LESS : CompareResult::GREATER;
|
||||
case Path::GREATER_OR_EQUAL: return inverse ? CompareResult::LESS_OR_EQUAL : CompareResult::GREATER_OR_EQUAL;
|
||||
}
|
||||
__builtin_unreachable();
|
||||
}
|
||||
|
||||
std::optional<ComparisonGraph::Path> ComparisonGraph::findPath(const size_t start, const size_t finish) const
|
||||
{
|
||||
const auto it = dists.find(std::make_pair(start, finish));
|
||||
if (it == std::end(dists))
|
||||
return {};
|
||||
|
||||
/// Since path can be only GREATER or GREATER_OR_EQUALS,
|
||||
/// we can strengthen the condition.
|
||||
return not_equal.contains({start, finish}) ? Path::GREATER : it->second;
|
||||
}
|
||||
|
||||
ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, const ASTPtr & right) const
|
||||
{
|
||||
size_t start = 0;
|
||||
size_t finish = 0;
|
||||
|
||||
/// TODO: check full ast
|
||||
const auto it_left = graph.ast_hash_to_component.find(left->getTreeHash());
|
||||
const auto it_right = graph.ast_hash_to_component.find(right->getTreeHash());
|
||||
|
||||
if (it_left == std::end(graph.ast_hash_to_component) || it_right == std::end(graph.ast_hash_to_component))
|
||||
{
|
||||
CompareResult result = CompareResult::UNKNOWN;
|
||||
{
|
||||
const auto left_bound = getConstLowerBound(left);
|
||||
const auto right_bound = getConstUpperBound(right);
|
||||
|
||||
if (left_bound && right_bound)
|
||||
{
|
||||
if (greater(left_bound->first, right_bound->first))
|
||||
result = CompareResult::GREATER;
|
||||
else if (equals(left_bound->first, right_bound->first))
|
||||
result = left_bound->second || right_bound->second
|
||||
? CompareResult::GREATER : CompareResult::GREATER_OR_EQUAL;
|
||||
}
|
||||
}
|
||||
{
|
||||
const auto left_bound = getConstUpperBound(left);
|
||||
const auto right_bound = getConstLowerBound(right);
|
||||
|
||||
if (left_bound && right_bound)
|
||||
{
|
||||
if (less(left_bound->first, right_bound->first))
|
||||
result = CompareResult::LESS;
|
||||
else if (equals(left_bound->first, right_bound->first))
|
||||
result = left_bound->second || right_bound->second
|
||||
? CompareResult::LESS : CompareResult::LESS_OR_EQUAL;
|
||||
}
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
else
|
||||
{
|
||||
start = it_left->second;
|
||||
finish = it_right->second;
|
||||
}
|
||||
|
||||
if (start == finish)
|
||||
return CompareResult::EQUAL;
|
||||
|
||||
if (auto path = findPath(start, finish))
|
||||
return pathToCompareResult(*path, /*inverse=*/ false);
|
||||
|
||||
if (auto path = findPath(finish, start))
|
||||
return pathToCompareResult(*path, /*inverse=*/ true);
|
||||
|
||||
if (not_equal.contains({start, finish}))
|
||||
return CompareResult::NOT_EQUAL;
|
||||
|
||||
return CompareResult::UNKNOWN;
|
||||
}
|
||||
|
||||
bool ComparisonGraph::isPossibleCompare(const CompareResult expected, const ASTPtr & left, const ASTPtr & right) const
|
||||
{
|
||||
const auto result = compare(left, right);
|
||||
|
||||
if (expected == CompareResult::UNKNOWN || result == CompareResult::UNKNOWN)
|
||||
return true;
|
||||
|
||||
if (expected == result)
|
||||
return true;
|
||||
|
||||
static const std::set<std::pair<CompareResult, CompareResult>> possible_pairs =
|
||||
{
|
||||
{CompareResult::EQUAL, CompareResult::LESS_OR_EQUAL},
|
||||
{CompareResult::EQUAL, CompareResult::GREATER_OR_EQUAL},
|
||||
{CompareResult::LESS_OR_EQUAL, CompareResult::LESS},
|
||||
{CompareResult::LESS_OR_EQUAL, CompareResult::EQUAL},
|
||||
{CompareResult::LESS_OR_EQUAL, CompareResult::NOT_EQUAL},
|
||||
{CompareResult::GREATER_OR_EQUAL, CompareResult::GREATER},
|
||||
{CompareResult::GREATER_OR_EQUAL, CompareResult::EQUAL},
|
||||
{CompareResult::GREATER_OR_EQUAL, CompareResult::NOT_EQUAL},
|
||||
{CompareResult::LESS, CompareResult::LESS},
|
||||
{CompareResult::LESS, CompareResult::LESS_OR_EQUAL},
|
||||
{CompareResult::LESS, CompareResult::NOT_EQUAL},
|
||||
{CompareResult::GREATER, CompareResult::GREATER},
|
||||
{CompareResult::GREATER, CompareResult::GREATER_OR_EQUAL},
|
||||
{CompareResult::GREATER, CompareResult::NOT_EQUAL},
|
||||
{CompareResult::NOT_EQUAL, CompareResult::LESS},
|
||||
{CompareResult::NOT_EQUAL, CompareResult::GREATER},
|
||||
{CompareResult::NOT_EQUAL, CompareResult::LESS_OR_EQUAL},
|
||||
{CompareResult::NOT_EQUAL, CompareResult::GREATER_OR_EQUAL},
|
||||
};
|
||||
|
||||
return possible_pairs.contains({expected, result});
|
||||
}
|
||||
|
||||
bool ComparisonGraph::isAlwaysCompare(const CompareResult expected, const ASTPtr & left, const ASTPtr & right) const
|
||||
{
|
||||
const auto result = compare(left, right);
|
||||
|
||||
if (expected == CompareResult::UNKNOWN || result == CompareResult::UNKNOWN)
|
||||
return false;
|
||||
|
||||
if (expected == result)
|
||||
return true;
|
||||
|
||||
static const std::set<std::pair<CompareResult, CompareResult>> possible_pairs =
|
||||
{
|
||||
{CompareResult::LESS_OR_EQUAL, CompareResult::LESS},
|
||||
{CompareResult::LESS_OR_EQUAL, CompareResult::EQUAL},
|
||||
{CompareResult::GREATER_OR_EQUAL, CompareResult::GREATER},
|
||||
{CompareResult::GREATER_OR_EQUAL, CompareResult::EQUAL},
|
||||
{CompareResult::NOT_EQUAL, CompareResult::GREATER},
|
||||
{CompareResult::NOT_EQUAL, CompareResult::LESS},
|
||||
};
|
||||
|
||||
return possible_pairs.contains({expected, result});
|
||||
}
|
||||
|
||||
|
||||
ASTs ComparisonGraph::getEqual(const ASTPtr & ast) const
|
||||
{
|
||||
const auto res = getComponentId(ast);
|
||||
if (!res)
|
||||
return {};
|
||||
else
|
||||
return getComponent(res.value());
|
||||
}
|
||||
|
||||
std::optional<size_t> ComparisonGraph::getComponentId(const ASTPtr & ast) const
|
||||
{
|
||||
const auto hash_it = graph.ast_hash_to_component.find(ast->getTreeHash());
|
||||
if (hash_it == std::end(graph.ast_hash_to_component))
|
||||
return {};
|
||||
|
||||
const size_t index = hash_it->second;
|
||||
if (std::any_of(
|
||||
std::cbegin(graph.vertices[index].asts),
|
||||
std::cend(graph.vertices[index].asts),
|
||||
[ast](const ASTPtr & constraint_ast)
|
||||
{
|
||||
return constraint_ast->getTreeHash() == ast->getTreeHash() &&
|
||||
constraint_ast->getColumnName() == ast->getColumnName();
|
||||
}))
|
||||
{
|
||||
return index;
|
||||
}
|
||||
else
|
||||
{
|
||||
return {};
|
||||
}
|
||||
}
|
||||
|
||||
bool ComparisonGraph::hasPath(const size_t left, const size_t right) const
|
||||
{
|
||||
return findPath(left, right) || findPath(right, left);
|
||||
}
|
||||
|
||||
ASTs ComparisonGraph::getComponent(const size_t id) const
|
||||
{
|
||||
return graph.vertices[id].asts;
|
||||
}
|
||||
|
||||
bool ComparisonGraph::EqualComponent::hasConstant() const
|
||||
{
|
||||
return constant_index.has_value();
|
||||
}
|
||||
|
||||
ASTPtr ComparisonGraph::EqualComponent::getConstant() const
|
||||
{
|
||||
assert(constant_index);
|
||||
return asts[*constant_index];
|
||||
}
|
||||
|
||||
void ComparisonGraph::EqualComponent::buildConstants()
|
||||
{
|
||||
constant_index.reset();
|
||||
for (size_t i = 0; i < asts.size(); ++i)
|
||||
{
|
||||
if (asts[i]->as<ASTLiteral>())
|
||||
{
|
||||
constant_index = i;
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ComparisonGraph::CompareResult ComparisonGraph::atomToCompareResult(const CNFQuery::AtomicFormula & atom)
|
||||
{
|
||||
if (const auto * func = atom.ast->as<ASTFunction>())
|
||||
{
|
||||
auto expected = functionNameToCompareResult(func->name);
|
||||
if (atom.negative)
|
||||
expected = inverseCompareResult(expected);
|
||||
return expected;
|
||||
}
|
||||
|
||||
return ComparisonGraph::CompareResult::UNKNOWN;
|
||||
}
|
||||
|
||||
ComparisonGraph::CompareResult ComparisonGraph::functionNameToCompareResult(const std::string & name)
|
||||
{
|
||||
static const std::unordered_map<std::string, CompareResult> relation_to_compare =
|
||||
{
|
||||
{"equals", CompareResult::EQUAL},
|
||||
{"notEquals", CompareResult::NOT_EQUAL},
|
||||
{"less", CompareResult::LESS},
|
||||
{"lessOrEquals", CompareResult::LESS_OR_EQUAL},
|
||||
{"greaterOrEquals", CompareResult::GREATER_OR_EQUAL},
|
||||
{"greater", CompareResult::GREATER},
|
||||
};
|
||||
|
||||
const auto it = relation_to_compare.find(name);
|
||||
return it == std::end(relation_to_compare) ? CompareResult::UNKNOWN : it->second;
|
||||
}
|
||||
|
||||
ComparisonGraph::CompareResult ComparisonGraph::inverseCompareResult(const CompareResult result)
|
||||
{
|
||||
static const std::unordered_map<CompareResult, CompareResult> inverse_relations =
|
||||
{
|
||||
{CompareResult::NOT_EQUAL, CompareResult::EQUAL},
|
||||
{CompareResult::EQUAL, CompareResult::NOT_EQUAL},
|
||||
{CompareResult::GREATER_OR_EQUAL, CompareResult::LESS},
|
||||
{CompareResult::GREATER, CompareResult::LESS_OR_EQUAL},
|
||||
{CompareResult::LESS, CompareResult::GREATER_OR_EQUAL},
|
||||
{CompareResult::LESS_OR_EQUAL, CompareResult::GREATER},
|
||||
{CompareResult::UNKNOWN, CompareResult::UNKNOWN},
|
||||
};
|
||||
return inverse_relations.at(result);
|
||||
}
|
||||
|
||||
std::optional<ASTPtr> ComparisonGraph::getEqualConst(const ASTPtr & ast) const
|
||||
{
|
||||
const auto hash_it = graph.ast_hash_to_component.find(ast->getTreeHash());
|
||||
if (hash_it == std::end(graph.ast_hash_to_component))
|
||||
return std::nullopt;
|
||||
|
||||
const size_t index = hash_it->second;
|
||||
return graph.vertices[index].hasConstant()
|
||||
? std::optional<ASTPtr>{graph.vertices[index].getConstant()}
|
||||
: std::nullopt;
|
||||
}
|
||||
|
||||
std::optional<std::pair<Field, bool>> ComparisonGraph::getConstUpperBound(const ASTPtr & ast) const
|
||||
{
|
||||
if (const auto * literal = ast->as<ASTLiteral>())
|
||||
return std::make_pair(literal->value, false);
|
||||
|
||||
const auto it = graph.ast_hash_to_component.find(ast->getTreeHash());
|
||||
if (it == std::end(graph.ast_hash_to_component))
|
||||
return std::nullopt;
|
||||
|
||||
const size_t to = it->second;
|
||||
const ssize_t from = ast_const_upper_bound[to];
|
||||
if (from == -1)
|
||||
return std::nullopt;
|
||||
|
||||
return std::make_pair(graph.vertices[from].getConstant()->as<ASTLiteral>()->value, dists.at({from, to}) == Path::GREATER);
|
||||
}
|
||||
|
||||
std::optional<std::pair<Field, bool>> ComparisonGraph::getConstLowerBound(const ASTPtr & ast) const
|
||||
{
|
||||
if (const auto * literal = ast->as<ASTLiteral>())
|
||||
return std::make_pair(literal->value, false);
|
||||
|
||||
const auto it = graph.ast_hash_to_component.find(ast->getTreeHash());
|
||||
if (it == std::end(graph.ast_hash_to_component))
|
||||
return std::nullopt;
|
||||
|
||||
const size_t from = it->second;
|
||||
const ssize_t to = ast_const_lower_bound[from];
|
||||
if (to == -1)
|
||||
return std::nullopt;
|
||||
|
||||
return std::make_pair(graph.vertices[to].getConstant()->as<ASTLiteral>()->value, dists.at({from, to}) == Path::GREATER);
|
||||
}
|
||||
|
||||
void ComparisonGraph::dfsOrder(const Graph & asts_graph, size_t v, std::vector<bool> & visited, std::vector<size_t> & order)
|
||||
{
|
||||
visited[v] = true;
|
||||
for (const auto & edge : asts_graph.edges[v])
|
||||
if (!visited[edge.to])
|
||||
dfsOrder(asts_graph, edge.to, visited, order);
|
||||
|
||||
order.push_back(v);
|
||||
}
|
||||
|
||||
ComparisonGraph::Graph ComparisonGraph::reverseGraph(const Graph & asts_graph)
|
||||
{
|
||||
Graph g;
|
||||
g.ast_hash_to_component = asts_graph.ast_hash_to_component;
|
||||
g.vertices = asts_graph.vertices;
|
||||
g.edges.resize(g.vertices.size());
|
||||
|
||||
for (size_t v = 0; v < asts_graph.vertices.size(); ++v)
|
||||
for (const auto & edge : asts_graph.edges[v])
|
||||
g.edges[edge.to].push_back(Edge{edge.type, v});
|
||||
|
||||
return g;
|
||||
}
|
||||
|
||||
std::vector<ASTs> ComparisonGraph::getVertices() const
|
||||
{
|
||||
std::vector<ASTs> result;
|
||||
for (const auto & vertex : graph.vertices)
|
||||
{
|
||||
result.emplace_back();
|
||||
for (const auto & ast : vertex.asts)
|
||||
result.back().push_back(ast);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
void ComparisonGraph::dfsComponents(
|
||||
const Graph & reversed_graph, size_t v,
|
||||
OptionalIndices & components, const size_t component)
|
||||
{
|
||||
components[v] = component;
|
||||
for (const auto & edge : reversed_graph.edges[v])
|
||||
if (!components[edge.to])
|
||||
dfsComponents(reversed_graph, edge.to, components, component);
|
||||
}
|
||||
|
||||
ComparisonGraph::Graph ComparisonGraph::buildGraphFromAstsGraph(const Graph & asts_graph)
|
||||
{
|
||||
/// Find strongly connected component by using 2 dfs traversals.
|
||||
/// https://en.wikipedia.org/wiki/Kosaraju%27s_algorithm
|
||||
const auto n = asts_graph.vertices.size();
|
||||
|
||||
std::vector<size_t> order;
|
||||
{
|
||||
std::vector<bool> visited(n, false);
|
||||
for (size_t v = 0; v < n; ++v)
|
||||
{
|
||||
if (!visited[v])
|
||||
dfsOrder(asts_graph, v, visited, order);
|
||||
}
|
||||
}
|
||||
|
||||
OptionalIndices components(n);
|
||||
size_t component = 0;
|
||||
{
|
||||
const Graph reversed_graph = reverseGraph(asts_graph);
|
||||
for (auto it = order.rbegin(); it != order.rend(); ++it)
|
||||
{
|
||||
if (!components[*it])
|
||||
{
|
||||
dfsComponents(reversed_graph, *it, components, component);
|
||||
++component;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Graph result;
|
||||
result.vertices.resize(component);
|
||||
result.edges.resize(component);
|
||||
for (const auto & [hash, index] : asts_graph.ast_hash_to_component)
|
||||
{
|
||||
assert(components[index]);
|
||||
result.ast_hash_to_component[hash] = *components[index];
|
||||
result.vertices[*components[index]].asts.insert(
|
||||
std::end(result.vertices[*components[index]].asts),
|
||||
std::begin(asts_graph.vertices[index].asts),
|
||||
std::end(asts_graph.vertices[index].asts)); // asts_graph has only one ast per vertex
|
||||
}
|
||||
|
||||
/// Calculate constants
|
||||
for (auto & vertex : result.vertices)
|
||||
vertex.buildConstants();
|
||||
|
||||
/// For each edge in initial graph, we add an edge between components in condensation graph.
|
||||
for (size_t v = 0; v < n; ++v)
|
||||
{
|
||||
for (const auto & edge : asts_graph.edges[v])
|
||||
result.edges[*components[v]].push_back(Edge{edge.type, *components[edge.to]});
|
||||
|
||||
/// TODO: make edges unique (left most strict)
|
||||
}
|
||||
|
||||
/// If we have constansts in two components, we can compare them and add and extra edge.
|
||||
for (size_t v = 0; v < result.vertices.size(); ++v)
|
||||
{
|
||||
for (size_t u = 0; u < result.vertices.size(); ++u)
|
||||
{
|
||||
if (v != u && result.vertices[v].hasConstant() && result.vertices[u].hasConstant())
|
||||
{
|
||||
const auto * left = result.vertices[v].getConstant()->as<ASTLiteral>();
|
||||
const auto * right = result.vertices[u].getConstant()->as<ASTLiteral>();
|
||||
|
||||
/// Only GREATER. Equal constant fields = equal literals so it was already considered above.
|
||||
if (greater(left->value, right->value))
|
||||
result.edges[v].push_back(Edge{Edge::GREATER, u});
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
std::map<std::pair<size_t, size_t>, ComparisonGraph::Path> ComparisonGraph::buildDistsFromGraph(const Graph & g)
|
||||
{
|
||||
/// Min path : -1 means GREATER, 0 means GREATER_OR_EQUALS.
|
||||
/// We use Floyd–Warshall algorithm to find distances between all pairs of vertices.
|
||||
/// https://en.wikipedia.org/wiki/Floyd–Warshall_algorithm
|
||||
|
||||
constexpr auto inf = std::numeric_limits<Int8>::max();
|
||||
const size_t n = g.vertices.size();
|
||||
std::vector<std::vector<Int8>> results(n, std::vector<Int8>(n, inf));
|
||||
|
||||
for (size_t v = 0; v < n; ++v)
|
||||
{
|
||||
results[v][v] = 0;
|
||||
for (const auto & edge : g.edges[v])
|
||||
results[v][edge.to] = std::min(results[v][edge.to], static_cast<Int8>(edge.type == Edge::GREATER ? -1 : 0));
|
||||
}
|
||||
|
||||
for (size_t k = 0; k < n; ++k)
|
||||
for (size_t v = 0; v < n; ++v)
|
||||
for (size_t u = 0; u < n; ++u)
|
||||
if (results[v][k] != inf && results[k][u] != inf)
|
||||
results[v][u] = std::min(results[v][u], std::min(results[v][k], results[k][u]));
|
||||
|
||||
std::map<std::pair<size_t, size_t>, Path> path;
|
||||
for (size_t v = 0; v < n; ++v)
|
||||
for (size_t u = 0; u < n; ++u)
|
||||
if (results[v][u] != inf)
|
||||
path[std::make_pair(v, u)] = (results[v][u] == -1 ? Path::GREATER : Path::GREATER_OR_EQUAL);
|
||||
|
||||
return path;
|
||||
}
|
||||
|
||||
std::pair<std::vector<ssize_t>, std::vector<ssize_t>> ComparisonGraph::buildConstBounds() const
|
||||
{
|
||||
const size_t n = graph.vertices.size();
|
||||
std::vector<ssize_t> lower(n, -1);
|
||||
std::vector<ssize_t> upper(n, -1);
|
||||
|
||||
auto get_value = [this](const size_t vertex) -> Field
|
||||
{
|
||||
return graph.vertices[vertex].getConstant()->as<ASTLiteral>()->value;
|
||||
};
|
||||
|
||||
for (const auto & [edge, path] : dists)
|
||||
{
|
||||
const auto [from, to] = edge;
|
||||
|
||||
if (graph.vertices[to].hasConstant())
|
||||
{
|
||||
if (lower[from] == -1
|
||||
|| greater(get_value(to), get_value(lower[from]))
|
||||
|| (equals(get_value(to), get_value(lower[from])) && path == Path::GREATER))
|
||||
lower[from] = to;
|
||||
}
|
||||
|
||||
if (graph.vertices[from].hasConstant())
|
||||
{
|
||||
if (upper[to] == -1
|
||||
|| less(get_value(from), get_value(upper[to]))
|
||||
|| (equals(get_value(from), get_value(upper[to])) && path == Path::GREATER))
|
||||
upper[to] = from;
|
||||
}
|
||||
}
|
||||
|
||||
return {lower, upper};
|
||||
}
|
||||
|
||||
}
|
176
src/Interpreters/ComparisonGraph.h
Normal file
176
src/Interpreters/ComparisonGraph.h
Normal file
@ -0,0 +1,176 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Interpreters/TreeCNFConverter.h>
|
||||
#include <unordered_map>
|
||||
#include <map>
|
||||
#include <vector>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/*
|
||||
* Graph of relations between terms in constraints.
|
||||
* Allows to compare terms and get equal terms.
|
||||
*/
|
||||
class ComparisonGraph
|
||||
{
|
||||
public:
|
||||
/// atomic_formulas are extracted from constraints.
|
||||
ComparisonGraph(const std::vector<ASTPtr> & atomic_formulas);
|
||||
|
||||
enum class CompareResult
|
||||
{
|
||||
LESS,
|
||||
LESS_OR_EQUAL,
|
||||
EQUAL,
|
||||
GREATER_OR_EQUAL,
|
||||
GREATER,
|
||||
NOT_EQUAL,
|
||||
UNKNOWN,
|
||||
};
|
||||
|
||||
static CompareResult atomToCompareResult(const CNFQuery::AtomicFormula & atom);
|
||||
static CompareResult functionNameToCompareResult(const std::string & name);
|
||||
static CompareResult inverseCompareResult(const CompareResult result);
|
||||
|
||||
CompareResult compare(const ASTPtr & left, const ASTPtr & right) const;
|
||||
|
||||
/// It's possible that left <expected> right
|
||||
bool isPossibleCompare(const CompareResult expected, const ASTPtr & left, const ASTPtr & right) const;
|
||||
|
||||
/// It's always true that left <expected> right
|
||||
bool isAlwaysCompare(const CompareResult expected, const ASTPtr & left, const ASTPtr & right) const;
|
||||
|
||||
/// Returns all expressions from component to which @ast belongs if any.
|
||||
std::vector<ASTPtr> getEqual(const ASTPtr & ast) const;
|
||||
|
||||
/// Returns constant expression from component to which @ast belongs if any.
|
||||
std::optional<ASTPtr> getEqualConst(const ASTPtr & ast) const;
|
||||
|
||||
/// Finds component id to which @ast belongs if any.
|
||||
std::optional<std::size_t> getComponentId(const ASTPtr & ast) const;
|
||||
|
||||
/// Returns all expressions from component.
|
||||
std::vector<ASTPtr> getComponent(const std::size_t id) const;
|
||||
|
||||
size_t getNumOfComponents() const { return graph.vertices.size(); }
|
||||
|
||||
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.
|
||||
/// Returns: {constant, is strict less/greater}
|
||||
std::optional<std::pair<Field, bool>> getConstUpperBound(const ASTPtr & ast) const;
|
||||
std::optional<std::pair<Field, bool>> getConstLowerBound(const ASTPtr & ast) const;
|
||||
|
||||
/// Returns all expression in graph.
|
||||
std::vector<ASTs> getVertices() const;
|
||||
|
||||
private:
|
||||
/// Strongly connected component
|
||||
struct EqualComponent
|
||||
{
|
||||
/// All these expressions are considered as equal.
|
||||
std::vector<ASTPtr> asts;
|
||||
std::optional<size_t> constant_index;
|
||||
|
||||
bool hasConstant() const;
|
||||
ASTPtr getConstant() const;
|
||||
void buildConstants();
|
||||
};
|
||||
|
||||
/// Edge (from, to, type) means that it's always true that @from <op> @to,
|
||||
/// where @op is the operation of type @type.
|
||||
///
|
||||
/// TODO: move to diff for int and double:
|
||||
/// GREATER and GREATER_OR_EQUAL with +const or 0 --- ok
|
||||
/// with -const --- not ok
|
||||
/// EQUAL is ok only for 0
|
||||
struct Edge
|
||||
{
|
||||
enum Type
|
||||
{
|
||||
GREATER,
|
||||
GREATER_OR_EQUAL,
|
||||
EQUAL,
|
||||
};
|
||||
|
||||
Type type;
|
||||
size_t to;
|
||||
};
|
||||
|
||||
struct Graph
|
||||
{
|
||||
struct ASTHash
|
||||
{
|
||||
size_t operator() (const IAST::Hash & hash) const
|
||||
{
|
||||
return hash.first;
|
||||
}
|
||||
};
|
||||
|
||||
std::unordered_map<IAST::Hash, size_t, ASTHash> ast_hash_to_component;
|
||||
std::vector<EqualComponent> vertices;
|
||||
std::vector<std::vector<Edge>> edges;
|
||||
};
|
||||
|
||||
/// Receives graph, in which each vertex corresponds to one expression.
|
||||
/// Then finds strongly connected components and builds graph on them.
|
||||
static Graph buildGraphFromAstsGraph(const Graph & asts_graph);
|
||||
|
||||
static Graph reverseGraph(const Graph & asts_graph);
|
||||
|
||||
/// The first part of finding strongly connected components.
|
||||
/// Finds order of exit from vertices of dfs traversal of graph.
|
||||
static void dfsOrder(const Graph & asts_graph, size_t v, std::vector<bool> & visited, std::vector<size_t> & order);
|
||||
|
||||
using OptionalIndices = std::vector<std::optional<size_t>>;
|
||||
|
||||
/// The second part of finding strongly connected components.
|
||||
/// Assigns index of component for each vertex.
|
||||
static void dfsComponents(
|
||||
const Graph & reversed_graph, size_t v,
|
||||
OptionalIndices & components, const size_t component);
|
||||
|
||||
enum class Path
|
||||
{
|
||||
GREATER,
|
||||
GREATER_OR_EQUAL,
|
||||
};
|
||||
|
||||
static CompareResult pathToCompareResult(Path path, bool inverse);
|
||||
std::optional<Path> findPath(const size_t start, const size_t finish) const;
|
||||
|
||||
/// Calculate @dists.
|
||||
static std::map<std::pair<size_t, size_t>, Path> buildDistsFromGraph(const Graph & g);
|
||||
|
||||
/// Calculate @ast_const_lower_bound and @ast_const_lower_bound.
|
||||
std::pair<std::vector<ssize_t>, std::vector<ssize_t>> buildConstBounds() const;
|
||||
|
||||
/// Direct acyclic graph in which each vertex corresponds
|
||||
/// to one equivalence class of expressions.
|
||||
/// Each edge sets the relation between classes (GREATER or GREATER_OR_EQUAL).
|
||||
Graph graph;
|
||||
|
||||
/// Precalculated distances between each pair of vertices.
|
||||
/// Distance can be either 0 or -1.
|
||||
/// 0 means GREATER_OR_EQUAL.
|
||||
/// -1 means GREATER.
|
||||
std::map<std::pair<size_t, size_t>, Path> dists;
|
||||
|
||||
/// Explicitly collected components, for which it's known
|
||||
/// that expressions in them are unequal.
|
||||
std::set<std::pair<size_t, size_t>> not_equal;
|
||||
|
||||
/// Maximal constant value for each component that
|
||||
/// is lower bound for all expressions in component.
|
||||
std::vector<ssize_t> ast_const_lower_bound;
|
||||
|
||||
/// Minimal constant value for each component that
|
||||
/// is upper bound for all expressions in component.
|
||||
std::vector<ssize_t> ast_const_upper_bound;
|
||||
};
|
||||
|
||||
}
|
@ -416,7 +416,7 @@ ASTPtr InterpreterCreateQuery::formatConstraints(const ConstraintsDescription &
|
||||
{
|
||||
auto res = std::make_shared<ASTExpressionList>();
|
||||
|
||||
for (const auto & constraint : constraints.constraints)
|
||||
for (const auto & constraint : constraints.getConstraints())
|
||||
res->children.push_back(constraint->clone());
|
||||
|
||||
return res;
|
||||
@ -564,11 +564,12 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
|
||||
|
||||
ConstraintsDescription InterpreterCreateQuery::getConstraintsDescription(const ASTExpressionList * constraints)
|
||||
{
|
||||
ConstraintsDescription res;
|
||||
ASTs constraints_data;
|
||||
if (constraints)
|
||||
for (const auto & constraint : constraints->children)
|
||||
res.constraints.push_back(std::dynamic_pointer_cast<ASTConstraintDeclaration>(constraint->clone()));
|
||||
return res;
|
||||
constraints_data.push_back(constraint->clone());
|
||||
|
||||
return ConstraintsDescription{constraints_data};
|
||||
}
|
||||
|
||||
|
||||
|
331
src/Interpreters/SubstituteColumnOptimizer.cpp
Normal file
331
src/Interpreters/SubstituteColumnOptimizer.cpp
Normal file
@ -0,0 +1,331 @@
|
||||
#include <Interpreters/SubstituteColumnOptimizer.h>
|
||||
#include <Interpreters/TreeCNFConverter.h>
|
||||
#include <Interpreters/ComparisonGraph.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Storages/IStorage.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
constexpr auto COMPONENT_PART = "__component_";
|
||||
constexpr UInt64 COLUMN_PENALTY = 10 * 1024 * 1024;
|
||||
constexpr Int64 INDEX_PRICE = -1'000'000'000'000'000'000;
|
||||
|
||||
class ComponentMatcher
|
||||
{
|
||||
public:
|
||||
using Visitor = InDepthNodeVisitor<ComponentMatcher, true>;
|
||||
|
||||
struct Data
|
||||
{
|
||||
const ComparisonGraph & graph;
|
||||
std::set<UInt64> & components;
|
||||
std::unordered_map<String, String> & old_name;
|
||||
std::unordered_map<String, UInt64> & component;
|
||||
UInt64 & current_id;
|
||||
|
||||
Data(const ComparisonGraph & graph_,
|
||||
std::set<UInt64> & components_,
|
||||
std::unordered_map<String, String> & old_name_,
|
||||
std::unordered_map<String, UInt64> & component_,
|
||||
UInt64 & current_id_)
|
||||
: graph(graph_)
|
||||
, components(components_)
|
||||
, old_name(old_name_)
|
||||
, component(component_)
|
||||
, current_id(current_id_)
|
||||
{
|
||||
}
|
||||
};
|
||||
|
||||
static void visit(ASTPtr & ast, Data & data)
|
||||
{
|
||||
if (auto id = data.graph.getComponentId(ast))
|
||||
{
|
||||
const String name = COMPONENT_PART + std::to_string(*id) + "_" + std::to_string(++data.current_id);
|
||||
data.old_name[name] = ast->getAliasOrColumnName();
|
||||
data.component[name] = *id;
|
||||
data.components.insert(*id);
|
||||
ast = std::make_shared<ASTIdentifier>(name);
|
||||
}
|
||||
}
|
||||
|
||||
static bool needChildVisit(const ASTPtr &, const ASTPtr &)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
using ComponentVisitor = ComponentMatcher::Visitor;
|
||||
|
||||
struct ColumnPrice
|
||||
{
|
||||
Int64 compressed_size;
|
||||
Int64 uncompressed_size;
|
||||
|
||||
ColumnPrice(const Int64 compressed_size_, const Int64 uncompressed_size_)
|
||||
: compressed_size(compressed_size_)
|
||||
, uncompressed_size(uncompressed_size_)
|
||||
{
|
||||
}
|
||||
|
||||
ColumnPrice() : ColumnPrice(0, 0) {}
|
||||
|
||||
bool operator<(const ColumnPrice & that) const
|
||||
{
|
||||
return std::tie(compressed_size, uncompressed_size) < std::tie(that.compressed_size, that.uncompressed_size);
|
||||
}
|
||||
|
||||
ColumnPrice & operator+=(const ColumnPrice & that)
|
||||
{
|
||||
compressed_size += that.compressed_size;
|
||||
uncompressed_size += that.uncompressed_size;
|
||||
return *this;
|
||||
}
|
||||
|
||||
ColumnPrice & operator-=(const ColumnPrice & that)
|
||||
{
|
||||
compressed_size -= that.compressed_size;
|
||||
uncompressed_size -= that.uncompressed_size;
|
||||
return *this;
|
||||
}
|
||||
};
|
||||
|
||||
using ColumnPriceByName = std::unordered_map<String, ColumnPrice>;
|
||||
|
||||
class SubstituteColumnMatcher
|
||||
{
|
||||
public:
|
||||
using Visitor = InDepthNodeVisitor<SubstituteColumnMatcher, false>;
|
||||
|
||||
struct Data
|
||||
{
|
||||
std::unordered_map<UInt64, ASTPtr> id_to_expression_map;
|
||||
std::unordered_map<String, UInt64> name_to_component_id;
|
||||
std::unordered_map<String, String> old_name;
|
||||
bool is_select;
|
||||
};
|
||||
|
||||
static void visit(ASTPtr & ast, Data & data)
|
||||
{
|
||||
const auto * identifier = ast->as<ASTIdentifier>();
|
||||
if (identifier && data.name_to_component_id.contains(identifier->name()))
|
||||
{
|
||||
const String & name = identifier->name();
|
||||
const auto component_id = data.name_to_component_id.at(name);
|
||||
auto new_ast = data.id_to_expression_map.at(component_id)->clone();
|
||||
|
||||
if (data.is_select)
|
||||
new_ast->setAlias(data.old_name.at(name));
|
||||
|
||||
ast = new_ast;
|
||||
}
|
||||
}
|
||||
|
||||
static bool needChildVisit(const ASTPtr &, const ASTPtr &)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
using SubstituteColumnVisitor = SubstituteColumnMatcher::Visitor;
|
||||
|
||||
ColumnPrice calculatePrice(
|
||||
const ColumnPriceByName & column_prices,
|
||||
const IdentifierNameSet & identifiers)
|
||||
{
|
||||
ColumnPrice result(0, 0);
|
||||
for (const auto & ident : identifiers)
|
||||
{
|
||||
auto it = column_prices.find(ident);
|
||||
if (it != column_prices.end())
|
||||
result += it->second;
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
/// We need to choose one expression in each component,
|
||||
/// so that total price of all read columns will be minimal.
|
||||
/// Bruteforce equal ASTs in each component and calculate
|
||||
/// price of all columns on which ast depends.
|
||||
/// TODO: branch-and-bound
|
||||
void bruteforce(
|
||||
const ComparisonGraph & graph,
|
||||
const std::vector<UInt64> & components,
|
||||
size_t current_component,
|
||||
const ColumnPriceByName & column_prices,
|
||||
ColumnPrice current_price,
|
||||
std::vector<ASTPtr> & expressions_stack,
|
||||
ColumnPrice & min_price,
|
||||
std::vector<ASTPtr> & min_expressions)
|
||||
{
|
||||
if (current_component == components.size())
|
||||
{
|
||||
if (current_price < min_price)
|
||||
{
|
||||
min_price = current_price;
|
||||
min_expressions = expressions_stack;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
for (const auto & ast : graph.getComponent(components[current_component]))
|
||||
{
|
||||
IdentifierNameSet identifiers;
|
||||
ast->collectIdentifierNames(identifiers);
|
||||
ColumnPrice expression_price = calculatePrice(column_prices, identifiers);
|
||||
|
||||
expressions_stack.push_back(ast);
|
||||
current_price += expression_price;
|
||||
|
||||
ColumnPriceByName new_prices(column_prices);
|
||||
/// Update prices of already counted columns.
|
||||
for (const auto & identifier : identifiers)
|
||||
new_prices[identifier] = ColumnPrice(0, 0);
|
||||
|
||||
bruteforce(graph,
|
||||
components,
|
||||
current_component + 1,
|
||||
new_prices,
|
||||
current_price,
|
||||
expressions_stack,
|
||||
min_price,
|
||||
min_expressions);
|
||||
|
||||
current_price -= expression_price;
|
||||
expressions_stack.pop_back();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
SubstituteColumnOptimizer::SubstituteColumnOptimizer(
|
||||
ASTSelectQuery * select_query_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const ConstStoragePtr & storage_)
|
||||
: select_query(select_query_)
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, storage(storage_)
|
||||
{
|
||||
}
|
||||
|
||||
void SubstituteColumnOptimizer::perform()
|
||||
{
|
||||
if (!storage)
|
||||
return;
|
||||
|
||||
const auto column_sizes = storage->getColumnSizes();
|
||||
if (column_sizes.empty())
|
||||
return;
|
||||
|
||||
const auto & compare_graph = metadata_snapshot->getConstraints().getGraph();
|
||||
|
||||
// Fill aliases
|
||||
if (select_query->select())
|
||||
{
|
||||
auto * list = select_query->refSelect()->as<ASTExpressionList>();
|
||||
if (!list)
|
||||
throw Exception("List of selected columns must be ASTExpressionList", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
for (ASTPtr & ast : list->children)
|
||||
ast->setAlias(ast->getAliasOrColumnName());
|
||||
}
|
||||
|
||||
auto run_for_all = [&](const auto func)
|
||||
{
|
||||
if (select_query->where())
|
||||
func(select_query->refWhere(), false);
|
||||
if (select_query->prewhere())
|
||||
func(select_query->refPrewhere(), false);
|
||||
if (select_query->select())
|
||||
func(select_query->refSelect(), true);
|
||||
if (select_query->having())
|
||||
func(select_query->refHaving(), false);
|
||||
};
|
||||
|
||||
std::set<UInt64> components;
|
||||
std::unordered_map<String, String> old_name;
|
||||
std::unordered_map<String, UInt64> name_to_component;
|
||||
|
||||
UInt64 counter_id = 0;
|
||||
|
||||
ComponentVisitor::Data component_data(
|
||||
compare_graph, components, old_name, name_to_component, counter_id);
|
||||
|
||||
IdentifierNameSet identifiers;
|
||||
auto preprocess = [&](ASTPtr & ast, bool)
|
||||
{
|
||||
ComponentVisitor(component_data).visit(ast);
|
||||
ast->collectIdentifierNames(identifiers);
|
||||
};
|
||||
|
||||
run_for_all(preprocess);
|
||||
|
||||
const auto primary_key = metadata_snapshot->getColumnsRequiredForPrimaryKey();
|
||||
const std::unordered_set<std::string_view> primary_key_set(std::begin(primary_key), std::end(primary_key));
|
||||
ColumnPriceByName column_prices;
|
||||
|
||||
for (const auto & [column_name, column_size] : column_sizes)
|
||||
column_prices[column_name] = ColumnPrice(column_size.data_compressed + COLUMN_PENALTY, column_size.data_uncompressed);
|
||||
|
||||
for (const auto & column_name : primary_key)
|
||||
column_prices[column_name] = ColumnPrice(INDEX_PRICE, INDEX_PRICE);
|
||||
|
||||
for (const auto & column_name : identifiers)
|
||||
column_prices[column_name] = ColumnPrice(0, 0);
|
||||
|
||||
std::unordered_map<UInt64, ASTPtr> id_to_expression_map;
|
||||
std::vector<UInt64> components_list;
|
||||
|
||||
for (const UInt64 component_id : components)
|
||||
{
|
||||
auto component = compare_graph.getComponent(component_id);
|
||||
if (component.size() == 1)
|
||||
id_to_expression_map[component_id] = component.front();
|
||||
else
|
||||
components_list.push_back(component_id);
|
||||
}
|
||||
|
||||
std::vector<ASTPtr> expressions_stack;
|
||||
ColumnPrice min_price(std::numeric_limits<Int64>::max(), std::numeric_limits<Int64>::max());
|
||||
std::vector<ASTPtr> min_expressions;
|
||||
|
||||
bruteforce(compare_graph,
|
||||
components_list,
|
||||
0,
|
||||
column_prices,
|
||||
ColumnPrice(0, 0),
|
||||
expressions_stack,
|
||||
min_price,
|
||||
min_expressions);
|
||||
|
||||
for (size_t i = 0; i < components_list.size(); ++i)
|
||||
id_to_expression_map[components_list[i]] = min_expressions[i];
|
||||
|
||||
auto process = [&](ASTPtr & ast, bool is_select)
|
||||
{
|
||||
SubstituteColumnVisitor::Data substitute_data{id_to_expression_map, name_to_component, old_name, is_select};
|
||||
SubstituteColumnVisitor(substitute_data).visit(ast);
|
||||
};
|
||||
|
||||
run_for_all(process);
|
||||
}
|
||||
|
||||
}
|
35
src/Interpreters/SubstituteColumnOptimizer.h
Normal file
35
src/Interpreters/SubstituteColumnOptimizer.h
Normal file
@ -0,0 +1,35 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Interpreters/Aliases.h>
|
||||
#include <Interpreters/SelectQueryOptions.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct StorageInMemoryMetadata;
|
||||
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
|
||||
|
||||
/// Optimizer that tries to replace columns to equal columns (according to constraints)
|
||||
/// with lower size (accorsing to compressed and uncomressed size).
|
||||
class SubstituteColumnOptimizer
|
||||
{
|
||||
public:
|
||||
SubstituteColumnOptimizer(
|
||||
ASTSelectQuery * select_query,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const ConstStoragePtr & storage);
|
||||
|
||||
void perform();
|
||||
|
||||
private:
|
||||
ASTSelectQuery * select_query;
|
||||
const StorageMetadataPtr & metadata_snapshot;
|
||||
ConstStoragePtr storage;
|
||||
};
|
||||
|
||||
}
|
469
src/Interpreters/TreeCNFConverter.cpp
Normal file
469
src/Interpreters/TreeCNFConverter.cpp
Normal file
@ -0,0 +1,469 @@
|
||||
#include <Interpreters/TreeCNFConverter.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Common/checkStackSize.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int INCORRECT_QUERY;
|
||||
extern const int TOO_MANY_TEMPORARY_COLUMNS;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
bool isLogicalFunction(const ASTFunction & func)
|
||||
{
|
||||
return func.name == "and" || func.name == "or" || func.name == "not";
|
||||
}
|
||||
|
||||
size_t countAtoms(const ASTPtr & node)
|
||||
{
|
||||
checkStackSize();
|
||||
if (node->as<ASTIdentifier>())
|
||||
return 1;
|
||||
|
||||
const auto * func = node->as<ASTFunction>();
|
||||
if (func && !isLogicalFunction(*func))
|
||||
return 1;
|
||||
|
||||
size_t num_atoms = 0;
|
||||
for (const auto & child : node->children)
|
||||
num_atoms += countAtoms(child);
|
||||
return num_atoms;
|
||||
}
|
||||
|
||||
/// Splits AND(a, b, c) to AND(a, AND(b, c)) for AND/OR
|
||||
void splitMultiLogic(ASTPtr & node)
|
||||
{
|
||||
checkStackSize();
|
||||
auto * func = node->as<ASTFunction>();
|
||||
|
||||
if (func && (func->name == "and" || func->name == "or"))
|
||||
{
|
||||
if (func->arguments->children.size() < 2)
|
||||
throw Exception("Bad AND or OR function. Expected at least 2 arguments", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
if (func->arguments->children.size() > 2)
|
||||
{
|
||||
ASTPtr res = func->arguments->children[0]->clone();
|
||||
for (size_t i = 1; i < func->arguments->children.size(); ++i)
|
||||
res = makeASTFunction(func->name, res, func->arguments->children[i]->clone());
|
||||
|
||||
node = res;
|
||||
}
|
||||
|
||||
auto * new_func = node->as<ASTFunction>();
|
||||
for (auto & child : new_func->arguments->children)
|
||||
splitMultiLogic(child);
|
||||
}
|
||||
else if (func && func->name == "not")
|
||||
{
|
||||
for (auto & child : func->arguments->children)
|
||||
splitMultiLogic(child);
|
||||
}
|
||||
}
|
||||
|
||||
/// Push NOT to leafs, remove NOT NOT ...
|
||||
void traversePushNot(ASTPtr & node, bool add_negation)
|
||||
{
|
||||
checkStackSize();
|
||||
auto * func = node->as<ASTFunction>();
|
||||
|
||||
if (func && (func->name == "and" || func->name == "or"))
|
||||
{
|
||||
if (add_negation)
|
||||
{
|
||||
if (func->arguments->children.size() != 2)
|
||||
throw Exception("Bad AND or OR function. Expected at least 2 arguments", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
/// apply De Morgan's Law
|
||||
node = makeASTFunction(
|
||||
(func->name == "and" ? "or" : "and"),
|
||||
func->arguments->children[0]->clone(),
|
||||
func->arguments->children[1]->clone());
|
||||
}
|
||||
|
||||
auto * new_func = node->as<ASTFunction>();
|
||||
for (auto & child : new_func->arguments->children)
|
||||
traversePushNot(child, add_negation);
|
||||
}
|
||||
else if (func && func->name == "not")
|
||||
{
|
||||
if (func->arguments->children.size() != 1)
|
||||
throw Exception("Bad NOT function. Expected 1 argument", ErrorCodes::INCORRECT_QUERY);
|
||||
/// delete NOT
|
||||
node = func->arguments->children[0]->clone();
|
||||
|
||||
traversePushNot(node, !add_negation);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (add_negation)
|
||||
node = makeASTFunction("not", node->clone());
|
||||
}
|
||||
}
|
||||
|
||||
/// Push Or inside And (actually pull AND to top)
|
||||
bool traversePushOr(ASTPtr & node, size_t num_atoms, size_t max_atoms)
|
||||
{
|
||||
if (max_atoms && num_atoms > max_atoms)
|
||||
return false;
|
||||
|
||||
checkStackSize();
|
||||
auto * func = node->as<ASTFunction>();
|
||||
|
||||
if (func && (func->name == "or" || func->name == "and"))
|
||||
{
|
||||
for (auto & child : func->arguments->children)
|
||||
if (!traversePushOr(child, num_atoms, max_atoms))
|
||||
return false;
|
||||
}
|
||||
|
||||
if (func && func->name == "or")
|
||||
{
|
||||
assert(func->arguments->children.size() == 2);
|
||||
size_t and_node_id = func->arguments->children.size();
|
||||
for (size_t i = 0; i < func->arguments->children.size(); ++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 == func->arguments->children.size())
|
||||
return true;
|
||||
|
||||
const size_t other_node_id = 1 - and_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) )
|
||||
node = makeASTFunction(
|
||||
"and",
|
||||
makeASTFunction("or", a->clone(), b),
|
||||
makeASTFunction("or", a, c));
|
||||
|
||||
/// Count all atoms from 'a', because it was cloned.
|
||||
num_atoms += countAtoms(a);
|
||||
return traversePushOr(node, num_atoms, max_atoms);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
/// transform ast into cnf groups
|
||||
void traverseCNF(const ASTPtr & node, CNFQuery::AndGroup & and_group, CNFQuery::OrGroup & or_group)
|
||||
{
|
||||
checkStackSize();
|
||||
|
||||
auto * func = node->as<ASTFunction>();
|
||||
if (func && func->name == "and")
|
||||
{
|
||||
for (auto & child : func->arguments->children)
|
||||
{
|
||||
CNFQuery::OrGroup group;
|
||||
traverseCNF(child, and_group, group);
|
||||
if (!group.empty())
|
||||
and_group.insert(std::move(group));
|
||||
}
|
||||
}
|
||||
else if (func && func->name == "or")
|
||||
{
|
||||
for (auto & child : func->arguments->children)
|
||||
{
|
||||
traverseCNF(child, and_group, or_group);
|
||||
}
|
||||
}
|
||||
else if (func && func->name == "not")
|
||||
{
|
||||
if (func->arguments->children.size() != 1)
|
||||
throw Exception("Bad NOT function. Expected 1 argument", ErrorCodes::INCORRECT_QUERY);
|
||||
or_group.insert(CNFQuery::AtomicFormula{true, func->arguments->children.front()});
|
||||
}
|
||||
else
|
||||
{
|
||||
or_group.insert(CNFQuery::AtomicFormula{false, node});
|
||||
}
|
||||
}
|
||||
|
||||
void traverseCNF(const ASTPtr & node, CNFQuery::AndGroup & result)
|
||||
{
|
||||
CNFQuery::OrGroup or_group;
|
||||
traverseCNF(node, result, or_group);
|
||||
if (!or_group.empty())
|
||||
result.insert(or_group);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
std::optional<CNFQuery> TreeCNFConverter::tryConvertToCNF(
|
||||
const ASTPtr & query, size_t max_growth_multipler)
|
||||
{
|
||||
auto cnf = query->clone();
|
||||
size_t num_atoms = countAtoms(cnf);
|
||||
|
||||
splitMultiLogic(cnf);
|
||||
traversePushNot(cnf, false);
|
||||
|
||||
size_t max_atoms = max_growth_multipler
|
||||
? std::max(MAX_ATOMS_WITHOUT_CHECK, num_atoms * max_growth_multipler)
|
||||
: 0;
|
||||
|
||||
if (!traversePushOr(cnf, num_atoms, max_atoms))
|
||||
return {};
|
||||
|
||||
CNFQuery::AndGroup and_group;
|
||||
traverseCNF(cnf, and_group);
|
||||
|
||||
CNFQuery result{std::move(and_group)};
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
CNFQuery TreeCNFConverter::toCNF(
|
||||
const ASTPtr & query, size_t max_growth_multipler)
|
||||
{
|
||||
auto cnf = tryConvertToCNF(query, max_growth_multipler);
|
||||
if (!cnf)
|
||||
throw Exception(ErrorCodes::TOO_MANY_TEMPORARY_COLUMNS,
|
||||
"Cannot expression '{}' to CNF, because it produces to many clauses."
|
||||
"Size of formula inCNF can be exponential of size of source formula.");
|
||||
|
||||
return *cnf;
|
||||
}
|
||||
|
||||
ASTPtr TreeCNFConverter::fromCNF(const CNFQuery & cnf)
|
||||
{
|
||||
const auto & groups = cnf.getStatements();
|
||||
if (groups.empty())
|
||||
return nullptr;
|
||||
|
||||
ASTs or_groups;
|
||||
for (const auto & group : groups)
|
||||
{
|
||||
if (group.size() == 1)
|
||||
{
|
||||
if ((*group.begin()).negative)
|
||||
or_groups.push_back(makeASTFunction("not", (*group.begin()).ast->clone()));
|
||||
else
|
||||
or_groups.push_back((*group.begin()).ast->clone());
|
||||
}
|
||||
else if (group.size() > 1)
|
||||
{
|
||||
or_groups.push_back(makeASTFunction("or"));
|
||||
auto * func = or_groups.back()->as<ASTFunction>();
|
||||
for (const auto & atom : group)
|
||||
{
|
||||
if (atom.negative)
|
||||
func->arguments->children.push_back(makeASTFunction("not", atom.ast->clone()));
|
||||
else
|
||||
func->arguments->children.push_back(atom.ast->clone());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (or_groups.size() == 1)
|
||||
return or_groups.front();
|
||||
|
||||
ASTPtr res = makeASTFunction("and");
|
||||
auto * func = res->as<ASTFunction>();
|
||||
for (const auto & group : or_groups)
|
||||
func->arguments->children.push_back(group);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
static void pushPullNotInAtom(CNFQuery::AtomicFormula & atom, const std::unordered_map<std::string, std::string> & inverse_relations)
|
||||
{
|
||||
auto * func = atom.ast->as<ASTFunction>();
|
||||
if (!func)
|
||||
return;
|
||||
if (auto it = inverse_relations.find(func->name); it != std::end(inverse_relations))
|
||||
{
|
||||
/// inverse func
|
||||
atom.ast = atom.ast->clone();
|
||||
auto * new_func = atom.ast->as<ASTFunction>();
|
||||
new_func->name = it->second;
|
||||
/// add not
|
||||
atom.negative = !atom.negative;
|
||||
}
|
||||
}
|
||||
|
||||
static void pullNotOut(CNFQuery::AtomicFormula & atom)
|
||||
{
|
||||
static const std::unordered_map<std::string, std::string> inverse_relations = {
|
||||
{"notEquals", "equals"},
|
||||
{"greaterOrEquals", "less"},
|
||||
{"greater", "lessOrEquals"},
|
||||
{"notIn", "in"},
|
||||
{"notLike", "like"},
|
||||
{"notEmpty", "empty"},
|
||||
};
|
||||
|
||||
pushPullNotInAtom(atom, inverse_relations);
|
||||
}
|
||||
|
||||
void pushNotIn(CNFQuery::AtomicFormula & atom)
|
||||
{
|
||||
if (!atom.negative)
|
||||
return;
|
||||
|
||||
static const std::unordered_map<std::string, std::string> inverse_relations = {
|
||||
{"equals", "notEquals"},
|
||||
{"less", "greaterOrEquals"},
|
||||
{"lessOrEquals", "greater"},
|
||||
{"in", "notIn"},
|
||||
{"like", "notLike"},
|
||||
{"empty", "notEmpty"},
|
||||
{"notEquals", "equals"},
|
||||
{"greaterOrEquals", "less"},
|
||||
{"greater", "lessOrEquals"},
|
||||
{"notIn", "in"},
|
||||
{"notLike", "like"},
|
||||
{"notEmpty", "empty"},
|
||||
};
|
||||
|
||||
pushPullNotInAtom(atom, inverse_relations);
|
||||
}
|
||||
|
||||
CNFQuery & CNFQuery::pullNotOutFunctions()
|
||||
{
|
||||
transformAtoms([](const AtomicFormula & atom) -> AtomicFormula
|
||||
{
|
||||
AtomicFormula result{atom.negative, atom.ast->clone()};
|
||||
pullNotOut(result);
|
||||
return result;
|
||||
});
|
||||
return *this;
|
||||
}
|
||||
|
||||
CNFQuery & CNFQuery::pushNotInFuntions()
|
||||
{
|
||||
transformAtoms([](const AtomicFormula & atom) -> AtomicFormula
|
||||
{
|
||||
AtomicFormula result{atom.negative, atom.ast->clone()};
|
||||
pushNotIn(result);
|
||||
return result;
|
||||
});
|
||||
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;
|
||||
}
|
||||
|
||||
bool isSubset(const CNFQuery::OrGroup & left, const CNFQuery::OrGroup & right)
|
||||
{
|
||||
if (left.size() > right.size())
|
||||
return false;
|
||||
for (const auto & elem : left)
|
||||
if (!right.contains(elem))
|
||||
return false;
|
||||
return true;
|
||||
}
|
||||
|
||||
CNFQuery::AndGroup filterSubsets(const CNFQuery::AndGroup & groups)
|
||||
{
|
||||
CNFQuery::AndGroup result;
|
||||
for (const CNFQuery::OrGroup & group : groups)
|
||||
{
|
||||
bool insert = true;
|
||||
|
||||
for (const CNFQuery::OrGroup & other_group : groups)
|
||||
{
|
||||
if (isSubset(other_group, group) && group != other_group)
|
||||
{
|
||||
insert = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (insert)
|
||||
result.insert(group);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
||||
CNFQuery & CNFQuery::reduce()
|
||||
{
|
||||
while (true)
|
||||
{
|
||||
AndGroup new_statements = reduceOnce(statements);
|
||||
if (statements == new_statements)
|
||||
{
|
||||
statements = filterSubsets(statements);
|
||||
return *this;
|
||||
}
|
||||
else
|
||||
statements = new_statements;
|
||||
}
|
||||
}
|
||||
|
||||
std::string CNFQuery::dump() const
|
||||
{
|
||||
WriteBufferFromOwnString res;
|
||||
bool first = true;
|
||||
for (const auto & group : statements)
|
||||
{
|
||||
if (!first)
|
||||
res << " AND ";
|
||||
first = false;
|
||||
res << "(";
|
||||
bool first_in_group = true;
|
||||
for (const auto & atom : group)
|
||||
{
|
||||
if (!first_in_group)
|
||||
res << " OR ";
|
||||
first_in_group = false;
|
||||
if (atom.negative)
|
||||
res << " NOT ";
|
||||
res << atom.ast->getColumnName();
|
||||
}
|
||||
res << ")";
|
||||
}
|
||||
|
||||
return res.str();
|
||||
}
|
||||
|
||||
}
|
167
src/Interpreters/TreeCNFConverter.h
Normal file
167
src/Interpreters/TreeCNFConverter.h
Normal file
@ -0,0 +1,167 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <vector>
|
||||
#include <set>
|
||||
#include <unordered_map>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class CNFQuery
|
||||
{
|
||||
public:
|
||||
struct AtomicFormula
|
||||
{
|
||||
bool negative = false;
|
||||
ASTPtr ast;
|
||||
|
||||
/// for set
|
||||
bool operator<(const AtomicFormula & rhs) const
|
||||
{
|
||||
return ast->getTreeHash() == rhs.ast->getTreeHash()
|
||||
? negative < rhs.negative
|
||||
: ast->getTreeHash() < rhs.ast->getTreeHash();
|
||||
}
|
||||
|
||||
bool operator==(const AtomicFormula & rhs) const
|
||||
{
|
||||
return negative == rhs.negative &&
|
||||
ast->getTreeHash() == rhs.ast->getTreeHash() &&
|
||||
ast->getColumnName() == rhs.ast->getColumnName();
|
||||
}
|
||||
};
|
||||
|
||||
using OrGroup = std::set<AtomicFormula>;
|
||||
using AndGroup = std::set<OrGroup>;
|
||||
|
||||
CNFQuery(AndGroup && statements_) : statements(std::move(statements_)) { }
|
||||
|
||||
template <typename P>
|
||||
CNFQuery & filterAlwaysTrueGroups(P predicate_is_unknown) /// delete always true groups
|
||||
{
|
||||
AndGroup filtered;
|
||||
for (const auto & or_group : statements)
|
||||
{
|
||||
if (predicate_is_unknown(or_group))
|
||||
filtered.insert(or_group);
|
||||
}
|
||||
std::swap(statements, filtered);
|
||||
return *this;
|
||||
}
|
||||
|
||||
template <typename P>
|
||||
CNFQuery & filterAlwaysFalseAtoms(P predicate_is_unknown) /// delete always false atoms
|
||||
{
|
||||
AndGroup filtered;
|
||||
for (const auto & or_group : statements)
|
||||
{
|
||||
OrGroup filtered_group;
|
||||
for (auto ast : or_group)
|
||||
{
|
||||
if (predicate_is_unknown(ast))
|
||||
filtered_group.insert(ast);
|
||||
}
|
||||
if (!filtered_group.empty())
|
||||
filtered.insert(filtered_group);
|
||||
else
|
||||
{
|
||||
/// all atoms false -> group false -> CNF false
|
||||
filtered.clear();
|
||||
filtered_group.clear();
|
||||
filtered_group.insert(AtomicFormula{false, std::make_shared<ASTLiteral>(static_cast<UInt8>(0))});
|
||||
filtered.insert(filtered_group);
|
||||
std::swap(statements, filtered);
|
||||
return *this;
|
||||
}
|
||||
}
|
||||
std::swap(statements, filtered);
|
||||
return *this;
|
||||
}
|
||||
|
||||
template <typename F>
|
||||
const CNFQuery & iterateGroups(F func) const
|
||||
{
|
||||
for (const auto & group : statements)
|
||||
func(group);
|
||||
return *this;
|
||||
}
|
||||
|
||||
CNFQuery & appendGroup(AndGroup&& and_group)
|
||||
{
|
||||
for (auto && or_group : and_group)
|
||||
statements.emplace(std::move(or_group));
|
||||
return *this;
|
||||
}
|
||||
|
||||
template <typename F>
|
||||
CNFQuery & transformGroups(F func)
|
||||
{
|
||||
AndGroup result;
|
||||
for (const auto & group : statements)
|
||||
{
|
||||
auto new_group = func(group);
|
||||
if (!new_group.empty())
|
||||
result.insert(std::move(new_group));
|
||||
}
|
||||
std::swap(statements, result);
|
||||
return *this;
|
||||
}
|
||||
|
||||
template <typename F>
|
||||
CNFQuery & transformAtoms(F func)
|
||||
{
|
||||
transformGroups([func](const OrGroup & group) -> OrGroup
|
||||
{
|
||||
OrGroup result;
|
||||
for (const auto & atom : group)
|
||||
{
|
||||
auto new_atom = func(atom);
|
||||
if (new_atom.ast)
|
||||
result.insert(std::move(new_atom));
|
||||
}
|
||||
return result;
|
||||
});
|
||||
return *this;
|
||||
}
|
||||
|
||||
const AndGroup & getStatements() const { return statements; }
|
||||
|
||||
std::string dump() const;
|
||||
|
||||
/// Converts != -> NOT =; <,>= -> (NOT) <; >,<= -> (NOT) <= for simpler matching
|
||||
CNFQuery & pullNotOutFunctions();
|
||||
/// Revert pullNotOutFunctions actions
|
||||
CNFQuery & pushNotInFuntions();
|
||||
|
||||
/// (a OR b OR ...) AND (NOT a OR b OR ...) -> (b OR ...)
|
||||
CNFQuery & reduce();
|
||||
|
||||
private:
|
||||
AndGroup statements;
|
||||
};
|
||||
|
||||
class TreeCNFConverter
|
||||
{
|
||||
public:
|
||||
static constexpr size_t DEFAULT_MAX_GROWTH_MULTIPLIER = 20;
|
||||
static constexpr size_t MAX_ATOMS_WITHOUT_CHECK = 200;
|
||||
|
||||
/// @max_growth_multipler means that it's allowed to grow size of formula only
|
||||
/// in that amount of times. It's needed to avoid exponential explosion of formula.
|
||||
/// CNF of boolean formula with N clauses can have 2^N clauses.
|
||||
/// If amount of atomic formulas will be exceeded nullopt will be returned.
|
||||
/// 0 - means unlimited.
|
||||
static std::optional<CNFQuery> tryConvertToCNF(
|
||||
const ASTPtr & query, size_t max_growth_multipler = DEFAULT_MAX_GROWTH_MULTIPLIER);
|
||||
|
||||
static CNFQuery toCNF(
|
||||
const ASTPtr & query, size_t max_growth_multipler = DEFAULT_MAX_GROWTH_MULTIPLIER);
|
||||
|
||||
static ASTPtr fromCNF(const CNFQuery & cnf);
|
||||
};
|
||||
|
||||
void pushNotIn(CNFQuery::AtomicFormula & atom);
|
||||
|
||||
}
|
@ -4,6 +4,9 @@
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
#include <Interpreters/OptimizeIfChains.h>
|
||||
#include <Interpreters/OptimizeIfWithConstantConditionVisitor.h>
|
||||
#include <Interpreters/WhereConstraintsOptimizer.h>
|
||||
#include <Interpreters/SubstituteColumnOptimizer.h>
|
||||
#include <Interpreters/TreeCNFConverter.h>
|
||||
#include <Interpreters/ArithmeticOperationsInAgrFuncOptimize.h>
|
||||
#include <Interpreters/DuplicateOrderByVisitor.h>
|
||||
#include <Interpreters/GroupByFunctionKeysVisitor.h>
|
||||
@ -539,6 +542,44 @@ void optimizeLimitBy(const ASTSelectQuery * select_query)
|
||||
elems = std::move(unique_elems);
|
||||
}
|
||||
|
||||
/// Use constraints to get rid of useless parts of query
|
||||
void optimizeWithConstraints(ASTSelectQuery * select_query,
|
||||
Aliases & /*aliases*/,
|
||||
const NameSet & /*source_columns_set*/,
|
||||
const std::vector<TableWithColumnNamesAndTypes> & /*tables_with_columns*/,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const bool optimize_append_index)
|
||||
{
|
||||
WhereConstraintsOptimizer(select_query, metadata_snapshot, optimize_append_index).perform();
|
||||
}
|
||||
|
||||
void optimizeSubstituteColumn(ASTSelectQuery * select_query,
|
||||
Aliases & /*aliases*/,
|
||||
const NameSet & /*source_columns_set*/,
|
||||
const std::vector<TableWithColumnNamesAndTypes> & /*tables_with_columns*/,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const ConstStoragePtr & storage)
|
||||
{
|
||||
SubstituteColumnOptimizer(select_query, metadata_snapshot, storage).perform();
|
||||
}
|
||||
|
||||
/// Transform WHERE to CNF for more convenient optimization.
|
||||
bool convertQueryToCNF(ASTSelectQuery * select_query)
|
||||
{
|
||||
if (select_query->where())
|
||||
{
|
||||
auto cnf_form = TreeCNFConverter::tryConvertToCNF(select_query->where());
|
||||
if (!cnf_form)
|
||||
return false;
|
||||
|
||||
cnf_form->pushNotInFuntions();
|
||||
select_query->refWhere() = TreeCNFConverter::fromCNF(*cnf_form);
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
/// Remove duplicated columns from USING(...).
|
||||
void optimizeUsing(const ASTSelectQuery * select_query)
|
||||
{
|
||||
@ -700,6 +741,20 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result,
|
||||
if (settings.optimize_arithmetic_operations_in_aggregate_functions)
|
||||
optimizeAggregationFunctions(query);
|
||||
|
||||
bool converted_to_cnf = false;
|
||||
if (settings.convert_query_to_cnf)
|
||||
converted_to_cnf = convertQueryToCNF(select_query);
|
||||
|
||||
if (converted_to_cnf && settings.optimize_using_constraints)
|
||||
{
|
||||
optimizeWithConstraints(select_query, result.aliases, result.source_columns_set,
|
||||
tables_with_columns, result.metadata_snapshot, settings.optimize_append_index);
|
||||
|
||||
if (settings.optimize_substitute_columns)
|
||||
optimizeSubstituteColumn(select_query, result.aliases, result.source_columns_set,
|
||||
tables_with_columns, result.metadata_snapshot, result.storage);
|
||||
}
|
||||
|
||||
/// GROUP BY injective function elimination.
|
||||
optimizeGroupBy(select_query, context);
|
||||
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Interpreters/Aliases.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
namespace DB
|
||||
@ -15,6 +16,7 @@ struct TreeRewriterResult;
|
||||
class TreeOptimizer
|
||||
{
|
||||
public:
|
||||
|
||||
static void apply(
|
||||
ASTPtr & query,
|
||||
TreeRewriterResult & result,
|
||||
|
182
src/Interpreters/WhereConstraintsOptimizer.cpp
Normal file
182
src/Interpreters/WhereConstraintsOptimizer.cpp
Normal file
@ -0,0 +1,182 @@
|
||||
#include <Interpreters/WhereConstraintsOptimizer.h>
|
||||
|
||||
#include <Interpreters/TreeCNFConverter.h>
|
||||
#include <Interpreters/ComparisonGraph.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
#include <Interpreters/AddIndexConstraintsOptimizer.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Poco/Logger.h>
|
||||
|
||||
#include <Parsers/queryToString.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
WhereConstraintsOptimizer::WhereConstraintsOptimizer(
|
||||
ASTSelectQuery * select_query_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
bool optimize_append_index_)
|
||||
: select_query(select_query_)
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, optimize_append_index(optimize_append_index_)
|
||||
{
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
enum class MatchState
|
||||
{
|
||||
FULL_MATCH, /// a = b
|
||||
NOT_MATCH, /// a = not b
|
||||
NONE, /// other
|
||||
};
|
||||
|
||||
MatchState match(CNFQuery::AtomicFormula a, CNFQuery::AtomicFormula b)
|
||||
{
|
||||
bool match_means_ok = (a.negative == b.negative);
|
||||
if (a.ast->getTreeHash() == b.ast->getTreeHash())
|
||||
return match_means_ok ? MatchState::FULL_MATCH : MatchState::NOT_MATCH;
|
||||
|
||||
return MatchState::NONE;
|
||||
}
|
||||
|
||||
bool checkIfGroupAlwaysTrueFullMatch(const CNFQuery::OrGroup & group, const ConstraintsDescription & constraints_description)
|
||||
{
|
||||
/// We have constraints in CNF.
|
||||
/// CNF is always true => Each OR group in CNF is always true.
|
||||
/// So, we try to check whether we have al least one OR group from CNF as subset in our group.
|
||||
/// If we've found one then our group is always true too.
|
||||
|
||||
const auto & constraints_data = constraints_description.getConstraintData();
|
||||
std::vector<size_t> found(constraints_data.size());
|
||||
for (size_t i = 0; i < constraints_data.size(); ++i)
|
||||
found[i] = constraints_data[i].size();
|
||||
|
||||
for (const auto & atom : group)
|
||||
{
|
||||
const auto constraint_atom_ids = constraints_description.getAtomIds(atom.ast);
|
||||
if (constraint_atom_ids)
|
||||
{
|
||||
const auto constraint_atoms = constraints_description.getAtomsById(*constraint_atom_ids);
|
||||
for (size_t i = 0; i < constraint_atoms.size(); ++i)
|
||||
{
|
||||
if (match(constraint_atoms[i], atom) == MatchState::FULL_MATCH)
|
||||
{
|
||||
if ((--found[(*constraint_atom_ids)[i].group_id]) == 0)
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
bool checkIfGroupAlwaysTrueGraph(const CNFQuery::OrGroup & group, const ComparisonGraph & graph)
|
||||
{
|
||||
/// We try to find at least one atom that is always true by using comparison graph.
|
||||
for (const auto & atom : group)
|
||||
{
|
||||
const auto * func = atom.ast->as<ASTFunction>();
|
||||
if (func && func->arguments->children.size() == 2)
|
||||
{
|
||||
const auto expected = ComparisonGraph::atomToCompareResult(atom);
|
||||
if (graph.isAlwaysCompare(expected, func->arguments->children[0], func->arguments->children[1]))
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
bool checkIfAtomAlwaysFalseFullMatch(const CNFQuery::AtomicFormula & atom, const ConstraintsDescription & constraints_description)
|
||||
{
|
||||
const auto constraint_atom_ids = constraints_description.getAtomIds(atom.ast);
|
||||
if (constraint_atom_ids)
|
||||
{
|
||||
for (const auto & constraint_atom : constraints_description.getAtomsById(*constraint_atom_ids))
|
||||
{
|
||||
const auto match_result = match(constraint_atom, atom);
|
||||
if (match_result == MatchState::NOT_MATCH)
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
bool checkIfAtomAlwaysFalseGraph(const CNFQuery::AtomicFormula & atom, const ComparisonGraph & graph)
|
||||
{
|
||||
const auto * func = atom.ast->as<ASTFunction>();
|
||||
if (func && func->arguments->children.size() == 2)
|
||||
{
|
||||
/// TODO: special support for !=
|
||||
const auto expected = ComparisonGraph::atomToCompareResult(atom);
|
||||
return !graph.isPossibleCompare(expected, func->arguments->children[0], func->arguments->children[1]);
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
void replaceToConstants(ASTPtr & term, const ComparisonGraph & graph)
|
||||
{
|
||||
const auto equal_constant = graph.getEqualConst(term);
|
||||
if (equal_constant)
|
||||
{
|
||||
term = (*equal_constant)->clone();
|
||||
}
|
||||
else
|
||||
{
|
||||
for (auto & child : term->children)
|
||||
replaceToConstants(child, graph);
|
||||
}
|
||||
}
|
||||
|
||||
CNFQuery::AtomicFormula replaceTermsToConstants(const CNFQuery::AtomicFormula & atom, const ComparisonGraph & graph)
|
||||
{
|
||||
CNFQuery::AtomicFormula result;
|
||||
result.negative = atom.negative;
|
||||
result.ast = atom.ast->clone();
|
||||
|
||||
replaceToConstants(result.ast, graph);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void WhereConstraintsOptimizer::perform()
|
||||
{
|
||||
if (select_query->where() && metadata_snapshot)
|
||||
{
|
||||
const auto & compare_graph = metadata_snapshot->getConstraints().getGraph();
|
||||
auto cnf = TreeCNFConverter::toCNF(select_query->where());
|
||||
cnf.pullNotOutFunctions()
|
||||
.filterAlwaysTrueGroups([&compare_graph, this](const auto & group)
|
||||
{
|
||||
/// remove always true groups from CNF
|
||||
return !checkIfGroupAlwaysTrueFullMatch(group, metadata_snapshot->getConstraints()) && !checkIfGroupAlwaysTrueGraph(group, compare_graph);
|
||||
})
|
||||
.filterAlwaysFalseAtoms([&compare_graph, this](const auto & atom)
|
||||
{
|
||||
/// remove always false atoms from CNF
|
||||
return !checkIfAtomAlwaysFalseFullMatch(atom, metadata_snapshot->getConstraints()) && !checkIfAtomAlwaysFalseGraph(atom, compare_graph);
|
||||
})
|
||||
.transformAtoms([&compare_graph](const auto & atom)
|
||||
{
|
||||
return replaceTermsToConstants(atom, compare_graph);
|
||||
})
|
||||
.reduce()
|
||||
.pushNotInFuntions();
|
||||
|
||||
if (optimize_append_index)
|
||||
AddIndexConstraintsOptimizer(metadata_snapshot).perform(cnf);
|
||||
|
||||
select_query->setExpression(ASTSelectQuery::Expression::WHERE, TreeCNFConverter::fromCNF(cnf));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
30
src/Interpreters/WhereConstraintsOptimizer.h
Normal file
30
src/Interpreters/WhereConstraintsOptimizer.h
Normal file
@ -0,0 +1,30 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct StorageInMemoryMetadata;
|
||||
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
|
||||
|
||||
/// Optimizer that can remove useless parts of conditions
|
||||
/// in WHERE clause according to table constraints.
|
||||
class WhereConstraintsOptimizer final
|
||||
{
|
||||
public:
|
||||
WhereConstraintsOptimizer(
|
||||
ASTSelectQuery * select_query,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
bool optimize_append_index_);
|
||||
|
||||
void perform();
|
||||
|
||||
private:
|
||||
ASTSelectQuery * select_query;
|
||||
const StorageMetadataPtr & metadata_snapshot;
|
||||
bool optimize_append_index;
|
||||
};
|
||||
|
||||
}
|
183
src/Interpreters/tests/gtest_comparison_graph.cpp
Normal file
183
src/Interpreters/tests/gtest_comparison_graph.cpp
Normal file
@ -0,0 +1,183 @@
|
||||
#include <Interpreters/ComparisonGraph.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
using namespace DB;
|
||||
|
||||
static ComparisonGraph getGraph(const String & query)
|
||||
{
|
||||
ParserExpressionList parser(false);
|
||||
ASTPtr ast = parseQuery(parser, query, 0, 0);
|
||||
return ComparisonGraph(ast->children);
|
||||
}
|
||||
|
||||
TEST(ComparisonGraph, Bounds)
|
||||
{
|
||||
String query = "x <= 1, 1 < c, 3 < c, c < d, d < e, e < 7, e < 10, 10 <= y";
|
||||
auto graph = getGraph(query);
|
||||
|
||||
auto d = std::make_shared<ASTIdentifier>("d");
|
||||
|
||||
{
|
||||
auto res = graph.getConstLowerBound(d);
|
||||
ASSERT_TRUE(res.has_value());
|
||||
|
||||
const auto & [lower, strict] = *res;
|
||||
|
||||
ASSERT_EQ(get<UInt64>(lower), 3);
|
||||
ASSERT_TRUE(strict);
|
||||
}
|
||||
|
||||
{
|
||||
auto res = graph.getConstUpperBound(d);
|
||||
ASSERT_TRUE(res.has_value());
|
||||
|
||||
const auto & [upper, strict] = *res;
|
||||
|
||||
ASSERT_EQ(get<UInt64>(upper), 7);
|
||||
ASSERT_TRUE(strict);
|
||||
}
|
||||
|
||||
{
|
||||
auto x = std::make_shared<ASTIdentifier>("x");
|
||||
auto y = std::make_shared<ASTIdentifier>("y");
|
||||
|
||||
ASSERT_EQ(graph.compare(x, y), ComparisonGraph::CompareResult::LESS);
|
||||
ASSERT_EQ(graph.compare(y, x), ComparisonGraph::CompareResult::GREATER);
|
||||
}
|
||||
}
|
||||
|
||||
using Components = std::set<std::set<String>>;
|
||||
|
||||
static std::set<String> componentToStrings(const ASTs & comp)
|
||||
{
|
||||
std::set<String> res;
|
||||
for (const auto & ast : comp)
|
||||
res.insert(ast->getColumnName());
|
||||
return res;
|
||||
}
|
||||
|
||||
static void checkComponents(const String & query, const Components & expected)
|
||||
{
|
||||
auto graph = getGraph(query);
|
||||
|
||||
size_t num_components = graph.getNumOfComponents();
|
||||
ASSERT_EQ(num_components, expected.size());
|
||||
|
||||
Components res;
|
||||
for (size_t i = 0; i < num_components; ++i)
|
||||
res.insert(componentToStrings(graph.getComponent(i)));
|
||||
|
||||
ASSERT_EQ(res, expected);
|
||||
}
|
||||
|
||||
TEST(ComparisonGraph, Components)
|
||||
{
|
||||
{
|
||||
String query = "a >= b, b >= c, c >= d, d >= b, d >= e, a >= e";
|
||||
Components expected = {{"a"}, {"b", "c", "d"}, {"e"}};
|
||||
checkComponents(query, expected);
|
||||
}
|
||||
|
||||
{
|
||||
String query = "a >= b, b >= a, b >= c, c >= d, d >= c";
|
||||
Components expected = {{"a", "b"}, {"c", "d"}};
|
||||
checkComponents(query, expected);
|
||||
}
|
||||
}
|
||||
|
||||
TEST(ComparisonGraph, Compare)
|
||||
{
|
||||
using CompareResult = ComparisonGraph::CompareResult;
|
||||
|
||||
{
|
||||
String query = "a >= b, c >= b";
|
||||
auto graph = getGraph(query);
|
||||
|
||||
auto a = std::make_shared<ASTIdentifier>("a");
|
||||
auto c = std::make_shared<ASTIdentifier>("c");
|
||||
|
||||
ASSERT_EQ(graph.compare(a, c), CompareResult::UNKNOWN);
|
||||
}
|
||||
|
||||
{
|
||||
String query = "a >= b, b > c";
|
||||
auto graph = getGraph(query);
|
||||
|
||||
auto a = std::make_shared<ASTIdentifier>("a");
|
||||
auto b = std::make_shared<ASTIdentifier>("b");
|
||||
auto c = std::make_shared<ASTIdentifier>("c");
|
||||
|
||||
ASSERT_EQ(graph.compare(a, c), CompareResult::GREATER);
|
||||
ASSERT_EQ(graph.compare(a, b), CompareResult::GREATER_OR_EQUAL);
|
||||
ASSERT_EQ(graph.compare(b, c), CompareResult::GREATER);
|
||||
}
|
||||
|
||||
{
|
||||
String query = "a != b, c < a";
|
||||
auto graph = getGraph(query);
|
||||
|
||||
auto a = std::make_shared<ASTIdentifier>("a");
|
||||
auto b = std::make_shared<ASTIdentifier>("b");
|
||||
auto c = std::make_shared<ASTIdentifier>("c");
|
||||
|
||||
ASSERT_EQ(graph.compare(a, b), CompareResult::NOT_EQUAL);
|
||||
ASSERT_EQ(graph.compare(a, c), CompareResult::GREATER);
|
||||
ASSERT_EQ(graph.compare(b, c), CompareResult::UNKNOWN);
|
||||
}
|
||||
|
||||
{
|
||||
/// These constraints are inconsistent.
|
||||
String query = "a >= b, b >= a, a != b";
|
||||
ASSERT_THROW(getGraph(query), Exception);
|
||||
}
|
||||
|
||||
{
|
||||
/// These constraints are inconsistent.
|
||||
String query = "a > b, b > c, c > a";
|
||||
ASSERT_THROW(getGraph(query), Exception);
|
||||
}
|
||||
|
||||
{
|
||||
String query = "a >= 3, b > a, c >= 3, d >= c";
|
||||
auto graph = getGraph(query);
|
||||
|
||||
auto a = std::make_shared<ASTIdentifier>("a");
|
||||
auto b = std::make_shared<ASTIdentifier>("b");
|
||||
auto d = std::make_shared<ASTIdentifier>("d");
|
||||
auto lit_2 = std::make_shared<ASTLiteral>(2u);
|
||||
auto lit_3 = std::make_shared<ASTLiteral>(3u);
|
||||
auto lit_4 = std::make_shared<ASTLiteral>(4u);
|
||||
|
||||
ASSERT_EQ(graph.compare(lit_3, a), CompareResult::LESS_OR_EQUAL);
|
||||
ASSERT_FALSE(graph.isAlwaysCompare(CompareResult::LESS, lit_3, a));
|
||||
ASSERT_TRUE(graph.isAlwaysCompare(CompareResult::LESS, lit_2, a));
|
||||
|
||||
ASSERT_EQ(graph.compare(b, lit_2), CompareResult::GREATER);
|
||||
ASSERT_EQ(graph.compare(b, lit_3), CompareResult::GREATER);
|
||||
ASSERT_EQ(graph.compare(b, lit_4), CompareResult::UNKNOWN);
|
||||
|
||||
ASSERT_EQ(graph.compare(d, lit_2), CompareResult::GREATER);
|
||||
ASSERT_EQ(graph.compare(d, lit_3), CompareResult::GREATER_OR_EQUAL);
|
||||
ASSERT_EQ(graph.compare(d, lit_4), CompareResult::UNKNOWN);
|
||||
}
|
||||
|
||||
{
|
||||
String query = "a >= 5, a <= 10";
|
||||
auto graph = getGraph(query);
|
||||
|
||||
auto a = std::make_shared<ASTIdentifier>("a");
|
||||
auto lit_8 = std::make_shared<ASTLiteral>(8);
|
||||
auto lit_3 = std::make_shared<ASTLiteral>(3);
|
||||
auto lit_15 = std::make_shared<ASTLiteral>(15);
|
||||
|
||||
ASSERT_EQ(graph.compare(a, lit_8), CompareResult::UNKNOWN);
|
||||
ASSERT_EQ(graph.compare(a, lit_3), CompareResult::GREATER);
|
||||
ASSERT_EQ(graph.compare(a, lit_15), CompareResult::LESS);
|
||||
}
|
||||
}
|
@ -11,6 +11,7 @@ ASTPtr ASTConstraintDeclaration::clone() const
|
||||
auto res = std::make_shared<ASTConstraintDeclaration>();
|
||||
|
||||
res->name = name;
|
||||
res->type = type;
|
||||
|
||||
if (expr)
|
||||
res->set(res->expr, expr->clone());
|
||||
@ -21,7 +22,7 @@ ASTPtr ASTConstraintDeclaration::clone() const
|
||||
void ASTConstraintDeclaration::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const
|
||||
{
|
||||
s.ostr << backQuoteIfNeed(name);
|
||||
s.ostr << (s.hilite ? hilite_keyword : "") << " CHECK " << (s.hilite ? hilite_none : "");
|
||||
s.ostr << (s.hilite ? hilite_keyword : "") << (type == Type::CHECK ? " CHECK " : " ASSUME ") << (s.hilite ? hilite_none : "");
|
||||
expr->formatImpl(s, state, frame);
|
||||
}
|
||||
|
||||
|
@ -10,7 +10,14 @@ namespace DB
|
||||
class ASTConstraintDeclaration : public IAST
|
||||
{
|
||||
public:
|
||||
enum class Type : UInt8
|
||||
{
|
||||
CHECK,
|
||||
ASSUME,
|
||||
};
|
||||
|
||||
String name;
|
||||
Type type;
|
||||
IAST * expr;
|
||||
|
||||
String getID(char) const override { return "Constraint"; }
|
||||
|
@ -143,24 +143,32 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
||||
bool ParserConstraintDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ParserKeyword s_check("CHECK");
|
||||
ParserKeyword s_assume("ASSUME");
|
||||
|
||||
ParserIdentifier name_p;
|
||||
ParserLogicalOrExpression expression_p;
|
||||
|
||||
ASTPtr name;
|
||||
ASTPtr expr;
|
||||
ASTConstraintDeclaration::Type type = ASTConstraintDeclaration::Type::CHECK;
|
||||
|
||||
if (!name_p.parse(pos, name, expected))
|
||||
return false;
|
||||
|
||||
if (!s_check.ignore(pos, expected))
|
||||
return false;
|
||||
{
|
||||
if (s_assume.ignore(pos, expected))
|
||||
type = ASTConstraintDeclaration::Type::ASSUME;
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
if (!expression_p.parse(pos, expr, expected))
|
||||
return false;
|
||||
|
||||
auto constraint = std::make_shared<ASTConstraintDeclaration>();
|
||||
constraint->name = name->as<ASTIdentifier &>().name();
|
||||
constraint->type = type;
|
||||
constraint->set(constraint->expr, expr);
|
||||
node = constraint;
|
||||
|
||||
|
@ -29,7 +29,7 @@ CheckConstraintsTransform::CheckConstraintsTransform(
|
||||
ContextPtr context_)
|
||||
: ExceptionKeepingTransform(header, header)
|
||||
, table_id(table_id_)
|
||||
, constraints(constraints_)
|
||||
, constraints_to_check(constraints_.filterConstraints(ConstraintsDescription::ConstraintType::CHECK))
|
||||
, expressions(constraints_.getExpressions(context_, header.getNamesAndTypesList()))
|
||||
{
|
||||
}
|
||||
@ -45,7 +45,7 @@ void CheckConstraintsTransform::transform(Chunk & chunk)
|
||||
auto constraint_expr = expressions[i];
|
||||
constraint_expr->execute(block_to_calculate);
|
||||
|
||||
auto * constraint_ptr = constraints.constraints[i]->as<ASTConstraintDeclaration>();
|
||||
auto * constraint_ptr = constraints_to_check[i]->as<ASTConstraintDeclaration>();
|
||||
|
||||
ColumnWithTypeAndName res_column = block_to_calculate.getByName(constraint_ptr->expr->getColumnName());
|
||||
|
||||
|
@ -27,7 +27,7 @@ public:
|
||||
|
||||
private:
|
||||
StorageID table_id;
|
||||
const ConstraintsDescription constraints;
|
||||
const ASTs constraints_to_check;
|
||||
const ConstraintsExpressions expressions;
|
||||
size_t rows_written = 0;
|
||||
};
|
||||
|
@ -549,9 +549,10 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
|
||||
}
|
||||
else if (type == ADD_CONSTRAINT)
|
||||
{
|
||||
auto constraints = metadata.constraints.getConstraints();
|
||||
if (std::any_of(
|
||||
metadata.constraints.constraints.cbegin(),
|
||||
metadata.constraints.constraints.cend(),
|
||||
constraints.cbegin(),
|
||||
constraints.cend(),
|
||||
[this](const ASTPtr & constraint_ast)
|
||||
{
|
||||
return constraint_ast->as<ASTConstraintDeclaration &>().name == constraint_name;
|
||||
@ -563,28 +564,30 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
auto insert_it = metadata.constraints.constraints.end();
|
||||
|
||||
metadata.constraints.constraints.emplace(insert_it, std::dynamic_pointer_cast<ASTConstraintDeclaration>(constraint_decl));
|
||||
auto insert_it = constraints.end();
|
||||
constraints.emplace(insert_it, constraint_decl);
|
||||
metadata.constraints = ConstraintsDescription(constraints);
|
||||
}
|
||||
else if (type == DROP_CONSTRAINT)
|
||||
{
|
||||
auto constraints = metadata.constraints.getConstraints();
|
||||
auto erase_it = std::find_if(
|
||||
metadata.constraints.constraints.begin(),
|
||||
metadata.constraints.constraints.end(),
|
||||
constraints.begin(),
|
||||
constraints.end(),
|
||||
[this](const ASTPtr & constraint_ast)
|
||||
{
|
||||
return constraint_ast->as<ASTConstraintDeclaration &>().name == constraint_name;
|
||||
});
|
||||
|
||||
if (erase_it == metadata.constraints.constraints.end())
|
||||
if (erase_it == constraints.end())
|
||||
{
|
||||
if (if_exists)
|
||||
return;
|
||||
throw Exception("Wrong constraint name. Cannot find constraint `" + constraint_name + "` to drop.",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
metadata.constraints.constraints.erase(erase_it);
|
||||
constraints.erase(erase_it);
|
||||
metadata.constraints = ConstraintsDescription(constraints);
|
||||
}
|
||||
else if (type == ADD_PROJECTION)
|
||||
{
|
||||
@ -654,8 +657,10 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
|
||||
if (metadata.table_ttl.definition_ast)
|
||||
rename_visitor.visit(metadata.table_ttl.definition_ast);
|
||||
|
||||
for (auto & constraint : metadata.constraints.constraints)
|
||||
auto constraints_data = metadata.constraints.getConstraints();
|
||||
for (auto & constraint : constraints_data)
|
||||
rename_visitor.visit(constraint);
|
||||
metadata.constraints = ConstraintsDescription(constraints_data);
|
||||
|
||||
if (metadata.isSortingKeyDefined())
|
||||
rename_visitor.visit(metadata.sorting_key.definition_ast);
|
||||
|
@ -7,12 +7,17 @@
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
#include <Core/Defines.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
String ConstraintsDescription::toString() const
|
||||
{
|
||||
@ -41,6 +46,84 @@ ConstraintsDescription ConstraintsDescription::parse(const String & str)
|
||||
return res;
|
||||
}
|
||||
|
||||
ASTs ConstraintsDescription::filterConstraints(ConstraintType selection) const
|
||||
{
|
||||
const auto ast_to_decr_constraint_type = [](ASTConstraintDeclaration::Type constraint_type) -> UInt8
|
||||
{
|
||||
switch (constraint_type)
|
||||
{
|
||||
case ASTConstraintDeclaration::Type::CHECK:
|
||||
return static_cast<UInt8>(ConstraintType::CHECK);
|
||||
case ASTConstraintDeclaration::Type::ASSUME:
|
||||
return static_cast<UInt8>(ConstraintType::ASSUME);
|
||||
}
|
||||
throw Exception("Unknown constraint type.", ErrorCodes::LOGICAL_ERROR);
|
||||
};
|
||||
|
||||
ASTs res;
|
||||
res.reserve(constraints.size());
|
||||
for (const auto & constraint : constraints)
|
||||
{
|
||||
if ((ast_to_decr_constraint_type(constraint->as<ASTConstraintDeclaration>()->type) & static_cast<UInt8>(selection)) != 0)
|
||||
{
|
||||
res.push_back(constraint);
|
||||
}
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
std::vector<std::vector<CNFQuery::AtomicFormula>> ConstraintsDescription::buildConstraintData() const
|
||||
{
|
||||
std::vector<std::vector<CNFQuery::AtomicFormula>> constraint_data;
|
||||
for (const auto & constraint : filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE))
|
||||
{
|
||||
const auto cnf = TreeCNFConverter::toCNF(constraint->as<ASTConstraintDeclaration>()->expr->ptr())
|
||||
.pullNotOutFunctions(); /// TODO: move prepare stage to ConstraintsDescription
|
||||
for (const auto & group : cnf.getStatements())
|
||||
constraint_data.emplace_back(std::begin(group), std::end(group));
|
||||
}
|
||||
|
||||
return constraint_data;
|
||||
}
|
||||
|
||||
std::vector<CNFQuery::AtomicFormula> ConstraintsDescription::getAtomicConstraintData() const
|
||||
{
|
||||
std::vector<CNFQuery::AtomicFormula> constraint_data;
|
||||
for (const auto & constraint : filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE))
|
||||
{
|
||||
const auto cnf = TreeCNFConverter::toCNF(constraint->as<ASTConstraintDeclaration>()->expr->ptr())
|
||||
.pullNotOutFunctions();
|
||||
for (const auto & group : cnf.getStatements())
|
||||
{
|
||||
if (group.size() == 1)
|
||||
constraint_data.push_back(*group.begin());
|
||||
}
|
||||
}
|
||||
|
||||
return constraint_data;
|
||||
}
|
||||
|
||||
std::unique_ptr<ComparisonGraph> ConstraintsDescription::buildGraph() const
|
||||
{
|
||||
static const NameSet relations = { "equals", "less", "lessOrEquals", "greaterOrEquals", "greater" };
|
||||
|
||||
std::vector<ASTPtr> constraints_for_graph;
|
||||
auto atomic_formulas = getAtomicConstraintData();
|
||||
for (const auto & atomic_formula : atomic_formulas)
|
||||
{
|
||||
CNFQuery::AtomicFormula atom{atomic_formula.negative, atomic_formula.ast->clone()};
|
||||
pushNotIn(atom);
|
||||
auto * func = atom.ast->as<ASTFunction>();
|
||||
if (func && relations.count(func->name))
|
||||
{
|
||||
assert(!atom.negative);
|
||||
constraints_for_graph.push_back(atom.ast);
|
||||
}
|
||||
}
|
||||
|
||||
return std::make_unique<ComparisonGraph>(constraints_for_graph);
|
||||
}
|
||||
|
||||
ConstraintsExpressions ConstraintsDescription::getExpressions(const DB::ContextPtr context,
|
||||
const DB::NamesAndTypesList & source_columns_) const
|
||||
{
|
||||
@ -48,20 +131,62 @@ ConstraintsExpressions ConstraintsDescription::getExpressions(const DB::ContextP
|
||||
res.reserve(constraints.size());
|
||||
for (const auto & constraint : constraints)
|
||||
{
|
||||
// TreeRewriter::analyze has query as non-const argument so to avoid accidental query changes we clone it
|
||||
auto * constraint_ptr = constraint->as<ASTConstraintDeclaration>();
|
||||
ASTPtr expr = constraint_ptr->expr->clone();
|
||||
auto syntax_result = TreeRewriter(context).analyze(expr, source_columns_);
|
||||
res.push_back(ExpressionAnalyzer(constraint_ptr->expr->clone(), syntax_result, context).getActions(false, true, CompileExpressions::yes));
|
||||
if (constraint_ptr->type == ASTConstraintDeclaration::Type::CHECK)
|
||||
{
|
||||
// TreeRewriter::analyze has query as non-const argument so to avoid accidental query changes we clone it
|
||||
ASTPtr expr = constraint_ptr->expr->clone();
|
||||
auto syntax_result = TreeRewriter(context).analyze(expr, source_columns_);
|
||||
res.push_back(ExpressionAnalyzer(constraint_ptr->expr->clone(), syntax_result, context).getActions(false, true, CompileExpressions::yes));
|
||||
}
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
const ComparisonGraph & ConstraintsDescription::getGraph() const
|
||||
{
|
||||
return *graph;
|
||||
}
|
||||
|
||||
const std::vector<std::vector<CNFQuery::AtomicFormula>> & ConstraintsDescription::getConstraintData() const
|
||||
{
|
||||
return cnf_constraints;
|
||||
}
|
||||
|
||||
const std::vector<ASTPtr> & ConstraintsDescription::getConstraints() const
|
||||
{
|
||||
return constraints;
|
||||
}
|
||||
|
||||
std::optional<ConstraintsDescription::AtomIds> ConstraintsDescription::getAtomIds(const ASTPtr & ast) const
|
||||
{
|
||||
const auto hash = ast->getTreeHash();
|
||||
auto it = ast_to_atom_ids.find(hash);
|
||||
if (it != ast_to_atom_ids.end())
|
||||
return it->second;
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
std::vector<CNFQuery::AtomicFormula> ConstraintsDescription::getAtomsById(const ConstraintsDescription::AtomIds & ids) const
|
||||
{
|
||||
std::vector<CNFQuery::AtomicFormula> result;
|
||||
for (const auto & id : ids)
|
||||
result.push_back(cnf_constraints[id.group_id][id.atom_id]);
|
||||
return result;
|
||||
}
|
||||
|
||||
ConstraintsDescription::ConstraintsDescription(const ASTs & constraints_)
|
||||
: constraints(constraints_)
|
||||
{
|
||||
update();
|
||||
}
|
||||
|
||||
ConstraintsDescription::ConstraintsDescription(const ConstraintsDescription & other)
|
||||
{
|
||||
constraints.reserve(other.constraints.size());
|
||||
for (const auto & constraint : other.constraints)
|
||||
constraints.emplace_back(constraint->clone());
|
||||
update();
|
||||
}
|
||||
|
||||
ConstraintsDescription & ConstraintsDescription::operator=(const ConstraintsDescription & other)
|
||||
@ -69,7 +194,27 @@ ConstraintsDescription & ConstraintsDescription::operator=(const ConstraintsDesc
|
||||
constraints.resize(other.constraints.size());
|
||||
for (size_t i = 0; i < constraints.size(); ++i)
|
||||
constraints[i] = other.constraints[i]->clone();
|
||||
update();
|
||||
return *this;
|
||||
}
|
||||
|
||||
void ConstraintsDescription::update()
|
||||
{
|
||||
if (constraints.empty())
|
||||
{
|
||||
cnf_constraints.clear();
|
||||
ast_to_atom_ids.clear();
|
||||
graph = std::make_unique<ComparisonGraph>(std::vector<ASTPtr>());
|
||||
return;
|
||||
}
|
||||
|
||||
cnf_constraints = buildConstraintData();
|
||||
ast_to_atom_ids.clear();
|
||||
for (size_t i = 0; i < cnf_constraints.size(); ++i)
|
||||
for (size_t j = 0; j < cnf_constraints[i].size(); ++j)
|
||||
ast_to_atom_ids[cnf_constraints[i][j].ast->getTreeHash()].push_back({i, j});
|
||||
|
||||
graph = buildGraph();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,6 +2,8 @@
|
||||
|
||||
#include <Parsers/ASTConstraintDeclaration.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/TreeCNFConverter.h>
|
||||
#include <Interpreters/ComparisonGraph.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -10,19 +12,56 @@ using ConstraintsExpressions = std::vector<ExpressionActionsPtr>;
|
||||
|
||||
struct ConstraintsDescription
|
||||
{
|
||||
std::vector<ASTPtr> constraints;
|
||||
|
||||
ConstraintsDescription() = default;
|
||||
public:
|
||||
ConstraintsDescription() { update(); }
|
||||
ConstraintsDescription(const ASTs & constraints_);
|
||||
ConstraintsDescription(const ConstraintsDescription & other);
|
||||
ConstraintsDescription & operator=(const ConstraintsDescription & other);
|
||||
|
||||
bool empty() const { return constraints.empty(); }
|
||||
String toString() const;
|
||||
|
||||
static ConstraintsDescription parse(const String & str);
|
||||
|
||||
enum class ConstraintType : UInt8
|
||||
{
|
||||
CHECK = 1,
|
||||
ASSUME = 2,
|
||||
ALWAYS_TRUE = CHECK | ASSUME,
|
||||
ALL = CHECK | ASSUME,
|
||||
};
|
||||
|
||||
ASTs filterConstraints(ConstraintType selection) const;
|
||||
|
||||
const ASTs & getConstraints() const;
|
||||
|
||||
const std::vector<std::vector<CNFQuery::AtomicFormula>> & getConstraintData() const;
|
||||
std::vector<CNFQuery::AtomicFormula> getAtomicConstraintData() const;
|
||||
|
||||
const ComparisonGraph & getGraph() const;
|
||||
|
||||
ConstraintsExpressions getExpressions(ContextPtr context, const NamesAndTypesList & source_columns_) const;
|
||||
|
||||
ConstraintsDescription(const ConstraintsDescription & other);
|
||||
ConstraintsDescription & operator=(const ConstraintsDescription & other);
|
||||
struct AtomId
|
||||
{
|
||||
size_t group_id;
|
||||
size_t atom_id;
|
||||
};
|
||||
|
||||
using AtomIds = std::vector<AtomId>;
|
||||
|
||||
std::optional<AtomIds> getAtomIds(const ASTPtr & ast) const;
|
||||
std::vector<CNFQuery::AtomicFormula> getAtomsById(const AtomIds & ids) const;
|
||||
|
||||
private:
|
||||
std::vector<std::vector<CNFQuery::AtomicFormula>> buildConstraintData() const;
|
||||
std::unique_ptr<ComparisonGraph> buildGraph() const;
|
||||
void update();
|
||||
|
||||
ASTs constraints;
|
||||
std::vector<std::vector<CNFQuery::AtomicFormula>> cnf_constraints;
|
||||
std::map<IAST::Hash, AtomIds> ast_to_atom_ids;
|
||||
std::unique_ptr<ComparisonGraph> graph;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -775,30 +775,61 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
|
||||
|
||||
/// Let's start analyzing all useful indices
|
||||
|
||||
struct DataSkippingIndexAndCondition
|
||||
struct IndexStat
|
||||
{
|
||||
MergeTreeIndexPtr index;
|
||||
MergeTreeIndexConditionPtr condition;
|
||||
std::atomic<size_t> total_granules{0};
|
||||
std::atomic<size_t> granules_dropped{0};
|
||||
std::atomic<size_t> total_parts{0};
|
||||
std::atomic<size_t> parts_dropped{0};
|
||||
};
|
||||
|
||||
struct DataSkippingIndexAndCondition
|
||||
{
|
||||
MergeTreeIndexPtr index;
|
||||
MergeTreeIndexConditionPtr condition;
|
||||
IndexStat stat;
|
||||
|
||||
DataSkippingIndexAndCondition(MergeTreeIndexPtr index_, MergeTreeIndexConditionPtr condition_)
|
||||
: index(index_), condition(condition_)
|
||||
{
|
||||
}
|
||||
};
|
||||
|
||||
struct MergedDataSkippingIndexAndCondition
|
||||
{
|
||||
std::vector<MergeTreeIndexPtr> indices;
|
||||
MergeTreeIndexMergedConditionPtr condition;
|
||||
IndexStat stat;
|
||||
|
||||
void addIndex(const MergeTreeIndexPtr & index)
|
||||
{
|
||||
indices.push_back(index);
|
||||
condition->addIndex(indices.back());
|
||||
}
|
||||
};
|
||||
|
||||
std::list<DataSkippingIndexAndCondition> useful_indices;
|
||||
std::map<std::pair<String, size_t>, MergedDataSkippingIndexAndCondition> merged_indices;
|
||||
|
||||
if (use_skip_indexes)
|
||||
{
|
||||
for (const auto & index : metadata_snapshot->getSecondaryIndices())
|
||||
{
|
||||
auto index_helper = MergeTreeIndexFactory::instance().get(index);
|
||||
auto condition = index_helper->createIndexCondition(query_info, context);
|
||||
if (!condition->alwaysUnknownOrTrue())
|
||||
useful_indices.emplace_back(index_helper, condition);
|
||||
if (index_helper->isMergeable())
|
||||
{
|
||||
auto [it, inserted] = merged_indices.try_emplace({index_helper->index.type, index_helper->getGranularity()});
|
||||
if (inserted)
|
||||
it->second.condition = index_helper->createIndexMergedCondtition(query_info, metadata_snapshot);
|
||||
|
||||
it->second.addIndex(index_helper);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto condition = index_helper->createIndexCondition(query_info, context);
|
||||
if (!condition->alwaysUnknownOrTrue())
|
||||
useful_indices.emplace_back(index_helper, condition);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -875,7 +906,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
|
||||
if (ranges.ranges.empty())
|
||||
break;
|
||||
|
||||
index_and_condition.total_parts.fetch_add(1, std::memory_order_relaxed);
|
||||
index_and_condition.stat.total_parts.fetch_add(1, std::memory_order_relaxed);
|
||||
|
||||
size_t total_granules = 0;
|
||||
size_t granules_dropped = 0;
|
||||
@ -892,11 +923,34 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
|
||||
uncompressed_cache.get(),
|
||||
log);
|
||||
|
||||
index_and_condition.total_granules.fetch_add(total_granules, std::memory_order_relaxed);
|
||||
index_and_condition.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed);
|
||||
index_and_condition.stat.total_granules.fetch_add(total_granules, std::memory_order_relaxed);
|
||||
index_and_condition.stat.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed);
|
||||
|
||||
if (ranges.ranges.empty())
|
||||
index_and_condition.parts_dropped.fetch_add(1, std::memory_order_relaxed);
|
||||
index_and_condition.stat.parts_dropped.fetch_add(1, std::memory_order_relaxed);
|
||||
}
|
||||
|
||||
for (auto & [_, indices_and_condition] : merged_indices)
|
||||
{
|
||||
if (ranges.ranges.empty())
|
||||
break;
|
||||
|
||||
indices_and_condition.stat.total_parts.fetch_add(1, std::memory_order_relaxed);
|
||||
|
||||
size_t total_granules = 0;
|
||||
size_t granules_dropped = 0;
|
||||
ranges.ranges = filterMarksUsingMergedIndex(
|
||||
indices_and_condition.indices, indices_and_condition.condition,
|
||||
part, ranges.ranges,
|
||||
settings, reader_settings,
|
||||
total_granules, granules_dropped,
|
||||
mark_cache.get(), uncompressed_cache.get(), log);
|
||||
|
||||
indices_and_condition.stat.total_granules.fetch_add(total_granules, std::memory_order_relaxed);
|
||||
indices_and_condition.stat.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed);
|
||||
|
||||
if (ranges.ranges.empty())
|
||||
indices_and_condition.stat.parts_dropped.fetch_add(1, std::memory_order_relaxed);
|
||||
}
|
||||
|
||||
if (!ranges.ranges.empty())
|
||||
@ -977,8 +1031,8 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
|
||||
log,
|
||||
"Index {} has dropped {}/{} granules.",
|
||||
backQuote(index_name),
|
||||
index_and_condition.granules_dropped,
|
||||
index_and_condition.total_granules);
|
||||
index_and_condition.stat.granules_dropped,
|
||||
index_and_condition.stat.total_granules);
|
||||
|
||||
std::string description
|
||||
= index_and_condition.index->index.type + " GRANULARITY " + std::to_string(index_and_condition.index->index.granularity);
|
||||
@ -987,8 +1041,25 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
|
||||
.type = ReadFromMergeTree::IndexType::Skip,
|
||||
.name = index_name,
|
||||
.description = std::move(description), //-V1030
|
||||
.num_parts_after = index_and_condition.total_parts - index_and_condition.parts_dropped,
|
||||
.num_granules_after = index_and_condition.total_granules - index_and_condition.granules_dropped});
|
||||
.num_parts_after = index_and_condition.stat.total_parts - index_and_condition.stat.parts_dropped,
|
||||
.num_granules_after = index_and_condition.stat.total_granules - index_and_condition.stat.granules_dropped});
|
||||
}
|
||||
|
||||
for (const auto & [type_with_granularity, index_and_condition] : merged_indices)
|
||||
{
|
||||
const auto & index_name = "Merged";
|
||||
LOG_DEBUG(log, "Index {} has dropped {}/{} granules.",
|
||||
backQuote(index_name),
|
||||
index_and_condition.stat.granules_dropped, index_and_condition.stat.total_granules);
|
||||
|
||||
std::string description = "MERGED GRANULARITY " + std::to_string(type_with_granularity.second);
|
||||
|
||||
index_stats.emplace_back(ReadFromMergeTree::IndexStat{
|
||||
.type = ReadFromMergeTree::IndexType::Skip,
|
||||
.name = index_name,
|
||||
.description = std::move(description), //-V1030
|
||||
.num_parts_after = index_and_condition.stat.total_parts - index_and_condition.stat.parts_dropped,
|
||||
.num_granules_after = index_and_condition.stat.total_granules - index_and_condition.stat.granules_dropped});
|
||||
}
|
||||
|
||||
return parts_with_ranges;
|
||||
@ -1504,6 +1575,106 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex(
|
||||
return res;
|
||||
}
|
||||
|
||||
MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingMergedIndex(
|
||||
MergeTreeIndices indices,
|
||||
MergeTreeIndexMergedConditionPtr condition,
|
||||
MergeTreeData::DataPartPtr part,
|
||||
const MarkRanges & ranges,
|
||||
const Settings & settings,
|
||||
const MergeTreeReaderSettings & reader_settings,
|
||||
size_t & total_granules,
|
||||
size_t & granules_dropped,
|
||||
MarkCache * mark_cache,
|
||||
UncompressedCache * uncompressed_cache,
|
||||
Poco::Logger * log)
|
||||
{
|
||||
for (const auto & index_helper : indices)
|
||||
{
|
||||
if (!part->volume->getDisk()->exists(part->getFullRelativePath() + index_helper->getFileName() + ".idx"))
|
||||
{
|
||||
LOG_DEBUG(log, "File for index {} does not exist. Skipping it.", backQuote(index_helper->index.name));
|
||||
return ranges;
|
||||
}
|
||||
}
|
||||
|
||||
auto index_granularity = indices.front()->index.granularity;
|
||||
|
||||
const size_t min_marks_for_seek = roundRowsOrBytesToMarks(
|
||||
settings.merge_tree_min_rows_for_seek,
|
||||
settings.merge_tree_min_bytes_for_seek,
|
||||
part->index_granularity_info.fixed_index_granularity,
|
||||
part->index_granularity_info.index_granularity_bytes);
|
||||
|
||||
size_t marks_count = part->getMarksCount();
|
||||
size_t final_mark = part->index_granularity.hasFinalMark();
|
||||
size_t index_marks_count = (marks_count - final_mark + index_granularity - 1) / index_granularity;
|
||||
|
||||
std::vector<std::unique_ptr<MergeTreeIndexReader>> readers;
|
||||
for (const auto & index_helper : indices)
|
||||
{
|
||||
readers.emplace_back(
|
||||
std::make_unique<MergeTreeIndexReader>(
|
||||
index_helper,
|
||||
part,
|
||||
index_marks_count,
|
||||
ranges,
|
||||
mark_cache,
|
||||
uncompressed_cache,
|
||||
reader_settings));
|
||||
}
|
||||
|
||||
MarkRanges res;
|
||||
|
||||
/// Some granules can cover two or more ranges,
|
||||
/// this variable is stored to avoid reading the same granule twice.
|
||||
MergeTreeIndexGranules granules(indices.size(), nullptr);
|
||||
bool granules_filled = false;
|
||||
size_t last_index_mark = 0;
|
||||
for (const auto & range : ranges)
|
||||
{
|
||||
MarkRange index_range(
|
||||
range.begin / index_granularity,
|
||||
(range.end + index_granularity - 1) / index_granularity);
|
||||
|
||||
if (last_index_mark != index_range.begin || !granules_filled)
|
||||
for (auto & reader : readers)
|
||||
reader->seek(index_range.begin);
|
||||
|
||||
total_granules += index_range.end - index_range.begin;
|
||||
|
||||
for (size_t index_mark = index_range.begin; index_mark < index_range.end; ++index_mark)
|
||||
{
|
||||
if (index_mark != index_range.begin || !granules_filled || last_index_mark != index_range.begin)
|
||||
{
|
||||
for (size_t i = 0; i < readers.size(); ++i)
|
||||
{
|
||||
granules[i] = readers[i]->read();
|
||||
granules_filled = true;
|
||||
}
|
||||
}
|
||||
|
||||
MarkRange data_range(
|
||||
std::max(range.begin, index_mark * index_granularity),
|
||||
std::min(range.end, (index_mark + 1) * index_granularity));
|
||||
|
||||
if (!condition->mayBeTrueOnGranule(granules))
|
||||
{
|
||||
++granules_dropped;
|
||||
continue;
|
||||
}
|
||||
|
||||
if (res.empty() || res.back().end - data_range.begin > min_marks_for_seek)
|
||||
res.push_back(data_range);
|
||||
else
|
||||
res.back().end = data_range.end;
|
||||
}
|
||||
|
||||
last_index_mark = index_range.end - 1;
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
void MergeTreeDataSelectExecutor::selectPartsToRead(
|
||||
MergeTreeData::DataPartsVector & parts,
|
||||
const std::optional<std::unordered_set<String>> & part_values,
|
||||
|
@ -94,6 +94,19 @@ private:
|
||||
UncompressedCache * uncompressed_cache,
|
||||
Poco::Logger * log);
|
||||
|
||||
static MarkRanges filterMarksUsingMergedIndex(
|
||||
MergeTreeIndices indices,
|
||||
MergeTreeIndexMergedConditionPtr condition,
|
||||
MergeTreeData::DataPartPtr part,
|
||||
const MarkRanges & ranges,
|
||||
const Settings & settings,
|
||||
const MergeTreeReaderSettings & reader_settings,
|
||||
size_t & total_granules,
|
||||
size_t & granules_dropped,
|
||||
MarkCache * mark_cache,
|
||||
UncompressedCache * uncompressed_cache,
|
||||
Poco::Logger * log);
|
||||
|
||||
struct PartFilterCounters
|
||||
{
|
||||
size_t num_initial_selected_parts = 0;
|
||||
|
115
src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp
Normal file
115
src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp
Normal file
@ -0,0 +1,115 @@
|
||||
#include <Storages/MergeTree/MergeTreeIndexHypothesis.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.h>
|
||||
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
MergeTreeIndexGranuleHypothesis::MergeTreeIndexGranuleHypothesis(const String & index_name_)
|
||||
: index_name(index_name_), is_empty(true), met(false)
|
||||
{
|
||||
}
|
||||
|
||||
MergeTreeIndexGranuleHypothesis::MergeTreeIndexGranuleHypothesis(const String & index_name_, const bool met_)
|
||||
: index_name(index_name_), is_empty(false), met(met_)
|
||||
{
|
||||
}
|
||||
|
||||
void MergeTreeIndexGranuleHypothesis::serializeBinary(WriteBuffer & ostr) const
|
||||
{
|
||||
const auto & size_type = DataTypePtr(std::make_shared<DataTypeUInt8>());
|
||||
size_type->getDefaultSerialization()->serializeBinary(static_cast<UInt8>(met), ostr);
|
||||
}
|
||||
|
||||
void MergeTreeIndexGranuleHypothesis::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version)
|
||||
{
|
||||
if (version != 1)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index version {}.", version);
|
||||
|
||||
Field field_met;
|
||||
const auto & size_type = DataTypePtr(std::make_shared<DataTypeUInt8>());
|
||||
size_type->getDefaultSerialization()->deserializeBinary(field_met, istr);
|
||||
met = field_met.get<UInt8>();
|
||||
is_empty = false;
|
||||
}
|
||||
|
||||
MergeTreeIndexAggregatorHypothesis::MergeTreeIndexAggregatorHypothesis(const String & index_name_, const String & column_name_)
|
||||
: index_name(index_name_), column_name(column_name_)
|
||||
{
|
||||
}
|
||||
|
||||
MergeTreeIndexGranulePtr MergeTreeIndexAggregatorHypothesis::getGranuleAndReset()
|
||||
{
|
||||
const auto granule = std::make_shared<MergeTreeIndexGranuleHypothesis>(index_name, met);
|
||||
met = true;
|
||||
is_empty = true;
|
||||
return granule;
|
||||
}
|
||||
|
||||
void MergeTreeIndexAggregatorHypothesis::update(const Block & block, size_t * pos, size_t limit)
|
||||
{
|
||||
size_t rows_read = std::min(limit, block.rows() - *pos);
|
||||
if (rows_read == 0)
|
||||
return;
|
||||
const auto & column = block.getByName(column_name).column->cut(*pos, rows_read);
|
||||
|
||||
if (!column->hasEqualValues() || column->get64(0) == 0)
|
||||
met = false;
|
||||
|
||||
is_empty = false;
|
||||
*pos += rows_read;
|
||||
}
|
||||
|
||||
MergeTreeIndexGranulePtr MergeTreeIndexHypothesis::createIndexGranule() const
|
||||
{
|
||||
return std::make_shared<MergeTreeIndexGranuleHypothesis>(index.name);
|
||||
}
|
||||
|
||||
MergeTreeIndexAggregatorPtr MergeTreeIndexHypothesis::createIndexAggregator() const
|
||||
{
|
||||
return std::make_shared<MergeTreeIndexAggregatorHypothesis>(index.name, index.sample_block.getNames().front());
|
||||
}
|
||||
|
||||
MergeTreeIndexConditionPtr MergeTreeIndexHypothesis::createIndexCondition(
|
||||
const SelectQueryInfo &, ContextPtr) const
|
||||
{
|
||||
throw Exception("Not supported", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
MergeTreeIndexMergedConditionPtr MergeTreeIndexHypothesis::createIndexMergedCondtition(
|
||||
const SelectQueryInfo & query_info, StorageMetadataPtr storage_metadata) const
|
||||
{
|
||||
return std::make_shared<MergeTreeIndexhypothesisMergedCondition>(
|
||||
query_info, storage_metadata->getConstraints(), index.granularity);
|
||||
}
|
||||
|
||||
bool MergeTreeIndexHypothesis::mayBenefitFromIndexForIn(const ASTPtr &) const
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
MergeTreeIndexPtr hypothesisIndexCreator(const IndexDescription & index)
|
||||
{
|
||||
return std::make_shared<MergeTreeIndexHypothesis>(index);
|
||||
}
|
||||
|
||||
void hypothesisIndexValidator(const IndexDescription & index, bool /*attach*/)
|
||||
{
|
||||
if (index.expression_list_ast->children.size() != 1)
|
||||
throw Exception("Hypothesis index needs exactly one expression", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
}
|
81
src/Storages/MergeTree/MergeTreeIndexHypothesis.h
Normal file
81
src/Storages/MergeTree/MergeTreeIndexHypothesis.h
Normal file
@ -0,0 +1,81 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/MergeTree/MergeTreeIndices.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <memory>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class MergeTreeIndexHyposesis;
|
||||
|
||||
struct MergeTreeIndexGranuleHypothesis : public IMergeTreeIndexGranule
|
||||
{
|
||||
explicit MergeTreeIndexGranuleHypothesis(
|
||||
const String & index_name_);
|
||||
|
||||
MergeTreeIndexGranuleHypothesis(
|
||||
const String & index_name_,
|
||||
const bool met_);
|
||||
|
||||
void serializeBinary(WriteBuffer & ostr) const override;
|
||||
void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override;
|
||||
|
||||
bool empty() const override { return is_empty; }
|
||||
|
||||
~MergeTreeIndexGranuleHypothesis() override = default;
|
||||
|
||||
const String & index_name;
|
||||
bool is_empty = true;
|
||||
bool met = true;
|
||||
};
|
||||
|
||||
|
||||
struct MergeTreeIndexAggregatorHypothesis : IMergeTreeIndexAggregator
|
||||
{
|
||||
explicit MergeTreeIndexAggregatorHypothesis(
|
||||
const String & index_name_, const String & column_name_);
|
||||
|
||||
~MergeTreeIndexAggregatorHypothesis() override = default;
|
||||
|
||||
bool empty() const override { return is_empty; }
|
||||
|
||||
MergeTreeIndexGranulePtr getGranuleAndReset() override;
|
||||
|
||||
void update(const Block & block, size_t * pos, size_t limit) override;
|
||||
|
||||
private:
|
||||
const String & index_name;
|
||||
String column_name;
|
||||
|
||||
bool met = true;
|
||||
bool is_empty = true;
|
||||
};
|
||||
|
||||
class MergeTreeIndexHypothesis : public IMergeTreeIndex
|
||||
{
|
||||
public:
|
||||
MergeTreeIndexHypothesis(
|
||||
const IndexDescription & index_)
|
||||
: IMergeTreeIndex(index_)
|
||||
{}
|
||||
|
||||
~MergeTreeIndexHypothesis() override = default;
|
||||
|
||||
bool isMergeable() const override { return true; }
|
||||
|
||||
MergeTreeIndexGranulePtr createIndexGranule() const override;
|
||||
MergeTreeIndexAggregatorPtr createIndexAggregator() const override;
|
||||
|
||||
MergeTreeIndexConditionPtr createIndexCondition(
|
||||
const SelectQueryInfo & query, ContextPtr context) const override;
|
||||
|
||||
MergeTreeIndexMergedConditionPtr createIndexMergedCondtition(
|
||||
const SelectQueryInfo & query_info, StorageMetadataPtr storage_metadata) const override;
|
||||
|
||||
bool mayBenefitFromIndexForIn(const ASTPtr & node) const override;
|
||||
|
||||
size_t max_rows = 0;
|
||||
};
|
||||
|
||||
}
|
@ -0,0 +1,204 @@
|
||||
#include <Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.h>
|
||||
|
||||
#include <Storages/MergeTree/MergeTreeIndexHypothesis.h>
|
||||
#include <Interpreters/TreeCNFConverter.h>
|
||||
#include <Interpreters/ComparisonGraph.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
MergeTreeIndexhypothesisMergedCondition::MergeTreeIndexhypothesisMergedCondition(
|
||||
const SelectQueryInfo & query, const ConstraintsDescription & constraints, size_t granularity_)
|
||||
: IMergeTreeIndexMergedCondition(granularity_)
|
||||
{
|
||||
const auto & select = query.query->as<ASTSelectQuery &>();
|
||||
|
||||
if (select.where() && select.prewhere())
|
||||
expression_ast = makeASTFunction(
|
||||
"and",
|
||||
select.where()->clone(),
|
||||
select.prewhere()->clone());
|
||||
else if (select.where())
|
||||
expression_ast = select.where()->clone();
|
||||
else if (select.prewhere())
|
||||
expression_ast = select.prewhere()->clone();
|
||||
|
||||
expression_cnf = std::make_unique<CNFQuery>(
|
||||
expression_ast ? TreeCNFConverter::toCNF(expression_ast) : CNFQuery::AndGroup{});
|
||||
|
||||
addConstraints(constraints);
|
||||
}
|
||||
|
||||
void MergeTreeIndexhypothesisMergedCondition::addIndex(const MergeTreeIndexPtr & index)
|
||||
{
|
||||
if (!index->isMergeable() || index->getGranularity() != granularity)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Index {} can not be merged", index->index.type);
|
||||
|
||||
const auto hypothesis_index = std::dynamic_pointer_cast<const MergeTreeIndexHypothesis>(index);
|
||||
if (!hypothesis_index)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Only hypothesis index is supported here");
|
||||
|
||||
static const NameSet relations = { "equals", "notEquals", "less", "lessOrEquals", "greaterOrEquals", "greater"};
|
||||
|
||||
// TODO: move to index hypothesis
|
||||
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())
|
||||
{
|
||||
if (group.size() == 1)
|
||||
{
|
||||
hypotheses_data.push_back(group);
|
||||
CNFQuery::AtomicFormula atomic_formula = *group.begin();
|
||||
CNFQuery::AtomicFormula atom{atomic_formula.negative, atomic_formula.ast->clone()};
|
||||
pushNotIn(atom);
|
||||
assert(!atom.negative);
|
||||
|
||||
const auto * func = atom.ast->as<ASTFunction>();
|
||||
if (func && relations.count(func->name))
|
||||
compare_hypotheses_data.push_back(atom.ast);
|
||||
}
|
||||
}
|
||||
|
||||
index_to_compare_atomic_hypotheses.push_back(compare_hypotheses_data);
|
||||
index_to_atomic_hypotheses.push_back(hypotheses_data);
|
||||
}
|
||||
|
||||
void MergeTreeIndexhypothesisMergedCondition::addConstraints(const ConstraintsDescription & constraints_description)
|
||||
{
|
||||
auto atomic_constraints_data = constraints_description.getAtomicConstraintData();
|
||||
for (const auto & atomic_formula : atomic_constraints_data)
|
||||
{
|
||||
CNFQuery::AtomicFormula atom{atomic_formula.negative, atomic_formula.ast->clone()};
|
||||
pushNotIn(atom);
|
||||
atomic_constraints.push_back(atom.ast);
|
||||
}
|
||||
}
|
||||
|
||||
/// Replaces < -> <=, > -> >= and assumes that all hypotheses are true then checks if path exists
|
||||
bool MergeTreeIndexhypothesisMergedCondition::alwaysUnknownOrTrue() const
|
||||
{
|
||||
std::vector<ASTPtr> active_atomic_formulas(atomic_constraints);
|
||||
for (const auto & hypothesis : index_to_compare_atomic_hypotheses)
|
||||
{
|
||||
active_atomic_formulas.insert(
|
||||
std::end(active_atomic_formulas),
|
||||
std::begin(hypothesis),
|
||||
std::end(hypothesis));
|
||||
}
|
||||
|
||||
/// 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 (const auto & atomic_formula : or_group)
|
||||
{
|
||||
CNFQuery::AtomicFormula atom{atomic_formula.negative, atomic_formula.ast->clone()};
|
||||
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 MergeTreeIndexhypothesisMergedCondition::mayBeTrueOnGranule(const MergeTreeIndexGranules & granules) const
|
||||
{
|
||||
std::vector<bool> values;
|
||||
for (const auto & index_granule : granules)
|
||||
{
|
||||
const auto granule = std::dynamic_pointer_cast<const MergeTreeIndexGranuleHypothesis>(index_granule);
|
||||
if (!granule)
|
||||
throw Exception("Only hypothesis index is supported here.", ErrorCodes::LOGICAL_ERROR);
|
||||
values.push_back(granule->met);
|
||||
}
|
||||
|
||||
if (const auto it = answer_cache.find(values); it != std::end(answer_cache))
|
||||
return it->second;
|
||||
|
||||
const auto & graph = getGraph(values);
|
||||
|
||||
bool always_false = false;
|
||||
expression_cnf->iterateGroups(
|
||||
[&](const CNFQuery::OrGroup & or_group)
|
||||
{
|
||||
if (always_false)
|
||||
return;
|
||||
|
||||
for (const auto & atomic_formula : or_group)
|
||||
{
|
||||
CNFQuery::AtomicFormula atom{atomic_formula.negative, atomic_formula.ast->clone()};
|
||||
pushNotIn(atom);
|
||||
const auto * func = atom.ast->as<ASTFunction>();
|
||||
if (func && func->arguments->children.size() == 2)
|
||||
{
|
||||
const auto expected = ComparisonGraph::atomToCompareResult(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;
|
||||
}
|
||||
}
|
||||
}
|
||||
always_false = true;
|
||||
});
|
||||
|
||||
answer_cache[values] = !always_false;
|
||||
return !always_false;
|
||||
}
|
||||
|
||||
std::unique_ptr<ComparisonGraph> MergeTreeIndexhypothesisMergedCondition::buildGraph(const std::vector<bool> & values) const
|
||||
{
|
||||
std::vector<ASTPtr> active_atomic_formulas(atomic_constraints);
|
||||
for (size_t i = 0; i < values.size(); ++i)
|
||||
{
|
||||
if (values[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]));
|
||||
}
|
||||
return std::make_unique<ComparisonGraph>(active_atomic_formulas);
|
||||
}
|
||||
|
||||
const ComparisonGraph & MergeTreeIndexhypothesisMergedCondition::getGraph(const std::vector<bool> & values) const
|
||||
{
|
||||
if (!graph_cache.contains(values))
|
||||
graph_cache[values] = buildGraph(values);
|
||||
return *graph_cache.at(values);
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,37 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/MergeTree/MergeTreeIndices.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Interpreters/ComparisonGraph.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// MergedCondition for Indexhypothesis.
|
||||
class MergeTreeIndexhypothesisMergedCondition : public IMergeTreeIndexMergedCondition
|
||||
{
|
||||
public:
|
||||
MergeTreeIndexhypothesisMergedCondition(
|
||||
const SelectQueryInfo & query, const ConstraintsDescription & constraints, size_t granularity_);
|
||||
|
||||
void addIndex(const MergeTreeIndexPtr & index) override;
|
||||
bool alwaysUnknownOrTrue() const override;
|
||||
bool mayBeTrueOnGranule(const MergeTreeIndexGranules & granules) const override;
|
||||
|
||||
private:
|
||||
void addConstraints(const ConstraintsDescription & constraints_description);
|
||||
std::unique_ptr<ComparisonGraph> buildGraph(const std::vector<bool> & values) const;
|
||||
const ComparisonGraph & getGraph(const std::vector<bool> & values) const;
|
||||
|
||||
ASTPtr expression_ast;
|
||||
std::unique_ptr<CNFQuery> expression_cnf;
|
||||
|
||||
mutable std::unordered_map<std::vector<bool>, std::unique_ptr<ComparisonGraph>> graph_cache;
|
||||
mutable std::unordered_map<std::vector<bool>, bool> answer_cache;
|
||||
|
||||
std::vector<std::vector<ASTPtr>> index_to_compare_atomic_hypotheses;
|
||||
std::vector<std::vector<CNFQuery::OrGroup>> index_to_atomic_hypotheses;
|
||||
std::vector<ASTPtr> atomic_constraints;
|
||||
};
|
||||
|
||||
}
|
@ -98,6 +98,9 @@ MergeTreeIndexFactory::MergeTreeIndexFactory()
|
||||
|
||||
registerCreator("bloom_filter", bloomFilterIndexCreatorNew);
|
||||
registerValidator("bloom_filter", bloomFilterIndexValidatorNew);
|
||||
|
||||
registerCreator("hypothesis", hypothesisIndexCreator);
|
||||
registerValidator("hypothesis", hypothesisIndexValidator);
|
||||
}
|
||||
|
||||
MergeTreeIndexFactory & MergeTreeIndexFactory::instance()
|
||||
|
@ -18,6 +18,11 @@ constexpr auto INDEX_FILE_PREFIX = "skp_idx_";
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
using MergeTreeIndexVersion = uint8_t;
|
||||
struct MergeTreeIndexFormat
|
||||
{
|
||||
@ -87,6 +92,32 @@ public:
|
||||
};
|
||||
|
||||
using MergeTreeIndexConditionPtr = std::shared_ptr<IMergeTreeIndexCondition>;
|
||||
using MergeTreeIndexConditions = std::vector<MergeTreeIndexConditionPtr>;
|
||||
|
||||
struct IMergeTreeIndex;
|
||||
using MergeTreeIndexPtr = std::shared_ptr<const IMergeTreeIndex>;
|
||||
|
||||
/// IndexCondition that checks several indexes at the same time.
|
||||
class IMergeTreeIndexMergedCondition
|
||||
{
|
||||
public:
|
||||
explicit IMergeTreeIndexMergedCondition(size_t granularity_)
|
||||
: granularity(granularity_)
|
||||
{
|
||||
}
|
||||
|
||||
virtual ~IMergeTreeIndexMergedCondition() = default;
|
||||
|
||||
virtual void addIndex(const MergeTreeIndexPtr & index) = 0;
|
||||
virtual bool alwaysUnknownOrTrue() const = 0;
|
||||
virtual bool mayBeTrueOnGranule(const MergeTreeIndexGranules & granules) const = 0;
|
||||
|
||||
protected:
|
||||
const size_t granularity;
|
||||
};
|
||||
|
||||
using MergeTreeIndexMergedConditionPtr = std::shared_ptr<IMergeTreeIndexMergedCondition>;
|
||||
using MergeTreeIndexMergedConditions = std::vector<IMergeTreeIndexMergedCondition>;
|
||||
|
||||
|
||||
struct IMergeTreeIndex
|
||||
@ -100,6 +131,9 @@ struct IMergeTreeIndex
|
||||
|
||||
/// Returns filename without extension.
|
||||
String getFileName() const { return INDEX_FILE_PREFIX + index.name; }
|
||||
size_t getGranularity() const { return index.granularity; }
|
||||
|
||||
virtual bool isMergeable() const { return false; }
|
||||
|
||||
/// Returns extension for serialization.
|
||||
/// Reimplement if you want new index format.
|
||||
@ -126,7 +160,14 @@ struct IMergeTreeIndex
|
||||
virtual MergeTreeIndexAggregatorPtr createIndexAggregator() const = 0;
|
||||
|
||||
virtual MergeTreeIndexConditionPtr createIndexCondition(
|
||||
const SelectQueryInfo & query_info, ContextPtr context) const = 0;
|
||||
const SelectQueryInfo & query_info, ContextPtr context) const = 0;
|
||||
|
||||
virtual MergeTreeIndexMergedConditionPtr createIndexMergedCondtition(
|
||||
const SelectQueryInfo & /*query_info*/, StorageMetadataPtr /*storage_metadata*/) const
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
|
||||
"MergedCondition is not implemented for index of type {}", index.type);
|
||||
}
|
||||
|
||||
Names getColumnsRequiredForIndexCalc() const { return index.expression->getRequiredColumns(); }
|
||||
|
||||
@ -177,4 +218,7 @@ void bloomFilterIndexValidator(const IndexDescription & index, bool attach);
|
||||
MergeTreeIndexPtr bloomFilterIndexCreatorNew(const IndexDescription & index);
|
||||
void bloomFilterIndexValidatorNew(const IndexDescription & index, bool attach);
|
||||
|
||||
MergeTreeIndexPtr hypothesisIndexCreator(const IndexDescription & index);
|
||||
void hypothesisIndexValidator(const IndexDescription & index, bool attach);
|
||||
|
||||
}
|
||||
|
@ -14,7 +14,6 @@
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <base/map.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -707,9 +707,11 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
metadata.projections.add(std::move(projection));
|
||||
}
|
||||
|
||||
auto constraints = metadata.constraints.getConstraints();
|
||||
if (args.query.columns_list && args.query.columns_list->constraints)
|
||||
for (auto & constraint : args.query.columns_list->constraints->children)
|
||||
metadata.constraints.constraints.push_back(constraint);
|
||||
constraints.push_back(constraint);
|
||||
metadata.constraints = ConstraintsDescription(constraints);
|
||||
|
||||
auto column_ttl_asts = args.columns.getColumnTTLs();
|
||||
for (const auto & [name, ast] : column_ttl_asts)
|
||||
|
@ -0,0 +1,45 @@
|
||||
4
|
||||
0
|
||||
0
|
||||
0
|
||||
4
|
||||
4
|
||||
4
|
||||
0
|
||||
4
|
||||
1
|
||||
1
|
||||
1
|
||||
0
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
0
|
||||
0
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
0
|
||||
0
|
||||
1
|
||||
1
|
||||
0
|
||||
1
|
||||
1
|
||||
0
|
||||
1
|
||||
1
|
||||
0
|
||||
SELECT count() AS `count()`
|
||||
FROM constraint_test_constants
|
||||
WHERE (c > 100) OR (b > 100)
|
||||
SELECT count() AS `count()`
|
||||
FROM constraint_test_constants
|
||||
WHERE c > 100
|
||||
SELECT count() AS `count()`
|
||||
FROM constraint_test_constants
|
||||
WHERE c > 100
|
||||
SELECT count() AS `count()`
|
||||
FROM constraint_test_constants
|
@ -0,0 +1,105 @@
|
||||
DROP TABLE IF EXISTS constraint_test_assumption;
|
||||
DROP TABLE IF EXISTS constraint_test_transitivity;
|
||||
DROP TABLE IF EXISTS constraint_test_transitivity2;
|
||||
|
||||
SET convert_query_to_cnf = 1;
|
||||
SET optimize_using_constraints = 1;
|
||||
SET optimize_move_to_prewhere = 1;
|
||||
SET optimize_substitute_columns = 1;
|
||||
SET optimize_append_index = 1;
|
||||
|
||||
CREATE TABLE constraint_test_assumption (URL String, a Int32, CONSTRAINT c1 ASSUME domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT c2 ASSUME URL > 'zzz' AND startsWith(URL, 'test') = True) ENGINE = TinyLog;
|
||||
|
||||
--- Add wrong rows in order to check optimization
|
||||
INSERT INTO constraint_test_assumption (URL, a) VALUES ('1', 1);
|
||||
INSERT INTO constraint_test_assumption (URL, a) VALUES ('2', 2);
|
||||
INSERT INTO constraint_test_assumption (URL, a) VALUES ('yandex.ru', 3);
|
||||
INSERT INTO constraint_test_assumption (URL, a) VALUES ('3', 4);
|
||||
|
||||
SELECT count() FROM constraint_test_assumption WHERE domainWithoutWWW(URL) = 'yandex.ru'; --- assumption -> 4
|
||||
SELECT count() FROM constraint_test_assumption WHERE NOT (domainWithoutWWW(URL) = 'yandex.ru'); --- assumption -> 0
|
||||
SELECT count() FROM constraint_test_assumption WHERE domainWithoutWWW(URL) != 'yandex.ru'; --- assumption -> 0
|
||||
SELECT count() FROM constraint_test_assumption WHERE domainWithoutWWW(URL) = 'nothing'; --- not optimized -> 0
|
||||
|
||||
SELECT count() FROM constraint_test_assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND URL > 'zzz'); ---> assumption -> 4
|
||||
SELECT count() FROM constraint_test_assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND NOT URL <= 'zzz'); ---> assumption -> 4
|
||||
SELECT count() FROM constraint_test_assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND URL > 'zzz') OR (a = 10 AND a + 5 < 100); ---> assumption -> 4
|
||||
SELECT count() FROM constraint_test_assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND URL = '111'); ---> assumption & no assumption -> 0
|
||||
SELECT count() FROM constraint_test_assumption WHERE (startsWith(URL, 'test') = True); ---> assumption -> 4
|
||||
|
||||
DROP TABLE constraint_test_assumption;
|
||||
|
||||
CREATE TABLE constraint_test_transitivity (a Int64, b Int64, c Int64, d Int32, CONSTRAINT c1 ASSUME a = b AND c = d, CONSTRAINT c2 ASSUME b = c) ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO constraint_test_transitivity (a, b, c, d) VALUES (1, 2, 3, 4);
|
||||
|
||||
SELECT count() FROM constraint_test_transitivity WHERE a = d; ---> assumption -> 1
|
||||
|
||||
DROP TABLE constraint_test_transitivity;
|
||||
|
||||
|
||||
CREATE TABLE constraint_test_strong_connectivity (a String, b String, c String, d String, CONSTRAINT c1 ASSUME a <= b AND b <= c AND c <= d AND d <= a) ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO constraint_test_strong_connectivity (a, b, c, d) VALUES ('1', '2', '3', '4');
|
||||
|
||||
SELECT count() FROM constraint_test_strong_connectivity WHERE a = d; ---> assumption -> 1
|
||||
SELECT count() FROM constraint_test_strong_connectivity WHERE a = c AND b = d; ---> assumption -> 1
|
||||
SELECT count() FROM constraint_test_strong_connectivity WHERE a < c OR b < d; ---> assumption -> 0
|
||||
SELECT count() FROM constraint_test_strong_connectivity WHERE a <= c OR b <= d; ---> assumption -> 1
|
||||
|
||||
DROP TABLE constraint_test_strong_connectivity;
|
||||
|
||||
CREATE TABLE constraint_test_transitivity2 (a String, b String, c String, d String, CONSTRAINT c1 ASSUME a > b AND b >= c AND c > d AND a >= d) ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO constraint_test_transitivity2 (a, b, c, d) VALUES ('1', '2', '3', '4');
|
||||
|
||||
SELECT count() FROM constraint_test_transitivity2 WHERE a > d; ---> assumption -> 1
|
||||
SELECT count() FROM constraint_test_transitivity2 WHERE a >= d; ---> assumption -> 1
|
||||
SELECT count() FROM constraint_test_transitivity2 WHERE d < a; ---> assumption -> 1
|
||||
SELECT count() FROM constraint_test_transitivity2 WHERE a < d; ---> assumption -> 0
|
||||
SELECT count() FROM constraint_test_transitivity2 WHERE a = d; ---> assumption -> 0
|
||||
SELECT count() FROM constraint_test_transitivity2 WHERE a != d; ---> assumption -> 1
|
||||
|
||||
DROP TABLE constraint_test_transitivity2;
|
||||
|
||||
CREATE TABLE constraint_test_transitivity3 (a Int64, b Int64, c Int64, CONSTRAINT c1 ASSUME b > 10 AND 1 > a) ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO constraint_test_transitivity3 (a, b, c) VALUES (4, 0, 2);
|
||||
|
||||
SELECT count() FROM constraint_test_transitivity3 WHERE a < b; ---> assumption -> 1
|
||||
SELECT count() FROM constraint_test_transitivity3 WHERE b >= a; ---> assumption -> 1
|
||||
|
||||
DROP TABLE constraint_test_transitivity3;
|
||||
|
||||
|
||||
CREATE TABLE constraint_test_constants_repl (a Int64, b Int64, c Int64, d Int64, CONSTRAINT c1 ASSUME a - b = 10 AND c + d = 20) ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO constraint_test_constants_repl (a, b, c, d) VALUES (1, 2, 3, 4);
|
||||
|
||||
SELECT count() FROM constraint_test_constants_repl WHERE a - b = 10; ---> assumption -> 1
|
||||
SELECT count() FROM constraint_test_constants_repl WHERE a - b < 0; ---> assumption -> 0
|
||||
SELECT count() FROM constraint_test_constants_repl WHERE a - b = c + d; ---> assumption -> 0
|
||||
SELECT count() FROM constraint_test_constants_repl WHERE (a - b) * 2 = c + d; ---> assumption -> 1
|
||||
|
||||
DROP TABLE constraint_test_constants_repl;
|
||||
|
||||
CREATE TABLE constraint_test_constants (a Int64, b Int64, c Int64, CONSTRAINT c1 ASSUME b > 10 AND a >= 10) ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO constraint_test_constants (a, b, c) VALUES (0, 0, 0);
|
||||
|
||||
SELECT count() FROM constraint_test_constants WHERE 9 < b; ---> assumption -> 1
|
||||
SELECT count() FROM constraint_test_constants WHERE 11 < b; ---> assumption -> 0
|
||||
SELECT count() FROM constraint_test_constants WHERE 10 <= b; ---> assumption -> 1
|
||||
SELECT count() FROM constraint_test_constants WHERE 9 < a; ---> assumption -> 1
|
||||
SELECT count() FROM constraint_test_constants WHERE 10 < a; ---> assumption -> 0
|
||||
SELECT count() FROM constraint_test_constants WHERE 10 <= a; ---> assumption -> 1
|
||||
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);
|
||||
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) AND (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) AND (c <= 100);
|
||||
|
||||
DROP TABLE constraint_test_constants;
|
@ -0,0 +1,14 @@
|
||||
SELECT count()
|
||||
FROM t_constraints_where
|
||||
WHERE 0
|
||||
SELECT count()
|
||||
FROM t_constraints_where
|
||||
WHERE 0
|
||||
SELECT count()
|
||||
FROM t_constraints_where
|
||||
WHERE 0
|
||||
SELECT count()
|
||||
FROM t_constraints_where
|
||||
WHERE b < 8
|
||||
SELECT count()
|
||||
FROM t_constraints_where
|
@ -0,0 +1,23 @@
|
||||
SET convert_query_to_cnf = 1;
|
||||
SET optimize_using_constraints = 1;
|
||||
|
||||
DROP TABLE IF EXISTS t_constraints_where;
|
||||
|
||||
CREATE TABLE t_constraints_where(a UInt32, b UInt32, CONSTRAINT c1 ASSUME b >= 5, CONSTRAINT c2 ASSUME b <= 10) ENGINE = Memory;
|
||||
|
||||
INSERT INTO t_constraints_where VALUES (1, 7);
|
||||
|
||||
EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b > 15; -- assumption -> 0
|
||||
EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b = 20; -- assumption -> 0
|
||||
EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b < 2; -- assumption -> 0
|
||||
EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b > 20 OR b < 8; -- assumption -> remove (b < 20)
|
||||
|
||||
DROP TABLE t_constraints_where;
|
||||
|
||||
CREATE TABLE t_constraints_where(a UInt32, b UInt32, CONSTRAINT c1 ASSUME b < 10) ENGINE = Memory;
|
||||
|
||||
INSERT INTO t_constraints_where VALUES (1, 7);
|
||||
|
||||
EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b = 1 OR b < 18 OR b > 5; -- assumtion -> (b < 20) -> 0;
|
||||
|
||||
DROP TABLE t_constraints_where;
|
@ -0,0 +1,53 @@
|
||||
SELECT
|
||||
(b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`,
|
||||
(b AS b) + 3 AS `plus(b, 3)`
|
||||
FROM column_swap_test_test
|
||||
WHERE b = 1
|
||||
SELECT
|
||||
(b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`,
|
||||
(b AS b) + 3 AS `plus(b, 3)`
|
||||
FROM column_swap_test_test
|
||||
WHERE b = 0
|
||||
SELECT
|
||||
(b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`,
|
||||
(b AS b) + 3 AS `plus(b, 3)`
|
||||
FROM column_swap_test_test
|
||||
WHERE b = 0
|
||||
SELECT
|
||||
(b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`,
|
||||
(b AS b) + 3 AS `plus(b, 3)`
|
||||
FROM column_swap_test_test
|
||||
WHERE b = 1
|
||||
SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`
|
||||
FROM column_swap_test_test
|
||||
WHERE b = 0
|
||||
SELECT
|
||||
(cityHash64(a) AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`,
|
||||
a AS a
|
||||
FROM column_swap_test_test
|
||||
WHERE cityHash64(a) = 0
|
||||
SELECT
|
||||
(cityHash64(a) AS b) + 10 AS `plus(b, 10)`,
|
||||
a AS a
|
||||
FROM column_swap_test_test
|
||||
WHERE cityHash64(a) = 0
|
||||
SELECT
|
||||
a AS `substring(reverse(b), 1, 1)`,
|
||||
a AS a
|
||||
FROM column_swap_test_test
|
||||
WHERE a = \'c\'
|
||||
SELECT
|
||||
a AS `substring(reverse(b), 1, 1)`,
|
||||
a AS a
|
||||
FROM column_swap_test_test
|
||||
WHERE a = \'c\'
|
||||
SELECT
|
||||
a AS t1,
|
||||
a AS t2
|
||||
FROM column_swap_test_test
|
||||
WHERE a = \'c\'
|
||||
SELECT a AS `substring(reverse(b), 1, 1)`
|
||||
FROM column_swap_test_test
|
||||
WHERE a = \'c\'
|
||||
SELECT a
|
||||
FROM t_bad_constraint
|
42
tests/queries/0_stateless/01623_constraints_column_swap.sql
Normal file
42
tests/queries/0_stateless/01623_constraints_column_swap.sql
Normal file
@ -0,0 +1,42 @@
|
||||
SET convert_query_to_cnf = 1;
|
||||
SET optimize_using_constraints = 1;
|
||||
SET optimize_move_to_prewhere = 1;
|
||||
SET optimize_substitute_columns = 1;
|
||||
SET optimize_append_index = 1;
|
||||
|
||||
DROP TABLE IF EXISTS column_swap_test_test;
|
||||
|
||||
CREATE TABLE column_swap_test_test (i Int64, a String, b UInt64, CONSTRAINT c1 ASSUME b = cityHash64(a)) ENGINE = MergeTree() ORDER BY i;
|
||||
INSERT INTO column_swap_test_test VALUES (1, 'cat', 1), (2, 'dog', 2);
|
||||
INSERT INTO column_swap_test_test SELECT number AS i, format('test {} kek {}', toString(number), toString(number + 10)) AS a, 1 AS b FROM system.numbers LIMIT 1000000;
|
||||
|
||||
EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 1;
|
||||
EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 0;
|
||||
EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 0;
|
||||
EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 1;
|
||||
|
||||
EXPLAIN SYNTAX SELECT cityHash64(a) + 10 FROM column_swap_test_test WHERE cityHash64(a) = 0;
|
||||
EXPLAIN SYNTAX SELECT cityHash64(a) + 10, a FROM column_swap_test_test WHERE cityHash64(a) = 0;
|
||||
EXPLAIN SYNTAX SELECT b + 10, a FROM column_swap_test_test WHERE b = 0;
|
||||
|
||||
DROP TABLE column_swap_test_test;
|
||||
|
||||
CREATE TABLE column_swap_test_test (i Int64, a String, b String, CONSTRAINT c1 ASSUME a = substring(reverse(b), 1, 1)) ENGINE = MergeTree() ORDER BY i;
|
||||
INSERT INTO column_swap_test_test SELECT number AS i, toString(number) AS a, format('test {} kek {}', toString(number), toString(number + 10)) b FROM system.numbers LIMIT 1000000;
|
||||
|
||||
EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1), a FROM column_swap_test_test WHERE a = 'c';
|
||||
EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1), a FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c';
|
||||
EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1) AS t1, a AS t2 FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c';
|
||||
EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1) FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c';
|
||||
|
||||
DROP TABLE column_swap_test_test;
|
||||
|
||||
DROP TABLE IF EXISTS t_bad_constraint;
|
||||
|
||||
CREATE TABLE t_bad_constraint(a UInt32, s String, CONSTRAINT c1 ASSUME a = toUInt32(s)) ENGINE = MergeTree ORDER BY tuple();
|
||||
|
||||
INSERT INTO t_bad_constraint SELECT number, randomPrintableASCII(100) FROM numbers(10000);
|
||||
|
||||
EXPLAIN SYNTAX SELECT a FROM t_bad_constraint;
|
||||
|
||||
DROP TABLE t_bad_constraint;
|
16
tests/queries/0_stateless/01624_soft_constraints.reference
Normal file
16
tests/queries/0_stateless/01624_soft_constraints.reference
Normal file
@ -0,0 +1,16 @@
|
||||
"rows_read": 4,
|
||||
"rows_read": 2,
|
||||
"rows_read": 4,
|
||||
"rows_read": 2,
|
||||
"rows_read": 2,
|
||||
"rows_read": 2,
|
||||
"rows_read": 4,
|
||||
"rows_read": 2,
|
||||
"rows_read": 4,
|
||||
"rows_read": 4,
|
||||
"rows_read": 1,
|
||||
"rows_read": 4,
|
||||
"rows_read": 3,
|
||||
"rows_read": 4,
|
||||
"rows_read": 4,
|
||||
"rows_read": 3,
|
96
tests/queries/0_stateless/01624_soft_constraints.sh
Executable file
96
tests/queries/0_stateless/01624_soft_constraints.sh
Executable file
@ -0,0 +1,96 @@
|
||||
#!/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; SET optimize_substitute_columns = 1; SET optimize_append_index = 1"
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="
|
||||
$SETTINGS;
|
||||
DROP TABLE IF EXISTS hypothesis_test_test;
|
||||
DROP TABLE IF EXISTS hypothesis_test_test2;
|
||||
DROP TABLE IF EXISTS hypothesis_test_test3;
|
||||
|
||||
CREATE TABLE hypothesis_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 -n --query="$SETTINGS;
|
||||
INSERT INTO hypothesis_test_test VALUES (0, 1, 2, 2), (1, 2, 1, 2), (2, 2, 2, 1), (3, 1, 2, 3)"
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test WHERE b > a FORMAT JSON" | grep "rows_read" # 4
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test WHERE b <= a FORMAT JSON" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test WHERE b >= a FORMAT JSON" | grep "rows_read" # 4
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test WHERE b = a FORMAT JSON" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test WHERE c < a FORMAT JSON" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test WHERE c = a FORMAT JSON" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test WHERE c > a FORMAT JSON" | grep "rows_read" # 4
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test WHERE c < a FORMAT JSON" | grep "rows_read"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="
|
||||
$SETTINGS;
|
||||
CREATE TABLE hypothesis_test_test2 (
|
||||
i UInt64,
|
||||
a UInt64,
|
||||
b UInt64,
|
||||
INDEX t (a != b) TYPE hypothesis GRANULARITY 1
|
||||
) ENGINE = MergeTree() ORDER BY i SETTINGS index_granularity=1;
|
||||
"
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="$SETTINGS;
|
||||
INSERT INTO hypothesis_test_test2 VALUES (0, 1, 2), (1, 2, 1), (2, 2, 2), (3, 1, 0)"
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test2 WHERE a < b FORMAT JSON" | grep "rows_read" # 4
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test2 WHERE a <= b FORMAT JSON" | grep "rows_read" # 4
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test2 WHERE a = b FORMAT JSON" | grep "rows_read" # 1
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test2 WHERE a != b FORMAT JSON" | grep "rows_read" # 4
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="
|
||||
$SETTINGS;
|
||||
CREATE TABLE hypothesis_test_test3 (
|
||||
i UInt64,
|
||||
a UInt64,
|
||||
b UInt64,
|
||||
INDEX t (a = b) TYPE hypothesis GRANULARITY 1
|
||||
) ENGINE = MergeTree() ORDER BY i SETTINGS index_granularity=1;
|
||||
"
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="
|
||||
$SETTINGS;
|
||||
INSERT INTO hypothesis_test_test3 VALUES (0, 1, 2), (1, 2, 1), (2, 2, 2), (3, 1, 0)"
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="$SETTINGS;SELECT count() FROM hypothesis_test_test3 WHERE a < b FORMAT JSON" | grep "rows_read" # 3
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="$SETTINGS;SELECT count() FROM hypothesis_test_test3 WHERE a <= b FORMAT JSON" | grep "rows_read" # 4
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="$SETTINGS;SELECT count() FROM hypothesis_test_test3 WHERE a = b FORMAT JSON" | grep "rows_read" # 4
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="$SETTINGS;SELECT count() FROM hypothesis_test_test3 WHERE a != b FORMAT JSON" | grep "rows_read" # 3
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="
|
||||
$SETTINGS;
|
||||
DROP TABLE hypothesis_test_test;
|
||||
DROP TABLE hypothesis_test_test2;
|
||||
DROP TABLE hypothesis_test_test3;"
|
@ -0,0 +1,15 @@
|
||||
SELECT i AS i
|
||||
FROM index_append_test_test
|
||||
PREWHERE a = 0
|
||||
WHERE (a = 0) AND indexHint((i + 40) > 0)
|
||||
SELECT i AS i
|
||||
FROM index_append_test_test
|
||||
PREWHERE a < 0
|
||||
SELECT i AS i
|
||||
FROM index_append_test_test
|
||||
PREWHERE a >= 0
|
||||
WHERE (a >= 0) AND indexHint((i + 40) > 0)
|
||||
SELECT i AS i
|
||||
FROM index_append_test_test
|
||||
PREWHERE (2 * b) < 100
|
||||
WHERE ((2 * b) < 100) AND indexHint(i < 100)
|
17
tests/queries/0_stateless/01625_constraints_index_append.sql
Normal file
17
tests/queries/0_stateless/01625_constraints_index_append.sql
Normal file
@ -0,0 +1,17 @@
|
||||
SET convert_query_to_cnf = 1;
|
||||
SET optimize_using_constraints = 1;
|
||||
SET optimize_move_to_prewhere = 1;
|
||||
SET optimize_substitute_columns = 1;
|
||||
SET optimize_append_index = 1;
|
||||
|
||||
DROP TABLE IF EXISTS index_append_test_test;
|
||||
|
||||
CREATE TABLE index_append_test_test (i Int64, a UInt32, b UInt64, CONSTRAINT c1 ASSUME i <= 2 * b AND i + 40 > a) ENGINE = MergeTree() ORDER BY i;
|
||||
INSERT INTO index_append_test_test VALUES (1, 10, 1), (2, 20, 2);
|
||||
|
||||
EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a = 0;
|
||||
EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a < 0;
|
||||
EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a >= 0;
|
||||
EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE 2 * b < 100;
|
||||
|
||||
DROP TABLE index_append_test_test;
|
73
tests/queries/0_stateless/01626_cnf_fuzz_long.python
Normal file
73
tests/queries/0_stateless/01626_cnf_fuzz_long.python
Normal file
@ -0,0 +1,73 @@
|
||||
#!/usr/bin/env python3
|
||||
import os
|
||||
from random import randint, choices
|
||||
import sys
|
||||
|
||||
CURDIR = os.path.dirname(os.path.realpath(__file__))
|
||||
sys.path.insert(0, os.path.join(CURDIR, 'helpers'))
|
||||
|
||||
from pure_http_client import ClickHouseClient
|
||||
|
||||
client = ClickHouseClient()
|
||||
|
||||
N = 10
|
||||
create_query = "CREATE TABLE t_cnf_fuzz(" + ", ".join([f"c{i} UInt8" for i in range(N)]) + ") ENGINE = Memory"
|
||||
|
||||
client.query("DROP TABLE IF EXISTS t_cnf_fuzz")
|
||||
client.query(create_query)
|
||||
|
||||
# Insert all possible combinations of bool columns.
|
||||
insert_query = "INSERT INTO t_cnf_fuzz VALUES "
|
||||
for i in range(2**N):
|
||||
values = []
|
||||
cur = i
|
||||
for _ in range(N):
|
||||
values.append(cur % 2)
|
||||
cur //= 2
|
||||
|
||||
insert_query += "(" + ", ".join(map(lambda x: str(x), values)) + ")"
|
||||
|
||||
client.query(insert_query)
|
||||
|
||||
# Let's try to covert DNF to CNF,
|
||||
# because it's a worst case in a sense.
|
||||
|
||||
MAX_CLAUSES = 10
|
||||
MAX_ATOMS = 5
|
||||
|
||||
def generate_dnf():
|
||||
clauses = []
|
||||
num_clauses = randint(1, MAX_CLAUSES)
|
||||
for _ in range(num_clauses):
|
||||
num_atoms = randint(1, MAX_ATOMS)
|
||||
atom_ids = choices(range(N), k=num_atoms)
|
||||
negates = choices([0, 1], k=num_atoms)
|
||||
atoms = [f"(NOT c{i})" if neg else f"c{i}" for (i, neg) in zip(atom_ids, negates)]
|
||||
clauses.append("(" + " AND ".join(atoms) + ")")
|
||||
|
||||
return " OR ".join(clauses)
|
||||
|
||||
select_query = "SELECT count() FROM t_cnf_fuzz WHERE {} SETTINGS convert_query_to_cnf = {}"
|
||||
|
||||
fail_report = """
|
||||
Failed query: '{}'.
|
||||
Result without optimization: {}.
|
||||
Result with optimization: {}.
|
||||
"""
|
||||
|
||||
T = 500
|
||||
for _ in range(T):
|
||||
condition = generate_dnf()
|
||||
|
||||
query = select_query.format(condition, 0)
|
||||
res = client.query(query).strip()
|
||||
|
||||
query_cnf = select_query.format(condition, 1)
|
||||
res_cnf = client.query(query_cnf).strip()
|
||||
|
||||
if res != res_cnf:
|
||||
print(fail_report.format(query_cnf, res, res_cnf))
|
||||
exit(1)
|
||||
|
||||
client.query("DROP TABLE t_cnf_fuzz")
|
||||
print("OK")
|
1
tests/queries/0_stateless/01626_cnf_fuzz_long.reference
Normal file
1
tests/queries/0_stateless/01626_cnf_fuzz_long.reference
Normal file
@ -0,0 +1 @@
|
||||
OK
|
10
tests/queries/0_stateless/01626_cnf_fuzz_long.sh
Executable file
10
tests/queries/0_stateless/01626_cnf_fuzz_long.sh
Executable file
@ -0,0 +1,10 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest, long
|
||||
# Tag no-fasttest: Require python libraries like scipy, pandas and numpy
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
# We should have correct env vars from shell_config.sh to run this test
|
||||
python3 "$CURDIR"/01626_cnf_fuzz_long.python
|
18
tests/queries/0_stateless/01626_cnf_test.reference
Normal file
18
tests/queries/0_stateless/01626_cnf_test.reference
Normal 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 > 2) OR (i > 5) OR (i > 3)) AND ((i > 2) OR (i > 5) OR (i > 4)) AND ((i > 2) OR (i > 6) OR (i > 3)) AND ((i > 2) OR (i > 6) OR (i > 4)) AND ((i > 1) OR (i > 5) OR (i > 3)) AND ((i > 1) OR (i > 5) OR (i > 4)) AND ((i > 1) OR (i > 6) OR (i > 3)) AND ((i > 1) OR (i > 6) OR (i > 4))
|
||||
SELECT i
|
||||
FROM cnf_test
|
||||
WHERE ((i <= 3) OR (i <= 2) OR (i <= 5)) AND ((i <= 3) OR (i <= 2) OR (i <= 6)) AND ((i <= 3) OR (i <= 5) OR (i <= 1)) AND ((i <= 3) OR (i <= 6) OR (i <= 1)) AND ((i <= 2) OR (i <= 5) OR (i <= 4)) AND ((i <= 2) OR (i <= 6) OR (i <= 4)) AND ((i <= 5) OR (i <= 1) OR (i <= 4)) AND ((i <= 6) OR (i <= 1) OR (i <= 4))
|
||||
SELECT i
|
||||
FROM cnf_test
|
||||
WHERE ((i > 2) OR (i > 5) OR (i > 3)) AND ((i > 2) OR (i > 5) OR (i > 4)) AND ((i > 2) OR (i > 5) OR (i > 8)) AND ((i > 2) OR (i > 6) OR (i > 3)) AND ((i > 2) OR (i > 6) OR (i > 4)) AND ((i > 2) OR (i > 6) OR (i > 8)) AND ((i > 1) OR (i > 5) OR (i > 3)) AND ((i > 1) OR (i > 5) OR (i > 4)) AND ((i > 1) OR (i > 5) OR (i > 8)) AND ((i > 1) OR (i > 6) OR (i > 3)) AND ((i > 1) OR (i > 6) OR (i > 4)) AND ((i > 1) OR (i > 6) OR (i > 8)) AND ((i > 5) OR (i > 3) OR (i > 7)) AND ((i > 5) OR (i > 4) OR (i > 7)) AND ((i > 5) OR (i > 8) OR (i > 7)) AND ((i > 6) OR (i > 3) OR (i > 7)) AND ((i > 6) OR (i > 4) OR (i > 7)) AND ((i > 6) OR (i > 8) OR (i > 7))
|
||||
SELECT i
|
||||
FROM cnf_test
|
||||
WHERE ((i > 2) OR (i > 1) OR (i > 7)) AND (i <= 5) AND (i <= 6) AND ((i > 3) OR (i > 4) OR (i > 8))
|
18
tests/queries/0_stateless/01626_cnf_test.sql
Normal file
18
tests/queries/0_stateless/01626_cnf_test.sql
Normal file
@ -0,0 +1,18 @@
|
||||
SET convert_query_to_cnf = 1;
|
||||
|
||||
DROP TABLE IF EXISTS cnf_test;
|
||||
|
||||
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));
|
||||
|
||||
DROP TABLE cnf_test;
|
@ -1,6 +1,6 @@
|
||||
import os
|
||||
import io
|
||||
import sys
|
||||
import os
|
||||
import io
|
||||
import sys
|
||||
import requests
|
||||
import time
|
||||
import pandas as pd
|
||||
@ -8,6 +8,7 @@ import pandas as pd
|
||||
CLICKHOUSE_HOST = os.environ.get('CLICKHOUSE_HOST', '127.0.0.1')
|
||||
CLICKHOUSE_PORT_HTTP = os.environ.get('CLICKHOUSE_PORT_HTTP', '8123')
|
||||
CLICKHOUSE_SERVER_URL_STR = 'http://' + ':'.join(str(s) for s in [CLICKHOUSE_HOST, CLICKHOUSE_PORT_HTTP]) + "/"
|
||||
CLICKHOUSE_DATABASE = os.environ.get('CLICKHOUSE_DATABASE', 'test')
|
||||
|
||||
class ClickHouseClient:
|
||||
def __init__(self, host = CLICKHOUSE_SERVER_URL_STR):
|
||||
@ -19,8 +20,12 @@ class ClickHouseClient:
|
||||
|
||||
for i in range(NUMBER_OF_TRIES):
|
||||
r = requests.post(
|
||||
self.host,
|
||||
params = {'timeout_before_checking_execution_speed': 120, 'max_execution_time': 6000},
|
||||
self.host,
|
||||
params = {
|
||||
'timeout_before_checking_execution_speed': 120,
|
||||
'max_execution_time': 6000,
|
||||
'database': CLICKHOUSE_DATABASE
|
||||
},
|
||||
timeout = connection_timeout,
|
||||
data = query)
|
||||
if r.status_code == 200:
|
||||
@ -35,7 +40,7 @@ class ClickHouseClient:
|
||||
raise ValueError(r.text)
|
||||
|
||||
def query_return_df(self, query, connection_timeout = 1500):
|
||||
data = self.query(query, connection_timeout)
|
||||
data = self.query(query, connection_timeout)
|
||||
df = pd.read_csv(io.StringIO(data), sep = '\t')
|
||||
return df
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user