mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 10:02:01 +00:00
fix
This commit is contained in:
parent
cdf78828da
commit
8e4dc3aa25
@ -38,8 +38,7 @@ ComparisonGraph::ComparisonGraph(const std::vector<ASTPtr> & atomic_formulas)
|
||||
{
|
||||
if (atomic_formulas.empty())
|
||||
return;
|
||||
static const std::unordered_map<std::string, Edge::Type> relation_to_enum =
|
||||
{
|
||||
static const std::unordered_map<std::string, Edge::Type> relation_to_enum = {
|
||||
{"equals", Edge::Type::EQUAL},
|
||||
{"less", Edge::Type::LESS},
|
||||
{"lessOrEquals", Edge::Type::LESS_OR_EQUAL},
|
||||
@ -51,16 +50,14 @@ ComparisonGraph::ComparisonGraph(const std::vector<ASTPtr> & atomic_formulas)
|
||||
const auto atom = ComparisonGraph::normalizeAtom(atom_raw);
|
||||
|
||||
const auto bad_term = std::numeric_limits<std::size_t>::max();
|
||||
auto get_index = [](const ASTPtr & ast, Graph & asts_graph) -> std::size_t
|
||||
{
|
||||
auto get_index = [](const ASTPtr & ast, Graph & asts_graph) -> std::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)
|
||||
{
|
||||
[ast](const ASTPtr & constraint_ast) {
|
||||
return constraint_ast->getTreeHash() == ast->getTreeHash()
|
||||
&& constraint_ast->getColumnName() == ast->getColumnName();
|
||||
}))
|
||||
@ -80,7 +77,7 @@ ComparisonGraph::ComparisonGraph(const std::vector<ASTPtr> & atomic_formulas)
|
||||
};
|
||||
|
||||
const auto * func = atom->as<ASTFunction>();
|
||||
if (func)
|
||||
if (func && func->arguments->children.size() == 2)
|
||||
{
|
||||
const size_t index_left = get_index(func->arguments->children[0], g);
|
||||
const size_t index_right = get_index(func->arguments->children[1], g);
|
||||
@ -98,8 +95,7 @@ ComparisonGraph::ComparisonGraph(const std::vector<ASTPtr> & atomic_formulas)
|
||||
}
|
||||
else if (func->name == "notEquals")
|
||||
{
|
||||
not_equal.emplace(index_left, index_right);
|
||||
not_equal.emplace(index_right, index_left);
|
||||
/// Do nothing.
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -108,8 +104,19 @@ ComparisonGraph::ComparisonGraph(const std::vector<ASTPtr> & atomic_formulas)
|
||||
graph = ComparisonGraph::BuildGraphFromAstsGraph(g);
|
||||
dists = ComparisonGraph::BuildDistsFromGraph(graph);
|
||||
std::tie(ast_const_lower_bound, ast_const_upper_bound) = buildConstBounds();
|
||||
}
|
||||
|
||||
for (const auto & atom_raw : atomic_formulas)
|
||||
{
|
||||
const auto atom = ComparisonGraph::normalizeAtom(atom_raw);
|
||||
if (const auto * func = atom->as<ASTFunction>(); func)
|
||||
{
|
||||
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());
|
||||
not_equal.emplace(index_left, index_right);
|
||||
not_equal.emplace(index_right, index_left);
|
||||
}
|
||||
}
|
||||
}
|
||||
/// returns {is less, is strict}
|
||||
/// {true, true} = <
|
||||
/// {true, false} = =<
|
||||
|
@ -23,7 +23,7 @@ void splitMultiLogic(ASTPtr & node)
|
||||
|
||||
if (func->arguments->children.size() > 2)
|
||||
{
|
||||
ASTPtr res = func->arguments->children.front()->clone();
|
||||
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());
|
||||
@ -32,6 +32,7 @@ void splitMultiLogic(ASTPtr & node)
|
||||
}
|
||||
|
||||
auto * new_func = node->as<ASTFunction>();
|
||||
Poco::Logger::get("new_func_children").information(std::to_string(new_func->arguments->children.size()));
|
||||
for (auto & child : new_func->arguments->children)
|
||||
splitMultiLogic(child);
|
||||
}
|
||||
@ -121,7 +122,11 @@ void pushOr(ASTPtr & query)
|
||||
|
||||
const auto * and_func = or_func->arguments->children[and_node_id]->as<ASTFunction>();
|
||||
if (and_func->arguments->children.size() != 2)
|
||||
{
|
||||
Poco::Logger::get("$$$$$").information(and_func->name);
|
||||
Poco::Logger::get("CHILDREN: ").information(std::to_string(and_func->arguments->children.size()));
|
||||
throw Exception("Bad AND function", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
auto a = or_func->arguments->children[other_node_id];
|
||||
auto b = and_func->arguments->children[0];
|
||||
|
@ -45,16 +45,24 @@ MatchState match(CNFQuery::AtomicFormula a, CNFQuery::AtomicFormula b)
|
||||
|
||||
bool checkIfGroupAlwaysTrueFullMatch(const CNFQuery::OrGroup & group, const ConstraintsDescription & constraints_description)
|
||||
{
|
||||
const auto & constraints_data = constraints_description.getConstraintData();
|
||||
std::vector<size_t> found(constraints_data.size(), 0);
|
||||
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)
|
||||
{
|
||||
for (const auto & constraint_atom : constraints_description.getAtomsById(constraint_atom_ids.value()))
|
||||
const auto constraint_atoms = constraints_description.getAtomsById(constraint_atom_ids.value());
|
||||
for (size_t i = 0; i < constraint_atoms.size(); ++i)
|
||||
{
|
||||
const auto match_result = match(constraint_atom, atom);
|
||||
if (match_result == MatchState::FULL_MATCH)
|
||||
return true;
|
||||
if (match(constraint_atoms[i], atom) == MatchState::FULL_MATCH)
|
||||
{
|
||||
if ((--found[(*constraint_atom_ids)[i].and_group]) == 0)
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -24,6 +24,7 @@ namespace
|
||||
const String COMPONENT = "__aorLwT30aH_comp";
|
||||
const String COMPONENT_SEPARATOR = "_";
|
||||
constexpr UInt64 COLUMN_PENALTY = 10 * 1024 * 1024;
|
||||
constexpr Int64 INDEX_PRICE = -1'000'000'000'000'000'000;
|
||||
|
||||
class ComponentMatcher
|
||||
{
|
||||
@ -88,10 +89,10 @@ void collectIdentifiers(const ASTPtr & ast, std::unordered_set<String> & identif
|
||||
|
||||
struct ColumnPrice
|
||||
{
|
||||
size_t compressed_size;
|
||||
size_t uncompressed_size;
|
||||
Int64 compressed_size;
|
||||
Int64 uncompressed_size;
|
||||
|
||||
ColumnPrice(const size_t compressed_size_, const size_t uncompressed_size_)
|
||||
ColumnPrice(const Int64 compressed_size_, const Int64 uncompressed_size_)
|
||||
: compressed_size(compressed_size_)
|
||||
, uncompressed_size(uncompressed_size_)
|
||||
{}
|
||||
@ -145,10 +146,16 @@ public:
|
||||
if (identifier && data.name_to_component_id.contains(identifier->name()))
|
||||
{
|
||||
const String & name = identifier->name();
|
||||
//Poco::Logger::get("NAME").information(name);
|
||||
const auto component_id = data.name_to_component_id.at(name);
|
||||
//Poco::Logger::get("COMP").information(std::to_string(component_id));
|
||||
auto new_ast = data.id_to_expression_map.at(component_id)->clone();
|
||||
//Poco::Logger::get("NEW_AST").information(new_ast->dumpTree());
|
||||
if (data.is_select)
|
||||
{
|
||||
new_ast->setAlias(data.old_name.at(name));
|
||||
//Poco::Logger::get("OLD").information(data.old_name.at(name));
|
||||
}
|
||||
ast = new_ast;
|
||||
}
|
||||
}
|
||||
@ -188,6 +195,7 @@ void bruteforce(
|
||||
{
|
||||
min_price = current_price;
|
||||
min_expressions = expressions_stack;
|
||||
//Poco::Logger::get("PRICE").information("UPDATE");
|
||||
}
|
||||
}
|
||||
else
|
||||
@ -290,7 +298,7 @@ void SubstituteColumnOptimizer::perform()
|
||||
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(0, 0);
|
||||
column_prices[column_name] = ColumnPrice(INDEX_PRICE, INDEX_PRICE);
|
||||
for (const auto & column_name : identifiers)
|
||||
column_prices[column_name] = ColumnPrice(0, 0);
|
||||
|
||||
@ -303,7 +311,7 @@ void SubstituteColumnOptimizer::perform()
|
||||
components_list.push_back(component);
|
||||
|
||||
std::vector<ASTPtr> expressions_stack;
|
||||
ColumnPrice min_price(std::numeric_limits<size_t>::max(), std::numeric_limits<size_t>::max());
|
||||
ColumnPrice min_price(std::numeric_limits<Int64>::max(), std::numeric_limits<Int64>::max());
|
||||
std::vector<ASTPtr> min_expressions;
|
||||
bruteforce(compare_graph,
|
||||
components_list,
|
||||
@ -314,9 +322,15 @@ void SubstituteColumnOptimizer::perform()
|
||||
min_price,
|
||||
min_expressions);
|
||||
|
||||
for (size_t i = 0; i < min_expressions.size(); ++i)
|
||||
for (size_t i = 0; i < components_list.size(); ++i)
|
||||
id_to_expression_map[components_list[i]] = min_expressions[i];
|
||||
|
||||
/*Poco::Logger::get("comp list").information("CL");
|
||||
for (const auto id : components_list)
|
||||
Poco::Logger::get("comp list").information(std::to_string(id));
|
||||
for (const auto & [k, v] : id_to_expression_map)
|
||||
Poco::Logger::get("id2expr").information(std::to_string(k) + " " + v->dumpTree());*/
|
||||
|
||||
auto process = [&](ASTPtr & ast, bool is_select)
|
||||
{
|
||||
SubstituteColumnVisitor::Data substitute_data{id_to_expression_map, name_to_component, old_name, is_select};
|
||||
|
@ -28,7 +28,7 @@
|
||||
0
|
||||
1
|
||||
1
|
||||
0
|
||||
1
|
||||
1
|
||||
0
|
||||
SELECT count()
|
||||
|
@ -91,7 +91,7 @@ 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
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user