ClickHouse/src/Interpreters/ExpressionActions.h

279 lines
8.6 KiB
C++
Raw Normal View History

#pragma once
#include <Core/Block.h>
2020-11-03 11:28:28 +00:00
#include <Core/ColumnNumbers.h>
2020-11-16 14:57:56 +00:00
#include <Interpreters/ActionsDAG.h>
2020-11-03 11:28:28 +00:00
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 TableJoin;
class IJoin;
using JoinPtr = std::shared_ptr<IJoin>;
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-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-17 12:34:31 +00:00
size_t pos = 0;
2020-11-09 15:01:08 +00:00
/// True if there is another action which will use this column.
/// Otherwise column will be removed.
2020-11-17 12:34:31 +00:00
bool needed_later = false;
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-17 12:34:31 +00:00
/// This map helps to find input position bu it's name.
/// Key is a view to input::result_name.
/// Result is a list because it is allowed for inputs to have same names.
using NameToInputMap = std::unordered_map<std::string_view, std::list<size_t>>;
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-17 12:34:31 +00:00
NameToInputMap input_positions;
2020-11-03 11:28:28 +00:00
ColumnNumbers result_positions;
2020-10-13 08:16:47 +00:00
Block sample_block;
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-11 16:52:27 +00:00
ActionsDAGPtr actions_dag;
2020-08-19 19:33:49 +00:00
2020-11-11 16:52:27 +00:00
explicit ExpressionActionsStep(ActionsDAGPtr actions_dag_, Names required_output_ = Names())
2020-08-19 19:33:49 +00:00
: Step(std::move(required_output_))
2020-11-11 16:52:27 +00:00
, actions_dag(std::move(actions_dag_))
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-11 16:52:27 +00:00
return actions_dag->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-11 16:52:27 +00:00
return actions_dag->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-11 16:52:27 +00:00
if (!actions_dag->getSettings().projected_output)
actions_dag->removeUnusedActions(required_output_);
2020-08-19 19:33:49 +00:00
}
void prependProjectInput() const override
{
2020-11-11 16:52:27 +00:00
actions_dag->projectInput();
2020-08-19 19:33:49 +00:00
}
std::string dump() const override
{
2020-11-11 16:52:27 +00:00
return actions_dag->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-11 16:52:27 +00:00
return typeid_cast<ExpressionActionsStep *>(steps.back().get())->actions_dag;
}
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;
};
}