mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 00:22:29 +00:00
Add support for substitute column
This commit is contained in:
parent
d6efe7fc21
commit
0fb9f9ffe6
@ -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);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -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;
|
||||||
|
@ -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:
|
||||||
|
@ -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,
|
||||||
|
@ -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;
|
||||||
|
@ -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)
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -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)
|
||||||
|
@ -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;
|
||||||
|
@ -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
|
||||||
|
@ -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;
|
||||||
|
Loading…
Reference in New Issue
Block a user