2013-05-24 10:49:19 +00:00
|
|
|
#pragma once
|
|
|
|
|
2019-05-17 14:34:25 +00:00
|
|
|
#include <Core/Block.h>
|
|
|
|
#include <Core/ColumnWithTypeAndName.h>
|
|
|
|
#include <Core/Names.h>
|
|
|
|
#include <Core/Settings.h>
|
2018-08-30 16:31:20 +00:00
|
|
|
#include <Common/SipHash.h>
|
2019-12-12 08:57:25 +00:00
|
|
|
#include <Common/UInt128.h>
|
2014-06-12 21:12:47 +00:00
|
|
|
#include <unordered_map>
|
2019-05-17 14:34:25 +00:00
|
|
|
#include <unordered_set>
|
2019-08-14 15:11:20 +00:00
|
|
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
2020-08-12 08:55:16 +00:00
|
|
|
#include <DataTypes/DataTypeArray.h>
|
2013-05-24 10:49:19 +00:00
|
|
|
|
2020-08-13 20:17:18 +00:00
|
|
|
#include <variant>
|
|
|
|
|
2020-04-16 12:31:57 +00:00
|
|
|
#if !defined(ARCADIA_BUILD)
|
|
|
|
# include "config_core.h"
|
|
|
|
#endif
|
|
|
|
|
2013-05-24 10:49:19 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
2013-05-28 14:24:20 +00:00
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
2020-05-20 20:16:32 +00:00
|
|
|
class Context;
|
2020-04-07 09:48:47 +00:00
|
|
|
class TableJoin;
|
2019-09-16 12:37:46 +00:00
|
|
|
class IJoin;
|
|
|
|
using JoinPtr = std::shared_ptr<IJoin>;
|
2014-06-13 02:05:05 +00:00
|
|
|
|
2019-12-08 21:06:37 +00:00
|
|
|
class IExecutableFunction;
|
|
|
|
using ExecutableFunctionPtr = std::shared_ptr<IExecutableFunction>;
|
2018-09-21 14:15:21 +00:00
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
class IFunctionBase;
|
|
|
|
using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
|
|
|
|
|
2019-12-08 21:06:37 +00:00
|
|
|
class IFunctionOverloadResolver;
|
|
|
|
using FunctionOverloadResolverPtr = std::shared_ptr<IFunctionOverloadResolver>;
|
2017-01-14 09:00:19 +00:00
|
|
|
|
|
|
|
class IDataType;
|
2017-12-18 01:11:48 +00:00
|
|
|
using DataTypePtr = std::shared_ptr<const IDataType>;
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2018-07-30 13:57:50 +00:00
|
|
|
class ExpressionActions;
|
2020-05-20 20:16:32 +00:00
|
|
|
class CompiledExpressionCache;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2020-08-13 20:17:18 +00:00
|
|
|
class ArrayJoinAction;
|
|
|
|
using ArrayJoinActionPtr = std::shared_ptr<ArrayJoinAction>;
|
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/** Action on the block.
|
2013-05-24 10:49:19 +00:00
|
|
|
*/
|
2014-06-12 18:41:09 +00:00
|
|
|
struct ExpressionAction
|
2013-05-24 10:49:19 +00:00
|
|
|
{
|
2018-07-30 13:57:50 +00:00
|
|
|
private:
|
|
|
|
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
2013-05-24 10:49:19 +00:00
|
|
|
public:
|
2017-04-01 07:20:54 +00:00
|
|
|
enum Type
|
|
|
|
{
|
|
|
|
ADD_COLUMN,
|
|
|
|
REMOVE_COLUMN,
|
|
|
|
COPY_COLUMN,
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
APPLY_FUNCTION,
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2020-08-13 20:17:18 +00:00
|
|
|
/// Replaces the source column with array into column with elements.
|
|
|
|
/// Duplicates the values in the remaining columns by the number of elements in the arrays.
|
|
|
|
/// Source column is removed from block.
|
2017-04-01 07:20:54 +00:00
|
|
|
ARRAY_JOIN,
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
JOIN,
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Reorder and rename the columns, delete the extra ones. The same column names are allowed in the result.
|
2017-04-01 07:20:54 +00:00
|
|
|
PROJECT,
|
2018-09-03 17:24:46 +00:00
|
|
|
/// Add columns with alias names. This columns are the same as non-aliased. PROJECT columns if you need to modify them.
|
|
|
|
ADD_ALIASES,
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2019-06-16 18:12:14 +00:00
|
|
|
Type type{};
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2020-08-13 20:17:18 +00:00
|
|
|
/// For ADD/REMOVE/ARRAY_JOIN/COPY_COLUMN.
|
2017-04-01 07:20:54 +00:00
|
|
|
std::string source_name;
|
|
|
|
std::string result_name;
|
|
|
|
DataTypePtr result_type;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2018-08-09 19:17:55 +00:00
|
|
|
/// If COPY_COLUMN can replace the result column.
|
|
|
|
bool can_replace = false;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// For ADD_COLUMN.
|
|
|
|
ColumnPtr added_column;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2020-02-05 10:12:19 +00:00
|
|
|
/// For APPLY_FUNCTION.
|
2019-12-12 14:16:59 +00:00
|
|
|
/// OverloadResolver is used before action was added to ExpressionActions (when we don't know types of arguments).
|
2019-12-08 21:06:37 +00:00
|
|
|
FunctionOverloadResolverPtr function_builder;
|
2018-12-12 03:13:02 +00:00
|
|
|
|
2018-09-27 15:16:47 +00:00
|
|
|
/// Can be used after action was added to ExpressionActions if we want to get function signature or properties like monotonicity.
|
2018-09-21 14:15:21 +00:00
|
|
|
FunctionBasePtr function_base;
|
2018-09-27 15:16:47 +00:00
|
|
|
/// Prepared function which is used in function execution.
|
2019-12-08 21:06:37 +00:00
|
|
|
ExecutableFunctionPtr function;
|
2017-04-01 07:20:54 +00:00
|
|
|
Names argument_names;
|
2018-08-29 20:05:31 +00:00
|
|
|
bool is_function_compiled = false;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// For JOIN
|
2020-04-07 09:48:47 +00:00
|
|
|
std::shared_ptr<const TableJoin> table_join;
|
2019-09-16 12:37:46 +00:00
|
|
|
JoinPtr join;
|
2014-06-12 19:23:06 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// For PROJECT.
|
|
|
|
NamesWithAliases projection;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// If result_name_ == "", as name "function_name(arguments separated by commas) is used".
|
2017-04-01 07:20:54 +00:00
|
|
|
static ExpressionAction applyFunction(
|
2019-12-08 21:06:37 +00:00
|
|
|
const FunctionOverloadResolverPtr & function_, const std::vector<std::string> & argument_names_, std::string result_name_ = "");
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2018-11-26 01:39:04 +00:00
|
|
|
static ExpressionAction addColumn(const ColumnWithTypeAndName & added_column_);
|
2017-04-01 07:20:54 +00:00
|
|
|
static ExpressionAction removeColumn(const std::string & removed_name);
|
2018-08-09 19:17:55 +00:00
|
|
|
static ExpressionAction copyColumn(const std::string & from_name, const std::string & to_name, bool can_replace = false);
|
2017-04-01 07:20:54 +00:00
|
|
|
static ExpressionAction project(const NamesWithAliases & projected_columns_);
|
|
|
|
static ExpressionAction project(const Names & projected_columns_);
|
2018-09-03 17:24:46 +00:00
|
|
|
static ExpressionAction addAliases(const NamesWithAliases & aliased_columns_);
|
2020-08-13 20:17:18 +00:00
|
|
|
static ExpressionAction arrayJoin(std::string source_name, std::string result_name);
|
2020-04-07 09:48:47 +00:00
|
|
|
static ExpressionAction ordinaryJoin(std::shared_ptr<TableJoin> table_join, JoinPtr join);
|
2014-06-13 02:05:05 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Which columns necessary to perform this action.
|
2017-04-01 07:20:54 +00:00
|
|
|
Names getNeededColumns() const;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
std::string toString() const;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2018-08-28 17:06:42 +00:00
|
|
|
bool operator==(const ExpressionAction & other) const;
|
|
|
|
|
|
|
|
struct ActionHash
|
|
|
|
{
|
2018-09-07 20:17:40 +00:00
|
|
|
UInt128 operator()(const ExpressionAction & action) const;
|
2018-08-28 17:06:42 +00:00
|
|
|
};
|
|
|
|
|
2014-06-12 18:41:09 +00:00
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
friend class ExpressionActions;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2019-08-16 10:29:33 +00:00
|
|
|
void prepare(Block & sample_block, const Settings & settings, NameSet & names_not_for_constant_folding);
|
2017-04-01 07:20:54 +00:00
|
|
|
void executeOnTotals(Block & block) const;
|
2020-01-13 18:00:32 +00:00
|
|
|
|
2020-08-08 01:01:47 +00:00
|
|
|
/// Executes action on block (modify it). Block could be split in case of JOIN. Then not_processed block is created.
|
2020-06-25 20:28:41 +00:00
|
|
|
void execute(Block & block, ExtraBlockPtr & not_processed) const;
|
|
|
|
void execute(Block & block, bool dry_run) const;
|
2014-06-12 18:41:09 +00:00
|
|
|
};
|
|
|
|
|
2020-08-12 13:30:02 +00:00
|
|
|
class ExpressionActions;
|
|
|
|
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/** Contains a sequence of actions on the block.
|
2014-06-12 18:41:09 +00:00
|
|
|
*/
|
|
|
|
class ExpressionActions
|
|
|
|
{
|
|
|
|
public:
|
2017-04-01 07:20:54 +00:00
|
|
|
using Actions = std::vector<ExpressionAction>;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2020-05-20 20:16:32 +00:00
|
|
|
ExpressionActions(const NamesAndTypesList & input_columns_, const Context & context_);
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// For constant columns the columns themselves can be contained in `input_columns_`.
|
2020-05-20 20:16:32 +00:00
|
|
|
ExpressionActions(const ColumnsWithTypeAndName & input_columns_, const Context & context_);
|
|
|
|
|
|
|
|
~ExpressionActions();
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2020-06-12 14:32:47 +00:00
|
|
|
ExpressionActions(const ExpressionActions & other) = default;
|
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Add the input column.
|
|
|
|
/// The name of the column must not match the names of the intermediate columns that occur when evaluating the expression.
|
|
|
|
/// The expression must not have any PROJECT actions.
|
2017-04-01 07:20:54 +00:00
|
|
|
void addInput(const ColumnWithTypeAndName & column);
|
2017-12-25 21:57:29 +00:00
|
|
|
void addInput(const NameAndTypePair & column);
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void add(const ExpressionAction & action);
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2018-02-19 20:23:25 +00:00
|
|
|
/// Adds new column names to out_new_columns (formed as a result of the added action).
|
2017-04-01 07:20:54 +00:00
|
|
|
void add(const ExpressionAction & action, Names & out_new_columns);
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Adds to the beginning the removal of all extra columns.
|
2017-04-01 07:20:54 +00:00
|
|
|
void prependProjectInput();
|
2014-04-30 19:19:29 +00:00
|
|
|
|
2020-08-13 18:40:21 +00:00
|
|
|
/// Splits actions into two parts. Returned half may be swapped with ARRAY JOIN.
|
2020-08-13 20:17:18 +00:00
|
|
|
/// Returns nullptr if no actions may be moved before ARRAY JOIN.
|
2020-08-12 13:30:02 +00:00
|
|
|
ExpressionActionsPtr splitActionsBeforeArrayJoin(const NameSet & array_joined_columns);
|
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// - Adds actions to delete all but the specified columns.
|
|
|
|
/// - Removes unused input columns.
|
|
|
|
/// - Can somehow optimize the expression.
|
|
|
|
/// - Does not reorder the columns.
|
|
|
|
/// - Does not remove "unexpected" columns (for example, added by functions).
|
|
|
|
/// - If output_columns is empty, leaves one arbitrary column (so that the number of rows in the block is not lost).
|
2017-04-01 07:20:54 +00:00
|
|
|
void finalize(const Names & output_columns);
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
const Actions & getActions() const { return actions; }
|
2017-02-09 17:29:36 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Get a list of input columns.
|
2017-04-01 07:20:54 +00:00
|
|
|
Names getRequiredColumns() const
|
|
|
|
{
|
|
|
|
Names names;
|
2020-08-13 20:17:18 +00:00
|
|
|
for (const auto & input : input_columns)
|
|
|
|
names.push_back(input.name);
|
2017-04-01 07:20:54 +00:00
|
|
|
return names;
|
|
|
|
}
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-12-25 21:57:29 +00:00
|
|
|
const NamesAndTypesList & getRequiredColumnsWithTypes() const { return input_columns; }
|
2013-05-24 10:49:19 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Execute the expression on the block. The block must contain all the columns returned by getRequiredColumns.
|
2018-12-02 11:00:23 +00:00
|
|
|
void execute(Block & block, bool dry_run = false) const;
|
2013-05-24 10:49:19 +00:00
|
|
|
|
2020-06-26 12:24:13 +00:00
|
|
|
/// Execute the expression on the block with continuation. This method in only supported for single JOIN.
|
2020-06-25 19:26:03 +00:00
|
|
|
void execute(Block & block, ExtraBlockPtr & not_processed) const;
|
2020-01-13 18:00:32 +00:00
|
|
|
|
2020-06-18 13:00:16 +00:00
|
|
|
bool hasJoinOrArrayJoin() const;
|
|
|
|
|
2019-04-09 14:51:38 +00:00
|
|
|
/// Check if joined subquery has totals.
|
|
|
|
bool hasTotalsInJoin() const;
|
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/** Execute the expression on the block of total values.
|
|
|
|
* Almost the same as `execute`. The difference is only when JOIN is executed.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
void executeOnTotals(Block & block) const;
|
2015-04-16 09:55:24 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Obtain a sample block that contains the names and types of result columns.
|
2017-04-01 07:20:54 +00:00
|
|
|
const Block & getSampleBlock() const { return sample_block; }
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
std::string dumpActions() const;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-12-25 21:57:29 +00:00
|
|
|
static std::string getSmallestColumn(const NamesAndTypesList & columns);
|
2013-05-24 10:49:19 +00:00
|
|
|
|
2019-09-16 12:37:46 +00:00
|
|
|
JoinPtr getTableJoinAlgo() const;
|
2015-04-18 22:30:43 +00:00
|
|
|
|
2018-08-31 10:27:08 +00:00
|
|
|
const Settings & getSettings() const { return settings; }
|
2018-08-28 17:06:42 +00:00
|
|
|
|
2019-10-11 17:27:54 +00:00
|
|
|
/// Check if result block has no rows. True if it's definite, false if we can't say for sure.
|
|
|
|
/// Call it only after subqueries for join were executed.
|
|
|
|
bool resultIsAlwaysEmpty() const;
|
|
|
|
|
|
|
|
/// Check if column is always zero. True if it's definite, false if we can't say for sure.
|
|
|
|
/// Call it only after subqueries for sets were executed.
|
|
|
|
bool checkColumnIsAlwaysFalse(const String & column_name) const;
|
2018-09-07 20:17:40 +00:00
|
|
|
|
|
|
|
struct ActionsHash
|
|
|
|
{
|
2019-01-04 12:10:00 +00:00
|
|
|
UInt128 operator()(const ExpressionActions::Actions & elems) const
|
2018-09-07 20:17:40 +00:00
|
|
|
{
|
|
|
|
SipHash hash;
|
2019-01-04 12:10:00 +00:00
|
|
|
for (const ExpressionAction & act : elems)
|
2018-09-07 20:17:40 +00:00
|
|
|
hash.update(ExpressionAction::ActionHash{}(act));
|
|
|
|
UInt128 result;
|
|
|
|
hash.get128(result.low, result.high);
|
|
|
|
return result;
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2013-05-24 10:49:19 +00:00
|
|
|
private:
|
2019-07-31 10:22:56 +00:00
|
|
|
/// These columns have to be in input blocks (arguments of execute* methods)
|
2017-12-25 21:57:29 +00:00
|
|
|
NamesAndTypesList input_columns;
|
2019-07-31 10:22:56 +00:00
|
|
|
/// These actions will be executed on input blocks
|
2017-04-01 07:20:54 +00:00
|
|
|
Actions actions;
|
2019-07-31 10:22:56 +00:00
|
|
|
/// The example of result (output) block.
|
2017-04-01 07:20:54 +00:00
|
|
|
Block sample_block;
|
2019-08-16 10:29:33 +00:00
|
|
|
/// Columns which can't be used for constant folding.
|
|
|
|
NameSet names_not_for_constant_folding;
|
2019-07-31 10:22:56 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Settings settings;
|
2018-08-31 10:27:08 +00:00
|
|
|
#if USE_EMBEDDED_COMPILER
|
|
|
|
std::shared_ptr<CompiledExpressionCache> compilation_cache;
|
|
|
|
#endif
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void checkLimits(Block & block) const;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2018-02-19 20:23:25 +00:00
|
|
|
void addImpl(ExpressionAction action, Names & new_names);
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Move all arrayJoin as close as possible to the end.
|
2017-04-01 07:20:54 +00:00
|
|
|
void optimizeArrayJoin();
|
2013-05-24 10:49:19 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/** The sequence of transformations over the block.
|
|
|
|
* It is assumed that the result of each step is fed to the input of the next step.
|
|
|
|
* Used to execute parts of the query individually.
|
2014-06-12 18:41:09 +00:00
|
|
|
*
|
2017-06-02 21:37:28 +00:00
|
|
|
* For example, you can create a chain of two steps:
|
|
|
|
* 1) evaluate the expression in the WHERE clause,
|
|
|
|
* 2) calculate the expression in the SELECT section,
|
|
|
|
* and between the two steps do the filtering by value in the WHERE clause.
|
2013-06-20 12:27:33 +00:00
|
|
|
*/
|
2013-05-28 14:24:20 +00:00
|
|
|
struct ExpressionActionsChain
|
|
|
|
{
|
2020-08-12 08:55:16 +00:00
|
|
|
explicit ExpressionActionsChain(const Context & context_) : context(context_) {}
|
|
|
|
|
2020-08-13 20:17:18 +00:00
|
|
|
struct ExpressionActionsLink
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-08-13 20:17:18 +00:00
|
|
|
ExpressionActionsPtr actions;
|
2020-08-12 08:55:16 +00:00
|
|
|
|
2020-08-13 20:17:18 +00:00
|
|
|
const NamesAndTypesList & getRequiredColumns() const { return actions->getRequiredColumnsWithTypes(); }
|
|
|
|
const ColumnsWithTypeAndName & getResultColumns() const { return actions->getSampleBlock().getColumnsWithTypeAndName(); }
|
|
|
|
void finalize(const Names & required_output_) const { actions->finalize(required_output_); }
|
|
|
|
void prependProjectInput() const { actions->prependProjectInput(); }
|
|
|
|
std::string dump() const { return actions->dumpActions(); }
|
|
|
|
};
|
2020-08-12 08:55:16 +00:00
|
|
|
|
2020-08-13 20:17:18 +00:00
|
|
|
struct ArrayJoinLink
|
|
|
|
{
|
2020-08-12 08:55:16 +00:00
|
|
|
ArrayJoinActionPtr array_join;
|
|
|
|
NamesAndTypesList required_columns;
|
2020-08-13 20:17:18 +00:00
|
|
|
ColumnsWithTypeAndName result_columns;
|
|
|
|
|
|
|
|
ArrayJoinLink(ArrayJoinActionPtr array_join_, ColumnsWithTypeAndName required_columns_);
|
|
|
|
|
|
|
|
const NamesAndTypesList & getRequiredColumns() const { return required_columns; }
|
|
|
|
const ColumnsWithTypeAndName & getResultColumns() const { return result_columns; }
|
|
|
|
void finalize(const Names & required_output_);
|
|
|
|
void prependProjectInput() const {} /// TODO: remove unused columns before ARRAY JOIN ?
|
|
|
|
static std::string dump() { return "ARRAY JOIN"; }
|
|
|
|
};
|
|
|
|
|
|
|
|
struct Step
|
|
|
|
{
|
|
|
|
std::variant<ExpressionActionsLink, ArrayJoinLink> link;
|
2020-08-12 08:55:16 +00:00
|
|
|
|
2018-04-23 19:05:46 +00:00
|
|
|
/// Columns were added to the block before current step in addition to prev step output.
|
2018-04-12 09:45:24 +00:00
|
|
|
NameSet additional_input;
|
2018-04-23 19:05:46 +00:00
|
|
|
/// Columns which are required in the result of current step.
|
2017-04-01 07:20:54 +00:00
|
|
|
Names required_output;
|
2018-04-23 19:05:46 +00:00
|
|
|
/// True if column from required_output is needed only for current step and not used in next actions
|
|
|
|
/// (and can be removed from block). Example: filter column for where actions.
|
2018-04-20 19:38:34 +00:00
|
|
|
/// If not empty, has the same size with required_output; is filled in finalize().
|
2018-06-29 11:42:44 +00:00
|
|
|
std::vector<bool> can_remove_required_output;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2020-08-12 08:55:16 +00:00
|
|
|
public:
|
2020-08-13 20:17:18 +00:00
|
|
|
explicit Step(ExpressionActionsPtr actions, const Names & required_output_ = Names())
|
|
|
|
: link(ExpressionActionsLink{std::move(actions)})
|
2020-08-12 08:55:16 +00:00
|
|
|
, required_output(required_output_)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2020-08-13 20:17:18 +00:00
|
|
|
explicit Step(ArrayJoinActionPtr array_join, ColumnsWithTypeAndName required_columns);
|
2020-08-12 08:55:16 +00:00
|
|
|
|
2020-08-13 20:17:18 +00:00
|
|
|
const NamesAndTypesList & getRequiredColumns() const;
|
|
|
|
const ColumnsWithTypeAndName & getResultColumns() const;
|
|
|
|
/// Remove unused result and update required columns
|
2020-08-12 08:55:16 +00:00
|
|
|
void finalize(const Names & required_output_);
|
2020-08-13 20:17:18 +00:00
|
|
|
/// Add projections to expression
|
2020-08-12 13:30:02 +00:00
|
|
|
void prependProjectInput() const;
|
2020-08-13 20:17:18 +00:00
|
|
|
std::string dump() const;
|
2020-08-12 08:55:16 +00:00
|
|
|
|
2020-08-13 20:17:18 +00:00
|
|
|
ExpressionActionsPtr & actions() { return std::get<ExpressionActionsLink>(link).actions; }
|
|
|
|
const ExpressionActionsPtr & actions() const { return std::get<ExpressionActionsLink>(link).actions; }
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
using Steps = std::vector<Step>;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2018-08-30 16:31:20 +00:00
|
|
|
const Context & context;
|
2017-04-01 07:20:54 +00:00
|
|
|
Steps steps;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void addStep();
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void finalize();
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void clear()
|
|
|
|
{
|
|
|
|
steps.clear();
|
|
|
|
}
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2020-06-25 17:40:25 +00:00
|
|
|
ExpressionActionsPtr getLastActions(bool allow_empty = false)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-06-25 18:44:05 +00:00
|
|
|
if (steps.empty())
|
|
|
|
{
|
|
|
|
if (allow_empty)
|
|
|
|
return {};
|
2017-04-01 07:20:54 +00:00
|
|
|
throw Exception("Empty ExpressionActionsChain", ErrorCodes::LOGICAL_ERROR);
|
2020-06-25 18:44:05 +00:00
|
|
|
}
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2020-08-13 20:17:18 +00:00
|
|
|
return steps.back().actions();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Step & getLastStep()
|
|
|
|
{
|
|
|
|
if (steps.empty())
|
|
|
|
throw Exception("Empty ExpressionActionsChain", ErrorCodes::LOGICAL_ERROR);
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return steps.back();
|
|
|
|
}
|
2014-04-30 19:19:29 +00:00
|
|
|
|
2020-06-25 23:05:47 +00:00
|
|
|
Step & lastStep(const NamesAndTypesList & columns)
|
|
|
|
{
|
|
|
|
if (steps.empty())
|
|
|
|
steps.emplace_back(std::make_shared<ExpressionActions>(columns, context));
|
|
|
|
return steps.back();
|
|
|
|
}
|
|
|
|
|
2020-07-26 14:21:57 +00:00
|
|
|
std::string dumpChain() const;
|
2013-05-28 14:24:20 +00:00
|
|
|
};
|
|
|
|
|
2013-05-24 10:49:19 +00:00
|
|
|
}
|