ClickHouse/src/Interpreters/ComparisonGraph.cpp

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

803 lines
26 KiB
C++
Raw Normal View History

2021-03-04 12:11:43 +00:00
#include <Interpreters/ComparisonGraph.h>
2021-03-05 09:54:13 +00:00
#include <Parsers/IAST.h>
2021-03-05 12:46:42 +00:00
#include <Parsers/ASTLiteral.h>
2021-03-05 09:54:13 +00:00
#include <Parsers/ASTFunction.h>
2021-11-10 17:57:59 +00:00
#include <Parsers/queryToString.h>
2021-11-10 17:57:59 +00:00
#include <Common/FieldVisitorsAccurateComparison.h>
2021-03-04 12:11:43 +00:00
#include <Analyzer/FunctionNode.h>
#include <Analyzer/ConstantNode.h>
#include <Functions/FunctionFactory.h>
2021-03-04 12:11:43 +00:00
namespace DB
{
2021-11-10 17:57:59 +00:00
namespace ErrorCodes
{
extern const int VIOLATED_CONSTRAINT;
}
namespace
2021-03-04 12:11:43 +00:00
{
2021-11-10 17:57:59 +00:00
/// Make function a > b or a >= b
ASTPtr normalizeAtom(const ASTPtr & atom, ContextPtr)
2021-11-10 17:57:59 +00:00
{
static const std::map<std::string, std::string> inverse_relations =
{
{"lessOrEquals", "greaterOrEquals"},
{"less", "greater"},
2021-03-05 09:54:13 +00:00
};
ASTPtr res = atom->clone();
2021-11-10 17:57:59 +00:00
if (const auto * func = res->as<ASTFunction>())
2021-03-05 09:54:13 +00:00
{
2021-11-10 17:57:59 +00:00
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());
2021-03-05 09:54:13 +00:00
}
return res;
}
QueryTreeNodePtr normalizeAtom(const QueryTreeNodePtr & atom, const ContextPtr & context)
{
static const std::map<std::string, std::string> inverse_relations =
{
{"lessOrEquals", "greaterOrEquals"},
{"less", "greater"},
};
if (const auto * function_node = atom->as<FunctionNode>())
{
if (const auto it = inverse_relations.find(function_node->getFunctionName()); it != inverse_relations.end())
{
auto inverted_node = function_node->clone();
auto * inverted_function_node = inverted_node->as<FunctionNode>();
auto function_resolver = FunctionFactory::instance().get(it->second, context);
2023-03-16 14:57:07 +00:00
auto & arguments = inverted_function_node->getArguments().getNodes();
assert(arguments.size() == 2);
std::swap(arguments[0], arguments[1]);
inverted_function_node->resolveAsFunction(function_resolver);
return inverted_node;
}
}
return atom;
}
const FunctionNode * tryGetFunctionNode(const QueryTreeNodePtr & node)
{
return node->as<FunctionNode>();
}
const ASTFunction * tryGetFunctionNode(const ASTPtr & node)
{
return node->as<ASTFunction>();
}
std::string functionName(const QueryTreeNodePtr & node)
{
return node->as<FunctionNode &>().getFunctionName();
}
std::string functionName(const ASTPtr & node)
{
return node->as<ASTFunction &>().name;
}
const Field * tryGetConstantValue(const QueryTreeNodePtr & node)
{
if (const auto * constant = node->as<ConstantNode>())
return &constant->getValue();
return nullptr;
}
const Field * tryGetConstantValue(const ASTPtr & node)
{
if (const auto * constant = node->as<ASTLiteral>())
return &constant->value;
return nullptr;
}
template <typename Node>
const Field & getConstantValue(const Node & node)
{
const auto * constant = tryGetConstantValue(node);
assert(constant);
return *constant;
}
const auto & getNode(const Analyzer::CNF::AtomicFormula & atom)
{
return atom.node_with_hash.node;
}
const auto & getNode(const CNFQuery::AtomicFormula & atom)
{
return atom.ast;
}
std::string nodeToString(const ASTPtr & ast)
{
return queryToString(ast);
}
std::string nodeToString(const QueryTreeNodePtr & node)
{
return queryToString(node->toAST());
}
const auto & getArguments(const ASTFunction * function)
{
return function->arguments->children;
}
const auto & getArguments(const FunctionNode * function)
{
return function->getArguments().getNodes();
}
2021-11-10 17:57:59 +00:00
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); }
ComparisonGraphCompareResult functionNameToCompareResult(const std::string & name)
{
using enum ComparisonGraphCompareResult;
static const std::unordered_map<std::string, ComparisonGraphCompareResult> relation_to_compare =
{
{"equals", EQUAL},
{"notEquals", NOT_EQUAL},
{"less", LESS},
{"lessOrEquals", LESS_OR_EQUAL},
{"greaterOrEquals", GREATER_OR_EQUAL},
{"greater", GREATER},
};
const auto it = relation_to_compare.find(name);
return it == std::end(relation_to_compare) ? UNKNOWN : it->second;
}
ComparisonGraphCompareResult inverseCompareResult(ComparisonGraphCompareResult result)
{
using enum ComparisonGraphCompareResult;
static const std::unordered_map<ComparisonGraphCompareResult, ComparisonGraphCompareResult> inverse_relations =
{
{NOT_EQUAL, EQUAL},
{EQUAL, NOT_EQUAL},
{GREATER_OR_EQUAL, LESS},
{GREATER, LESS_OR_EQUAL},
{LESS, GREATER_OR_EQUAL},
{LESS_OR_EQUAL, GREATER},
{UNKNOWN, UNKNOWN},
};
return inverse_relations.at(result);
}
2021-11-10 17:57:59 +00:00
}
template <ComparisonGraphNodeType Node>
ComparisonGraph<Node>::ComparisonGraph(const NodeContainer & atomic_formulas, ContextPtr context)
2021-03-05 09:54:13 +00:00
{
2021-05-07 10:49:05 +00:00
if (atomic_formulas.empty())
return;
2021-11-10 17:57:59 +00:00
static const std::unordered_map<std::string, typename Edge::Type> relation_to_enum =
2021-05-10 10:27:47 +00:00
{
2021-11-10 17:57:59 +00:00
{"equals", Edge::EQUAL},
{"greater", Edge::GREATER},
{"greaterOrEquals", Edge::GREATER_OR_EQUAL},
2021-03-05 09:54:13 +00:00
};
2021-11-10 17:57:59 +00:00
/// 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).
2021-03-05 09:54:13 +00:00
Graph g;
2021-05-04 18:43:58 +00:00
for (const auto & atom_raw : atomic_formulas)
{
const auto atom = normalizeAtom(atom_raw, context);
2021-03-05 09:54:13 +00:00
auto get_index = [](const Node & node, Graph & nodes_graph) -> std::optional<size_t>
2021-05-10 10:27:47 +00:00
{
const auto it = nodes_graph.node_hash_to_component.find(Graph::getHash(node));
if (it != std::end(nodes_graph.node_hash_to_component))
2021-03-05 09:54:13 +00:00
{
if (!std::any_of(
std::cbegin(nodes_graph.vertices[it->second].nodes),
std::cend(nodes_graph.vertices[it->second].nodes),
[node](const Node & constraint_node)
2021-05-10 10:27:47 +00:00
{
if constexpr (with_ast)
return constraint_node->getTreeHash(/*ignore_aliases=*/ true) == node->getTreeHash(/*ignore_aliases=*/ true)
&& constraint_node->getColumnName() == node->getColumnName();
else
2023-03-16 14:57:07 +00:00
return constraint_node->isEqual(*node);
2021-03-05 09:54:13 +00:00
}))
{
2021-11-10 17:57:59 +00:00
return {};
2021-03-05 09:54:13 +00:00
}
return it->second;
}
else
{
nodes_graph.node_hash_to_component[Graph::getHash(node)] = nodes_graph.vertices.size();
nodes_graph.vertices.push_back(EqualComponent{{node}, std::nullopt});
nodes_graph.edges.emplace_back();
return nodes_graph.vertices.size() - 1;
2021-03-05 09:54:13 +00:00
}
};
const auto * function_node = tryGetFunctionNode(atom);
if (function_node)
2021-03-05 09:54:13 +00:00
{
const auto & arguments = getArguments(function_node);
if (arguments.size() == 2)
2021-05-04 10:47:23 +00:00
{
auto index_left = get_index(arguments[0], g);
auto index_right = get_index(arguments[1], g);
if (index_left && index_right)
2021-03-05 09:54:13 +00:00
{
if (const auto it = relation_to_enum.find(functionName(atom)); 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});
}
2021-03-05 09:54:13 +00:00
}
}
}
}
2021-11-10 17:57:59 +00:00
/// 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 = buildGraphFromNodesGraph(g);
2021-11-10 17:57:59 +00:00
dists = buildDistsFromGraph(graph);
std::tie(node_const_lower_bound, node_const_upper_bound) = buildConstBounds();
2021-03-05 09:54:13 +00:00
2021-11-10 17:57:59 +00:00
/// 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.
2021-05-08 09:19:18 +00:00
for (const auto & atom_raw : atomic_formulas)
{
const auto atom = normalizeAtom(atom_raw, context);
2021-11-10 17:57:59 +00:00
const auto * function_node = tryGetFunctionNode(atom);
if (function_node && not_equals_functions.contains(functionName(atom)))
2021-05-08 09:19:18 +00:00
{
const auto & arguments = getArguments(function_node);
if (arguments.size() == 2)
{
auto index_left = graph.node_hash_to_component.at(Graph::getHash(arguments[0]));
auto index_right = graph.node_hash_to_component.at(Graph::getHash(arguments[1]));
if (index_left == index_right)
{
throw Exception(ErrorCodes::VIOLATED_CONSTRAINT,
"Found expression '{}', but its arguments considered equal according to constraints",
nodeToString(atom));
}
2021-11-10 17:57:59 +00:00
not_equal.emplace(index_left, index_right);
not_equal.emplace(index_right, index_left);
}
2021-05-08 09:19:18 +00:00
}
}
}
2021-11-10 17:57:59 +00:00
template <ComparisonGraphNodeType Node>
ComparisonGraphCompareResult ComparisonGraph<Node>::pathToCompareResult(Path path, bool inverse)
2021-03-05 12:13:00 +00:00
{
2021-11-10 17:57:59 +00:00
switch (path)
2021-04-10 20:46:53 +00:00
{
case Path::GREATER: return inverse ? ComparisonGraphCompareResult::LESS : ComparisonGraphCompareResult::GREATER;
case Path::GREATER_OR_EQUAL: return inverse ? ComparisonGraphCompareResult::LESS_OR_EQUAL : ComparisonGraphCompareResult::GREATER_OR_EQUAL;
2021-04-10 20:46:53 +00:00
}
UNREACHABLE();
2021-11-10 17:57:59 +00:00
}
template <ComparisonGraphNodeType Node>
std::optional<typename ComparisonGraph<Node>::Path> ComparisonGraph<Node>::findPath(size_t start, size_t finish) const
2021-11-10 17:57:59 +00:00
{
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;
2021-03-05 12:13:00 +00:00
}
template <ComparisonGraphNodeType Node>
ComparisonGraphCompareResult ComparisonGraph<Node>::compare(const Node & left, const Node & right) const
2021-03-05 09:54:13 +00:00
{
size_t start = 0;
size_t finish = 0;
2021-11-10 17:57:59 +00:00
/// TODO: check full ast
const auto it_left = graph.node_hash_to_component.find(Graph::getHash(left));
const auto it_right = graph.node_hash_to_component.find(Graph::getHash(right));
2021-11-10 17:57:59 +00:00
if (it_left == std::end(graph.node_hash_to_component) || it_right == std::end(graph.node_hash_to_component))
2021-03-05 09:54:13 +00:00
{
auto result = ComparisonGraphCompareResult::UNKNOWN;
2021-03-05 09:54:13 +00:00
{
2021-11-10 17:57:59 +00:00
const auto left_bound = getConstLowerBound(left);
const auto right_bound = getConstUpperBound(right);
if (left_bound && right_bound)
2021-04-10 20:46:53 +00:00
{
2021-11-10 17:57:59 +00:00
if (greater(left_bound->first, right_bound->first))
result = ComparisonGraphCompareResult::GREATER;
2021-11-10 17:57:59 +00:00
else if (equals(left_bound->first, right_bound->first))
result = left_bound->second || right_bound->second
? ComparisonGraphCompareResult::GREATER : ComparisonGraphCompareResult::GREATER_OR_EQUAL;
2021-04-10 20:46:53 +00:00
}
2021-03-05 09:54:13 +00:00
}
{
2021-11-10 17:57:59 +00:00
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 = ComparisonGraphCompareResult::LESS;
2021-11-10 17:57:59 +00:00
else if (equals(left_bound->first, right_bound->first))
result = left_bound->second || right_bound->second
? ComparisonGraphCompareResult::LESS : ComparisonGraphCompareResult::LESS_OR_EQUAL;
2021-11-10 17:57:59 +00:00
}
2021-03-05 09:54:13 +00:00
}
2021-11-10 17:57:59 +00:00
return result;
}
else
{
start = it_left->second;
finish = it_right->second;
2021-03-05 09:54:13 +00:00
}
if (start == finish)
return ComparisonGraphCompareResult::EQUAL;
2021-03-05 09:54:13 +00:00
2021-11-10 17:57:59 +00:00
if (auto path = findPath(start, finish))
return pathToCompareResult(*path, /*inverse=*/ false);
2021-03-05 12:13:00 +00:00
2021-11-10 17:57:59 +00:00
if (auto path = findPath(finish, start))
return pathToCompareResult(*path, /*inverse=*/ true);
2021-03-05 12:13:00 +00:00
2021-05-04 10:47:23 +00:00
if (not_equal.contains({start, finish}))
return ComparisonGraphCompareResult::NOT_EQUAL;
2021-05-04 10:47:23 +00:00
return ComparisonGraphCompareResult::UNKNOWN;
2021-03-04 12:11:43 +00:00
}
template <ComparisonGraphNodeType Node>
bool ComparisonGraph<Node>::isPossibleCompare(ComparisonGraphCompareResult expected, const Node & left, const Node & right) const
2021-05-02 19:16:40 +00:00
{
const auto result = compare(left, right);
using enum ComparisonGraphCompareResult;
if (expected == UNKNOWN || result == UNKNOWN)
2021-05-02 19:16:40 +00:00
return true;
2021-11-10 17:57:59 +00:00
2021-05-02 19:16:40 +00:00
if (expected == result)
return true;
static const std::set<std::pair<ComparisonGraphCompareResult, ComparisonGraphCompareResult>> possible_pairs =
2021-11-10 17:57:59 +00:00
{
{EQUAL, LESS_OR_EQUAL},
{EQUAL, GREATER_OR_EQUAL},
{LESS_OR_EQUAL, LESS},
{LESS_OR_EQUAL, EQUAL},
{LESS_OR_EQUAL, NOT_EQUAL},
{GREATER_OR_EQUAL, GREATER},
{GREATER_OR_EQUAL, EQUAL},
{GREATER_OR_EQUAL, NOT_EQUAL},
{LESS, LESS},
{LESS, LESS_OR_EQUAL},
{LESS, NOT_EQUAL},
{GREATER, GREATER},
{GREATER, GREATER_OR_EQUAL},
{GREATER, NOT_EQUAL},
{NOT_EQUAL, LESS},
{NOT_EQUAL, GREATER},
{NOT_EQUAL, LESS_OR_EQUAL},
{NOT_EQUAL, GREATER_OR_EQUAL},
2021-05-02 19:16:40 +00:00
};
return possible_pairs.contains({expected, result});
}
template <ComparisonGraphNodeType Node>
bool ComparisonGraph<Node>::isAlwaysCompare(ComparisonGraphCompareResult expected, const Node & left, const Node & right) const
2021-05-02 19:16:40 +00:00
{
const auto result = compare(left, right);
using enum ComparisonGraphCompareResult;
if (expected == UNKNOWN || result == UNKNOWN)
2021-05-02 19:16:40 +00:00
return false;
2021-11-10 17:57:59 +00:00
2021-05-02 19:16:40 +00:00
if (expected == result)
return true;
static const std::set<std::pair<ComparisonGraphCompareResult, ComparisonGraphCompareResult>> possible_pairs =
2021-11-10 17:57:59 +00:00
{
{LESS_OR_EQUAL, LESS},
{LESS_OR_EQUAL, EQUAL},
{GREATER_OR_EQUAL, GREATER},
{GREATER_OR_EQUAL, EQUAL},
{NOT_EQUAL, GREATER},
{NOT_EQUAL, LESS},
2021-05-02 19:16:40 +00:00
};
return possible_pairs.contains({expected, result});
}
template <ComparisonGraphNodeType Node>
typename ComparisonGraph<Node>::NodeContainer ComparisonGraph<Node>::getEqual(const Node & node) const
2021-04-26 11:26:54 +00:00
{
const auto res = getComponentId(node);
2021-04-26 11:26:54 +00:00
if (!res)
return {};
else
return getComponent(res.value());
}
template <ComparisonGraphNodeType Node>
std::optional<size_t> ComparisonGraph<Node>::getComponentId(const Node & node) const
2021-03-04 12:11:43 +00:00
{
const auto hash_it = graph.node_hash_to_component.find(Graph::getHash(node));
if (hash_it == std::end(graph.node_hash_to_component))
2021-03-04 12:11:43 +00:00
return {};
2021-11-10 17:57:59 +00:00
2021-03-04 12:11:43 +00:00
const size_t index = hash_it->second;
if (std::any_of(
std::cbegin(graph.vertices[index].nodes),
std::cend(graph.vertices[index].nodes),
[node](const Node & constraint_node)
2021-04-26 11:26:54 +00:00
{
if constexpr (with_ast)
return constraint_node->getTreeHash(/*ignore_aliases=*/ true) == node->getTreeHash(/*ignore_aliases=*/ true)
&& constraint_node->getColumnName() == node->getColumnName();
else
return constraint_node->getTreeHash() == node->getTreeHash();
2021-05-04 18:43:58 +00:00
}))
{
2021-04-26 11:26:54 +00:00
return index;
2021-05-04 18:43:58 +00:00
}
else
{
2021-03-04 12:11:43 +00:00
return {};
}
}
template <ComparisonGraphNodeType Node>
bool ComparisonGraph<Node>::hasPath(size_t left, size_t right) const
2021-05-03 19:08:26 +00:00
{
2021-11-10 17:57:59 +00:00
return findPath(left, right) || findPath(right, left);
2021-05-03 19:08:26 +00:00
}
template <ComparisonGraphNodeType Node>
typename ComparisonGraph<Node>::NodeContainer ComparisonGraph<Node>::getComponent(size_t id) const
2021-04-26 11:26:54 +00:00
{
return graph.vertices[id].nodes;
2021-04-26 11:26:54 +00:00
}
template <ComparisonGraphNodeType Node>
bool ComparisonGraph<Node>::EqualComponent::hasConstant() const
2021-05-04 18:43:58 +00:00
{
2021-11-10 17:57:59 +00:00
return constant_index.has_value();
2021-04-10 15:47:50 +00:00
}
template <ComparisonGraphNodeType Node>
Node ComparisonGraph<Node>::EqualComponent::getConstant() const
2021-05-04 18:43:58 +00:00
{
2021-11-10 17:57:59 +00:00
assert(constant_index);
return nodes[*constant_index];
2021-04-10 15:47:50 +00:00
}
template <ComparisonGraphNodeType Node>
void ComparisonGraph<Node>::EqualComponent::buildConstants()
2021-05-04 18:43:58 +00:00
{
2021-11-10 17:57:59 +00:00
constant_index.reset();
for (size_t i = 0; i < nodes.size(); ++i)
2021-04-10 15:47:50 +00:00
{
if (tryGetConstantValue(nodes[i]) != nullptr)
2021-04-10 15:47:50 +00:00
{
constant_index = i;
return;
}
}
}
template <ComparisonGraphNodeType Node>
ComparisonGraphCompareResult ComparisonGraph<Node>::atomToCompareResult(const typename CNF::AtomicFormula & atom)
2021-11-10 17:57:59 +00:00
{
const auto & node = getNode(atom);
if (tryGetFunctionNode(node) != nullptr)
2021-11-10 17:57:59 +00:00
{
auto expected = functionNameToCompareResult(functionName(node));
2021-11-10 17:57:59 +00:00
if (atom.negative)
expected = inverseCompareResult(expected);
return expected;
}
return ComparisonGraphCompareResult::UNKNOWN;
2021-05-04 10:47:23 +00:00
}
template <ComparisonGraphNodeType Node>
std::optional<Node> ComparisonGraph<Node>::getEqualConst(const Node & node) const
2021-03-05 12:46:42 +00:00
{
const auto hash_it = graph.node_hash_to_component.find(Graph::getHash(node));
if (hash_it == std::end(graph.node_hash_to_component))
2021-03-05 12:46:42 +00:00
return std::nullopt;
2021-11-10 17:57:59 +00:00
2021-03-05 12:46:42 +00:00
const size_t index = hash_it->second;
2023-03-16 15:59:48 +00:00
if (!graph.vertices[index].hasConstant())
return std::nullopt;
if constexpr (with_ast)
return graph.vertices[index].getConstant();
else
{
const auto & constant = getConstantValue(graph.vertices[index].getConstant());
auto constant_node = std::make_shared<ConstantNode>(constant, node->getResultType());
return constant_node;
}
2021-03-05 12:46:42 +00:00
}
template <ComparisonGraphNodeType Node>
std::optional<std::pair<Field, bool>> ComparisonGraph<Node>::getConstUpperBound(const Node & node) const
2021-04-10 20:46:53 +00:00
{
if (const auto * constant = tryGetConstantValue(node))
return std::make_pair(*constant, false);
2021-11-10 17:57:59 +00:00
const auto it = graph.node_hash_to_component.find(Graph::getHash(node));
if (it == std::end(graph.node_hash_to_component))
2021-04-10 20:46:53 +00:00
return std::nullopt;
2021-11-10 17:57:59 +00:00
2021-04-10 20:46:53 +00:00
const size_t to = it->second;
const ssize_t from = node_const_upper_bound[to];
2021-04-10 20:46:53 +00:00
if (from == -1)
return std::nullopt;
2021-11-10 17:57:59 +00:00
return std::make_pair(getConstantValue(graph.vertices[from].getConstant()), dists.at({from, to}) == Path::GREATER);
2021-04-10 20:46:53 +00:00
}
template <ComparisonGraphNodeType Node>
std::optional<std::pair<Field, bool>> ComparisonGraph<Node>::getConstLowerBound(const Node & node) const
2021-04-10 20:46:53 +00:00
{
if (const auto * constant = tryGetConstantValue(node))
return std::make_pair(*constant, false);
2021-11-10 17:57:59 +00:00
const auto it = graph.node_hash_to_component.find(Graph::getHash(node));
if (it == std::end(graph.node_hash_to_component))
2021-04-10 20:46:53 +00:00
return std::nullopt;
2021-11-10 17:57:59 +00:00
2021-04-10 20:46:53 +00:00
const size_t from = it->second;
const ssize_t to = node_const_lower_bound[from];
2021-04-10 20:46:53 +00:00
if (to == -1)
2021-11-10 17:57:59 +00:00
return std::nullopt;
return std::make_pair(getConstantValue(graph.vertices[to].getConstant()), dists.at({from, to}) == Path::GREATER);
2021-04-10 20:46:53 +00:00
}
template <ComparisonGraphNodeType Node>
void ComparisonGraph<Node>::dfsOrder(const Graph & nodes_graph, size_t v, std::vector<bool> & visited, std::vector<size_t> & order)
2021-03-04 12:11:43 +00:00
{
2021-03-05 09:54:13 +00:00
visited[v] = true;
for (const auto & edge : nodes_graph.edges[v])
2021-03-05 09:54:13 +00:00
if (!visited[edge.to])
dfsOrder(nodes_graph, edge.to, visited, order);
2021-11-10 17:57:59 +00:00
2021-03-05 09:54:13 +00:00
order.push_back(v);
}
template <ComparisonGraphNodeType Node>
typename ComparisonGraph<Node>::Graph ComparisonGraph<Node>::reverseGraph(const Graph & nodes_graph)
2021-03-05 09:54:13 +00:00
{
Graph g;
g.node_hash_to_component = nodes_graph.node_hash_to_component;
g.vertices = nodes_graph.vertices;
2021-05-04 18:43:58 +00:00
g.edges.resize(g.vertices.size());
2021-11-10 17:57:59 +00:00
for (size_t v = 0; v < nodes_graph.vertices.size(); ++v)
for (const auto & edge : nodes_graph.edges[v])
2021-03-05 09:54:13 +00:00
g.edges[edge.to].push_back(Edge{edge.type, v});
2021-11-10 17:57:59 +00:00
return g;
2021-03-05 09:54:13 +00:00
}
template <ComparisonGraphNodeType Node>
std::vector<typename ComparisonGraph<Node>::NodeContainer> ComparisonGraph<Node>::getVertices() const
2021-04-28 17:35:51 +00:00
{
std::vector<NodeContainer> result;
2021-05-04 18:43:58 +00:00
for (const auto & vertex : graph.vertices)
2021-04-28 17:35:51 +00:00
{
result.emplace_back();
for (const auto & node : vertex.nodes)
result.back().push_back(node);
2021-04-28 17:35:51 +00:00
}
return result;
}
template <ComparisonGraphNodeType Node>
void ComparisonGraph<Node>::dfsComponents(
2021-11-10 17:57:59 +00:00
const Graph & reversed_graph, size_t v,
OptionalIndices & components, size_t component)
2021-03-05 09:54:13 +00:00
{
components[v] = component;
for (const auto & edge : reversed_graph.edges[v])
2021-11-10 17:57:59 +00:00
if (!components[edge.to])
dfsComponents(reversed_graph, edge.to, components, component);
2021-03-05 09:54:13 +00:00
}
template <ComparisonGraphNodeType Node>
typename ComparisonGraph<Node>::Graph ComparisonGraph<Node>::buildGraphFromNodesGraph(const Graph & nodes_graph)
2021-03-05 09:54:13 +00:00
{
2021-11-10 17:57:59 +00:00
/// Find strongly connected component by using 2 dfs traversals.
/// https://en.wikipedia.org/wiki/Kosaraju%27s_algorithm
const auto n = nodes_graph.vertices.size();
2021-03-05 09:54:13 +00:00
std::vector<size_t> order;
{
std::vector<bool> visited(n, false);
for (size_t v = 0; v < n; ++v)
{
if (!visited[v])
dfsOrder(nodes_graph, v, visited, order);
2021-03-05 09:54:13 +00:00
}
}
2021-11-10 17:57:59 +00:00
OptionalIndices components(n);
2021-03-05 09:54:13 +00:00
size_t component = 0;
{
const Graph reversed_graph = reverseGraph(nodes_graph);
2021-11-10 17:57:59 +00:00
for (auto it = order.rbegin(); it != order.rend(); ++it)
2021-03-05 09:54:13 +00:00
{
2021-11-10 17:57:59 +00:00
if (!components[*it])
2021-03-05 09:54:13 +00:00
{
2021-11-10 17:57:59 +00:00
dfsComponents(reversed_graph, *it, components, component);
2021-03-05 09:54:13 +00:00
++component;
}
}
}
Graph result;
2021-05-04 18:43:58 +00:00
result.vertices.resize(component);
2021-03-05 09:54:13 +00:00
result.edges.resize(component);
for (const auto & [hash, index] : nodes_graph.node_hash_to_component)
2021-03-05 09:54:13 +00:00
{
2021-11-10 17:57:59 +00:00
assert(components[index]);
result.node_hash_to_component[hash] = *components[index];
result.vertices[*components[index]].nodes.insert(
std::end(result.vertices[*components[index]].nodes),
std::begin(nodes_graph.vertices[index].nodes),
std::end(nodes_graph.vertices[index].nodes)); // asts_graph has only one ast per vertex
2021-03-05 09:54:13 +00:00
}
2021-04-10 15:47:50 +00:00
/// Calculate constants
2021-05-04 18:43:58 +00:00
for (auto & vertex : result.vertices)
2021-04-10 15:47:50 +00:00
vertex.buildConstants();
2021-03-05 09:54:13 +00:00
2021-11-10 17:57:59 +00:00
/// For each edge in initial graph, we add an edge between components in condensation graph.
2021-03-05 09:54:13 +00:00
for (size_t v = 0; v < n; ++v)
{
for (const auto & edge : nodes_graph.edges[v])
2021-11-10 17:57:59 +00:00
result.edges[*components[v]].push_back(Edge{edge.type, *components[edge.to]});
/// TODO: make edges unique (left most strict)
2021-03-05 09:54:13 +00:00
}
2021-11-10 17:57:59 +00:00
/// If we have constansts in two components, we can compare them and add and extra edge.
2021-05-04 18:43:58 +00:00
for (size_t v = 0; v < result.vertices.size(); ++v)
2021-04-10 15:47:50 +00:00
{
2021-05-04 18:43:58 +00:00
for (size_t u = 0; u < result.vertices.size(); ++u)
2021-04-10 15:47:50 +00:00
{
2021-11-10 17:57:59 +00:00
if (v != u && result.vertices[v].hasConstant() && result.vertices[u].hasConstant())
2021-04-10 15:47:50 +00:00
{
const auto & left = getConstantValue(result.vertices[v].getConstant());
const auto & right = getConstantValue(result.vertices[u].getConstant());
2021-04-10 15:47:50 +00:00
2021-11-10 17:57:59 +00:00
/// Only GREATER. Equal constant fields = equal literals so it was already considered above.
if (greater(left, right))
2021-11-10 17:57:59 +00:00
result.edges[v].push_back(Edge{Edge::GREATER, u});
2021-04-10 15:47:50 +00:00
}
}
}
2021-03-05 09:54:13 +00:00
return result;
2021-03-04 12:11:43 +00:00
}
template <ComparisonGraphNodeType Node>
std::map<std::pair<size_t, size_t>, typename ComparisonGraph<Node>::Path> ComparisonGraph<Node>::buildDistsFromGraph(const Graph & g)
2021-04-03 16:30:49 +00:00
{
2021-11-10 17:57:59 +00:00
/// Min path : -1 means GREATER, 0 means GREATER_OR_EQUALS.
/// We use FloydWarshall algorithm to find distances between all pairs of vertices.
/// https://en.wikipedia.org/wiki/FloydWarshall_algorithm
constexpr auto inf = std::numeric_limits<Int8>::max();
2021-05-06 08:29:24 +00:00
const size_t n = g.vertices.size();
2021-11-10 17:57:59 +00:00
std::vector<std::vector<Int8>> results(n, std::vector<Int8>(n, inf));
2021-04-03 16:30:49 +00:00
for (size_t v = 0; v < n; ++v)
2021-04-10 20:46:53 +00:00
{
results[v][v] = 0;
2021-04-03 16:30:49 +00:00
for (const auto & edge : g.edges[v])
2021-11-10 17:57:59 +00:00
results[v][edge.to] = std::min(results[v][edge.to], static_cast<Int8>(edge.type == Edge::GREATER ? -1 : 0));
2021-04-10 20:46:53 +00:00
}
2021-04-03 16:30:49 +00:00
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)
2021-11-10 17:57:59 +00:00
path[std::make_pair(v, u)] = (results[v][u] == -1 ? Path::GREATER : Path::GREATER_OR_EQUAL);
2021-05-04 10:47:23 +00:00
2021-04-03 16:30:49 +00:00
return path;
}
template <ComparisonGraphNodeType Node>
std::pair<std::vector<ssize_t>, std::vector<ssize_t>> ComparisonGraph<Node>::buildConstBounds() const
2021-04-10 20:46:53 +00:00
{
2021-05-04 18:43:58 +00:00
const size_t n = graph.vertices.size();
2021-04-10 20:46:53 +00:00
std::vector<ssize_t> lower(n, -1);
std::vector<ssize_t> upper(n, -1);
2021-05-04 18:43:58 +00:00
auto get_value = [this](const size_t vertex) -> Field
{
return getConstantValue(graph.vertices[vertex].getConstant());
2021-04-10 20:46:53 +00:00
};
for (const auto & [edge, path] : dists)
{
const auto [from, to] = edge;
2021-11-10 17:57:59 +00:00
2021-05-04 18:43:58 +00:00
if (graph.vertices[to].hasConstant())
{
2021-04-10 20:46:53 +00:00
if (lower[from] == -1
2021-11-10 17:57:59 +00:00
|| greater(get_value(to), get_value(lower[from]))
|| (equals(get_value(to), get_value(lower[from])) && path == Path::GREATER))
2021-04-10 20:46:53 +00:00
lower[from] = to;
}
2021-11-10 17:57:59 +00:00
2021-05-04 18:43:58 +00:00
if (graph.vertices[from].hasConstant())
{
2021-04-10 20:46:53 +00:00
if (upper[to] == -1
2021-11-10 17:57:59 +00:00
|| less(get_value(from), get_value(upper[to]))
|| (equals(get_value(from), get_value(upper[to])) && path == Path::GREATER))
2021-04-10 20:46:53 +00:00
upper[to] = from;
}
}
2023-03-16 14:57:07 +00:00
return {std::move(lower), std::move(upper)};
2021-04-10 20:46:53 +00:00
}
template class ComparisonGraph<ASTPtr>;
template class ComparisonGraph<QueryTreeNodePtr>;
2021-03-04 12:11:43 +00:00
}