ClickHouse/src/Analyzer/FunctionNode.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

255 lines
8.0 KiB
C++
Raw Normal View History

2022-07-14 11:20:16 +00:00
#include <Analyzer/FunctionNode.h>
#include <Common/SipHash.h>
2022-08-31 15:21:17 +00:00
#include <Common/FieldVisitorToString.h>
2022-07-14 11:20:16 +00:00
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypeSet.h>
2022-09-12 14:14:40 +00:00
#include <Parsers/ASTFunction.h>
2022-07-14 11:20:16 +00:00
#include <Functions/IFunction.h>
2022-09-12 14:14:40 +00:00
2022-07-14 11:20:16 +00:00
#include <AggregateFunctions/IAggregateFunction.h>
#include <Analyzer/Utils.h>
#include <Analyzer/ConstantNode.h>
2022-09-12 14:14:40 +00:00
#include <Analyzer/IdentifierNode.h>
2022-07-14 11:20:16 +00:00
namespace DB
{
2022-12-05 14:41:37 +00:00
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
2022-08-26 13:06:14 +00:00
FunctionNode::FunctionNode(String function_name_)
2022-10-07 10:44:28 +00:00
: IQueryTreeNode(children_size)
, function_name(function_name_)
2022-08-26 13:06:14 +00:00
{
children[parameters_child_index] = std::make_shared<ListNode>();
children[arguments_child_index] = std::make_shared<ListNode>();
}
const DataTypes & FunctionNode::getArgumentTypes() const
{
if (!function)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Function {} is not resolved",
function_name);
return function->getArgumentTypes();
}
2022-12-23 18:23:01 +00:00
ColumnsWithTypeAndName FunctionNode::getArgumentColumns() const
2022-07-14 11:20:16 +00:00
{
2022-12-23 18:23:01 +00:00
const auto & arguments = getArguments().getNodes();
size_t arguments_size = arguments.size();
2022-12-23 18:23:01 +00:00
ColumnsWithTypeAndName argument_columns;
argument_columns.reserve(arguments.size());
for (size_t i = 0; i < arguments_size; ++i)
2022-11-28 15:02:59 +00:00
{
const auto & argument = arguments[i];
ColumnWithTypeAndName argument_column;
if (isNameOfInFunction(function_name) && i == 1)
argument_column.type = std::make_shared<DataTypeSet>();
else
argument_column.type = argument->getResultType();
2023-03-08 13:39:48 +00:00
auto * constant = argument->as<ConstantNode>();
if (constant && !isNotCreatable(argument_column.type))
argument_column.column = argument_column.type->createColumnConst(1, constant->getValue());
argument_columns.push_back(std::move(argument_column));
2022-11-28 15:02:59 +00:00
}
2022-12-23 18:23:01 +00:00
return argument_columns;
2022-11-28 15:02:59 +00:00
}
void FunctionNode::resolveAsFunction(FunctionBasePtr function_value)
{
function_name = function_value->getName();
2022-07-14 11:20:16 +00:00
function = std::move(function_value);
2022-12-01 18:51:57 +00:00
kind = FunctionKind::ORDINARY;
2022-07-14 11:20:16 +00:00
}
2022-11-28 15:02:59 +00:00
void FunctionNode::resolveAsAggregateFunction(AggregateFunctionPtr aggregate_function_value)
2022-07-14 11:20:16 +00:00
{
2022-11-28 15:02:59 +00:00
function_name = aggregate_function_value->getName();
function = std::move(aggregate_function_value);
2022-12-01 18:51:57 +00:00
kind = FunctionKind::AGGREGATE;
2022-07-14 11:20:16 +00:00
}
2022-11-28 15:02:59 +00:00
void FunctionNode::resolveAsWindowFunction(AggregateFunctionPtr window_function_value)
2022-09-12 14:14:40 +00:00
{
2022-12-01 18:51:57 +00:00
if (!hasWindow())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Trying to resolve FunctionNode without window definition as a window function {}", window_function_value->getName());
2022-11-28 15:02:59 +00:00
resolveAsAggregateFunction(window_function_value);
2022-12-01 18:51:57 +00:00
kind = FunctionKind::WINDOW;
2022-09-12 14:14:40 +00:00
}
void FunctionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const
2022-07-14 11:20:16 +00:00
{
buffer << std::string(indent, ' ') << "FUNCTION id: " << format_state.getNodeId(this);
if (hasAlias())
buffer << ", alias: " << getAlias();
buffer << ", function_name: " << function_name;
2022-09-12 14:14:40 +00:00
std::string function_type = "ordinary";
if (isAggregateFunction())
function_type = "aggregate";
else if (isWindowFunction())
function_type = "window";
buffer << ", function_type: " << function_type;
if (nulls_action == NullsAction::RESPECT_NULLS)
buffer << ", nulls_action : RESPECT_NULLS";
else if (nulls_action == NullsAction::IGNORE_NULLS)
buffer << ", nulls_action : IGNORE_NULLS";
2022-11-28 15:02:59 +00:00
if (function)
2023-02-01 13:33:32 +00:00
buffer << ", result_type: " + getResultType()->getName();
2022-07-14 11:20:16 +00:00
const auto & parameters = getParameters();
if (!parameters.getNodes().empty())
{
buffer << '\n' << std::string(indent + 2, ' ') << "PARAMETERS\n";
parameters.dumpTreeImpl(buffer, format_state, indent + 4);
2022-07-14 11:20:16 +00:00
}
const auto & arguments = getArguments();
if (!arguments.getNodes().empty())
{
buffer << '\n' << std::string(indent + 2, ' ') << "ARGUMENTS\n";
arguments.dumpTreeImpl(buffer, format_state, indent + 4);
2022-07-14 11:20:16 +00:00
}
2022-09-12 14:14:40 +00:00
if (hasWindow())
{
buffer << '\n' << std::string(indent + 2, ' ') << "WINDOW\n";
getWindowNode()->dumpTreeImpl(buffer, format_state, indent + 4);
}
2022-07-14 11:20:16 +00:00
}
2022-07-15 13:32:53 +00:00
bool FunctionNode::isEqualImpl(const IQueryTreeNode & rhs) const
{
const auto & rhs_typed = assert_cast<const FunctionNode &>(rhs);
if (function_name != rhs_typed.function_name || isAggregateFunction() != rhs_typed.isAggregateFunction()
|| isOrdinaryFunction() != rhs_typed.isOrdinaryFunction() || isWindowFunction() != rhs_typed.isWindowFunction()
|| nulls_action != rhs_typed.nulls_action)
2022-07-15 13:32:53 +00:00
return false;
2022-12-15 19:47:35 +00:00
if (isResolved() != rhs_typed.isResolved())
return false;
if (!isResolved())
return true;
2022-11-28 15:02:59 +00:00
auto lhs_result_type = getResultType();
auto rhs_result_type = rhs.getResultType();
2022-07-15 13:32:53 +00:00
2022-11-28 15:02:59 +00:00
if (lhs_result_type && rhs_result_type && !lhs_result_type->equals(*rhs_result_type))
2022-08-31 15:21:17 +00:00
return false;
2022-11-28 15:02:59 +00:00
else if (lhs_result_type && !rhs_result_type)
2022-07-15 13:32:53 +00:00
return false;
2022-11-28 15:02:59 +00:00
else if (!lhs_result_type && rhs_result_type)
2022-07-15 13:32:53 +00:00
return false;
2022-08-31 15:21:17 +00:00
return true;
2022-07-15 13:32:53 +00:00
}
2022-07-14 11:20:16 +00:00
void FunctionNode::updateTreeHashImpl(HashState & hash_state) const
{
hash_state.update(function_name.size());
hash_state.update(function_name);
2022-10-03 12:22:38 +00:00
hash_state.update(isOrdinaryFunction());
2022-07-14 11:20:16 +00:00
hash_state.update(isAggregateFunction());
2022-09-12 14:14:40 +00:00
hash_state.update(isWindowFunction());
hash_state.update(nulls_action);
2022-07-14 11:20:16 +00:00
2022-12-15 19:47:35 +00:00
if (!isResolved())
return;
2022-11-28 15:02:59 +00:00
if (auto result_type = getResultType())
2022-07-14 11:20:16 +00:00
{
auto result_type_name = result_type->getName();
hash_state.update(result_type_name.size());
hash_state.update(result_type_name);
}
}
QueryTreeNodePtr FunctionNode::cloneImpl() const
{
auto result_function = std::make_shared<FunctionNode>(function_name);
/** This is valid for clone method to reuse same function pointers
* because ordinary functions or aggregate functions must be stateless.
*/
result_function->function = function;
result_function->kind = kind;
result_function->nulls_action = nulls_action;
2023-02-01 13:33:32 +00:00
result_function->wrap_with_nullable = wrap_with_nullable;
return result_function;
}
2023-03-14 09:14:58 +00:00
ASTPtr FunctionNode::toASTImpl(const ConvertToASTOptions & options) const
2022-07-14 11:20:16 +00:00
{
auto function_ast = std::make_shared<ASTFunction>();
function_ast->name = function_name;
function_ast->nulls_action = nulls_action;
if (isWindowFunction())
{
function_ast->is_window_function = true;
function_ast->kind = ASTFunction::Kind::WINDOW_FUNCTION;
}
2022-09-12 14:14:40 +00:00
2023-11-22 12:40:24 +00:00
const auto & arguments = getArguments();
auto new_options = options;
2023-11-22 12:40:24 +00:00
const auto & argument_nodes = arguments.getNodes();
2023-05-12 14:14:40 +00:00
/// To avoid surrounding constants with several internal casts.
2023-11-22 12:40:24 +00:00
if (function_name == "_CAST" && !argument_nodes.empty() && argument_nodes[0]->getNodeType() == QueryTreeNodeType::CONSTANT)
new_options.add_cast_for_constants = false;
2023-08-10 18:41:57 +00:00
/// Avoid cast for `IN tuple(...)` expression.
/// Tuples could be quite big, and adding a type may significantly increase query size.
2023-08-10 18:41:57 +00:00
/// It should be safe because set type for `column IN tuple` is deduced from `column` type.
2023-11-22 12:40:24 +00:00
if (isNameOfInFunction(function_name) && argument_nodes.size() > 1 && argument_nodes[1]->getNodeType() == QueryTreeNodeType::CONSTANT)
2023-08-10 18:41:57 +00:00
new_options.add_cast_for_constants = false;
2022-07-14 11:20:16 +00:00
const auto & parameters = getParameters();
if (!parameters.getNodes().empty())
{
function_ast->children.push_back(parameters.toAST(new_options));
2022-07-14 11:20:16 +00:00
function_ast->parameters = function_ast->children.back();
}
function_ast->children.push_back(arguments.toAST(new_options));
2022-08-26 16:23:26 +00:00
function_ast->arguments = function_ast->children.back();
2022-07-14 11:20:16 +00:00
2022-10-19 10:25:27 +00:00
auto window_node = getWindowNode();
if (window_node)
{
if (auto * identifier_node = window_node->as<IdentifierNode>())
function_ast->window_name = identifier_node->getIdentifier().getFullName();
else
function_ast->window_definition = window_node->toAST(new_options);
2022-10-19 10:25:27 +00:00
}
2022-07-14 11:20:16 +00:00
return function_ast;
}
}