2021-11-10 17:57:59 +00:00
|
|
|
#include <Interpreters/SubstituteColumnOptimizer.h>
|
2021-04-24 15:24:32 +00:00
|
|
|
#include <Interpreters/TreeCNFConverter.h>
|
|
|
|
#include <Interpreters/ComparisonGraph.h>
|
|
|
|
#include <Parsers/IAST_fwd.h>
|
|
|
|
#include <Parsers/ASTFunction.h>
|
2021-04-26 11:26:54 +00:00
|
|
|
#include <Parsers/ASTIdentifier.h>
|
2021-04-24 15:24:32 +00:00
|
|
|
#include <Storages/StorageInMemoryMetadata.h>
|
|
|
|
#include <Parsers/ASTSelectQuery.h>
|
|
|
|
#include <Poco/Logger.h>
|
|
|
|
#include <Interpreters/InDepthNodeVisitor.h>
|
|
|
|
#include <Storages/IStorage.h>
|
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
2021-05-06 08:29:24 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
}
|
2021-04-24 15:24:32 +00:00
|
|
|
|
|
|
|
namespace
|
|
|
|
{
|
2021-04-26 11:26:54 +00:00
|
|
|
|
2021-11-10 17:57:59 +00:00
|
|
|
constexpr auto COMPONENT_PART = "__component_";
|
2021-04-29 18:54:54 +00:00
|
|
|
constexpr UInt64 COLUMN_PENALTY = 10 * 1024 * 1024;
|
2021-05-08 09:19:18 +00:00
|
|
|
constexpr Int64 INDEX_PRICE = -1'000'000'000'000'000'000;
|
2021-04-26 11:26:54 +00:00
|
|
|
|
|
|
|
class ComponentMatcher
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
using Visitor = InDepthNodeVisitor<ComponentMatcher, true>;
|
|
|
|
|
|
|
|
struct Data
|
|
|
|
{
|
2023-03-17 13:38:01 +00:00
|
|
|
const ComparisonGraph<ASTPtr> & graph;
|
2021-04-29 18:12:51 +00:00
|
|
|
std::set<UInt64> & components;
|
2021-05-04 14:34:00 +00:00
|
|
|
std::unordered_map<String, String> & old_name;
|
|
|
|
std::unordered_map<String, UInt64> & component;
|
|
|
|
UInt64 & current_id;
|
|
|
|
|
2023-03-17 13:38:01 +00:00
|
|
|
Data(const ComparisonGraph<ASTPtr> & graph_,
|
2021-05-04 14:34:00 +00:00
|
|
|
std::set<UInt64> & components_,
|
|
|
|
std::unordered_map<String, String> & old_name_,
|
|
|
|
std::unordered_map<String, UInt64> & component_,
|
|
|
|
UInt64 & current_id_)
|
|
|
|
: graph(graph_)
|
|
|
|
, components(components_)
|
|
|
|
, old_name(old_name_)
|
|
|
|
, component(component_)
|
|
|
|
, current_id(current_id_)
|
2021-04-26 11:26:54 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
static void visit(ASTPtr & ast, Data & data)
|
|
|
|
{
|
2021-11-10 17:57:59 +00:00
|
|
|
if (auto id = data.graph.getComponentId(ast))
|
2021-04-29 18:12:51 +00:00
|
|
|
{
|
2021-11-10 17:57:59 +00:00
|
|
|
const String name = COMPONENT_PART + std::to_string(*id) + "_" + std::to_string(++data.current_id);
|
2021-05-04 14:34:00 +00:00
|
|
|
data.old_name[name] = ast->getAliasOrColumnName();
|
2021-11-10 17:57:59 +00:00
|
|
|
data.component[name] = *id;
|
|
|
|
data.components.insert(*id);
|
2021-05-04 14:34:00 +00:00
|
|
|
ast = std::make_shared<ASTIdentifier>(name);
|
2021-04-29 18:12:51 +00:00
|
|
|
}
|
2021-04-26 11:26:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
static bool needChildVisit(const ASTPtr &, const ASTPtr &)
|
|
|
|
{
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
using ComponentVisitor = ComponentMatcher::Visitor;
|
|
|
|
|
2021-04-29 18:12:51 +00:00
|
|
|
struct ColumnPrice
|
|
|
|
{
|
2021-05-08 09:19:18 +00:00
|
|
|
Int64 compressed_size;
|
|
|
|
Int64 uncompressed_size;
|
2021-04-29 18:12:51 +00:00
|
|
|
|
2021-05-08 09:19:18 +00:00
|
|
|
ColumnPrice(const Int64 compressed_size_, const Int64 uncompressed_size_)
|
2021-04-29 18:12:51 +00:00
|
|
|
: compressed_size(compressed_size_)
|
|
|
|
, uncompressed_size(uncompressed_size_)
|
2021-11-10 17:57:59 +00:00
|
|
|
{
|
|
|
|
}
|
2021-04-29 18:12:51 +00:00
|
|
|
|
2021-11-10 17:57:59 +00:00
|
|
|
ColumnPrice() : ColumnPrice(0, 0) {}
|
2021-04-29 18:12:51 +00:00
|
|
|
|
|
|
|
bool operator<(const ColumnPrice & that) const
|
2021-04-26 11:26:54 +00:00
|
|
|
{
|
2021-04-29 18:12:51 +00:00
|
|
|
return std::tie(compressed_size, uncompressed_size) < std::tie(that.compressed_size, that.uncompressed_size);
|
2021-04-26 11:26:54 +00:00
|
|
|
}
|
|
|
|
|
2021-04-29 18:12:51 +00:00
|
|
|
ColumnPrice & operator+=(const ColumnPrice & that)
|
|
|
|
{
|
|
|
|
compressed_size += that.compressed_size;
|
|
|
|
uncompressed_size += that.uncompressed_size;
|
|
|
|
return *this;
|
|
|
|
}
|
2021-04-26 11:26:54 +00:00
|
|
|
|
2021-04-29 18:12:51 +00:00
|
|
|
ColumnPrice & operator-=(const ColumnPrice & that)
|
|
|
|
{
|
|
|
|
compressed_size -= that.compressed_size;
|
|
|
|
uncompressed_size -= that.uncompressed_size;
|
|
|
|
return *this;
|
|
|
|
}
|
|
|
|
};
|
2021-04-26 11:26:54 +00:00
|
|
|
|
2021-11-10 17:57:59 +00:00
|
|
|
using ColumnPriceByName = std::unordered_map<String, ColumnPrice>;
|
|
|
|
|
2021-04-24 15:24:32 +00:00
|
|
|
class SubstituteColumnMatcher
|
|
|
|
{
|
|
|
|
public:
|
2021-04-26 11:26:54 +00:00
|
|
|
using Visitor = InDepthNodeVisitor<SubstituteColumnMatcher, false>;
|
2021-04-24 15:24:32 +00:00
|
|
|
|
|
|
|
struct Data
|
|
|
|
{
|
2021-05-04 14:34:00 +00:00
|
|
|
std::unordered_map<UInt64, ASTPtr> id_to_expression_map;
|
|
|
|
std::unordered_map<String, UInt64> name_to_component_id;
|
|
|
|
std::unordered_map<String, String> old_name;
|
|
|
|
bool is_select;
|
2021-04-24 15:24:32 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
static void visit(ASTPtr & ast, Data & data)
|
|
|
|
{
|
2021-04-26 11:26:54 +00:00
|
|
|
const auto * identifier = ast->as<ASTIdentifier>();
|
2021-05-04 14:34:00 +00:00
|
|
|
if (identifier && data.name_to_component_id.contains(identifier->name()))
|
|
|
|
{
|
2021-05-06 20:36:18 +00:00
|
|
|
const String & name = identifier->name();
|
2021-05-04 14:34:00 +00:00
|
|
|
const auto component_id = data.name_to_component_id.at(name);
|
2021-05-06 20:36:18 +00:00
|
|
|
auto new_ast = data.id_to_expression_map.at(component_id)->clone();
|
2021-11-10 17:57:59 +00:00
|
|
|
|
2021-05-04 14:34:00 +00:00
|
|
|
if (data.is_select)
|
2021-05-06 20:36:18 +00:00
|
|
|
new_ast->setAlias(data.old_name.at(name));
|
2021-11-10 17:57:59 +00:00
|
|
|
|
2021-05-06 20:36:18 +00:00
|
|
|
ast = new_ast;
|
2021-05-04 14:34:00 +00:00
|
|
|
}
|
2021-04-24 15:24:32 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
static bool needChildVisit(const ASTPtr &, const ASTPtr &)
|
|
|
|
{
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
using SubstituteColumnVisitor = SubstituteColumnMatcher::Visitor;
|
2021-04-29 18:12:51 +00:00
|
|
|
|
|
|
|
ColumnPrice calculatePrice(
|
2021-11-10 17:57:59 +00:00
|
|
|
const ColumnPriceByName & column_prices,
|
|
|
|
const IdentifierNameSet & identifiers)
|
2021-04-29 18:12:51 +00:00
|
|
|
{
|
|
|
|
ColumnPrice result(0, 0);
|
|
|
|
for (const auto & ident : identifiers)
|
2021-11-19 14:14:56 +00:00
|
|
|
{
|
|
|
|
auto it = column_prices.find(ident);
|
|
|
|
if (it != column_prices.end())
|
|
|
|
result += it->second;
|
|
|
|
}
|
|
|
|
|
2021-04-29 18:12:51 +00:00
|
|
|
return result;
|
|
|
|
}
|
|
|
|
|
2021-11-10 17:57:59 +00:00
|
|
|
/// We need to choose one expression in each component,
|
|
|
|
/// so that total price of all read columns will be minimal.
|
|
|
|
/// Bruteforce equal ASTs in each component and calculate
|
|
|
|
/// price of all columns on which ast depends.
|
|
|
|
/// TODO: branch-and-bound
|
2021-04-29 18:12:51 +00:00
|
|
|
void bruteforce(
|
2023-03-17 13:38:01 +00:00
|
|
|
const ComparisonGraph<ASTPtr> & graph,
|
2021-04-29 18:12:51 +00:00
|
|
|
const std::vector<UInt64> & components,
|
|
|
|
size_t current_component,
|
2021-11-10 17:57:59 +00:00
|
|
|
const ColumnPriceByName & column_prices,
|
2021-04-29 18:12:51 +00:00
|
|
|
ColumnPrice current_price,
|
|
|
|
std::vector<ASTPtr> & expressions_stack,
|
|
|
|
ColumnPrice & min_price,
|
|
|
|
std::vector<ASTPtr> & min_expressions)
|
|
|
|
{
|
|
|
|
if (current_component == components.size())
|
|
|
|
{
|
|
|
|
if (current_price < min_price)
|
|
|
|
{
|
|
|
|
min_price = current_price;
|
|
|
|
min_expressions = expressions_stack;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
for (const auto & ast : graph.getComponent(components[current_component]))
|
|
|
|
{
|
2021-11-10 17:57:59 +00:00
|
|
|
IdentifierNameSet identifiers;
|
|
|
|
ast->collectIdentifierNames(identifiers);
|
2021-04-29 18:12:51 +00:00
|
|
|
ColumnPrice expression_price = calculatePrice(column_prices, identifiers);
|
|
|
|
|
|
|
|
expressions_stack.push_back(ast);
|
|
|
|
current_price += expression_price;
|
|
|
|
|
2021-11-10 17:57:59 +00:00
|
|
|
ColumnPriceByName new_prices(column_prices);
|
|
|
|
/// Update prices of already counted columns.
|
2021-04-29 18:12:51 +00:00
|
|
|
for (const auto & identifier : identifiers)
|
|
|
|
new_prices[identifier] = 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();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-04-24 15:24:32 +00:00
|
|
|
}
|
|
|
|
|
2021-04-26 11:26:54 +00:00
|
|
|
|
2021-04-24 15:24:32 +00:00
|
|
|
SubstituteColumnOptimizer::SubstituteColumnOptimizer(
|
|
|
|
ASTSelectQuery * select_query_,
|
|
|
|
const StorageMetadataPtr & metadata_snapshot_,
|
|
|
|
const ConstStoragePtr & storage_)
|
|
|
|
: select_query(select_query_)
|
|
|
|
, metadata_snapshot(metadata_snapshot_)
|
|
|
|
, storage(storage_)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
void SubstituteColumnOptimizer::perform()
|
|
|
|
{
|
|
|
|
if (!storage)
|
|
|
|
return;
|
2021-11-10 17:57:59 +00:00
|
|
|
|
2021-04-29 18:12:51 +00:00
|
|
|
const auto column_sizes = storage->getColumnSizes();
|
|
|
|
if (column_sizes.empty())
|
|
|
|
return;
|
|
|
|
|
2021-05-07 10:49:05 +00:00
|
|
|
const auto & compare_graph = metadata_snapshot->getConstraints().getGraph();
|
2021-04-26 11:26:54 +00:00
|
|
|
|
2021-05-04 14:34:00 +00:00
|
|
|
// Fill aliases
|
|
|
|
if (select_query->select())
|
|
|
|
{
|
|
|
|
auto * list = select_query->refSelect()->as<ASTExpressionList>();
|
|
|
|
if (!list)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "List of selected columns must be ASTExpressionList");
|
2021-11-10 17:57:59 +00:00
|
|
|
|
2021-05-04 14:34:00 +00:00
|
|
|
for (ASTPtr & ast : list->children)
|
|
|
|
ast->setAlias(ast->getAliasOrColumnName());
|
|
|
|
}
|
|
|
|
|
2021-05-04 18:43:58 +00:00
|
|
|
auto run_for_all = [&](const auto func)
|
|
|
|
{
|
2021-04-26 11:26:54 +00:00
|
|
|
if (select_query->where())
|
2021-05-04 14:34:00 +00:00
|
|
|
func(select_query->refWhere(), false);
|
2021-04-26 11:26:54 +00:00
|
|
|
if (select_query->prewhere())
|
2021-05-04 14:34:00 +00:00
|
|
|
func(select_query->refPrewhere(), false);
|
2021-04-26 11:26:54 +00:00
|
|
|
if (select_query->select())
|
2021-05-04 14:34:00 +00:00
|
|
|
func(select_query->refSelect(), true);
|
2021-04-26 11:26:54 +00:00
|
|
|
if (select_query->having())
|
2021-05-04 14:34:00 +00:00
|
|
|
func(select_query->refHaving(), false);
|
2021-04-26 11:26:54 +00:00
|
|
|
};
|
|
|
|
|
2021-04-29 18:12:51 +00:00
|
|
|
std::set<UInt64> components;
|
2021-05-04 14:34:00 +00:00
|
|
|
std::unordered_map<String, String> old_name;
|
|
|
|
std::unordered_map<String, UInt64> name_to_component;
|
2021-11-10 17:57:59 +00:00
|
|
|
|
2021-05-04 14:34:00 +00:00
|
|
|
UInt64 counter_id = 0;
|
2021-11-10 17:57:59 +00:00
|
|
|
|
2021-05-04 14:34:00 +00:00
|
|
|
ComponentVisitor::Data component_data(
|
|
|
|
compare_graph, components, old_name, name_to_component, counter_id);
|
2021-11-10 17:57:59 +00:00
|
|
|
|
|
|
|
IdentifierNameSet identifiers;
|
2021-05-04 18:43:58 +00:00
|
|
|
auto preprocess = [&](ASTPtr & ast, bool)
|
|
|
|
{
|
2021-04-26 11:26:54 +00:00
|
|
|
ComponentVisitor(component_data).visit(ast);
|
2021-11-10 17:57:59 +00:00
|
|
|
ast->collectIdentifierNames(identifiers);
|
2021-04-26 11:26:54 +00:00
|
|
|
};
|
|
|
|
|
2021-04-29 18:12:51 +00:00
|
|
|
run_for_all(preprocess);
|
|
|
|
|
|
|
|
const auto primary_key = metadata_snapshot->getColumnsRequiredForPrimaryKey();
|
|
|
|
const std::unordered_set<std::string_view> primary_key_set(std::begin(primary_key), std::end(primary_key));
|
2021-11-10 17:57:59 +00:00
|
|
|
ColumnPriceByName column_prices;
|
|
|
|
|
2021-04-29 18:12:51 +00:00
|
|
|
for (const auto & [column_name, column_size] : column_sizes)
|
2021-11-10 17:57:59 +00:00
|
|
|
column_prices[column_name] = ColumnPrice(column_size.data_compressed + COLUMN_PENALTY, column_size.data_uncompressed);
|
|
|
|
|
2021-04-29 18:12:51 +00:00
|
|
|
for (const auto & column_name : primary_key)
|
2021-05-08 09:19:18 +00:00
|
|
|
column_prices[column_name] = ColumnPrice(INDEX_PRICE, INDEX_PRICE);
|
2021-11-10 17:57:59 +00:00
|
|
|
|
2021-04-29 18:12:51 +00:00
|
|
|
for (const auto & column_name : identifiers)
|
|
|
|
column_prices[column_name] = ColumnPrice(0, 0);
|
|
|
|
|
2021-05-04 14:34:00 +00:00
|
|
|
std::unordered_map<UInt64, ASTPtr> id_to_expression_map;
|
2021-04-29 18:12:51 +00:00
|
|
|
std::vector<UInt64> components_list;
|
2021-11-10 17:57:59 +00:00
|
|
|
|
|
|
|
for (const UInt64 component_id : components)
|
|
|
|
{
|
|
|
|
auto component = compare_graph.getComponent(component_id);
|
|
|
|
if (component.size() == 1)
|
|
|
|
id_to_expression_map[component_id] = component.front();
|
2021-04-29 18:12:51 +00:00
|
|
|
else
|
2021-11-10 17:57:59 +00:00
|
|
|
components_list.push_back(component_id);
|
|
|
|
}
|
2021-04-29 18:12:51 +00:00
|
|
|
|
|
|
|
std::vector<ASTPtr> expressions_stack;
|
2021-05-08 09:19:18 +00:00
|
|
|
ColumnPrice min_price(std::numeric_limits<Int64>::max(), std::numeric_limits<Int64>::max());
|
2021-04-29 18:12:51 +00:00
|
|
|
std::vector<ASTPtr> min_expressions;
|
2021-11-10 17:57:59 +00:00
|
|
|
|
2021-04-29 18:12:51 +00:00
|
|
|
bruteforce(compare_graph,
|
|
|
|
components_list,
|
|
|
|
0,
|
|
|
|
column_prices,
|
|
|
|
ColumnPrice(0, 0),
|
|
|
|
expressions_stack,
|
|
|
|
min_price,
|
|
|
|
min_expressions);
|
|
|
|
|
2021-05-08 09:19:18 +00:00
|
|
|
for (size_t i = 0; i < components_list.size(); ++i)
|
2021-05-04 14:34:00 +00:00
|
|
|
id_to_expression_map[components_list[i]] = min_expressions[i];
|
2021-04-29 18:12:51 +00:00
|
|
|
|
2021-05-04 18:43:58 +00:00
|
|
|
auto process = [&](ASTPtr & ast, bool is_select)
|
|
|
|
{
|
2021-05-04 14:34:00 +00:00
|
|
|
SubstituteColumnVisitor::Data substitute_data{id_to_expression_map, name_to_component, old_name, is_select};
|
2021-04-26 11:26:54 +00:00
|
|
|
SubstituteColumnVisitor(substitute_data).visit(ast);
|
|
|
|
};
|
|
|
|
|
|
|
|
run_for_all(process);
|
2021-04-24 15:24:32 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|