2013-05-24 10:49:19 +00:00
|
|
|
#pragma once
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/AggregateDescription.h>
|
|
|
|
#include <Interpreters/Settings.h>
|
|
|
|
#include <Core/Block.h>
|
2013-05-24 10:49:19 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
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
|
|
|
class ExpressionActions;
|
|
|
|
struct ExpressionActionsChain;
|
|
|
|
|
2017-01-14 09:00:19 +00:00
|
|
|
class Join;
|
|
|
|
using JoinPtr = std::shared_ptr<Join>;
|
|
|
|
|
|
|
|
class IAST;
|
|
|
|
using ASTPtr = std::shared_ptr<IAST>;
|
|
|
|
|
2017-07-15 03:48:36 +00:00
|
|
|
class Set;
|
|
|
|
using SetPtr = std::shared_ptr<Set>;
|
|
|
|
using PreparedSets = std::unordered_map<IAST*, SetPtr>;
|
|
|
|
|
2017-01-14 09:00:19 +00:00
|
|
|
class IBlockInputStream;
|
|
|
|
using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>;
|
|
|
|
|
|
|
|
class IStorage;
|
|
|
|
using StoragePtr = std::shared_ptr<IStorage>;
|
|
|
|
using Tables = std::map<String, StoragePtr>;
|
|
|
|
|
|
|
|
class ASTFunction;
|
|
|
|
class ASTExpressionList;
|
|
|
|
class ASTSelectQuery;
|
|
|
|
|
2016-01-13 00:32:59 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/** Information on what to do when executing a subquery in the [GLOBAL] IN/JOIN section.
|
2014-07-06 19:48:39 +00:00
|
|
|
*/
|
|
|
|
struct SubqueryForSet
|
|
|
|
{
|
2017-06-02 21:37:28 +00:00
|
|
|
/// The source is obtained using the InterpreterSelectQuery subquery.
|
2017-04-01 07:20:54 +00:00
|
|
|
BlockInputStreamPtr source;
|
|
|
|
Block source_sample;
|
2014-07-06 19:48:39 +00:00
|
|
|
|
2017-07-14 00:33:37 +00:00
|
|
|
/// If set, build it from result.
|
2017-04-01 07:20:54 +00:00
|
|
|
SetPtr set;
|
|
|
|
JoinPtr join;
|
2014-07-06 19:48:39 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// If set, put the result into the table.
|
|
|
|
/// This is a temporary table for transferring to remote servers for distributed query processing.
|
2017-04-01 07:20:54 +00:00
|
|
|
StoragePtr table;
|
2014-07-06 19:48:39 +00:00
|
|
|
};
|
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// ID of subquery -> what to do with it.
|
2016-01-13 00:32:59 +00:00
|
|
|
using SubqueriesForSets = std::unordered_map<String, SubqueryForSet>;
|
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
|
|
|
*/
|
|
|
|
class ExpressionAnalyzer : private boost::noncopyable
|
|
|
|
{
|
2016-01-13 00:32:59 +00:00
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
2016-01-13 00:32:59 +00:00
|
|
|
|
2013-05-24 10:49:19 +00:00
|
|
|
public:
|
2017-04-01 07:20:54 +00:00
|
|
|
ExpressionAnalyzer(
|
|
|
|
const ASTPtr & ast_,
|
|
|
|
const Context & context_,
|
2017-09-08 03:47:27 +00:00
|
|
|
const StoragePtr & storage_,
|
2017-04-01 07:20:54 +00:00
|
|
|
const NamesAndTypesList & columns_,
|
|
|
|
size_t subquery_depth_ = 0,
|
|
|
|
bool do_global_ = false);
|
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Does the expression have aggregate functions or a GROUP BY or HAVING section.
|
2017-04-01 07:20:54 +00:00
|
|
|
bool hasAggregation() const { return has_aggregation; }
|
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Get a list of aggregation keys and descriptions of aggregate functions if the query contains GROUP BY.
|
2017-04-01 07:20:54 +00:00
|
|
|
void getAggregateInfo(Names & key_names, AggregateDescriptions & aggregates) const;
|
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/** Get a set of columns that are enough to read from the table to evaluate the expression.
|
|
|
|
* Columns added from another table by JOIN are not counted.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2017-10-13 17:47:59 +00:00
|
|
|
Names getRequiredColumns() const;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-06-02 21:37:28 +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.
|
2017-04-01 07:20:54 +00:00
|
|
|
*
|
2017-06-02 21:37:28 +00:00
|
|
|
* Example usage:
|
2017-04-01 07:20:54 +00:00
|
|
|
* ExpressionActionsChain chain;
|
|
|
|
* analyzer.appendWhere(chain);
|
|
|
|
* chain.addStep();
|
|
|
|
* analyzer.appendSelect(chain);
|
|
|
|
* analyzer.appendOrderBy(chain);
|
|
|
|
* chain.finalize();
|
|
|
|
*
|
2017-06-02 21:37:28 +00:00
|
|
|
* 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.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Before aggregation:
|
2017-04-01 07:20:54 +00:00
|
|
|
bool appendArrayJoin(ExpressionActionsChain & chain, bool only_types);
|
|
|
|
bool appendJoin(ExpressionActionsChain & chain, bool only_types);
|
|
|
|
bool appendWhere(ExpressionActionsChain & chain, bool only_types);
|
|
|
|
bool appendGroupBy(ExpressionActionsChain & chain, bool only_types);
|
|
|
|
void appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types);
|
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// After aggregation:
|
2017-04-01 07:20:54 +00:00
|
|
|
bool appendHaving(ExpressionActionsChain & chain, bool only_types);
|
|
|
|
void appendSelect(ExpressionActionsChain & chain, bool only_types);
|
|
|
|
bool appendOrderBy(ExpressionActionsChain & chain, bool only_types);
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Deletes all columns except mentioned by SELECT, arranges the remaining columns and renames them to aliases.
|
2017-12-01 21:13:25 +00:00
|
|
|
void appendProjectResult(ExpressionActionsChain & chain) const;
|
2017-04-01 07:20:54 +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.
|
|
|
|
/// If project_result, only the calculated values in the desired order, renamed to aliases, remain in the output block.
|
|
|
|
/// Otherwise, only temporary columns will be deleted from the block.
|
2017-04-01 07:20:54 +00:00
|
|
|
ExpressionActionsPtr getActions(bool project_result);
|
|
|
|
|
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
|
|
|
*/
|
2017-10-13 17:47:59 +00:00
|
|
|
SubqueriesForSets getSubqueriesForSets() const { return subqueries_for_sets; }
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-07-14 00:33:37 +00:00
|
|
|
PreparedSets getPreparedSets() { return prepared_sets; }
|
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/** Tables that will need to be sent to remote servers for distributed query processing.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
const Tables & getExternalTables() const { return external_tables; }
|
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// If ast is a SELECT query, it gets the aliases and column types from the SELECT section.
|
2017-04-01 07:20:54 +00:00
|
|
|
Block getSelectSampleBlock();
|
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Create Set-s that we can from IN section to use the index on them.
|
2017-04-01 07:20:54 +00:00
|
|
|
void makeSetsForIndex();
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2014-06-12 21:12:47 +00:00
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
ASTPtr ast;
|
|
|
|
ASTSelectQuery * select_query;
|
|
|
|
const Context & context;
|
|
|
|
Settings settings;
|
|
|
|
size_t subquery_depth;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Columns that are mentioned in the expression, but were not specified in the constructor.
|
2017-04-01 07:20:54 +00:00
|
|
|
NameSet unknown_required_columns;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/** Original columns.
|
|
|
|
* First, all available columns of the table are placed here. Then (when parsing the query), unused columns are deleted.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
NamesAndTypesList columns;
|
2014-06-12 19:53:36 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Columns after ARRAY JOIN, JOIN, and/or aggregation.
|
2017-04-01 07:20:54 +00:00
|
|
|
NamesAndTypesList aggregated_columns;
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// The table from which the query is made.
|
2017-04-01 07:20:54 +00:00
|
|
|
const StoragePtr storage;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
bool has_aggregation = false;
|
|
|
|
NamesAndTypesList aggregation_keys;
|
|
|
|
AggregateDescriptions aggregate_descriptions;
|
2014-03-04 11:26:55 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
SubqueriesForSets subqueries_for_sets;
|
2014-06-13 06:39:15 +00:00
|
|
|
|
2017-07-14 00:33:37 +00:00
|
|
|
PreparedSets prepared_sets;
|
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// NOTE: So far, only one JOIN per query is supported.
|
2014-07-02 20:23:48 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/** Query of the form `SELECT expr(x) AS FROM t1 ANY LEFT JOIN (SELECT expr(x) AS k FROM t2) USING k`
|
|
|
|
* The join is made by column k.
|
|
|
|
* During the JOIN,
|
|
|
|
* - in the "right" table, it will be available by alias `k`, since `Project` action for the subquery was executed.
|
|
|
|
* - in the "left" table, it will be accessible by the name `expr(x)`, since `Project` action has not been executed yet.
|
|
|
|
* You must remember both of these options.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
Names join_key_names_left;
|
|
|
|
Names join_key_names_right;
|
2014-07-02 20:23:48 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
NamesAndTypesList columns_added_by_join;
|
2014-06-12 04:04:47 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
using Aliases = std::unordered_map<String, ASTPtr>;
|
|
|
|
Aliases aliases;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
using SetOfASTs = std::set<const IAST *>;
|
|
|
|
using MapOfASTs = std::map<ASTPtr, ASTPtr>;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Which column is needed to be ARRAY-JOIN'ed to get the specified.
|
|
|
|
/// For example, for `SELECT s.v ... ARRAY JOIN a AS s` will get "s.v" -> "a.v".
|
2017-04-01 07:20:54 +00:00
|
|
|
NameToNameMap array_join_result_to_source;
|
2013-10-21 11:33:25 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// For the ARRAY JOIN section, mapping from the alias to the full column name.
|
|
|
|
/// For example, for `ARRAY JOIN [1,2] AS b` "b" -> "array(1,2)" will enter here.
|
2017-04-01 07:20:54 +00:00
|
|
|
NameToNameMap array_join_alias_to_name;
|
2014-04-29 14:21:53 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// The backward mapping for array_join_alias_to_name.
|
2017-04-01 07:20:54 +00:00
|
|
|
NameToNameMap array_join_name_to_alias;
|
2015-10-22 20:56:52 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Do I need to prepare for execution global subqueries when analyzing the query.
|
2017-04-01 07:20:54 +00:00
|
|
|
bool do_global;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// All new temporary tables obtained by performing the GLOBAL IN/JOIN subqueries.
|
2017-04-01 07:20:54 +00:00
|
|
|
Tables external_tables;
|
|
|
|
size_t external_table_id = 1;
|
2014-07-04 20:30:06 +00:00
|
|
|
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void init();
|
2013-05-24 10:49:19 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
static NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypesList & cols);
|
|
|
|
NamesAndTypesList::iterator findColumn(const String & name) { return findColumn(name, columns); }
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/** Remove all unnecessary columns from the list of all available columns of the table (`columns`).
|
|
|
|
* At the same time, form a set of unknown columns (`unknown_required_columns`),
|
|
|
|
* as well as the columns added by JOIN (`columns_added_by_join`).
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
void collectUsedColumns();
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/** Find the columns that are obtained by JOIN.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
void collectJoinedColumns(NameSet & joined_columns, NamesAndTypesList & joined_columns_name_type);
|
2014-06-12 23:21:38 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/** Create a dictionary of aliases.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
void addASTAliases(ASTPtr & ast, int ignore_levels = 0);
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/** For star nodes(`*`), expand them to a list of all columns.
|
|
|
|
* For literal nodes, substitute aliases.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
void normalizeTree();
|
|
|
|
void normalizeTreeImpl(ASTPtr & ast, MapOfASTs & finished_asts, SetOfASTs & current_asts, std::string current_alias, size_t level);
|
2014-03-14 14:52:48 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Eliminates injective function calls and constant expressions from group by statement
|
|
|
|
void optimizeGroupBy();
|
2014-07-31 07:24:07 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Remove duplicate items from ORDER BY.
|
2017-04-01 07:20:54 +00:00
|
|
|
void optimizeOrderBy();
|
2015-03-12 03:31:15 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void optimizeLimitBy();
|
2016-12-28 15:12:54 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// remove Function_if AST if condition is constant
|
|
|
|
void optimizeIfWithConstantCondition();
|
|
|
|
void optimizeIfWithConstantConditionImpl(ASTPtr & current_ast, Aliases & aliases) const;
|
|
|
|
bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & value) const;
|
2016-11-24 14:18:44 +00:00
|
|
|
|
2017-07-14 00:33:37 +00:00
|
|
|
void makeSet(const ASTFunction * node, const Block & sample_block);
|
2014-06-12 21:12:47 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Adds a list of ALIAS columns from the table
|
2017-04-01 07:20:54 +00:00
|
|
|
void addAliasColumns();
|
2015-10-29 15:14:19 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Replacing scalar subqueries with constant values.
|
2017-04-01 07:20:54 +00:00
|
|
|
void executeScalarSubqueries();
|
|
|
|
void executeScalarSubqueriesImpl(ASTPtr & ast);
|
2015-07-25 11:47:50 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables.
|
2017-04-01 07:20:54 +00:00
|
|
|
void initGlobalSubqueriesAndExternalTables();
|
|
|
|
void initGlobalSubqueries(ASTPtr & ast);
|
2014-07-04 20:30:06 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Finds in the query the usage of external tables (as table identifiers). Fills in external_tables.
|
2017-04-01 07:20:54 +00:00
|
|
|
void findExternalTables(ASTPtr & ast);
|
2014-07-04 20:30:06 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/** Initialize InterpreterSelectQuery for a subquery in the GLOBAL IN/JOIN section,
|
|
|
|
* create a temporary table of type Memory and store it in the external_tables dictionary.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
void addExternalStorage(ASTPtr & subquery_or_table_name);
|
2014-03-14 14:52:48 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void getArrayJoinedColumns();
|
2017-09-08 03:47:27 +00:00
|
|
|
void getArrayJoinedColumnsImpl(const ASTPtr & ast);
|
2017-04-01 07:20:54 +00:00
|
|
|
void addMultipleArrayJoinAction(ExpressionActionsPtr & actions) const;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void addJoinAction(ExpressionActionsPtr & actions, bool only_types) const;
|
2014-06-13 02:05:05 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
struct ScopeStack;
|
2017-09-08 03:47:27 +00:00
|
|
|
void getActionsImpl(const ASTPtr & ast, bool no_subqueries, bool only_consts, ScopeStack & actions_stack);
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-09-08 03:47:27 +00:00
|
|
|
void getRootActions(const ASTPtr & ast, bool no_subqueries, bool only_consts, ExpressionActionsPtr & actions);
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-09-08 03:47:27 +00:00
|
|
|
void getActionsBeforeAggregation(const ASTPtr & ast, ExpressionActionsPtr & actions, bool no_subqueries);
|
2014-06-12 18:41:09 +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.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
void analyzeAggregation();
|
|
|
|
void getAggregates(const ASTPtr & ast, ExpressionActionsPtr & actions);
|
|
|
|
void assertNoAggregates(const ASTPtr & ast, const char * description);
|
2013-10-17 13:32:32 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/** Get a set of necessary columns to read from the table.
|
|
|
|
* In this case, the columns specified in ignored_names are considered unnecessary. And the ignored_names parameter can be modified.
|
|
|
|
* The set of columns available_joined_columns are the columns available from JOIN, they are not needed for reading from the main table.
|
|
|
|
* Put in required_joined_columns the set of columns available from JOIN and needed.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2017-09-08 03:47:27 +00:00
|
|
|
void getRequiredColumnsImpl(const ASTPtr & ast,
|
2017-04-01 07:20:54 +00:00
|
|
|
NameSet & required_columns, NameSet & ignored_names,
|
|
|
|
const NameSet & available_joined_columns, NameSet & required_joined_columns);
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Get the table from which the query is made
|
2017-04-01 07:20:54 +00:00
|
|
|
StoragePtr getTable();
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// columns - the columns that are present before the transformations begin.
|
2017-04-01 07:20:54 +00:00
|
|
|
void initChain(ExpressionActionsChain & chain, const NamesAndTypesList & columns) const;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void assertSelect() const;
|
|
|
|
void assertAggregation() const;
|
2014-03-31 14:49:43 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/** Create Set from an explicit enumeration of values in the query.
|
|
|
|
* If create_ordered_set = true - create a data structure suitable for using the index.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2017-07-14 00:33:37 +00:00
|
|
|
void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool create_ordered_set);
|
|
|
|
void makeSetsForIndexImpl(const ASTPtr & node, const Block & sample_block);
|
2017-06-19 02:14:18 +00:00
|
|
|
|
|
|
|
/** Translate qualified names such as db.table.column, table.column, table_alias.column
|
|
|
|
* to unqualified names. This is done in a poor transitional way:
|
|
|
|
* only one ("main") table is supported. Ambiguity is not detected or resolved.
|
|
|
|
*/
|
|
|
|
void translateQualifiedNames();
|
|
|
|
void translateQualifiedNamesImpl(ASTPtr & node, const String & database_name, const String & table_name, const String & alias);
|
2013-05-24 10:49:19 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|