|
|
|
@ -368,7 +368,7 @@ void ExpressionAnalyzer::translateQualifiedNamesImpl(ASTPtr & ast, const String
|
|
|
|
|
{
|
|
|
|
|
/// Do not go to FROM, JOIN, UNION.
|
|
|
|
|
if (!typeid_cast<const ASTTableExpression *>(child.get())
|
|
|
|
|
&& child.get() != select_query->next_union_all.get())
|
|
|
|
|
&& !typeid_cast<const ASTSelectQuery *>(child.get()))
|
|
|
|
|
{
|
|
|
|
|
translateQualifiedNamesImpl(child, database_name, table_name, alias);
|
|
|
|
|
}
|
|
|
|
@ -1108,35 +1108,6 @@ void ExpressionAnalyzer::normalizeTreeImpl(
|
|
|
|
|
normalizeTreeImpl(select->having_expression, finished_asts, current_asts, current_alias, level + 1);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Actions to be performed from the bottom up.
|
|
|
|
|
|
|
|
|
|
if (ASTFunction * node = typeid_cast<ASTFunction *>(ast.get()))
|
|
|
|
|
{
|
|
|
|
|
if (node->kind == ASTFunction::TABLE_FUNCTION)
|
|
|
|
|
{
|
|
|
|
|
}
|
|
|
|
|
else if (node->name == "lambda")
|
|
|
|
|
{
|
|
|
|
|
node->kind = ASTFunction::LAMBDA_EXPRESSION;
|
|
|
|
|
}
|
|
|
|
|
else if (AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
|
|
|
|
|
{
|
|
|
|
|
node->kind = ASTFunction::AGGREGATE_FUNCTION;
|
|
|
|
|
}
|
|
|
|
|
else if (node->name == "arrayJoin")
|
|
|
|
|
{
|
|
|
|
|
node->kind = ASTFunction::ARRAY_JOIN;
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
node->kind = ASTFunction::FUNCTION;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (node->parameters && node->kind != ASTFunction::AGGREGATE_FUNCTION)
|
|
|
|
|
throw Exception("The only parametric functions (functions with two separate parenthesis pairs) are aggregate functions"
|
|
|
|
|
", and '" + node->name + "' is not an aggregate function.", ErrorCodes::FUNCTION_CANNOT_HAVE_PARAMETERS);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
current_asts.erase(initial_ast.get());
|
|
|
|
|
current_asts.erase(ast.get());
|
|
|
|
|
finished_asts[initial_ast] = ast;
|
|
|
|
@ -1165,7 +1136,7 @@ void ExpressionAnalyzer::executeScalarSubqueries()
|
|
|
|
|
{
|
|
|
|
|
/// Do not go to FROM, JOIN, UNION.
|
|
|
|
|
if (!typeid_cast<const ASTTableExpression *>(child.get())
|
|
|
|
|
&& child.get() != select_query->next_union_all.get())
|
|
|
|
|
&& !typeid_cast<const ASTSelectQuery *>(child.get()))
|
|
|
|
|
{
|
|
|
|
|
executeScalarSubqueriesImpl(child);
|
|
|
|
|
}
|
|
|
|
@ -1181,7 +1152,6 @@ static ASTPtr addTypeConversion(std::unique_ptr<ASTLiteral> && ast, const String
|
|
|
|
|
func->alias = ast->alias;
|
|
|
|
|
func->prefer_alias_to_column_name = ast->prefer_alias_to_column_name;
|
|
|
|
|
ast->alias.clear();
|
|
|
|
|
func->kind = ASTFunction::FUNCTION;
|
|
|
|
|
func->name = "CAST";
|
|
|
|
|
auto exp_list = std::make_shared<ASTExpressionList>(ast->range);
|
|
|
|
|
func->arguments = exp_list;
|
|
|
|
@ -1260,7 +1230,6 @@ void ExpressionAnalyzer::executeScalarSubqueriesImpl(ASTPtr & ast)
|
|
|
|
|
auto tuple = std::make_shared<ASTFunction>(ast->range);
|
|
|
|
|
tuple->alias = subquery->alias;
|
|
|
|
|
ast = tuple;
|
|
|
|
|
tuple->kind = ASTFunction::FUNCTION;
|
|
|
|
|
tuple->name = "tuple";
|
|
|
|
|
auto exp_list = std::make_shared<ASTExpressionList>(ast->range);
|
|
|
|
|
tuple->arguments = exp_list;
|
|
|
|
@ -1286,8 +1255,7 @@ void ExpressionAnalyzer::executeScalarSubqueriesImpl(ASTPtr & ast)
|
|
|
|
|
*/
|
|
|
|
|
ASTFunction * func = typeid_cast<ASTFunction *>(ast.get());
|
|
|
|
|
|
|
|
|
|
if (func && func->kind == ASTFunction::FUNCTION
|
|
|
|
|
&& functionIsInOrGlobalInOperator(func->name))
|
|
|
|
|
if (func && functionIsInOrGlobalInOperator(func->name))
|
|
|
|
|
{
|
|
|
|
|
for (auto & child : ast->children)
|
|
|
|
|
{
|
|
|
|
@ -1495,7 +1463,7 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block &
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
const ASTFunction * func = typeid_cast<const ASTFunction *>(node.get());
|
|
|
|
|
if (func && func->kind == ASTFunction::FUNCTION && functionIsInOperator(func->name))
|
|
|
|
|
if (func && functionIsInOperator(func->name))
|
|
|
|
|
{
|
|
|
|
|
const IAST & args = *func->arguments;
|
|
|
|
|
const ASTPtr & arg = args.children.at(1);
|
|
|
|
@ -1976,11 +1944,11 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
|
|
|
|
|
}
|
|
|
|
|
else if (ASTFunction * node = typeid_cast<ASTFunction *>(ast.get()))
|
|
|
|
|
{
|
|
|
|
|
if (node->kind == ASTFunction::LAMBDA_EXPRESSION)
|
|
|
|
|
if (node->name == "lambda")
|
|
|
|
|
throw Exception("Unexpected lambda expression", ErrorCodes::UNEXPECTED_EXPRESSION);
|
|
|
|
|
|
|
|
|
|
/// Function arrayJoin.
|
|
|
|
|
if (node->kind == ASTFunction::ARRAY_JOIN)
|
|
|
|
|
if (node->name == "arrayJoin")
|
|
|
|
|
{
|
|
|
|
|
if (node->arguments->children.size() != 1)
|
|
|
|
|
throw Exception("arrayJoin requires exactly 1 argument", ErrorCodes::TYPE_MISMATCH);
|
|
|
|
@ -1999,193 +1967,193 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
|
|
|
|
|
return;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (node->kind == ASTFunction::FUNCTION)
|
|
|
|
|
if (functionIsInOrGlobalInOperator(node->name))
|
|
|
|
|
{
|
|
|
|
|
if (functionIsInOrGlobalInOperator(node->name))
|
|
|
|
|
if (!no_subqueries)
|
|
|
|
|
{
|
|
|
|
|
if (!no_subqueries)
|
|
|
|
|
{
|
|
|
|
|
/// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything).
|
|
|
|
|
getActionsImpl(node->arguments->children.at(0), no_subqueries, only_consts, actions_stack);
|
|
|
|
|
/// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything).
|
|
|
|
|
getActionsImpl(node->arguments->children.at(0), no_subqueries, only_consts, actions_stack);
|
|
|
|
|
|
|
|
|
|
/// Transform tuple or subquery into a set.
|
|
|
|
|
makeSet(node, actions_stack.getSampleBlock());
|
|
|
|
|
/// Transform tuple or subquery into a set.
|
|
|
|
|
makeSet(node, actions_stack.getSampleBlock());
|
|
|
|
|
}
|
|
|
|
|
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 insert an arbitrary column of the correct type.
|
|
|
|
|
ColumnWithTypeAndName fake_column;
|
|
|
|
|
fake_column.name = node->getColumnName();
|
|
|
|
|
fake_column.type = std::make_shared<DataTypeUInt8>();
|
|
|
|
|
actions_stack.addAction(ExpressionAction::addColumn(fake_column));
|
|
|
|
|
getActionsImpl(node->arguments->children.at(0), no_subqueries, only_consts, actions_stack);
|
|
|
|
|
}
|
|
|
|
|
return;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// A special function `indexHint`. Everything that is inside it is not calculated
|
|
|
|
|
/// (and is used only for index analysis, see PKCondition).
|
|
|
|
|
if (node->name == "indexHint")
|
|
|
|
|
{
|
|
|
|
|
actions_stack.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName(
|
|
|
|
|
ColumnConst::create(ColumnUInt8::create(1, 1), 1), std::make_shared<DataTypeUInt8>(), node->getColumnName())));
|
|
|
|
|
return;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
|
|
|
|
|
return;
|
|
|
|
|
|
|
|
|
|
const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get(node->name, context);
|
|
|
|
|
|
|
|
|
|
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 (auto & child : node->arguments->children)
|
|
|
|
|
{
|
|
|
|
|
ASTFunction * lambda = typeid_cast<ASTFunction *>(child.get());
|
|
|
|
|
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);
|
|
|
|
|
|
|
|
|
|
ASTFunction * lambda_args_tuple = typeid_cast<ASTFunction *>(lambda->arguments->children.at(0).get());
|
|
|
|
|
|
|
|
|
|
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 (prepared_sets.count(child.get()))
|
|
|
|
|
{
|
|
|
|
|
ColumnWithTypeAndName column;
|
|
|
|
|
column.type = std::make_shared<DataTypeSet>();
|
|
|
|
|
|
|
|
|
|
const SetPtr & set = prepared_sets[child.get()];
|
|
|
|
|
|
|
|
|
|
/// 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 record do not fuse together (they can have different types).
|
|
|
|
|
if (!set->empty())
|
|
|
|
|
column.name = getUniqueName(actions_stack.getSampleBlock(), "__set");
|
|
|
|
|
else
|
|
|
|
|
column.name = child->getColumnName();
|
|
|
|
|
|
|
|
|
|
if (!actions_stack.getSampleBlock().has(column.name))
|
|
|
|
|
{
|
|
|
|
|
column.column = ColumnSet::create(1, set);
|
|
|
|
|
|
|
|
|
|
actions_stack.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.
|
|
|
|
|
getActionsImpl(child, no_subqueries, only_consts, actions_stack);
|
|
|
|
|
std::string name = child->getColumnName();
|
|
|
|
|
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)
|
|
|
|
|
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 insert an arbitrary column of the correct type.
|
|
|
|
|
ColumnWithTypeAndName fake_column;
|
|
|
|
|
fake_column.name = node->getColumnName();
|
|
|
|
|
fake_column.type = std::make_shared<DataTypeUInt8>();
|
|
|
|
|
actions_stack.addAction(ExpressionAction::addColumn(fake_column));
|
|
|
|
|
getActionsImpl(node->arguments->children.at(0), no_subqueries, only_consts, actions_stack);
|
|
|
|
|
arguments_present = false;
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
throw Exception("Unknown identifier: " + name, ErrorCodes::UNKNOWN_IDENTIFIER);
|
|
|
|
|
}
|
|
|
|
|
return;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// A special function `indexHint`. Everything that is inside it is not calculated
|
|
|
|
|
/// (and is used only for index analysis, see PKCondition).
|
|
|
|
|
if (node->name == "indexHint")
|
|
|
|
|
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)
|
|
|
|
|
{
|
|
|
|
|
actions_stack.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName(
|
|
|
|
|
ColumnConst::create(ColumnUInt8::create(1, 1), 1), std::make_shared<DataTypeUInt8>(), node->getColumnName())));
|
|
|
|
|
return;
|
|
|
|
|
}
|
|
|
|
|
ASTPtr child = node->arguments->children[i];
|
|
|
|
|
|
|
|
|
|
const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get(node->name, context);
|
|
|
|
|
|
|
|
|
|
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 (auto & child : node->arguments->children)
|
|
|
|
|
{
|
|
|
|
|
ASTFunction * lambda = typeid_cast<ASTFunction *>(child.get());
|
|
|
|
|
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());
|
|
|
|
|
ASTFunction * lambda_args_tuple = typeid_cast<ASTFunction *>(lambda->arguments->children.at(0).get());
|
|
|
|
|
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 (prepared_sets.count(child.get()))
|
|
|
|
|
{
|
|
|
|
|
ColumnWithTypeAndName column;
|
|
|
|
|
column.type = std::make_shared<DataTypeSet>();
|
|
|
|
|
|
|
|
|
|
const SetPtr & set = prepared_sets[child.get()];
|
|
|
|
|
|
|
|
|
|
/// 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 record do not fuse together (they can have different types).
|
|
|
|
|
if (!set->empty())
|
|
|
|
|
column.name = getUniqueName(actions_stack.getSampleBlock(), "__set");
|
|
|
|
|
else
|
|
|
|
|
column.name = child->getColumnName();
|
|
|
|
|
|
|
|
|
|
if (!actions_stack.getSampleBlock().has(column.name))
|
|
|
|
|
for (size_t j = 0; j < lambda_arg_asts.size(); ++j)
|
|
|
|
|
{
|
|
|
|
|
column.column = ColumnSet::create(1, set);
|
|
|
|
|
ASTIdentifier * identifier = typeid_cast<ASTIdentifier *>(lambda_arg_asts[j].get());
|
|
|
|
|
if (!identifier)
|
|
|
|
|
throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH);
|
|
|
|
|
|
|
|
|
|
actions_stack.addAction(ExpressionAction::addColumn(column));
|
|
|
|
|
String arg_name = identifier->name;
|
|
|
|
|
|
|
|
|
|
lambda_arguments.emplace_back(arg_name, lambda_type->getArgumentTypes()[j]);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
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.
|
|
|
|
|
getActionsImpl(child, no_subqueries, only_consts, actions_stack);
|
|
|
|
|
std::string name = child->getColumnName();
|
|
|
|
|
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);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
actions_stack.pushLevel(lambda_arguments);
|
|
|
|
|
getActionsImpl(lambda->arguments->children.at(1), no_subqueries, only_consts, actions_stack);
|
|
|
|
|
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 (size_t j = 0; j < required.size(); ++j)
|
|
|
|
|
if (findColumn(required[j], lambda_arguments) == lambda_arguments.end())
|
|
|
|
|
captured.push_back(required[j]);
|
|
|
|
|
|
|
|
|
|
/// 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 && !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];
|
|
|
|
|
|
|
|
|
|
ASTFunction * lambda = typeid_cast<ASTFunction *>(child.get());
|
|
|
|
|
if (lambda && lambda->name == "lambda")
|
|
|
|
|
{
|
|
|
|
|
const DataTypeFunction * lambda_type = typeid_cast<const DataTypeFunction *>(argument_types[i].get());
|
|
|
|
|
ASTFunction * lambda_args_tuple = typeid_cast<ASTFunction *>(lambda->arguments->children.at(0).get());
|
|
|
|
|
ASTs lambda_arg_asts = lambda_args_tuple->arguments->children;
|
|
|
|
|
NamesAndTypesList lambda_arguments;
|
|
|
|
|
|
|
|
|
|
for (size_t j = 0; j < lambda_arg_asts.size(); ++j)
|
|
|
|
|
{
|
|
|
|
|
ASTIdentifier * identifier = typeid_cast<ASTIdentifier *>(lambda_arg_asts[j].get());
|
|
|
|
|
if (!identifier)
|
|
|
|
|
throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH);
|
|
|
|
|
|
|
|
|
|
String arg_name = identifier->name;
|
|
|
|
|
|
|
|
|
|
lambda_arguments.emplace_back(arg_name, lambda_type->getArgumentTypes()[j]);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
actions_stack.pushLevel(lambda_arguments);
|
|
|
|
|
getActionsImpl(lambda->arguments->children.at(1), no_subqueries, only_consts, actions_stack);
|
|
|
|
|
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 (size_t j = 0; j < required.size(); ++j)
|
|
|
|
|
if (findColumn(required[j], lambda_arguments) == lambda_arguments.end())
|
|
|
|
|
captured.push_back(required[j]);
|
|
|
|
|
|
|
|
|
|
/// 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 (size_t i = 0; i < argument_names.size(); ++i)
|
|
|
|
|
{
|
|
|
|
|
if (!actions_stack.getSampleBlock().has(argument_names[i]))
|
|
|
|
|
{
|
|
|
|
|
arguments_present = false;
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (arguments_present)
|
|
|
|
|
actions_stack.addAction(ExpressionAction::applyFunction(function_builder, argument_names, node->getColumnName()));
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (only_consts)
|
|
|
|
|
{
|
|
|
|
|
for (size_t i = 0; i < argument_names.size(); ++i)
|
|
|
|
|
{
|
|
|
|
|
if (!actions_stack.getSampleBlock().has(argument_names[i]))
|
|
|
|
|
{
|
|
|
|
|
arguments_present = false;
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (arguments_present)
|
|
|
|
|
actions_stack.addAction(ExpressionAction::applyFunction(function_builder, argument_names, node->getColumnName()));
|
|
|
|
|
}
|
|
|
|
|
else if (ASTLiteral * node = typeid_cast<ASTLiteral *>(ast.get()))
|
|
|
|
|
{
|
|
|
|
@ -2201,7 +2169,12 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
for (auto & child : ast->children)
|
|
|
|
|
getActionsImpl(child, no_subqueries, only_consts, actions_stack);
|
|
|
|
|
{
|
|
|
|
|
/// Do not go to FROM, JOIN, UNION.
|
|
|
|
|
if (!typeid_cast<const ASTTableExpression *>(child.get())
|
|
|
|
|
&& !typeid_cast<const ASTSelectQuery *>(child.get()))
|
|
|
|
|
getActionsImpl(child, no_subqueries, only_consts, actions_stack);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
@ -2223,7 +2196,7 @@ void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
const ASTFunction * node = typeid_cast<const ASTFunction *>(ast.get());
|
|
|
|
|
if (node && node->kind == ASTFunction::AGGREGATE_FUNCTION)
|
|
|
|
|
if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
|
|
|
|
|
{
|
|
|
|
|
has_aggregation = true;
|
|
|
|
|
AggregateDescription aggregate;
|
|
|
|
@ -2268,7 +2241,7 @@ void ExpressionAnalyzer::assertNoAggregates(const ASTPtr & ast, const char * des
|
|
|
|
|
{
|
|
|
|
|
const ASTFunction * node = typeid_cast<const ASTFunction *>(ast.get());
|
|
|
|
|
|
|
|
|
|
if (node && node->kind == ASTFunction::AGGREGATE_FUNCTION)
|
|
|
|
|
if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
|
|
|
|
|
throw Exception("Aggregate function " + node->getColumnName()
|
|
|
|
|
+ " is found " + String(description) + " in query", ErrorCodes::ILLEGAL_AGGREGATION);
|
|
|
|
|
|
|
|
|
@ -2570,7 +2543,7 @@ void ExpressionAnalyzer::getActionsBeforeAggregation(const ASTPtr & ast, Express
|
|
|
|
|
{
|
|
|
|
|
ASTFunction * node = typeid_cast<ASTFunction *>(ast.get());
|
|
|
|
|
|
|
|
|
|
if (node && node->kind == ASTFunction::AGGREGATE_FUNCTION)
|
|
|
|
|
if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
|
|
|
|
|
for (auto & argument : node->arguments->children)
|
|
|
|
|
getRootActions(argument, no_subqueries, false, actions);
|
|
|
|
|
else
|
|
|
|
@ -2836,7 +2809,7 @@ void ExpressionAnalyzer::getRequiredColumnsImpl(const ASTPtr & ast,
|
|
|
|
|
|
|
|
|
|
if (ASTFunction * node = typeid_cast<ASTFunction *>(ast.get()))
|
|
|
|
|
{
|
|
|
|
|
if (node->kind == ASTFunction::LAMBDA_EXPRESSION)
|
|
|
|
|
if (node->name == "lambda")
|
|
|
|
|
{
|
|
|
|
|
if (node->arguments->children.size() != 2)
|
|
|
|
|
throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|