2019-12-27 19:45:41 +00:00
|
|
|
#include <Core/Settings.h>
|
2020-04-15 20:28:05 +00:00
|
|
|
#include <Core/Defines.h>
|
2019-12-27 19:45:41 +00:00
|
|
|
#include <Core/NamesAndTypes.h>
|
2019-12-27 19:42:14 +00:00
|
|
|
|
2020-07-22 17:13:05 +00:00
|
|
|
#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>
|
|
|
|
#include <Interpreters/MarkTableIdentifiersVisitor.h>
|
|
|
|
#include <Interpreters/QueryNormalizer.h>
|
|
|
|
#include <Interpreters/ExecuteScalarSubqueriesVisitor.h>
|
|
|
|
#include <Interpreters/CollectJoinOnKeysVisitor.h>
|
|
|
|
#include <Interpreters/RequiredSourceColumnsVisitor.h>
|
|
|
|
#include <Interpreters/GetAggregatesVisitor.h>
|
2020-04-07 09:48:47 +00:00
|
|
|
#include <Interpreters/TableJoin.h>
|
2019-12-27 19:45:41 +00:00
|
|
|
#include <Interpreters/ExpressionActions.h> /// getSmallestColumn()
|
|
|
|
#include <Interpreters/getTableExpressions.h>
|
2020-07-22 17:13:05 +00:00
|
|
|
#include <Interpreters/TreeOptimizer.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 <DataTypes/NestedUtils.h>
|
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
|
|
|
|
|
|
|
#include <IO/WriteHelpers.h>
|
|
|
|
#include <Storages/IStorage.h>
|
|
|
|
|
2020-10-18 12:18:31 +00:00
|
|
|
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
2019-12-27 19:45:41 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int EMPTY_NESTED_TABLE;
|
2020-02-28 11:03:09 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2019-12-27 19:45:41 +00:00
|
|
|
extern const int INVALID_JOIN_ON_EXPRESSION;
|
|
|
|
extern const int EMPTY_LIST_OF_COLUMNS_QUERIED;
|
|
|
|
extern const int NOT_IMPLEMENTED;
|
|
|
|
extern const int UNKNOWN_IDENTIFIER;
|
|
|
|
extern const int EXPECTED_ALL_OR_ANY;
|
|
|
|
}
|
|
|
|
|
|
|
|
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 &)
|
|
|
|
{
|
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";
|
2020-06-22 14:55:49 +00:00
|
|
|
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
|
|
|
|
2020-06-22 14:55:49 +00:00
|
|
|
template <char const * func_suffix>
|
|
|
|
struct CustomizeFunctionsSuffixData
|
|
|
|
{
|
|
|
|
using TypeToVisit = ASTFunction;
|
|
|
|
|
|
|
|
const String & customized_func_suffix;
|
|
|
|
|
|
|
|
void visit(ASTFunction & func, ASTPtr &)
|
|
|
|
{
|
|
|
|
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
|
|
|
|
2020-10-18 12:18:31 +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 &)
|
|
|
|
{
|
|
|
|
if (AggregateFunctionFactory::instance().isAggregateFunctionName(func.name)
|
|
|
|
&& !endsWith(func.name, customized_func_suffix))
|
|
|
|
{
|
|
|
|
func.name = func.name + customized_func_suffix;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
using CustomizeAggregateFunctionsOrNullVisitor = InDepthNodeVisitor<OneTypeMatcher<CustomizeAggregateFunctionsSuffixData>, 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,
|
2020-06-05 21:17:00 +00:00
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
|
|
|
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);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
elements = std::move(new_elements);
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Replacing scalar subqueries with constant values.
|
2020-04-30 17:07:34 +00:00
|
|
|
void executeScalarSubqueries(ASTPtr & query, const Context & context, size_t subquery_depth, Scalars & scalars, bool only_analyze)
|
2019-12-27 19:45:41 +00:00
|
|
|
{
|
|
|
|
LogAST log;
|
2020-04-30 17:07:34 +00:00
|
|
|
ExecuteScalarSubqueriesVisitor::Data visitor_data{context, subquery_depth, scalars, only_analyze};
|
2019-12-27 19:45:41 +00:00
|
|
|
ExecuteScalarSubqueriesVisitor(visitor_data, log.stream()).visit(query);
|
|
|
|
}
|
|
|
|
|
2020-07-22 17:13:05 +00:00
|
|
|
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)
|
|
|
|
{
|
2020-03-23 02:12:31 +00:00
|
|
|
if (ASTPtr array_join_expression_list = select_query->arrayJoinExpressionList())
|
2019-12-27 19:45:41 +00:00
|
|
|
{
|
|
|
|
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())
|
|
|
|
{
|
2020-03-23 02:12:31 +00:00
|
|
|
ASTPtr expr = select_query->arrayJoinExpressionList()->children.at(0);
|
2019-12-27 19:45:41 +00:00
|
|
|
String source_name = expr->getColumnName();
|
|
|
|
String result_name = expr->getAliasOrColumnName();
|
|
|
|
|
|
|
|
/// 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)
|
|
|
|
{
|
2020-08-08 01:01:47 +00:00
|
|
|
auto split = Nested::splitName(column.name);
|
|
|
|
if (split.first == source_name && !split.second.empty())
|
2019-12-27 19:45:41 +00:00
|
|
|
{
|
2020-08-08 01:01:47 +00:00
|
|
|
result.array_join_result_to_source[Nested::concatenateName(result_name, split.second)] = column.name;
|
2019-12-27 19:45:41 +00:00
|
|
|
found = true;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if (!found)
|
|
|
|
throw Exception("No columns in nested table " + source_name, ErrorCodes::EMPTY_NESTED_TABLE);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
2020-02-02 14:07:50 +00:00
|
|
|
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;
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Find the columns that are obtained by JOIN.
|
2020-04-07 09:48:47 +00:00
|
|
|
void collectJoinedColumns(TableJoin & analyzed_join, const ASTSelectQuery & select_query,
|
2020-06-05 21:17:00 +00:00
|
|
|
const TablesWithColumns & tables, const Aliases & aliases)
|
2019-12-27 19:45:41 +00:00
|
|
|
{
|
|
|
|
const ASTTablesInSelectQueryElement * node = select_query.join();
|
|
|
|
if (!node)
|
|
|
|
return;
|
|
|
|
|
|
|
|
const auto & table_join = node->table_join->as<ASTTableJoin &>();
|
|
|
|
|
|
|
|
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};
|
|
|
|
CollectJoinOnKeysVisitor(data).visit(table_join.on_expression);
|
|
|
|
if (!data.has_some)
|
|
|
|
throw Exception("Cannot get JOIN keys from JOIN ON section: " + queryToString(table_join.on_expression),
|
|
|
|
ErrorCodes::INVALID_JOIN_ON_EXPRESSION);
|
|
|
|
if (is_asof)
|
|
|
|
data.asofToJoinKeys();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
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)
|
|
|
|
for (auto & arg : node->arguments->children)
|
|
|
|
assertNoAggregates(arg, "inside another aggregate function");
|
|
|
|
return data.aggregates;
|
2019-12-27 19:45:41 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|
|
|
|
|
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
|
2020-07-22 17:13:05 +00:00
|
|
|
void TreeRewriterResult::collectSourceColumns(bool add_special)
|
2020-03-03 14:25:45 +00:00
|
|
|
{
|
|
|
|
if (storage)
|
|
|
|
{
|
2020-06-17 16:39:58 +00:00
|
|
|
const ColumnsDescription & columns = metadata_snapshot->getColumns();
|
2020-03-03 14:25:45 +00:00
|
|
|
|
2020-04-29 12:15:23 +00:00
|
|
|
auto 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.
|
2020-07-22 17:13:05 +00:00
|
|
|
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.
|
|
|
|
if (columns_context.nameInclusion(name) > analyzed_join->rightKeyInclusion(name))
|
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
|
|
|
/// You need to read at least one column to find the number of rows.
|
2020-06-04 22:01:40 +00:00
|
|
|
if (is_select && required.empty())
|
2019-12-27 19:45:41 +00:00
|
|
|
{
|
2020-06-04 22:01:40 +00:00
|
|
|
optimize_trivial_count = true;
|
|
|
|
|
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 we have no information about columns sizes, choose a column of minimum size of its data type.
|
|
|
|
required.insert(ExpressionActions::getSmallestColumn(source_columns));
|
|
|
|
}
|
2020-09-21 10:13:01 +00:00
|
|
|
else if (is_select && metadata_snapshot)
|
|
|
|
{
|
|
|
|
const auto & partition_desc = metadata_snapshot->getPartitionKey();
|
|
|
|
if (partition_desc.expression)
|
|
|
|
{
|
|
|
|
const auto & partition_source_columns = partition_desc.expression->getRequiredColumns();
|
|
|
|
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())
|
|
|
|
{
|
|
|
|
optimize_trivial_count = false;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
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();)
|
|
|
|
{
|
2020-04-24 10:20:03 +00:00
|
|
|
auto column = storage_virtuals.tryGetByName(*it);
|
|
|
|
if (column)
|
2019-12-27 19:45:41 +00:00
|
|
|
{
|
2020-04-24 10:20:03 +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())
|
|
|
|
{
|
|
|
|
std::stringstream ss;
|
|
|
|
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 << "'";
|
|
|
|
|
|
|
|
if (!source_column_names.empty())
|
|
|
|
{
|
|
|
|
ss << ", source columns:";
|
|
|
|
for (const auto & name : source_column_names)
|
|
|
|
ss << " '" << name << "'";
|
|
|
|
}
|
|
|
|
else
|
|
|
|
ss << ", no source columns";
|
|
|
|
|
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
2020-06-04 22:01:40 +00:00
|
|
|
|
2020-07-22 17:13:05 +00:00
|
|
|
TreeRewriterResultPtr TreeRewriter::analyzeSelect(
|
2019-12-27 19:45:41 +00:00
|
|
|
ASTPtr & query,
|
2020-07-22 17:13:05 +00:00
|
|
|
TreeRewriterResult && result,
|
2020-02-26 19:33:09 +00:00
|
|
|
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>();
|
2020-02-26 19:33:09 +00:00
|
|
|
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;
|
|
|
|
|
2019-12-27 19:45:41 +00:00
|
|
|
const auto & settings = context.getSettingsRef();
|
|
|
|
|
2020-03-03 14:25:45 +00:00
|
|
|
const NameSet & source_columns_set = result.source_columns_set;
|
2020-05-20 09:29:23 +00:00
|
|
|
|
|
|
|
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
|
|
|
|
2020-02-27 14:33:03 +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)
|
2020-02-27 14:33:03 +00:00
|
|
|
{
|
2020-03-03 14:25:45 +00:00
|
|
|
result.analyzed_join->columns_from_joined_table = tables_with_columns[1].columns;
|
2020-02-27 14:33:03 +00:00
|
|
|
result.analyzed_join->deduplicateAndQualifyColumnNames(
|
|
|
|
source_columns_set, tables_with_columns[1].table.getQualifiedNamePrefix());
|
|
|
|
}
|
2019-12-27 19:45:41 +00:00
|
|
|
|
2020-06-05 21:17:00 +00:00
|
|
|
translateQualifiedNames(query, *select_query, source_columns_set, tables_with_columns);
|
2019-12-27 19:45:41 +00:00
|
|
|
|
2020-02-27 14:33:03 +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
|
|
|
|
2020-02-27 15:06:04 +00:00
|
|
|
normalize(query, result.aliases, settings);
|
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.
|
2020-02-26 19:33:09 +00:00
|
|
|
removeUnneededColumnsFromSelectClause(select_query, required_result_columns, remove_duplicates);
|
2019-12-27 19:45:41 +00:00
|
|
|
|
|
|
|
/// Executing scalar subqueries - replacing them with constant values.
|
2020-04-30 17:07:34 +00:00
|
|
|
executeScalarSubqueries(query, context, subquery_depth, result.scalars, select_options.only_analyze);
|
2019-12-27 19:45:41 +00:00
|
|
|
|
2020-07-22 17:13:05 +00:00
|
|
|
TreeOptimizer::apply(query, result.aliases, source_columns_set, tables_with_columns, context, result.rewrite_subqueries);
|
2020-01-04 04:31:45 +00:00
|
|
|
|
2020-07-22 17:13:05 +00:00
|
|
|
/// 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
|
|
|
|
2020-07-22 17:13:05 +00:00
|
|
|
setJoinStrictness(*select_query, settings.join_default_strictness, settings.any_join_distinct_right_table_keys,
|
|
|
|
result.analyzed_join->table_join);
|
|
|
|
collectJoinedColumns(*result.analyzed_join, *select_query, tables_with_columns, result.aliases);
|
2019-12-27 19:45:41 +00:00
|
|
|
|
2020-02-27 15:06:04 +00:00
|
|
|
result.aggregates = getAggregates(query, *select_query);
|
2020-06-04 22:01:40 +00:00
|
|
|
result.collectUsedColumns(query, true);
|
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() &&
|
2020-06-05 21:17:00 +00:00
|
|
|
(tables_with_columns.size() < 2 || isLeft(result.analyzed_join->kind()));
|
2020-06-04 22:01:40 +00:00
|
|
|
|
2020-07-22 17:13:05 +00:00
|
|
|
return std::make_shared<const TreeRewriterResult>(result);
|
2019-12-27 19:45:41 +00:00
|
|
|
}
|
|
|
|
|
2020-07-22 17:13:05 +00:00
|
|
|
TreeRewriterResultPtr TreeRewriter::analyze(
|
2020-06-17 16:39:58 +00:00
|
|
|
ASTPtr & query,
|
|
|
|
const NamesAndTypesList & source_columns,
|
|
|
|
ConstStoragePtr storage,
|
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
|
|
|
bool allow_aggregations) const
|
2020-02-26 19:33:09 +00:00
|
|
|
{
|
2020-02-27 18:05:49 +00:00
|
|
|
if (query->as<ASTSelectQuery>())
|
2020-02-26 19:33:09 +00:00
|
|
|
throw Exception("Not select analyze for select asts.", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
const auto & settings = context.getSettingsRef();
|
|
|
|
|
2020-07-22 17:13:05 +00:00
|
|
|
TreeRewriterResult result(source_columns, storage, metadata_snapshot, false);
|
2020-02-26 19:33:09 +00:00
|
|
|
|
2020-02-27 15:06:04 +00:00
|
|
|
normalize(query, result.aliases, settings);
|
2020-02-26 19:33:09 +00:00
|
|
|
|
2020-02-26 22:19:16 +00:00
|
|
|
/// Executing scalar subqueries. Column defaults could be a scalar subquery.
|
2020-04-30 17:09:19 +00:00
|
|
|
executeScalarSubqueries(query, context, 0, result.scalars, false);
|
2020-02-26 22:19:16 +00:00
|
|
|
|
2020-07-22 17:13:05 +00:00
|
|
|
TreeOptimizer::optimizeIf(query, result.aliases, settings.optimize_if_chain_to_multiif);
|
2020-02-27 14:33:03 +00:00
|
|
|
|
2020-05-14 23:32:45 +00:00
|
|
|
if (allow_aggregations)
|
2020-04-27 14:47:59 +00:00
|
|
|
{
|
|
|
|
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
|
2020-04-27 14:47:59 +00:00
|
|
|
assertNoAggregates(query, "in wrong place");
|
|
|
|
|
2020-06-04 22:01:40 +00:00
|
|
|
result.collectUsedColumns(query, false);
|
2020-07-22 17:13:05 +00:00
|
|
|
return std::make_shared<const TreeRewriterResult>(result);
|
2020-02-26 19:33:09 +00:00
|
|
|
}
|
|
|
|
|
2020-07-22 17:13:05 +00:00
|
|
|
void TreeRewriter::normalize(ASTPtr & query, Aliases & aliases, const Settings & settings)
|
2020-02-26 19:33:09 +00:00
|
|
|
{
|
2020-06-22 14:55:49 +00:00
|
|
|
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-02-26 19:33:09 +00:00
|
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
2020-10-18 12:18:31 +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-02-26 19:33:09 +00:00
|
|
|
/// Creates a dictionary `aliases`: alias -> ASTPtr
|
2020-03-18 21:38:27 +00:00
|
|
|
QueryAliasesVisitor(aliases).visit(query);
|
2020-02-26 19:33:09 +00:00
|
|
|
|
|
|
|
/// Mark table ASTIdentifiers with not a column marker
|
|
|
|
MarkTableIdentifiersVisitor::Data identifiers_data{aliases};
|
|
|
|
MarkTableIdentifiersVisitor(identifiers_data).visit(query);
|
|
|
|
|
|
|
|
/// Common subexpression elimination. Rewrite rules.
|
|
|
|
QueryNormalizer::Data normalizer_data(aliases, settings);
|
|
|
|
QueryNormalizer(normalizer_data).visit(query);
|
|
|
|
}
|
|
|
|
|
2019-12-27 19:45:41 +00:00
|
|
|
}
|