ClickHouse/dbms/src/Interpreters/ExpressionActions.h

273 lines
8.6 KiB
C++
Raw Normal View History

#pragma once
#include <Interpreters/Settings.h>
#include <Core/Names.h>
#include <Core/ColumnWithTypeAndName.h>
#include <Core/Block.h>
2014-06-12 21:12:47 +00:00
#include <unordered_set>
#include <unordered_map>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
using NameWithAlias = std::pair<std::string, std::string>;
using NamesWithAliases = std::vector<NameWithAlias>;
class Join;
2018-02-02 08:33:36 +00:00
class IFunctionBase;
using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
class IFunctionBuilder;
using FunctionBuilderPtr = std::shared_ptr<IFunctionBuilder>;
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 IDataType;
using DataTypePtr = std::shared_ptr<const IDataType>;
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 IBlockInputStream;
using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>;
2017-06-02 21:37:28 +00:00
/** Action on the block.
*/
struct ExpressionAction
{
public:
enum Type
{
ADD_COLUMN,
REMOVE_COLUMN,
COPY_COLUMN,
APPLY_FUNCTION,
2017-06-02 21:37:28 +00:00
/** Replaces the specified columns with arrays into columns with elements.
* Duplicates the values in the remaining columns by the number of elements in the arrays.
* Arrays must be parallel (have the same lengths).
*/
ARRAY_JOIN,
JOIN,
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.
PROJECT,
};
Type type;
/// For ADD/REMOVE/COPY_COLUMN.
std::string source_name;
std::string result_name;
DataTypePtr result_type;
/// For ADD_COLUMN.
ColumnPtr added_column;
/// For APPLY_FUNCTION and LEFT ARRAY JOIN.
2018-02-02 08:33:36 +00:00
FunctionBuilderPtr function_builder;
FunctionBasePtr function;
Names argument_names;
/// For ARRAY_JOIN
NameSet array_joined_columns;
bool array_join_is_left = false;
/// For JOIN
std::shared_ptr<const Join> join;
NamesAndTypesList columns_added_by_join;
/// For PROJECT.
NamesWithAliases projection;
2017-06-02 21:37:28 +00:00
/// If result_name_ == "", as name "function_name(arguments separated by commas) is used".
static ExpressionAction applyFunction(
2018-02-02 08:33:36 +00:00
const FunctionBuilderPtr & function_, const std::vector<std::string> & argument_names_, std::string result_name_ = "");
static ExpressionAction addColumn(const ColumnWithTypeAndName & added_column_);
static ExpressionAction removeColumn(const std::string & removed_name);
static ExpressionAction copyColumn(const std::string & from_name, const std::string & to_name);
static ExpressionAction project(const NamesWithAliases & projected_columns_);
static ExpressionAction project(const Names & projected_columns_);
static ExpressionAction arrayJoin(const NameSet & array_joined_columns, bool array_join_is_left, const Context & context);
static ExpressionAction ordinaryJoin(std::shared_ptr<const Join> join_, const NamesAndTypesList & columns_added_by_join_);
2017-06-02 21:37:28 +00:00
/// Which columns necessary to perform this action.
Names getNeededColumns() const;
std::string toString() const;
private:
friend class ExpressionActions;
void prepare(Block & sample_block);
void execute(Block & block) const;
void executeOnTotals(Block & block) const;
};
2017-06-02 21:37:28 +00:00
/** Contains a sequence of actions on the block.
*/
class ExpressionActions
{
public:
using Actions = std::vector<ExpressionAction>;
ExpressionActions(const NamesAndTypesList & input_columns_, const Settings & settings_)
: input_columns(input_columns_), settings(settings_)
{
for (const auto & input_elem : input_columns)
sample_block.insert(ColumnWithTypeAndName(nullptr, input_elem.type, input_elem.name));
}
2017-06-02 21:37:28 +00:00
/// For constant columns the columns themselves can be contained in `input_columns_`.
ExpressionActions(const ColumnsWithTypeAndName & input_columns_, const Settings & settings_)
: settings(settings_)
{
for (const auto & input_elem : input_columns_)
{
input_columns.emplace_back(input_elem.name, input_elem.type);
sample_block.insert(input_elem);
}
}
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.
void addInput(const ColumnWithTypeAndName & column);
void addInput(const NameAndTypePair & column);
void add(const ExpressionAction & action);
/// Adds new column names to out_new_columns (formed as a result of the added action).
void add(const ExpressionAction & action, Names & out_new_columns);
2017-06-02 21:37:28 +00:00
/// Adds to the beginning the removal of all extra columns.
void prependProjectInput();
2017-06-02 21:37:28 +00:00
/// Add the specified ARRAY JOIN action to the beginning. Change the appropriate input types to arrays.
/// If there are unknown columns in the ARRAY JOIN list, take their types from sample_block, and immediately after ARRAY JOIN remove them.
void prependArrayJoin(const ExpressionAction & action, const Block & sample_block);
2017-06-02 21:37:28 +00:00
/// If the last action is ARRAY JOIN, and it does not affect the columns from required_columns, discard and return it.
/// Change the corresponding output types to arrays.
bool popUnusedArrayJoin(const Names & required_columns, ExpressionAction & out_action);
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).
void finalize(const Names & output_columns);
const Actions & getActions() const { return actions; }
2017-06-02 21:37:28 +00:00
/// Get a list of input columns.
Names getRequiredColumns() const
{
Names names;
for (NamesAndTypesList::const_iterator it = input_columns.begin(); it != input_columns.end(); ++it)
names.push_back(it->name);
return names;
}
const NamesAndTypesList & getRequiredColumnsWithTypes() const { return input_columns; }
2017-06-02 21:37:28 +00:00
/// Execute the expression on the block. The block must contain all the columns returned by getRequiredColumns.
void execute(Block & block) 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.
*/
void executeOnTotals(Block & block) const;
2017-06-02 21:37:28 +00:00
/// Obtain a sample block that contains the names and types of result columns.
const Block & getSampleBlock() const { return sample_block; }
std::string dumpActions() const;
static std::string getSmallestColumn(const NamesAndTypesList & columns);
BlockInputStreamPtr createStreamWithNonJoinedDataIfFullOrRightJoin(const Block & source_header, size_t max_block_size) const;
2015-04-18 22:30:43 +00:00
private:
NamesAndTypesList input_columns;
Actions actions;
Block sample_block;
Settings settings;
void checkLimits(Block & block) const;
void addImpl(ExpressionAction action, Names & new_names);
2017-06-02 21:37:28 +00:00
/// Try to improve something without changing the lists of input and output columns.
void optimize();
2017-06-02 21:37:28 +00:00
/// Move all arrayJoin as close as possible to the end.
void optimizeArrayJoin();
};
2016-01-13 00:32:59 +00:00
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
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.
*
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.
*/
struct ExpressionActionsChain
{
struct Step
{
ExpressionActionsPtr actions;
Names required_output;
Step(const ExpressionActionsPtr & actions_ = nullptr, const Names & required_output_ = Names())
: actions(actions_), required_output(required_output_) {}
};
using Steps = std::vector<Step>;
Settings settings;
Steps steps;
void addStep();
void finalize();
void clear()
{
steps.clear();
}
ExpressionActionsPtr getLastActions()
{
if (steps.empty())
throw Exception("Empty ExpressionActionsChain", ErrorCodes::LOGICAL_ERROR);
return steps.back().actions;
}
Step & getLastStep()
{
if (steps.empty())
throw Exception("Empty ExpressionActionsChain", ErrorCodes::LOGICAL_ERROR);
return steps.back();
}
std::string dumpChain();
};
}