Add support for substitute column

This commit is contained in:
Antonio Andelic 2023-03-17 13:38:01 +00:00
parent d6efe7fc21
commit 0fb9f9ffe6
12 changed files with 580 additions and 37 deletions

View File

@ -9,7 +9,13 @@
#include <Analyzer/Passes/CNF.h> #include <Analyzer/Passes/CNF.h>
#include <Analyzer/Utils.h> #include <Analyzer/Utils.h>
#include <Storages/IStorage.h>
#include <Functions/FunctionFactory.h> #include <Functions/FunctionFactory.h>
#include "Analyzer/HashUtils.h"
#include "Analyzer/IQueryTreeNode.h"
#include "Interpreters/ComparisonGraph.h"
#include "base/types.h"
namespace DB namespace DB
{ {
@ -344,6 +350,286 @@ void addIndexConstraint(Analyzer::CNF & cnf, const QueryTreeNodes & table_expres
} }
} }
struct ColumnPrice
{
Int64 compressed_size{0};
Int64 uncompressed_size{0};
ColumnPrice(const Int64 compressed_size_, const Int64 uncompressed_size_)
: compressed_size(compressed_size_)
, uncompressed_size(uncompressed_size_)
{
}
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>;
using ColumnPriceByQueryNode = QueryTreeNodePtrWithHashMap<ColumnPrice>;
class ComponentCollectorVisitor : public ConstInDepthQueryTreeVisitor<ComponentCollectorVisitor>
{
public:
ComponentCollectorVisitor(
std::set<UInt64> & components_,
QueryTreeNodePtrWithHashMap<UInt64> & query_node_to_component_,
const ComparisonGraph<QueryTreeNodePtr> & graph_)
: components(components_), query_node_to_component(query_node_to_component_), graph(graph_)
{}
void visitImpl(const QueryTreeNodePtr & node)
{
if (auto id = graph.getComponentId(node))
{
query_node_to_component.emplace(node, *id);
components.insert(*id);
}
}
private:
std::set<UInt64> & components;
QueryTreeNodePtrWithHashMap<UInt64> & query_node_to_component;
const ComparisonGraph<QueryTreeNodePtr> & graph;
};
class ColumnNameCollectorVisitor : public ConstInDepthQueryTreeVisitor<ColumnNameCollectorVisitor>
{
public:
ColumnNameCollectorVisitor(
std::unordered_set<std::string> & column_names_,
const QueryTreeNodePtrWithHashMap<UInt64> & query_node_to_component_)
: column_names(column_names_), query_node_to_component(query_node_to_component_)
{}
bool needChildVisit(const VisitQueryTreeNodeType & parent, const VisitQueryTreeNodeType &)
{
return !query_node_to_component.contains(parent);
}
void visitImpl(const QueryTreeNodePtr & node)
{
if (query_node_to_component.contains(node))
return;
if (const auto * column_node = node->as<ColumnNode>())
column_names.insert(column_node->getColumnName());
}
private:
std::unordered_set<std::string> & column_names;
const QueryTreeNodePtrWithHashMap<UInt64> & query_node_to_component;
};
class SubstituteColumnVisitor : public InDepthQueryTreeVisitor<SubstituteColumnVisitor>
{
public:
SubstituteColumnVisitor(
const QueryTreeNodePtrWithHashMap<UInt64> & query_node_to_component_,
const std::unordered_map<UInt64, QueryTreeNodePtr> & id_to_query_node_map_,
ContextPtr context_)
: query_node_to_component(query_node_to_component_), id_to_query_node_map(id_to_query_node_map_), context(std::move(context_))
{}
void visitImpl(QueryTreeNodePtr & node)
{
auto component_id_it = query_node_to_component.find(node);
if (component_id_it == query_node_to_component.end())
return;
const auto component_id = component_id_it->second;
auto new_node = id_to_query_node_map.at(component_id)->clone();
if (!node->getResultType()->equals(*new_node->getResultType()))
{
node = buildCastFunction(new_node, node->getResultType(), context);
return;
}
node = std::move(new_node);
}
private:
const QueryTreeNodePtrWithHashMap<UInt64> & query_node_to_component;
const std::unordered_map<UInt64, QueryTreeNodePtr> & id_to_query_node_map;
ContextPtr context;
};
ColumnPrice calculatePrice(
const ColumnPriceByName & column_prices,
const std::unordered_set<std::string> & column_names)
{
ColumnPrice result(0, 0);
for (const auto & column : column_names)
{
if (auto it = column_prices.find(column); it != column_prices.end())
result += it->second;
}
return result;
}
void bruteForce(
const ComparisonGraph<QueryTreeNodePtr> & graph,
const std::vector<UInt64> & components,
size_t current_component,
const ColumnPriceByName & column_prices,
ColumnPrice current_price,
std::vector<QueryTreeNodePtr> & expressions_stack,
ColumnPrice & min_price,
std::vector<QueryTreeNodePtr> & min_expressions)
{
if (current_component == components.size())
{
if (current_price < min_price)
{
min_price = current_price;
min_expressions = expressions_stack;
}
return;
}
for (const auto & node : graph.getComponent(components[current_component]))
{
std::unordered_set<std::string> column_names;
ColumnNameCollectorVisitor column_name_collector{column_names, {}};
column_name_collector.visit(node);
ColumnPrice expression_price = calculatePrice(column_prices, column_names);
expressions_stack.push_back(node);
current_price += expression_price;
ColumnPriceByName new_prices(column_prices);
for (const auto & column : column_names)
new_prices.insert_or_assign(column, 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();
}
}
void substituteColumns(QueryNode & query_node, const QueryTreeNodes & table_expressions, const ContextPtr & context)
{
static constexpr UInt64 COLUMN_PENALTY = 10 * 1024 * 1024;
static constexpr Int64 INDEX_PRICE = -1'000'000'000'000'000'000;
for (const auto & table_expression : table_expressions)
{
auto snapshot = getStorageSnapshot(table_expression);
if (!snapshot || !snapshot->metadata)
continue;
const auto column_sizes = snapshot->storage.getColumnSizes();
if (column_sizes.empty())
return;
auto query_tree_constraint = snapshot->metadata->getConstraints().getQueryTreeData(context, table_expression);
const auto & graph = query_tree_constraint.getGraph();
auto run_for_all = [&](const auto function)
{
function(query_node.getProjectionNode());
if (query_node.hasWhere())
function(query_node.getWhere());
if (query_node.hasPrewhere())
function(query_node.getPrewhere());
if (query_node.hasHaving())
function(query_node.getHaving());
};
std::set<UInt64> components;
QueryTreeNodePtrWithHashMap<UInt64> query_node_to_component;
std::unordered_set<std::string> column_names;
run_for_all([&](QueryTreeNodePtr & node)
{
ComponentCollectorVisitor component_collector{components, query_node_to_component, graph};
component_collector.visit(node);
ColumnNameCollectorVisitor column_name_collector{column_names, query_node_to_component};
column_name_collector.visit(node);
});
ColumnPriceByName column_prices;
const auto primary_key = snapshot->metadata->getColumnsRequiredForPrimaryKey();
for (const auto & [column_name, column_size] : column_sizes)
column_prices.insert_or_assign(column_name, ColumnPrice(column_size.data_compressed + COLUMN_PENALTY, column_size.data_uncompressed));
for (const auto & column_name : primary_key)
column_prices.insert_or_assign(column_name, ColumnPrice(INDEX_PRICE, INDEX_PRICE));
for (const auto & column_name : column_names)
column_prices.insert_or_assign(column_name, ColumnPrice(0, 0));
std::unordered_map<UInt64, QueryTreeNodePtr> id_to_query_node_map;
std::vector<UInt64> components_list;
for (const auto component_id : components)
{
auto component = graph.getComponent(component_id);
if (component.size() == 1)
id_to_query_node_map[component_id] = component.front();
else
components_list.push_back(component_id);
}
std::vector<QueryTreeNodePtr> expressions_stack;
ColumnPrice min_price(std::numeric_limits<Int64>::max(), std::numeric_limits<Int64>::max());
std::vector<QueryTreeNodePtr> min_expressions;
bruteForce(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_query_node_map[components_list[i]] = min_expressions[i];
SubstituteColumnVisitor substitute_column{query_node_to_component, id_to_query_node_map, context};
run_for_all([&](QueryTreeNodePtr & node)
{
substitute_column.visit(node);
});
}
}
void optimizeWithConstraints(Analyzer::CNF & cnf, const QueryTreeNodes & table_expressions, const ContextPtr & context) void optimizeWithConstraints(Analyzer::CNF & cnf, const QueryTreeNodes & table_expressions, const ContextPtr & context)
{ {
cnf.pullNotOutFunctions(context); cnf.pullNotOutFunctions(context);
@ -410,11 +696,24 @@ public:
auto table_expressions = extractTableExpressions(query_node->getJoinTree()); auto table_expressions = extractTableExpressions(query_node->getJoinTree());
if (query_node->hasWhere()) const auto & context = getContext();
optimizeNode(query_node->getWhere(), table_expressions, getContext()); const auto & settings = context->getSettingsRef();
if (query_node->hasPrewhere()) bool has_filter = false;
optimizeNode(query_node->getPrewhere(), table_expressions, getContext()); const auto optimize_filter = [&](QueryTreeNodePtr & filter_node)
{
if (filter_node == nullptr)
return;
optimizeNode(query_node->getWhere(), table_expressions, context);
has_filter = true;
};
optimize_filter(query_node->getWhere());
optimize_filter(query_node->getPrewhere());
if (has_filter && settings.optimize_substitute_columns)
substituteColumns(*query_node, table_expressions, context);
} }
}; };

View File

@ -104,7 +104,7 @@ namespace
/// we can add to expression 'indexHint(I < A)' condition. /// we can add to expression 'indexHint(I < A)' condition.
CNFQuery::OrGroup createIndexHintGroup( CNFQuery::OrGroup createIndexHintGroup(
const CNFQuery::OrGroup & group, const CNFQuery::OrGroup & group,
const ComparisonGraph<> & graph, const ComparisonGraph<ASTPtr> & graph,
const ASTs & primary_key_only_asts) const ASTs & primary_key_only_asts)
{ {
CNFQuery::OrGroup result; CNFQuery::OrGroup result;

View File

@ -33,7 +33,7 @@ concept ComparisonGraphNodeType = std::same_as<T, ASTPtr> || std::same_as<T, Que
* Graph of relations between terms in constraints. * Graph of relations between terms in constraints.
* Allows to compare terms and get equal terms. * Allows to compare terms and get equal terms.
*/ */
template <ComparisonGraphNodeType Node = ASTPtr> template <ComparisonGraphNodeType Node>
class ComparisonGraph class ComparisonGraph
{ {
public: public:

View File

@ -32,13 +32,13 @@ public:
struct Data struct Data
{ {
const ComparisonGraph<> & graph; const ComparisonGraph<ASTPtr> & graph;
std::set<UInt64> & components; std::set<UInt64> & components;
std::unordered_map<String, String> & old_name; std::unordered_map<String, String> & old_name;
std::unordered_map<String, UInt64> & component; std::unordered_map<String, UInt64> & component;
UInt64 & current_id; UInt64 & current_id;
Data(const ComparisonGraph<> & graph_, Data(const ComparisonGraph<ASTPtr> & graph_,
std::set<UInt64> & components_, std::set<UInt64> & components_,
std::unordered_map<String, String> & old_name_, std::unordered_map<String, String> & old_name_,
std::unordered_map<String, UInt64> & component_, std::unordered_map<String, UInt64> & component_,
@ -165,7 +165,7 @@ ColumnPrice calculatePrice(
/// price of all columns on which ast depends. /// price of all columns on which ast depends.
/// TODO: branch-and-bound /// TODO: branch-and-bound
void bruteforce( void bruteforce(
const ComparisonGraph<> & graph, const ComparisonGraph<ASTPtr> & graph,
const std::vector<UInt64> & components, const std::vector<UInt64> & components,
size_t current_component, size_t current_component,
const ColumnPriceByName & column_prices, const ColumnPriceByName & column_prices,

View File

@ -74,7 +74,7 @@ bool checkIfGroupAlwaysTrueFullMatch(const CNFQuery::OrGroup & group, const Cons
return false; return false;
} }
bool checkIfGroupAlwaysTrueGraph(const CNFQuery::OrGroup & group, const ComparisonGraph<> & graph) bool checkIfGroupAlwaysTrueGraph(const CNFQuery::OrGroup & group, const ComparisonGraph<ASTPtr> & graph)
{ {
/// We try to find at least one atom that is always true by using comparison graph. /// We try to find at least one atom that is always true by using comparison graph.
for (const auto & atom : group) for (const auto & atom : group)
@ -82,7 +82,7 @@ bool checkIfGroupAlwaysTrueGraph(const CNFQuery::OrGroup & group, const Comparis
const auto * func = atom.ast->as<ASTFunction>(); const auto * func = atom.ast->as<ASTFunction>();
if (func && func->arguments->children.size() == 2) if (func && func->arguments->children.size() == 2)
{ {
const auto expected = ComparisonGraph<>::atomToCompareResult(atom); const auto expected = ComparisonGraph<ASTPtr>::atomToCompareResult(atom);
if (graph.isAlwaysCompare(expected, func->arguments->children[0], func->arguments->children[1])) if (graph.isAlwaysCompare(expected, func->arguments->children[0], func->arguments->children[1]))
return true; return true;
} }
@ -108,20 +108,20 @@ bool checkIfAtomAlwaysFalseFullMatch(const CNFQuery::AtomicFormula & atom, const
return false; return false;
} }
bool checkIfAtomAlwaysFalseGraph(const CNFQuery::AtomicFormula & atom, const ComparisonGraph<> & graph) bool checkIfAtomAlwaysFalseGraph(const CNFQuery::AtomicFormula & atom, const ComparisonGraph<ASTPtr> & graph)
{ {
const auto * func = atom.ast->as<ASTFunction>(); const auto * func = atom.ast->as<ASTFunction>();
if (func && func->arguments->children.size() == 2) if (func && func->arguments->children.size() == 2)
{ {
/// TODO: special support for != /// TODO: special support for !=
const auto expected = ComparisonGraph<>::atomToCompareResult(atom); const auto expected = ComparisonGraph<ASTPtr>::atomToCompareResult(atom);
return !graph.isPossibleCompare(expected, func->arguments->children[0], func->arguments->children[1]); return !graph.isPossibleCompare(expected, func->arguments->children[0], func->arguments->children[1]);
} }
return false; return false;
} }
void replaceToConstants(ASTPtr & term, const ComparisonGraph<> & graph) void replaceToConstants(ASTPtr & term, const ComparisonGraph<ASTPtr> & graph)
{ {
const auto equal_constant = graph.getEqualConst(term); const auto equal_constant = graph.getEqualConst(term);
if (equal_constant) if (equal_constant)
@ -135,7 +135,7 @@ void replaceToConstants(ASTPtr & term, const ComparisonGraph<> & graph)
} }
} }
CNFQuery::AtomicFormula replaceTermsToConstants(const CNFQuery::AtomicFormula & atom, const ComparisonGraph<> & graph) CNFQuery::AtomicFormula replaceTermsToConstants(const CNFQuery::AtomicFormula & atom, const ComparisonGraph<ASTPtr> & graph)
{ {
CNFQuery::AtomicFormula result; CNFQuery::AtomicFormula result;
result.negative = atom.negative; result.negative = atom.negative;

View File

@ -9,11 +9,11 @@
using namespace DB; using namespace DB;
static ComparisonGraph<> getGraph(const String & query) static ComparisonGraph<ASTPtr> getGraph(const String & query)
{ {
ParserExpressionList parser(false); ParserExpressionList parser(false);
ASTPtr ast = parseQuery(parser, query, 0, 0); ASTPtr ast = parseQuery(parser, query, 0, 0);
return ComparisonGraph<>(ast->children); return ComparisonGraph<ASTPtr>(ast->children);
} }
TEST(ComparisonGraph, Bounds) TEST(ComparisonGraph, Bounds)

View File

@ -110,7 +110,7 @@ std::vector<CNFQuery::AtomicFormula> ConstraintsDescription::getAtomicConstraint
return constraint_data; return constraint_data;
} }
std::unique_ptr<ComparisonGraph<>> ConstraintsDescription::buildGraph() const std::unique_ptr<ComparisonGraph<ASTPtr>> ConstraintsDescription::buildGraph() const
{ {
static const NameSet relations = { "equals", "less", "lessOrEquals", "greaterOrEquals", "greater" }; static const NameSet relations = { "equals", "less", "lessOrEquals", "greaterOrEquals", "greater" };
@ -128,7 +128,7 @@ std::unique_ptr<ComparisonGraph<>> ConstraintsDescription::buildGraph() const
} }
} }
return std::make_unique<ComparisonGraph<>>(constraints_for_graph); return std::make_unique<ComparisonGraph<ASTPtr>>(constraints_for_graph);
} }
ConstraintsExpressions ConstraintsDescription::getExpressions(const DB::ContextPtr context, ConstraintsExpressions ConstraintsDescription::getExpressions(const DB::ContextPtr context,
@ -150,7 +150,7 @@ ConstraintsExpressions ConstraintsDescription::getExpressions(const DB::ContextP
return res; return res;
} }
const ComparisonGraph<> & ConstraintsDescription::getGraph() const const ComparisonGraph<ASTPtr> & ConstraintsDescription::getGraph() const
{ {
return *graph; return *graph;
} }
@ -313,7 +313,7 @@ void ConstraintsDescription::update()
{ {
cnf_constraints.clear(); cnf_constraints.clear();
ast_to_atom_ids.clear(); ast_to_atom_ids.clear();
graph = std::make_unique<ComparisonGraph<>>(ASTs()); graph = std::make_unique<ComparisonGraph<ASTPtr>>(ASTs());
return; return;
} }

View File

@ -43,7 +43,7 @@ public:
const std::vector<std::vector<CNFQuery::AtomicFormula>> & getConstraintData() const; const std::vector<std::vector<CNFQuery::AtomicFormula>> & getConstraintData() const;
std::vector<CNFQuery::AtomicFormula> getAtomicConstraintData() const; std::vector<CNFQuery::AtomicFormula> getAtomicConstraintData() const;
const ComparisonGraph<> & getGraph() const; const ComparisonGraph<ASTPtr> & getGraph() const;
ConstraintsExpressions getExpressions(ContextPtr context, const NamesAndTypesList & source_columns_) const; ConstraintsExpressions getExpressions(ContextPtr context, const NamesAndTypesList & source_columns_) const;
@ -79,7 +79,7 @@ public:
private: private:
std::vector<std::vector<CNFQuery::AtomicFormula>> buildConstraintData() const; std::vector<std::vector<CNFQuery::AtomicFormula>> buildConstraintData() const;
std::unique_ptr<ComparisonGraph<>> buildGraph() const; std::unique_ptr<ComparisonGraph<ASTPtr>> buildGraph() const;
void update(); void update();
ASTs constraints; ASTs constraints;
@ -87,7 +87,7 @@ private:
std::vector<std::vector<CNFQuery::AtomicFormula>> cnf_constraints; std::vector<std::vector<CNFQuery::AtomicFormula>> cnf_constraints;
std::map<IAST::Hash, AtomIds> ast_to_atom_ids; std::map<IAST::Hash, AtomIds> ast_to_atom_ids;
std::unique_ptr<ComparisonGraph<>> graph; std::unique_ptr<ComparisonGraph<ASTPtr>> graph;
}; };
} }

View File

@ -108,7 +108,7 @@ bool MergeTreeIndexhypothesisMergedCondition::alwaysUnknownOrTrue() const
func->name = "greaterOrEquals"; func->name = "greaterOrEquals";
} }
const auto weak_graph = std::make_unique<ComparisonGraph<>>(active_atomic_formulas); const auto weak_graph = std::make_unique<ComparisonGraph<ASTPtr>>(active_atomic_formulas);
bool useless = true; bool useless = true;
expression_cnf->iterateGroups( expression_cnf->iterateGroups(
@ -146,7 +146,7 @@ bool MergeTreeIndexhypothesisMergedCondition::mayBeTrueOnGranule(const MergeTree
values.push_back(granule->met); values.push_back(granule->met);
} }
const ComparisonGraph<> * graph = nullptr; const ComparisonGraph<ASTPtr> * graph = nullptr;
{ {
std::lock_guard lock(cache_mutex); std::lock_guard lock(cache_mutex);
@ -170,7 +170,7 @@ bool MergeTreeIndexhypothesisMergedCondition::mayBeTrueOnGranule(const MergeTree
const auto * func = atom.ast->as<ASTFunction>(); const auto * func = atom.ast->as<ASTFunction>();
if (func && func->arguments->children.size() == 2) if (func && func->arguments->children.size() == 2)
{ {
const auto expected = ComparisonGraph<>::atomToCompareResult(atom); const auto expected = ComparisonGraph<ASTPtr>::atomToCompareResult(atom);
if (graph->isPossibleCompare(expected, func->arguments->children[0], func->arguments->children[1])) if (graph->isPossibleCompare(expected, func->arguments->children[0], func->arguments->children[1]))
{ {
/// If graph failed use matching. /// If graph failed use matching.
@ -188,7 +188,7 @@ bool MergeTreeIndexhypothesisMergedCondition::mayBeTrueOnGranule(const MergeTree
return !always_false; return !always_false;
} }
std::unique_ptr<ComparisonGraph<>> MergeTreeIndexhypothesisMergedCondition::buildGraph(const std::vector<bool> & values) const std::unique_ptr<ComparisonGraph<ASTPtr>> MergeTreeIndexhypothesisMergedCondition::buildGraph(const std::vector<bool> & values) const
{ {
ASTs active_atomic_formulas(atomic_constraints); ASTs active_atomic_formulas(atomic_constraints);
for (size_t i = 0; i < values.size(); ++i) for (size_t i = 0; i < values.size(); ++i)
@ -199,10 +199,10 @@ std::unique_ptr<ComparisonGraph<>> MergeTreeIndexhypothesisMergedCondition::buil
std::begin(index_to_compare_atomic_hypotheses[i]), std::begin(index_to_compare_atomic_hypotheses[i]),
std::end(index_to_compare_atomic_hypotheses[i])); std::end(index_to_compare_atomic_hypotheses[i]));
} }
return std::make_unique<ComparisonGraph<>>(active_atomic_formulas); return std::make_unique<ComparisonGraph<ASTPtr>>(active_atomic_formulas);
} }
const ComparisonGraph<> * MergeTreeIndexhypothesisMergedCondition::getGraph(const std::vector<bool> & values) const const ComparisonGraph<ASTPtr> * MergeTreeIndexhypothesisMergedCondition::getGraph(const std::vector<bool> & values) const
{ {
auto [it, inserted] = graph_cache.try_emplace(values); auto [it, inserted] = graph_cache.try_emplace(values);
if (inserted) if (inserted)

View File

@ -20,8 +20,8 @@ public:
private: private:
void addConstraints(const ConstraintsDescription & constraints_description); void addConstraints(const ConstraintsDescription & constraints_description);
std::unique_ptr<ComparisonGraph<>> buildGraph(const std::vector<bool> & values) const; std::unique_ptr<ComparisonGraph<ASTPtr>> buildGraph(const std::vector<bool> & values) const;
const ComparisonGraph<> * getGraph(const std::vector<bool> & values) const; const ComparisonGraph<ASTPtr> * getGraph(const std::vector<bool> & values) const;
ASTPtr expression_ast; ASTPtr expression_ast;
std::unique_ptr<CNFQuery> expression_cnf; std::unique_ptr<CNFQuery> expression_cnf;
@ -29,7 +29,7 @@ private:
/// Part analysis can be done in parallel. /// Part analysis can be done in parallel.
/// So, we have shared answer and graph cache. /// So, we have shared answer and graph cache.
mutable std::mutex cache_mutex; mutable std::mutex cache_mutex;
mutable std::unordered_map<std::vector<bool>, std::unique_ptr<ComparisonGraph<>>> graph_cache; mutable std::unordered_map<std::vector<bool>, std::unique_ptr<ComparisonGraph<ASTPtr>>> graph_cache;
mutable std::unordered_map<std::vector<bool>, bool> answer_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<ASTPtr>> index_to_compare_atomic_hypotheses;

View File

@ -3,51 +3,287 @@ SELECT
(b AS b) + 3 AS `plus(b, 3)` (b AS b) + 3 AS `plus(b, 3)`
FROM column_swap_test_test FROM column_swap_test_test
WHERE b = 1 WHERE b = 1
QUERY id: 0
PROJECTION COLUMNS
plus(cityHash64(a), 10) UInt64
plus(b, 3) UInt64
PROJECTION
LIST id: 1, nodes: 2
FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64
ARGUMENTS
LIST id: 3, nodes: 2
COLUMN id: 4, column_name: b, result_type: UInt64, source_id: 5
CONSTANT id: 6, constant_value: UInt64_10, constant_value_type: UInt8
FUNCTION id: 7, function_name: plus, function_type: ordinary, result_type: UInt64
ARGUMENTS
LIST id: 8, nodes: 2
COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5
CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8
JOIN TREE
TABLE id: 5, table_name: default.column_swap_test_test
WHERE
FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
LIST id: 12, nodes: 2
COLUMN id: 13, column_name: b, result_type: UInt64, source_id: 5
CONSTANT id: 14, constant_value: UInt64_1, constant_value_type: UInt8
SELECT SELECT
(b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`,
(b AS b) + 3 AS `plus(b, 3)` (b AS b) + 3 AS `plus(b, 3)`
FROM column_swap_test_test FROM column_swap_test_test
WHERE b = 0 WHERE b = 0
QUERY id: 0
PROJECTION COLUMNS
plus(cityHash64(a), 10) UInt64
plus(b, 3) UInt64
PROJECTION
LIST id: 1, nodes: 2
FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64
ARGUMENTS
LIST id: 3, nodes: 2
COLUMN id: 4, column_name: b, result_type: UInt64, source_id: 5
CONSTANT id: 6, constant_value: UInt64_10, constant_value_type: UInt8
FUNCTION id: 7, function_name: plus, function_type: ordinary, result_type: UInt64
ARGUMENTS
LIST id: 8, nodes: 2
COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5
CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8
JOIN TREE
TABLE id: 5, table_name: default.column_swap_test_test
WHERE
FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
LIST id: 12, nodes: 2
COLUMN id: 13, column_name: b, result_type: UInt64, source_id: 5
CONSTANT id: 14, constant_value: UInt64_0, constant_value_type: UInt8
SELECT SELECT
(b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`,
(b AS b) + 3 AS `plus(b, 3)` (b AS b) + 3 AS `plus(b, 3)`
FROM column_swap_test_test FROM column_swap_test_test
WHERE b = 0 WHERE b = 0
QUERY id: 0
PROJECTION COLUMNS
plus(cityHash64(a), 10) UInt64
plus(b, 3) UInt64
PROJECTION
LIST id: 1, nodes: 2
FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64
ARGUMENTS
LIST id: 3, nodes: 2
COLUMN id: 4, column_name: b, result_type: UInt64, source_id: 5
CONSTANT id: 6, constant_value: UInt64_10, constant_value_type: UInt8
FUNCTION id: 7, function_name: plus, function_type: ordinary, result_type: UInt64
ARGUMENTS
LIST id: 8, nodes: 2
COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5
CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8
JOIN TREE
TABLE id: 5, table_name: default.column_swap_test_test
WHERE
FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
LIST id: 12, nodes: 2
COLUMN id: 13, column_name: b, result_type: UInt64, source_id: 5
CONSTANT id: 14, constant_value: UInt64_0, constant_value_type: UInt8
SELECT SELECT
(b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`,
(b AS b) + 3 AS `plus(b, 3)` (b AS b) + 3 AS `plus(b, 3)`
FROM column_swap_test_test FROM column_swap_test_test
WHERE b = 1 WHERE b = 1
QUERY id: 0
PROJECTION COLUMNS
plus(cityHash64(a), 10) UInt64
plus(b, 3) UInt64
PROJECTION
LIST id: 1, nodes: 2
FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64
ARGUMENTS
LIST id: 3, nodes: 2
COLUMN id: 4, column_name: b, result_type: UInt64, source_id: 5
CONSTANT id: 6, constant_value: UInt64_10, constant_value_type: UInt8
FUNCTION id: 7, function_name: plus, function_type: ordinary, result_type: UInt64
ARGUMENTS
LIST id: 8, nodes: 2
COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5
CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8
JOIN TREE
TABLE id: 5, table_name: default.column_swap_test_test
WHERE
FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
LIST id: 12, nodes: 2
COLUMN id: 13, column_name: b, result_type: UInt64, source_id: 5
CONSTANT id: 14, constant_value: UInt64_1, constant_value_type: UInt8
SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)` SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`
FROM column_swap_test_test FROM column_swap_test_test
WHERE b = 0 WHERE b = 0
QUERY id: 0
PROJECTION COLUMNS
plus(cityHash64(a), 10) UInt64
PROJECTION
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64
ARGUMENTS
LIST id: 3, nodes: 2
COLUMN id: 4, column_name: b, result_type: UInt64, source_id: 5
CONSTANT id: 6, constant_value: UInt64_10, constant_value_type: UInt8
JOIN TREE
TABLE id: 5, table_name: default.column_swap_test_test
WHERE
FUNCTION id: 7, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
LIST id: 8, nodes: 2
COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5
CONSTANT id: 10, constant_value: UInt64_0, constant_value_type: UInt8
SELECT SELECT
(cityHash64(a) AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, (cityHash64(a) AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`,
a AS a a AS a
FROM column_swap_test_test FROM column_swap_test_test
WHERE cityHash64(a) = 0 WHERE cityHash64(a) = 0
QUERY id: 0
PROJECTION COLUMNS
plus(cityHash64(a), 10) UInt64
a String
PROJECTION
LIST id: 1, nodes: 2
FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64
ARGUMENTS
LIST id: 3, nodes: 2
FUNCTION id: 4, function_name: cityHash64, function_type: ordinary, result_type: UInt64
ARGUMENTS
LIST id: 5, nodes: 1
COLUMN id: 6, column_name: a, result_type: String, source_id: 7
CONSTANT id: 8, constant_value: UInt64_10, constant_value_type: UInt8
COLUMN id: 9, column_name: a, result_type: String, source_id: 7
JOIN TREE
TABLE id: 7, table_name: default.column_swap_test_test
WHERE
FUNCTION id: 10, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
LIST id: 11, nodes: 2
FUNCTION id: 12, function_name: cityHash64, function_type: ordinary, result_type: UInt64
ARGUMENTS
LIST id: 13, nodes: 1
COLUMN id: 14, column_name: a, result_type: String, source_id: 7
CONSTANT id: 15, constant_value: UInt64_0, constant_value_type: UInt8
SELECT SELECT
(cityHash64(a) AS b) + 10 AS `plus(b, 10)`, (cityHash64(a) AS b) + 10 AS `plus(b, 10)`,
a AS a a AS a
FROM column_swap_test_test FROM column_swap_test_test
WHERE cityHash64(a) = 0 WHERE cityHash64(a) = 0
QUERY id: 0
PROJECTION COLUMNS
plus(b, 10) UInt64
a String
PROJECTION
LIST id: 1, nodes: 2
FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64
ARGUMENTS
LIST id: 3, nodes: 2
FUNCTION id: 4, function_name: cityHash64, function_type: ordinary, result_type: UInt64
ARGUMENTS
LIST id: 5, nodes: 1
COLUMN id: 6, column_name: a, result_type: String, source_id: 7
CONSTANT id: 8, constant_value: UInt64_10, constant_value_type: UInt8
COLUMN id: 9, column_name: a, result_type: String, source_id: 7
JOIN TREE
TABLE id: 7, table_name: default.column_swap_test_test
WHERE
FUNCTION id: 10, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
LIST id: 11, nodes: 2
FUNCTION id: 12, function_name: cityHash64, function_type: ordinary, result_type: UInt64
ARGUMENTS
LIST id: 13, nodes: 1
COLUMN id: 14, column_name: a, result_type: String, source_id: 7
CONSTANT id: 15, constant_value: UInt64_0, constant_value_type: UInt8
SELECT SELECT
a AS `substring(reverse(b), 1, 1)`, a AS `substring(reverse(b), 1, 1)`,
a AS a a AS a
FROM column_swap_test_test FROM column_swap_test_test
WHERE a = \'c\' WHERE a = \'c\'
QUERY id: 0
PROJECTION COLUMNS
substring(reverse(b), 1, 1) String
a String
PROJECTION
LIST id: 1, nodes: 2
COLUMN id: 2, column_name: a, result_type: String, source_id: 3
COLUMN id: 4, column_name: a, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.column_swap_test_test
WHERE
FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
LIST id: 6, nodes: 2
COLUMN id: 7, column_name: a, result_type: String, source_id: 3
CONSTANT id: 8, constant_value: \'c\', constant_value_type: String
SELECT SELECT
a AS `substring(reverse(b), 1, 1)`, a AS `substring(reverse(b), 1, 1)`,
a AS a a AS a
FROM column_swap_test_test FROM column_swap_test_test
WHERE a = \'c\' WHERE a = \'c\'
QUERY id: 0
PROJECTION COLUMNS
substring(reverse(b), 1, 1) String
a String
PROJECTION
LIST id: 1, nodes: 2
COLUMN id: 2, column_name: a, result_type: String, source_id: 3
COLUMN id: 4, column_name: a, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.column_swap_test_test
WHERE
FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
LIST id: 6, nodes: 2
COLUMN id: 7, column_name: a, result_type: String, source_id: 3
CONSTANT id: 8, constant_value: \'c\', constant_value_type: String
SELECT SELECT
a AS t1, a AS t1,
a AS t2 a AS t2
FROM column_swap_test_test FROM column_swap_test_test
WHERE a = \'c\' WHERE a = \'c\'
QUERY id: 0
PROJECTION COLUMNS
t1 String
t2 String
PROJECTION
LIST id: 1, nodes: 2
COLUMN id: 2, column_name: a, result_type: String, source_id: 3
COLUMN id: 4, column_name: a, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.column_swap_test_test
WHERE
FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
LIST id: 6, nodes: 2
COLUMN id: 7, column_name: a, result_type: String, source_id: 3
CONSTANT id: 8, constant_value: \'c\', constant_value_type: String
SELECT a AS `substring(reverse(b), 1, 1)` SELECT a AS `substring(reverse(b), 1, 1)`
FROM column_swap_test_test FROM column_swap_test_test
WHERE a = \'c\' WHERE a = \'c\'
QUERY id: 0
PROJECTION COLUMNS
substring(reverse(b), 1, 1) String
PROJECTION
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: a, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.column_swap_test_test
WHERE
FUNCTION id: 4, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
LIST id: 5, nodes: 2
COLUMN id: 6, column_name: a, result_type: String, source_id: 3
CONSTANT id: 7, constant_value: \'c\', constant_value_type: String
SELECT a SELECT a
FROM t_bad_constraint FROM t_bad_constraint
QUERY id: 0
PROJECTION COLUMNS
a UInt32
PROJECTION
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: a, result_type: UInt32, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.t_bad_constraint

View File

@ -13,17 +13,20 @@ 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; 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) = 1;
--EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 1; EXPLAIN QUERY TREE 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 cityHash64(a) = 0;
--EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 0; EXPLAIN QUERY TREE 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 = 0;
--EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 0; EXPLAIN QUERY TREE 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, b + 3 FROM column_swap_test_test WHERE b = 1;
--EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 1; EXPLAIN QUERY TREE 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 FROM column_swap_test_test WHERE cityHash64(a) = 0;
EXPLAIN QUERY TREE 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 cityHash64(a) + 10, a FROM column_swap_test_test WHERE cityHash64(a) = 0;
EXPLAIN QUERY TREE 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; EXPLAIN SYNTAX SELECT b + 10, a FROM column_swap_test_test WHERE b = 0;
EXPLAIN QUERY TREE SELECT b + 10, a FROM column_swap_test_test WHERE b = 0;
DROP TABLE column_swap_test_test; DROP TABLE column_swap_test_test;
@ -31,9 +34,13 @@ CREATE TABLE column_swap_test_test (i Int64, a String, b String, CONSTRAINT c1 A
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; 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 a = 'c';
EXPLAIN QUERY TREE 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), a FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c';
EXPLAIN QUERY TREE 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) AS t1, a AS t2 FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c';
EXPLAIN QUERY TREE 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'; EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1) FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c';
EXPLAIN QUERY TREE 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 column_swap_test_test;
@ -44,5 +51,6 @@ CREATE TABLE t_bad_constraint(a UInt32, s String, CONSTRAINT c1 ASSUME a = toUIn
INSERT INTO t_bad_constraint SELECT number, randomPrintableASCII(100) FROM numbers(10000); INSERT INTO t_bad_constraint SELECT number, randomPrintableASCII(100) FROM numbers(10000);
EXPLAIN SYNTAX SELECT a FROM t_bad_constraint; EXPLAIN SYNTAX SELECT a FROM t_bad_constraint;
EXPLAIN QUERY TREE SELECT a FROM t_bad_constraint;
DROP TABLE t_bad_constraint; DROP TABLE t_bad_constraint;