2018-10-18 15:03:14 +00:00
|
|
|
#include <Core/Block.h>
|
|
|
|
|
2021-01-13 19:29:52 +00:00
|
|
|
#include <Parsers/ASTExpressionList.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Parsers/ASTFunction.h>
|
|
|
|
#include <Parsers/ASTIdentifier.h>
|
|
|
|
#include <Parsers/ASTLiteral.h>
|
2021-01-13 19:29:52 +00:00
|
|
|
#include <Parsers/ASTOrderByElement.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Parsers/ASTSelectQuery.h>
|
|
|
|
#include <Parsers/ASTSubquery.h>
|
2021-01-13 19:29:52 +00:00
|
|
|
#include <Parsers/ASTWindowDefinition.h>
|
2018-10-09 14:32:11 +00:00
|
|
|
#include <Parsers/DumpASTNode.h>
|
2013-05-24 10:49:19 +00:00
|
|
|
|
2017-12-25 00:38:45 +00:00
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
2020-12-10 23:56:57 +00:00
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
2018-10-12 15:41:28 +00:00
|
|
|
#include <Columns/IColumn.h>
|
2013-05-24 10:49:19 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/ExpressionAnalyzer.h>
|
|
|
|
#include <Interpreters/ExpressionActions.h>
|
2020-08-13 20:17:18 +00:00
|
|
|
#include <Interpreters/ArrayJoinAction.h>
|
2019-09-26 10:41:33 +00:00
|
|
|
#include <Interpreters/ExternalDictionariesLoader.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/Set.h>
|
2020-04-07 09:48:47 +00:00
|
|
|
#include <Interpreters/TableJoin.h>
|
2020-02-11 18:27:52 +00:00
|
|
|
#include <Interpreters/JoinSwitcher.h>
|
2020-04-07 09:48:47 +00:00
|
|
|
#include <Interpreters/HashJoin.h>
|
2020-02-17 17:21:03 +00:00
|
|
|
#include <Interpreters/MergeJoin.h>
|
2020-04-08 18:59:52 +00:00
|
|
|
#include <Interpreters/DictionaryReader.h>
|
2020-05-20 20:16:32 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2013-05-24 10:49:19 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
2017-07-10 23:30:17 +00:00
|
|
|
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
|
2015-04-16 06:12:35 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/StorageDistributed.h>
|
2020-04-08 18:59:52 +00:00
|
|
|
#include <Storages/StorageDictionary.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/StorageJoin.h>
|
2014-03-14 14:52:48 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <DataStreams/copyData.h>
|
2014-03-14 14:52:48 +00:00
|
|
|
|
2020-04-08 19:58:27 +00:00
|
|
|
#include <Dictionaries/DictionaryStructure.h>
|
2015-05-13 16:11:07 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2018-01-15 19:07:47 +00:00
|
|
|
#include <Common/StringUtils/StringUtils.h>
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <DataTypes/DataTypeFactory.h>
|
2018-07-26 12:58:23 +00:00
|
|
|
#include <Parsers/parseQuery.h>
|
2018-10-16 19:00:05 +00:00
|
|
|
|
2018-10-12 15:41:28 +00:00
|
|
|
#include <Interpreters/ActionsVisitor.h>
|
2019-08-12 19:27:09 +00:00
|
|
|
#include <Interpreters/GetAggregatesVisitor.h>
|
2021-01-31 19:25:47 +00:00
|
|
|
#include <Interpreters/GlobalSubqueriesVisitor.h>
|
|
|
|
#include <Interpreters/interpretSubquery.h>
|
|
|
|
#include <Interpreters/join_common.h>
|
|
|
|
#include <Interpreters/misc.h>
|
2013-05-24 10:49:19 +00:00
|
|
|
|
2020-12-18 17:13:28 +00:00
|
|
|
#include <IO/Operators.h>
|
|
|
|
#include <IO/WriteBufferFromString.h>
|
|
|
|
|
2021-01-13 18:37:54 +00:00
|
|
|
#include <Processors/Executors/PullingPipelineExecutor.h>
|
2021-01-20 12:12:18 +00:00
|
|
|
#include <Parsers/formatAST.h>
|
2021-01-13 18:37:54 +00:00
|
|
|
|
2013-05-24 10:49:19 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2018-10-10 16:23:27 +00:00
|
|
|
using LogAST = DebugASTLog<false>; /// set to true to enable logs
|
2018-10-09 14:32:11 +00:00
|
|
|
|
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2020-12-09 11:14:40 +00:00
|
|
|
extern const int BAD_ARGUMENTS;
|
2020-02-10 15:50:12 +00:00
|
|
|
extern const int ILLEGAL_PREWHERE;
|
2020-06-20 13:48:21 +00:00
|
|
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
2020-07-02 07:44:47 +00:00
|
|
|
extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER;
|
2020-12-09 11:14:40 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2020-12-21 09:59:11 +00:00
|
|
|
extern const int NOT_IMPLEMENTED;
|
2020-12-09 11:14:40 +00:00
|
|
|
extern const int UNKNOWN_IDENTIFIER;
|
|
|
|
extern const int UNKNOWN_TYPE_OF_AST_NODE;
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
2020-02-10 15:50:12 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
|
|
|
/// Check if there is an ignore function. It's used for disabling constant folding in query
|
|
|
|
/// predicates because some performance tests use ignore function as a non-optimize guard.
|
2020-11-03 11:28:28 +00:00
|
|
|
bool allowEarlyConstantFolding(const ActionsDAG & actions, const Settings & settings)
|
2020-02-10 15:50:12 +00:00
|
|
|
{
|
2020-02-10 19:55:13 +00:00
|
|
|
if (!settings.enable_early_constant_folding)
|
2020-02-10 15:50:12 +00:00
|
|
|
return false;
|
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
for (const auto & node : actions.getNodes())
|
2020-02-10 15:50:12 +00:00
|
|
|
{
|
2020-11-10 14:54:59 +00:00
|
|
|
if (node.type == ActionsDAG::ActionType::FUNCTION && node.function_base)
|
2020-02-10 15:50:12 +00:00
|
|
|
{
|
2021-02-10 21:25:50 +00:00
|
|
|
if (!node.function_base->isSuitableForConstantFolding())
|
|
|
|
return false;
|
2020-02-10 15:50:12 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2021-02-09 13:17:42 +00:00
|
|
|
|
|
|
|
/// Returns converting actions for tables that need to be performed before join
|
|
|
|
ActionsDAGPtr createJoinConvertingActions(const ColumnsWithTypeAndName & cols_src, const TableJoin::NameToTypeMap & mapping)
|
|
|
|
{
|
|
|
|
ColumnsWithTypeAndName cols_dst = cols_src;
|
|
|
|
for (auto & col : cols_dst)
|
|
|
|
{
|
|
|
|
if (auto it = mapping.find(col.name); it != mapping.end())
|
|
|
|
{
|
|
|
|
col.type = it->second;
|
|
|
|
col.column = nullptr;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return ActionsDAG::makeConvertingActions(cols_src, cols_dst, ActionsDAG::MatchColumnsMode::Name, true);
|
|
|
|
};
|
|
|
|
|
|
|
|
|
2020-02-10 15:50:12 +00:00
|
|
|
}
|
|
|
|
|
2020-06-20 13:48:21 +00:00
|
|
|
bool sanitizeBlock(Block & block, bool throw_if_cannot_create_column)
|
2020-02-10 15:50:12 +00:00
|
|
|
{
|
|
|
|
for (auto & col : block)
|
|
|
|
{
|
|
|
|
if (!col.column)
|
|
|
|
{
|
|
|
|
if (isNotCreatable(col.type->getTypeId()))
|
2020-06-20 13:48:21 +00:00
|
|
|
{
|
|
|
|
if (throw_if_cannot_create_column)
|
|
|
|
throw Exception("Cannot create column of type " + col.type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
2020-02-10 15:50:12 +00:00
|
|
|
return false;
|
2020-06-20 13:48:21 +00:00
|
|
|
}
|
|
|
|
|
2020-02-10 15:50:12 +00:00
|
|
|
col.column = col.type->createColumn();
|
|
|
|
}
|
2020-06-10 15:06:37 +00:00
|
|
|
else if (!col.column->empty())
|
2020-02-10 15:50:12 +00:00
|
|
|
col.column = col.column->cloneEmpty();
|
|
|
|
}
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2020-12-10 22:05:02 +00:00
|
|
|
ExpressionAnalyzer::ExtractedSettings::ExtractedSettings(const Settings & settings_)
|
|
|
|
: use_index_for_in_with_subqueries(settings_.use_index_for_in_with_subqueries)
|
|
|
|
, size_limits_for_set(settings_.max_rows_in_set, settings_.max_bytes_in_set, settings_.set_overflow_mode)
|
|
|
|
{}
|
|
|
|
|
2020-02-10 15:50:12 +00:00
|
|
|
|
2017-01-14 09:00:19 +00:00
|
|
|
ExpressionAnalyzer::ExpressionAnalyzer(
|
2018-08-27 17:58:43 +00:00
|
|
|
const ASTPtr & query_,
|
2020-07-22 17:13:05 +00:00
|
|
|
const TreeRewriterResultPtr & syntax_analyzer_result_,
|
2017-04-01 07:20:54 +00:00
|
|
|
const Context & context_,
|
|
|
|
size_t subquery_depth_,
|
2020-10-13 14:55:22 +00:00
|
|
|
bool do_global,
|
|
|
|
SubqueriesForSets subqueries_for_sets_)
|
2019-08-14 19:30:30 +00:00
|
|
|
: query(query_), context(context_), settings(context.getSettings())
|
2019-08-13 13:48:09 +00:00
|
|
|
, subquery_depth(subquery_depth_)
|
2018-11-09 17:23:48 +00:00
|
|
|
, syntax(syntax_analyzer_result_)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-10-13 14:55:22 +00:00
|
|
|
subqueries_for_sets = std::move(subqueries_for_sets_);
|
|
|
|
|
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.
|
2019-08-13 13:48:09 +00:00
|
|
|
initGlobalSubqueriesAndExternalTables(do_global);
|
2015-10-15 11:07:39 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// 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.
|
2017-04-01 07:20:54 +00:00
|
|
|
analyzeAggregation();
|
2014-07-04 19:13:38 +00:00
|
|
|
}
|
|
|
|
|
2018-10-12 15:41:28 +00:00
|
|
|
|
2014-07-04 19:13:38 +00:00
|
|
|
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-01 07:20:54 +00:00
|
|
|
*
|
2017-04-02 17:37:49 +00:00
|
|
|
* Everything below (compiling temporary ExpressionActions) - only for the purpose of query analysis (type output).
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
|
2019-03-11 13:22:51 +00:00
|
|
|
auto * select_query = query->as<ASTSelectQuery>();
|
2019-03-11 12:49:39 +00:00
|
|
|
|
2020-09-10 16:01:41 +00:00
|
|
|
auto temp_actions = std::make_shared<ActionsDAG>(sourceColumns());
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (select_query)
|
|
|
|
{
|
2020-08-11 12:03:18 +00:00
|
|
|
NamesAndTypesList array_join_columns;
|
2020-09-13 13:51:31 +00:00
|
|
|
columns_after_array_join = sourceColumns();
|
2020-08-11 12:03:18 +00:00
|
|
|
|
2018-12-19 13:13:51 +00:00
|
|
|
bool is_array_join_left;
|
2020-08-11 12:03:18 +00:00
|
|
|
if (ASTPtr array_join_expression_list = select_query->arrayJoinExpressionList(is_array_join_left))
|
2018-12-19 13:13:51 +00:00
|
|
|
{
|
2020-01-27 21:57:44 +00:00
|
|
|
getRootActionsNoMakeSet(array_join_expression_list, true, temp_actions, false);
|
2020-09-13 13:51:31 +00:00
|
|
|
|
|
|
|
auto array_join = addMultipleArrayJoinAction(temp_actions, is_array_join_left);
|
|
|
|
auto sample_columns = temp_actions->getResultColumns();
|
|
|
|
array_join->prepare(sample_columns);
|
|
|
|
temp_actions = std::make_shared<ActionsDAG>(sample_columns);
|
|
|
|
|
|
|
|
NamesAndTypesList new_columns_after_array_join;
|
|
|
|
NameSet added_columns;
|
2019-07-08 20:06:17 +00:00
|
|
|
|
2020-09-10 18:36:51 +00:00
|
|
|
for (auto & column : temp_actions->getResultColumns())
|
2020-09-13 13:51:31 +00:00
|
|
|
{
|
2019-07-08 20:06:17 +00:00
|
|
|
if (syntax->array_join_result_to_source.count(column.name))
|
2020-09-13 13:51:31 +00:00
|
|
|
{
|
|
|
|
new_columns_after_array_join.emplace_back(column.name, column.type);
|
|
|
|
added_columns.emplace(column.name);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
for (auto & column : columns_after_array_join)
|
|
|
|
if (added_columns.count(column.name) == 0)
|
|
|
|
new_columns_after_array_join.emplace_back(column.name, column.type);
|
|
|
|
|
|
|
|
columns_after_array_join.swap(new_columns_after_array_join);
|
2018-12-19 13:13:51 +00:00
|
|
|
}
|
|
|
|
|
2020-08-11 12:03:18 +00:00
|
|
|
columns_after_array_join.insert(columns_after_array_join.end(), array_join_columns.begin(), array_join_columns.end());
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
const ASTTablesInSelectQueryElement * join = select_query->join();
|
|
|
|
if (join)
|
|
|
|
{
|
2020-01-27 21:57:44 +00:00
|
|
|
getRootActionsNoMakeSet(analyzedJoin().leftKeysList(), true, temp_actions, false);
|
2020-09-10 18:36:51 +00:00
|
|
|
auto sample_columns = temp_actions->getResultColumns();
|
2021-02-09 13:17:42 +00:00
|
|
|
analyzedJoin().addJoinedColumnsAndCorrectTypes(sample_columns);
|
2020-09-10 18:36:51 +00:00
|
|
|
temp_actions = std::make_shared<ActionsDAG>(sample_columns);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2020-08-11 12:03:18 +00:00
|
|
|
|
|
|
|
columns_after_join = columns_after_array_join;
|
2021-02-09 13:17:42 +00:00
|
|
|
analyzedJoin().addJoinedColumnsAndCorrectTypes(columns_after_join, false);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2019-08-13 12:39:03 +00:00
|
|
|
has_aggregation = makeAggregateDescriptions(temp_actions);
|
|
|
|
if (select_query && (select_query->groupBy() || select_query->having()))
|
|
|
|
has_aggregation = true;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (has_aggregation)
|
|
|
|
{
|
|
|
|
|
|
|
|
/// Find out aggregation keys.
|
2020-05-15 23:53:47 +00:00
|
|
|
if (select_query)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-05-15 23:53:47 +00:00
|
|
|
if (select_query->groupBy())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-05-15 23:53:47 +00:00
|
|
|
NameSet unique_keys;
|
|
|
|
ASTs & group_asts = select_query->groupBy()->children;
|
|
|
|
for (ssize_t i = 0; i < ssize_t(group_asts.size()); ++i)
|
|
|
|
{
|
|
|
|
ssize_t size = group_asts.size();
|
|
|
|
getRootActionsNoMakeSet(group_asts[i], true, temp_actions, false);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-05-15 23:53:47 +00:00
|
|
|
const auto & column_name = group_asts[i]->getColumnName();
|
2020-09-10 18:36:51 +00:00
|
|
|
const auto & index = temp_actions->getIndex();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-09-10 18:36:51 +00:00
|
|
|
auto it = index.find(column_name);
|
|
|
|
if (it == index.end())
|
2020-05-15 23:53:47 +00:00
|
|
|
throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
const auto & node = *it;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-05-15 23:53:47 +00:00
|
|
|
/// Constant expressions have non-null column pointer at this stage.
|
2020-09-10 18:36:51 +00:00
|
|
|
if (node->column && isColumnConst(*node->column))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-05-15 23:53:47 +00:00
|
|
|
/// 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());
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-05-15 23:53:47 +00:00
|
|
|
group_asts.pop_back();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-05-15 23:53:47 +00:00
|
|
|
--i;
|
|
|
|
continue;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2020-09-10 18:36:51 +00:00
|
|
|
NameAndTypePair key{column_name, node->result_type};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-05-15 23:53:47 +00:00
|
|
|
/// Aggregation keys are uniqued.
|
|
|
|
if (!unique_keys.count(key.name))
|
|
|
|
{
|
|
|
|
unique_keys.insert(key.name);
|
|
|
|
aggregation_keys.push_back(key);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-05-15 23:53:47 +00:00
|
|
|
/// Key is no longer needed, therefore we can save a little by moving it.
|
|
|
|
aggregated_columns.push_back(std::move(key));
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2020-05-15 23:53:47 +00:00
|
|
|
if (group_asts.empty())
|
|
|
|
{
|
|
|
|
select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, {});
|
|
|
|
has_aggregation = select_query->having() || !aggregate_descriptions.empty();
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
2020-05-15 23:53:47 +00:00
|
|
|
else
|
2020-09-10 18:36:51 +00:00
|
|
|
aggregated_columns = temp_actions->getNamesAndTypesList();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-03-08 23:48:08 +00:00
|
|
|
for (const auto & desc : aggregate_descriptions)
|
2018-04-04 18:56:30 +00:00
|
|
|
aggregated_columns.emplace_back(desc.column_name, desc.function->getReturnType());
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2018-04-04 18:56:30 +00:00
|
|
|
else
|
|
|
|
{
|
2020-09-10 18:36:51 +00:00
|
|
|
aggregated_columns = temp_actions->getNamesAndTypesList();
|
2018-04-04 18:56:30 +00:00
|
|
|
}
|
2013-05-24 10:49:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-08-13 13:48:09 +00:00
|
|
|
void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables(bool do_global)
|
2014-07-04 20:30:06 +00:00
|
|
|
{
|
2018-10-17 10:59:05 +00:00
|
|
|
if (do_global)
|
|
|
|
{
|
2018-12-10 13:02:45 +00:00
|
|
|
GlobalSubqueriesVisitor::Data subqueries_data(context, subquery_depth, isRemoteStorage(),
|
2018-10-17 10:59:05 +00:00
|
|
|
external_tables, subqueries_for_sets, has_global_subqueries);
|
2018-12-07 15:36:54 +00:00
|
|
|
GlobalSubqueriesVisitor(subqueries_data).visit(query);
|
2018-10-17 10:59:05 +00:00
|
|
|
}
|
2015-11-04 22:02:52 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-08-14 19:30:30 +00:00
|
|
|
void SelectQueryExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name)
|
2018-01-21 07:30:07 +00:00
|
|
|
{
|
2019-01-22 12:33:56 +00:00
|
|
|
auto set_key = PreparedSetKey::forSubquery(*subquery_or_table_name);
|
2019-11-22 15:45:03 +00:00
|
|
|
|
2019-01-22 12:33:56 +00:00
|
|
|
if (prepared_sets.count(set_key))
|
|
|
|
return; /// Already prepared.
|
2018-01-21 07:30:07 +00:00
|
|
|
|
2019-11-22 15:45:03 +00:00
|
|
|
if (auto set_ptr_from_storage_set = isPlainStorageSetInSubquery(subquery_or_table_name))
|
|
|
|
{
|
|
|
|
prepared_sets.insert({set_key, set_ptr_from_storage_set});
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
2020-02-20 02:56:20 +00:00
|
|
|
auto interpreter_subquery = interpretSubquery(subquery_or_table_name, context, {}, query_options);
|
2021-01-13 18:37:54 +00:00
|
|
|
auto io = interpreter_subquery->execute();
|
|
|
|
PullingPipelineExecutor executor(io.pipeline);
|
2018-01-21 07:30:07 +00:00
|
|
|
|
2020-04-09 08:27:55 +00:00
|
|
|
SetPtr set = std::make_shared<Set>(settings.size_limits_for_set, true, context.getSettingsRef().transform_null_in);
|
2021-01-13 18:37:54 +00:00
|
|
|
set->setHeader(executor.getHeader());
|
2019-01-22 12:33:56 +00:00
|
|
|
|
2021-01-13 18:37:54 +00:00
|
|
|
Block block;
|
|
|
|
while (executor.pull(block))
|
2018-01-21 07:30:07 +00:00
|
|
|
{
|
|
|
|
/// 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))
|
2018-01-21 07:30:07 +00:00
|
|
|
return;
|
|
|
|
}
|
2019-11-01 10:58:29 +00:00
|
|
|
|
|
|
|
set->finishInsert();
|
2018-01-21 07:30:07 +00:00
|
|
|
|
2019-01-22 12:33:56 +00:00
|
|
|
prepared_sets[set_key] = std::move(set);
|
2018-01-21 07:30:07 +00:00
|
|
|
}
|
|
|
|
|
2019-11-22 15:45:03 +00:00
|
|
|
SetPtr SelectQueryExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & subquery_or_table_name)
|
2019-11-13 12:43:55 +00:00
|
|
|
{
|
|
|
|
const auto * table = subquery_or_table_name->as<ASTIdentifier>();
|
|
|
|
if (!table)
|
2019-11-22 15:45:03 +00:00
|
|
|
return nullptr;
|
2020-03-13 10:30:55 +00:00
|
|
|
auto table_id = context.resolveStorageID(subquery_or_table_name);
|
2020-05-28 23:01:18 +00:00
|
|
|
const auto storage = DatabaseCatalog::instance().getTable(table_id, context);
|
2019-11-22 15:45:03 +00:00
|
|
|
if (storage->getName() != "Set")
|
|
|
|
return nullptr;
|
|
|
|
const auto storage_set = std::dynamic_pointer_cast<StorageSet>(storage);
|
|
|
|
return storage_set->getSet();
|
2019-11-13 12:43:55 +00:00
|
|
|
}
|
|
|
|
|
2018-01-21 07:30:07 +00:00
|
|
|
|
2020-01-11 09:50:41 +00:00
|
|
|
/// Performance optimisation for IN() if storage supports it.
|
2019-08-14 19:30:30 +00:00
|
|
|
void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node)
|
2014-03-31 14:49:43 +00:00
|
|
|
{
|
2019-08-12 19:27:09 +00:00
|
|
|
if (!node || !storage() || !storage()->supportsIndexForIn())
|
|
|
|
return;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
for (auto & child : node->children)
|
2018-02-14 17:39:16 +00:00
|
|
|
{
|
2019-01-22 12:33:56 +00:00
|
|
|
/// Don't descend into subqueries.
|
2019-03-11 13:22:51 +00:00
|
|
|
if (child->as<ASTSubquery>())
|
2018-03-16 06:33:23 +00:00
|
|
|
continue;
|
|
|
|
|
2019-01-22 12:33:56 +00:00
|
|
|
/// Don't descend into lambda functions
|
2019-03-11 13:22:51 +00:00
|
|
|
const auto * func = child->as<ASTFunction>();
|
2018-03-16 06:33:23 +00:00
|
|
|
if (func && func->name == "lambda")
|
|
|
|
continue;
|
|
|
|
|
2019-08-12 19:27:09 +00:00
|
|
|
makeSetsForIndex(child);
|
2018-02-14 17:39:16 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-03-11 13:22:51 +00:00
|
|
|
const auto * func = node->as<ASTFunction>();
|
2020-11-21 08:25:45 +00:00
|
|
|
if (func && functionIsInOrGlobalInOperator(func->name))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-07-14 00:33:37 +00:00
|
|
|
const IAST & args = *func->arguments;
|
2019-08-12 19:27:09 +00:00
|
|
|
const ASTPtr & left_in_operand = args.children.at(0);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-06-17 09:38:47 +00:00
|
|
|
if (storage()->mayBenefitFromIndexForIn(left_in_operand, context, metadata_snapshot))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-03-16 06:39:32 +00:00
|
|
|
const ASTPtr & arg = args.children.at(1);
|
2019-03-11 13:22:51 +00:00
|
|
|
if (arg->as<ASTSubquery>() || arg->as<ASTIdentifier>())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-11-22 15:45:03 +00:00
|
|
|
if (settings.use_index_for_in_with_subqueries)
|
2019-01-22 12:33:56 +00:00
|
|
|
tryMakeSetForIndexFromSubquery(arg);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2020-09-10 16:01:41 +00:00
|
|
|
auto temp_actions = std::make_shared<ActionsDAG>(columns_after_join);
|
2019-08-12 19:27:09 +00:00
|
|
|
getRootActions(left_in_operand, true, temp_actions);
|
2019-01-22 12:33:56 +00:00
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
if (temp_actions->getIndex().contains(left_in_operand->getColumnName()))
|
2020-09-30 11:45:47 +00:00
|
|
|
makeExplicitSet(func, *temp_actions, true, context,
|
2019-01-22 12:33:56 +00:00
|
|
|
settings.size_limits_for_set, prepared_sets);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2013-08-01 13:29:32 +00:00
|
|
|
}
|
|
|
|
|
2018-10-16 12:34:20 +00:00
|
|
|
|
2020-09-10 16:01:41 +00:00
|
|
|
void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts)
|
2018-07-24 12:41:35 +00:00
|
|
|
{
|
2018-10-16 12:34:20 +00:00
|
|
|
LogAST log;
|
2019-08-15 13:54:59 +00:00
|
|
|
ActionsVisitor::Data visitor_data(context, settings.size_limits_for_set, subquery_depth,
|
2020-09-10 16:01:41 +00:00
|
|
|
sourceColumns(), std::move(actions), prepared_sets, subqueries_for_sets,
|
2020-01-27 21:57:44 +00:00
|
|
|
no_subqueries, false, only_consts, !isRemoteStorage());
|
|
|
|
ActionsVisitor(visitor_data, log.stream()).visit(ast);
|
2020-09-10 16:01:41 +00:00
|
|
|
actions = visitor_data.getActions();
|
2020-01-27 21:57:44 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-09-10 16:01:41 +00:00
|
|
|
void ExpressionAnalyzer::getRootActionsNoMakeSet(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts)
|
2020-01-27 21:57:44 +00:00
|
|
|
{
|
|
|
|
LogAST log;
|
|
|
|
ActionsVisitor::Data visitor_data(context, settings.size_limits_for_set, subquery_depth,
|
2020-09-10 16:01:41 +00:00
|
|
|
sourceColumns(), std::move(actions), prepared_sets, subqueries_for_sets,
|
2020-01-27 21:57:44 +00:00
|
|
|
no_subqueries, true, only_consts, !isRemoteStorage());
|
2019-08-15 13:54:59 +00:00
|
|
|
ActionsVisitor(visitor_data, log.stream()).visit(ast);
|
2020-09-13 13:51:31 +00:00
|
|
|
actions = visitor_data.getActions();
|
2018-10-16 12:34:20 +00:00
|
|
|
}
|
|
|
|
|
2020-11-02 12:07:01 +00:00
|
|
|
void ExpressionAnalyzer::getRootActionsForHaving(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts)
|
2020-10-23 19:08:38 +00:00
|
|
|
{
|
|
|
|
LogAST log;
|
|
|
|
ActionsVisitor::Data visitor_data(context, settings.size_limits_for_set, subquery_depth,
|
|
|
|
sourceColumns(), std::move(actions), prepared_sets, subqueries_for_sets,
|
2020-11-02 12:07:01 +00:00
|
|
|
no_subqueries, false, only_consts, true);
|
2019-08-15 13:54:59 +00:00
|
|
|
ActionsVisitor(visitor_data, log.stream()).visit(ast);
|
2020-09-13 13:51:31 +00:00
|
|
|
actions = visitor_data.getActions();
|
2018-10-16 12:34:20 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-09-10 18:36:51 +00:00
|
|
|
bool ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions)
|
2018-10-16 12:34:20 +00:00
|
|
|
{
|
2019-08-13 12:39:03 +00:00
|
|
|
for (const ASTFunction * node : aggregates())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
AggregateDescription aggregate;
|
2020-12-04 02:15:44 +00:00
|
|
|
if (node->arguments) getRootActionsNoMakeSet(node->arguments, true, actions);
|
2020-10-21 18:17:27 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
aggregate.column_name = node->getColumnName();
|
|
|
|
|
2020-12-04 02:15:44 +00:00
|
|
|
const ASTs & arguments = node->arguments ? node->arguments->children : ASTs();
|
2017-04-01 07:20:54 +00:00
|
|
|
aggregate.argument_names.resize(arguments.size());
|
|
|
|
DataTypes types(arguments.size());
|
|
|
|
|
2020-10-16 13:03:54 +00:00
|
|
|
const auto & index = actions->getIndex();
|
2017-04-01 07:20:54 +00:00
|
|
|
for (size_t i = 0; i < arguments.size(); ++i)
|
|
|
|
{
|
|
|
|
const std::string & name = arguments[i]->getColumnName();
|
2020-10-16 13:03:54 +00:00
|
|
|
|
|
|
|
auto it = index.find(name);
|
|
|
|
if (it == index.end())
|
2020-12-09 11:14:40 +00:00
|
|
|
{
|
|
|
|
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER,
|
|
|
|
"Unknown identifier '{}' in aggregate function '{}'",
|
|
|
|
name, node->formatForErrorMessage());
|
|
|
|
}
|
2020-10-16 13:03:54 +00:00
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
types[i] = (*it)->result_type;
|
2017-04-01 07:20:54 +00:00
|
|
|
aggregate.argument_names[i] = name;
|
|
|
|
}
|
|
|
|
|
2020-06-14 07:44:02 +00:00
|
|
|
AggregateFunctionProperties properties;
|
2017-07-10 23:30:17 +00:00
|
|
|
aggregate.parameters = (node->parameters) ? getAggregateFunctionParametersArray(node->parameters) : Array();
|
2020-06-14 07:44:02 +00:00
|
|
|
aggregate.function = AggregateFunctionFactory::instance().get(node->name, types, aggregate.parameters, properties);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
aggregate_descriptions.push_back(aggregate);
|
|
|
|
}
|
2016-03-05 02:30:20 +00:00
|
|
|
|
2019-08-13 12:39:03 +00:00
|
|
|
return !aggregates().empty();
|
2016-03-05 02:30:20 +00:00
|
|
|
}
|
|
|
|
|
2021-01-20 05:29:27 +00:00
|
|
|
void makeWindowDescriptionFromAST(WindowDescription & desc, const IAST * ast)
|
2021-01-13 19:29:52 +00:00
|
|
|
{
|
|
|
|
const auto & definition = ast->as<const ASTWindowDefinition &>();
|
|
|
|
|
|
|
|
if (definition.partition_by)
|
|
|
|
{
|
|
|
|
for (const auto & column_ast : definition.partition_by->children)
|
|
|
|
{
|
|
|
|
const auto * with_alias = dynamic_cast<const ASTWithAlias *>(
|
|
|
|
column_ast.get());
|
|
|
|
if (!with_alias)
|
|
|
|
{
|
|
|
|
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
|
|
|
"Expected a column in PARTITION BY in window definition,"
|
|
|
|
" got '{}'",
|
|
|
|
column_ast->formatForErrorMessage());
|
|
|
|
}
|
|
|
|
desc.partition_by.push_back(SortColumnDescription(
|
|
|
|
with_alias->getColumnName(), 1 /* direction */,
|
|
|
|
1 /* nulls_direction */));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (definition.order_by)
|
|
|
|
{
|
|
|
|
for (const auto & column_ast
|
|
|
|
: definition.order_by->children)
|
|
|
|
{
|
|
|
|
// Parser should have checked that we have a proper element here.
|
|
|
|
const auto & order_by_element
|
|
|
|
= column_ast->as<ASTOrderByElement &>();
|
|
|
|
// Ignore collation for now.
|
|
|
|
desc.order_by.push_back(
|
|
|
|
SortColumnDescription(
|
|
|
|
order_by_element.children.front()->getColumnName(),
|
|
|
|
order_by_element.direction,
|
|
|
|
order_by_element.nulls_direction));
|
|
|
|
}
|
|
|
|
}
|
2016-03-05 02:30:20 +00:00
|
|
|
|
2021-01-13 19:29:52 +00:00
|
|
|
desc.full_sort_description = desc.partition_by;
|
|
|
|
desc.full_sort_description.insert(desc.full_sort_description.end(),
|
|
|
|
desc.order_by.begin(), desc.order_by.end());
|
2021-01-19 01:00:39 +00:00
|
|
|
|
2021-01-27 00:08:15 +00:00
|
|
|
if (definition.frame.type != WindowFrame::FrameType::Rows
|
|
|
|
&& definition.frame.type != WindowFrame::FrameType::Range)
|
2021-01-19 01:00:39 +00:00
|
|
|
{
|
|
|
|
std::string name = definition.frame.type == WindowFrame::FrameType::Rows
|
|
|
|
? "ROWS"
|
|
|
|
: definition.frame.type == WindowFrame::FrameType::Groups
|
|
|
|
? "GROUPS" : "RANGE";
|
|
|
|
|
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
|
|
|
|
"Window frame '{}' is not implemented (while processing '{}')",
|
|
|
|
name, ast->formatForErrorMessage());
|
|
|
|
}
|
|
|
|
|
|
|
|
desc.frame = definition.frame;
|
2021-01-13 19:29:52 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
|
2020-12-09 11:14:40 +00:00
|
|
|
{
|
2020-12-18 01:25:21 +00:00
|
|
|
// Convenient to check here because at least we have the Context.
|
2020-12-22 01:37:45 +00:00
|
|
|
if (!syntax->window_function_asts.empty() &&
|
2020-12-18 01:25:21 +00:00
|
|
|
!context.getSettingsRef().allow_experimental_window_functions)
|
|
|
|
{
|
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
|
2021-02-11 15:07:42 +00:00
|
|
|
"The support for window functions is experimental and will change"
|
|
|
|
" in backwards-incompatible ways in the future releases. Set"
|
|
|
|
" allow_experimental_window_functions = 1 to enable it."
|
|
|
|
" While processing '{}'",
|
2020-12-22 01:37:45 +00:00
|
|
|
syntax->window_function_asts[0]->formatForErrorMessage());
|
2020-12-18 01:25:21 +00:00
|
|
|
}
|
|
|
|
|
2021-01-13 19:29:52 +00:00
|
|
|
// Window definitions from the WINDOW clause
|
|
|
|
const auto * select_query = query->as<ASTSelectQuery>();
|
|
|
|
if (select_query && select_query->window())
|
2020-12-09 11:14:40 +00:00
|
|
|
{
|
2021-01-13 19:29:52 +00:00
|
|
|
for (const auto & ptr : select_query->window()->children)
|
2020-12-09 11:14:40 +00:00
|
|
|
{
|
2021-01-13 19:29:52 +00:00
|
|
|
const auto & elem = ptr->as<const ASTWindowListElement &>();
|
|
|
|
WindowDescription desc;
|
|
|
|
desc.window_name = elem.name;
|
2021-01-20 05:29:27 +00:00
|
|
|
makeWindowDescriptionFromAST(desc, elem.definition.get());
|
2020-12-10 19:06:52 +00:00
|
|
|
|
2021-01-13 19:29:52 +00:00
|
|
|
auto [it, inserted] = window_descriptions.insert(
|
|
|
|
{desc.window_name, desc});
|
|
|
|
|
|
|
|
if (!inserted)
|
2020-12-10 19:06:52 +00:00
|
|
|
{
|
2021-01-13 19:29:52 +00:00
|
|
|
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
|
|
|
"Window '{}' is defined twice in the WINDOW clause",
|
|
|
|
desc.window_name);
|
2020-12-09 11:14:40 +00:00
|
|
|
}
|
|
|
|
}
|
2021-01-13 19:29:52 +00:00
|
|
|
}
|
2020-12-16 21:44:05 +00:00
|
|
|
|
2021-01-13 19:29:52 +00:00
|
|
|
// Window functions
|
|
|
|
for (const ASTFunction * function_node : syntax->window_function_asts)
|
|
|
|
{
|
|
|
|
assert(function_node->is_window_function);
|
2020-12-09 11:14:40 +00:00
|
|
|
|
|
|
|
WindowFunctionDescription window_function;
|
2020-12-16 21:44:05 +00:00
|
|
|
window_function.function_node = function_node;
|
2020-12-09 11:14:40 +00:00
|
|
|
window_function.column_name
|
|
|
|
= window_function.function_node->getColumnName();
|
|
|
|
window_function.function_parameters
|
|
|
|
= window_function.function_node->parameters
|
|
|
|
? getAggregateFunctionParametersArray(
|
|
|
|
window_function.function_node->parameters)
|
|
|
|
: Array();
|
|
|
|
|
2020-12-16 21:44:05 +00:00
|
|
|
// Requiring a constant reference to a shared pointer to non-const AST
|
|
|
|
// doesn't really look sane, but the visitor does indeed require it.
|
2020-12-18 17:13:28 +00:00
|
|
|
// Hence we clone the node (not very sane either, I know).
|
2020-12-16 21:44:05 +00:00
|
|
|
getRootActionsNoMakeSet(window_function.function_node->clone(),
|
|
|
|
true, actions);
|
2020-12-10 19:06:52 +00:00
|
|
|
|
2020-12-09 11:14:40 +00:00
|
|
|
const ASTs & arguments
|
|
|
|
= window_function.function_node->arguments->children;
|
|
|
|
window_function.argument_types.resize(arguments.size());
|
|
|
|
window_function.argument_names.resize(arguments.size());
|
|
|
|
const auto & index = actions->getIndex();
|
|
|
|
for (size_t i = 0; i < arguments.size(); ++i)
|
|
|
|
{
|
|
|
|
const std::string & name = arguments[i]->getColumnName();
|
|
|
|
|
|
|
|
auto it = index.find(name);
|
|
|
|
if (it == index.end())
|
|
|
|
{
|
|
|
|
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER,
|
|
|
|
"Unknown identifier '{}' in window function '{}'",
|
|
|
|
name, window_function.function_node->formatForErrorMessage());
|
|
|
|
}
|
|
|
|
|
|
|
|
window_function.argument_types[i] = (*it)->result_type;
|
|
|
|
window_function.argument_names[i] = name;
|
|
|
|
}
|
|
|
|
|
|
|
|
AggregateFunctionProperties properties;
|
|
|
|
window_function.aggregate_function
|
|
|
|
= AggregateFunctionFactory::instance().get(
|
|
|
|
window_function.function_node->name,
|
|
|
|
window_function.argument_types,
|
|
|
|
window_function.function_parameters, properties);
|
|
|
|
|
2020-12-22 01:37:45 +00:00
|
|
|
|
2021-01-13 19:29:52 +00:00
|
|
|
// Find the window corresponding to this function. It may be either
|
|
|
|
// referenced by name and previously defined in WINDOW clause, or it
|
|
|
|
// may be defined inline.
|
|
|
|
if (!function_node->window_name.empty())
|
2020-12-22 01:37:45 +00:00
|
|
|
{
|
2021-01-13 19:29:52 +00:00
|
|
|
auto it = window_descriptions.find(function_node->window_name);
|
|
|
|
if (it == std::end(window_descriptions))
|
|
|
|
{
|
|
|
|
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER,
|
|
|
|
"Window '{}' is not defined (referenced by '{}')",
|
|
|
|
function_node->window_name,
|
|
|
|
function_node->formatForErrorMessage());
|
|
|
|
}
|
|
|
|
|
|
|
|
it->second.window_functions.push_back(window_function);
|
2020-12-22 01:37:45 +00:00
|
|
|
}
|
2021-01-13 19:29:52 +00:00
|
|
|
else
|
|
|
|
{
|
|
|
|
const auto & definition = function_node->window_definition->as<
|
|
|
|
const ASTWindowDefinition &>();
|
|
|
|
WindowDescription desc;
|
|
|
|
desc.window_name = definition.getDefaultWindowName();
|
2021-01-20 05:29:27 +00:00
|
|
|
makeWindowDescriptionFromAST(desc, &definition);
|
2020-12-22 01:37:45 +00:00
|
|
|
|
2021-01-13 19:29:52 +00:00
|
|
|
auto [it, inserted] = window_descriptions.insert(
|
|
|
|
{desc.window_name, desc});
|
|
|
|
|
|
|
|
if (!inserted)
|
|
|
|
{
|
|
|
|
assert(it->second.full_sort_description
|
|
|
|
== desc.full_sort_description);
|
|
|
|
}
|
2020-12-09 11:14:40 +00:00
|
|
|
|
2021-01-13 19:29:52 +00:00
|
|
|
it->second.window_functions.push_back(window_function);
|
|
|
|
}
|
|
|
|
}
|
2020-12-09 11:14:40 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-08-13 13:48:09 +00:00
|
|
|
const ASTSelectQuery * ExpressionAnalyzer::getSelectQuery() const
|
2013-05-28 11:54:37 +00:00
|
|
|
{
|
2019-03-11 13:22:51 +00:00
|
|
|
const auto * select_query = query->as<ASTSelectQuery>();
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!select_query)
|
|
|
|
throw Exception("Not a select query", ErrorCodes::LOGICAL_ERROR);
|
2019-08-13 13:48:09 +00:00
|
|
|
return select_query;
|
2013-05-28 11:54:37 +00:00
|
|
|
}
|
2013-06-14 16:38:54 +00:00
|
|
|
|
2019-08-14 19:30:30 +00:00
|
|
|
const ASTSelectQuery * SelectQueryExpressionAnalyzer::getAggregatingQuery() const
|
2013-05-24 10:49:19 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!has_aggregation)
|
|
|
|
throw Exception("No aggregation", ErrorCodes::LOGICAL_ERROR);
|
2019-08-13 13:48:09 +00:00
|
|
|
return getSelectQuery();
|
2013-05-28 11:54:37 +00:00
|
|
|
}
|
2013-06-14 16:38:54 +00:00
|
|
|
|
2016-07-23 02:25:09 +00:00
|
|
|
/// "Big" ARRAY JOIN.
|
2020-09-10 16:01:41 +00:00
|
|
|
ArrayJoinActionPtr ExpressionAnalyzer::addMultipleArrayJoinAction(ActionsDAGPtr & actions, bool array_join_is_left) const
|
2013-10-17 13:32:32 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
NameSet result_columns;
|
2018-11-08 15:43:14 +00:00
|
|
|
for (const auto & result_source : syntax->array_join_result_to_source)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
/// Assign new names to columns, if needed.
|
|
|
|
if (result_source.first != result_source.second)
|
2020-09-10 16:01:41 +00:00
|
|
|
actions->addAlias(result_source.second, result_source.first);
|
2015-10-22 20:56:52 +00:00
|
|
|
|
2017-04-02 17:37:49 +00:00
|
|
|
/// Make ARRAY JOIN (replace arrays with their insides) for the columns in these new names.
|
2017-04-01 07:20:54 +00:00
|
|
|
result_columns.insert(result_source.first);
|
|
|
|
}
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2020-08-11 12:03:18 +00:00
|
|
|
return std::make_shared<ArrayJoinAction>(result_columns, array_join_is_left, context);
|
2013-10-17 13:32:32 +00:00
|
|
|
}
|
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
ArrayJoinActionPtr SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, ActionsDAGPtr & before_array_join, bool only_types)
|
2013-07-26 16:33:05 +00:00
|
|
|
{
|
2019-08-13 13:48:09 +00:00
|
|
|
const auto * select_query = getSelectQuery();
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2018-12-19 13:13:51 +00:00
|
|
|
bool is_array_join_left;
|
2020-03-23 02:12:31 +00:00
|
|
|
ASTPtr array_join_expression_list = select_query->arrayJoinExpressionList(is_array_join_left);
|
2018-12-19 13:13:51 +00:00
|
|
|
if (!array_join_expression_list)
|
2020-08-11 12:03:18 +00:00
|
|
|
return nullptr;
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2020-06-25 23:05:47 +00:00
|
|
|
ExpressionActionsChain::Step & step = chain.lastStep(sourceColumns());
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2020-08-13 20:17:18 +00:00
|
|
|
getRootActions(array_join_expression_list, only_types, step.actions());
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2020-08-13 20:17:18 +00:00
|
|
|
auto array_join = addMultipleArrayJoinAction(step.actions(), is_array_join_left);
|
2020-09-11 12:24:41 +00:00
|
|
|
before_array_join = chain.getLastActions();
|
2020-08-12 08:55:16 +00:00
|
|
|
|
2020-08-19 19:33:49 +00:00
|
|
|
chain.steps.push_back(std::make_unique<ExpressionActionsChain::ArrayJoinStep>(
|
2020-09-08 10:40:53 +00:00
|
|
|
array_join, step.getResultColumns()));
|
2020-08-12 08:55:16 +00:00
|
|
|
|
|
|
|
chain.addStep();
|
2013-10-21 11:33:25 +00:00
|
|
|
|
2020-08-11 12:03:18 +00:00
|
|
|
return array_join;
|
2013-07-26 16:33:05 +00:00
|
|
|
}
|
|
|
|
|
2021-02-09 13:17:42 +00:00
|
|
|
bool SelectQueryExpressionAnalyzer::appendJoinLeftKeys(ExpressionActionsChain & chain, bool only_types)
|
2020-06-25 23:05:47 +00:00
|
|
|
{
|
2020-08-11 12:03:18 +00:00
|
|
|
ExpressionActionsChain::Step & step = chain.lastStep(columns_after_array_join);
|
2020-06-25 23:05:47 +00:00
|
|
|
|
2020-08-13 20:17:18 +00:00
|
|
|
getRootActions(analyzedJoin().leftKeysList(), only_types, step.actions());
|
2020-06-25 23:05:47 +00:00
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2021-02-09 13:17:42 +00:00
|
|
|
JoinPtr SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, ActionsDAGPtr & left_actions)
|
2019-08-16 16:50:30 +00:00
|
|
|
{
|
2021-02-09 13:17:42 +00:00
|
|
|
JoinPtr table_join = makeTableJoin(*syntax->ast_join);
|
|
|
|
if (syntax->analyzed_join->needConvert())
|
2021-01-31 19:25:47 +00:00
|
|
|
{
|
2021-02-09 13:17:42 +00:00
|
|
|
left_actions = createJoinConvertingActions(chain.getLastStep().getResultColumns(),
|
|
|
|
syntax->analyzed_join->getLeftMapping());
|
|
|
|
chain.steps.push_back(std::make_unique<ExpressionActionsChain::ExpressionActionsStep>(left_actions));
|
2021-01-31 19:25:47 +00:00
|
|
|
chain.addStep();
|
|
|
|
}
|
2020-09-08 10:40:53 +00:00
|
|
|
|
2021-01-31 19:25:47 +00:00
|
|
|
ExpressionActionsChain::Step & step = chain.lastStep(columns_after_array_join);
|
|
|
|
chain.steps.push_back(std::make_unique<ExpressionActionsChain::JoinStep>(syntax->analyzed_join, table_join, step.getResultColumns()));
|
2020-09-08 10:40:53 +00:00
|
|
|
chain.addStep();
|
|
|
|
return table_join;
|
2019-08-16 16:50:30 +00:00
|
|
|
}
|
2018-08-28 13:57:31 +00:00
|
|
|
|
2020-04-08 18:59:52 +00:00
|
|
|
static JoinPtr tryGetStorageJoin(std::shared_ptr<TableJoin> analyzed_join)
|
2019-08-16 16:50:30 +00:00
|
|
|
{
|
2020-04-08 18:59:52 +00:00
|
|
|
if (auto * table = analyzed_join->joined_storage.get())
|
|
|
|
if (auto * storage_join = dynamic_cast<StorageJoin *>(table))
|
|
|
|
return storage_join->getJoin(analyzed_join);
|
2019-08-16 16:50:30 +00:00
|
|
|
return {};
|
|
|
|
}
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
static ExpressionActionsPtr createJoinedBlockActions(const Context & context, const TableJoin & analyzed_join)
|
2019-09-04 16:37:05 +00:00
|
|
|
{
|
|
|
|
ASTPtr expression_list = analyzed_join.rightKeysList();
|
2020-07-22 17:13:05 +00:00
|
|
|
auto syntax_result = TreeRewriter(context).analyze(expression_list, analyzed_join.columnsFromJoinedTable());
|
2019-09-04 16:37:05 +00:00
|
|
|
return ExpressionAnalyzer(expression_list, syntax_result, context).getActions(true, false);
|
|
|
|
}
|
|
|
|
|
2020-04-09 20:00:57 +00:00
|
|
|
static bool allowDictJoin(StoragePtr joined_storage, const Context & context, String & dict_name, String & key_name)
|
2020-04-08 19:58:27 +00:00
|
|
|
{
|
2020-04-22 06:01:33 +00:00
|
|
|
const auto * dict = dynamic_cast<const StorageDictionary *>(joined_storage.get());
|
2020-04-08 19:58:27 +00:00
|
|
|
if (!dict)
|
|
|
|
return false;
|
|
|
|
|
2020-09-22 11:56:40 +00:00
|
|
|
dict_name = dict->resolvedDictionaryName();
|
2020-04-08 19:58:27 +00:00
|
|
|
auto dictionary = context.getExternalDictionariesLoader().getDictionary(dict_name);
|
|
|
|
if (!dictionary)
|
|
|
|
return false;
|
|
|
|
|
|
|
|
const DictionaryStructure & structure = dictionary->getStructure();
|
2020-04-09 20:00:57 +00:00
|
|
|
if (structure.id)
|
|
|
|
{
|
|
|
|
key_name = structure.id->name;
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
return false;
|
2020-04-08 19:58:27 +00:00
|
|
|
}
|
|
|
|
|
2020-04-09 20:00:57 +00:00
|
|
|
static std::shared_ptr<IJoin> makeJoin(std::shared_ptr<TableJoin> analyzed_join, const Block & sample_block, const Context & context)
|
2020-02-17 17:08:31 +00:00
|
|
|
{
|
2020-02-20 11:26:00 +00:00
|
|
|
bool allow_merge_join = analyzed_join->allowMergeJoin();
|
|
|
|
|
2020-04-09 20:00:57 +00:00
|
|
|
/// HashJoin with Dictionary optimisation
|
2020-04-08 19:58:27 +00:00
|
|
|
String dict_name;
|
2020-04-09 20:00:57 +00:00
|
|
|
String key_name;
|
|
|
|
if (analyzed_join->joined_storage && allowDictJoin(analyzed_join->joined_storage, context, dict_name, key_name))
|
2020-04-08 18:59:52 +00:00
|
|
|
{
|
2020-04-09 20:00:57 +00:00
|
|
|
Names original_names;
|
|
|
|
NamesAndTypesList result_columns;
|
|
|
|
if (analyzed_join->allowDictJoin(key_name, sample_block, original_names, result_columns))
|
|
|
|
{
|
|
|
|
analyzed_join->dictionary_reader = std::make_shared<DictionaryReader>(dict_name, original_names, result_columns, context);
|
|
|
|
return std::make_shared<HashJoin>(analyzed_join, sample_block);
|
|
|
|
}
|
2020-04-08 18:59:52 +00:00
|
|
|
}
|
|
|
|
|
2020-02-20 11:26:00 +00:00
|
|
|
if (analyzed_join->forceHashJoin() || (analyzed_join->preferMergeJoin() && !allow_merge_join))
|
2020-04-07 09:48:47 +00:00
|
|
|
return std::make_shared<HashJoin>(analyzed_join, sample_block);
|
2020-02-20 11:26:00 +00:00
|
|
|
else if (analyzed_join->forceMergeJoin() || (analyzed_join->preferMergeJoin() && allow_merge_join))
|
2020-02-17 17:08:31 +00:00
|
|
|
return std::make_shared<MergeJoin>(analyzed_join, sample_block);
|
|
|
|
return std::make_shared<JoinSwitcher>(analyzed_join, sample_block);
|
|
|
|
}
|
|
|
|
|
2021-02-09 13:17:42 +00:00
|
|
|
JoinPtr SelectQueryExpressionAnalyzer::makeTableJoin(const ASTTablesInSelectQueryElement & join_element)
|
2019-08-16 16:50:30 +00:00
|
|
|
{
|
|
|
|
/// Two JOINs are not supported with the same subquery, but different USINGs.
|
|
|
|
auto join_hash = join_element.getTreeHash();
|
|
|
|
String join_subquery_id = toString(join_hash.first) + "_" + toString(join_hash.second);
|
|
|
|
|
2019-09-09 19:43:37 +00:00
|
|
|
SubqueryForSet & subquery_for_join = subqueries_for_sets[join_subquery_id];
|
2019-08-16 16:50:30 +00:00
|
|
|
|
2020-04-08 18:59:52 +00:00
|
|
|
/// Use StorageJoin if any.
|
2019-09-09 19:43:37 +00:00
|
|
|
if (!subquery_for_join.join)
|
2020-04-08 18:59:52 +00:00
|
|
|
subquery_for_join.join = tryGetStorageJoin(syntax->analyzed_join);
|
2018-07-30 13:57:50 +00:00
|
|
|
|
2019-09-09 19:43:37 +00:00
|
|
|
if (!subquery_for_join.join)
|
2019-09-04 16:20:02 +00:00
|
|
|
{
|
|
|
|
/// Actions which need to be calculated on joined block.
|
2019-09-04 16:37:05 +00:00
|
|
|
ExpressionActionsPtr joined_block_actions = createJoinedBlockActions(context, analyzedJoin());
|
2019-09-04 16:20:02 +00:00
|
|
|
|
2020-04-08 18:59:52 +00:00
|
|
|
Names original_right_columns;
|
2020-09-15 13:25:14 +00:00
|
|
|
if (!subquery_for_join.source)
|
2019-09-09 19:43:37 +00:00
|
|
|
{
|
2020-04-08 18:59:52 +00:00
|
|
|
NamesWithAliases required_columns_with_aliases = analyzedJoin().getRequiredColumns(
|
|
|
|
joined_block_actions->getSampleBlock(), joined_block_actions->getRequiredColumns());
|
|
|
|
for (auto & pr : required_columns_with_aliases)
|
|
|
|
original_right_columns.push_back(pr.first);
|
|
|
|
|
|
|
|
/** 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.
|
|
|
|
*/
|
|
|
|
auto interpreter = interpretSubquery(join_element.table_expression, context, original_right_columns, query_options);
|
|
|
|
|
|
|
|
subquery_for_join.makeSource(interpreter, std::move(required_columns_with_aliases));
|
2019-09-09 19:43:37 +00:00
|
|
|
}
|
2018-07-30 13:57:50 +00:00
|
|
|
|
2019-09-04 16:20:02 +00:00
|
|
|
/// TODO You do not need to set this up when JOIN is only needed on remote servers.
|
2021-01-31 19:25:47 +00:00
|
|
|
subquery_for_join.addJoinActions(joined_block_actions); /// changes subquery_for_join.sample_block inside
|
|
|
|
|
2021-02-09 13:17:42 +00:00
|
|
|
if (syntax->analyzed_join->needConvert())
|
|
|
|
{
|
|
|
|
auto right_actions = createJoinConvertingActions(subquery_for_join.sample_block.getColumnsWithTypeAndName(),
|
|
|
|
syntax->analyzed_join->getRightMapping());
|
|
|
|
subquery_for_join.addJoinActions(std::make_shared<ExpressionActions>(right_actions));
|
|
|
|
}
|
2021-01-31 19:25:47 +00:00
|
|
|
|
2020-04-09 20:00:57 +00:00
|
|
|
subquery_for_join.join = makeJoin(syntax->analyzed_join, subquery_for_join.sample_block, context);
|
2020-04-08 18:59:52 +00:00
|
|
|
|
|
|
|
/// Do not make subquery for join over dictionary.
|
|
|
|
if (syntax->analyzed_join->dictionary_reader)
|
|
|
|
{
|
|
|
|
JoinPtr join = subquery_for_join.join;
|
|
|
|
subqueries_for_sets.erase(join_subquery_id);
|
|
|
|
return join;
|
|
|
|
}
|
2019-09-04 16:20:02 +00:00
|
|
|
}
|
|
|
|
|
2019-09-16 12:37:46 +00:00
|
|
|
return subquery_for_join.join;
|
2019-08-16 16:50:30 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere(
|
2018-11-08 16:39:43 +00:00
|
|
|
ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns)
|
2018-04-06 13:58:06 +00:00
|
|
|
{
|
2019-08-13 13:48:09 +00:00
|
|
|
const auto * select_query = getSelectQuery();
|
2020-11-03 11:28:28 +00:00
|
|
|
ActionsDAGPtr prewhere_actions;
|
2018-04-06 13:58:06 +00:00
|
|
|
|
2019-04-09 14:22:35 +00:00
|
|
|
if (!select_query->prewhere())
|
2020-09-11 12:24:41 +00:00
|
|
|
return prewhere_actions;
|
2018-04-06 13:58:06 +00:00
|
|
|
|
2020-06-25 23:05:47 +00:00
|
|
|
auto & step = chain.lastStep(sourceColumns());
|
2020-08-13 20:17:18 +00:00
|
|
|
getRootActions(select_query->prewhere(), only_types, step.actions());
|
2019-04-09 14:22:35 +00:00
|
|
|
String prewhere_column_name = select_query->prewhere()->getColumnName();
|
2018-04-12 09:45:24 +00:00
|
|
|
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
|
|
|
|
2021-01-28 09:40:08 +00:00
|
|
|
const auto & index = step.actions()->getIndex();
|
|
|
|
auto it = index.find(prewhere_column_name);
|
|
|
|
if (it == index.end())
|
|
|
|
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown identifier: '{}'", prewhere_column_name);
|
|
|
|
|
|
|
|
auto filter_type = (*it)->result_type;
|
2020-07-02 07:44:47 +00:00
|
|
|
if (!filter_type->canBeUsedInBooleanContext())
|
2020-06-30 14:44:49 +00:00
|
|
|
throw Exception("Invalid type for filter in PREWHERE: " + filter_type->getName(),
|
2020-07-02 07:44:47 +00:00
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER);
|
2020-06-30 14:44:49 +00:00
|
|
|
|
2018-04-12 09:45:24 +00:00
|
|
|
{
|
|
|
|
/// Remove unused source_columns from prewhere actions.
|
2020-09-11 12:24:41 +00:00
|
|
|
auto tmp_actions_dag = std::make_shared<ActionsDAG>(sourceColumns());
|
|
|
|
getRootActions(select_query->prewhere(), only_types, tmp_actions_dag);
|
2020-11-03 11:28:28 +00:00
|
|
|
tmp_actions_dag->removeUnusedActions({prewhere_column_name});
|
2020-11-11 17:17:26 +00:00
|
|
|
auto tmp_actions = std::make_shared<ExpressionActions>(tmp_actions_dag);
|
|
|
|
auto required_columns = tmp_actions->getRequiredColumns();
|
2018-04-12 09:45:24 +00:00
|
|
|
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.
|
2018-11-08 16:39:43 +00:00
|
|
|
for (const auto & column : additional_required_columns)
|
2018-09-10 10:06:12 +00:00
|
|
|
{
|
|
|
|
if (required_source_columns.count(column))
|
|
|
|
{
|
|
|
|
step.required_output.push_back(column);
|
|
|
|
step.can_remove_required_output.push_back(true);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-09-11 12:24:41 +00:00
|
|
|
auto names = step.actions()->getNames();
|
2018-04-12 09:45:24 +00:00
|
|
|
NameSet name_set(names.begin(), names.end());
|
|
|
|
|
2019-08-09 14:50:04 +00:00
|
|
|
for (const auto & column : sourceColumns())
|
2018-04-12 09:45:24 +00:00
|
|
|
if (required_source_columns.count(column.name) == 0)
|
|
|
|
name_set.erase(column.name);
|
|
|
|
|
|
|
|
Names required_output(name_set.begin(), name_set.end());
|
2020-09-11 12:24:41 +00:00
|
|
|
prewhere_actions = chain.getLastActions();
|
2020-11-03 11:28:28 +00:00
|
|
|
prewhere_actions->removeUnusedActions(required_output);
|
2018-04-12 09:45:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
{
|
|
|
|
/// 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.
|
2020-11-03 11:28:28 +00:00
|
|
|
ColumnsWithTypeAndName columns = prewhere_actions->getResultColumns();
|
2020-09-11 12:24:41 +00:00
|
|
|
auto required_columns = prewhere_actions->getRequiredColumns();
|
2020-11-03 11:28:28 +00:00
|
|
|
NameSet prewhere_input_names;
|
2018-04-12 09:45:24 +00:00
|
|
|
NameSet unused_source_columns;
|
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
for (const auto & col : required_columns)
|
|
|
|
prewhere_input_names.insert(col.name);
|
|
|
|
|
2019-08-09 14:50:04 +00:00
|
|
|
for (const auto & column : sourceColumns())
|
2018-04-12 09:45:24 +00:00
|
|
|
{
|
|
|
|
if (prewhere_input_names.count(column.name) == 0)
|
|
|
|
{
|
|
|
|
columns.emplace_back(column.type, column.name);
|
|
|
|
unused_source_columns.emplace(column.name);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-08-19 19:33:49 +00:00
|
|
|
chain.steps.emplace_back(std::make_unique<ExpressionActionsChain::ExpressionActionsStep>(
|
2020-09-11 12:24:41 +00:00
|
|
|
std::make_shared<ActionsDAG>(std::move(columns))));
|
2020-08-19 19:33:49 +00:00
|
|
|
chain.steps.back()->additional_input = std::move(unused_source_columns);
|
2020-09-11 12:24:41 +00:00
|
|
|
chain.getLastActions();
|
|
|
|
chain.addStep();
|
2018-04-12 09:45:24 +00:00
|
|
|
}
|
2018-04-06 13:58:06 +00:00
|
|
|
|
2020-09-11 12:24:41 +00:00
|
|
|
return prewhere_actions;
|
2018-04-06 13:58:06 +00:00
|
|
|
}
|
2016-07-22 20:39:28 +00:00
|
|
|
|
2020-11-11 16:52:27 +00:00
|
|
|
void SelectQueryExpressionAnalyzer::appendPreliminaryFilter(ExpressionActionsChain & chain, ActionsDAGPtr actions_dag, String column_name)
|
2019-10-03 11:58:52 +00:00
|
|
|
{
|
2020-06-25 23:05:47 +00:00
|
|
|
ExpressionActionsChain::Step & step = chain.lastStep(sourceColumns());
|
2019-10-03 11:58:52 +00:00
|
|
|
|
2019-10-05 19:34:57 +00:00
|
|
|
// FIXME: assert(filter_info);
|
2020-09-11 12:24:41 +00:00
|
|
|
auto * expression_step = typeid_cast<ExpressionActionsChain::ExpressionActionsStep *>(&step);
|
2020-11-11 16:52:27 +00:00
|
|
|
expression_step->actions_dag = std::move(actions_dag);
|
2019-10-03 11:58:52 +00:00
|
|
|
step.required_output.push_back(std::move(column_name));
|
|
|
|
step.can_remove_required_output = {true};
|
|
|
|
|
|
|
|
chain.addStep();
|
|
|
|
}
|
|
|
|
|
2019-08-14 19:30:30 +00:00
|
|
|
bool SelectQueryExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, bool only_types)
|
2013-05-28 11:54:37 +00:00
|
|
|
{
|
2019-08-13 13:48:09 +00:00
|
|
|
const auto * select_query = getSelectQuery();
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2019-04-09 14:22:35 +00:00
|
|
|
if (!select_query->where())
|
2017-04-01 07:20:54 +00:00
|
|
|
return false;
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2020-08-11 12:03:18 +00:00
|
|
|
ExpressionActionsChain::Step & step = chain.lastStep(columns_after_join);
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2020-10-21 18:17:27 +00:00
|
|
|
getRootActions(select_query->where(), only_types, step.actions());
|
|
|
|
|
2020-06-30 14:44:49 +00:00
|
|
|
auto where_column_name = select_query->where()->getColumnName();
|
|
|
|
step.required_output.push_back(where_column_name);
|
2018-06-29 11:42:44 +00:00
|
|
|
step.can_remove_required_output = {true};
|
2018-04-23 19:05:46 +00:00
|
|
|
|
2021-01-28 09:40:08 +00:00
|
|
|
const auto & index = step.actions()->getIndex();
|
|
|
|
auto it = index.find(where_column_name);
|
|
|
|
if (it == index.end())
|
|
|
|
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown identifier: '{}'", where_column_name);
|
|
|
|
|
|
|
|
auto filter_type = (*it)->result_type;
|
2020-07-02 07:44:47 +00:00
|
|
|
if (!filter_type->canBeUsedInBooleanContext())
|
2020-06-30 14:48:00 +00:00
|
|
|
throw Exception("Invalid type for filter in WHERE: " + filter_type->getName(),
|
2020-07-02 07:44:47 +00:00
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER);
|
2020-06-30 14:44:49 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return true;
|
2013-05-28 11:54:37 +00:00
|
|
|
}
|
|
|
|
|
2020-02-16 19:46:45 +00:00
|
|
|
bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain, bool only_types, bool optimize_aggregation_in_order,
|
|
|
|
ManyExpressionActions & group_by_elements_actions)
|
2013-05-28 11:54:37 +00:00
|
|
|
{
|
2019-08-13 13:48:09 +00:00
|
|
|
const auto * select_query = getAggregatingQuery();
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2019-04-09 14:22:35 +00:00
|
|
|
if (!select_query->groupBy())
|
2017-04-01 07:20:54 +00:00
|
|
|
return false;
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2020-08-11 12:03:18 +00:00
|
|
|
ExpressionActionsChain::Step & step = chain.lastStep(columns_after_join);
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2019-04-09 14:22:35 +00:00
|
|
|
ASTs asts = select_query->groupBy()->children;
|
2020-03-08 23:48:08 +00:00
|
|
|
for (const auto & ast : asts)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-03-08 23:48:08 +00:00
|
|
|
step.required_output.emplace_back(ast->getColumnName());
|
2020-08-13 20:17:18 +00:00
|
|
|
getRootActions(ast, only_types, step.actions());
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2020-02-16 19:46:45 +00:00
|
|
|
if (optimize_aggregation_in_order)
|
|
|
|
{
|
|
|
|
for (auto & child : asts)
|
|
|
|
{
|
2020-09-11 12:24:41 +00:00
|
|
|
auto actions_dag = std::make_shared<ActionsDAG>(columns_after_join);
|
|
|
|
getRootActions(child, only_types, actions_dag);
|
2020-11-10 16:27:55 +00:00
|
|
|
group_by_elements_actions.emplace_back(std::make_shared<ExpressionActions>(actions_dag));
|
2020-02-16 19:46:45 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return true;
|
2013-05-28 11:54:37 +00:00
|
|
|
}
|
|
|
|
|
2019-08-14 19:30:30 +00:00
|
|
|
void SelectQueryExpressionAnalyzer::appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types)
|
2013-05-28 11:54:37 +00:00
|
|
|
{
|
2019-08-13 13:48:09 +00:00
|
|
|
const auto * select_query = getAggregatingQuery();
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2020-08-11 12:03:18 +00:00
|
|
|
ExpressionActionsChain::Step & step = chain.lastStep(columns_after_join);
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2020-03-08 23:48:08 +00:00
|
|
|
for (const auto & desc : aggregate_descriptions)
|
|
|
|
for (const auto & name : desc.argument_names)
|
|
|
|
step.required_output.emplace_back(name);
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2019-08-13 12:39:03 +00:00
|
|
|
/// Collect aggregates removing duplicates by node.getColumnName()
|
|
|
|
/// It's not clear why we recollect aggregates (for query parts) while we're able to use previously collected ones (for entire query)
|
|
|
|
/// @note The original recollection logic didn't remove duplicates.
|
|
|
|
GetAggregatesVisitor::Data data;
|
|
|
|
GetAggregatesVisitor(data).visit(select_query->select());
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2019-04-09 14:22:35 +00:00
|
|
|
if (select_query->having())
|
2019-08-13 12:39:03 +00:00
|
|
|
GetAggregatesVisitor(data).visit(select_query->having());
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2019-04-09 14:22:35 +00:00
|
|
|
if (select_query->orderBy())
|
2019-08-13 12:39:03 +00:00
|
|
|
GetAggregatesVisitor(data).visit(select_query->orderBy());
|
|
|
|
|
|
|
|
/// TODO: data.aggregates -> aggregates()
|
|
|
|
for (const ASTFunction * node : data.aggregates)
|
2020-12-04 02:15:44 +00:00
|
|
|
if (node->arguments)
|
|
|
|
for (auto & argument : node->arguments->children)
|
|
|
|
getRootActions(argument, only_types, step.actions());
|
2020-12-09 11:14:40 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void SelectQueryExpressionAnalyzer::appendWindowFunctionsArguments(
|
2020-12-10 19:06:52 +00:00
|
|
|
ExpressionActionsChain & chain, bool /* only_types */)
|
2020-12-09 11:14:40 +00:00
|
|
|
{
|
|
|
|
ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns);
|
|
|
|
|
2021-01-13 19:29:52 +00:00
|
|
|
// (1) Add actions for window functions and the columns they require.
|
|
|
|
// (2) Mark the columns that are really required. We have to mark them as
|
|
|
|
// required because we finish the expression chain before processing the
|
|
|
|
// window functions.
|
|
|
|
// The required columns are:
|
|
|
|
// (a) window function arguments,
|
|
|
|
// (b) the columns from PARTITION BY and ORDER BY.
|
|
|
|
|
|
|
|
// (1a) Actions for PARTITION BY and ORDER BY for windows defined in the
|
|
|
|
// WINDOW clause. The inline window definitions will be processed
|
|
|
|
// recursively together with (1b) as ASTFunction::window_definition.
|
|
|
|
if (getSelectQuery()->window())
|
|
|
|
{
|
|
|
|
getRootActionsNoMakeSet(getSelectQuery()->window(),
|
|
|
|
true /* no_subqueries */, step.actions());
|
|
|
|
}
|
|
|
|
|
2020-12-22 19:01:15 +00:00
|
|
|
for (const auto & [_, w] : window_descriptions)
|
2020-12-10 19:06:52 +00:00
|
|
|
{
|
2020-12-22 19:01:15 +00:00
|
|
|
for (const auto & f : w.window_functions)
|
|
|
|
{
|
2021-01-13 19:29:52 +00:00
|
|
|
// (1b) Actions for function arguments, and also the inline window
|
|
|
|
// definitions (1a).
|
2020-12-22 19:01:15 +00:00
|
|
|
// Requiring a constant reference to a shared pointer to non-const AST
|
|
|
|
// doesn't really look sane, but the visitor does indeed require it.
|
|
|
|
getRootActionsNoMakeSet(f.function_node->clone(),
|
|
|
|
true /* no_subqueries */, step.actions());
|
|
|
|
|
2021-01-13 19:29:52 +00:00
|
|
|
// (2b) Required function argument columns.
|
2020-12-22 19:01:15 +00:00
|
|
|
for (const auto & a : f.function_node->arguments->children)
|
|
|
|
{
|
|
|
|
step.required_output.push_back(a->getColumnName());
|
|
|
|
}
|
2020-12-10 19:06:52 +00:00
|
|
|
}
|
|
|
|
|
2021-01-13 19:29:52 +00:00
|
|
|
// (2a) Required PARTITION BY and ORDER BY columns.
|
2020-12-10 19:06:52 +00:00
|
|
|
for (const auto & c : w.full_sort_description)
|
|
|
|
{
|
|
|
|
step.required_output.push_back(c.column_name);
|
|
|
|
}
|
|
|
|
}
|
2013-05-28 11:54:37 +00:00
|
|
|
}
|
|
|
|
|
2019-08-14 19:30:30 +00:00
|
|
|
bool SelectQueryExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_types)
|
2013-05-28 11:54:37 +00:00
|
|
|
{
|
2019-08-13 13:48:09 +00:00
|
|
|
const auto * select_query = getAggregatingQuery();
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2019-04-09 14:22:35 +00:00
|
|
|
if (!select_query->having())
|
2017-04-01 07:20:54 +00:00
|
|
|
return false;
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2020-06-25 23:05:47 +00:00
|
|
|
ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns);
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2020-11-02 12:07:01 +00:00
|
|
|
getRootActionsForHaving(select_query->having(), only_types, step.actions());
|
2020-10-21 18:17:27 +00:00
|
|
|
step.required_output.push_back(select_query->having()->getColumnName());
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return true;
|
2013-05-24 10:49:19 +00:00
|
|
|
}
|
|
|
|
|
2019-08-14 19:30:30 +00:00
|
|
|
void SelectQueryExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, bool only_types)
|
2013-05-28 11:54:37 +00:00
|
|
|
{
|
2019-08-13 13:48:09 +00:00
|
|
|
const auto * select_query = getSelectQuery();
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2020-06-25 23:05:47 +00:00
|
|
|
ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns);
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2020-08-13 20:17:18 +00:00
|
|
|
getRootActions(select_query->select(), only_types, step.actions());
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2019-04-09 14:22:35 +00:00
|
|
|
for (const auto & child : select_query->select()->children)
|
2020-12-09 11:14:40 +00:00
|
|
|
{
|
2020-12-24 04:03:33 +00:00
|
|
|
if (const auto * function = typeid_cast<const ASTFunction *>(child.get());
|
|
|
|
function
|
|
|
|
&& function->is_window_function)
|
|
|
|
{
|
|
|
|
// Skip window function columns here -- they are calculated after
|
|
|
|
// other SELECT expressions by a special step.
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
|
2018-03-01 01:25:06 +00:00
|
|
|
step.required_output.push_back(child->getColumnName());
|
2020-12-09 11:14:40 +00:00
|
|
|
}
|
2013-05-28 11:54:37 +00:00
|
|
|
}
|
2013-05-24 10:49:19 +00:00
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only_types, bool optimize_read_in_order,
|
2020-02-10 19:55:13 +00:00
|
|
|
ManyExpressionActions & order_by_elements_actions)
|
2013-05-24 10:49:19 +00:00
|
|
|
{
|
2019-08-13 13:48:09 +00:00
|
|
|
const auto * select_query = getSelectQuery();
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2019-04-09 14:22:35 +00:00
|
|
|
if (!select_query->orderBy())
|
2020-11-03 11:28:28 +00:00
|
|
|
{
|
|
|
|
auto actions = chain.getLastActions();
|
|
|
|
chain.addStep();
|
|
|
|
return actions;
|
|
|
|
}
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2020-06-25 23:05:47 +00:00
|
|
|
ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns);
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2020-08-13 20:17:18 +00:00
|
|
|
getRootActions(select_query->orderBy(), only_types, step.actions());
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
bool with_fill = false;
|
|
|
|
NameSet order_by_keys;
|
2019-04-09 14:22:35 +00:00
|
|
|
for (auto & child : select_query->orderBy()->children)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-04-09 14:22:35 +00:00
|
|
|
const auto * ast = child->as<ASTOrderByElement>();
|
2020-03-08 23:48:08 +00:00
|
|
|
if (!ast || ast->children.empty())
|
2017-04-01 07:20:54 +00:00
|
|
|
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());
|
2020-11-03 11:28:28 +00:00
|
|
|
|
|
|
|
if (ast->with_fill)
|
|
|
|
with_fill = true;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2019-12-20 13:15:17 +00:00
|
|
|
if (optimize_read_in_order)
|
|
|
|
{
|
|
|
|
for (auto & child : select_query->orderBy()->children)
|
2019-12-10 23:18:24 +00:00
|
|
|
{
|
2020-09-11 12:24:41 +00:00
|
|
|
auto actions_dag = std::make_shared<ActionsDAG>(columns_after_join);
|
|
|
|
getRootActions(child, only_types, actions_dag);
|
2020-11-10 16:27:55 +00:00
|
|
|
order_by_elements_actions.emplace_back(std::make_shared<ExpressionActions>(actions_dag));
|
2019-12-10 23:18:24 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2020-11-03 11:28:28 +00:00
|
|
|
|
|
|
|
NameSet non_constant_inputs;
|
|
|
|
if (with_fill)
|
|
|
|
{
|
|
|
|
for (const auto & column : step.getResultColumns())
|
|
|
|
if (!order_by_keys.count(column.name))
|
|
|
|
non_constant_inputs.insert(column.name);
|
|
|
|
}
|
|
|
|
|
|
|
|
auto actions = chain.getLastActions();
|
|
|
|
chain.addStep(non_constant_inputs);
|
|
|
|
return actions;
|
2013-05-28 11:54:37 +00:00
|
|
|
}
|
|
|
|
|
2019-08-14 19:30:30 +00:00
|
|
|
bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain, bool only_types)
|
2018-03-01 05:24:56 +00:00
|
|
|
{
|
2019-08-13 13:48:09 +00:00
|
|
|
const auto * select_query = getSelectQuery();
|
2018-03-01 05:24:56 +00:00
|
|
|
|
2019-04-09 14:22:35 +00:00
|
|
|
if (!select_query->limitBy())
|
2018-03-01 05:24:56 +00:00
|
|
|
return false;
|
|
|
|
|
2020-06-25 23:05:47 +00:00
|
|
|
ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns);
|
2018-03-01 05:24:56 +00:00
|
|
|
|
2020-08-13 20:17:18 +00:00
|
|
|
getRootActions(select_query->limitBy(), only_types, step.actions());
|
2018-03-01 05:24:56 +00:00
|
|
|
|
2019-05-23 16:25:53 +00:00
|
|
|
NameSet aggregated_names;
|
|
|
|
for (const auto & column : aggregated_columns)
|
|
|
|
{
|
2019-05-25 10:51:48 +00:00
|
|
|
step.required_output.push_back(column.name);
|
|
|
|
aggregated_names.insert(column.name);
|
2019-05-23 16:25:53 +00:00
|
|
|
}
|
|
|
|
|
2019-04-09 14:22:35 +00:00
|
|
|
for (const auto & child : select_query->limitBy()->children)
|
2019-05-25 10:51:48 +00:00
|
|
|
{
|
|
|
|
auto child_name = child->getColumnName();
|
|
|
|
if (!aggregated_names.count(child_name))
|
|
|
|
step.required_output.push_back(std::move(child_name));
|
|
|
|
}
|
2018-03-01 05:24:56 +00:00
|
|
|
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
ActionsDAGPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) const
|
2013-05-28 11:54:37 +00:00
|
|
|
{
|
2019-08-13 13:48:09 +00:00
|
|
|
const auto * select_query = getSelectQuery();
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2020-06-25 23:05:47 +00:00
|
|
|
ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns);
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
NamesWithAliases result_columns;
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2019-04-09 14:22:35 +00:00
|
|
|
ASTs asts = select_query->select()->children;
|
2020-03-08 23:48:08 +00:00
|
|
|
for (const auto & ast : asts)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-03-08 23:48:08 +00:00
|
|
|
String result_name = ast->getAliasOrColumnName();
|
2019-01-30 15:51:39 +00:00
|
|
|
if (required_result_columns.empty() || required_result_columns.count(result_name))
|
2018-02-26 09:05:06 +00:00
|
|
|
{
|
2020-04-01 12:16:39 +00:00
|
|
|
std::string source_name = ast->getColumnName();
|
|
|
|
|
|
|
|
/*
|
|
|
|
* For temporary columns created by ExpressionAnalyzer for literals,
|
|
|
|
* use the correct source column. Using the default display name
|
|
|
|
* returned by getColumnName is not enough, and we have to use the
|
|
|
|
* column id set by EA. In principle, this logic applies to all kinds
|
|
|
|
* of columns, not only literals. Literals are especially problematic
|
|
|
|
* for two reasons:
|
|
|
|
* 1) confusing different literal columns leads to weird side
|
|
|
|
* effects (see 01101_literal_columns_clash);
|
|
|
|
* 2) the disambiguation mechanism in SyntaxAnalyzer, that, among
|
|
|
|
* other things, creates unique aliases for columns with same
|
|
|
|
* names from different tables, is applied before these temporary
|
|
|
|
* columns are created by ExpressionAnalyzer.
|
|
|
|
* Similar problems should also manifest for function columns, which
|
|
|
|
* are likewise created at a later stage by EA.
|
|
|
|
* In general, we need to have explicit separation between display
|
|
|
|
* names and identifiers for columns. This code is a workaround for
|
|
|
|
* a particular subclass of problems, and not a proper solution.
|
|
|
|
*/
|
2020-04-22 06:01:33 +00:00
|
|
|
if (const auto * as_literal = ast->as<ASTLiteral>())
|
2020-04-01 12:16:39 +00:00
|
|
|
{
|
|
|
|
source_name = as_literal->unique_column_name;
|
|
|
|
assert(!source_name.empty());
|
|
|
|
}
|
|
|
|
|
|
|
|
result_columns.emplace_back(source_name, result_name);
|
2018-02-26 09:05:06 +00:00
|
|
|
step.required_output.push_back(result_columns.back().second);
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2020-09-13 13:51:31 +00:00
|
|
|
auto actions = chain.getLastActions();
|
2020-11-03 11:28:28 +00:00
|
|
|
actions->project(result_columns);
|
2020-09-13 13:51:31 +00:00
|
|
|
return actions;
|
2013-05-24 10:49:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-09-04 13:45:39 +00:00
|
|
|
void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const ASTPtr & expr, bool only_types)
|
2018-09-03 13:36:58 +00:00
|
|
|
{
|
2020-06-25 23:05:47 +00:00
|
|
|
ExpressionActionsChain::Step & step = chain.lastStep(sourceColumns());
|
2020-08-13 20:17:18 +00:00
|
|
|
getRootActions(expr, only_types, step.actions());
|
2018-09-03 13:36:58 +00:00
|
|
|
step.required_output.push_back(expr->getColumnName());
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool project_result)
|
2013-05-24 10:49:19 +00:00
|
|
|
{
|
2020-09-11 12:24:41 +00:00
|
|
|
auto actions_dag = std::make_shared<ActionsDAG>(aggregated_columns);
|
2017-04-01 07:20:54 +00:00
|
|
|
NamesWithAliases result_columns;
|
|
|
|
Names result_names;
|
|
|
|
|
|
|
|
ASTs asts;
|
|
|
|
|
2019-03-11 13:22:51 +00:00
|
|
|
if (const auto * node = query->as<ASTExpressionList>())
|
2017-04-01 07:20:54 +00:00
|
|
|
asts = node->children;
|
|
|
|
else
|
2018-08-27 17:58:43 +00:00
|
|
|
asts = ASTs(1, query);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-03-08 23:48:08 +00:00
|
|
|
for (const auto & ast : asts)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-03-08 23:48:08 +00:00
|
|
|
std::string name = ast->getColumnName();
|
2017-04-01 07:20:54 +00:00
|
|
|
std::string alias;
|
2018-09-03 17:24:46 +00:00
|
|
|
if (add_aliases)
|
2020-03-08 23:48:08 +00:00
|
|
|
alias = ast->getAliasOrColumnName();
|
2017-04-01 07:20:54 +00:00
|
|
|
else
|
|
|
|
alias = name;
|
|
|
|
result_columns.emplace_back(name, alias);
|
|
|
|
result_names.push_back(alias);
|
2020-09-11 12:24:41 +00:00
|
|
|
getRootActions(ast, false, actions_dag);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2018-09-03 17:24:46 +00:00
|
|
|
if (add_aliases)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-09-03 17:24:46 +00:00
|
|
|
if (project_result)
|
2020-11-03 11:28:28 +00:00
|
|
|
actions_dag->project(result_columns);
|
2018-09-03 17:24:46 +00:00
|
|
|
else
|
2020-11-03 11:28:28 +00:00
|
|
|
actions_dag->addAliases(result_columns);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2018-09-05 13:04:28 +00:00
|
|
|
|
|
|
|
if (!(add_aliases && project_result))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-04-02 17:37:49 +00:00
|
|
|
/// We will not delete the original columns.
|
2019-08-09 14:50:04 +00:00
|
|
|
for (const auto & column_name_type : sourceColumns())
|
2017-04-01 07:20:54 +00:00
|
|
|
result_names.push_back(column_name_type.name);
|
|
|
|
}
|
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
actions_dag->removeUnusedActions(result_names);
|
|
|
|
return actions_dag;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool project_result)
|
|
|
|
{
|
2020-11-10 16:27:55 +00:00
|
|
|
return std::make_shared<ExpressionActions>(getActionsDAG(add_aliases, project_result));
|
2013-05-24 10:49:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
ExpressionActionsPtr ExpressionAnalyzer::getConstActions()
|
|
|
|
{
|
2020-09-11 12:24:41 +00:00
|
|
|
auto actions = std::make_shared<ActionsDAG>(NamesAndTypesList());
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2018-10-16 12:34:20 +00:00
|
|
|
getRootActions(query, true, actions, true);
|
2020-11-10 16:27:55 +00:00
|
|
|
return std::make_shared<ExpressionActions>(actions);
|
2013-05-24 10:49:19 +00:00
|
|
|
}
|
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
ActionsDAGPtr SelectQueryExpressionAnalyzer::simpleSelectActions()
|
2020-02-10 15:50:12 +00:00
|
|
|
{
|
|
|
|
ExpressionActionsChain new_chain(context);
|
|
|
|
appendSelect(new_chain, false);
|
|
|
|
return new_chain.getLastActions();
|
|
|
|
}
|
|
|
|
|
2020-02-10 19:55:13 +00:00
|
|
|
ExpressionAnalysisResult::ExpressionAnalysisResult(
|
2020-02-10 15:50:12 +00:00
|
|
|
SelectQueryExpressionAnalyzer & query_analyzer,
|
2020-06-17 11:05:11 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
2020-02-10 15:50:12 +00:00
|
|
|
bool first_stage_,
|
|
|
|
bool second_stage_,
|
|
|
|
bool only_types,
|
|
|
|
const FilterInfoPtr & filter_info_,
|
|
|
|
const Block & source_header)
|
|
|
|
: first_stage(first_stage_)
|
|
|
|
, second_stage(second_stage_)
|
2020-02-10 19:55:13 +00:00
|
|
|
, need_aggregate(query_analyzer.hasAggregation())
|
2020-12-09 11:14:40 +00:00
|
|
|
, has_window(query_analyzer.hasWindow())
|
2020-02-10 15:50:12 +00:00
|
|
|
{
|
|
|
|
/// first_stage: Do I need to perform the first part of the pipeline - running on remote servers during distributed processing.
|
|
|
|
/// second_stage: Do I need to execute the second part of the pipeline - running on the initiating server during distributed processing.
|
|
|
|
|
|
|
|
/** First we compose a chain of actions and remember the necessary steps from it.
|
|
|
|
* Regardless of from_stage and to_stage, we will compose a complete sequence of actions to perform optimization and
|
|
|
|
* throw out unnecessary columns based on the entire query. In unnecessary parts of the query, we will not execute subqueries.
|
|
|
|
*/
|
|
|
|
|
2020-02-10 19:55:13 +00:00
|
|
|
const ASTSelectQuery & query = *query_analyzer.getSelectQuery();
|
|
|
|
const Context & context = query_analyzer.context;
|
|
|
|
const Settings & settings = context.getSettingsRef();
|
2020-03-24 18:06:55 +00:00
|
|
|
const ConstStoragePtr & storage = query_analyzer.storage();
|
2020-02-10 19:55:13 +00:00
|
|
|
|
2020-02-10 15:50:12 +00:00
|
|
|
bool finalized = false;
|
|
|
|
size_t where_step_num = 0;
|
|
|
|
|
2020-03-23 02:12:31 +00:00
|
|
|
auto finalize_chain = [&](ExpressionActionsChain & chain)
|
2020-02-10 15:50:12 +00:00
|
|
|
{
|
2020-04-16 18:03:27 +00:00
|
|
|
chain.finalize();
|
|
|
|
|
2020-02-10 15:50:12 +00:00
|
|
|
if (!finalized)
|
|
|
|
{
|
2020-11-03 11:28:28 +00:00
|
|
|
finalize(chain, where_step_num);
|
2020-04-16 18:03:27 +00:00
|
|
|
finalized = true;
|
2020-02-10 15:50:12 +00:00
|
|
|
}
|
2020-04-16 18:03:27 +00:00
|
|
|
|
|
|
|
chain.clear();
|
2020-02-10 15:50:12 +00:00
|
|
|
};
|
|
|
|
|
2020-05-20 09:29:23 +00:00
|
|
|
if (storage)
|
|
|
|
{
|
|
|
|
query_analyzer.makeSetsForIndex(query.where());
|
|
|
|
query_analyzer.makeSetsForIndex(query.prewhere());
|
|
|
|
}
|
|
|
|
|
2020-02-10 15:50:12 +00:00
|
|
|
{
|
|
|
|
ExpressionActionsChain chain(context);
|
|
|
|
Names additional_required_columns_after_prewhere;
|
|
|
|
|
2020-03-23 02:12:31 +00:00
|
|
|
if (storage && (query.sampleSize() || settings.parallel_replicas_count > 1))
|
2020-02-10 15:50:12 +00:00
|
|
|
{
|
2020-06-17 12:07:09 +00:00
|
|
|
Names columns_for_sampling = metadata_snapshot->getColumnsRequiredForSampling();
|
2020-02-10 15:50:12 +00:00
|
|
|
additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(),
|
|
|
|
columns_for_sampling.begin(), columns_for_sampling.end());
|
|
|
|
}
|
|
|
|
|
|
|
|
if (storage && query.final())
|
|
|
|
{
|
2020-06-17 11:05:11 +00:00
|
|
|
Names columns_for_final = metadata_snapshot->getColumnsRequiredForFinal();
|
2020-02-10 15:50:12 +00:00
|
|
|
additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(),
|
|
|
|
columns_for_final.begin(), columns_for_final.end());
|
|
|
|
}
|
|
|
|
|
|
|
|
if (storage && filter_info_)
|
|
|
|
{
|
|
|
|
filter_info = filter_info_;
|
2020-11-11 16:52:27 +00:00
|
|
|
query_analyzer.appendPreliminaryFilter(chain, filter_info->actions_dag, filter_info->column_name);
|
2020-02-10 15:50:12 +00:00
|
|
|
}
|
|
|
|
|
2020-09-11 12:24:41 +00:00
|
|
|
if (auto actions = query_analyzer.appendPrewhere(chain, !first_stage, additional_required_columns_after_prewhere))
|
2020-02-10 15:50:12 +00:00
|
|
|
{
|
2020-11-03 19:05:47 +00:00
|
|
|
prewhere_info = std::make_shared<PrewhereDAGInfo>(actions, query.prewhere()->getColumnName());
|
2020-02-10 15:50:12 +00:00
|
|
|
|
2020-02-10 19:55:13 +00:00
|
|
|
if (allowEarlyConstantFolding(*prewhere_info->prewhere_actions, settings))
|
2020-02-10 15:50:12 +00:00
|
|
|
{
|
|
|
|
Block before_prewhere_sample = source_header;
|
|
|
|
if (sanitizeBlock(before_prewhere_sample))
|
|
|
|
{
|
2020-11-11 17:17:26 +00:00
|
|
|
ExpressionActions(prewhere_info->prewhere_actions).execute(before_prewhere_sample);
|
2020-02-10 15:50:12 +00:00
|
|
|
auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName());
|
|
|
|
/// If the filter column is a constant, record it.
|
|
|
|
if (column_elem.column)
|
|
|
|
prewhere_constant_filter_description = ConstantFilterDescription(*column_elem.column);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-08-12 08:55:16 +00:00
|
|
|
array_join = query_analyzer.appendArrayJoin(chain, before_array_join, only_types || !first_stage);
|
2020-02-10 15:50:12 +00:00
|
|
|
|
2020-06-25 20:59:10 +00:00
|
|
|
if (query_analyzer.hasTableJoin())
|
2020-02-10 15:50:12 +00:00
|
|
|
{
|
2021-02-09 13:17:42 +00:00
|
|
|
query_analyzer.appendJoinLeftKeys(chain, only_types || !first_stage);
|
2020-09-08 10:40:53 +00:00
|
|
|
before_join = chain.getLastActions();
|
2021-02-09 13:17:42 +00:00
|
|
|
join = query_analyzer.appendJoin(chain, converting_join_columns);
|
2020-02-10 15:50:12 +00:00
|
|
|
chain.addStep();
|
|
|
|
}
|
|
|
|
|
|
|
|
if (query_analyzer.appendWhere(chain, only_types || !first_stage))
|
|
|
|
{
|
|
|
|
where_step_num = chain.steps.size() - 1;
|
|
|
|
before_where = chain.getLastActions();
|
2020-02-10 19:55:13 +00:00
|
|
|
if (allowEarlyConstantFolding(*before_where, settings))
|
2020-02-10 15:50:12 +00:00
|
|
|
{
|
|
|
|
Block before_where_sample;
|
|
|
|
if (chain.steps.size() > 1)
|
2020-08-19 19:33:49 +00:00
|
|
|
before_where_sample = Block(chain.steps[chain.steps.size() - 2]->getResultColumns());
|
2020-02-10 15:50:12 +00:00
|
|
|
else
|
|
|
|
before_where_sample = source_header;
|
|
|
|
if (sanitizeBlock(before_where_sample))
|
|
|
|
{
|
2020-11-11 17:17:26 +00:00
|
|
|
ExpressionActions(before_where).execute(before_where_sample);
|
2020-02-10 15:50:12 +00:00
|
|
|
auto & column_elem = before_where_sample.getByName(query.where()->getColumnName());
|
|
|
|
/// If the filter column is a constant, record it.
|
|
|
|
if (column_elem.column)
|
|
|
|
where_constant_filter_description = ConstantFilterDescription(*column_elem.column);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
chain.addStep();
|
|
|
|
}
|
|
|
|
|
|
|
|
if (need_aggregate)
|
|
|
|
{
|
2020-02-16 19:46:45 +00:00
|
|
|
/// TODO correct conditions
|
|
|
|
optimize_aggregation_in_order =
|
|
|
|
context.getSettingsRef().optimize_aggregation_in_order
|
|
|
|
&& storage && query.groupBy();
|
|
|
|
|
|
|
|
query_analyzer.appendGroupBy(chain, only_types || !first_stage, optimize_aggregation_in_order, group_by_elements_actions);
|
2020-02-10 15:50:12 +00:00
|
|
|
query_analyzer.appendAggregateFunctionsArguments(chain, only_types || !first_stage);
|
|
|
|
before_aggregation = chain.getLastActions();
|
|
|
|
|
2020-03-23 02:12:31 +00:00
|
|
|
finalize_chain(chain);
|
2020-02-10 15:50:12 +00:00
|
|
|
|
|
|
|
if (query_analyzer.appendHaving(chain, only_types || !second_stage))
|
|
|
|
{
|
|
|
|
before_having = chain.getLastActions();
|
|
|
|
chain.addStep();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-06-16 20:13:18 +00:00
|
|
|
bool join_allow_read_in_order = true;
|
2020-06-25 17:40:25 +00:00
|
|
|
if (hasJoin())
|
2020-06-16 20:13:18 +00:00
|
|
|
{
|
|
|
|
/// You may find it strange but we support read_in_order for HashJoin and do not support for MergeJoin.
|
2020-09-18 16:25:20 +00:00
|
|
|
join_has_delayed_stream = query_analyzer.analyzedJoin().needStreamWithNonJoinedRows();
|
|
|
|
join_allow_read_in_order = typeid_cast<HashJoin *>(join.get()) && !join_has_delayed_stream;
|
2020-06-16 20:13:18 +00:00
|
|
|
}
|
|
|
|
|
2020-02-10 15:50:12 +00:00
|
|
|
optimize_read_in_order =
|
2020-02-10 19:55:13 +00:00
|
|
|
settings.optimize_read_in_order
|
2020-02-10 15:50:12 +00:00
|
|
|
&& storage && query.orderBy()
|
|
|
|
&& !query_analyzer.hasAggregation()
|
|
|
|
&& !query.final()
|
2020-06-16 20:13:18 +00:00
|
|
|
&& join_allow_read_in_order;
|
2020-02-10 15:50:12 +00:00
|
|
|
|
|
|
|
/// If there is aggregation, we execute expressions in SELECT and ORDER BY on the initiating server, otherwise on the source servers.
|
|
|
|
query_analyzer.appendSelect(chain, only_types || (need_aggregate ? !second_stage : !first_stage));
|
2020-12-09 11:14:40 +00:00
|
|
|
|
2020-12-25 03:13:30 +00:00
|
|
|
// Window functions are processed in a separate expression chain after
|
|
|
|
// the main SELECT, similar to what we do for aggregate functions.
|
2020-12-24 04:03:33 +00:00
|
|
|
if (has_window)
|
|
|
|
{
|
2021-01-13 19:29:52 +00:00
|
|
|
query_analyzer.makeWindowDescriptions(chain.getLastActions());
|
|
|
|
|
2020-12-24 04:03:33 +00:00
|
|
|
query_analyzer.appendWindowFunctionsArguments(chain, only_types || !first_stage);
|
|
|
|
|
2020-12-28 09:56:38 +00:00
|
|
|
// Build a list of output columns of the window step.
|
|
|
|
// 1) We need the columns that are the output of ExpressionActions.
|
2020-12-24 04:03:33 +00:00
|
|
|
for (const auto & x : chain.getLastActions()->getNamesAndTypesList())
|
|
|
|
{
|
|
|
|
query_analyzer.columns_after_window.push_back(x);
|
|
|
|
}
|
2020-12-28 09:56:38 +00:00
|
|
|
// 2) We also have to manually add the output of the window function
|
|
|
|
// to the list of the output columns of the window step, because the
|
|
|
|
// window functions are not in the ExpressionActions.
|
|
|
|
for (const auto & [_, w] : query_analyzer.window_descriptions)
|
|
|
|
{
|
|
|
|
for (const auto & f : w.window_functions)
|
|
|
|
{
|
|
|
|
query_analyzer.columns_after_window.push_back(
|
|
|
|
{f.column_name, f.aggregate_function->getReturnType()});
|
|
|
|
}
|
|
|
|
}
|
2020-12-24 04:03:33 +00:00
|
|
|
|
2020-12-25 03:13:30 +00:00
|
|
|
before_window = chain.getLastActions();
|
2020-12-24 04:03:33 +00:00
|
|
|
finalize_chain(chain);
|
|
|
|
|
|
|
|
auto & step = chain.lastStep(query_analyzer.columns_after_window);
|
2020-12-25 03:13:30 +00:00
|
|
|
|
|
|
|
// The output of this expression chain is the result of
|
|
|
|
// SELECT (before "final projection" i.e. renaming the columns), so
|
|
|
|
// we have to mark the expressions that are required in the output,
|
|
|
|
// again. We did it for the previous expression chain ("select w/o
|
|
|
|
// window functions") earlier, in appendSelect(). But that chain also
|
|
|
|
// produced the expressions required to calculate window functions.
|
|
|
|
// They are not needed in the final SELECT result. Knowing the correct
|
|
|
|
// list of columns is important when we apply SELECT DISTINCT later.
|
2020-12-24 04:03:33 +00:00
|
|
|
const auto * select_query = query_analyzer.getSelectQuery();
|
|
|
|
for (const auto & child : select_query->select()->children)
|
|
|
|
{
|
2020-12-25 03:13:30 +00:00
|
|
|
step.required_output.push_back(child->getColumnName());
|
2020-12-24 04:03:33 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
selected_columns = chain.getLastStep().required_output;
|
|
|
|
|
2020-12-25 03:13:30 +00:00
|
|
|
has_order_by = query.orderBy() != nullptr;
|
|
|
|
before_order_by = query_analyzer.appendOrderBy(
|
|
|
|
chain,
|
|
|
|
only_types || (need_aggregate ? !second_stage : !first_stage),
|
|
|
|
optimize_read_in_order,
|
|
|
|
order_by_elements_actions);
|
|
|
|
|
2020-02-10 15:50:12 +00:00
|
|
|
if (query_analyzer.appendLimitBy(chain, only_types || !second_stage))
|
|
|
|
{
|
|
|
|
before_limit_by = chain.getLastActions();
|
|
|
|
chain.addStep();
|
|
|
|
}
|
|
|
|
|
2020-09-13 13:51:31 +00:00
|
|
|
final_projection = query_analyzer.appendProjectResult(chain);
|
2020-02-10 15:50:12 +00:00
|
|
|
|
2020-03-23 02:12:31 +00:00
|
|
|
finalize_chain(chain);
|
2020-02-10 15:50:12 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// Before executing WHERE and HAVING, remove the extra columns from the block (mostly the aggregation keys).
|
|
|
|
removeExtraColumns();
|
|
|
|
|
|
|
|
checkActions();
|
|
|
|
}
|
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
void ExpressionAnalysisResult::finalize(const ExpressionActionsChain & chain, size_t where_step_num)
|
2020-02-10 15:50:12 +00:00
|
|
|
{
|
|
|
|
if (hasPrewhere())
|
|
|
|
{
|
2020-08-19 19:33:49 +00:00
|
|
|
const ExpressionActionsChain::Step & step = *chain.steps.at(0);
|
2020-02-10 15:50:12 +00:00
|
|
|
prewhere_info->remove_prewhere_column = step.can_remove_required_output.at(0);
|
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
NameSet columns_to_remove;
|
2020-02-10 15:50:12 +00:00
|
|
|
for (size_t i = 1; i < step.required_output.size(); ++i)
|
|
|
|
{
|
|
|
|
if (step.can_remove_required_output[i])
|
2020-11-03 11:28:28 +00:00
|
|
|
columns_to_remove.insert(step.required_output[i]);
|
2020-02-10 15:50:12 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
columns_to_remove_after_prewhere = std::move(columns_to_remove);
|
|
|
|
}
|
|
|
|
else if (hasFilter())
|
|
|
|
{
|
|
|
|
/// Can't have prewhere and filter set simultaneously
|
2020-08-19 19:33:49 +00:00
|
|
|
filter_info->do_remove_column = chain.steps.at(0)->can_remove_required_output.at(0);
|
2020-02-10 15:50:12 +00:00
|
|
|
}
|
|
|
|
if (hasWhere())
|
2020-08-19 19:33:49 +00:00
|
|
|
remove_where_filter = chain.steps.at(where_step_num)->can_remove_required_output.at(0);
|
2020-02-10 15:50:12 +00:00
|
|
|
}
|
|
|
|
|
2020-04-22 06:01:33 +00:00
|
|
|
void ExpressionAnalysisResult::removeExtraColumns() const
|
2020-02-10 15:50:12 +00:00
|
|
|
{
|
|
|
|
if (hasFilter())
|
2020-11-11 16:52:27 +00:00
|
|
|
filter_info->actions_dag->projectInput();
|
2020-02-10 15:50:12 +00:00
|
|
|
if (hasWhere())
|
2020-11-03 11:28:28 +00:00
|
|
|
before_where->projectInput();
|
2020-02-10 15:50:12 +00:00
|
|
|
if (hasHaving())
|
2020-11-03 11:28:28 +00:00
|
|
|
before_having->projectInput();
|
2020-02-10 15:50:12 +00:00
|
|
|
}
|
|
|
|
|
2020-04-22 06:01:33 +00:00
|
|
|
void ExpressionAnalysisResult::checkActions() const
|
2020-02-10 15:50:12 +00:00
|
|
|
{
|
|
|
|
/// Check that PREWHERE doesn't contain unusual actions. Unusual actions are that can change number of rows.
|
|
|
|
if (hasPrewhere())
|
|
|
|
{
|
2020-11-03 11:28:28 +00:00
|
|
|
auto check_actions = [](const ActionsDAGPtr & actions)
|
2020-02-10 15:50:12 +00:00
|
|
|
{
|
|
|
|
if (actions)
|
2020-11-03 11:28:28 +00:00
|
|
|
for (const auto & node : actions->getNodes())
|
2020-11-10 14:54:59 +00:00
|
|
|
if (node.type == ActionsDAG::ActionType::ARRAY_JOIN)
|
2020-09-08 10:40:53 +00:00
|
|
|
throw Exception("PREWHERE cannot contain ARRAY JOIN action", ErrorCodes::ILLEGAL_PREWHERE);
|
2020-02-10 15:50:12 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
check_actions(prewhere_info->prewhere_actions);
|
|
|
|
check_actions(prewhere_info->alias_actions);
|
|
|
|
check_actions(prewhere_info->remove_columns_actions);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-12-02 18:16:31 +00:00
|
|
|
std::string ExpressionAnalysisResult::dump() const
|
|
|
|
{
|
2020-12-18 17:13:28 +00:00
|
|
|
WriteBufferFromOwnString ss;
|
2020-12-02 18:16:31 +00:00
|
|
|
|
|
|
|
ss << "need_aggregate " << need_aggregate << "\n";
|
|
|
|
ss << "has_order_by " << has_order_by << "\n";
|
2020-12-09 11:14:40 +00:00
|
|
|
ss << "has_window " << has_window << "\n";
|
2020-12-02 18:16:31 +00:00
|
|
|
|
|
|
|
if (before_array_join)
|
|
|
|
{
|
|
|
|
ss << "before_array_join " << before_array_join->dumpDAG() << "\n";
|
|
|
|
}
|
|
|
|
|
|
|
|
if (array_join)
|
|
|
|
{
|
|
|
|
ss << "array_join " << "FIXME doesn't have dump" << "\n";
|
|
|
|
}
|
|
|
|
|
|
|
|
if (before_join)
|
|
|
|
{
|
|
|
|
ss << "before_join " << before_join->dumpDAG() << "\n";
|
|
|
|
}
|
|
|
|
|
|
|
|
if (before_where)
|
|
|
|
{
|
|
|
|
ss << "before_where " << before_where->dumpDAG() << "\n";
|
|
|
|
}
|
|
|
|
|
|
|
|
if (prewhere_info)
|
|
|
|
{
|
|
|
|
ss << "prewhere_info " << prewhere_info->dump() << "\n";
|
|
|
|
}
|
|
|
|
|
|
|
|
if (filter_info)
|
|
|
|
{
|
|
|
|
ss << "filter_info " << filter_info->dump() << "\n";
|
|
|
|
}
|
|
|
|
|
|
|
|
if (before_aggregation)
|
|
|
|
{
|
|
|
|
ss << "before_aggregation " << before_aggregation->dumpDAG() << "\n";
|
|
|
|
}
|
|
|
|
|
|
|
|
if (before_having)
|
|
|
|
{
|
|
|
|
ss << "before_having " << before_having->dumpDAG() << "\n";
|
|
|
|
}
|
|
|
|
|
2020-12-09 11:14:40 +00:00
|
|
|
if (before_window)
|
|
|
|
{
|
|
|
|
ss << "before_window " << before_window->dumpDAG() << "\n";
|
|
|
|
}
|
|
|
|
|
2020-12-25 03:13:30 +00:00
|
|
|
if (before_order_by)
|
2020-12-02 18:16:31 +00:00
|
|
|
{
|
2020-12-25 03:13:30 +00:00
|
|
|
ss << "before_order_by " << before_order_by->dumpDAG() << "\n";
|
2020-12-02 18:16:31 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
if (before_limit_by)
|
|
|
|
{
|
|
|
|
ss << "before_limit_by " << before_limit_by->dumpDAG() << "\n";
|
|
|
|
}
|
|
|
|
|
|
|
|
if (final_projection)
|
|
|
|
{
|
|
|
|
ss << "final_projection " << final_projection->dumpDAG() << "\n";
|
|
|
|
}
|
|
|
|
|
2020-12-25 03:13:30 +00:00
|
|
|
if (!selected_columns.empty())
|
|
|
|
{
|
|
|
|
ss << "selected_columns ";
|
|
|
|
for (size_t i = 0; i < selected_columns.size(); i++)
|
|
|
|
{
|
|
|
|
if (i > 0)
|
|
|
|
{
|
|
|
|
ss << ", ";
|
|
|
|
}
|
|
|
|
ss << backQuote(selected_columns[i]);
|
|
|
|
}
|
|
|
|
ss << "\n";
|
|
|
|
}
|
|
|
|
|
2020-12-02 18:16:31 +00:00
|
|
|
return ss.str();
|
|
|
|
}
|
|
|
|
|
2013-05-24 10:49:19 +00:00
|
|
|
}
|