2020-11-16 14:57:56 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <Core/ColumnsWithTypeAndName.h>
|
|
|
|
#include <Core/NamesAndTypes.h>
|
|
|
|
#include <Core/Names.h>
|
2021-04-10 23:33:54 +00:00
|
|
|
#include <Interpreters/Context_fwd.h>
|
2020-11-16 14:57:56 +00:00
|
|
|
|
|
|
|
#if !defined(ARCADIA_BUILD)
|
|
|
|
# include "config_core.h"
|
|
|
|
#endif
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
class ActionsDAG;
|
|
|
|
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
|
|
|
|
|
|
|
|
class IExecutableFunction;
|
|
|
|
using ExecutableFunctionPtr = std::shared_ptr<IExecutableFunction>;
|
|
|
|
|
|
|
|
class IFunctionBase;
|
|
|
|
using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
|
|
|
|
|
|
|
|
class IFunctionOverloadResolver;
|
|
|
|
using FunctionOverloadResolverPtr = std::shared_ptr<IFunctionOverloadResolver>;
|
|
|
|
|
|
|
|
class IDataType;
|
|
|
|
using DataTypePtr = std::shared_ptr<const IDataType>;
|
|
|
|
|
|
|
|
class CompiledExpressionCache;
|
|
|
|
|
2021-04-14 14:07:56 +00:00
|
|
|
namespace JSONBuilder
|
|
|
|
{
|
|
|
|
class JSONMap;
|
|
|
|
|
|
|
|
class IItem;
|
|
|
|
using ItemPtr = std::unique_ptr<IItem>;
|
|
|
|
}
|
|
|
|
|
2020-11-16 14:57:56 +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.
|
|
|
|
class ActionsDAG
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
|
|
|
|
enum class ActionType
|
|
|
|
{
|
|
|
|
/// 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,
|
|
|
|
FUNCTION,
|
|
|
|
};
|
|
|
|
|
2021-04-09 16:18:45 +00:00
|
|
|
static const char * typeToString(ActionType type);
|
|
|
|
|
2021-03-02 17:08:59 +00:00
|
|
|
struct Node;
|
|
|
|
using NodeRawPtrs = std::vector<Node *>;
|
|
|
|
using NodeRawConstPtrs = std::vector<const Node *>;
|
|
|
|
|
2020-11-16 14:57:56 +00:00
|
|
|
struct Node
|
|
|
|
{
|
2021-03-02 17:08:59 +00:00
|
|
|
NodeRawConstPtrs children;
|
2020-11-16 14:57:56 +00:00
|
|
|
|
2021-05-08 15:20:40 +00:00
|
|
|
ActionType type{};
|
2020-11-16 14:57:56 +00:00
|
|
|
|
|
|
|
std::string result_name;
|
|
|
|
DataTypePtr result_type;
|
|
|
|
|
|
|
|
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;
|
|
|
|
/// If function is a compiled statement.
|
|
|
|
bool is_function_compiled = false;
|
|
|
|
|
|
|
|
/// For COLUMN node and propagated constants.
|
|
|
|
ColumnPtr column;
|
|
|
|
/// 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;
|
2021-04-09 16:18:45 +00:00
|
|
|
|
2021-04-14 14:07:56 +00:00
|
|
|
void toTree(JSONBuilder::JSONMap & map) const;
|
2020-11-16 14:57:56 +00:00
|
|
|
};
|
|
|
|
|
2021-02-26 16:29:56 +00:00
|
|
|
/// NOTE: std::list is an implementation detail.
|
|
|
|
/// It allows to add and remove new nodes inplace without reallocation.
|
|
|
|
/// Raw pointers to nodes remain valid.
|
2020-11-16 14:57:56 +00:00
|
|
|
using Nodes = std::list<Node>;
|
|
|
|
|
|
|
|
private:
|
|
|
|
Nodes nodes;
|
2021-03-02 17:08:59 +00:00
|
|
|
NodeRawConstPtrs index;
|
|
|
|
NodeRawConstPtrs inputs;
|
2020-11-16 14:57:56 +00:00
|
|
|
|
2021-03-02 17:08:59 +00:00
|
|
|
bool project_input = false;
|
|
|
|
bool projected_output = false;
|
2020-11-16 14:57:56 +00:00
|
|
|
|
|
|
|
public:
|
|
|
|
ActionsDAG() = default;
|
2020-11-26 16:16:44 +00:00
|
|
|
ActionsDAG(ActionsDAG &&) = default;
|
2020-11-16 14:57:56 +00:00
|
|
|
ActionsDAG(const ActionsDAG &) = delete;
|
|
|
|
ActionsDAG & operator=(const ActionsDAG &) = delete;
|
2020-11-17 12:39:41 +00:00
|
|
|
explicit ActionsDAG(const NamesAndTypesList & inputs_);
|
|
|
|
explicit ActionsDAG(const ColumnsWithTypeAndName & inputs_);
|
2020-11-16 14:57:56 +00:00
|
|
|
|
|
|
|
const Nodes & getNodes() const { return nodes; }
|
2021-03-02 17:08:59 +00:00
|
|
|
const NodeRawConstPtrs & getIndex() const { return index; }
|
|
|
|
const NodeRawConstPtrs & getInputs() const { return inputs; }
|
2020-11-16 14:57:56 +00:00
|
|
|
|
|
|
|
NamesAndTypesList getRequiredColumns() const;
|
|
|
|
ColumnsWithTypeAndName getResultColumns() const;
|
|
|
|
NamesAndTypesList getNamesAndTypesList() const;
|
|
|
|
|
|
|
|
Names getNames() const;
|
|
|
|
std::string dumpNames() const;
|
|
|
|
std::string dumpDAG() const;
|
|
|
|
|
2021-03-02 17:08:59 +00:00
|
|
|
const Node & addInput(std::string name, DataTypePtr type);
|
|
|
|
const Node & addInput(ColumnWithTypeAndName column);
|
2021-03-18 11:37:45 +00:00
|
|
|
const Node & addColumn(ColumnWithTypeAndName column);
|
2021-03-02 17:08:59 +00:00
|
|
|
const Node & addAlias(const Node & child, std::string alias);
|
|
|
|
const Node & addArrayJoin(const Node & child, std::string result_name);
|
2020-11-16 14:57:56 +00:00
|
|
|
const Node & addFunction(
|
|
|
|
const FunctionOverloadResolverPtr & function,
|
2021-03-02 17:08:59 +00:00
|
|
|
NodeRawConstPtrs children,
|
|
|
|
std::string result_name);
|
2020-11-16 14:57:56 +00:00
|
|
|
|
2021-03-11 17:03:39 +00:00
|
|
|
/// Index can contain any column returned from DAG.
|
|
|
|
/// You may manually change it if needed.
|
2021-03-03 20:01:07 +00:00
|
|
|
NodeRawConstPtrs & getIndex() { return index; }
|
2021-03-11 17:03:39 +00:00
|
|
|
/// Find first column by name in index. This search is linear.
|
|
|
|
const Node & findInIndex(const std::string & name) const;
|
|
|
|
/// Same, but return nullptr if node not found.
|
|
|
|
const Node * tryFindInIndex(const std::string & name) const;
|
2021-03-18 11:33:07 +00:00
|
|
|
/// Find first node with the same name in index and replace it.
|
2021-03-11 17:03:39 +00:00
|
|
|
/// If was not found, add node to index end.
|
|
|
|
void addOrReplaceInIndex(const Node & node);
|
2021-03-03 20:01:07 +00:00
|
|
|
|
2020-11-16 14:57:56 +00:00
|
|
|
/// Call addAlias several times.
|
|
|
|
void addAliases(const NamesWithAliases & aliases);
|
|
|
|
/// Add alias actions and remove unused columns from index. Also specify result columns order in index.
|
|
|
|
void project(const NamesWithAliases & projection);
|
|
|
|
|
|
|
|
/// If column is not in index, try to find it in nodes and insert back into index.
|
|
|
|
bool tryRestoreColumn(const std::string & column_name);
|
2021-02-04 11:44:00 +00:00
|
|
|
/// Find column in result. Remove it from index.
|
|
|
|
/// If columns is in inputs and has no dependent nodes, remove it from inputs too.
|
|
|
|
/// Return true if column was removed from inputs.
|
|
|
|
bool removeUnusedResult(const std::string & column_name);
|
2020-11-16 14:57:56 +00:00
|
|
|
|
2021-03-05 15:21:14 +00:00
|
|
|
void projectInput(bool project = true) { project_input = project; }
|
2021-03-18 11:33:07 +00:00
|
|
|
bool isInputProjected() const { return project_input; }
|
|
|
|
bool isOutputProjected() const { return projected_output; }
|
2021-03-05 15:21:14 +00:00
|
|
|
|
2020-11-16 14:57:56 +00:00
|
|
|
void removeUnusedActions(const Names & required_names);
|
2021-03-09 16:54:06 +00:00
|
|
|
void removeUnusedActions(const NameSet & required_names);
|
2020-11-16 14:57:56 +00:00
|
|
|
|
2021-04-29 07:38:47 +00:00
|
|
|
NameSet
|
|
|
|
foldActionsByProjection(const NameSet & keys, const Block & projection_block_for_keys, const String & predicate_column_name = {});
|
2021-04-29 15:31:08 +00:00
|
|
|
void reorderAggregationKeysForProjection(const std::unordered_map<std::string_view, size_t> & key_names_pos_map);
|
2021-04-29 07:38:47 +00:00
|
|
|
void addAggregatesViaProjection(const Block & aggregates);
|
|
|
|
|
2020-11-16 14:57:56 +00:00
|
|
|
bool hasArrayJoin() const;
|
2020-11-27 09:50:56 +00:00
|
|
|
bool hasStatefulFunctions() const;
|
2021-01-28 11:00:24 +00:00
|
|
|
bool trivial() const; /// If actions has no functions or array join.
|
2020-11-16 14:57:56 +00:00
|
|
|
|
2021-03-05 13:56:44 +00:00
|
|
|
#if USE_EMBEDDED_COMPILER
|
2021-03-03 20:01:07 +00:00
|
|
|
void compileExpressions(size_t min_count_to_compile_expression);
|
2021-03-05 13:56:44 +00:00
|
|
|
#endif
|
2020-11-16 14:57:56 +00:00
|
|
|
|
|
|
|
ActionsDAGPtr clone() const;
|
|
|
|
|
2021-02-04 20:36:50 +00:00
|
|
|
/// For apply materialize() function for every output.
|
|
|
|
/// Also add aliases so the result names remain unchanged.
|
|
|
|
void addMaterializingOutputActions();
|
2020-11-17 14:51:05 +00:00
|
|
|
|
2021-03-18 11:33:07 +00:00
|
|
|
/// Apply materialize() function to node. Result node has the same name.
|
2021-03-09 16:54:06 +00:00
|
|
|
const Node & materializeNode(const Node & node);
|
|
|
|
|
2020-11-17 14:51:05 +00:00
|
|
|
enum class MatchColumnsMode
|
|
|
|
{
|
|
|
|
/// Require same number of columns in source and result. Match columns by corresponding positions, regardless to names.
|
|
|
|
Position,
|
|
|
|
/// Find columns in source by their names. Allow excessive columns in source.
|
|
|
|
Name,
|
|
|
|
};
|
|
|
|
|
2020-11-20 16:52:50 +00:00
|
|
|
/// Create ActionsDAG which converts block structure from source to result.
|
|
|
|
/// It is needed to convert result from different sources to the same structure, e.g. for UNION query.
|
|
|
|
/// Conversion should be possible with only usage of CAST function and renames.
|
2021-02-18 11:49:32 +00:00
|
|
|
/// @param ignore_constant_values - Do not check that constants are same. Use value from result_header.
|
|
|
|
/// @param add_casted_columns - Create new columns with converted values instead of replacing original.
|
2020-11-17 14:51:05 +00:00
|
|
|
static ActionsDAGPtr makeConvertingActions(
|
|
|
|
const ColumnsWithTypeAndName & source,
|
|
|
|
const ColumnsWithTypeAndName & result,
|
|
|
|
MatchColumnsMode mode,
|
2021-02-18 11:49:32 +00:00
|
|
|
bool ignore_constant_values = false,
|
|
|
|
bool add_casted_columns = false,
|
|
|
|
NameToNameMap * new_names = nullptr);
|
2020-11-17 14:51:05 +00:00
|
|
|
|
2021-02-04 14:25:11 +00:00
|
|
|
/// Create expression which add const column and then materialize it.
|
|
|
|
static ActionsDAGPtr makeAddingColumnActions(ColumnWithTypeAndName column);
|
|
|
|
|
2021-01-19 10:03:25 +00:00
|
|
|
/// Create ActionsDAG which represents expression equivalent to applying first and second actions consequently.
|
2020-12-01 11:19:03 +00:00
|
|
|
/// Is used to replace `(first -> second)` expression chain to single `merge(first, second)` expression.
|
|
|
|
/// If first.settings.project_input is set, then outputs of `first` must include inputs of `second`.
|
|
|
|
/// Otherwise, any two actions may be combined.
|
|
|
|
static ActionsDAGPtr merge(ActionsDAG && first, ActionsDAG && second);
|
2020-11-26 16:16:44 +00:00
|
|
|
|
2021-02-04 11:44:00 +00:00
|
|
|
using SplitResult = std::pair<ActionsDAGPtr, ActionsDAGPtr>;
|
|
|
|
|
2021-01-19 10:03:25 +00:00
|
|
|
/// Split ActionsDAG into two DAGs, where first part contains all nodes from split_nodes and their children.
|
|
|
|
/// Execution of first then second parts on block is equivalent to execution of initial DAG.
|
|
|
|
/// First DAG and initial DAG have equal inputs, second DAG and initial DAG has equal index (outputs).
|
|
|
|
/// Second DAG inputs may contain less inputs then first DAG (but also include other columns).
|
2021-02-04 11:44:00 +00:00
|
|
|
SplitResult split(std::unordered_set<const Node *> split_nodes) const;
|
2021-01-18 14:59:59 +00:00
|
|
|
|
2021-01-19 10:03:25 +00:00
|
|
|
/// Splits actions into two parts. Returned first half may be swapped with ARRAY JOIN.
|
2021-02-04 11:44:00 +00:00
|
|
|
SplitResult splitActionsBeforeArrayJoin(const NameSet & array_joined_columns) const;
|
2021-01-19 10:03:25 +00:00
|
|
|
|
|
|
|
/// Splits actions into two parts. First part has minimal size sufficient for calculation of column_name.
|
|
|
|
/// Index of initial actions must contain column_name.
|
2021-02-04 11:44:00 +00:00
|
|
|
SplitResult splitActionsForFilter(const std::string & column_name) const;
|
2021-01-19 10:03:25 +00:00
|
|
|
|
2021-02-10 16:26:49 +00:00
|
|
|
/// Create actions which may calculate part of filter using only available_inputs.
|
|
|
|
/// If nothing may be calculated, returns nullptr.
|
|
|
|
/// Otherwise, return actions which inputs are from available_inputs.
|
2021-04-08 09:54:39 +00:00
|
|
|
/// Returned actions add single column which may be used for filter. Added column will be the first one.
|
2021-02-10 16:26:49 +00:00
|
|
|
/// Also, replace some nodes of current inputs to constant 1 in case they are filtered.
|
2021-03-17 16:08:46 +00:00
|
|
|
///
|
2021-03-17 18:42:42 +00:00
|
|
|
/// @param all_inputs should contain inputs from previous step, which will be used for result actions.
|
2021-03-17 16:08:46 +00:00
|
|
|
/// It is expected that all_inputs contain columns from available_inputs.
|
|
|
|
/// This parameter is needed to enforce result actions save columns order in block.
|
2021-03-17 18:42:42 +00:00
|
|
|
/// Otherwise for some queries, e.g. with GROUP BY, columns will be mixed.
|
2021-03-17 16:08:46 +00:00
|
|
|
/// Example: SELECT sum(x), y, z FROM tab WHERE z > 0 and sum(x) > 0
|
|
|
|
/// Pushed condition: z > 0
|
|
|
|
/// GROUP BY step will transform columns `x, y, z` -> `sum(x), y, z`
|
|
|
|
/// If we just add filter step with actions `z -> z > 0` before GROUP BY,
|
2021-04-08 09:54:39 +00:00
|
|
|
/// columns will be transformed like `x, y, z` -> `z > 0, z, x, y` -(remove filter)-> `z, x, y`.
|
2021-03-17 16:08:46 +00:00
|
|
|
/// To avoid it, add inputs from `all_inputs` list,
|
2021-04-08 09:54:39 +00:00
|
|
|
/// so actions `x, y, z -> z > 0, x, y, z` -(remove filter)-> `x, y, z` will not change columns order.
|
2021-03-17 16:08:46 +00:00
|
|
|
ActionsDAGPtr cloneActionsForFilterPushDown(
|
|
|
|
const std::string & filter_name,
|
|
|
|
bool can_remove_filter,
|
|
|
|
const Names & available_inputs,
|
|
|
|
const ColumnsWithTypeAndName & all_inputs);
|
2021-02-10 16:26:49 +00:00
|
|
|
|
2020-11-16 14:57:56 +00:00
|
|
|
private:
|
2021-03-02 17:08:59 +00:00
|
|
|
Node & addNode(Node node);
|
2020-11-16 14:57:56 +00:00
|
|
|
|
2021-02-10 16:26:49 +00:00
|
|
|
void removeUnusedActions(bool allow_remove_inputs = true);
|
2020-11-16 14:57:56 +00:00
|
|
|
|
2021-03-05 13:56:44 +00:00
|
|
|
#if USE_EMBEDDED_COMPILER
|
2021-03-03 20:01:07 +00:00
|
|
|
void compileFunctions(size_t min_count_to_compile_expression);
|
2021-03-05 13:56:44 +00:00
|
|
|
#endif
|
2021-02-20 16:13:36 +00:00
|
|
|
|
2021-03-11 17:03:39 +00:00
|
|
|
static ActionsDAGPtr cloneActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs);
|
2020-11-16 14:57:56 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|