ClickHouse/src/Interpreters/ExpressionAnalyzer.h

405 lines
16 KiB
C++
Raw Normal View History

#pragma once
2022-05-13 14:55:50 +00:00
#include <Core/ColumnNumbers.h>
#include <Columns/FilterDescription.h>
#include <Interpreters/ActionsVisitor.h>
#include <Interpreters/AggregateDescription.h>
2021-01-31 19:25:47 +00:00
#include <Interpreters/DatabaseCatalog.h>
2019-08-14 16:53:30 +00:00
#include <Interpreters/SubqueryForSet.h>
2021-01-31 19:25:47 +00:00
#include <Interpreters/TreeRewriter.h>
#include <Interpreters/WindowDescription.h>
2021-01-31 19:25:47 +00:00
#include <Interpreters/join_common.h>
2019-03-11 14:01:45 +00:00
#include <Parsers/IAST_fwd.h>
#include <Storages/IStorage_fwd.h>
#include <Storages/SelectQueryInfo.h>
2019-03-11 14:01:45 +00:00
namespace DB
{
class Block;
struct Settings;
2016-01-13 00:32:59 +00:00
struct ExpressionActionsChain;
class ExpressionActions;
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
2019-12-10 23:18:24 +00:00
using ManyExpressionActions = std::vector<ExpressionActionsPtr>;
2016-01-13 00:32:59 +00:00
struct ASTTableJoin;
class IJoin;
using JoinPtr = std::shared_ptr<IJoin>;
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
class ASTFunction;
class ASTExpressionList;
class ASTSelectQuery;
struct ASTTablesInSelectQueryElement;
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
struct StorageInMemoryMetadata;
2020-06-18 11:02:31 +00:00
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
class ArrayJoinAction;
using ArrayJoinActionPtr = std::shared_ptr<ArrayJoinAction>;
2020-09-10 16:01:41 +00:00
class ActionsDAG;
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
/// Create columns in block or return false if not possible
2020-06-20 13:48:21 +00:00
bool sanitizeBlock(Block & block, bool throw_if_cannot_create_column = false);
/// ExpressionAnalyzer sources, intermediates and results. It splits data and logic, allows to test them separately.
struct ExpressionAnalyzerData
2018-04-24 07:16:39 +00:00
{
2021-04-28 17:32:12 +00:00
~ExpressionAnalyzerData();
SubqueriesForSets subqueries_for_sets;
PreparedSets prepared_sets;
2021-04-27 14:00:16 +00:00
std::unique_ptr<QueryPlan> joined_plan;
2020-12-04 09:24:42 +00:00
/// Columns after ARRAY JOIN. If there is no ARRAY JOIN, it's source_columns.
NamesAndTypesList columns_after_array_join;
/// Columns after Columns after ARRAY JOIN and JOIN. If there is no JOIN, it's columns_after_array_join.
NamesAndTypesList columns_after_join;
/// Columns after ARRAY JOIN, JOIN, and/or aggregation.
NamesAndTypesList aggregated_columns;
2020-12-24 04:03:33 +00:00
/// Columns after window functions.
NamesAndTypesList columns_after_window;
/// Keys of ORDER BY
NameSet order_by_keys;
bool has_aggregation = false;
NamesAndTypesList aggregation_keys;
2021-09-06 02:18:42 +00:00
NamesAndTypesLists aggregation_keys_list;
2022-05-13 14:55:50 +00:00
ColumnNumbersList aggregation_keys_indexes_list;
2021-07-03 07:45:37 +00:00
bool has_const_aggregation_keys = false;
AggregateDescriptions aggregate_descriptions;
2020-12-09 11:14:40 +00:00
WindowDescriptions window_descriptions;
NamesAndTypesList window_columns;
bool has_global_subqueries = false;
/// All new temporary tables obtained by performing the GLOBAL IN/JOIN subqueries.
2020-03-10 19:36:17 +00:00
TemporaryTablesMapping external_tables;
2022-05-13 23:04:12 +00:00
GroupByKind group_by_kind = GroupByKind::NONE;
};
2018-04-24 07:16:39 +00:00
2017-06-02 21:37:28 +00:00
/** Transforms an expression from a syntax tree into a sequence of actions to execute it.
*
2017-06-02 21:37:28 +00:00
* NOTE: if `ast` is a SELECT query from a table, the structure of this table should not change during the lifetime of ExpressionAnalyzer.
*/
class ExpressionAnalyzer : protected ExpressionAnalyzerData, private boost::noncopyable, protected WithContext
{
2016-01-13 00:32:59 +00:00
private:
/// Extracts settings to enlight which are used (and avoid copy of others).
struct ExtractedSettings
{
const bool use_index_for_in_with_subqueries;
const SizeLimits size_limits_for_set;
2021-07-30 12:19:39 +00:00
const UInt64 distributed_group_by_no_merge;
explicit ExtractedSettings(const Settings & settings_);
};
2016-01-13 00:32:59 +00:00
public:
/// Ctor for non-select queries. Generally its usage is:
/// auto actions = ExpressionAnalyzer(query, syntax, context).getActions();
ExpressionAnalyzer(const ASTPtr & query_, const TreeRewriterResultPtr & syntax_analyzer_result_, ContextPtr context_)
: ExpressionAnalyzer(query_, syntax_analyzer_result_, context_, 0, false, false, {}, {})
{
}
2021-04-29 14:14:23 +00:00
~ExpressionAnalyzer();
void appendExpression(ExpressionActionsChain & chain, const ASTPtr & expr, bool only_types);
2017-06-02 21:37:28 +00:00
/// If `ast` is not a SELECT query, just gets all the actions to evaluate the expression.
/// If add_aliases, only the calculated values in the desired order and add aliases.
/// If also project_result, than only aliases remain in the output block.
2017-06-02 21:37:28 +00:00
/// Otherwise, only temporary columns will be deleted from the block.
2020-11-03 11:28:28 +00:00
ActionsDAGPtr getActionsDAG(bool add_aliases, bool project_result = true);
ExpressionActionsPtr getActions(bool add_aliases, bool project_result = true, CompileExpressions compile_expressions = CompileExpressions::no);
2017-06-02 21:37:28 +00:00
/// Actions that can be performed on an empty block: adding constants and applying functions that depend only on constants.
/// Does not execute subqueries.
2021-03-04 08:53:31 +00:00
ExpressionActionsPtr getConstActions(const ColumnsWithTypeAndName & constant_inputs = {});
2017-06-02 21:37:28 +00:00
/** Sets that require a subquery to be create.
* Only the sets needed to perform actions returned from already executed `append*` or `getActions`.
* That is, you need to call getSetsWithSubqueries after all calls of `append*` or `getActions`
* and create all the returned sets before performing the actions.
*/
SubqueriesForSets & getSubqueriesForSets() { return subqueries_for_sets; }
PreparedSets & getPreparedSets() { return prepared_sets; }
/// Get intermediates for tests
const ExpressionAnalyzerData & getAnalyzedData() const { return *this; }
2020-12-22 19:01:15 +00:00
/// A list of windows for window functions.
const WindowDescriptions & windowDescriptions() const { return window_descriptions; }
2021-01-13 19:29:52 +00:00
void makeWindowDescriptions(ActionsDAGPtr actions);
2021-03-03 08:36:20 +00:00
/**
* Create Set from a subquery or a table expression in the query. The created set is suitable for using the index.
* The set will not be created if its size hits the limit.
*/
void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options = {});
/**
* Checks if subquery is not a plain StorageSet.
* Because while making set we will read data from StorageSet which is not allowed.
* Returns valid SetPtr from StorageSet if the latter is used after IN or nullptr otherwise.
*/
SetPtr isPlainStorageSetInSubquery(const ASTPtr & subquery_or_table_name);
protected:
ExpressionAnalyzer(
const ASTPtr & query_,
const TreeRewriterResultPtr & syntax_analyzer_result_,
ContextPtr context_,
size_t subquery_depth_,
2020-10-13 14:55:22 +00:00
bool do_global_,
bool is_explain_,
SubqueriesForSets subqueries_for_sets_,
PreparedSets prepared_sets_);
2018-08-27 17:58:43 +00:00
ASTPtr query;
const ExtractedSettings settings;
size_t subquery_depth;
TreeRewriterResultPtr syntax;
const ConstStoragePtr & storage() const { return syntax->storage; } /// The main table in FROM clause, if exists.
const TableJoin & analyzedJoin() const { return *syntax->analyzed_join; }
const NamesAndTypesList & sourceColumns() const { return syntax->required_source_columns; }
const std::vector<const ASTFunction *> & aggregates() const { return syntax->aggregates; }
2017-06-02 21:37:28 +00:00
/// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables.
void initGlobalSubqueriesAndExternalTables(bool do_global, bool is_explain);
2020-09-10 16:01:41 +00:00
ArrayJoinActionPtr addMultipleArrayJoinAction(ActionsDAGPtr & actions, bool is_left) const;
void getRootActions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions, bool only_consts = false);
2020-01-27 21:57:44 +00:00
/** Similar to getRootActions but do not make sets when analyzing IN functions. It's used in
* analyzeAggregation which happens earlier than analyzing PREWHERE and WHERE. If we did, the
* prepared sets would not be applicable for MergeTree index optimization.
*/
void getRootActionsNoMakeSet(const ASTPtr & ast, ActionsDAGPtr & actions, bool only_consts = false);
2020-01-27 21:57:44 +00:00
void getRootActionsForHaving(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions, bool only_consts = false);
2020-01-27 21:57:44 +00:00
2017-06-02 21:37:28 +00:00
/** Add aggregation keys to aggregation_keys, aggregate functions to aggregate_descriptions,
* Create a set of columns aggregated_columns resulting after the aggregation, if any,
* or after all the actions that are normally performed before aggregation.
* Set has_aggregation = true if there is GROUP BY or at least one aggregate function.
*/
void analyzeAggregation(ActionsDAGPtr & temp_actions);
void makeAggregateDescriptions(ActionsDAGPtr & actions, AggregateDescriptions & descriptions);
2019-08-13 13:48:09 +00:00
const ASTSelectQuery * getSelectQuery() const;
bool isRemoteStorage() const { return syntax->is_remote_storage; }
NamesAndTypesList getColumnsAfterArrayJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns);
NamesAndTypesList analyzeJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns);
};
class SelectQueryExpressionAnalyzer;
/// Result of SelectQueryExpressionAnalyzer: expressions for InterpreterSelectQuery
struct ExpressionAnalysisResult
{
2020-12-02 18:16:31 +00:00
std::string dump() const;
/// Do I need to perform the first part of the pipeline - running on remote servers during distributed processing.
bool first_stage = false;
/// Do I need to execute the second part of the pipeline - running on the initiating server during distributed processing.
bool second_stage = false;
bool need_aggregate = false;
bool has_order_by = false;
2020-12-09 11:14:40 +00:00
bool has_window = false;
2021-05-04 12:40:34 +00:00
String where_column_name;
bool remove_where_filter = false;
bool optimize_read_in_order = false;
2020-02-16 19:46:45 +00:00
bool optimize_aggregation_in_order = false;
2020-09-18 16:25:20 +00:00
bool join_has_delayed_stream = false;
bool use_grouping_set_key = false;
2020-11-03 11:28:28 +00:00
ActionsDAGPtr before_array_join;
ArrayJoinActionPtr array_join;
2020-11-03 11:28:28 +00:00
ActionsDAGPtr before_join;
ActionsDAGPtr converting_join_columns;
2020-09-08 10:40:53 +00:00
JoinPtr join;
2020-11-03 11:28:28 +00:00
ActionsDAGPtr before_where;
ActionsDAGPtr before_aggregation;
ActionsDAGPtr before_having;
String having_column_name;
bool remove_having_filter = false;
2020-12-09 11:14:40 +00:00
ActionsDAGPtr before_window;
2020-12-25 03:13:30 +00:00
ActionsDAGPtr before_order_by;
2020-11-03 11:28:28 +00:00
ActionsDAGPtr before_limit_by;
ActionsDAGPtr final_projection;
2020-12-25 03:13:30 +00:00
/// Columns from the SELECT list, before renaming them to aliases. Used to
/// perform SELECT DISTINCT.
Names selected_columns;
2021-05-03 13:00:59 +00:00
/// Columns to read from storage if any.
Names required_columns;
/// Columns will be removed after prewhere actions execution.
2020-11-03 11:28:28 +00:00
NameSet columns_to_remove_after_prewhere;
2021-06-25 14:49:28 +00:00
PrewhereInfoPtr prewhere_info;
FilterDAGInfoPtr filter_info;
ConstantFilterDescription prewhere_constant_filter_description;
ConstantFilterDescription where_constant_filter_description;
/// Actions by every element of ORDER BY
ManyExpressionActions order_by_elements_actions;
2020-02-16 19:46:45 +00:00
ManyExpressionActions group_by_elements_actions;
ExpressionAnalysisResult() = default;
ExpressionAnalysisResult(
SelectQueryExpressionAnalyzer & query_analyzer,
2020-06-17 11:05:11 +00:00
const StorageMetadataPtr & metadata_snapshot,
bool first_stage,
bool second_stage,
bool only_types,
const FilterDAGInfoPtr & filter_info,
const Block & source_header);
/// Filter for row-level security.
bool hasFilter() const { return filter_info.get(); }
2020-06-25 17:40:25 +00:00
bool hasJoin() const { return join.get(); }
bool hasPrewhere() const { return prewhere_info.get(); }
bool hasWhere() const { return before_where.get(); }
bool hasHaving() const { return before_having.get(); }
bool hasLimitBy() const { return before_limit_by.get(); }
2020-04-22 06:01:33 +00:00
void removeExtraColumns() const;
void checkActions() const;
void finalize(
const ExpressionActionsChain & chain,
ssize_t & prewhere_step_num,
ssize_t & where_step_num,
ssize_t & having_step_num,
const ASTSelectQuery & query);
};
/// SelectQuery specific ExpressionAnalyzer part.
class SelectQueryExpressionAnalyzer : public ExpressionAnalyzer
{
public:
friend struct ExpressionAnalysisResult;
SelectQueryExpressionAnalyzer(
const ASTPtr & query_,
const TreeRewriterResultPtr & syntax_analyzer_result_,
ContextPtr context_,
const StorageMetadataPtr & metadata_snapshot_,
const NameSet & required_result_columns_ = {},
bool do_global_ = false,
2020-10-13 14:55:22 +00:00
const SelectQueryOptions & options_ = {},
SubqueriesForSets subqueries_for_sets_ = {},
PreparedSets prepared_sets_ = {})
: ExpressionAnalyzer(
query_,
syntax_analyzer_result_,
context_,
options_.subquery_depth,
do_global_,
options_.is_explain,
std::move(subqueries_for_sets_),
std::move(prepared_sets_))
, metadata_snapshot(metadata_snapshot_)
, required_result_columns(required_result_columns_)
, query_options(options_)
{
}
/// Does the expression have aggregate functions or a GROUP BY or HAVING section.
bool hasAggregation() const { return has_aggregation; }
2021-01-13 19:29:52 +00:00
bool hasWindow() const { return !syntax->window_function_asts.empty(); }
bool hasGlobalSubqueries() { return has_global_subqueries; }
2020-06-25 20:59:10 +00:00
bool hasTableJoin() const { return syntax->ast_join; }
bool useGroupingSetKey() const { return aggregation_keys_list.size() > 1; }
const NamesAndTypesList & aggregationKeys() const { return aggregation_keys; }
2021-07-03 07:45:37 +00:00
bool hasConstAggregationKeys() const { return has_const_aggregation_keys; }
2021-09-06 02:18:42 +00:00
const NamesAndTypesLists & aggregationKeysList() const { return aggregation_keys_list; }
const AggregateDescriptions & aggregates() const { return aggregate_descriptions; }
2021-04-30 10:18:01 +00:00
std::unique_ptr<QueryPlan> getJoinedPlan();
/// Tables that will need to be sent to remote servers for distributed query processing.
2020-03-10 19:36:17 +00:00
const TemporaryTablesMapping & getExternalTables() const { return external_tables; }
2020-11-03 11:28:28 +00:00
ActionsDAGPtr simpleSelectActions();
/// These appends are public only for tests
void appendSelect(ExpressionActionsChain & chain, bool only_types);
/// Deletes all columns except mentioned by SELECT, arranges the remaining columns and renames them to aliases.
2020-11-03 11:28:28 +00:00
ActionsDAGPtr appendProjectResult(ExpressionActionsChain & chain) const;
/// Create Set-s that we make from IN section to use index on them.
void makeSetsForIndex(const ASTPtr & node);
private:
StorageMetadataPtr metadata_snapshot;
/// If non-empty, ignore all expressions not from this list.
NameSet required_result_columns;
SelectQueryOptions query_options;
JoinPtr makeTableJoin(
const ASTTablesInSelectQueryElement & join_element,
const ColumnsWithTypeAndName & left_columns,
ActionsDAGPtr & left_convert_actions);
const ASTSelectQuery * getAggregatingQuery() const;
/** These methods allow you to build a chain of transformations over a block, that receives values in the desired sections of the query.
*
* Example usage:
* ExpressionActionsChain chain;
* analyzer.appendWhere(chain);
* chain.addStep();
* analyzer.appendSelect(chain);
* analyzer.appendOrderBy(chain);
* chain.finalize();
*
* If only_types = true set, does not execute subqueries in the relevant parts of the query. The actions got this way
* shouldn't be executed, they are only needed to get a list of columns with their types.
*/
/// Before aggregation:
2020-11-03 11:28:28 +00:00
ArrayJoinActionPtr appendArrayJoin(ExpressionActionsChain & chain, ActionsDAGPtr & before_array_join, bool only_types);
bool appendJoinLeftKeys(ExpressionActionsChain & chain, bool only_types);
JoinPtr appendJoin(ExpressionActionsChain & chain, ActionsDAGPtr & converting_join_columns);
/// remove_filter is set in ExpressionActionsChain::finalize();
/// Columns in `additional_required_columns` will not be removed (they can be used for e.g. sampling or FINAL modifier).
2020-11-03 11:28:28 +00:00
ActionsDAGPtr appendPrewhere(ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns);
bool appendWhere(ExpressionActionsChain & chain, bool only_types);
2020-02-16 19:46:45 +00:00
bool appendGroupBy(ExpressionActionsChain & chain, bool only_types, bool optimize_aggregation_in_order, ManyExpressionActions &);
void appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types);
2020-12-09 11:14:40 +00:00
void appendWindowFunctionsArguments(ExpressionActionsChain & chain, bool only_types);
/// After aggregation:
bool appendHaving(ExpressionActionsChain & chain, bool only_types);
/// appendSelect
2020-11-03 11:28:28 +00:00
ActionsDAGPtr appendOrderBy(ExpressionActionsChain & chain, bool only_types, bool optimize_read_in_order, ManyExpressionActions &);
bool appendLimitBy(ExpressionActionsChain & chain, bool only_types);
/// appendProjectResult
};
}