ClickHouse/src/Interpreters/TreeRewriter.cpp

1322 lines
49 KiB
C++
Raw Normal View History

2019-12-27 19:45:41 +00:00
#include <Core/Settings.h>
#include <Core/NamesAndTypes.h>
2019-12-27 19:42:14 +00:00
#include <Common/checkStackSize.h>
#include <Interpreters/TreeRewriter.h>
2019-12-27 19:45:41 +00:00
#include <Interpreters/LogicalExpressionsOptimizer.h>
#include <Interpreters/QueryAliasesVisitor.h>
#include <Interpreters/ArrayJoinedColumnsVisitor.h>
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
#include <Interpreters/Context.h>
2021-02-14 11:09:36 +00:00
#include <Interpreters/FunctionNameNormalizer.h>
2019-12-27 19:45:41 +00:00
#include <Interpreters/MarkTableIdentifiersVisitor.h>
#include <Interpreters/QueryNormalizer.h>
#include <Interpreters/ExecuteScalarSubqueriesVisitor.h>
#include <Interpreters/CollectJoinOnKeysVisitor.h>
#include <Interpreters/RequiredSourceColumnsVisitor.h>
#include <Interpreters/GetAggregatesVisitor.h>
#include <Interpreters/UserDefinedSQLFunctionVisitor.h>
#include <Interpreters/TableJoin.h>
2019-12-27 19:45:41 +00:00
#include <Interpreters/ExpressionActions.h> /// getSmallestColumn()
#include <Interpreters/getTableExpressions.h>
#include <Interpreters/TreeOptimizer.h>
2020-12-12 16:42:15 +00:00
#include <Interpreters/replaceAliasColumnsInQuery.h>
2019-12-27 19:45:41 +00:00
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/queryToString.h>
#include <Parsers/ASTLiteral.h>
2019-12-27 19:45:41 +00:00
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypeNullable.h>
#include <IO/WriteHelpers.h>
#include <Storages/IStorage.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
2019-12-27 19:45:41 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int EMPTY_LIST_OF_COLUMNS_QUERIED;
2019-12-27 19:45:41 +00:00
extern const int EMPTY_NESTED_TABLE;
extern const int EXPECTED_ALL_OR_ANY;
2019-12-27 19:45:41 +00:00
extern const int INVALID_JOIN_ON_EXPRESSION;
extern const int LOGICAL_ERROR;
2019-12-27 19:45:41 +00:00
extern const int NOT_IMPLEMENTED;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
2019-12-27 19:45:41 +00:00
extern const int UNKNOWN_IDENTIFIER;
}
namespace
{
using LogAST = DebugASTLog<false>; /// set to true to enable logs
2020-04-06 13:30:16 +00:00
/// Select implementation of a function based on settings.
2019-12-27 19:45:41 +00:00
/// Important that it is done as query rewrite. It means rewritten query
/// will be sent to remote servers during distributed query execution,
/// and on all remote servers, function implementation will be same.
2020-04-06 13:30:16 +00:00
template <char const * func_name>
2019-12-27 19:45:41 +00:00
struct CustomizeFunctionsData
{
using TypeToVisit = ASTFunction;
2020-04-06 13:30:16 +00:00
const String & customized_func_name;
2019-12-27 19:45:41 +00:00
void visit(ASTFunction & func, ASTPtr &) const
2019-12-27 19:45:41 +00:00
{
2020-04-06 13:30:16 +00:00
if (Poco::toLower(func.name) == func_name)
{
func.name = customized_func_name;
}
2019-12-27 19:45:41 +00:00
}
};
2020-04-06 13:30:16 +00:00
char countdistinct[] = "countdistinct";
using CustomizeCountDistinctVisitor = InDepthNodeVisitor<OneTypeMatcher<CustomizeFunctionsData<countdistinct>>, true>;
char countifdistinct[] = "countifdistinct";
using CustomizeCountIfDistinctVisitor = InDepthNodeVisitor<OneTypeMatcher<CustomizeFunctionsData<countifdistinct>>, true>;
2020-04-06 13:30:16 +00:00
char in[] = "in";
using CustomizeInVisitor = InDepthNodeVisitor<OneTypeMatcher<CustomizeFunctionsData<in>>, true>;
char notIn[] = "notin";
using CustomizeNotInVisitor = InDepthNodeVisitor<OneTypeMatcher<CustomizeFunctionsData<notIn>>, true>;
char globalIn[] = "globalin";
using CustomizeGlobalInVisitor = InDepthNodeVisitor<OneTypeMatcher<CustomizeFunctionsData<globalIn>>, true>;
char globalNotIn[] = "globalnotin";
using CustomizeGlobalNotInVisitor = InDepthNodeVisitor<OneTypeMatcher<CustomizeFunctionsData<globalNotIn>>, true>;
2019-12-27 19:45:41 +00:00
template <char const * func_suffix>
struct CustomizeFunctionsSuffixData
{
using TypeToVisit = ASTFunction;
const String & customized_func_suffix;
void visit(ASTFunction & func, ASTPtr &) const
{
if (endsWith(Poco::toLower(func.name), func_suffix))
{
size_t prefix_len = func.name.length() - strlen(func_suffix);
func.name = func.name.substr(0, prefix_len) + customized_func_suffix;
}
}
};
/// Swap 'if' and 'distinct' suffixes to make execution more optimal.
char ifDistinct[] = "ifdistinct";
using CustomizeIfDistinctVisitor = InDepthNodeVisitor<OneTypeMatcher<CustomizeFunctionsSuffixData<ifDistinct>>, true>;
2019-12-27 19:45:41 +00:00
/// Used to rewrite all aggregate functions to add -OrNull suffix to them if setting `aggregate_functions_null_for_empty` is set.
struct CustomizeAggregateFunctionsSuffixData
{
using TypeToVisit = ASTFunction;
const String & customized_func_suffix;
void visit(ASTFunction & func, ASTPtr &) const
{
2020-11-04 12:15:31 +00:00
const auto & instance = AggregateFunctionFactory::instance();
if (instance.isAggregateFunctionName(func.name) && !endsWith(func.name, customized_func_suffix))
{
2020-11-04 12:15:31 +00:00
auto properties = instance.tryGetProperties(func.name);
if (properties && !properties->returns_default_when_only_null)
{
2020-12-10 08:26:45 +00:00
func.name += customized_func_suffix;
2020-12-09 04:30:38 +00:00
}
}
}
};
// Used to rewrite aggregate functions with -OrNull suffix in some cases, such as sumIfOrNull, we should rewrite to sumOrNullIf
2020-12-09 04:30:38 +00:00
struct CustomizeAggregateFunctionsMoveSuffixData
{
using TypeToVisit = ASTFunction;
2020-12-09 04:30:38 +00:00
const String & customized_func_suffix;
2020-12-09 04:30:38 +00:00
String moveSuffixAhead(const String & name) const
{
auto prefix = name.substr(0, name.size() - customized_func_suffix.size());
2020-12-09 04:30:38 +00:00
auto prefix_size = prefix.size();
if (endsWith(prefix, "MergeState"))
return prefix.substr(0, prefix_size - 10) + customized_func_suffix + "MergeState";
if (endsWith(prefix, "Merge"))
return prefix.substr(0, prefix_size - 5) + customized_func_suffix + "Merge";
if (endsWith(prefix, "State"))
return prefix.substr(0, prefix_size - 5) + customized_func_suffix + "State";
if (endsWith(prefix, "If"))
return prefix.substr(0, prefix_size - 2) + customized_func_suffix + "If";
return name;
}
void visit(ASTFunction & func, ASTPtr &) const
{
const auto & instance = AggregateFunctionFactory::instance();
if (instance.isAggregateFunctionName(func.name))
{
2020-12-10 08:26:45 +00:00
if (endsWith(func.name, customized_func_suffix))
2020-12-09 04:30:38 +00:00
{
2020-12-09 07:24:36 +00:00
auto properties = instance.tryGetProperties(func.name);
if (properties && !properties->returns_default_when_only_null)
{
func.name = moveSuffixAhead(func.name);
}
}
}
}
};
2021-04-15 16:40:49 +00:00
struct FuseSumCountAggregates
{
std::vector<ASTFunction *> sums {};
std::vector<ASTFunction *> counts {};
std::vector<ASTFunction *> avgs {};
2021-04-15 16:40:49 +00:00
void addFuncNode(ASTFunction * func)
2021-03-08 03:58:18 +00:00
{
2021-04-15 16:40:49 +00:00
if (func->name == "sum")
sums.push_back(func);
else if (func->name == "count")
counts.push_back(func);
else
{
assert(func->name == "avg");
avgs.push_back(func);
}
2021-03-08 03:58:18 +00:00
}
2021-03-08 03:58:18 +00:00
bool canBeFused() const
{
2021-04-15 16:40:49 +00:00
// Need at least two different kinds of functions to fuse.
2021-03-08 03:58:18 +00:00
if (sums.empty() && counts.empty())
return false;
if (sums.empty() && avgs.empty())
return false;
if (counts.empty() && avgs.empty())
return false;
return true;
}
2021-03-08 03:58:18 +00:00
};
2021-04-15 16:40:49 +00:00
struct FuseSumCountAggregatesVisitorData
2021-03-08 03:58:18 +00:00
{
using TypeToVisit = ASTFunction;
2021-04-15 16:40:49 +00:00
std::unordered_map<String, FuseSumCountAggregates> fuse_map;
void visit(ASTFunction & func, ASTPtr &)
{
if (func.name == "sum" || func.name == "avg" || func.name == "count")
{
2021-03-09 09:03:24 +00:00
if (func.arguments->children.empty())
2021-03-08 03:58:18 +00:00
return;
2021-04-16 10:35:02 +00:00
// Probably we can extend it to match count() for non-nullable argument
2021-04-15 16:40:49 +00:00
// to sum/avg with any other argument. Now we require strict match.
const auto argument = func.arguments->children.at(0)->getColumnName();
auto it = fuse_map.find(argument);
if (it != fuse_map.end())
{
2021-04-15 16:40:49 +00:00
it->second.addFuncNode(&func);
}
else
{
2021-04-15 16:40:49 +00:00
FuseSumCountAggregates funcs{};
funcs.addFuncNode(&func);
fuse_map[argument] = funcs;
}
}
}
};
using CustomizeAggregateFunctionsOrNullVisitor = InDepthNodeVisitor<OneTypeMatcher<CustomizeAggregateFunctionsSuffixData>, true>;
2020-12-09 04:30:38 +00:00
using CustomizeAggregateFunctionsMoveOrNullVisitor = InDepthNodeVisitor<OneTypeMatcher<CustomizeAggregateFunctionsMoveSuffixData>, true>;
2021-04-15 16:40:49 +00:00
using FuseSumCountAggregatesVisitor = InDepthNodeVisitor<OneTypeMatcher<FuseSumCountAggregatesVisitorData>, true>;
2019-12-27 19:45:41 +00:00
/// Translate qualified names such as db.table.column, table.column, table_alias.column to names' normal form.
/// Expand asterisks and qualified asterisks with column names.
/// There would be columns in normal form & column aliases after translation. Column & column alias would be normalized in QueryNormalizer.
void translateQualifiedNames(ASTPtr & query, const ASTSelectQuery & select_query, const NameSet & source_columns_set,
const TablesWithColumns & tables_with_columns)
2019-12-27 19:45:41 +00:00
{
LogAST log;
2020-02-27 13:31:32 +00:00
TranslateQualifiedNamesVisitor::Data visitor_data(source_columns_set, tables_with_columns);
2019-12-27 19:45:41 +00:00
TranslateQualifiedNamesVisitor visitor(visitor_data, log.stream());
visitor.visit(query);
/// This may happen after expansion of COLUMNS('regexp').
if (select_query.select()->children.empty())
throw Exception("Empty list of columns in SELECT query", ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED);
}
2021-04-15 16:40:49 +00:00
// Replaces one avg/sum/count function with an appropriate expression with
// sumCount().
void replaceWithSumCount(String column_name, ASTFunction & func)
2021-03-08 03:58:18 +00:00
{
auto func_base = makeASTFunction("sumCount", std::make_shared<ASTIdentifier>(column_name));
auto exp_list = std::make_shared<ASTExpressionList>();
if (func.name == "sum" || func.name == "count")
2021-03-08 03:58:18 +00:00
{
/// Rewrite "sum" to sumCount().1, rewrite "count" to sumCount().2
UInt8 idx = (func.name == "sum" ? 1 : 2);
func.name = "tupleElement";
exp_list->children.push_back(func_base);
exp_list->children.push_back(std::make_shared<ASTLiteral>(idx));
}
else
{
/// Rewrite "avg" to sumCount().1 / sumCount().2
auto new_arg1 = makeASTFunction("tupleElement", func_base, std::make_shared<ASTLiteral>(UInt8(1)));
auto new_arg2 = makeASTFunction("tupleElement", func_base, std::make_shared<ASTLiteral>(UInt8(2)));
func.name = "divide";
exp_list->children.push_back(new_arg1);
exp_list->children.push_back(new_arg2);
}
func.arguments = exp_list;
func.children.push_back(func.arguments);
}
2021-03-08 03:58:18 +00:00
2021-04-15 16:40:49 +00:00
void fuseSumCountAggregates(std::unordered_map<String, FuseSumCountAggregates> & fuse_map)
{
for (auto & it : fuse_map)
{
if (it.second.canBeFused())
{
for (auto & func: it.second.sums)
2021-04-15 16:40:49 +00:00
replaceWithSumCount(it.first, *func);
for (auto & func: it.second.avgs)
2021-04-15 16:40:49 +00:00
replaceWithSumCount(it.first, *func);
for (auto & func: it.second.counts)
2021-04-15 16:40:49 +00:00
replaceWithSumCount(it.first, *func);
2021-03-08 03:58:18 +00:00
}
}
}
2019-12-27 19:45:41 +00:00
bool hasArrayJoin(const ASTPtr & ast)
{
if (const ASTFunction * function = ast->as<ASTFunction>())
if (function->name == "arrayJoin")
return true;
for (const auto & child : ast->children)
if (!child->as<ASTSelectQuery>() && hasArrayJoin(child))
return true;
return false;
}
/// Keep number of columns for 'GLOBAL IN (SELECT 1 AS a, a)'
void renameDuplicatedColumns(const ASTSelectQuery * select_query)
{
ASTs & elements = select_query->select()->children;
std::set<String> all_column_names;
std::set<String> assigned_column_names;
for (auto & expr : elements)
all_column_names.insert(expr->getAliasOrColumnName());
for (auto & expr : elements)
{
auto name = expr->getAliasOrColumnName();
if (!assigned_column_names.insert(name).second)
{
size_t i = 1;
while (all_column_names.end() != all_column_names.find(name + "_" + toString(i)))
++i;
name = name + "_" + toString(i);
expr = expr->clone(); /// Cancels fuse of the same expressions in the tree.
expr->setAlias(name);
all_column_names.insert(name);
assigned_column_names.insert(name);
}
}
}
/// Sometimes we have to calculate more columns in SELECT clause than will be returned from query.
/// This is the case when we have DISTINCT or arrayJoin: we require more columns in SELECT even if we need less columns in result.
2020-01-11 09:50:41 +00:00
/// Also we have to remove duplicates in case of GLOBAL subqueries. Their results are placed into tables so duplicates are impossible.
2019-12-27 19:45:41 +00:00
void removeUnneededColumnsFromSelectClause(const ASTSelectQuery * select_query, const Names & required_result_columns, bool remove_dups)
{
ASTs & elements = select_query->select()->children;
std::map<String, size_t> required_columns_with_duplicate_count;
if (!required_result_columns.empty())
{
/// Some columns may be queried multiple times, like SELECT x, y, y FROM table.
for (const auto & name : required_result_columns)
{
if (remove_dups)
required_columns_with_duplicate_count[name] = 1;
else
++required_columns_with_duplicate_count[name];
}
}
else if (remove_dups)
{
/// Even if we have no requirements there could be duplicates cause of asterisks. SELECT *, t.*
for (const auto & elem : elements)
required_columns_with_duplicate_count.emplace(elem->getAliasOrColumnName(), 1);
}
else
return;
ASTs new_elements;
new_elements.reserve(elements.size());
for (const auto & elem : elements)
{
String name = elem->getAliasOrColumnName();
auto it = required_columns_with_duplicate_count.find(name);
if (required_columns_with_duplicate_count.end() != it && it->second)
{
new_elements.push_back(elem);
--it->second;
}
else if (select_query->distinct || hasArrayJoin(elem))
{
new_elements.push_back(elem);
}
else
{
ASTFunction * func = elem->as<ASTFunction>();
2021-04-12 12:15:55 +00:00
/// Never remove untuple. It's result column may be in required columns.
/// It is not easy to analyze untuple here, because types were not calculated yes.
if (func && func->name == "untuple")
2021-04-12 12:15:55 +00:00
new_elements.push_back(elem);
}
2019-12-27 19:45:41 +00:00
}
elements = std::move(new_elements);
}
/// Replacing scalar subqueries with constant values.
2021-06-01 12:20:52 +00:00
void executeScalarSubqueries(ASTPtr & query, ContextPtr context, size_t subquery_depth, Scalars & scalars, bool only_analyze)
2019-12-27 19:45:41 +00:00
{
LogAST log;
2021-06-01 12:20:52 +00:00
ExecuteScalarSubqueriesVisitor::Data visitor_data{WithContext{context}, subquery_depth, scalars, only_analyze};
2019-12-27 19:45:41 +00:00
ExecuteScalarSubqueriesVisitor(visitor_data, log.stream()).visit(query);
}
void getArrayJoinedColumns(ASTPtr & query, TreeRewriterResult & result, const ASTSelectQuery * select_query,
2019-12-27 19:45:41 +00:00
const NamesAndTypesList & source_columns, const NameSet & source_columns_set)
{
if (!select_query->arrayJoinExpressionList().first)
return;
ArrayJoinedColumnsVisitor::Data visitor_data{
result.aliases, result.array_join_name_to_alias, result.array_join_alias_to_name, result.array_join_result_to_source};
ArrayJoinedColumnsVisitor(visitor_data).visit(query);
/// If the result of ARRAY JOIN is not used, it is necessary to ARRAY-JOIN any column,
/// to get the correct number of rows.
if (result.array_join_result_to_source.empty())
2019-12-27 19:45:41 +00:00
{
if (select_query->arrayJoinExpressionList().first->children.empty())
throw DB::Exception("ARRAY JOIN requires an argument", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
ASTPtr expr = select_query->arrayJoinExpressionList().first->children.at(0);
String source_name = expr->getColumnName();
String result_name = expr->getAliasOrColumnName();
2019-12-27 19:45:41 +00:00
/// This is an array.
if (!expr->as<ASTIdentifier>() || source_columns_set.count(source_name))
{
result.array_join_result_to_source[result_name] = source_name;
}
else /// This is a nested table.
{
bool found = false;
for (const auto & column : source_columns)
2019-12-27 19:45:41 +00:00
{
auto split = Nested::splitName(column.name);
if (split.first == source_name && !split.second.empty())
2019-12-27 19:45:41 +00:00
{
result.array_join_result_to_source[Nested::concatenateName(result_name, split.second)] = column.name;
found = true;
break;
2019-12-27 19:45:41 +00:00
}
}
if (!found)
throw Exception("No columns in nested table " + source_name, ErrorCodes::EMPTY_NESTED_TABLE);
2019-12-27 19:45:41 +00:00
}
}
}
void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_default_strictness, bool old_any, ASTTableJoin & out_table_join)
{
const ASTTablesInSelectQueryElement * node = select_query.join();
if (!node)
return;
auto & table_join = const_cast<ASTTablesInSelectQueryElement *>(node)->table_join->as<ASTTableJoin &>();
if (table_join.strictness == ASTTableJoin::Strictness::Unspecified &&
table_join.kind != ASTTableJoin::Kind::Cross)
{
if (join_default_strictness == JoinStrictness::ANY)
table_join.strictness = ASTTableJoin::Strictness::Any;
else if (join_default_strictness == JoinStrictness::ALL)
table_join.strictness = ASTTableJoin::Strictness::All;
else
throw Exception("Expected ANY or ALL in JOIN section, because setting (join_default_strictness) is empty",
DB::ErrorCodes::EXPECTED_ALL_OR_ANY);
}
if (old_any)
{
if (table_join.strictness == ASTTableJoin::Strictness::Any &&
table_join.kind == ASTTableJoin::Kind::Inner)
{
table_join.strictness = ASTTableJoin::Strictness::Semi;
table_join.kind = ASTTableJoin::Kind::Left;
}
if (table_join.strictness == ASTTableJoin::Strictness::Any)
table_join.strictness = ASTTableJoin::Strictness::RightAny;
}
else
{
if (table_join.strictness == ASTTableJoin::Strictness::Any)
if (table_join.kind == ASTTableJoin::Kind::Full)
throw Exception("ANY FULL JOINs are not implemented.", ErrorCodes::NOT_IMPLEMENTED);
}
2019-12-27 19:45:41 +00:00
out_table_join = table_join;
}
2021-09-15 11:16:10 +00:00
/// Convert to Disjunctive Normal Form https://en.wikipedia.org/wiki/Disjunctive_normal_form
/// based on sample https://github.com/ilejn/ndf
2021-06-25 12:03:10 +00:00
class DNF
{
bool node_added = false;
void normTree(ASTPtr node)
{
auto * func = node->as<ASTFunction>();
if (func && func->children.size() == 1)
2021-06-25 12:03:10 +00:00
{
for (bool touched = true; touched;)
{
touched = false;
ASTs new_children;
const auto * func_args = func->arguments->as<ASTExpressionList>();
for (const auto & child : func_args->children)
2021-06-25 12:03:10 +00:00
{
auto * child_func = child->as<ASTFunction>();
if (child_func && func->children.size() == 1
&& ((func->name == "or" && child_func->name == "or") || (func->name == "and" && child_func->name == "and")))
2021-06-25 12:03:10 +00:00
{
std::copy(child_func->arguments->children.begin(),
child_func->arguments->children.end(),
2021-06-25 12:03:10 +00:00
std::back_inserter(new_children));
touched = true;
}
else
{
new_children.push_back(child);
}
}
func->arguments->children = std::move(new_children);
2021-06-25 12:03:10 +00:00
}
for (auto & child : func->arguments->children)
2021-06-25 12:03:10 +00:00
{
normTree(child);
}
}
// LOG_TRACE(&Poco::Logger::get("TreeRewrite"), "bottom of normTree: {}", node->dumpTree());
}
ASTPtr distribute(ASTPtr node)
{
checkStackSize();
2021-06-25 12:03:10 +00:00
const auto * function = node->as<ASTFunction>();
if (function && function->children.size() == 1)
{
if (function->name == "and")
{
2021-09-22 20:46:28 +00:00
auto * func_args = function->arguments->as<ASTExpressionList>();
if (!func_args)
2021-06-25 12:03:10 +00:00
{
return node;
}
2021-09-22 20:46:28 +00:00
ASTs distr_lst;
for (const auto & arg : func_args->children)
{
distr_lst.push_back(distribute(arg));
}
func_args->children = distr_lst;
auto or_child = std::find_if(func_args->children.begin(), func_args->children.end(), [](ASTPtr arg)
2021-06-25 12:03:10 +00:00
{
const auto * f = arg->as<ASTFunction>();
return f && f->name == "or" && f->children.size() == 1;
});
if (or_child == func_args->children.end())
2021-06-25 12:03:10 +00:00
{
return node;
}
ASTs rest_children;
for (const auto & arg : func_args->children)
2021-06-25 12:03:10 +00:00
{
// LOG_DEBUG(&Poco::Logger::get("toDNF"), "IDs {} vs. {}", arg->getTreeHash(), (*or_child)->getTreeHash());
if (arg->getTreeHash() != (*or_child)->getTreeHash())
{
rest_children.push_back(arg);
}
}
if (rest_children.empty())
{
return node;
}
const auto * or_child_function = (*or_child)->as<ASTFunction>();
if (!or_child_function)
{
return node;
}
auto rest = rest_children.size() > 1 ?
makeASTFunction("and", rest_children):
rest_children[0];
const auto * or_child_expression_list = or_child_function->children[0]->as<ASTExpressionList>();
assert(or_child_expression_list);
if (or_child_expression_list)
{
ASTs lst;
for (const auto & arg : or_child_expression_list->children)
{
ASTs arg_rest_lst;
arg_rest_lst.push_back(arg);
arg_rest_lst.push_back(rest);
auto and_node = makeASTFunction("and", arg_rest_lst);
lst.push_back(distribute(and_node));
}
if (lst.empty())
{
return node;
}
auto ret = lst.size()>1 ?
makeASTFunction("or", lst) :
lst[0];
node_added = true;
return ret;
}
}
else if (function->name == "or")
{
const auto * expression_list = function->children[0]->as<ASTExpressionList>();
if (!expression_list)
{
return node;
}
ASTs lst;
for (const auto & arg : expression_list->children)
{
lst.push_back(distribute(arg));
}
auto ret = lst.size() > 1
? makeASTFunction("or", lst)
: lst[0];
return ret;
}
}
return node;
}
public:
void process(const ASTSelectQuery & select_query, const TablesWithColumns & tables)
{
const ASTTablesInSelectQueryElement * node = select_query.join();
if (!node || tables.size() < 2)
{
return;
}
auto & table_join = node->table_join->as<ASTTableJoin &>();
if (!table_join.on_expression || table_join.strictness == ASTTableJoin::Strictness::Asof)
{
return;
}
normTree(table_join.on_expression);
auto distributed_expression = distribute(table_join.on_expression);
normTree(distributed_expression);
LOG_TRACE(&Poco::Logger::get("TreeRewrite"), "bottom of toDNF: {}, node_added {}",
distributed_expression->dumpTree(), node_added);
table_join.on_expression = distributed_expression;
table_join.converted_to_dnf = node_added;
}
};
2019-12-27 19:45:41 +00:00
/// Find the columns that are obtained by JOIN.
void collectJoinedColumns(TableJoin & analyzed_join, const ASTTableJoin & table_join,
const TablesWithColumns & tables, const Aliases & aliases)
2019-12-27 19:45:41 +00:00
{
assert(tables.size() >= 2);
2019-12-27 19:45:41 +00:00
if (table_join.using_expression_list)
{
const auto & keys = table_join.using_expression_list->as<ASTExpressionList &>();
for (const auto & key : keys.children)
analyzed_join.addUsingKey(key);
}
else if (table_join.on_expression)
{
bool is_asof = (table_join.strictness == ASTTableJoin::Strictness::Asof);
CollectJoinOnKeysVisitor::Data data{analyzed_join, tables[0], tables[1], aliases, is_asof};
2019-12-27 19:45:41 +00:00
CollectJoinOnKeysVisitor(data).visit(table_join.on_expression);
2021-09-06 10:59:18 +00:00
for (const auto & onexpr : analyzed_join.getClauses())
{
2021-09-06 10:59:18 +00:00
if (onexpr.key_names_left.empty())
2021-06-25 12:03:10 +00:00
{
throw Exception("Cannot get JOIN keys from JOIN ON section: " + queryToString(table_join.on_expression),
ErrorCodes::INVALID_JOIN_ON_EXPRESSION);
}
}
2021-09-15 11:16:10 +00:00
data.optimize();
2019-12-27 19:45:41 +00:00
if (is_asof)
{
if (!analyzed_join.oneDisjunct())
throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "ASOF join doesn't support multiple ORs for keys in JOIN ON section");
2019-12-27 19:45:41 +00:00
data.asofToJoinKeys();
}
if (!analyzed_join.oneDisjunct() && !analyzed_join.forceHashJoin())
throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "Only `hash` join supports multiple ORs for keys in JOIN ON section");
2019-12-27 19:45:41 +00:00
}
}
2020-12-12 16:42:15 +00:00
2020-02-27 15:06:04 +00:00
std::vector<const ASTFunction *> getAggregates(ASTPtr & query, const ASTSelectQuery & select_query)
2019-12-27 19:45:41 +00:00
{
2020-02-27 15:06:04 +00:00
/// There can not be aggregate functions inside the WHERE and PREWHERE.
if (select_query.where())
assertNoAggregates(select_query.where(), "in WHERE");
if (select_query.prewhere())
assertNoAggregates(select_query.prewhere(), "in PREWHERE");
GetAggregatesVisitor::Data data;
GetAggregatesVisitor(data).visit(query);
/// There can not be other aggregate functions within the aggregate functions.
for (const ASTFunction * node : data.aggregates)
2020-12-09 11:14:40 +00:00
{
if (node->arguments)
2020-12-09 11:14:40 +00:00
{
for (auto & arg : node->arguments->children)
{
assertNoAggregates(arg, "inside another aggregate function");
// We also can't have window functions inside aggregate functions,
// because the window functions are calculated later.
2020-12-18 17:13:28 +00:00
assertNoWindows(arg, "inside an aggregate function");
}
2020-12-09 11:14:40 +00:00
}
}
2020-02-27 15:06:04 +00:00
return data.aggregates;
2019-12-27 19:45:41 +00:00
}
2020-12-09 11:14:40 +00:00
std::vector<const ASTFunction *> getWindowFunctions(ASTPtr & query, const ASTSelectQuery & select_query)
{
/// There can not be window functions inside the WHERE, PREWHERE and HAVING
if (select_query.having())
assertNoWindows(select_query.having(), "in HAVING");
2020-12-09 11:14:40 +00:00
if (select_query.where())
assertNoWindows(select_query.where(), "in WHERE");
if (select_query.prewhere())
assertNoWindows(select_query.prewhere(), "in PREWHERE");
2021-01-13 19:29:52 +00:00
if (select_query.window())
assertNoWindows(select_query.window(), "in WINDOW");
2020-12-09 11:14:40 +00:00
GetAggregatesVisitor::Data data;
GetAggregatesVisitor(data).visit(query);
/// Window functions cannot be inside aggregates or other window functions.
/// Aggregate functions can be inside window functions because they are
/// calculated earlier.
2020-12-09 11:14:40 +00:00
for (const ASTFunction * node : data.window_functions)
{
if (node->arguments)
2020-12-09 11:14:40 +00:00
{
for (auto & arg : node->arguments->children)
{
assertNoWindows(arg, "inside another window function");
}
2020-12-09 11:14:40 +00:00
}
2021-01-13 19:29:52 +00:00
if (node->window_definition)
{
2021-01-13 19:29:52 +00:00
assertNoWindows(node->window_definition, "inside window definition");
}
2020-12-09 11:14:40 +00:00
}
return data.window_functions;
}
class MarkTupleLiteralsAsLegacyData
{
public:
using TypeToVisit = ASTLiteral;
2021-08-04 17:38:10 +00:00
static void visit(ASTLiteral & literal, ASTPtr &)
{
if (literal.value.getType() == Field::Types::Tuple)
literal.use_legacy_column_name_of_tuple = true;
}
};
using MarkTupleLiteralsAsLegacyMatcher = OneTypeMatcher<MarkTupleLiteralsAsLegacyData>;
using MarkTupleLiteralsAsLegacyVisitor = InDepthNodeVisitor<MarkTupleLiteralsAsLegacyMatcher, true>;
void markTupleLiteralsAsLegacy(ASTPtr & query)
{
MarkTupleLiteralsAsLegacyVisitor::Data data;
MarkTupleLiteralsAsLegacyVisitor(data).visit(query);
}
2019-12-27 19:45:41 +00:00
}
TreeRewriterResult::TreeRewriterResult(
const NamesAndTypesList & source_columns_,
ConstStoragePtr storage_,
const StorageMetadataPtr & metadata_snapshot_,
bool add_special)
: storage(storage_)
, metadata_snapshot(metadata_snapshot_)
, source_columns(source_columns_)
{
collectSourceColumns(add_special);
is_remote_storage = storage && storage->isRemote();
}
2020-03-03 14:25:45 +00:00
/// Add columns from storage to source_columns list. Deduplicate resulted list.
2020-04-29 12:15:23 +00:00
/// Special columns are non physical columns, for example ALIAS
void TreeRewriterResult::collectSourceColumns(bool add_special)
2020-03-03 14:25:45 +00:00
{
if (storage)
{
const ColumnsDescription & columns = metadata_snapshot->getColumns();
2020-03-03 14:25:45 +00:00
2020-12-22 16:40:53 +00:00
NamesAndTypesList columns_from_storage;
if (storage->supportsSubcolumns())
columns_from_storage = add_special ? columns.getAllWithSubcolumns() : columns.getAllPhysicalWithSubcolumns();
else
columns_from_storage = add_special ? columns.getAll() : columns.getAllPhysical();
2020-03-03 14:25:45 +00:00
if (source_columns.empty())
source_columns.swap(columns_from_storage);
else
source_columns.insert(source_columns.end(), columns_from_storage.begin(), columns_from_storage.end());
}
source_columns_set = removeDuplicateColumns(source_columns);
}
2019-12-27 19:45:41 +00:00
/// Calculate which columns are required to execute the expression.
/// Then, delete all other columns from the list of available columns.
/// After execution, columns will only contain the list of columns needed to read from the table.
void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select)
2019-12-27 19:45:41 +00:00
{
2020-01-11 09:50:41 +00:00
/// We calculate required_source_columns with source_columns modifications and swap them on exit
2019-12-27 19:45:41 +00:00
required_source_columns = source_columns;
RequiredSourceColumnsVisitor::Data columns_context;
RequiredSourceColumnsVisitor(columns_context).visit(query);
NameSet source_column_names;
for (const auto & column : source_columns)
source_column_names.insert(column.name);
NameSet required = columns_context.requiredColumns();
if (columns_context.has_table_join)
{
2020-01-11 09:50:41 +00:00
NameSet available_columns;
2019-12-27 19:45:41 +00:00
for (const auto & name : source_columns)
2020-01-11 09:50:41 +00:00
available_columns.insert(name.name);
2019-12-27 19:45:41 +00:00
/// Add columns obtained by JOIN (if needed).
for (const auto & joined_column : analyzed_join->columnsFromJoinedTable())
{
2020-04-22 06:01:33 +00:00
const auto & name = joined_column.name;
2020-01-11 09:50:41 +00:00
if (available_columns.count(name))
2019-12-27 19:45:41 +00:00
continue;
if (required.count(name))
{
/// Optimisation: do not add columns needed only in JOIN ON section.
/// Does not work well if AST was altered, the problem here is DNFing applied to table_join.on_query,
/// not to original query, so calculation of 'how many times a column occurred in a query' is not affected by DNFing,
/// but calculation of 'how many times it occurred in join' is affected.
2021-06-25 12:03:10 +00:00
if (columns_context.converted_to_dnf || columns_context.nameInclusion(name) > analyzed_join->rightKeyInclusion(name))
2019-12-27 19:45:41 +00:00
analyzed_join->addJoinedColumn(joined_column);
required.erase(name);
}
}
}
NameSet array_join_sources;
if (columns_context.has_array_join)
{
/// Insert the columns required for the ARRAY JOIN calculation into the required columns list.
for (const auto & result_source : array_join_result_to_source)
array_join_sources.insert(result_source.second);
for (const auto & column_name_type : source_columns)
if (array_join_sources.count(column_name_type.name))
required.insert(column_name_type.name);
}
/// Figure out if we're able to use the trivial count optimization.
has_explicit_columns = !required.empty();
if (is_select && !has_explicit_columns)
2019-12-27 19:45:41 +00:00
{
2020-06-04 22:01:40 +00:00
optimize_trivial_count = true;
/// You need to read at least one column to find the number of rows.
2019-12-27 19:45:41 +00:00
/// We will find a column with minimum <compressed_size, type_size, uncompressed_size>.
/// Because it is the column that is cheapest to read.
struct ColumnSizeTuple
{
size_t compressed_size;
size_t type_size;
size_t uncompressed_size;
String name;
2020-06-04 22:01:40 +00:00
2019-12-27 19:45:41 +00:00
bool operator<(const ColumnSizeTuple & that) const
{
return std::tie(compressed_size, type_size, uncompressed_size)
< std::tie(that.compressed_size, that.type_size, that.uncompressed_size);
}
};
2020-06-04 22:01:40 +00:00
2019-12-27 19:45:41 +00:00
std::vector<ColumnSizeTuple> columns;
if (storage)
{
auto column_sizes = storage->getColumnSizes();
for (auto & source_column : source_columns)
{
auto c = column_sizes.find(source_column.name);
if (c == column_sizes.end())
continue;
size_t type_size = source_column.type->haveMaximumSizeOfValue() ? source_column.type->getMaximumSizeOfValueInMemory() : 100;
columns.emplace_back(ColumnSizeTuple{c->second.data_compressed, type_size, c->second.data_uncompressed, source_column.name});
}
}
2020-06-04 22:01:40 +00:00
2020-03-09 00:28:05 +00:00
if (!columns.empty())
2019-12-27 19:45:41 +00:00
required.insert(std::min_element(columns.begin(), columns.end())->name);
else if (!source_columns.empty())
2019-12-27 19:45:41 +00:00
/// If we have no information about columns sizes, choose a column of minimum size of its data type.
required.insert(ExpressionActions::getSmallestColumn(source_columns));
}
else if (is_select && metadata_snapshot && !columns_context.has_array_join)
2020-09-21 10:13:01 +00:00
{
const auto & partition_desc = metadata_snapshot->getPartitionKey();
if (partition_desc.expression)
{
2021-03-03 08:36:20 +00:00
auto partition_source_columns = partition_desc.expression->getRequiredColumns();
partition_source_columns.push_back("_part");
partition_source_columns.push_back("_partition_id");
partition_source_columns.push_back("_part_uuid");
2021-04-27 08:15:59 +00:00
partition_source_columns.push_back("_partition_value");
2020-09-21 10:13:01 +00:00
optimize_trivial_count = true;
for (const auto & required_column : required)
{
if (std::find(partition_source_columns.begin(), partition_source_columns.end(), required_column)
== partition_source_columns.end())
{
2020-12-13 09:33:02 +00:00
optimize_trivial_count = false;
break;
2020-09-21 10:13:01 +00:00
}
}
}
}
2019-12-27 19:45:41 +00:00
NameSet unknown_required_source_columns = required;
for (NamesAndTypesList::iterator it = source_columns.begin(); it != source_columns.end();)
{
const String & column_name = it->name;
unknown_required_source_columns.erase(column_name);
if (!required.count(column_name))
source_columns.erase(it++);
else
++it;
}
/// If there are virtual columns among the unknown columns. Remove them from the list of unknown and add
/// in columns list, so that when further processing they are also considered.
if (storage)
{
2020-04-27 13:55:30 +00:00
const auto storage_virtuals = storage->getVirtuals();
2019-12-27 19:45:41 +00:00
for (auto it = unknown_required_source_columns.begin(); it != unknown_required_source_columns.end();)
{
auto column = storage_virtuals.tryGetByName(*it);
if (column)
2019-12-27 19:45:41 +00:00
{
source_columns.push_back(*column);
2019-12-27 19:45:41 +00:00
unknown_required_source_columns.erase(it++);
}
else
++it;
}
}
if (!unknown_required_source_columns.empty())
{
2020-11-10 18:22:26 +00:00
WriteBufferFromOwnString ss;
2019-12-27 19:45:41 +00:00
ss << "Missing columns:";
for (const auto & name : unknown_required_source_columns)
ss << " '" << name << "'";
ss << " while processing query: '" << queryToString(query) << "'";
ss << ", required columns:";
for (const auto & name : columns_context.requiredColumns())
ss << " '" << name << "'";
2020-12-21 13:46:55 +00:00
if (storage)
2019-12-27 19:45:41 +00:00
{
2021-01-29 08:14:34 +00:00
std::vector<String> hint_name{};
2020-12-21 13:46:55 +00:00
for (const auto & name : columns_context.requiredColumns())
{
auto hints = storage->getHints(name);
2021-01-29 08:14:34 +00:00
hint_name.insert(hint_name.end(), hints.begin(), hints.end());
}
if (!hint_name.empty())
{
ss << ", maybe you meant: ";
2021-01-29 08:14:34 +00:00
ss << toString(hint_name);
2020-12-21 13:46:55 +00:00
}
2019-12-27 19:45:41 +00:00
}
else
{
2020-12-21 13:46:55 +00:00
if (!source_column_names.empty())
for (const auto & name : columns_context.requiredColumns())
ss << " '" << name << "'";
else
ss << ", no source columns";
2019-12-27 19:45:41 +00:00
}
if (columns_context.has_table_join)
{
ss << ", joined columns:";
for (const auto & column : analyzed_join->columnsFromJoinedTable())
ss << " '" << column.name << "'";
}
if (!array_join_sources.empty())
{
ss << ", arrayJoin columns:";
for (const auto & name : array_join_sources)
ss << " '" << name << "'";
}
throw Exception(ss.str(), ErrorCodes::UNKNOWN_IDENTIFIER);
}
required_source_columns.swap(source_columns);
2021-06-25 12:03:10 +00:00
for (const auto & column : required_source_columns)
{
source_column_names.insert(column.name);
}
2019-12-27 19:45:41 +00:00
}
2020-12-12 16:42:15 +00:00
NameSet TreeRewriterResult::getArrayJoinSourceNameSet() const
{
NameSet forbidden_columns;
for (const auto & elem : array_join_result_to_source)
forbidden_columns.insert(elem.first);
return forbidden_columns;
}
2020-06-04 22:01:40 +00:00
TreeRewriterResultPtr TreeRewriter::analyzeSelect(
2019-12-27 19:45:41 +00:00
ASTPtr & query,
TreeRewriterResult && result,
const SelectQueryOptions & select_options,
2020-03-03 14:25:45 +00:00
const std::vector<TableWithColumnNamesAndTypes> & tables_with_columns,
2020-04-08 18:59:52 +00:00
const Names & required_result_columns,
std::shared_ptr<TableJoin> table_join) const
2019-12-27 19:45:41 +00:00
{
auto * select_query = query->as<ASTSelectQuery>();
if (!select_query)
throw Exception("Select analyze for not select asts.", ErrorCodes::LOGICAL_ERROR);
size_t subquery_depth = select_options.subquery_depth;
bool remove_duplicates = select_options.remove_duplicates;
const auto & settings = getContext()->getSettingsRef();
2019-12-27 19:45:41 +00:00
2020-03-03 14:25:45 +00:00
const NameSet & source_columns_set = result.source_columns_set;
if (table_join)
{
result.analyzed_join = table_join;
result.analyzed_join->resetCollected();
}
else /// TODO: remove. For now ExpressionAnalyzer expects some not empty object here
2020-04-08 18:59:52 +00:00
result.analyzed_join = std::make_shared<TableJoin>();
2019-12-27 19:45:41 +00:00
if (remove_duplicates)
renameDuplicatedColumns(select_query);
2019-12-27 19:45:41 +00:00
2020-03-03 14:25:45 +00:00
if (tables_with_columns.size() > 1)
{
const auto & right_table = tables_with_columns[1];
auto & cols_from_joined = result.analyzed_join->columns_from_joined_table;
cols_from_joined = right_table.columns;
/// query can use materialized or aliased columns from right joined table,
/// we want to request it for right table
cols_from_joined.insert(cols_from_joined.end(), right_table.hidden_columns.begin(), right_table.hidden_columns.end());
result.analyzed_join->deduplicateAndQualifyColumnNames(
source_columns_set, right_table.table.getQualifiedNamePrefix());
}
2019-12-27 19:45:41 +00:00
translateQualifiedNames(query, *select_query, source_columns_set, tables_with_columns);
2019-12-27 19:45:41 +00:00
/// Optimizes logical expressions.
LogicalExpressionsOptimizer(select_query, settings.optimize_min_equality_disjunction_chain_length.value).perform();
2019-12-27 19:45:41 +00:00
NameSet all_source_columns_set = source_columns_set;
if (table_join)
{
for (const auto & [name, _] : table_join->columns_from_joined_table)
all_source_columns_set.insert(name);
2021-06-25 12:03:10 +00:00
DNF().process(*select_query, tables_with_columns);
}
2021-06-07 20:59:38 +00:00
normalize(query, result.aliases, all_source_columns_set, select_options.ignore_alias, settings, /* allow_self_aliases = */ true);
2019-12-27 19:45:41 +00:00
/// Remove unneeded columns according to 'required_result_columns'.
/// Leave all selected columns in case of DISTINCT; columns that contain arrayJoin function inside.
/// Must be after 'normalizeTree' (after expanding aliases, for aliases not get lost)
/// and before 'executeScalarSubqueries', 'analyzeAggregation', etc. to avoid excessive calculations.
removeUnneededColumnsFromSelectClause(select_query, required_result_columns, remove_duplicates);
2019-12-27 19:45:41 +00:00
/// Executing scalar subqueries - replacing them with constant values.
executeScalarSubqueries(query, getContext(), subquery_depth, result.scalars, select_options.only_analyze);
2019-12-27 19:45:41 +00:00
if (settings.legacy_column_name_of_tuple_literal)
markTupleLiteralsAsLegacy(query);
2021-05-21 18:48:19 +00:00
TreeOptimizer::apply(query, result, tables_with_columns, getContext());
/// array_join_alias_to_name, array_join_result_to_source.
getArrayJoinedColumns(query, result, select_query, result.source_columns, source_columns_set);
2019-12-27 19:45:41 +00:00
2021-05-21 17:01:21 +00:00
setJoinStrictness(
*select_query, settings.join_default_strictness, settings.any_join_distinct_right_table_keys, result.analyzed_join->table_join);
auto * table_join_ast = select_query->join() ? select_query->join()->table_join->as<ASTTableJoin>() : nullptr;
if (table_join_ast && tables_with_columns.size() >= 2)
collectJoinedColumns(*result.analyzed_join, *table_join_ast, tables_with_columns, result.aliases);
2019-12-27 19:45:41 +00:00
2021-06-17 15:00:03 +00:00
result.aggregates = getAggregates(query, *select_query);
result.window_function_asts = getWindowFunctions(query, *select_query);
result.collectUsedColumns(query, true);
result.required_source_columns_before_expanding_alias_columns = result.required_source_columns.getNames();
2021-01-12 13:54:11 +00:00
/// rewrite filters for select query, must go after getArrayJoinedColumns
bool is_initiator = getContext()->getClientInfo().distributed_depth == 0;
if (settings.optimize_respect_aliases && result.metadata_snapshot && is_initiator)
2020-12-12 16:42:15 +00:00
{
std::unordered_set<IAST *> excluded_nodes;
{
/// Do not replace ALIASed columns in JOIN ON/USING sections
if (table_join_ast && table_join_ast->on_expression)
excluded_nodes.insert(table_join_ast->on_expression.get());
if (table_join_ast && table_join_ast->using_expression_list)
excluded_nodes.insert(table_join_ast->using_expression_list.get());
}
bool is_changed = replaceAliasColumnsInQuery(query, result.metadata_snapshot->getColumns(),
result.array_join_result_to_source, getContext(), excluded_nodes);
2021-07-18 15:27:19 +00:00
/// If query is changed, we need to redo some work to correct name resolution.
if (is_changed)
2021-07-18 15:27:19 +00:00
{
result.aggregates = getAggregates(query, *select_query);
result.window_function_asts = getWindowFunctions(query, *select_query);
result.collectUsedColumns(query, true);
}
2020-12-12 16:42:15 +00:00
}
2020-06-25 20:59:10 +00:00
result.ast_join = select_query->join();
2020-06-04 22:01:40 +00:00
if (result.optimize_trivial_count)
result.optimize_trivial_count = settings.optimize_trivial_count_query &&
2020-09-21 10:13:01 +00:00
!select_query->groupBy() && !select_query->having() &&
2020-06-04 22:01:40 +00:00
!select_query->sampleSize() && !select_query->sampleOffset() && !select_query->final() &&
(tables_with_columns.size() < 2 || isLeft(result.analyzed_join->kind()));
2020-06-04 22:01:40 +00:00
return std::make_shared<const TreeRewriterResult>(result);
2019-12-27 19:45:41 +00:00
}
TreeRewriterResultPtr TreeRewriter::analyze(
ASTPtr & query,
const NamesAndTypesList & source_columns,
ConstStoragePtr storage,
const StorageMetadataPtr & metadata_snapshot,
2021-06-08 09:54:00 +00:00
bool allow_aggregations,
bool allow_self_aliases) const
{
2020-02-27 18:05:49 +00:00
if (query->as<ASTSelectQuery>())
throw Exception("Not select analyze for select asts.", ErrorCodes::LOGICAL_ERROR);
const auto & settings = getContext()->getSettingsRef();
TreeRewriterResult result(source_columns, storage, metadata_snapshot, false);
2021-06-08 09:54:00 +00:00
normalize(query, result.aliases, result.source_columns_set, false, settings, allow_self_aliases);
2020-02-26 22:19:16 +00:00
/// Executing scalar subqueries. Column defaults could be a scalar subquery.
executeScalarSubqueries(query, getContext(), 0, result.scalars, false);
2020-02-26 22:19:16 +00:00
if (settings.legacy_column_name_of_tuple_literal)
markTupleLiteralsAsLegacy(query);
TreeOptimizer::optimizeIf(query, result.aliases, settings.optimize_if_chain_to_multiif);
2020-05-14 23:32:45 +00:00
if (allow_aggregations)
{
GetAggregatesVisitor::Data data;
GetAggregatesVisitor(data).visit(query);
/// There can not be other aggregate functions within the aggregate functions.
for (const ASTFunction * node : data.aggregates)
for (auto & arg : node->arguments->children)
assertNoAggregates(arg, "inside another aggregate function");
result.aggregates = data.aggregates;
}
2020-05-12 20:44:48 +00:00
else
assertNoAggregates(query, "in wrong place");
2020-12-13 09:33:02 +00:00
result.collectUsedColumns(query, false);
return std::make_shared<const TreeRewriterResult>(result);
}
void TreeRewriter::normalize(
2021-06-07 20:59:38 +00:00
ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, bool ignore_alias, const Settings & settings, bool allow_self_aliases)
{
UserDefinedSQLFunctionVisitor::Data data_user_defined_functions_visitor;
UserDefinedSQLFunctionVisitor(data_user_defined_functions_visitor).visit(query);
CustomizeCountDistinctVisitor::Data data_count_distinct{settings.count_distinct_implementation};
CustomizeCountDistinctVisitor(data_count_distinct).visit(query);
CustomizeCountIfDistinctVisitor::Data data_count_if_distinct{settings.count_distinct_implementation.toString() + "If"};
CustomizeCountIfDistinctVisitor(data_count_if_distinct).visit(query);
CustomizeIfDistinctVisitor::Data data_distinct_if{"DistinctIf"};
CustomizeIfDistinctVisitor(data_distinct_if).visit(query);
2020-04-06 13:30:16 +00:00
if (settings.transform_null_in)
{
CustomizeInVisitor::Data data_null_in{"nullIn"};
CustomizeInVisitor(data_null_in).visit(query);
CustomizeNotInVisitor::Data data_not_null_in{"notNullIn"};
CustomizeNotInVisitor(data_not_null_in).visit(query);
CustomizeGlobalInVisitor::Data data_global_null_in{"globalNullIn"};
CustomizeGlobalInVisitor(data_global_null_in).visit(query);
CustomizeGlobalNotInVisitor::Data data_global_not_null_in{"globalNotNullIn"};
CustomizeGlobalNotInVisitor(data_global_not_null_in).visit(query);
}
2021-04-15 16:40:49 +00:00
// Try to fuse sum/avg/count with identical arguments to one sumCount call,
// if we have at least two different functions. E.g. we will replace sum(x)
// and count(x) with sumCount(x).1 and sumCount(x).2, and sumCount() will
// be calculated only once because of CSE.
if (settings.optimize_fuse_sum_count_avg || settings.optimize_syntax_fuse_functions)
{
2021-04-15 16:40:49 +00:00
FuseSumCountAggregatesVisitor::Data data;
FuseSumCountAggregatesVisitor(data).visit(query);
fuseSumCountAggregates(data.fuse_map);
}
2021-03-08 03:58:18 +00:00
/// Rewrite all aggregate functions to add -OrNull suffix to them
if (settings.aggregate_functions_null_for_empty)
{
CustomizeAggregateFunctionsOrNullVisitor::Data data_or_null{"OrNull"};
CustomizeAggregateFunctionsOrNullVisitor(data_or_null).visit(query);
}
2020-12-09 04:30:38 +00:00
/// Move -OrNull suffix ahead, this should execute after add -OrNull suffix
CustomizeAggregateFunctionsMoveOrNullVisitor::Data data_or_null{"OrNull"};
CustomizeAggregateFunctionsMoveOrNullVisitor(data_or_null).visit(query);
/// Creates a dictionary `aliases`: alias -> ASTPtr
QueryAliasesVisitor(aliases).visit(query);
/// Mark table ASTIdentifiers with not a column marker
MarkTableIdentifiersVisitor::Data identifiers_data{aliases};
MarkTableIdentifiersVisitor(identifiers_data).visit(query);
2021-02-14 11:09:36 +00:00
/// Rewrite function names to their canonical ones.
2021-02-18 03:27:24 +00:00
if (settings.normalize_function_names)
FunctionNameNormalizer().visit(query.get());
2021-02-14 11:09:36 +00:00
/// Common subexpression elimination. Rewrite rules.
2021-06-07 20:59:38 +00:00
QueryNormalizer::Data normalizer_data(aliases, source_columns_set, ignore_alias, settings, allow_self_aliases);
QueryNormalizer(normalizer_data).visit(query);
}
2019-12-27 19:45:41 +00:00
}