2013-05-24 10:49:19 +00:00
|
|
|
#pragma once
|
|
|
|
|
2019-05-17 14:34:25 +00:00
|
|
|
#include <Core/Settings.h>
|
|
|
|
#include <DataStreams/IBlockStream_fwd.h>
|
2020-02-10 15:50:12 +00:00
|
|
|
#include <Columns/FilterDescription.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/AggregateDescription.h>
|
2018-11-02 18:53:23 +00:00
|
|
|
#include <Interpreters/SyntaxAnalyzer.h>
|
2019-08-14 16:53:30 +00:00
|
|
|
#include <Interpreters/SubqueryForSet.h>
|
2019-03-11 14:01:45 +00:00
|
|
|
#include <Parsers/IAST_fwd.h>
|
2019-05-17 14:34:25 +00:00
|
|
|
#include <Storages/IStorage_fwd.h>
|
2020-02-10 15:50:12 +00:00
|
|
|
#include <Storages/SelectQueryInfo.h>
|
2020-03-10 19:36:17 +00:00
|
|
|
#include <Interpreters/DatabaseCatalog.h>
|
2019-03-11 14:01:45 +00:00
|
|
|
|
2013-05-24 10:49:19 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2018-10-18 15:03:14 +00:00
|
|
|
class Block;
|
2017-01-14 09:00:19 +00:00
|
|
|
class Context;
|
2014-07-06 19:48:39 +00:00
|
|
|
|
2016-01-13 00:32:59 +00:00
|
|
|
struct ExpressionActionsChain;
|
2018-10-18 15:03:14 +00:00
|
|
|
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
|
|
|
|
2018-10-18 15:03:14 +00:00
|
|
|
struct ASTTableJoin;
|
2019-09-16 12:37:46 +00:00
|
|
|
class IJoin;
|
|
|
|
using JoinPtr = std::shared_ptr<IJoin>;
|
2017-01-14 09:00:19 +00:00
|
|
|
|
|
|
|
class ASTFunction;
|
|
|
|
class ASTExpressionList;
|
|
|
|
class ASTSelectQuery;
|
2019-08-16 16:50:30 +00:00
|
|
|
struct ASTTablesInSelectQueryElement;
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2020-06-17 09:38:47 +00:00
|
|
|
struct StorageInMemoryMetadata;
|
|
|
|
using StorageMetadataPtr = std::shared_ptr<StorageInMemoryMetadata>;
|
|
|
|
|
2020-02-10 15:50:12 +00:00
|
|
|
/// Create columns in block or return false if not possible
|
|
|
|
bool sanitizeBlock(Block & block);
|
|
|
|
|
2018-11-12 15:40:33 +00:00
|
|
|
/// ExpressionAnalyzer sources, intermediates and results. It splits data and logic, allows to test them separately.
|
2018-10-18 15:03:14 +00:00
|
|
|
struct ExpressionAnalyzerData
|
2018-04-24 07:16:39 +00:00
|
|
|
{
|
2018-10-18 15:03:14 +00:00
|
|
|
SubqueriesForSets subqueries_for_sets;
|
|
|
|
PreparedSets prepared_sets;
|
|
|
|
|
|
|
|
/// Columns after ARRAY JOIN, JOIN, and/or aggregation.
|
|
|
|
NamesAndTypesList aggregated_columns;
|
|
|
|
NamesAndTypesList array_join_columns;
|
|
|
|
|
|
|
|
bool has_aggregation = false;
|
|
|
|
NamesAndTypesList aggregation_keys;
|
|
|
|
AggregateDescriptions aggregate_descriptions;
|
|
|
|
|
|
|
|
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;
|
2018-10-18 15:03:14 +00:00
|
|
|
};
|
2018-04-24 07:16:39 +00:00
|
|
|
|
2014-07-06 19:48: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.
|
2014-03-19 11:44:41 +00:00
|
|
|
*
|
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.
|
2013-05-24 10:49:19 +00:00
|
|
|
*/
|
2019-08-14 19:30:30 +00:00
|
|
|
class ExpressionAnalyzer : protected ExpressionAnalyzerData, private boost::noncopyable
|
2013-05-24 10:49:19 +00:00
|
|
|
{
|
2016-01-13 00:32:59 +00:00
|
|
|
private:
|
2018-10-18 15:03:14 +00:00
|
|
|
/// 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;
|
|
|
|
|
2019-08-03 11:02:40 +00:00
|
|
|
ExtractedSettings(const Settings & settings_)
|
|
|
|
: use_index_for_in_with_subqueries(settings_.use_index_for_in_with_subqueries),
|
2019-09-09 19:43:37 +00:00
|
|
|
size_limits_for_set(settings_.max_rows_in_set, settings_.max_bytes_in_set, settings_.set_overflow_mode)
|
2018-10-18 15:03:14 +00:00
|
|
|
{}
|
|
|
|
};
|
2016-01-13 00:32:59 +00:00
|
|
|
|
2013-05-24 10:49:19 +00:00
|
|
|
public:
|
2019-08-14 19:30:30 +00:00
|
|
|
/// Ctor for non-select queries. Generally its usage is:
|
|
|
|
/// auto actions = ExpressionAnalyzer(query, syntax, context).getActions();
|
2017-04-01 07:20:54 +00:00
|
|
|
ExpressionAnalyzer(
|
2018-08-27 17:58:43 +00:00
|
|
|
const ASTPtr & query_,
|
2018-11-08 15:43:14 +00:00
|
|
|
const SyntaxAnalyzerResultPtr & syntax_analyzer_result_,
|
2019-08-14 19:30:30 +00:00
|
|
|
const Context & context_)
|
|
|
|
: ExpressionAnalyzer(query_, syntax_analyzer_result_, context_, 0, false)
|
|
|
|
{}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-09-04 13:45:39 +00:00
|
|
|
void appendExpression(ExpressionActionsChain & chain, const ASTPtr & expr, bool only_types);
|
2018-09-03 13:36:58 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// If `ast` is not a SELECT query, just gets all the actions to evaluate the expression.
|
2018-09-03 17:24:46 +00:00
|
|
|
/// 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.
|
2018-09-03 17:24:46 +00:00
|
|
|
ExpressionActionsPtr getActions(bool add_aliases, bool project_result = true);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
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.
|
2017-04-01 07:20:54 +00:00
|
|
|
ExpressionActionsPtr getConstActions();
|
|
|
|
|
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.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2018-09-04 11:38:41 +00:00
|
|
|
const SubqueriesForSets & getSubqueriesForSets() const { return subqueries_for_sets; }
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-10-19 15:33:40 +00:00
|
|
|
/// Get intermediates for tests
|
|
|
|
const ExpressionAnalyzerData & getAnalyzedData() const { return *this; }
|
|
|
|
|
2019-08-14 19:30:30 +00:00
|
|
|
protected:
|
|
|
|
ExpressionAnalyzer(
|
|
|
|
const ASTPtr & query_,
|
|
|
|
const SyntaxAnalyzerResultPtr & syntax_analyzer_result_,
|
|
|
|
const Context & context_,
|
|
|
|
size_t subquery_depth_,
|
|
|
|
bool do_global_);
|
2018-10-10 17:07:21 +00:00
|
|
|
|
2018-08-27 17:58:43 +00:00
|
|
|
ASTPtr query;
|
2017-04-01 07:20:54 +00:00
|
|
|
const Context & context;
|
2018-10-18 15:03:14 +00:00
|
|
|
const ExtractedSettings settings;
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t subquery_depth;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2018-11-08 15:43:14 +00:00
|
|
|
SyntaxAnalyzerResultPtr syntax;
|
2018-03-04 16:15:31 +00:00
|
|
|
|
2020-03-24 18:06:55 +00:00
|
|
|
const ConstStoragePtr & storage() const { return syntax->storage; } /// The main table in FROM clause, if exists.
|
2020-04-07 09:48:47 +00:00
|
|
|
const TableJoin & analyzedJoin() const { return *syntax->analyzed_join; }
|
2019-08-09 14:50:04 +00:00
|
|
|
const NamesAndTypesList & sourceColumns() const { return syntax->required_source_columns; }
|
2019-08-13 12:39:03 +00:00
|
|
|
const std::vector<const ASTFunction *> & aggregates() const { return syntax->aggregates; }
|
2019-12-20 13:15:17 +00:00
|
|
|
NamesAndTypesList sourceWithJoinedColumns() const;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables.
|
2019-08-13 13:48:09 +00:00
|
|
|
void initGlobalSubqueriesAndExternalTables(bool do_global);
|
2014-07-04 20:30:06 +00:00
|
|
|
|
2018-12-19 13:13:51 +00:00
|
|
|
void addMultipleArrayJoinAction(ExpressionActionsPtr & actions, bool is_left) const;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2019-09-16 12:37:46 +00:00
|
|
|
void addJoinAction(ExpressionActionsPtr & actions, JoinPtr = {}) const;
|
2014-06-13 02:05:05 +00:00
|
|
|
|
2018-10-16 12:34:20 +00:00
|
|
|
void getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts = false);
|
2014-06-12 18:41:09 +00:00
|
|
|
|
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, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts = false);
|
|
|
|
|
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.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
void analyzeAggregation();
|
2019-08-13 12:39:03 +00:00
|
|
|
bool makeAggregateDescriptions(ExpressionActionsPtr & actions);
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// columns - the columns that are present before the transformations begin.
|
2017-12-25 21:57:29 +00:00
|
|
|
void initChain(ExpressionActionsChain & chain, const NamesAndTypesList & columns) const;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2019-08-13 13:48:09 +00:00
|
|
|
const ASTSelectQuery * getSelectQuery() const;
|
2019-08-14 19:30:30 +00:00
|
|
|
|
|
|
|
bool isRemoteStorage() const;
|
|
|
|
};
|
|
|
|
|
2020-02-10 15:50:12 +00:00
|
|
|
class SelectQueryExpressionAnalyzer;
|
|
|
|
|
|
|
|
/// Result of SelectQueryExpressionAnalyzer: expressions for InterpreterSelectQuery
|
|
|
|
struct ExpressionAnalysisResult
|
|
|
|
{
|
2020-02-10 19:55:13 +00:00
|
|
|
/// 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;
|
|
|
|
|
2020-02-10 15:50:12 +00:00
|
|
|
bool need_aggregate = false;
|
|
|
|
bool has_order_by = false;
|
|
|
|
|
|
|
|
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-02-10 15:50:12 +00:00
|
|
|
|
|
|
|
ExpressionActionsPtr before_join; /// including JOIN
|
|
|
|
ExpressionActionsPtr before_where;
|
|
|
|
ExpressionActionsPtr before_aggregation;
|
|
|
|
ExpressionActionsPtr before_having;
|
|
|
|
ExpressionActionsPtr before_order_and_select;
|
|
|
|
ExpressionActionsPtr before_limit_by;
|
|
|
|
ExpressionActionsPtr final_projection;
|
|
|
|
|
|
|
|
/// Columns from the SELECT list, before renaming them to aliases.
|
|
|
|
Names selected_columns;
|
|
|
|
|
|
|
|
/// Columns will be removed after prewhere actions execution.
|
|
|
|
Names columns_to_remove_after_prewhere;
|
|
|
|
|
|
|
|
PrewhereInfoPtr prewhere_info;
|
|
|
|
FilterInfoPtr filter_info;
|
|
|
|
ConstantFilterDescription prewhere_constant_filter_description;
|
|
|
|
ConstantFilterDescription where_constant_filter_description;
|
2020-02-10 19:55:13 +00:00
|
|
|
/// 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;
|
2020-02-10 15:50:12 +00:00
|
|
|
|
|
|
|
ExpressionAnalysisResult() = default;
|
|
|
|
|
|
|
|
ExpressionAnalysisResult(
|
|
|
|
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);
|
|
|
|
|
2020-06-13 20:59:20 +00:00
|
|
|
/// Filter for row-level security.
|
2020-02-10 15:50:12 +00:00
|
|
|
bool hasFilter() const { return filter_info.get(); }
|
2020-06-13 20:59:20 +00:00
|
|
|
|
2020-02-10 15:50:12 +00:00
|
|
|
bool hasJoin() const { return before_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;
|
2020-02-10 15:50:12 +00:00
|
|
|
void finalize(const ExpressionActionsChain & chain, const Context & context, size_t where_step_num);
|
|
|
|
};
|
|
|
|
|
2019-08-14 19:30:30 +00:00
|
|
|
/// SelectQuery specific ExpressionAnalyzer part.
|
|
|
|
class SelectQueryExpressionAnalyzer : public ExpressionAnalyzer
|
|
|
|
{
|
|
|
|
public:
|
2020-02-10 15:50:12 +00:00
|
|
|
friend struct ExpressionAnalysisResult;
|
|
|
|
|
2019-08-14 19:30:30 +00:00
|
|
|
SelectQueryExpressionAnalyzer(
|
|
|
|
const ASTPtr & query_,
|
|
|
|
const SyntaxAnalyzerResultPtr & syntax_analyzer_result_,
|
|
|
|
const Context & context_,
|
2020-06-17 09:38:47 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot_,
|
2019-08-14 19:30:30 +00:00
|
|
|
const NameSet & required_result_columns_ = {},
|
2020-02-20 02:56:20 +00:00
|
|
|
bool do_global_ = false,
|
|
|
|
const SelectQueryOptions & options_ = {})
|
2020-06-17 09:38:47 +00:00
|
|
|
: ExpressionAnalyzer(query_, syntax_analyzer_result_, context_, options_.subquery_depth, do_global_)
|
|
|
|
, metadata_snapshot(metadata_snapshot_)
|
|
|
|
, required_result_columns(required_result_columns_)
|
|
|
|
, query_options(options_)
|
2020-02-20 02:56:20 +00:00
|
|
|
{
|
|
|
|
}
|
2019-08-14 19:30:30 +00:00
|
|
|
|
|
|
|
/// Does the expression have aggregate functions or a GROUP BY or HAVING section.
|
|
|
|
bool hasAggregation() const { return has_aggregation; }
|
|
|
|
bool hasGlobalSubqueries() { return has_global_subqueries; }
|
|
|
|
|
2020-02-10 19:55:13 +00:00
|
|
|
const NamesAndTypesList & aggregationKeys() const { return aggregation_keys; }
|
|
|
|
const AggregateDescriptions & aggregates() const { return aggregate_descriptions; }
|
2019-08-14 19:30:30 +00:00
|
|
|
|
|
|
|
const PreparedSets & getPreparedSets() const { return prepared_sets; }
|
|
|
|
|
|
|
|
/// 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; }
|
2019-08-14 19:30:30 +00:00
|
|
|
|
2020-02-10 15:50:12 +00:00
|
|
|
ExpressionActionsPtr simpleSelectActions();
|
2019-08-14 19:30:30 +00:00
|
|
|
|
2020-02-10 15:50:12 +00:00
|
|
|
/// These appends are public only for tests
|
2019-08-14 19:30:30 +00:00
|
|
|
void appendSelect(ExpressionActionsChain & chain, bool only_types);
|
|
|
|
/// Deletes all columns except mentioned by SELECT, arranges the remaining columns and renames them to aliases.
|
|
|
|
void appendProjectResult(ExpressionActionsChain & chain) const;
|
|
|
|
|
|
|
|
private:
|
2020-06-17 09:38:47 +00:00
|
|
|
StorageMetadataPtr metadata_snapshot;
|
2019-08-14 19:30:30 +00:00
|
|
|
/// If non-empty, ignore all expressions not from this list.
|
|
|
|
NameSet required_result_columns;
|
2020-02-20 02:56:20 +00:00
|
|
|
SelectQueryOptions query_options;
|
2014-03-31 14:49:43 +00:00
|
|
|
|
2018-01-23 08:18:12 +00:00
|
|
|
/**
|
2019-07-11 13:51:54 +00:00
|
|
|
* Create Set from a subquery or a table expression in the query. The created set is suitable for using the index.
|
2018-01-23 08:18:12 +00:00
|
|
|
* The set will not be created if its size hits the limit.
|
|
|
|
*/
|
2018-07-02 19:37:04 +00:00
|
|
|
void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name);
|
2018-01-21 07:30:07 +00:00
|
|
|
|
2019-11-13 12:43:55 +00:00
|
|
|
/**
|
2019-12-03 02:09:05 +00:00
|
|
|
* 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.
|
|
|
|
*/
|
2020-03-08 23:48:08 +00:00
|
|
|
SetPtr isPlainStorageSetInSubquery(const ASTPtr & subquery_or_table_name);
|
2019-11-13 12:43:55 +00:00
|
|
|
|
2020-05-20 09:29:23 +00:00
|
|
|
/// Create Set-s that we make from IN section to use index on them.
|
|
|
|
void makeSetsForIndex(const ASTPtr & node);
|
|
|
|
|
2019-09-16 12:37:46 +00:00
|
|
|
JoinPtr makeTableJoin(const ASTTablesInSelectQueryElement & join_element);
|
2019-08-16 16:50:30 +00:00
|
|
|
|
2019-08-14 19:30:30 +00:00
|
|
|
const ASTSelectQuery * getAggregatingQuery() const;
|
2020-02-10 15:50:12 +00:00
|
|
|
|
|
|
|
/** 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:
|
|
|
|
bool appendArrayJoin(ExpressionActionsChain & chain, bool only_types);
|
|
|
|
bool appendJoin(ExpressionActionsChain & chain, bool only_types);
|
|
|
|
/// Add preliminary rows filtration. Actions are created in other expression analyzer to prevent any possible alias injection.
|
|
|
|
void appendPreliminaryFilter(ExpressionActionsChain & chain, ExpressionActionsPtr actions, String column_name);
|
|
|
|
/// 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).
|
|
|
|
bool 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 &);
|
2020-02-10 15:50:12 +00:00
|
|
|
void appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types);
|
|
|
|
|
|
|
|
/// After aggregation:
|
|
|
|
bool appendHaving(ExpressionActionsChain & chain, bool only_types);
|
|
|
|
/// appendSelect
|
2020-02-10 19:55:13 +00:00
|
|
|
bool appendOrderBy(ExpressionActionsChain & chain, bool only_types, bool optimize_read_in_order, ManyExpressionActions &);
|
2020-02-10 15:50:12 +00:00
|
|
|
bool appendLimitBy(ExpressionActionsChain & chain, bool only_types);
|
|
|
|
/// appendProjectResult
|
2013-05-24 10:49:19 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|