ClickHouse/dbms/src/Interpreters/ExpressionAnalyzer.cpp

1064 lines
39 KiB
C++
Raw Normal View History

#include <Poco/Util/Application.h>
2016-11-20 12:43:20 +00:00
#include <Poco/String.h>
#include <Core/Block.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTAsterisk.h>
#include <Parsers/ASTQualifiedAsterisk.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTSelectQuery.h>
2018-02-26 09:05:06 +00:00
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTOrderByElement.h>
#include <Parsers/formatAST.h>
#include <Parsers/DumpASTNode.h>
2017-12-25 00:38:45 +00:00
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypesNumber.h>
2018-08-28 14:40:07 +00:00
#include <Columns/IColumn.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/InJoinSubqueriesPreprocessor.h>
#include <Interpreters/LogicalExpressionsOptimizer.h>
#include <Interpreters/PredicateExpressionsOptimizer.h>
#include <Interpreters/ExternalDictionaries.h>
#include <Interpreters/Set.h>
#include <Interpreters/Join.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
2015-04-16 06:12:35 +00:00
#include <Storages/StorageDistributed.h>
#include <Storages/StorageMemory.h>
#include <Storages/StorageJoin.h>
#include <DataStreams/copyData.h>
#include <Dictionaries/IDictionary.h>
#include <Common/typeid_cast.h>
#include <Common/StringUtils/StringUtils.h>
2017-06-06 17:18:32 +00:00
#include <ext/range.h>
#include <DataTypes/DataTypeFactory.h>
#include <Functions/FunctionsMiscellaneous.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/parseQuery.h>
2018-07-27 16:21:43 +00:00
#include <Parsers/queryToString.h>
#include <Interpreters/interpretSubquery.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/QueryNormalizer.h>
#include <Interpreters/ActionsVisitor.h>
#include <Interpreters/ExternalTablesVisitor.h>
#include <Interpreters/GlobalSubqueriesVisitor.h>
#include <Interpreters/RequiredSourceColumnsVisitor.h>
namespace DB
{
2018-10-10 16:23:27 +00:00
using LogAST = DebugASTLog<false>; /// set to true to enable logs
namespace ErrorCodes
{
extern const int UNKNOWN_IDENTIFIER;
extern const int ILLEGAL_AGGREGATION;
extern const int EXPECTED_ALL_OR_ANY;
}
Squashed commit of the following: commit e712f469a55ff34ad34b482b15cc4153b7ad7233 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:59:13 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a002823084e3a79bffcc17d479620a68eb0644b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:58:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9e06f407c8ee781ed8ddf98bdfcc31846bf2a0fe Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:55:14 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9581620f1e839f456fa7894aa1f996d5162ac6cd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:54:22 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a8564c68cb6cc3649fafaf401256d43c9a2e777 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:47:34 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit cf60632d78ec656be3304ef4565e859bb6ce80ba Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:40:09 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit ee3d1dc6e0c4ca60e3ac1e0c30d4b3ed1e66eca0 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:22:49 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 65592ef7116a90104fcd524b53ef8b7cf22640f2 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:18:17 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 37972c257320d3b7e7b294e0fdeffff218647bfd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:17:06 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit dd909d149974ce5bed2456de1261aa5a368fd3ff Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:16:28 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 3cf43266ca7e30adf01212b1a739ba5fe43639fd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:15:42 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 6731a3df96d1609286e2536b6432916af7743f0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:13:35 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 1b5727e0d56415b7add4cb76110105358663602c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:11:18 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit bbcf726a55685b8e72f5b40ba0bf1904bd1c0407 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:09:04 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit c03b477d5e2e65014e8906ecfa2efb67ee295af1 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:06:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2986e2fb0466bc18d73693dcdded28fccc0dc66b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:05:44 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 5d6cdef13d2e02bd5c4954983334e9162ab2635b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:04:53 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit f2b819b25ce8b2ccdcb201eefb03e1e6f5aab590 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:01:47 2017 +0300 Less dependencies [#CLICKHOUSE-2]
2017-01-14 09:00:19 +00:00
ExpressionAnalyzer::ExpressionAnalyzer(
2018-08-27 17:58:43 +00:00
const ASTPtr & query_,
const SyntaxAnalyzerResultPtr & syntax_analyzer_result_,
const Context & context_,
const NamesAndTypesList & additional_source_columns,
const NameSet & required_result_columns_,
size_t subquery_depth_,
2018-02-08 17:10:35 +00:00
bool do_global_,
const SubqueriesForSets & subqueries_for_sets_)
: ExpressionAnalyzerData(syntax_analyzer_result_->source_columns, required_result_columns_, subqueries_for_sets_)
, query(query_), context(context_), settings(context.getSettings())
, subquery_depth(subquery_depth_), do_global(do_global_)
2018-11-09 17:23:48 +00:00
, syntax(syntax_analyzer_result_)
{
storage = syntax->storage;
rewrite_subqueries = syntax->rewrite_subqueries;
Squashed commit of the following: commit e712f469a55ff34ad34b482b15cc4153b7ad7233 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:59:13 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a002823084e3a79bffcc17d479620a68eb0644b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:58:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9e06f407c8ee781ed8ddf98bdfcc31846bf2a0fe Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:55:14 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9581620f1e839f456fa7894aa1f996d5162ac6cd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:54:22 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a8564c68cb6cc3649fafaf401256d43c9a2e777 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:47:34 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit cf60632d78ec656be3304ef4565e859bb6ce80ba Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:40:09 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit ee3d1dc6e0c4ca60e3ac1e0c30d4b3ed1e66eca0 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:22:49 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 65592ef7116a90104fcd524b53ef8b7cf22640f2 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:18:17 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 37972c257320d3b7e7b294e0fdeffff218647bfd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:17:06 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit dd909d149974ce5bed2456de1261aa5a368fd3ff Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:16:28 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 3cf43266ca7e30adf01212b1a739ba5fe43639fd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:15:42 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 6731a3df96d1609286e2536b6432916af7743f0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:13:35 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 1b5727e0d56415b7add4cb76110105358663602c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:11:18 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit bbcf726a55685b8e72f5b40ba0bf1904bd1c0407 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:09:04 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit c03b477d5e2e65014e8906ecfa2efb67ee295af1 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:06:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2986e2fb0466bc18d73693dcdded28fccc0dc66b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:05:44 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 5d6cdef13d2e02bd5c4954983334e9162ab2635b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:04:53 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit f2b819b25ce8b2ccdcb201eefb03e1e6f5aab590 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:01:47 2017 +0300 Less dependencies [#CLICKHOUSE-2]
2017-01-14 09:00:19 +00:00
2018-11-02 18:53:23 +00:00
select_query = typeid_cast<ASTSelectQuery *>(query.get());
2018-02-27 19:00:55 +00:00
if (!additional_source_columns.empty())
{
source_columns.insert(source_columns.end(), additional_source_columns.begin(), additional_source_columns.end());
2018-02-28 01:29:55 +00:00
removeDuplicateColumns(source_columns);
}
/// Delete the unnecessary from `source_columns` list. Form `columns_added_by_join`.
collectUsedColumns();
2017-04-02 17:37:49 +00:00
/// external_tables, subqueries_for_sets for global subqueries.
/// Replaces global subqueries with the generated names of temporary tables that will be sent to remote servers.
initGlobalSubqueriesAndExternalTables();
/// has_aggregation, aggregation_keys, aggregate_descriptions, aggregated_columns.
2017-04-02 17:37:49 +00:00
/// This analysis should be performed after processing global subqueries, because otherwise,
/// if the aggregate function contains a global subquery, then `analyzeAggregation` method will save
/// in `aggregate_descriptions` the information about the parameters of this aggregate function, among which
/// global subquery. Then, when you call `initGlobalSubqueriesAndExternalTables` method, this
/// the global subquery will be replaced with a temporary table, resulting in aggregate_descriptions
/// will contain out-of-date information, which will lead to an error when the query is executed.
analyzeAggregation();
}
2018-10-17 11:06:46 +00:00
bool ExpressionAnalyzer::isRemoteStorage() const
{
2018-10-17 11:06:46 +00:00
return storage && storage->isRemote();
}
void ExpressionAnalyzer::analyzeAggregation()
{
2017-04-02 17:37:49 +00:00
/** Find aggregation keys (aggregation_keys), information about aggregate functions (aggregate_descriptions),
* as well as a set of columns obtained after the aggregation, if any,
* or after all the actions that are usually performed before aggregation (aggregated_columns).
*
2017-04-02 17:37:49 +00:00
* Everything below (compiling temporary ExpressionActions) - only for the purpose of query analysis (type output).
*/
if (select_query && (select_query->group_expression_list || select_query->having_expression))
has_aggregation = true;
ExpressionActionsPtr temp_actions = std::make_shared<ExpressionActions>(source_columns, context);
if (select_query)
{
bool is_array_join_left;
ASTPtr array_join_expression_list = select_query->array_join_expression_list(is_array_join_left);
if (array_join_expression_list)
{
getRootActions(array_join_expression_list, true, temp_actions);
addMultipleArrayJoinAction(temp_actions, is_array_join_left);
array_join_columns = temp_actions->getSampleBlock().getNamesAndTypesList();
}
const ASTTablesInSelectQueryElement * join = select_query->join();
if (join)
{
const auto table_join = static_cast<const ASTTableJoin &>(*join->table_join);
if (table_join.using_expression_list)
getRootActions(table_join.using_expression_list, true, temp_actions);
if (table_join.on_expression)
2018-11-09 17:23:48 +00:00
for (const auto & key_ast : analyzedJoin().key_asts_left)
getRootActions(key_ast, true, temp_actions);
addJoinAction(temp_actions, true);
}
}
2018-08-27 17:58:43 +00:00
getAggregates(query, temp_actions);
if (has_aggregation)
{
assertSelect();
/// Find out aggregation keys.
if (select_query->group_expression_list)
{
NameSet unique_keys;
ASTs & group_asts = select_query->group_expression_list->children;
for (ssize_t i = 0; i < ssize_t(group_asts.size()); ++i)
{
ssize_t size = group_asts.size();
getRootActions(group_asts[i], true, temp_actions);
const auto & column_name = group_asts[i]->getColumnName();
const auto & block = temp_actions->getSampleBlock();
if (!block.has(column_name))
throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER);
const auto & col = block.getByName(column_name);
/// Constant expressions have non-null column pointer at this stage.
if (col.column && col.column->isColumnConst())
{
/// But don't remove last key column if no aggregate functions, otherwise aggregation will not work.
if (!aggregate_descriptions.empty() || size > 1)
{
if (i + 1 < static_cast<ssize_t>(size))
group_asts[i] = std::move(group_asts.back());
group_asts.pop_back();
--i;
continue;
}
}
NameAndTypePair key{column_name, col.type};
/// Aggregation keys are uniqued.
if (!unique_keys.count(key.name))
{
unique_keys.insert(key.name);
aggregation_keys.push_back(key);
/// Key is no longer needed, therefore we can save a little by moving it.
aggregated_columns.push_back(std::move(key));
}
}
if (group_asts.empty())
{
select_query->group_expression_list = nullptr;
has_aggregation = select_query->having_expression || aggregate_descriptions.size();
}
}
for (size_t i = 0; i < aggregate_descriptions.size(); ++i)
{
AggregateDescription & desc = aggregate_descriptions[i];
aggregated_columns.emplace_back(desc.column_name, desc.function->getReturnType());
}
}
else
{
aggregated_columns = temp_actions->getSampleBlock().getNamesAndTypesList();
}
}
void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables()
{
2017-04-02 17:37:49 +00:00
/// Adds existing external tables (not subqueries) to the external_tables dictionary.
ExternalTablesVisitor::Data tables_data{context, external_tables};
ExternalTablesVisitor(tables_data).visit(query);
2018-10-17 10:59:05 +00:00
if (do_global)
{
GlobalSubqueriesVisitor::Data subqueries_data(context, subquery_depth, isRemoteStorage(),
2018-10-17 10:59:05 +00:00
external_tables, subqueries_for_sets, has_global_subqueries);
GlobalSubqueriesVisitor(subqueries_data).visit(query);
2018-10-17 10:59:05 +00:00
}
}
2014-04-10 08:57:01 +00:00
void ExpressionAnalyzer::makeSetsForIndex()
{
if (storage && select_query && storage->supportsIndexForIn())
{
if (select_query->where_expression)
makeSetsForIndexImpl(select_query->where_expression);
if (select_query->prewhere_expression)
makeSetsForIndexImpl(select_query->prewhere_expression);
}
}
void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name)
{
auto set_key = PreparedSetKey::forSubquery(*subquery_or_table_name);
if (prepared_sets.count(set_key))
return; /// Already prepared.
auto interpreter_subquery = interpretSubquery(subquery_or_table_name, context, subquery_depth + 1, {});
BlockIO res = interpreter_subquery->execute();
SetPtr set = std::make_shared<Set>(settings.size_limits_for_set, true);
2018-04-19 21:34:04 +00:00
set->setHeader(res.in->getHeader());
while (Block block = res.in->read())
{
/// If the limits have been exceeded, give up and let the default subquery processing actions take place.
2018-07-02 18:57:14 +00:00
if (!set->insertFromBlock(block))
return;
}
prepared_sets[set_key] = std::move(set);
}
void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node)
{
for (auto & child : node->children)
{
/// Don't descend into subqueries.
if (typeid_cast<ASTSubquery *>(child.get()))
continue;
/// Don't descend into lambda functions
const ASTFunction * func = typeid_cast<const ASTFunction *>(child.get());
if (func && func->name == "lambda")
continue;
makeSetsForIndexImpl(child);
}
const ASTFunction * func = typeid_cast<const ASTFunction *>(node.get());
if (func && functionIsInOperator(func->name))
{
const IAST & args = *func->arguments;
2018-03-16 06:39:32 +00:00
if (storage && storage->mayBenefitFromIndexForIn(args.children.at(0)))
{
2018-03-16 06:39:32 +00:00
const ASTPtr & arg = args.children.at(1);
if (typeid_cast<ASTSubquery *>(arg.get()) || isIdentifier(arg))
{
if (settings.use_index_for_in_with_subqueries)
tryMakeSetForIndexFromSubquery(arg);
}
else
{
NamesAndTypesList temp_columns = source_columns;
temp_columns.insert(temp_columns.end(), array_join_columns.begin(), array_join_columns.end());
for (const auto & joined_column : columns_added_by_join)
temp_columns.push_back(joined_column.name_and_type);
ExpressionActionsPtr temp_actions = std::make_shared<ExpressionActions>(temp_columns, context);
getRootActions(func->arguments->children.at(0), true, temp_actions);
Block sample_block_with_calculated_columns = temp_actions->getSampleBlock();
if (sample_block_with_calculated_columns.has(args.children.at(0)->getColumnName()))
makeExplicitSet(func, sample_block_with_calculated_columns, true, context,
settings.size_limits_for_set, prepared_sets);
}
}
}
}
void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts)
{
LogAST log;
ActionsVisitor actions_visitor(context, settings.size_limits_for_set, subquery_depth,
source_columns, actions, prepared_sets, subqueries_for_sets,
2018-10-17 11:06:46 +00:00
no_subqueries, only_consts, !isRemoteStorage(), log.stream());
actions_visitor.visit(ast);
actions = actions_visitor.popActionsLevel();
}
void ExpressionAnalyzer::getActionsFromJoinKeys(const ASTTableJoin & table_join, bool no_subqueries, ExpressionActionsPtr & actions)
{
bool only_consts = false;
LogAST log;
ActionsVisitor actions_visitor(context, settings.size_limits_for_set, subquery_depth,
source_columns, actions, prepared_sets, subqueries_for_sets,
2018-10-17 11:06:46 +00:00
no_subqueries, only_consts, !isRemoteStorage(), log.stream());
if (table_join.using_expression_list)
actions_visitor.visit(table_join.using_expression_list);
else if (table_join.on_expression)
{
2018-11-09 17:23:48 +00:00
for (const auto & ast : analyzedJoin().key_asts_left)
actions_visitor.visit(ast);
}
actions = actions_visitor.popActionsLevel();
}
2016-03-05 02:30:20 +00:00
void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr & actions)
{
2017-04-02 17:37:49 +00:00
/// There can not be aggregate functions inside the WHERE and PREWHERE.
if (select_query && (ast.get() == select_query->where_expression.get() || ast.get() == select_query->prewhere_expression.get()))
{
assertNoAggregates(ast, "in WHERE or PREWHERE");
return;
}
2017-04-02 17:37:49 +00:00
/// If we are not analyzing a SELECT query, but a separate expression, then there can not be aggregate functions in it.
if (!select_query)
{
assertNoAggregates(ast, "in wrong place");
return;
}
const ASTFunction * node = typeid_cast<const ASTFunction *>(ast.get());
if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
{
has_aggregation = true;
AggregateDescription aggregate;
aggregate.column_name = node->getColumnName();
2017-04-02 17:37:49 +00:00
/// Make unique aggregate functions.
for (size_t i = 0; i < aggregate_descriptions.size(); ++i)
if (aggregate_descriptions[i].column_name == aggregate.column_name)
return;
const ASTs & arguments = node->arguments->children;
aggregate.argument_names.resize(arguments.size());
DataTypes types(arguments.size());
for (size_t i = 0; i < arguments.size(); ++i)
{
2017-04-02 17:37:49 +00:00
/// There can not be other aggregate functions within the aggregate functions.
assertNoAggregates(arguments[i], "inside another aggregate function");
getRootActions(arguments[i], true, actions);
const std::string & name = arguments[i]->getColumnName();
types[i] = actions->getSampleBlock().getByName(name).type;
aggregate.argument_names[i] = name;
}
aggregate.parameters = (node->parameters) ? getAggregateFunctionParametersArray(node->parameters) : Array();
aggregate.function = AggregateFunctionFactory::instance().get(node->name, types, aggregate.parameters);
aggregate_descriptions.push_back(aggregate);
}
else
{
for (const auto & child : ast->children)
if (!typeid_cast<const ASTSubquery *>(child.get())
&& !typeid_cast<const ASTSelectQuery *>(child.get()))
getAggregates(child, actions);
}
}
2016-03-05 02:30:20 +00:00
void ExpressionAnalyzer::assertNoAggregates(const ASTPtr & ast, const char * description)
{
const ASTFunction * node = typeid_cast<const ASTFunction *>(ast.get());
2016-03-05 02:30:20 +00:00
if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
throw Exception("Aggregate function " + node->getColumnName()
+ " is found " + String(description) + " in query", ErrorCodes::ILLEGAL_AGGREGATION);
2016-03-05 02:30:20 +00:00
for (const auto & child : ast->children)
if (!typeid_cast<const ASTSubquery *>(child.get())
&& !typeid_cast<const ASTSelectQuery *>(child.get()))
assertNoAggregates(child, description);
2016-03-05 02:30:20 +00:00
}
2015-04-18 22:30:43 +00:00
void ExpressionAnalyzer::assertSelect() const
{
if (!select_query)
throw Exception("Not a select query", ErrorCodes::LOGICAL_ERROR);
}
2013-06-14 16:38:54 +00:00
2015-04-18 22:30:43 +00:00
void ExpressionAnalyzer::assertAggregation() const
{
if (!has_aggregation)
throw Exception("No aggregation", ErrorCodes::LOGICAL_ERROR);
}
2013-06-14 16:38:54 +00:00
void ExpressionAnalyzer::initChain(ExpressionActionsChain & chain, const NamesAndTypesList & columns) const
{
if (chain.steps.empty())
{
chain.steps.emplace_back(std::make_shared<ExpressionActions>(columns, context));
}
}
2016-07-23 02:25:09 +00:00
/// "Big" ARRAY JOIN.
void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActionsPtr & actions, bool array_join_is_left) const
{
NameSet result_columns;
for (const auto & result_source : syntax->array_join_result_to_source)
{
/// Assign new names to columns, if needed.
if (result_source.first != result_source.second)
actions->add(ExpressionAction::copyColumn(result_source.second, result_source.first));
2017-04-02 17:37:49 +00:00
/// Make ARRAY JOIN (replace arrays with their insides) for the columns in these new names.
result_columns.insert(result_source.first);
}
actions->add(ExpressionAction::arrayJoin(result_columns, array_join_is_left, context));
}
bool ExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, bool only_types)
{
assertSelect();
bool is_array_join_left;
ASTPtr array_join_expression_list = select_query->array_join_expression_list(is_array_join_left);
if (!array_join_expression_list)
return false;
2018-02-26 09:05:06 +00:00
initChain(chain, source_columns);
ExpressionActionsChain::Step & step = chain.steps.back();
getRootActions(array_join_expression_list, only_types, step.actions);
addMultipleArrayJoinAction(step.actions, is_array_join_left);
return true;
}
2015-04-18 22:30:43 +00:00
void ExpressionAnalyzer::addJoinAction(ExpressionActionsPtr & actions, bool only_types) const
{
2018-11-07 12:55:19 +00:00
NamesAndTypesList columns_added_by_join_list;
for (const auto & joined_column : columns_added_by_join)
columns_added_by_join_list.push_back(joined_column.name_and_type);
if (only_types)
actions->add(ExpressionAction::ordinaryJoin(nullptr, analyzedJoin().key_names_left, columns_added_by_join_list));
else
for (auto & subquery_for_set : subqueries_for_sets)
if (subquery_for_set.second.join)
2018-11-09 17:23:48 +00:00
actions->add(ExpressionAction::ordinaryJoin(subquery_for_set.second.join, analyzedJoin().key_names_left,
columns_added_by_join_list));
}
static void appendRequiredColumns(NameSet & required_columns, const Block & sample, const AnalyzedJoin & analyzed_join)
{
for (auto & column : analyzed_join.key_names_right)
if (!sample.has(column))
required_columns.insert(column);
for (auto & column : analyzed_join.columns_from_joined_table)
if (!sample.has(column.name_and_type.name))
required_columns.insert(column.name_and_type.name);
}
bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_types)
{
assertSelect();
if (!select_query->join())
return false;
2018-02-26 09:05:06 +00:00
initChain(chain, source_columns);
ExpressionActionsChain::Step & step = chain.steps.back();
const auto & join_element = static_cast<const ASTTablesInSelectQueryElement &>(*select_query->join());
auto & join_params = static_cast<ASTTableJoin &>(*join_element.table_join);
if (join_params.strictness == ASTTableJoin::Strictness::Unspecified && join_params.kind != ASTTableJoin::Kind::Cross)
{
if (settings.join_default_strictness == "ANY")
join_params.strictness = ASTTableJoin::Strictness::Any;
else if (settings.join_default_strictness == "ALL")
join_params.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);
}
const auto & table_to_join = static_cast<const ASTTableExpression &>(*join_element.table_expression);
getActionsFromJoinKeys(join_params, only_types, step.actions);
2017-04-02 17:37:49 +00:00
/// Two JOINs are not supported with the same subquery, but different USINGs.
2018-02-23 08:05:21 +00:00
auto join_hash = join_element.getTreeHash();
2018-02-23 08:05:21 +00:00
SubqueryForSet & subquery_for_set = subqueries_for_sets[toString(join_hash.first) + "_" + toString(join_hash.second)];
2017-04-02 17:37:49 +00:00
/// Special case - if table name is specified on the right of JOIN, then the table has the type Join (the previously prepared mapping).
/// TODO This syntax does not support specifying a database name.
if (table_to_join.database_and_table_name)
{
DatabaseAndTableWithAlias database_table(table_to_join.database_and_table_name);
StoragePtr table = context.tryGetTable(database_table.database, database_table.table);
if (table)
{
2017-11-04 16:46:14 +00:00
StorageJoin * storage_join = dynamic_cast<StorageJoin *>(table.get());
if (storage_join)
{
storage_join->assertCompatible(join_params.kind, join_params.strictness);
2017-04-02 17:37:49 +00:00
/// TODO Check the set of keys.
JoinPtr & join = storage_join->getJoin();
subquery_for_set.join = join;
}
}
}
if (!subquery_for_set.join)
{
auto & analyzed_join = analyzedJoin();
/// Actions which need to be calculated on joined block.
ExpressionActionsPtr joined_block_actions =
analyzed_join.createJoinedBlockActions(columns_added_by_join, select_query, context);
2017-04-02 17:37:49 +00:00
/** For GLOBAL JOINs (in the case, for example, of the push method for executing GLOBAL subqueries), the following occurs
* - in the addExternalStorage function, the JOIN (SELECT ...) subquery is replaced with JOIN _data1,
* in the subquery_for_set object this subquery is exposed as source and the temporary table _data1 as the `table`.
* - this function shows the expression JOIN _data1.
*/
if (!subquery_for_set.source)
{
ASTPtr table;
if (table_to_join.subquery)
table = table_to_join.subquery;
else if (table_to_join.table_function)
table = table_to_join.table_function;
else if (table_to_join.database_and_table_name)
table = table_to_join.database_and_table_name;
Names action_columns = joined_block_actions->getRequiredColumns();
NameSet required_columns(action_columns.begin(), action_columns.end());
appendRequiredColumns(required_columns, joined_block_actions->getSampleBlock(), analyzed_join);
Names original_columns = analyzed_join.getOriginalColumnNames(required_columns);
auto interpreter = interpretSubquery(table, context, subquery_depth, original_columns);
subquery_for_set.makeSource(interpreter, analyzed_join.columns_from_joined_table, required_columns);
2018-07-31 11:31:18 +00:00
}
Block sample_block = subquery_for_set.renamedSampleBlock();
2018-11-08 09:00:25 +00:00
joined_block_actions->execute(sample_block);
2018-10-02 18:14:37 +00:00
2017-04-02 17:37:49 +00:00
/// TODO You do not need to set this up when JOIN is only needed on remote servers.
subquery_for_set.join = std::make_shared<Join>(analyzedJoin().key_names_right, settings.join_use_nulls,
settings.size_limits_for_join, join_params.kind, join_params.strictness);
subquery_for_set.join->setSampleBlock(sample_block);
2018-11-08 09:00:25 +00:00
subquery_for_set.joined_block_actions = joined_block_actions;
}
addJoinAction(step.actions, false);
return true;
}
bool ExpressionAnalyzer::appendPrewhere(
ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns)
{
assertSelect();
if (!select_query->prewhere_expression)
return false;
initChain(chain, source_columns);
2018-04-12 09:45:24 +00:00
auto & step = chain.getLastStep();
getRootActions(select_query->prewhere_expression, only_types, step.actions);
2018-04-12 09:45:24 +00:00
String prewhere_column_name = select_query->prewhere_expression->getColumnName();
step.required_output.push_back(prewhere_column_name);
2018-06-29 11:42:44 +00:00
step.can_remove_required_output.push_back(true);
2018-04-12 09:45:24 +00:00
{
/// Remove unused source_columns from prewhere actions.
auto tmp_actions = std::make_shared<ExpressionActions>(source_columns, context);
getRootActions(select_query->prewhere_expression, only_types, tmp_actions);
2018-04-12 09:45:24 +00:00
tmp_actions->finalize({prewhere_column_name});
auto required_columns = tmp_actions->getRequiredColumns();
NameSet required_source_columns(required_columns.begin(), required_columns.end());
2018-10-04 08:58:19 +00:00
/// Add required columns to required output in order not to remove them after prewhere execution.
/// TODO: add sampling and final execution to common chain.
for (const auto & column : additional_required_columns)
{
if (required_source_columns.count(column))
{
step.required_output.push_back(column);
step.can_remove_required_output.push_back(true);
}
}
2018-04-12 09:45:24 +00:00
auto names = step.actions->getSampleBlock().getNames();
NameSet name_set(names.begin(), names.end());
for (const auto & column : source_columns)
if (required_source_columns.count(column.name) == 0)
name_set.erase(column.name);
Names required_output(name_set.begin(), name_set.end());
step.actions->finalize(required_output);
}
{
/// Add empty action with input = {prewhere actions output} + {unused source columns}
/// Reasons:
/// 1. Remove remove source columns which are used only in prewhere actions during prewhere actions execution.
/// Example: select A prewhere B > 0. B can be removed at prewhere step.
/// 2. Store side columns which were calculated during prewhere actions execution if they are used.
/// Example: select F(A) prewhere F(A) > 0. F(A) can be saved from prewhere step.
2018-06-25 13:08:35 +00:00
/// 3. Check if we can remove filter column at prewhere step. If we can, action will store single REMOVE_COLUMN.
2018-04-12 09:45:24 +00:00
ColumnsWithTypeAndName columns = step.actions->getSampleBlock().getColumnsWithTypeAndName();
auto required_columns = step.actions->getRequiredColumns();
NameSet prewhere_input_names(required_columns.begin(), required_columns.end());
NameSet unused_source_columns;
for (const auto & column : source_columns)
{
if (prewhere_input_names.count(column.name) == 0)
{
columns.emplace_back(column.type, column.name);
unused_source_columns.emplace(column.name);
}
}
chain.steps.emplace_back(std::make_shared<ExpressionActions>(std::move(columns), context));
2018-04-12 09:45:24 +00:00
chain.steps.back().additional_input = std::move(unused_source_columns);
}
return true;
}
bool ExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, bool only_types)
{
assertSelect();
if (!select_query->where_expression)
return false;
2018-02-26 09:05:06 +00:00
initChain(chain, source_columns);
ExpressionActionsChain::Step & step = chain.steps.back();
step.required_output.push_back(select_query->where_expression->getColumnName());
2018-06-29 11:42:44 +00:00
step.can_remove_required_output = {true};
2018-04-23 19:05:46 +00:00
getRootActions(select_query->where_expression, only_types, step.actions);
return true;
}
bool ExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain, bool only_types)
{
assertAggregation();
if (!select_query->group_expression_list)
return false;
2018-02-26 09:05:06 +00:00
initChain(chain, source_columns);
ExpressionActionsChain::Step & step = chain.steps.back();
ASTs asts = select_query->group_expression_list->children;
for (size_t i = 0; i < asts.size(); ++i)
{
step.required_output.push_back(asts[i]->getColumnName());
getRootActions(asts[i], only_types, step.actions);
}
return true;
}
void ExpressionAnalyzer::appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types)
{
assertAggregation();
2018-02-26 09:05:06 +00:00
initChain(chain, source_columns);
ExpressionActionsChain::Step & step = chain.steps.back();
for (size_t i = 0; i < aggregate_descriptions.size(); ++i)
{
for (size_t j = 0; j < aggregate_descriptions[i].argument_names.size(); ++j)
{
step.required_output.push_back(aggregate_descriptions[i].argument_names[j]);
}
}
getActionsBeforeAggregation(select_query->select_expression_list, step.actions, only_types);
if (select_query->having_expression)
getActionsBeforeAggregation(select_query->having_expression, step.actions, only_types);
if (select_query->order_expression_list)
getActionsBeforeAggregation(select_query->order_expression_list, step.actions, only_types);
}
bool ExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_types)
{
assertAggregation();
if (!select_query->having_expression)
return false;
initChain(chain, aggregated_columns);
ExpressionActionsChain::Step & step = chain.steps.back();
step.required_output.push_back(select_query->having_expression->getColumnName());
getRootActions(select_query->having_expression, only_types, step.actions);
return true;
}
void ExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, bool only_types)
{
assertSelect();
initChain(chain, aggregated_columns);
ExpressionActionsChain::Step & step = chain.steps.back();
getRootActions(select_query->select_expression_list, only_types, step.actions);
2018-02-26 09:55:14 +00:00
for (const auto & child : select_query->select_expression_list->children)
2018-03-01 01:25:06 +00:00
step.required_output.push_back(child->getColumnName());
}
bool ExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only_types)
{
assertSelect();
if (!select_query->order_expression_list)
return false;
initChain(chain, aggregated_columns);
ExpressionActionsChain::Step & step = chain.steps.back();
getRootActions(select_query->order_expression_list, only_types, step.actions);
ASTs asts = select_query->order_expression_list->children;
for (size_t i = 0; i < asts.size(); ++i)
{
ASTOrderByElement * ast = typeid_cast<ASTOrderByElement *>(asts[i].get());
if (!ast || ast->children.size() < 1)
throw Exception("Bad order expression AST", ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE);
ASTPtr order_expression = ast->children.at(0);
step.required_output.push_back(order_expression->getColumnName());
}
return true;
}
bool ExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain, bool only_types)
{
assertSelect();
if (!select_query->limit_by_expression_list)
return false;
initChain(chain, aggregated_columns);
ExpressionActionsChain::Step & step = chain.steps.back();
getRootActions(select_query->limit_by_expression_list, only_types, step.actions);
for (const auto & child : select_query->limit_by_expression_list->children)
step.required_output.push_back(child->getColumnName());
return true;
}
2017-12-01 21:13:25 +00:00
void ExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) const
{
assertSelect();
initChain(chain, aggregated_columns);
ExpressionActionsChain::Step & step = chain.steps.back();
NamesWithAliases result_columns;
ASTs asts = select_query->select_expression_list->children;
for (size_t i = 0; i < asts.size(); ++i)
{
2018-02-26 09:05:06 +00:00
String result_name = asts[i]->getAliasOrColumnName();
if (required_result_columns.empty() || required_result_columns.count(result_name))
2018-02-26 09:05:06 +00:00
{
result_columns.emplace_back(asts[i]->getColumnName(), result_name);
step.required_output.push_back(result_columns.back().second);
}
}
step.actions->add(ExpressionAction::project(result_columns));
}
void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const ASTPtr & expr, bool only_types)
{
initChain(chain, source_columns);
ExpressionActionsChain::Step & step = chain.steps.back();
getRootActions(expr, only_types, step.actions);
step.required_output.push_back(expr->getColumnName());
}
void ExpressionAnalyzer::getActionsBeforeAggregation(const ASTPtr & ast, ExpressionActionsPtr & actions, bool no_subqueries)
{
ASTFunction * node = typeid_cast<ASTFunction *>(ast.get());
if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
for (auto & argument : node->arguments->children)
getRootActions(argument, no_subqueries, actions);
else
for (auto & child : ast->children)
getActionsBeforeAggregation(child, actions, no_subqueries);
}
ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool project_result)
{
ExpressionActionsPtr actions = std::make_shared<ExpressionActions>(source_columns, context);
NamesWithAliases result_columns;
Names result_names;
ASTs asts;
2018-08-27 17:58:43 +00:00
if (auto node = typeid_cast<const ASTExpressionList *>(query.get()))
asts = node->children;
else
2018-08-27 17:58:43 +00:00
asts = ASTs(1, query);
for (size_t i = 0; i < asts.size(); ++i)
{
std::string name = asts[i]->getColumnName();
std::string alias;
if (add_aliases)
alias = asts[i]->getAliasOrColumnName();
else
alias = name;
result_columns.emplace_back(name, alias);
result_names.push_back(alias);
getRootActions(asts[i], false, actions);
}
if (add_aliases)
{
if (project_result)
actions->add(ExpressionAction::project(result_columns));
else
actions->add(ExpressionAction::addAliases(result_columns));
}
2018-09-05 13:04:28 +00:00
if (!(add_aliases && project_result))
{
2017-04-02 17:37:49 +00:00
/// We will not delete the original columns.
2018-02-26 09:05:06 +00:00
for (const auto & column_name_type : source_columns)
result_names.push_back(column_name_type.name);
}
actions->finalize(result_names);
return actions;
}
ExpressionActionsPtr ExpressionAnalyzer::getConstActions()
{
ExpressionActionsPtr actions = std::make_shared<ExpressionActions>(NamesAndTypesList(), context);
getRootActions(query, true, actions, true);
return actions;
}
2015-04-18 22:30:43 +00:00
void ExpressionAnalyzer::getAggregateInfo(Names & key_names, AggregateDescriptions & aggregates) const
{
for (const auto & name_and_type : aggregation_keys)
key_names.emplace_back(name_and_type.name);
2015-03-12 02:22:55 +00:00
aggregates = aggregate_descriptions;
}
void ExpressionAnalyzer::collectUsedColumns()
{
2017-04-02 17:37:49 +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.
*/
RequiredSourceColumnsVisitor::Data columns_context;
RequiredSourceColumnsVisitor(columns_context).visit(query);
NameSet required = columns_context.requiredColumns();
#if 0
std::cerr << "Query: " << query << std::endl;
std::cerr << "CTX: " << columns_context << std::endl;
std::cerr << "source_columns: ";
for (const auto & name : source_columns)
std::cerr << "'" << name.name << "' ";
std::cerr << "required: ";
for (const auto & name : required)
std::cerr << "'" << name << "' ";
std::cerr << std::endl;
#endif
if (columns_context.has_table_join)
{
const AnalyzedJoin & analyzed_join = analyzedJoin();
#if 0
std::cerr << "key_names_left: ";
for (const auto & name : analyzed_join.key_names_left)
std::cerr << "'" << name << "' ";
std::cerr << "key_names_right: ";
for (const auto & name : analyzed_join.key_names_right)
std::cerr << "'" << name << "' ";
std::cerr << "columns_from_joined_table: ";
for (const auto & column : analyzed_join.columns_from_joined_table)
std::cerr << "'" << column.name_and_type.name << '/' << column.original_name << "' ";
std::cerr << "available_joined_columns: ";
for (const auto & column : analyzed_join.available_joined_columns)
std::cerr << "'" << column.name_and_type.name << '/' << column.original_name << "' ";
std::cerr << std::endl;
#endif
NameSet avaliable_columns;
for (const auto & name : source_columns)
avaliable_columns.insert(name.name);
/** You also need to ignore the identifiers of the columns that are obtained by JOIN.
* (Do not assume that they are required for reading from the "left" table).
*/
columns_added_by_join.clear();
for (const auto & joined_column : analyzed_join.available_joined_columns)
{
auto & name = joined_column.name_and_type.name;
if (required.count(name) && !avaliable_columns.count(name))
{
columns_added_by_join.push_back(joined_column);
required.erase(name);
}
}
}
2018-07-27 20:01:54 +00:00
if (columns_context.has_array_join)
{
/// Insert the columns required for the ARRAY JOIN calculation into the required columns list.
NameSet array_join_sources;
for (const auto & result_source : syntax->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);
}
2017-04-02 17:37:49 +00:00
/// You need to read at least one column to find the number of rows.
2018-02-22 10:54:28 +00:00
if (select_query && required.empty())
2018-02-26 09:05:06 +00:00
required.insert(ExpressionActions::getSmallestColumn(source_columns));
2018-02-28 01:29:55 +00:00
NameSet unknown_required_source_columns = required;
2018-02-26 09:05:06 +00:00
for (NamesAndTypesList::iterator it = source_columns.begin(); it != source_columns.end();)
{
2019-02-12 15:08:21 +00:00
const String & column_name = it->name;
unknown_required_source_columns.erase(column_name);
2019-02-12 15:08:21 +00:00
if (!required.count(column_name))
2018-02-26 09:05:06 +00:00
source_columns.erase(it++);
else
++it;
}
2018-02-26 21:00:42 +00:00
/// 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)
{
2018-02-26 09:05:06 +00:00
for (auto it = unknown_required_source_columns.begin(); it != unknown_required_source_columns.end();)
{
if (storage->hasColumn(*it))
{
2018-02-26 09:05:06 +00:00
source_columns.push_back(storage->getColumn(*it));
unknown_required_source_columns.erase(it++);
}
else
++it;
}
}
2018-02-28 01:29:55 +00:00
if (!unknown_required_source_columns.empty())
{
std::stringstream ss;
2019-02-08 17:21:28 +00:00
ss << "query: '" << query << "' ";
ss << columns_context;
ss << "source_columns: ";
for (const auto & name : source_columns)
ss << "'" << name.name << "' ";
throw Exception("Unknown identifier: " + *unknown_required_source_columns.begin()
2019-02-10 18:14:23 +00:00
+ (select_query && !select_query->tables ? ". Note that there are no tables (FROM clause) in your query" : "")
+ ", context: " + ss.str(), ErrorCodes::UNKNOWN_IDENTIFIER);
}
}
}