ClickHouse/src/Interpreters/ExpressionActions.h

531 lines
17 KiB
C++
Raw Normal View History

#pragma once
#include <Core/Block.h>
#include <Core/ColumnWithTypeAndName.h>
#include <Core/Names.h>
#include <Core/Settings.h>
2020-11-03 11:28:28 +00:00
#include <Core/ColumnNumbers.h>
#include <Common/SipHash.h>
#include <Common/UInt128.h>
2014-06-12 21:12:47 +00:00
#include <unordered_map>
#include <unordered_set>
#include <Parsers/ASTTablesInSelectQuery.h>
2020-08-12 08:55:16 +00:00
#include <DataTypes/DataTypeArray.h>
2020-11-03 11:28:28 +00:00
#include <boost/multi_index_container.hpp>
#include <boost/multi_index/sequenced_index.hpp>
#include <boost/multi_index/hashed_index.hpp>
#include <boost/multi_index/identity.hpp>
2020-08-13 20:17:18 +00:00
#include <variant>
#if !defined(ARCADIA_BUILD)
# include "config_core.h"
#endif
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
class Context;
class TableJoin;
class IJoin;
using JoinPtr = std::shared_ptr<IJoin>;
2019-12-08 21:06:37 +00:00
class IExecutableFunction;
using ExecutableFunctionPtr = std::shared_ptr<IExecutableFunction>;
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>;
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 ExpressionActions;
class CompiledExpressionCache;
2020-08-13 20:17:18 +00:00
class ArrayJoinAction;
using ArrayJoinActionPtr = std::shared_ptr<ArrayJoinAction>;
2020-08-12 13:30:02 +00:00
class ExpressionActions;
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
2020-11-03 11:28:28 +00:00
class ActionsDAG;
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
2020-11-10 14:54:59 +00:00
/// Directed acyclic graph of expressions.
/// This is an intermediate representation of actions which is usually built from expression list AST.
/// Node of DAG describe calculation of a single column with known type, name, and constant value (if applicable).
///
/// DAG representation is useful in case we need to know explicit dependencies between actions.
/// It is helpful when it is needed to optimize actions, remove unused expressions, compile subexpressions,
/// split or merge parts of graph, calculate expressions on partial input.
///
/// Built DAG is used by ExpressionActions, which calculates expressions on block.
2020-09-10 07:30:03 +00:00
class ActionsDAG
{
public:
2020-11-10 14:54:59 +00:00
enum class ActionType
2020-09-10 07:30:03 +00:00
{
/// Column which must be in input.
INPUT,
/// Constant column with known value.
COLUMN,
/// Another one name for column.
ALIAS,
/// Function arrayJoin. Specially separated because it changes the number of rows.
ARRAY_JOIN,
2020-09-10 16:01:41 +00:00
FUNCTION,
2020-09-10 07:30:03 +00:00
};
struct Node
{
std::vector<Node *> children;
2020-11-10 14:54:59 +00:00
ActionType type;
2020-09-10 07:30:03 +00:00
std::string result_name;
DataTypePtr result_type;
2020-09-10 16:01:41 +00:00
FunctionOverloadResolverPtr function_builder;
/// Can be used after action was added to ExpressionActions if we want to get function signature or properties like monotonicity.
FunctionBasePtr function_base;
/// Prepared function which is used in function execution.
ExecutableFunctionPtr function;
2020-10-13 08:16:47 +00:00
/// If function is a compiled statement.
bool is_function_compiled = false;
2020-09-10 16:01:41 +00:00
2020-09-10 07:30:03 +00:00
/// For COLUMN node and propagated constants.
ColumnPtr column;
2020-09-10 16:01:41 +00:00
/// Some functions like `ignore()` always return constant but can't be replaced by constant it.
/// We calculate such constants in order to avoid unnecessary materialization, but prohibit it's folding.
bool allow_constant_folding = true;
2020-09-10 07:30:03 +00:00
};
2020-11-11 14:07:51 +00:00
/// Index is used to:
/// * find Node buy it's result_name
/// * specify order of columns in result
/// It represents a set of available columns.
/// Removing of column from index is equivalent to removing of column from final result.
///
/// DAG allows actions with duplicating result names. In this case index will point to last added Node.
/// It does not cause any problems as long as execution of actions does not depend on action names anymore.
///
/// Index is a list of nodes + [map: name -> list::iterator].
2020-11-09 15:01:08 +00:00
/// List is ordered, may contain nodes with same names, or one node several times.
2020-11-03 11:28:28 +00:00
class Index
{
2020-11-09 15:01:08 +00:00
private:
std::list<Node *> list;
2020-11-10 14:54:59 +00:00
/// Map key is a string_view to Node::result_name for node from value.
/// Map always point to existing node, so key always valid (nodes live longer then index).
2020-11-09 15:01:08 +00:00
std::unordered_map<std::string_view, std::list<Node *>::iterator> map;
2020-11-03 11:28:28 +00:00
public:
auto size() const { return list.size(); }
bool contains(std::string_view key) const { return map.count(key) != 0; }
std::list<Node *>::iterator begin() { return list.begin(); }
std::list<Node *>::iterator end() { return list.end(); }
std::list<Node *>::const_iterator begin() const { return list.begin(); }
std::list<Node *>::const_iterator end() const { return list.end(); }
std::list<Node *>::const_iterator find(std::string_view key) const
{
auto it = map.find(key);
if (it == map.end())
return list.end();
return it->second;
}
/// Insert method doesn't check if map already have node with the same name.
/// If node with the same name exists, it is removed from map, but not list.
/// It is expected and used for project(), when result may have several columns with the same name.
void insert(Node * node) { map[node->result_name] = list.emplace(list.end(), node); }
2020-11-10 14:54:59 +00:00
/// If node with same name exists in index, replace it. Otherwise insert new node to index.
void replace(Node * node)
{
if (auto handle = map.extract(node->result_name))
{
handle.key() = node->result_name; /// Change string_view
*handle.mapped() = node;
map.insert(std::move(handle));
}
else
insert(node);
}
2020-11-03 11:28:28 +00:00
void remove(Node * node)
{
auto it = map.find(node->result_name);
if (it != map.end())
return;
list.erase(it->second);
map.erase(it);
}
2020-11-10 14:54:59 +00:00
void swap(Index & other)
{
list.swap(other.list);
map.swap(other.map);
}
2020-11-03 11:28:28 +00:00
};
using Nodes = std::list<Node>;
2020-09-10 07:30:03 +00:00
2020-11-10 16:27:55 +00:00
struct ActionsSettings
{
size_t max_temporary_columns = 0;
size_t max_temporary_non_const_columns = 0;
size_t min_count_to_compile_expression = 0;
bool compile_expressions = false;
bool project_input = false;
bool projected_output = false;
};
2020-09-10 07:30:03 +00:00
private:
2020-11-03 11:28:28 +00:00
Nodes nodes;
2020-09-10 07:30:03 +00:00
Index index;
2020-11-10 16:27:55 +00:00
ActionsSettings settings;
2020-11-03 11:28:28 +00:00
#if USE_EMBEDDED_COMPILER
std::shared_ptr<CompiledExpressionCache> compilation_cache;
#endif
2020-09-10 07:30:03 +00:00
public:
ActionsDAG() = default;
ActionsDAG(const ActionsDAG &) = delete;
ActionsDAG & operator=(const ActionsDAG &) = delete;
2020-10-07 18:37:27 +00:00
explicit ActionsDAG(const NamesAndTypesList & inputs);
explicit ActionsDAG(const ColumnsWithTypeAndName & inputs);
2020-09-10 07:30:03 +00:00
2020-11-03 11:28:28 +00:00
const Nodes & getNodes() const { return nodes; }
2020-09-10 07:30:03 +00:00
const Index & getIndex() const { return index; }
2020-11-03 11:28:28 +00:00
NamesAndTypesList getRequiredColumns() const;
2020-09-10 18:36:51 +00:00
ColumnsWithTypeAndName getResultColumns() const;
2020-09-10 16:01:41 +00:00
NamesAndTypesList getNamesAndTypesList() const;
2020-11-03 11:28:28 +00:00
2020-09-11 12:24:41 +00:00
Names getNames() const;
2020-09-10 16:01:41 +00:00
std::string dumpNames() const;
2020-11-03 11:28:28 +00:00
std::string dumpDAG() const;
2020-09-10 16:01:41 +00:00
2020-09-10 07:30:03 +00:00
const Node & addInput(std::string name, DataTypePtr type);
2020-09-13 13:51:31 +00:00
const Node & addInput(ColumnWithTypeAndName column);
2020-09-10 16:01:41 +00:00
const Node & addColumn(ColumnWithTypeAndName column);
2020-09-13 13:51:31 +00:00
const Node & addAlias(const std::string & name, std::string alias, bool can_replace = false);
2020-11-03 11:28:28 +00:00
const Node & addArrayJoin(const std::string & source_name, std::string result_name);
2020-09-10 16:01:41 +00:00
const Node & addFunction(
const FunctionOverloadResolverPtr & function,
const Names & argument_names,
std::string result_name,
2020-11-03 11:28:28 +00:00
const Context & context);
2020-09-10 16:01:41 +00:00
2020-11-03 11:28:28 +00:00
/// Call addAlias several times.
void addAliases(const NamesWithAliases & aliases);
2020-11-11 14:26:18 +00:00
/// Add alias actions and remove unused columns from index. Also specify result columns order in index.
2020-11-03 11:28:28 +00:00
void project(const NamesWithAliases & projection);
/// Removes column from index.
void removeColumn(const std::string & column_name);
/// If column is not in index, try to find it in nodes and insert back into index.
bool tryRestoreColumn(const std::string & column_name);
2020-11-10 16:27:55 +00:00
void projectInput() { settings.project_input = true; }
2020-11-03 11:28:28 +00:00
void removeUnusedActions(const Names & required_names);
2020-11-11 13:50:29 +00:00
/// Changed structure of empty block as if actions are executed.
/// Does not actually execute actions, but only creates empty columns.
void updateHeader(Block & block) const;
2020-11-03 11:28:28 +00:00
/// Splits actions into two parts. Returned half may be swapped with ARRAY JOIN.
/// Returns nullptr if no actions may be moved before ARRAY JOIN.
ActionsDAGPtr splitActionsBeforeArrayJoin(const NameSet & array_joined_columns);
bool hasArrayJoin() const;
2020-11-09 15:01:08 +00:00
bool empty() const; /// If actions only contain inputs.
2020-11-10 16:27:55 +00:00
const ActionsSettings & getSettings() const { return settings; }
void compileExpressions();
2020-11-03 11:28:28 +00:00
ActionsDAGPtr clone() const;
2020-09-10 07:30:03 +00:00
private:
2020-09-11 12:24:41 +00:00
Node & addNode(Node node, bool can_replace = false);
2020-09-10 07:30:03 +00:00
Node & getNode(const std::string & name);
2020-11-03 11:28:28 +00:00
ActionsDAGPtr cloneEmpty() const
{
auto actions = std::make_shared<ActionsDAG>();
2020-11-10 16:27:55 +00:00
actions->settings = settings;
2020-11-03 11:28:28 +00:00
#if USE_EMBEDDED_COMPILER
actions->compilation_cache = compilation_cache;
#endif
return actions;
}
void removeUnusedActions(const std::vector<Node *> & required_nodes);
2020-11-05 16:11:14 +00:00
void removeUnusedActions();
2020-11-03 11:28:28 +00:00
void addAliases(const NamesWithAliases & aliases, std::vector<Node *> & result_nodes);
2020-11-05 16:11:14 +00:00
void compileFunctions();
2020-09-10 07:30:03 +00:00
};
2020-09-11 12:24:41 +00:00
2020-11-11 11:15:25 +00:00
/// Sequence of actions on the block.
/// Is used to calculate expressions.
///
/// Takes ActionsDAG and orders actions using top-sort.
class ExpressionActions
{
2020-11-03 11:28:28 +00:00
public:
2020-10-07 18:37:27 +00:00
using Node = ActionsDAG::Node;
using Index = ActionsDAG::Index;
2020-11-03 11:28:28 +00:00
struct Argument
{
2020-11-09 15:01:08 +00:00
/// Position in ExecutionContext::columns
2020-11-03 11:28:28 +00:00
size_t pos;
2020-11-09 15:01:08 +00:00
/// True if there is another action which will use this column.
/// Otherwise column will be removed.
bool needed_later;
2020-11-03 11:28:28 +00:00
};
using Arguments = std::vector<Argument>;
2020-10-07 18:37:27 +00:00
struct Action
{
2020-11-03 11:28:28 +00:00
const Node * node;
Arguments arguments;
2020-10-07 18:37:27 +00:00
size_t result_position;
2020-11-03 11:28:28 +00:00
std::string toString() const;
2020-10-07 18:37:27 +00:00
};
2020-10-07 18:37:27 +00:00
using Actions = std::vector<Action>;
2020-11-03 11:28:28 +00:00
private:
2020-10-07 18:37:27 +00:00
2020-11-10 16:27:55 +00:00
ActionsDAGPtr actions_dag;
2020-10-07 18:37:27 +00:00
Actions actions;
2020-11-10 19:09:18 +00:00
size_t num_columns = 0;
2020-06-12 14:32:47 +00:00
2020-10-07 18:37:27 +00:00
NamesAndTypesList required_columns;
2020-11-03 11:28:28 +00:00
ColumnNumbers result_positions;
2020-10-13 08:16:47 +00:00
Block sample_block;
2020-11-03 11:28:28 +00:00
friend class ActionsDAG;
2020-10-07 18:37:27 +00:00
public:
~ExpressionActions();
2020-11-10 16:27:55 +00:00
explicit ExpressionActions(ActionsDAGPtr actions_dag_);
ExpressionActions(const ExpressionActions &) = default;
ExpressionActions & operator=(const ExpressionActions &) = default;
2020-11-03 11:28:28 +00:00
const Actions & getActions() const { return actions; }
2020-11-10 16:27:55 +00:00
const std::list<Node> & getNodes() const { return actions_dag->getNodes(); }
const ActionsDAG & getActionsDAG() const { return *actions_dag; }
2017-06-02 21:37:28 +00:00
/// Get a list of input columns.
2020-10-07 18:37:27 +00:00
Names getRequiredColumns() const;
2020-10-13 08:16:47 +00:00
const NamesAndTypesList & getRequiredColumnsWithTypes() const { return required_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.
2020-11-03 11:28:28 +00:00
void execute(Block & block, size_t & num_rows, bool dry_run = false) const;
2020-11-09 15:01:08 +00:00
/// The same, but without `num_rows`. If result block is empty, adds `_dummy` column to keep block size.
void execute(Block & block, bool dry_run = false) const;
2020-09-08 10:40:53 +00:00
bool hasArrayJoin() const;
2017-06-02 21:37:28 +00:00
/// Obtain a sample block that contains the names and types of result columns.
2020-10-13 08:16:47 +00:00
const Block & getSampleBlock() const { return sample_block; }
std::string dumpActions() const;
static std::string getSmallestColumn(const NamesAndTypesList & columns);
2019-10-11 17:27:54 +00:00
/// 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;
2020-11-03 11:28:28 +00:00
ExpressionActionsPtr clone() const;
2019-07-31 10:22:56 +00:00
2020-11-03 11:28:28 +00:00
private:
2020-11-10 16:27:55 +00:00
ExpressionActions() = default;
2020-11-10 20:36:38 +00:00
void checkLimits(const ColumnsWithTypeAndName & columns) const;
2020-11-10 16:27:55 +00:00
void linearizeActions();
};
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
{
2020-08-12 08:55:16 +00:00
explicit ExpressionActionsChain(const Context & context_) : context(context_) {}
2020-08-13 20:17:18 +00:00
struct Step
{
2020-08-19 19:33:49 +00:00
virtual ~Step() = default;
explicit Step(Names required_output_) : required_output(std::move(required_output_)) {}
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.
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;
2020-11-03 11:28:28 +00:00
virtual NamesAndTypesList getRequiredColumns() const = 0;
virtual ColumnsWithTypeAndName getResultColumns() const = 0;
2020-08-19 19:33:49 +00:00
/// Remove unused result and update required columns
virtual void finalize(const Names & required_output_) = 0;
/// Add projections to expression
virtual void prependProjectInput() const = 0;
virtual std::string dump() const = 0;
/// Only for ExpressionActionsStep
2020-09-11 12:24:41 +00:00
ActionsDAGPtr & actions();
const ActionsDAGPtr & actions() const;
2020-08-19 19:33:49 +00:00
};
struct ExpressionActionsStep : public Step
{
2020-11-03 11:28:28 +00:00
ActionsDAGPtr actions;
2020-08-19 19:33:49 +00:00
2020-09-11 12:24:41 +00:00
explicit ExpressionActionsStep(ActionsDAGPtr actions_, Names required_output_ = Names())
2020-08-19 19:33:49 +00:00
: Step(std::move(required_output_))
2020-11-03 11:28:28 +00:00
, actions(std::move(actions_))
2020-08-19 19:33:49 +00:00
{
}
2020-11-03 11:28:28 +00:00
NamesAndTypesList getRequiredColumns() const override
2020-08-12 08:55:16 +00:00
{
2020-11-03 11:28:28 +00:00
return actions->getRequiredColumns();
2020-08-12 08:55:16 +00:00
}
2020-11-03 11:28:28 +00:00
ColumnsWithTypeAndName getResultColumns() const override
2020-08-19 19:33:49 +00:00
{
2020-11-03 11:28:28 +00:00
return actions->getResultColumns();
2020-08-19 19:33:49 +00:00
}
2020-08-12 08:55:16 +00:00
2020-08-19 19:33:49 +00:00
void finalize(const Names & required_output_) override
{
2020-11-10 16:27:55 +00:00
if (!actions->getSettings().projected_output)
2020-11-03 11:28:28 +00:00
actions->removeUnusedActions(required_output_);
2020-08-19 19:33:49 +00:00
}
void prependProjectInput() const override
{
2020-11-03 11:28:28 +00:00
actions->projectInput();
2020-08-19 19:33:49 +00:00
}
std::string dump() const override
{
2020-11-10 16:27:55 +00:00
return actions->dumpDAG();
2020-08-19 19:33:49 +00:00
}
};
struct ArrayJoinStep : public Step
{
ArrayJoinActionPtr array_join;
NamesAndTypesList required_columns;
ColumnsWithTypeAndName result_columns;
2020-09-08 10:40:53 +00:00
ArrayJoinStep(ArrayJoinActionPtr array_join_, ColumnsWithTypeAndName required_columns_);
2020-08-12 08:55:16 +00:00
2020-11-03 11:28:28 +00:00
NamesAndTypesList getRequiredColumns() const override { return required_columns; }
ColumnsWithTypeAndName getResultColumns() const override { return result_columns; }
2020-08-19 19:33:49 +00:00
void finalize(const Names & required_output_) override;
void prependProjectInput() const override {} /// TODO: remove unused columns before ARRAY JOIN ?
std::string dump() const override { return "ARRAY JOIN"; }
};
2020-09-08 10:40:53 +00:00
struct JoinStep : public Step
{
std::shared_ptr<TableJoin> analyzed_join;
JoinPtr join;
NamesAndTypesList required_columns;
ColumnsWithTypeAndName result_columns;
JoinStep(std::shared_ptr<TableJoin> analyzed_join_, JoinPtr join_, ColumnsWithTypeAndName required_columns_);
2020-11-03 11:28:28 +00:00
NamesAndTypesList getRequiredColumns() const override { return required_columns; }
ColumnsWithTypeAndName getResultColumns() const override { return result_columns; }
2020-09-08 10:40:53 +00:00
void finalize(const Names & required_output_) override;
void prependProjectInput() const override {} /// TODO: remove unused columns before JOIN ?
std::string dump() const override { return "JOIN"; }
};
2020-08-19 19:33:49 +00:00
using StepPtr = std::unique_ptr<Step>;
using Steps = std::vector<StepPtr>;
const Context & context;
Steps steps;
2020-11-03 11:28:28 +00:00
void addStep(NameSet non_constant_inputs = {});
void finalize();
void clear()
{
steps.clear();
}
2020-11-03 11:28:28 +00:00
ActionsDAGPtr getLastActions(bool allow_empty = false)
{
2020-06-25 18:44:05 +00:00
if (steps.empty())
{
if (allow_empty)
return {};
throw Exception("Empty ExpressionActionsChain", ErrorCodes::LOGICAL_ERROR);
2020-06-25 18:44:05 +00:00
}
2020-11-03 11:28:28 +00:00
return typeid_cast<ExpressionActionsStep *>(steps.back().get())->actions;
}
Step & getLastStep()
{
if (steps.empty())
throw Exception("Empty ExpressionActionsChain", ErrorCodes::LOGICAL_ERROR);
2020-08-19 19:58:23 +00:00
return *steps.back();
}
Step & lastStep(const NamesAndTypesList & columns)
{
if (steps.empty())
2020-09-11 12:24:41 +00:00
steps.emplace_back(std::make_unique<ExpressionActionsStep>(std::make_shared<ActionsDAG>(columns)));
2020-08-19 19:58:23 +00:00
return *steps.back();
}
2020-07-26 14:21:57 +00:00
std::string dumpChain() const;
};
}