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>
|
2020-11-03 11:28:28 +00:00
|
|
|
#include <Core/ColumnNumbers.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-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>
|
|
|
|
|
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>;
|
|
|
|
|
2020-08-12 13:30:02 +00:00
|
|
|
class ExpressionActions;
|
|
|
|
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
class ActionsDAG;
|
|
|
|
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
|
|
|
|
|
2020-09-10 07:30:03 +00:00
|
|
|
class ActionsDAG
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
|
|
|
|
enum class Type
|
|
|
|
{
|
|
|
|
/// 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;
|
|
|
|
|
|
|
|
Type type;
|
|
|
|
|
|
|
|
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-03 11:28:28 +00:00
|
|
|
class Index
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
Node *& operator[](std::string_view key)
|
|
|
|
{
|
|
|
|
auto res = map.emplace(key, list.end());
|
|
|
|
if (res.second)
|
|
|
|
res.first->second = list.emplace(list.end(), nullptr);
|
|
|
|
|
|
|
|
return *res.first->second;
|
|
|
|
}
|
|
|
|
|
|
|
|
void swap(Index & other)
|
|
|
|
{
|
|
|
|
list.swap(other.list);
|
|
|
|
map.swap(other.map);
|
|
|
|
}
|
|
|
|
|
|
|
|
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); }
|
|
|
|
void remove(Node * node)
|
|
|
|
{
|
|
|
|
auto it = map.find(node->result_name);
|
|
|
|
if (it != map.end())
|
|
|
|
return;
|
|
|
|
|
|
|
|
list.erase(it->second);
|
|
|
|
map.erase(it);
|
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
|
|
|
std::list<Node *> list;
|
|
|
|
std::unordered_map<std::string_view, std::list<Node *>::iterator> map;
|
|
|
|
};
|
|
|
|
|
|
|
|
using Nodes = std::list<Node>;
|
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-03 11:28:28 +00:00
|
|
|
size_t max_temporary_columns = 0;
|
|
|
|
size_t max_temporary_non_const_columns = 0;
|
|
|
|
|
|
|
|
#if USE_EMBEDDED_COMPILER
|
|
|
|
std::shared_ptr<CompiledExpressionCache> compilation_cache;
|
|
|
|
#endif
|
|
|
|
|
|
|
|
bool project_input = false;
|
|
|
|
bool projected_output = false;
|
|
|
|
|
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 dump() const;
|
|
|
|
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);
|
|
|
|
/// Adds alias actions and removes unused columns from index.
|
|
|
|
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);
|
|
|
|
|
|
|
|
void projectInput() { project_input = true; }
|
|
|
|
void removeUnusedActions(const Names & required_names);
|
2020-11-03 13:30:55 +00:00
|
|
|
ExpressionActionsPtr buildExpressions() 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;
|
|
|
|
bool empty() const;
|
|
|
|
bool projectedOutput() const { return projected_output; }
|
|
|
|
|
|
|
|
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>();
|
|
|
|
actions->max_temporary_columns = max_temporary_columns;
|
|
|
|
actions->max_temporary_non_const_columns = max_temporary_non_const_columns;
|
|
|
|
|
|
|
|
#if USE_EMBEDDED_COMPILER
|
|
|
|
actions->compilation_cache = compilation_cache;
|
|
|
|
#endif
|
|
|
|
return actions;
|
|
|
|
}
|
|
|
|
|
|
|
|
ExpressionActionsPtr linearizeActions() const;
|
|
|
|
void removeUnusedActions(const std::vector<Node *> & required_nodes);
|
|
|
|
void addAliases(const NamesWithAliases & aliases, std::vector<Node *> & result_nodes);
|
2020-09-10 07:30:03 +00:00
|
|
|
};
|
|
|
|
|
2020-09-11 12:24:41 +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
|
|
|
|
{
|
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;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
struct Argument
|
|
|
|
{
|
|
|
|
size_t pos;
|
|
|
|
bool remove;
|
|
|
|
};
|
|
|
|
|
|
|
|
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
|
|
|
};
|
2014-06-12 18:41:09 +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
|
|
|
struct ExecutionContext
|
|
|
|
{
|
2020-11-03 11:28:28 +00:00
|
|
|
ColumnsWithTypeAndName & inputs;
|
2020-10-07 18:37:27 +00:00
|
|
|
ColumnsWithTypeAndName columns;
|
2020-11-03 11:28:28 +00:00
|
|
|
std::vector<ssize_t> inputs_pos;
|
2020-10-07 18:37:27 +00:00
|
|
|
size_t num_rows;
|
|
|
|
};
|
|
|
|
|
|
|
|
std::list<Node> nodes;
|
|
|
|
Actions actions;
|
2020-10-13 08:16:47 +00:00
|
|
|
size_t num_columns;
|
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;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
/// This flag means that all columns except input will be removed from block before execution.
|
|
|
|
bool project_input = false;
|
|
|
|
|
|
|
|
size_t max_temporary_non_const_columns = 0;
|
|
|
|
|
|
|
|
friend class ActionsDAG;
|
|
|
|
|
2020-10-07 18:37:27 +00:00
|
|
|
public:
|
|
|
|
~ExpressionActions();
|
2020-11-03 11:28:28 +00:00
|
|
|
ExpressionActions() = default;
|
|
|
|
ExpressionActions(const ExpressionActions &) = delete;
|
|
|
|
ExpressionActions & operator=(const ExpressionActions &) = delete;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
const Actions & getActions() const { return actions; }
|
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.
|
2020-11-03 11:28:28 +00:00
|
|
|
void projectInput() { project_input = true; }
|
2020-08-12 13:30:02 +00:00
|
|
|
|
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).
|
2020-10-07 18:37:27 +00:00
|
|
|
// void finalize(const Names & output_columns);
|
2017-02-09 17:29:36 +00:00
|
|
|
|
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; }
|
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;
|
2020-11-03 11:28:28 +00:00
|
|
|
void execute(Block & block, size_t & num_rows, bool dry_run = false) const;
|
2013-05-24 10:49:19 +00:00
|
|
|
|
2020-09-08 10:40:53 +00:00
|
|
|
bool hasArrayJoin() 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.
|
2020-10-13 08:16:47 +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-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;
|
2018-09-07 20:17:40 +00:00
|
|
|
|
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:
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2020-10-13 08:16:47 +00:00
|
|
|
void checkLimits(ExecutionContext & execution_context) const;
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2020-10-13 08:16:47 +00:00
|
|
|
static void executeAction(const Action & action, ExecutionContext & execution_context, bool dry_run);
|
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 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.
|
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-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-03 11:28:28 +00:00
|
|
|
if (!actions->projectedOutput())
|
|
|
|
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-03 11:28:28 +00:00
|
|
|
return actions->dump();
|
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"; }
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
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"; }
|
|
|
|
};
|
2014-06-12 18:41:09 +00:00
|
|
|
|
2020-08-19 19:33:49 +00:00
|
|
|
using StepPtr = std::unique_ptr<Step>;
|
|
|
|
using Steps = std::vector<StepPtr>;
|
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
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
void addStep(NameSet non_constant_inputs = {});
|
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-11-03 11:28:28 +00:00
|
|
|
ActionsDAGPtr 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-11-03 11:28:28 +00:00
|
|
|
return typeid_cast<ExpressionActionsStep *>(steps.back().get())->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
|
|
|
|
2020-08-19 19:58:23 +00:00
|
|
|
return *steps.back();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2014-04-30 19:19:29 +00:00
|
|
|
|
2020-06-25 23:05:47 +00:00
|
|
|
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-06-25 23:05:47 +00:00
|
|
|
}
|
|
|
|
|
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
|
|
|
}
|