ClickHouse/src/Interpreters/ActionsVisitor.h

215 lines
6.9 KiB
C++
Raw Normal View History

#pragma once
#include <Interpreters/Context_fwd.h>
#include <Interpreters/InDepthNodeVisitor.h>
2019-01-18 16:30:35 +00:00
#include <Interpreters/PreparedSets.h>
#include <Interpreters/SubqueryForSet.h>
#include <Parsers/IAST.h>
namespace DB
{
class ASTFunction;
class ExpressionActions;
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
2020-09-10 07:30:03 +00:00
class ActionsDAG;
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
2020-09-30 13:31:06 +00:00
class IFunctionOverloadResolver;
using FunctionOverloadResolverPtr = std::shared_ptr<IFunctionOverloadResolver>;
/// The case of an explicit enumeration of values.
SetPtr makeExplicitSet(
2020-09-30 11:45:47 +00:00
const ASTFunction * node, const ActionsDAG & actions, bool create_ordered_set,
ContextPtr context, const SizeLimits & limits, PreparedSets & prepared_sets);
/** Create a block for set from expression.
* 'set_element_types' - types of what are on the left hand side of IN.
* 'right_arg' - list of values: 1, 2, 3 or list of tuples: (1, 2), (3, 4), (5, 6).
*
* We need special implementation for ASTFunction, because in case, when we interpret
* large tuple or array as function, `evaluateConstantExpression` works extremely slow.
*
* Note: this and following functions are used in third-party applications in Arcadia, so
* they should be declared in header file.
*
*/
Block createBlockForSet(
const DataTypePtr & left_arg_type,
const std::shared_ptr<ASTFunction> & right_arg,
const DataTypes & set_element_types,
ContextPtr context);
/** Create a block for set from literal.
* 'set_element_types' - types of what are on the left hand side of IN.
* 'right_arg' - Literal - Tuple or Array.
*/
Block createBlockForSet(
const DataTypePtr & left_arg_type,
const ASTPtr & right_arg,
const DataTypes & set_element_types,
ContextPtr context);
/** For ActionsVisitor
* A stack of ExpressionActions corresponding to nested lambda expressions.
* The new action should be added to the highest possible level.
* For example, in the expression "select arrayMap(x -> x + column1 * column2, array1)"
* calculation of the product must be done outside the lambda expression (it does not depend on x),
* and the calculation of the sum is inside (depends on x).
*/
struct ScopeStack : WithContext
{
2021-03-03 20:01:07 +00:00
class Index;
using IndexPtr = std::unique_ptr<Index>;
2020-09-13 13:51:31 +00:00
struct Level
{
2020-11-11 16:52:27 +00:00
ActionsDAGPtr actions_dag;
2021-03-03 20:01:07 +00:00
IndexPtr index;
2020-09-13 13:51:31 +00:00
NameSet inputs;
2021-03-03 20:01:07 +00:00
Level();
Level(Level &&);
~Level();
2020-09-13 13:51:31 +00:00
};
using Levels = std::vector<Level>;
Levels stack;
ScopeStack(ActionsDAGPtr actions_dag, ContextPtr context_);
void pushLevel(const NamesAndTypesList & input_columns);
size_t getColumnLevel(const std::string & name);
2020-09-10 16:01:41 +00:00
void addColumn(ColumnWithTypeAndName column);
2020-09-10 07:30:03 +00:00
void addAlias(const std::string & name, std::string alias);
2020-11-03 11:28:28 +00:00
void addArrayJoin(const std::string & source_name, std::string result_name);
2020-09-10 16:01:41 +00:00
void addFunction(
const FunctionOverloadResolverPtr & function,
const Names & argument_names,
2020-11-03 11:28:28 +00:00
std::string result_name);
2020-09-10 07:30:03 +00:00
ActionsDAGPtr popLevel();
2020-09-30 11:45:47 +00:00
const ActionsDAG & getLastActions() const;
2021-03-03 20:01:07 +00:00
const Index & getLastActionsIndex() const;
2020-09-10 16:01:41 +00:00
std::string dumpNames() const;
};
class ASTIdentifier;
class ASTFunction;
class ASTLiteral;
/// Collect ExpressionAction from AST. Returns PreparedSets and SubqueriesForSets too.
class ActionsMatcher
{
public:
using Visitor = ConstInDepthNodeVisitor<ActionsMatcher, true>;
struct Data : public WithContext
2019-08-14 16:53:30 +00:00
{
SizeLimits set_size_limit;
size_t subquery_depth;
const NamesAndTypesList & source_columns;
PreparedSets & prepared_sets;
SubqueriesForSets & subqueries_for_sets;
bool no_subqueries;
2020-01-27 21:57:44 +00:00
bool no_makeset;
bool only_consts;
2020-11-02 12:07:01 +00:00
bool create_source_for_in;
size_t visit_depth;
ScopeStack actions_stack;
2020-04-01 12:16:39 +00:00
/*
* Remember the last unique column suffix to avoid quadratic behavior
* when we add lots of column with same prefix. One counter for all
* prefixes is good enough.
*/
int next_unique_suffix;
Data(
ContextPtr context_,
SizeLimits set_size_limit_,
size_t subquery_depth_,
const NamesAndTypesList & source_columns_,
ActionsDAGPtr actions_dag,
PreparedSets & prepared_sets_,
SubqueriesForSets & subqueries_for_sets_,
bool no_subqueries_,
bool no_makeset_,
bool only_consts_,
bool create_source_for_in_);
2020-09-10 07:30:03 +00:00
/// Does result of the calculation already exists in the block.
2020-09-30 12:32:37 +00:00
bool hasColumn(const String & column_name) const;
2020-09-10 16:01:41 +00:00
void addColumn(ColumnWithTypeAndName column)
{
actions_stack.addColumn(std::move(column));
}
2020-09-10 07:30:03 +00:00
void addAlias(const std::string & name, std::string alias)
2019-08-15 14:22:33 +00:00
{
2020-09-10 07:30:03 +00:00
actions_stack.addAlias(name, std::move(alias));
2019-08-15 14:22:33 +00:00
}
2020-09-10 07:30:03 +00:00
void addArrayJoin(const std::string & source_name, std::string result_name)
{
2020-11-03 11:28:28 +00:00
actions_stack.addArrayJoin(source_name, std::move(result_name));
}
2020-04-14 17:41:06 +00:00
2020-09-10 16:01:41 +00:00
void addFunction(const FunctionOverloadResolverPtr & function,
const Names & argument_names,
std::string result_name)
{
2020-11-03 11:28:28 +00:00
actions_stack.addFunction(function, argument_names, std::move(result_name));
2020-09-10 16:01:41 +00:00
}
ActionsDAGPtr getActions()
{
return actions_stack.popLevel();
}
2020-04-14 17:41:06 +00:00
/*
* Generate a column name that is not present in the sample block, using
* the given prefix and an optional numeric suffix.
*/
String getUniqueName(const String & prefix)
{
auto result = prefix;
// First, try the name without any suffix, because it is currently
// used both as a display name and a column id.
2020-09-10 16:01:41 +00:00
while (hasColumn(result))
2020-04-14 17:41:06 +00:00
{
result = prefix + "_" + toString(next_unique_suffix);
++next_unique_suffix;
}
return result;
}
};
static void visit(const ASTPtr & ast, Data & data);
static bool needChildVisit(const ASTPtr & node, const ASTPtr & child);
private:
static void visit(const ASTIdentifier & identifier, const ASTPtr & ast, Data & data);
static void visit(const ASTFunction & node, const ASTPtr & ast, Data & data);
static void visit(const ASTLiteral & literal, const ASTPtr & ast, Data & data);
2020-10-21 18:17:27 +00:00
static void visit(ASTExpressionList & expression_list, const ASTPtr & ast, Data & data);
2019-10-27 18:12:40 +00:00
static SetPtr makeSet(const ASTFunction & node, Data & data, bool no_subqueries);
2020-10-21 18:17:27 +00:00
static ASTs doUntuple(const ASTFunction * function, ActionsMatcher::Data & data);
static std::optional<NameAndTypePair> getNameAndTypeFromAST(const ASTPtr & ast, Data & data);
};
using ActionsVisitor = ActionsMatcher::Visitor;
}