ClickHouse/src/Storages/ConstraintsDescription.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

330 lines
11 KiB
C++
Raw Normal View History

#include <Storages/ConstraintsDescription.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/TreeRewriter.h>
#include <Parsers/formatAST.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ASTExpressionList.h>
2021-04-03 12:12:45 +00:00
#include <Parsers/ASTFunction.h>
#include <Core/Defines.h>
#include <Analyzer/QueryTreeBuilder.h>
#include <Analyzer/FunctionNode.h>
2023-03-16 14:57:07 +00:00
#include <Analyzer/TableNode.h>
#include <Analyzer/QueryNode.h>
#include <Analyzer/Passes/QueryAnalysisPass.h>
2023-03-16 14:57:07 +00:00
#include <Interpreters/Context.h>
namespace DB
{
2021-05-06 08:29:24 +00:00
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
String ConstraintsDescription::toString() const
{
if (constraints.empty())
return {};
ASTExpressionList list;
for (const auto & constraint : constraints)
list.children.push_back(constraint);
2023-07-19 18:02:09 +00:00
return serializeAST(list);
}
ConstraintsDescription ConstraintsDescription::parse(const String & str)
{
if (str.empty())
return {};
ConstraintsDescription res;
ParserConstraintDeclarationList parser;
ASTPtr list = parseQuery(parser, str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
for (const auto & constraint : list->children)
2020-05-12 11:26:44 +00:00
res.constraints.push_back(constraint);
return res;
}
2021-01-04 20:55:32 +00:00
ASTs ConstraintsDescription::filterConstraints(ConstraintType selection) const
2021-01-03 15:02:00 +00:00
{
2021-11-10 17:57:59 +00:00
const auto ast_to_decr_constraint_type = [](ASTConstraintDeclaration::Type constraint_type) -> UInt8
2021-01-04 20:55:32 +00:00
{
switch (constraint_type)
{
case ASTConstraintDeclaration::Type::CHECK:
2021-11-10 17:57:59 +00:00
return static_cast<UInt8>(ConstraintType::CHECK);
2021-01-04 20:55:32 +00:00
case ASTConstraintDeclaration::Type::ASSUME:
2021-11-10 17:57:59 +00:00
return static_cast<UInt8>(ConstraintType::ASSUME);
2021-01-04 20:55:32 +00:00
}
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown constraint type.");
2021-01-04 20:55:32 +00:00
};
2021-01-03 15:02:00 +00:00
ASTs res;
res.reserve(constraints.size());
for (const auto & constraint : constraints)
{
2021-11-10 17:57:59 +00:00
if ((ast_to_decr_constraint_type(constraint->as<ASTConstraintDeclaration>()->type) & static_cast<UInt8>(selection)) != 0)
2021-05-04 18:43:58 +00:00
{
2021-01-03 15:02:00 +00:00
res.push_back(constraint);
}
}
return res;
}
2021-04-26 14:19:18 +00:00
std::vector<std::vector<CNFQuery::AtomicFormula>> ConstraintsDescription::buildConstraintData() const
2021-04-03 12:12:45 +00:00
{
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())
2021-05-05 11:17:49 +00:00
.pullNotOutFunctions();
2021-05-04 18:43:58 +00:00
for (const auto & group : cnf.getStatements())
{
2021-04-03 12:12:45 +00:00
if (group.size() == 1)
constraint_data.push_back(*group.begin());
}
}
return constraint_data;
}
2023-03-17 13:38:01 +00:00
std::unique_ptr<ComparisonGraph<ASTPtr>> ConstraintsDescription::buildGraph() const
2021-04-03 12:12:45 +00:00
{
2021-11-10 17:57:59 +00:00
static const NameSet relations = { "equals", "less", "lessOrEquals", "greaterOrEquals", "greater" };
2021-04-03 12:12:45 +00:00
ASTs constraints_for_graph;
2021-04-03 12:12:45 +00:00
auto atomic_formulas = getAtomicConstraintData();
2021-05-06 08:29:24 +00:00
for (const auto & atomic_formula : atomic_formulas)
2021-04-03 12:12:45 +00:00
{
2021-05-06 08:29:24 +00:00
CNFQuery::AtomicFormula atom{atomic_formula.negative, atomic_formula.ast->clone()};
pushNotIn(atom);
auto * func = atom.ast->as<ASTFunction>();
if (func && relations.contains(func->name))
2021-04-03 12:12:45 +00:00
{
2021-11-10 17:57:59 +00:00
assert(!atom.negative);
2021-05-06 08:29:24 +00:00
constraints_for_graph.push_back(atom.ast);
2021-04-03 12:12:45 +00:00
}
}
2023-03-17 13:38:01 +00:00
return std::make_unique<ComparisonGraph<ASTPtr>>(constraints_for_graph);
2021-04-03 12:12:45 +00:00
}
ConstraintsExpressions ConstraintsDescription::getExpressions(const DB::ContextPtr context,
2019-05-25 14:07:45 +00:00
const DB::NamesAndTypesList & source_columns_) const
{
ConstraintsExpressions res;
res.reserve(constraints.size());
for (const auto & constraint : constraints)
{
2020-05-12 16:38:11 +00:00
auto * constraint_ptr = constraint->as<ASTConstraintDeclaration>();
2021-01-03 15:02:00 +00:00
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));
2021-01-03 15:02:00 +00:00
}
2019-05-25 14:07:45 +00:00
}
return res;
}
2023-03-17 13:38:01 +00:00
const ComparisonGraph<ASTPtr> & ConstraintsDescription::getGraph() const
2021-04-26 14:19:18 +00:00
{
return *graph;
}
const std::vector<std::vector<CNFQuery::AtomicFormula>> & ConstraintsDescription::getConstraintData() const
{
return cnf_constraints;
}
const ASTs & ConstraintsDescription::getConstraints() const
2021-04-26 14:19:18 +00:00
{
return constraints;
}
2021-05-05 11:17:49 +00:00
std::optional<ConstraintsDescription::AtomIds> ConstraintsDescription::getAtomIds(const ASTPtr & ast) const
{
const auto hash = ast->getTreeHash(/*ignore_aliases=*/ true);
2021-11-10 17:57:59 +00:00
auto it = ast_to_atom_ids.find(hash);
if (it != ast_to_atom_ids.end())
return it->second;
2021-05-05 11:17:49 +00:00
return std::nullopt;
}
std::vector<CNFQuery::AtomicFormula> ConstraintsDescription::getAtomsById(const ConstraintsDescription::AtomIds & ids) const
{
std::vector<CNFQuery::AtomicFormula> result;
for (const auto & id : ids)
2021-11-10 17:57:59 +00:00
result.push_back(cnf_constraints[id.group_id][id.atom_id]);
2021-05-05 11:17:49 +00:00
return result;
}
2023-03-16 14:57:07 +00:00
ConstraintsDescription::QueryTreeData ConstraintsDescription::getQueryTreeData(const ContextPtr & context, const QueryTreeNodePtr & table_node) const
{
2023-03-16 14:57:07 +00:00
QueryTreeData data;
std::vector<Analyzer::CNF::AtomicFormula> atomic_constraints_data;
QueryAnalysisPass pass(table_node);
for (const auto & constraint : filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE))
{
2023-03-16 14:57:07 +00:00
auto query_tree = buildQueryTree(constraint->as<ASTConstraintDeclaration>()->expr->ptr(), context);
pass.run(query_tree, context);
2023-03-16 14:57:07 +00:00
const auto cnf = Analyzer::CNF::toCNF(query_tree, context)
.pullNotOutFunctions(context);
for (const auto & group : cnf.getStatements())
{
data.cnf_constraints.emplace_back(group.begin(), group.end());
2023-03-16 14:57:07 +00:00
if (group.size() == 1)
atomic_constraints_data.emplace_back(*group.begin());
}
2023-03-16 14:57:07 +00:00
data.constraints.push_back(std::move(query_tree));
}
2023-03-16 14:57:07 +00:00
for (size_t i = 0; i < data.cnf_constraints.size(); ++i)
for (size_t j = 0; j < data.cnf_constraints[i].size(); ++j)
data.query_node_to_atom_ids[data.cnf_constraints[i][j].node_with_hash].push_back({i, j});
/// build graph
if (constraints.empty())
{
data.graph = std::make_unique<ComparisonGraph<QueryTreeNodePtr>>(QueryTreeNodes(), context);
}
else
{
static const NameSet relations = { "equals", "less", "lessOrEquals", "greaterOrEquals", "greater" };
QueryTreeNodes constraints_for_graph;
for (const auto & atomic_formula : atomic_constraints_data)
{
2023-03-16 14:57:07 +00:00
Analyzer::CNF::AtomicFormula atom{atomic_formula.negative, atomic_formula.node_with_hash.node->clone()};
atom = Analyzer::CNF::pushNotIntoFunction(atom, context);
2023-03-16 14:57:07 +00:00
auto * function_node = atom.node_with_hash.node->as<FunctionNode>();
if (function_node && relations.contains(function_node->getFunctionName()))
{
2023-03-16 14:57:07 +00:00
assert(!atom.negative);
constraints_for_graph.push_back(atom.node_with_hash.node);
}
}
2023-03-16 14:57:07 +00:00
data.graph = std::make_unique<ComparisonGraph<QueryTreeNodePtr>>(constraints_for_graph, context);
}
2023-03-16 14:57:07 +00:00
return data;
}
2023-03-16 14:57:07 +00:00
const QueryTreeNodes & ConstraintsDescription::QueryTreeData::getConstraints() const
{
return constraints;
}
const std::vector<std::vector<Analyzer::CNF::AtomicFormula>> & ConstraintsDescription::QueryTreeData::getConstraintData() const
{
return cnf_constraints;
}
const ComparisonGraph<QueryTreeNodePtr> & ConstraintsDescription::QueryTreeData::getGraph() const
{
return *graph;
}
std::optional<ConstraintsDescription::AtomIds> ConstraintsDescription::QueryTreeData::getAtomIds(const QueryTreeNodePtrWithHash & node_with_hash) const
{
auto it = query_node_to_atom_ids.find(node_with_hash);
if (it != query_node_to_atom_ids.end())
return it->second;
return std::nullopt;
}
std::vector<Analyzer::CNF::AtomicFormula> ConstraintsDescription::QueryTreeData::getAtomsById(const AtomIds & ids) const
{
std::vector<Analyzer::CNF::AtomicFormula> result;
for (const auto & id : ids)
result.push_back(cnf_constraints[id.group_id][id.atom_id]);
return result;
}
2021-11-10 17:57:59 +00:00
ConstraintsDescription::ConstraintsDescription(const ASTs & constraints_)
: constraints(constraints_)
2021-04-26 14:19:18 +00:00
{
update();
}
2020-06-05 17:29:40 +00:00
ConstraintsDescription::ConstraintsDescription(const ConstraintsDescription & other)
{
constraints.reserve(other.constraints.size());
for (const auto & constraint : other.constraints)
constraints.emplace_back(constraint->clone());
2021-04-26 14:19:18 +00:00
update();
2020-06-05 17:29:40 +00:00
}
ConstraintsDescription & ConstraintsDescription::operator=(const ConstraintsDescription & other)
{
constraints.resize(other.constraints.size());
for (size_t i = 0; i < constraints.size(); ++i)
constraints[i] = other.constraints[i]->clone();
2021-04-26 14:19:18 +00:00
update();
2020-06-05 17:29:40 +00:00
return *this;
}
ConstraintsDescription::ConstraintsDescription(ConstraintsDescription && other) noexcept
: constraints(std::move(other.constraints))
{
update();
}
ConstraintsDescription & ConstraintsDescription::operator=(ConstraintsDescription && other) noexcept
{
constraints = std::move(other.constraints);
update();
return *this;
}
2021-04-26 14:19:18 +00:00
void ConstraintsDescription::update()
{
2021-05-07 10:49:05 +00:00
if (constraints.empty())
{
cnf_constraints.clear();
ast_to_atom_ids.clear();
2023-03-17 13:38:01 +00:00
graph = std::make_unique<ComparisonGraph<ASTPtr>>(ASTs());
2021-05-07 10:49:05 +00:00
return;
}
2021-11-10 17:57:59 +00:00
2021-04-26 14:19:18 +00:00
cnf_constraints = buildConstraintData();
2021-05-05 11:17:49 +00:00
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(/*ignore_aliases=*/ true)].push_back({i, j});
2021-11-10 17:57:59 +00:00
2021-04-26 14:19:18 +00:00
graph = buildGraph();
}
}