mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #6499 from 4ertus2/refactoring
ExpressionAnalyzer and ActionsVisitor refactoring
This commit is contained in:
commit
bd2c5e14cd
@ -39,7 +39,7 @@ try
|
||||
|
||||
NamesAndTypesList source_columns = {{"number", std::make_shared<DataTypeUInt64>()}};
|
||||
auto syntax_result = SyntaxAnalyzer(context, {}).analyze(ast, source_columns);
|
||||
ExpressionAnalyzer analyzer(ast, syntax_result, context);
|
||||
SelectQueryExpressionAnalyzer analyzer(ast, syntax_result, context);
|
||||
ExpressionActionsChain chain(context);
|
||||
analyzer.appendSelect(chain, false);
|
||||
analyzer.appendProjectResult(chain);
|
||||
|
@ -44,7 +44,7 @@ try
|
||||
|
||||
NamesAndTypesList source_columns = {{"number", std::make_shared<DataTypeUInt64>()}};
|
||||
auto syntax_result = SyntaxAnalyzer(context, {}).analyze(ast, source_columns);
|
||||
ExpressionAnalyzer analyzer(ast, syntax_result, context);
|
||||
SelectQueryExpressionAnalyzer analyzer(ast, syntax_result, context);
|
||||
ExpressionActionsChain chain(context);
|
||||
analyzer.appendSelect(chain, false);
|
||||
analyzer.appendProjectResult(chain);
|
||||
|
@ -1,3 +1,5 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionsMiscellaneous.h>
|
||||
|
||||
@ -19,8 +21,6 @@
|
||||
#include <Storages/StorageSet.h>
|
||||
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Parsers/DumpASTNode.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
@ -228,346 +228,350 @@ const Block & ScopeStack::getSampleBlock() const
|
||||
return stack.back().actions->getSampleBlock();
|
||||
}
|
||||
|
||||
|
||||
ActionsVisitor::ActionsVisitor(
|
||||
const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_,
|
||||
const NamesAndTypesList & source_columns_, const ExpressionActionsPtr & actions,
|
||||
PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_,
|
||||
bool no_subqueries_, bool only_consts_, bool no_storage_or_local_, std::ostream * ostr_)
|
||||
: context(context_),
|
||||
set_size_limit(set_size_limit_),
|
||||
subquery_depth(subquery_depth_),
|
||||
source_columns(source_columns_),
|
||||
prepared_sets(prepared_sets_),
|
||||
subqueries_for_sets(subqueries_for_sets_),
|
||||
no_subqueries(no_subqueries_),
|
||||
only_consts(only_consts_),
|
||||
no_storage_or_local(no_storage_or_local_),
|
||||
visit_depth(0),
|
||||
ostr(ostr_),
|
||||
actions_stack(actions, context)
|
||||
struct CachedColumnName
|
||||
{
|
||||
String cached;
|
||||
|
||||
const String & get(const ASTPtr & ast)
|
||||
{
|
||||
if (cached.empty())
|
||||
cached = ast->getColumnName();
|
||||
return cached;
|
||||
}
|
||||
};
|
||||
|
||||
bool ActionsMatcher::needChildVisit(const ASTPtr & node, const ASTPtr & child)
|
||||
{
|
||||
/// Visit children themself
|
||||
if (node->as<ASTIdentifier>() ||
|
||||
node->as<ASTFunction>() ||
|
||||
node->as<ASTLiteral>())
|
||||
return false;
|
||||
|
||||
/// Do not go to FROM, JOIN, UNION.
|
||||
if (child->as<ASTTableExpression>() ||
|
||||
child->as<ASTSelectQuery>())
|
||||
return false;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
void ActionsVisitor::visit(const ASTPtr & ast)
|
||||
void ActionsMatcher::visit(const ASTPtr & ast, Data & data)
|
||||
{
|
||||
DumpASTNode dump(*ast, ostr, visit_depth, "getActions");
|
||||
if (const auto * identifier = ast->as<ASTIdentifier>())
|
||||
visit(*identifier, ast, data);
|
||||
else if (const auto * node = ast->as<ASTFunction>())
|
||||
visit(*node, ast, data);
|
||||
else if (const auto * literal = ast->as<ASTLiteral>())
|
||||
visit(*literal, ast, data);
|
||||
}
|
||||
|
||||
String ast_column_name;
|
||||
auto getColumnName = [&ast, &ast_column_name]()
|
||||
{
|
||||
if (ast_column_name.empty())
|
||||
ast_column_name = ast->getColumnName();
|
||||
|
||||
return ast_column_name;
|
||||
};
|
||||
|
||||
/// If the result of the calculation already exists in the block.
|
||||
if ((ast->as<ASTFunction>() || ast->as<ASTLiteral>()) && actions_stack.getSampleBlock().has(getColumnName()))
|
||||
void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr & ast, Data & data)
|
||||
{
|
||||
CachedColumnName column_name;
|
||||
if (data.hasColumn(column_name.get(ast)))
|
||||
return;
|
||||
|
||||
if (const auto * identifier = ast->as<ASTIdentifier>())
|
||||
if (!data.only_consts)
|
||||
{
|
||||
if (!only_consts && !actions_stack.getSampleBlock().has(getColumnName()))
|
||||
{
|
||||
/// The requested column is not in the block.
|
||||
/// If such a column exists in the table, then the user probably forgot to surround it with an aggregate function or add it to GROUP BY.
|
||||
/// The requested column is not in the block.
|
||||
/// If such a column exists in the table, then the user probably forgot to surround it with an aggregate function or add it to GROUP BY.
|
||||
|
||||
bool found = false;
|
||||
for (const auto & column_name_type : source_columns)
|
||||
if (column_name_type.name == getColumnName())
|
||||
found = true;
|
||||
bool found = false;
|
||||
for (const auto & column_name_type : data.source_columns)
|
||||
if (column_name_type.name == column_name.get(ast))
|
||||
found = true;
|
||||
|
||||
if (found)
|
||||
throw Exception("Column " + getColumnName() + " is not under aggregate function and not in GROUP BY.",
|
||||
ErrorCodes::NOT_AN_AGGREGATE);
|
||||
if (found)
|
||||
throw Exception("Column " + column_name.get(ast) + " is not under aggregate function and not in GROUP BY.",
|
||||
ErrorCodes::NOT_AN_AGGREGATE);
|
||||
|
||||
/// Special check for WITH statement alias. Add alias action to be able to use this alias.
|
||||
if (identifier->prefer_alias_to_column_name && !identifier->alias.empty())
|
||||
actions_stack.addAction(ExpressionAction::addAliases({{identifier->name, identifier->alias}}));
|
||||
}
|
||||
/// Special check for WITH statement alias. Add alias action to be able to use this alias.
|
||||
if (identifier.prefer_alias_to_column_name && !identifier.alias.empty())
|
||||
data.addAction(ExpressionAction::addAliases({{identifier.name, identifier.alias}}));
|
||||
}
|
||||
else if (const auto * node = ast->as<ASTFunction>())
|
||||
}
|
||||
|
||||
void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & data)
|
||||
{
|
||||
CachedColumnName column_name;
|
||||
if (data.hasColumn(column_name.get(ast)))
|
||||
return;
|
||||
|
||||
if (node.name == "lambda")
|
||||
throw Exception("Unexpected lambda expression", ErrorCodes::UNEXPECTED_EXPRESSION);
|
||||
|
||||
/// Function arrayJoin.
|
||||
if (node.name == "arrayJoin")
|
||||
{
|
||||
if (node->name == "lambda")
|
||||
throw Exception("Unexpected lambda expression", ErrorCodes::UNEXPECTED_EXPRESSION);
|
||||
if (node.arguments->children.size() != 1)
|
||||
throw Exception("arrayJoin requires exactly 1 argument", ErrorCodes::TYPE_MISMATCH);
|
||||
|
||||
/// Function arrayJoin.
|
||||
if (node->name == "arrayJoin")
|
||||
ASTPtr arg = node.arguments->children.at(0);
|
||||
visit(arg, data);
|
||||
if (!data.only_consts)
|
||||
{
|
||||
if (node->arguments->children.size() != 1)
|
||||
throw Exception("arrayJoin requires exactly 1 argument", ErrorCodes::TYPE_MISMATCH);
|
||||
String result_name = column_name.get(ast);
|
||||
data.addAction(ExpressionAction::copyColumn(arg->getColumnName(), result_name));
|
||||
NameSet joined_columns;
|
||||
joined_columns.insert(result_name);
|
||||
data.addAction(ExpressionAction::arrayJoin(joined_columns, false, data.context));
|
||||
}
|
||||
|
||||
ASTPtr arg = node->arguments->children.at(0);
|
||||
visit(arg);
|
||||
if (!only_consts)
|
||||
return;
|
||||
}
|
||||
|
||||
SetPtr prepared_set;
|
||||
if (functionIsInOrGlobalInOperator(node.name))
|
||||
{
|
||||
/// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything).
|
||||
visit(node.arguments->children.at(0), data);
|
||||
|
||||
if (!data.no_subqueries)
|
||||
{
|
||||
/// Transform tuple or subquery into a set.
|
||||
prepared_set = makeSet(node, data);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!data.only_consts)
|
||||
{
|
||||
String result_name = getColumnName();
|
||||
actions_stack.addAction(ExpressionAction::copyColumn(arg->getColumnName(), result_name));
|
||||
NameSet joined_columns;
|
||||
joined_columns.insert(result_name);
|
||||
actions_stack.addAction(ExpressionAction::arrayJoin(joined_columns, false, context));
|
||||
}
|
||||
/// We are in the part of the tree that we are not going to compute. You just need to define types.
|
||||
/// Do not subquery and create sets. We treat "IN" as "ignoreExceptNull" function.
|
||||
|
||||
data.addAction(ExpressionAction::applyFunction(
|
||||
FunctionFactory::instance().get("ignoreExceptNull", data.context),
|
||||
{ node.arguments->children.at(0)->getColumnName() },
|
||||
column_name.get(ast)));
|
||||
}
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
SetPtr prepared_set;
|
||||
if (functionIsInOrGlobalInOperator(node->name))
|
||||
/// A special function `indexHint`. Everything that is inside it is not calculated
|
||||
/// (and is used only for index analysis, see KeyCondition).
|
||||
if (node.name == "indexHint")
|
||||
{
|
||||
data.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName(
|
||||
ColumnConst::create(ColumnUInt8::create(1, 1), 1), std::make_shared<DataTypeUInt8>(),
|
||||
column_name.get(ast))));
|
||||
return;
|
||||
}
|
||||
|
||||
if (AggregateFunctionFactory::instance().isAggregateFunctionName(node.name))
|
||||
return;
|
||||
|
||||
/// Context object that we pass to function should live during query.
|
||||
const Context & function_context = data.context.hasQueryContext()
|
||||
? data.context.getQueryContext()
|
||||
: data.context;
|
||||
|
||||
FunctionBuilderPtr function_builder;
|
||||
try
|
||||
{
|
||||
function_builder = FunctionFactory::instance().get(node.name, function_context);
|
||||
}
|
||||
catch (DB::Exception & e)
|
||||
{
|
||||
auto hints = AggregateFunctionFactory::instance().getHints(node.name);
|
||||
if (!hints.empty())
|
||||
e.addMessage("Or unknown aggregate function " + node.name + ". Maybe you meant: " + toString(hints));
|
||||
e.rethrow();
|
||||
}
|
||||
|
||||
Names argument_names;
|
||||
DataTypes argument_types;
|
||||
bool arguments_present = true;
|
||||
|
||||
/// If the function has an argument-lambda expression, you need to determine its type before the recursive call.
|
||||
bool has_lambda_arguments = false;
|
||||
|
||||
for (size_t arg = 0; arg < node.arguments->children.size(); ++arg)
|
||||
{
|
||||
auto & child = node.arguments->children[arg];
|
||||
auto child_column_name = child->getColumnName();
|
||||
|
||||
const auto * lambda = child->as<ASTFunction>();
|
||||
if (lambda && lambda->name == "lambda")
|
||||
{
|
||||
/// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything).
|
||||
visit(node->arguments->children.at(0));
|
||||
/// If the argument is a lambda expression, just remember its approximate type.
|
||||
if (lambda->arguments->children.size() != 2)
|
||||
throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!no_subqueries)
|
||||
const auto * lambda_args_tuple = lambda->arguments->children.at(0)->as<ASTFunction>();
|
||||
|
||||
if (!lambda_args_tuple || lambda_args_tuple->name != "tuple")
|
||||
throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH);
|
||||
|
||||
has_lambda_arguments = true;
|
||||
argument_types.emplace_back(std::make_shared<DataTypeFunction>(DataTypes(lambda_args_tuple->arguments->children.size())));
|
||||
/// Select the name in the next cycle.
|
||||
argument_names.emplace_back();
|
||||
}
|
||||
else if (functionIsInOrGlobalInOperator(node.name) && arg == 1 && prepared_set)
|
||||
{
|
||||
ColumnWithTypeAndName column;
|
||||
column.type = std::make_shared<DataTypeSet>();
|
||||
|
||||
/// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name,
|
||||
/// so that sets with the same literal representation do not fuse together (they can have different types).
|
||||
if (!prepared_set->empty())
|
||||
column.name = getUniqueName(data.getSampleBlock(), "__set");
|
||||
else
|
||||
column.name = child_column_name;
|
||||
|
||||
if (!data.hasColumn(column.name))
|
||||
{
|
||||
/// Transform tuple or subquery into a set.
|
||||
prepared_set = makeSet(node, actions_stack.getSampleBlock());
|
||||
column.column = ColumnSet::create(1, prepared_set);
|
||||
|
||||
data.addAction(ExpressionAction::addColumn(column));
|
||||
}
|
||||
|
||||
argument_types.push_back(column.type);
|
||||
argument_names.push_back(column.name);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// If the argument is not a lambda expression, call it recursively and find out its type.
|
||||
visit(child, data);
|
||||
std::string name = child_column_name;
|
||||
if (data.hasColumn(name))
|
||||
{
|
||||
argument_types.push_back(data.getSampleBlock().getByName(name).type);
|
||||
argument_names.push_back(name);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!only_consts)
|
||||
{
|
||||
/// We are in the part of the tree that we are not going to compute. You just need to define types.
|
||||
/// Do not subquery and create sets. We treat "IN" as "ignoreExceptNull" function.
|
||||
|
||||
actions_stack.addAction(ExpressionAction::applyFunction(
|
||||
FunctionFactory::instance().get("ignoreExceptNull", context),
|
||||
{ node->arguments->children.at(0)->getColumnName() },
|
||||
getColumnName()));
|
||||
}
|
||||
return;
|
||||
if (data.only_consts)
|
||||
arguments_present = false;
|
||||
else
|
||||
throw Exception("Unknown identifier: " + name, ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// A special function `indexHint`. Everything that is inside it is not calculated
|
||||
/// (and is used only for index analysis, see KeyCondition).
|
||||
if (node->name == "indexHint")
|
||||
if (data.only_consts && !arguments_present)
|
||||
return;
|
||||
|
||||
if (has_lambda_arguments && !data.only_consts)
|
||||
{
|
||||
function_builder->getLambdaArgumentTypes(argument_types);
|
||||
|
||||
/// Call recursively for lambda expressions.
|
||||
for (size_t i = 0; i < node.arguments->children.size(); ++i)
|
||||
{
|
||||
actions_stack.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName(
|
||||
ColumnConst::create(ColumnUInt8::create(1, 1), 1), std::make_shared<DataTypeUInt8>(),
|
||||
getColumnName())));
|
||||
return;
|
||||
}
|
||||
|
||||
if (AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
|
||||
return;
|
||||
|
||||
/// Context object that we pass to function should live during query.
|
||||
const Context & function_context = context.hasQueryContext()
|
||||
? context.getQueryContext()
|
||||
: context;
|
||||
|
||||
FunctionBuilderPtr function_builder;
|
||||
try
|
||||
{
|
||||
function_builder = FunctionFactory::instance().get(node->name, function_context);
|
||||
}
|
||||
catch (DB::Exception & e)
|
||||
{
|
||||
auto hints = AggregateFunctionFactory::instance().getHints(node->name);
|
||||
if (!hints.empty())
|
||||
e.addMessage("Or unknown aggregate function " + node->name + ". Maybe you meant: " + toString(hints));
|
||||
e.rethrow();
|
||||
}
|
||||
|
||||
Names argument_names;
|
||||
DataTypes argument_types;
|
||||
bool arguments_present = true;
|
||||
|
||||
/// If the function has an argument-lambda expression, you need to determine its type before the recursive call.
|
||||
bool has_lambda_arguments = false;
|
||||
|
||||
for (size_t arg = 0; arg < node->arguments->children.size(); ++arg)
|
||||
{
|
||||
auto & child = node->arguments->children[arg];
|
||||
auto child_column_name = child->getColumnName();
|
||||
ASTPtr child = node.arguments->children[i];
|
||||
|
||||
const auto * lambda = child->as<ASTFunction>();
|
||||
if (lambda && lambda->name == "lambda")
|
||||
{
|
||||
/// If the argument is a lambda expression, just remember its approximate type.
|
||||
if (lambda->arguments->children.size() != 2)
|
||||
throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const DataTypeFunction * lambda_type = typeid_cast<const DataTypeFunction *>(argument_types[i].get());
|
||||
const auto * lambda_args_tuple = lambda->arguments->children.at(0)->as<ASTFunction>();
|
||||
const ASTs & lambda_arg_asts = lambda_args_tuple->arguments->children;
|
||||
NamesAndTypesList lambda_arguments;
|
||||
|
||||
if (!lambda_args_tuple || lambda_args_tuple->name != "tuple")
|
||||
throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH);
|
||||
|
||||
has_lambda_arguments = true;
|
||||
argument_types.emplace_back(std::make_shared<DataTypeFunction>(DataTypes(lambda_args_tuple->arguments->children.size())));
|
||||
/// Select the name in the next cycle.
|
||||
argument_names.emplace_back();
|
||||
}
|
||||
else if (functionIsInOrGlobalInOperator(node->name) && arg == 1 && prepared_set)
|
||||
{
|
||||
ColumnWithTypeAndName column;
|
||||
column.type = std::make_shared<DataTypeSet>();
|
||||
|
||||
/// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name,
|
||||
/// so that sets with the same literal representation do not fuse together (they can have different types).
|
||||
if (!prepared_set->empty())
|
||||
column.name = getUniqueName(actions_stack.getSampleBlock(), "__set");
|
||||
else
|
||||
column.name = child_column_name;
|
||||
|
||||
if (!actions_stack.getSampleBlock().has(column.name))
|
||||
for (size_t j = 0; j < lambda_arg_asts.size(); ++j)
|
||||
{
|
||||
column.column = ColumnSet::create(1, prepared_set);
|
||||
auto opt_arg_name = tryGetIdentifierName(lambda_arg_asts[j]);
|
||||
if (!opt_arg_name)
|
||||
throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH);
|
||||
|
||||
actions_stack.addAction(ExpressionAction::addColumn(column));
|
||||
lambda_arguments.emplace_back(*opt_arg_name, lambda_type->getArgumentTypes()[j]);
|
||||
}
|
||||
|
||||
argument_types.push_back(column.type);
|
||||
argument_names.push_back(column.name);
|
||||
data.actions_stack.pushLevel(lambda_arguments);
|
||||
visit(lambda->arguments->children.at(1), data);
|
||||
ExpressionActionsPtr lambda_actions = data.actions_stack.popLevel();
|
||||
|
||||
String result_name = lambda->arguments->children.at(1)->getColumnName();
|
||||
lambda_actions->finalize(Names(1, result_name));
|
||||
DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type;
|
||||
|
||||
Names captured;
|
||||
Names required = lambda_actions->getRequiredColumns();
|
||||
for (const auto & required_arg : required)
|
||||
if (findColumn(required_arg, lambda_arguments) == lambda_arguments.end())
|
||||
captured.push_back(required_arg);
|
||||
|
||||
/// We can not name `getColumnName()`,
|
||||
/// because it does not uniquely define the expression (the types of arguments can be different).
|
||||
String lambda_name = getUniqueName(data.getSampleBlock(), "__lambda");
|
||||
|
||||
auto function_capture = std::make_shared<FunctionCapture>(
|
||||
lambda_actions, captured, lambda_arguments, result_type, result_name);
|
||||
data.addAction(ExpressionAction::applyFunction(function_capture, captured, lambda_name));
|
||||
|
||||
argument_types[i] = std::make_shared<DataTypeFunction>(lambda_type->getArgumentTypes(), result_type);
|
||||
argument_names[i] = lambda_name;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// If the argument is not a lambda expression, call it recursively and find out its type.
|
||||
visit(child);
|
||||
std::string name = child_column_name;
|
||||
if (actions_stack.getSampleBlock().has(name))
|
||||
{
|
||||
argument_types.push_back(actions_stack.getSampleBlock().getByName(name).type);
|
||||
argument_names.push_back(name);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (only_consts)
|
||||
arguments_present = false;
|
||||
else
|
||||
throw Exception("Unknown identifier: " + name, ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (only_consts && !arguments_present)
|
||||
return;
|
||||
|
||||
if (has_lambda_arguments && !only_consts)
|
||||
{
|
||||
function_builder->getLambdaArgumentTypes(argument_types);
|
||||
|
||||
/// Call recursively for lambda expressions.
|
||||
for (size_t i = 0; i < node->arguments->children.size(); ++i)
|
||||
{
|
||||
ASTPtr child = node->arguments->children[i];
|
||||
|
||||
const auto * lambda = child->as<ASTFunction>();
|
||||
if (lambda && lambda->name == "lambda")
|
||||
{
|
||||
const DataTypeFunction * lambda_type = typeid_cast<const DataTypeFunction *>(argument_types[i].get());
|
||||
const auto * lambda_args_tuple = lambda->arguments->children.at(0)->as<ASTFunction>();
|
||||
const ASTs & lambda_arg_asts = lambda_args_tuple->arguments->children;
|
||||
NamesAndTypesList lambda_arguments;
|
||||
|
||||
for (size_t j = 0; j < lambda_arg_asts.size(); ++j)
|
||||
{
|
||||
auto opt_arg_name = tryGetIdentifierName(lambda_arg_asts[j]);
|
||||
if (!opt_arg_name)
|
||||
throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH);
|
||||
|
||||
lambda_arguments.emplace_back(*opt_arg_name, lambda_type->getArgumentTypes()[j]);
|
||||
}
|
||||
|
||||
actions_stack.pushLevel(lambda_arguments);
|
||||
visit(lambda->arguments->children.at(1));
|
||||
ExpressionActionsPtr lambda_actions = actions_stack.popLevel();
|
||||
|
||||
String result_name = lambda->arguments->children.at(1)->getColumnName();
|
||||
lambda_actions->finalize(Names(1, result_name));
|
||||
DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type;
|
||||
|
||||
Names captured;
|
||||
Names required = lambda_actions->getRequiredColumns();
|
||||
for (const auto & required_arg : required)
|
||||
if (findColumn(required_arg, lambda_arguments) == lambda_arguments.end())
|
||||
captured.push_back(required_arg);
|
||||
|
||||
/// We can not name `getColumnName()`,
|
||||
/// because it does not uniquely define the expression (the types of arguments can be different).
|
||||
String lambda_name = getUniqueName(actions_stack.getSampleBlock(), "__lambda");
|
||||
|
||||
auto function_capture = std::make_shared<FunctionCapture>(
|
||||
lambda_actions, captured, lambda_arguments, result_type, result_name);
|
||||
actions_stack.addAction(ExpressionAction::applyFunction(function_capture, captured, lambda_name));
|
||||
|
||||
argument_types[i] = std::make_shared<DataTypeFunction>(lambda_type->getArgumentTypes(), result_type);
|
||||
argument_names[i] = lambda_name;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (only_consts)
|
||||
{
|
||||
for (const auto & argument_name : argument_names)
|
||||
{
|
||||
if (!actions_stack.getSampleBlock().has(argument_name))
|
||||
{
|
||||
arguments_present = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (arguments_present)
|
||||
{
|
||||
actions_stack.addAction(
|
||||
ExpressionAction::applyFunction(function_builder, argument_names, getColumnName()));
|
||||
}
|
||||
}
|
||||
else if (const auto * literal = ast->as<ASTLiteral>())
|
||||
{
|
||||
DataTypePtr type = applyVisitor(FieldToDataType(), literal->value);
|
||||
|
||||
ColumnWithTypeAndName column;
|
||||
column.column = type->createColumnConst(1, convertFieldToType(literal->value, *type));
|
||||
column.type = type;
|
||||
column.name = getColumnName();
|
||||
|
||||
actions_stack.addAction(ExpressionAction::addColumn(column));
|
||||
}
|
||||
else
|
||||
if (data.only_consts)
|
||||
{
|
||||
for (auto & child : ast->children)
|
||||
for (const auto & argument_name : argument_names)
|
||||
{
|
||||
/// Do not go to FROM, JOIN, UNION.
|
||||
if (!child->as<ASTTableExpression>() && !child->as<ASTSelectQuery>())
|
||||
visit(child);
|
||||
if (!data.hasColumn(argument_name))
|
||||
{
|
||||
arguments_present = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (arguments_present)
|
||||
{
|
||||
data.addAction(ExpressionAction::applyFunction(function_builder, argument_names, column_name.get(ast)));
|
||||
}
|
||||
}
|
||||
|
||||
SetPtr ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_block)
|
||||
void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & ast, Data & data)
|
||||
{
|
||||
CachedColumnName column_name;
|
||||
if (data.hasColumn(column_name.get(ast)))
|
||||
return;
|
||||
|
||||
DataTypePtr type = applyVisitor(FieldToDataType(), literal.value);
|
||||
|
||||
ColumnWithTypeAndName column;
|
||||
column.column = type->createColumnConst(1, convertFieldToType(literal.value, *type));
|
||||
column.type = type;
|
||||
column.name = column_name.get(ast);
|
||||
|
||||
data.addAction(ExpressionAction::addColumn(column));
|
||||
}
|
||||
|
||||
SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data)
|
||||
{
|
||||
/** You need to convert the right argument to a set.
|
||||
* This can be a table name, a value, a value enumeration, or a subquery.
|
||||
* The enumeration of values is parsed as a function `tuple`.
|
||||
*/
|
||||
const IAST & args = *node->arguments;
|
||||
const IAST & args = *node.arguments;
|
||||
const ASTPtr & arg = args.children.at(1);
|
||||
const Block & sample_block = data.getSampleBlock();
|
||||
|
||||
/// If the subquery or table name for SELECT.
|
||||
const auto * identifier = arg->as<ASTIdentifier>();
|
||||
if (arg->as<ASTSubquery>() || identifier)
|
||||
{
|
||||
auto set_key = PreparedSetKey::forSubquery(*arg);
|
||||
if (prepared_sets.count(set_key))
|
||||
return prepared_sets.at(set_key);
|
||||
if (data.prepared_sets.count(set_key))
|
||||
return data.prepared_sets.at(set_key);
|
||||
|
||||
/// A special case is if the name of the table is specified on the right side of the IN statement,
|
||||
/// and the table has the type Set (a previously prepared set).
|
||||
if (identifier)
|
||||
{
|
||||
DatabaseAndTableWithAlias database_table(*identifier);
|
||||
StoragePtr table = context.tryGetTable(database_table.database, database_table.table);
|
||||
StoragePtr table = data.context.tryGetTable(database_table.database, database_table.table);
|
||||
|
||||
if (table)
|
||||
{
|
||||
StorageSet * storage_set = dynamic_cast<StorageSet *>(table.get());
|
||||
if (storage_set)
|
||||
{
|
||||
prepared_sets[set_key] = storage_set->getSet();
|
||||
data.prepared_sets[set_key] = storage_set->getSet();
|
||||
return storage_set->getSet();
|
||||
}
|
||||
}
|
||||
@ -576,25 +580,25 @@ SetPtr ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bl
|
||||
/// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery.
|
||||
String set_id = arg->getColumnName();
|
||||
|
||||
SubqueryForSet & subquery_for_set = subqueries_for_sets[set_id];
|
||||
SubqueryForSet & subquery_for_set = data.subqueries_for_sets[set_id];
|
||||
|
||||
/// If you already created a Set with the same subquery / table.
|
||||
if (subquery_for_set.set)
|
||||
{
|
||||
prepared_sets[set_key] = subquery_for_set.set;
|
||||
data.prepared_sets[set_key] = subquery_for_set.set;
|
||||
return subquery_for_set.set;
|
||||
}
|
||||
|
||||
SetPtr set = std::make_shared<Set>(set_size_limit, false);
|
||||
SetPtr set = std::make_shared<Set>(data.set_size_limit, false);
|
||||
|
||||
/** The following happens for GLOBAL INs:
|
||||
* - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1,
|
||||
* in the subquery_for_set object, this subquery is set as source and the temporary table _data1 as the table.
|
||||
* - this function shows the expression IN_data1.
|
||||
*/
|
||||
if (!subquery_for_set.source && no_storage_or_local)
|
||||
if (!subquery_for_set.source && data.no_storage_or_local)
|
||||
{
|
||||
auto interpreter = interpretSubquery(arg, context, subquery_depth, {});
|
||||
auto interpreter = interpretSubquery(arg, data.context, data.subquery_depth, {});
|
||||
subquery_for_set.source = std::make_shared<LazyBlockInputStream>(
|
||||
interpreter->getSampleBlock(), [interpreter]() mutable { return interpreter->execute().in; });
|
||||
|
||||
@ -627,13 +631,13 @@ SetPtr ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bl
|
||||
}
|
||||
|
||||
subquery_for_set.set = set;
|
||||
prepared_sets[set_key] = set;
|
||||
data.prepared_sets[set_key] = set;
|
||||
return set;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// An explicit enumeration of values in parentheses.
|
||||
return makeExplicitSet(node, sample_block, false, context, set_size_limit, prepared_sets);
|
||||
return makeExplicitSet(&node, sample_block, false, data.context, data.set_size_limit, data.prepared_sets);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Interpreters/PreparedSets.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/SubqueryForSet.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -52,38 +53,81 @@ struct ScopeStack
|
||||
const Block & getSampleBlock() const;
|
||||
};
|
||||
|
||||
class ASTIdentifier;
|
||||
class ASTFunction;
|
||||
class ASTLiteral;
|
||||
|
||||
/// Collect ExpressionAction from AST. Returns PreparedSets and SubqueriesForSets too.
|
||||
class ActionsVisitor
|
||||
class ActionsMatcher
|
||||
{
|
||||
public:
|
||||
ActionsVisitor(const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_,
|
||||
const NamesAndTypesList & source_columns_, const ExpressionActionsPtr & actions,
|
||||
PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_,
|
||||
bool no_subqueries_, bool only_consts_, bool no_storage_or_local_, std::ostream * ostr_ = nullptr);
|
||||
using Visitor = ConstInDepthNodeVisitor<ActionsMatcher, true>;
|
||||
|
||||
void visit(const ASTPtr & ast, ExpressionActionsPtr & actions)
|
||||
struct Data
|
||||
{
|
||||
visit(ast);
|
||||
actions = actions_stack.popLevel();
|
||||
}
|
||||
const Context & context;
|
||||
SizeLimits set_size_limit;
|
||||
size_t subquery_depth;
|
||||
const NamesAndTypesList & source_columns;
|
||||
PreparedSets & prepared_sets;
|
||||
SubqueriesForSets & subqueries_for_sets;
|
||||
bool no_subqueries;
|
||||
bool only_consts;
|
||||
bool no_storage_or_local;
|
||||
size_t visit_depth;
|
||||
ScopeStack actions_stack;
|
||||
|
||||
Data(const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_,
|
||||
const NamesAndTypesList & source_columns_, const ExpressionActionsPtr & actions,
|
||||
PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_,
|
||||
bool no_subqueries_, bool only_consts_, bool no_storage_or_local_)
|
||||
: context(context_),
|
||||
set_size_limit(set_size_limit_),
|
||||
subquery_depth(subquery_depth_),
|
||||
source_columns(source_columns_),
|
||||
prepared_sets(prepared_sets_),
|
||||
subqueries_for_sets(subqueries_for_sets_),
|
||||
no_subqueries(no_subqueries_),
|
||||
only_consts(only_consts_),
|
||||
no_storage_or_local(no_storage_or_local_),
|
||||
visit_depth(0),
|
||||
actions_stack(actions, context)
|
||||
{}
|
||||
|
||||
void updateActions(ExpressionActionsPtr & actions)
|
||||
{
|
||||
actions = actions_stack.popLevel();
|
||||
}
|
||||
|
||||
void addAction(const ExpressionAction & action)
|
||||
{
|
||||
actions_stack.addAction(action);
|
||||
}
|
||||
|
||||
const Block & getSampleBlock() const
|
||||
{
|
||||
return actions_stack.getSampleBlock();
|
||||
}
|
||||
|
||||
/// Does result of the calculation already exists in the block.
|
||||
bool hasColumn(const String & columnName) const
|
||||
{
|
||||
return actions_stack.getSampleBlock().has(columnName);
|
||||
}
|
||||
};
|
||||
|
||||
static void visit(const ASTPtr & ast, Data & data);
|
||||
static bool needChildVisit(const ASTPtr & node, const ASTPtr & child);
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
SizeLimits set_size_limit;
|
||||
size_t subquery_depth;
|
||||
const NamesAndTypesList & source_columns;
|
||||
PreparedSets & prepared_sets;
|
||||
SubqueriesForSets & subqueries_for_sets;
|
||||
const bool no_subqueries;
|
||||
const bool only_consts;
|
||||
const bool no_storage_or_local;
|
||||
mutable size_t visit_depth;
|
||||
std::ostream * ostr;
|
||||
ScopeStack actions_stack;
|
||||
|
||||
void visit(const ASTPtr & ast);
|
||||
SetPtr makeSet(const ASTFunction * node, const Block & sample_block);
|
||||
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);
|
||||
|
||||
static SetPtr makeSet(const ASTFunction & node, Data & data);
|
||||
};
|
||||
|
||||
using ActionsVisitor = ActionsMatcher::Visitor;
|
||||
|
||||
}
|
||||
|
@ -80,7 +80,7 @@ ExpressionActionsPtr AnalyzedJoin::createJoinedBlockActions(
|
||||
|
||||
ASTPtr query = expression_list;
|
||||
auto syntax_result = SyntaxAnalyzer(context).analyze(query, columns_from_joined_table, required_columns);
|
||||
ExpressionAnalyzer analyzer(query, syntax_result, context, required_columns_set);
|
||||
ExpressionAnalyzer analyzer(query, syntax_result, context);
|
||||
return analyzer.getActions(true, false);
|
||||
}
|
||||
|
||||
|
@ -35,6 +35,7 @@ private:
|
||||
friend class SyntaxAnalyzer;
|
||||
friend struct SyntaxAnalyzerResult;
|
||||
friend class ExpressionAnalyzer;
|
||||
friend class SelectQueryExpressionAnalyzer;
|
||||
|
||||
Names key_names_left;
|
||||
Names key_names_right; /// Duplicating names are qualified.
|
||||
|
@ -54,8 +54,8 @@
|
||||
#include <Interpreters/interpretSubquery.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/QueryNormalizer.h>
|
||||
|
||||
#include <Interpreters/ActionsVisitor.h>
|
||||
|
||||
#include <Interpreters/ExternalTablesVisitor.h>
|
||||
#include <Interpreters/GlobalSubqueriesVisitor.h>
|
||||
#include <Interpreters/GetAggregatesVisitor.h>
|
||||
@ -76,11 +76,9 @@ ExpressionAnalyzer::ExpressionAnalyzer(
|
||||
const ASTPtr & query_,
|
||||
const SyntaxAnalyzerResultPtr & syntax_analyzer_result_,
|
||||
const Context & context_,
|
||||
const NameSet & required_result_columns_,
|
||||
size_t subquery_depth_,
|
||||
bool do_global)
|
||||
: ExpressionAnalyzerData(required_result_columns_)
|
||||
, query(query_), context(context_), settings(context.getSettings())
|
||||
: query(query_), context(context_), settings(context.getSettings())
|
||||
, subquery_depth(subquery_depth_)
|
||||
, syntax(syntax_analyzer_result_)
|
||||
{
|
||||
@ -236,7 +234,7 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables(bool do_global)
|
||||
}
|
||||
|
||||
|
||||
void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name)
|
||||
void SelectQueryExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name)
|
||||
{
|
||||
auto set_key = PreparedSetKey::forSubquery(*subquery_or_table_name);
|
||||
if (prepared_sets.count(set_key))
|
||||
@ -262,7 +260,7 @@ void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_
|
||||
|
||||
|
||||
/// Perfomance optimisation for IN() if storage supports it.
|
||||
void ExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node)
|
||||
void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node)
|
||||
{
|
||||
if (!node || !storage() || !storage()->supportsIndexForIn())
|
||||
return;
|
||||
@ -317,10 +315,11 @@ void ExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node)
|
||||
void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts)
|
||||
{
|
||||
LogAST log;
|
||||
ActionsVisitor actions_visitor(context, settings.size_limits_for_set, subquery_depth,
|
||||
ActionsVisitor::Data visitor_data(context, settings.size_limits_for_set, subquery_depth,
|
||||
sourceColumns(), actions, prepared_sets, subqueries_for_sets,
|
||||
no_subqueries, only_consts, !isRemoteStorage(), log.stream());
|
||||
actions_visitor.visit(ast, actions);
|
||||
no_subqueries, only_consts, !isRemoteStorage());
|
||||
ActionsVisitor(visitor_data, log.stream()).visit(ast);
|
||||
visitor_data.updateActions(actions);
|
||||
}
|
||||
|
||||
|
||||
@ -361,7 +360,7 @@ const ASTSelectQuery * ExpressionAnalyzer::getSelectQuery() const
|
||||
return select_query;
|
||||
}
|
||||
|
||||
const ASTSelectQuery * ExpressionAnalyzer::getAggregatingQuery() const
|
||||
const ASTSelectQuery * SelectQueryExpressionAnalyzer::getAggregatingQuery() const
|
||||
{
|
||||
if (!has_aggregation)
|
||||
throw Exception("No aggregation", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -393,7 +392,7 @@ void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActionsPtr & actio
|
||||
actions->add(ExpressionAction::arrayJoin(result_columns, array_join_is_left, context));
|
||||
}
|
||||
|
||||
bool ExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, bool only_types)
|
||||
bool SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, bool only_types)
|
||||
{
|
||||
const auto * select_query = getSelectQuery();
|
||||
|
||||
@ -435,7 +434,7 @@ static void appendRequiredColumns(
|
||||
required_columns.insert(column.name);
|
||||
}
|
||||
|
||||
bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_types)
|
||||
bool SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_types)
|
||||
{
|
||||
const auto * select_query = getSelectQuery();
|
||||
|
||||
@ -552,7 +551,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ExpressionAnalyzer::appendPrewhere(
|
||||
bool SelectQueryExpressionAnalyzer::appendPrewhere(
|
||||
ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns)
|
||||
{
|
||||
const auto * select_query = getSelectQuery();
|
||||
@ -626,7 +625,7 @@ bool ExpressionAnalyzer::appendPrewhere(
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, bool only_types)
|
||||
bool SelectQueryExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, bool only_types)
|
||||
{
|
||||
const auto * select_query = getSelectQuery();
|
||||
|
||||
@ -644,7 +643,7 @@ bool ExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, bool only_t
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain, bool only_types)
|
||||
bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain, bool only_types)
|
||||
{
|
||||
const auto * select_query = getAggregatingQuery();
|
||||
|
||||
@ -664,7 +663,7 @@ bool ExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain, bool only
|
||||
return true;
|
||||
}
|
||||
|
||||
void ExpressionAnalyzer::appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types)
|
||||
void SelectQueryExpressionAnalyzer::appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types)
|
||||
{
|
||||
const auto * select_query = getAggregatingQuery();
|
||||
|
||||
@ -697,7 +696,7 @@ void ExpressionAnalyzer::appendAggregateFunctionsArguments(ExpressionActionsChai
|
||||
getRootActions(argument, only_types, step.actions);
|
||||
}
|
||||
|
||||
bool ExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_types)
|
||||
bool SelectQueryExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_types)
|
||||
{
|
||||
const auto * select_query = getAggregatingQuery();
|
||||
|
||||
@ -713,7 +712,7 @@ bool ExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_
|
||||
return true;
|
||||
}
|
||||
|
||||
void ExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, bool only_types)
|
||||
void SelectQueryExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, bool only_types)
|
||||
{
|
||||
const auto * select_query = getSelectQuery();
|
||||
|
||||
@ -726,7 +725,7 @@ void ExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, bool only_
|
||||
step.required_output.push_back(child->getColumnName());
|
||||
}
|
||||
|
||||
bool ExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only_types)
|
||||
bool SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only_types)
|
||||
{
|
||||
const auto * select_query = getSelectQuery();
|
||||
|
||||
@ -750,7 +749,7 @@ bool ExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain, bool only_types)
|
||||
bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain, bool only_types)
|
||||
{
|
||||
const auto * select_query = getSelectQuery();
|
||||
|
||||
@ -779,7 +778,7 @@ bool ExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain, bool only
|
||||
return true;
|
||||
}
|
||||
|
||||
void ExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) const
|
||||
void SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) const
|
||||
{
|
||||
const auto * select_query = getSelectQuery();
|
||||
|
||||
@ -867,7 +866,7 @@ ExpressionActionsPtr ExpressionAnalyzer::getConstActions()
|
||||
return actions;
|
||||
}
|
||||
|
||||
void ExpressionAnalyzer::getAggregateInfo(Names & key_names, AggregateDescriptions & aggregates) const
|
||||
void SelectQueryExpressionAnalyzer::getAggregateInfo(Names & key_names, AggregateDescriptions & aggregates) const
|
||||
{
|
||||
for (const auto & name_and_type : aggregation_keys)
|
||||
key_names.emplace_back(name_and_type.name);
|
||||
|
@ -31,9 +31,6 @@ using SyntaxAnalyzerResultPtr = std::shared_ptr<const SyntaxAnalyzerResult>;
|
||||
/// ExpressionAnalyzer sources, intermediates and results. It splits data and logic, allows to test them separately.
|
||||
struct ExpressionAnalyzerData
|
||||
{
|
||||
/// If non-empty, ignore all expressions in not from this list.
|
||||
NameSet required_result_columns;
|
||||
|
||||
SubqueriesForSets subqueries_for_sets;
|
||||
PreparedSets prepared_sets;
|
||||
|
||||
@ -49,11 +46,6 @@ struct ExpressionAnalyzerData
|
||||
|
||||
/// All new temporary tables obtained by performing the GLOBAL IN/JOIN subqueries.
|
||||
Tables external_tables;
|
||||
|
||||
protected:
|
||||
ExpressionAnalyzerData(const NameSet & required_result_columns_)
|
||||
: required_result_columns(required_result_columns_)
|
||||
{}
|
||||
};
|
||||
|
||||
|
||||
@ -61,7 +53,7 @@ protected:
|
||||
*
|
||||
* NOTE: if `ast` is a SELECT query from a table, the structure of this table should not change during the lifetime of ExpressionAnalyzer.
|
||||
*/
|
||||
class ExpressionAnalyzer : private ExpressionAnalyzerData, private boost::noncopyable
|
||||
class ExpressionAnalyzer : protected ExpressionAnalyzerData, private boost::noncopyable
|
||||
{
|
||||
private:
|
||||
/// Extracts settings to enlight which are used (and avoid copy of others).
|
||||
@ -83,51 +75,14 @@ private:
|
||||
};
|
||||
|
||||
public:
|
||||
/// Ctor for non-select queries. Generally its usage is:
|
||||
/// auto actions = ExpressionAnalyzer(query, syntax, context).getActions();
|
||||
ExpressionAnalyzer(
|
||||
const ASTPtr & query_,
|
||||
const SyntaxAnalyzerResultPtr & syntax_analyzer_result_,
|
||||
const Context & context_,
|
||||
const NameSet & required_result_columns_ = {},
|
||||
size_t subquery_depth_ = 0,
|
||||
bool do_global_ = false);
|
||||
|
||||
/// Does the expression have aggregate functions or a GROUP BY or HAVING section.
|
||||
bool hasAggregation() const { return has_aggregation; }
|
||||
|
||||
/// Get a list of aggregation keys and descriptions of aggregate functions if the query contains GROUP BY.
|
||||
void getAggregateInfo(Names & key_names, AggregateDescriptions & aggregates) const;
|
||||
|
||||
/** These methods allow you to build a chain of transformations over a block, that receives values in the desired sections of the query.
|
||||
*
|
||||
* Example usage:
|
||||
* ExpressionActionsChain chain;
|
||||
* analyzer.appendWhere(chain);
|
||||
* chain.addStep();
|
||||
* analyzer.appendSelect(chain);
|
||||
* analyzer.appendOrderBy(chain);
|
||||
* chain.finalize();
|
||||
*
|
||||
* If only_types = true set, does not execute subqueries in the relevant parts of the query. The actions got this way
|
||||
* shouldn't be executed, they are only needed to get a list of columns with their types.
|
||||
*/
|
||||
|
||||
/// Before aggregation:
|
||||
bool appendArrayJoin(ExpressionActionsChain & chain, bool only_types);
|
||||
bool appendJoin(ExpressionActionsChain & chain, bool only_types);
|
||||
/// remove_filter is set in ExpressionActionsChain::finalize();
|
||||
/// Columns in `additional_required_columns` will not be removed (they can be used for e.g. sampling or FINAL modifier).
|
||||
bool appendPrewhere(ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns);
|
||||
bool appendWhere(ExpressionActionsChain & chain, bool only_types);
|
||||
bool appendGroupBy(ExpressionActionsChain & chain, bool only_types);
|
||||
void appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types);
|
||||
|
||||
/// After aggregation:
|
||||
bool appendHaving(ExpressionActionsChain & chain, bool only_types);
|
||||
void appendSelect(ExpressionActionsChain & chain, bool only_types);
|
||||
bool appendOrderBy(ExpressionActionsChain & chain, bool only_types);
|
||||
bool appendLimitBy(ExpressionActionsChain & chain, bool only_types);
|
||||
/// Deletes all columns except mentioned by SELECT, arranges the remaining columns and renames them to aliases.
|
||||
void appendProjectResult(ExpressionActionsChain & chain) const;
|
||||
const Context & context_)
|
||||
: ExpressionAnalyzer(query_, syntax_analyzer_result_, context_, 0, false)
|
||||
{}
|
||||
|
||||
void appendExpression(ExpressionActionsChain & chain, const ASTPtr & expr, bool only_types);
|
||||
|
||||
@ -148,21 +103,17 @@ public:
|
||||
*/
|
||||
const SubqueriesForSets & getSubqueriesForSets() const { return subqueries_for_sets; }
|
||||
|
||||
const PreparedSets & getPreparedSets() const { return prepared_sets; }
|
||||
|
||||
/** Tables that will need to be sent to remote servers for distributed query processing.
|
||||
*/
|
||||
const Tables & getExternalTables() const { return external_tables; }
|
||||
|
||||
/// Get intermediates for tests
|
||||
const ExpressionAnalyzerData & getAnalyzedData() const { return *this; }
|
||||
|
||||
/// Create Set-s that we can from IN section to use the index on them.
|
||||
void makeSetsForIndex(const ASTPtr & node);
|
||||
protected:
|
||||
ExpressionAnalyzer(
|
||||
const ASTPtr & query_,
|
||||
const SyntaxAnalyzerResultPtr & syntax_analyzer_result_,
|
||||
const Context & context_,
|
||||
size_t subquery_depth_,
|
||||
bool do_global_);
|
||||
|
||||
bool hasGlobalSubqueries() { return has_global_subqueries; }
|
||||
|
||||
private:
|
||||
ASTPtr query;
|
||||
const Context & context;
|
||||
const ExtractedSettings settings;
|
||||
@ -197,7 +148,75 @@ private:
|
||||
void initChain(ExpressionActionsChain & chain, const NamesAndTypesList & columns) const;
|
||||
|
||||
const ASTSelectQuery * getSelectQuery() const;
|
||||
const ASTSelectQuery * getAggregatingQuery() const;
|
||||
|
||||
bool isRemoteStorage() const;
|
||||
};
|
||||
|
||||
/// SelectQuery specific ExpressionAnalyzer part.
|
||||
class SelectQueryExpressionAnalyzer : public ExpressionAnalyzer
|
||||
{
|
||||
public:
|
||||
SelectQueryExpressionAnalyzer(
|
||||
const ASTPtr & query_,
|
||||
const SyntaxAnalyzerResultPtr & syntax_analyzer_result_,
|
||||
const Context & context_,
|
||||
const NameSet & required_result_columns_ = {},
|
||||
size_t subquery_depth_ = 0,
|
||||
bool do_global_ = false)
|
||||
: ExpressionAnalyzer(query_, syntax_analyzer_result_, context_, subquery_depth_, do_global_)
|
||||
, required_result_columns(required_result_columns_)
|
||||
{}
|
||||
|
||||
/// Does the expression have aggregate functions or a GROUP BY or HAVING section.
|
||||
bool hasAggregation() const { return has_aggregation; }
|
||||
bool hasGlobalSubqueries() { return has_global_subqueries; }
|
||||
|
||||
/// Get a list of aggregation keys and descriptions of aggregate functions if the query contains GROUP BY.
|
||||
void getAggregateInfo(Names & key_names, AggregateDescriptions & aggregates) const;
|
||||
|
||||
const PreparedSets & getPreparedSets() const { return prepared_sets; }
|
||||
|
||||
/// Tables that will need to be sent to remote servers for distributed query processing.
|
||||
const Tables & getExternalTables() const { return external_tables; }
|
||||
|
||||
/** These methods allow you to build a chain of transformations over a block, that receives values in the desired sections of the query.
|
||||
*
|
||||
* Example usage:
|
||||
* ExpressionActionsChain chain;
|
||||
* analyzer.appendWhere(chain);
|
||||
* chain.addStep();
|
||||
* analyzer.appendSelect(chain);
|
||||
* analyzer.appendOrderBy(chain);
|
||||
* chain.finalize();
|
||||
*
|
||||
* If only_types = true set, does not execute subqueries in the relevant parts of the query. The actions got this way
|
||||
* shouldn't be executed, they are only needed to get a list of columns with their types.
|
||||
*/
|
||||
|
||||
/// Before aggregation:
|
||||
bool appendArrayJoin(ExpressionActionsChain & chain, bool only_types);
|
||||
bool appendJoin(ExpressionActionsChain & chain, bool only_types);
|
||||
/// remove_filter is set in ExpressionActionsChain::finalize();
|
||||
/// Columns in `additional_required_columns` will not be removed (they can be used for e.g. sampling or FINAL modifier).
|
||||
bool appendPrewhere(ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns);
|
||||
bool appendWhere(ExpressionActionsChain & chain, bool only_types);
|
||||
bool appendGroupBy(ExpressionActionsChain & chain, bool only_types);
|
||||
void appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types);
|
||||
|
||||
/// After aggregation:
|
||||
bool appendHaving(ExpressionActionsChain & chain, bool only_types);
|
||||
void appendSelect(ExpressionActionsChain & chain, bool only_types);
|
||||
bool appendOrderBy(ExpressionActionsChain & chain, bool only_types);
|
||||
bool appendLimitBy(ExpressionActionsChain & chain, bool only_types);
|
||||
/// Deletes all columns except mentioned by SELECT, arranges the remaining columns and renames them to aliases.
|
||||
void appendProjectResult(ExpressionActionsChain & chain) const;
|
||||
|
||||
/// Create Set-s that we can from IN section to use the index on them.
|
||||
void makeSetsForIndex(const ASTPtr & node);
|
||||
|
||||
private:
|
||||
/// If non-empty, ignore all expressions not from this list.
|
||||
NameSet required_result_columns;
|
||||
|
||||
/**
|
||||
* Create Set from a subquery or a table expression in the query. The created set is suitable for using the index.
|
||||
@ -205,7 +224,7 @@ private:
|
||||
*/
|
||||
void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name);
|
||||
|
||||
bool isRemoteStorage() const;
|
||||
const ASTSelectQuery * getAggregatingQuery() const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -144,7 +144,7 @@ String generateFilterActions(ExpressionActionsPtr & actions, const StoragePtr &
|
||||
|
||||
/// Using separate expression analyzer to prevent any possible alias injection
|
||||
auto syntax_result = SyntaxAnalyzer(context).analyze(query_ast, storage->getColumns().getAllPhysical());
|
||||
ExpressionAnalyzer analyzer(query_ast, syntax_result, context);
|
||||
SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, context);
|
||||
ExpressionActionsChain new_chain(context);
|
||||
analyzer.appendSelect(new_chain, false);
|
||||
actions = new_chain.getLastActions();
|
||||
@ -296,7 +296,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
|
||||
syntax_analyzer_result = SyntaxAnalyzer(context, options).analyze(
|
||||
query_ptr, source_header.getNamesAndTypesList(), required_result_column_names, storage, NamesAndTypesList());
|
||||
query_analyzer = std::make_unique<ExpressionAnalyzer>(
|
||||
query_analyzer = std::make_unique<SelectQueryExpressionAnalyzer>(
|
||||
query_ptr, syntax_analyzer_result, context,
|
||||
NameSet(required_result_column_names.begin(), required_result_column_names.end()),
|
||||
options.subquery_depth, !options.only_analyze);
|
||||
|
@ -248,7 +248,7 @@ private:
|
||||
Context context;
|
||||
NamesAndTypesList source_columns;
|
||||
SyntaxAnalyzerResultPtr syntax_analyzer_result;
|
||||
std::unique_ptr<ExpressionAnalyzer> query_analyzer;
|
||||
std::unique_ptr<SelectQueryExpressionAnalyzer> query_analyzer;
|
||||
SelectQueryInfo query_info;
|
||||
|
||||
/// How many streams we ask for storage to produce, and in how many threads we will do further processing.
|
||||
|
@ -56,7 +56,7 @@ int main(int argc, char ** argv)
|
||||
};
|
||||
|
||||
auto syntax_result = SyntaxAnalyzer(context, {}).analyze(ast, columns);
|
||||
ExpressionAnalyzer analyzer(ast, syntax_result, context);
|
||||
SelectQueryExpressionAnalyzer analyzer(ast, syntax_result, context);
|
||||
ExpressionActionsChain chain(context);
|
||||
analyzer.appendSelect(chain, false);
|
||||
analyzer.appendProjectResult(chain);
|
||||
|
Loading…
Reference in New Issue
Block a user