mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Reject NULL actions on normal functions and fix serialization
This commit is contained in:
parent
9432061652
commit
7fa626bfef
@ -118,6 +118,7 @@ namespace ErrorCodes
|
|||||||
extern const int ILLEGAL_COLUMN;
|
extern const int ILLEGAL_COLUMN;
|
||||||
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
|
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
|
||||||
extern const int FUNCTION_CANNOT_HAVE_PARAMETERS;
|
extern const int FUNCTION_CANNOT_HAVE_PARAMETERS;
|
||||||
|
extern const int SYNTAX_ERROR;
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h first.
|
/** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h first.
|
||||||
@ -4593,6 +4594,19 @@ ProjectionNames QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_nod
|
|||||||
return result_projection_names;
|
return result_projection_names;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
void checkFunctionNodeHasEmptyNullsAction(FunctionNode const & node)
|
||||||
|
{
|
||||||
|
if (node.getNullsAction() != NullsAction::EMPTY)
|
||||||
|
throw Exception(
|
||||||
|
ErrorCodes::SYNTAX_ERROR,
|
||||||
|
"Function with name '{}' cannot use {} NULLS",
|
||||||
|
node.getFunctionName(),
|
||||||
|
node.getNullsAction() == NullsAction::IGNORE_NULLS ? "IGNORE" : "RESPECT");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/** Resolve function node in scope.
|
/** Resolve function node in scope.
|
||||||
* During function node resolve, function node can be replaced with another expression (if it match lambda or sql user defined function),
|
* During function node resolve, function node can be replaced with another expression (if it match lambda or sql user defined function),
|
||||||
* with constant (if it allow constant folding), or with expression list. It is caller responsibility to handle such cases appropriately.
|
* with constant (if it allow constant folding), or with expression list. It is caller responsibility to handle such cases appropriately.
|
||||||
@ -4751,6 +4765,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
|
|||||||
|
|
||||||
if (is_special_function_exists)
|
if (is_special_function_exists)
|
||||||
{
|
{
|
||||||
|
checkFunctionNodeHasEmptyNullsAction(*function_node_ptr);
|
||||||
/// Rewrite EXISTS (subquery) into 1 IN (SELECT 1 FROM (subquery) LIMIT 1).
|
/// Rewrite EXISTS (subquery) into 1 IN (SELECT 1 FROM (subquery) LIMIT 1).
|
||||||
auto & exists_subquery_argument = function_node_ptr->getArguments().getNodes().at(0);
|
auto & exists_subquery_argument = function_node_ptr->getArguments().getNodes().at(0);
|
||||||
|
|
||||||
@ -4771,6 +4786,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
|
|||||||
|
|
||||||
if (is_special_function_if && !function_node_ptr->getArguments().getNodes().empty())
|
if (is_special_function_if && !function_node_ptr->getArguments().getNodes().empty())
|
||||||
{
|
{
|
||||||
|
checkFunctionNodeHasEmptyNullsAction(*function_node_ptr);
|
||||||
/** Handle special case with constant If function, even if some of the arguments are invalid.
|
/** Handle special case with constant If function, even if some of the arguments are invalid.
|
||||||
*
|
*
|
||||||
* SELECT if(hasColumnInTable('system', 'numbers', 'not_existing_column'), not_existing_column, 5) FROM system.numbers;
|
* SELECT if(hasColumnInTable('system', 'numbers', 'not_existing_column'), not_existing_column, 5) FROM system.numbers;
|
||||||
@ -4836,6 +4852,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
|
|||||||
/// Replace right IN function argument if it is table or table function with subquery that read ordinary columns
|
/// Replace right IN function argument if it is table or table function with subquery that read ordinary columns
|
||||||
if (is_special_function_in)
|
if (is_special_function_in)
|
||||||
{
|
{
|
||||||
|
checkFunctionNodeHasEmptyNullsAction(function_node);
|
||||||
if (scope.context->getSettingsRef().transform_null_in)
|
if (scope.context->getSettingsRef().transform_null_in)
|
||||||
{
|
{
|
||||||
static constexpr std::array<std::pair<std::string_view, std::string_view>, 4> in_function_to_replace_null_in_function_map =
|
static constexpr std::array<std::pair<std::string_view, std::string_view>, 4> in_function_to_replace_null_in_function_map =
|
||||||
@ -5014,6 +5031,8 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
|
|||||||
lambda_expression_untyped->formatASTForErrorMessage(),
|
lambda_expression_untyped->formatASTForErrorMessage(),
|
||||||
scope.scope_node->formatASTForErrorMessage());
|
scope.scope_node->formatASTForErrorMessage());
|
||||||
|
|
||||||
|
checkFunctionNodeHasEmptyNullsAction(function_node);
|
||||||
|
|
||||||
if (!parameters.empty())
|
if (!parameters.empty())
|
||||||
{
|
{
|
||||||
throw Exception(
|
throw Exception(
|
||||||
@ -5043,6 +5062,8 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
|
|||||||
"Function 'untuple' must have 1 argument. In scope {}",
|
"Function 'untuple' must have 1 argument. In scope {}",
|
||||||
scope.scope_node->formatASTForErrorMessage());
|
scope.scope_node->formatASTForErrorMessage());
|
||||||
|
|
||||||
|
checkFunctionNodeHasEmptyNullsAction(function_node);
|
||||||
|
|
||||||
const auto & untuple_argument = function_arguments[0];
|
const auto & untuple_argument = function_arguments[0];
|
||||||
auto result_type = untuple_argument->getResultType();
|
auto result_type = untuple_argument->getResultType();
|
||||||
const auto * tuple_data_type = typeid_cast<const DataTypeTuple *>(result_type.get());
|
const auto * tuple_data_type = typeid_cast<const DataTypeTuple *>(result_type.get());
|
||||||
@ -5093,6 +5114,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
|
|||||||
throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION,
|
throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION,
|
||||||
"Function GROUPING can have up to 64 arguments, but {} provided",
|
"Function GROUPING can have up to 64 arguments, but {} provided",
|
||||||
function_arguments_size);
|
function_arguments_size);
|
||||||
|
checkFunctionNodeHasEmptyNullsAction(function_node);
|
||||||
|
|
||||||
bool force_grouping_standard_compatibility = scope.context->getSettingsRef().force_grouping_standard_compatibility;
|
bool force_grouping_standard_compatibility = scope.context->getSettingsRef().force_grouping_standard_compatibility;
|
||||||
auto grouping_function = std::make_shared<FunctionGrouping>(force_grouping_standard_compatibility);
|
auto grouping_function = std::make_shared<FunctionGrouping>(force_grouping_standard_compatibility);
|
||||||
@ -5146,7 +5168,11 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
|
|||||||
is_executable_udf = false;
|
is_executable_udf = false;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!function)
|
if (function)
|
||||||
|
{
|
||||||
|
checkFunctionNodeHasEmptyNullsAction(function_node);
|
||||||
|
}
|
||||||
|
else
|
||||||
{
|
{
|
||||||
if (!AggregateFunctionFactory::instance().isAggregateFunctionName(function_name))
|
if (!AggregateFunctionFactory::instance().isAggregateFunctionName(function_name))
|
||||||
{
|
{
|
||||||
|
@ -846,6 +846,19 @@ void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr &, Dat
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
void checkFunctionHasEmptyNullsAction(const ASTFunction & node)
|
||||||
|
{
|
||||||
|
if (node.nulls_action != NullsAction::EMPTY)
|
||||||
|
throw Exception(
|
||||||
|
ErrorCodes::SYNTAX_ERROR,
|
||||||
|
"Function {} cannot use {} NULLS",
|
||||||
|
node.name,
|
||||||
|
node.nulls_action == NullsAction::IGNORE_NULLS ? "IGNORE" : "RESPECT");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & data)
|
void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & data)
|
||||||
{
|
{
|
||||||
auto column_name = ast->getColumnName();
|
auto column_name = ast->getColumnName();
|
||||||
@ -860,6 +873,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
|||||||
{
|
{
|
||||||
if (node.arguments->children.size() != 1)
|
if (node.arguments->children.size() != 1)
|
||||||
throw Exception(ErrorCodes::TYPE_MISMATCH, "arrayJoin requires exactly 1 argument");
|
throw Exception(ErrorCodes::TYPE_MISMATCH, "arrayJoin requires exactly 1 argument");
|
||||||
|
checkFunctionHasEmptyNullsAction(node);
|
||||||
|
|
||||||
ASTPtr arg = node.arguments->children.at(0);
|
ASTPtr arg = node.arguments->children.at(0);
|
||||||
visit(arg, data);
|
visit(arg, data);
|
||||||
@ -871,6 +885,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
|||||||
|
|
||||||
if (node.name == "grouping")
|
if (node.name == "grouping")
|
||||||
{
|
{
|
||||||
|
checkFunctionHasEmptyNullsAction(node);
|
||||||
if (data.only_consts)
|
if (data.only_consts)
|
||||||
return; // Can not perform constant folding, because this function can be executed only after GROUP BY
|
return; // Can not perform constant folding, because this function can be executed only after GROUP BY
|
||||||
|
|
||||||
@ -922,6 +937,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
|||||||
FutureSetPtr prepared_set;
|
FutureSetPtr prepared_set;
|
||||||
if (checkFunctionIsInOrGlobalInOperator(node))
|
if (checkFunctionIsInOrGlobalInOperator(node))
|
||||||
{
|
{
|
||||||
|
checkFunctionHasEmptyNullsAction(node);
|
||||||
/// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything).
|
/// 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);
|
visit(node.arguments->children.at(0), data);
|
||||||
|
|
||||||
@ -952,6 +968,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
|||||||
/// A special function `indexHint`. Everything that is inside it is not calculated
|
/// A special function `indexHint`. Everything that is inside it is not calculated
|
||||||
if (node.name == "indexHint")
|
if (node.name == "indexHint")
|
||||||
{
|
{
|
||||||
|
checkFunctionHasEmptyNullsAction(node);
|
||||||
if (data.only_consts)
|
if (data.only_consts)
|
||||||
{
|
{
|
||||||
/// We need to collect constants inside `indexHint` for index analysis.
|
/// We need to collect constants inside `indexHint` for index analysis.
|
||||||
@ -1090,6 +1107,8 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
|||||||
throw Exception(ErrorCodes::FUNCTION_CANNOT_HAVE_PARAMETERS, "Function {} is not parametric", node.name);
|
throw Exception(ErrorCodes::FUNCTION_CANNOT_HAVE_PARAMETERS, "Function {} is not parametric", node.name);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
checkFunctionHasEmptyNullsAction(node);
|
||||||
|
|
||||||
Names argument_names;
|
Names argument_names;
|
||||||
DataTypes argument_types;
|
DataTypes argument_types;
|
||||||
bool arguments_present = true;
|
bool arguments_present = true;
|
||||||
|
@ -539,9 +539,9 @@ void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const
|
|||||||
writeChar(')', ostr);
|
writeChar(')', ostr);
|
||||||
|
|
||||||
if (nulls_action == NullsAction::RESPECT_NULLS)
|
if (nulls_action == NullsAction::RESPECT_NULLS)
|
||||||
writeCString(" RESPECT NULLS ", ostr);
|
writeCString(" RESPECT NULLS", ostr);
|
||||||
else if (nulls_action == NullsAction::IGNORE_NULLS)
|
else if (nulls_action == NullsAction::IGNORE_NULLS)
|
||||||
writeCString(" IGNORE NULLS ", ostr);
|
writeCString(" IGNORE NULLS", ostr);
|
||||||
|
|
||||||
if (is_window_function)
|
if (is_window_function)
|
||||||
{
|
{
|
||||||
@ -565,9 +565,9 @@ void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const
|
|||||||
void ASTFunction::finishFormatWithWindow(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
|
void ASTFunction::finishFormatWithWindow(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
|
||||||
{
|
{
|
||||||
if (nulls_action == NullsAction::RESPECT_NULLS)
|
if (nulls_action == NullsAction::RESPECT_NULLS)
|
||||||
settings.ostr << " RESPECT NULLS ";
|
settings.ostr << " RESPECT NULLS";
|
||||||
else if (nulls_action == NullsAction::IGNORE_NULLS)
|
else if (nulls_action == NullsAction::IGNORE_NULLS)
|
||||||
settings.ostr << " IGNORE NULLS ";
|
settings.ostr << " IGNORE NULLS";
|
||||||
|
|
||||||
if (!is_window_function)
|
if (!is_window_function)
|
||||||
return;
|
return;
|
||||||
|
@ -1,5 +1,5 @@
|
|||||||
SELECT first_value(number) RESPECT NULLS \nFROM numbers(1)
|
SELECT first_value(number) RESPECT NULLS\nFROM numbers(1)
|
||||||
SELECT first_value(number) IGNORE NULLS \nFROM numbers(1)
|
SELECT first_value(number) IGNORE NULLS\nFROM numbers(1)
|
||||||
SELECT any(number) RESPECT NULLS \nFROM numbers(1)
|
SELECT any(number) RESPECT NULLS\nFROM numbers(1)
|
||||||
SELECT LAST_VALUE(number) RESPECT NULLS \nFROM numbers(1)
|
SELECT LAST_VALUE(number) RESPECT NULLS\nFROM numbers(1)
|
||||||
SELECT sum(number) RESPECT NULLS \nFROM numbers(1)
|
SELECT sum(number) RESPECT NULLS\nFROM numbers(1)
|
||||||
|
@ -1,9 +1,19 @@
|
|||||||
SELECT first_value(number) RESPECT NULLS IGNORE NULLS from numbers(1); -- { clientError SYNTAX_ERROR }
|
SELECT first_value(number) RESPECT NULLS IGNORE NULLS from numbers(1); -- { clientError SYNTAX_ERROR }
|
||||||
|
|
||||||
SELECT formatQuery('Select first_value(number) RESPECT NULLS from numbers(1)');
|
SELECT formatQuery('SELECT first_value(number) RESPECT NULLS from numbers(1)');
|
||||||
SELECT formatQuery('Select first_value(number) IGNORE NULLS from numbers(1)');
|
SELECT formatQuery('SELECT first_value(number) IGNORE NULLS from numbers(1)');
|
||||||
SELECT formatQuery('Select any (number) RESPECT NULLS from numbers(1)');
|
SELECT formatQuery('SELECT any (number) RESPECT NULLS from numbers(1)');
|
||||||
SELECT formatQuery('Select LAST_VALUE(number) RESPECT NULLS from numbers(1)');
|
SELECT formatQuery('SELECT LAST_VALUE(number) RESPECT NULLS from numbers(1)');
|
||||||
|
|
||||||
-- The parser doesn't know if this function supports "RESPECT/IGNORE" NULLS
|
-- The parser doesn't know if this function supports "RESPECT/IGNORE" NULLS
|
||||||
SELECT formatQuery('Select sum(number) RESPECT NULLS from numbers(1)');
|
SELECT formatQuery('SELECT sum(number) RESPECT NULLS from numbers(1)');
|
||||||
|
|
||||||
|
-- Normal functions should throw in the server
|
||||||
|
SELECT toDateTimeNonExistingFunction(now()) RESPECT NULLS b; -- { serverError UNKNOWN_FUNCTION }
|
||||||
|
SELECT toDateTime(now()) RESPECT NULLS b; -- { serverError SYNTAX_ERROR }
|
||||||
|
SELECT count() from numbers(10) where in(number, (0)) RESPECT NULLS; -- { serverError SYNTAX_ERROR }
|
||||||
|
SELECT if(number > 0, number, 0) respect nulls from numbers(0); -- { serverError SYNTAX_ERROR }
|
||||||
|
WITH (x -> x + 1) AS lambda SELECT lambda(number) RESPECT NULLS FROM numbers(10) SETTINGS allow_experimental_analyzer = 1; -- { serverError SYNTAX_ERROR }
|
||||||
|
SELECT * from system.one WHERE indexHint(dummy = 1) RESPECT NULLS; -- { serverError SYNTAX_ERROR }
|
||||||
|
SELECT arrayJoin([[3,4,5], [6,7], [2], [1,1]]) IGNORE NULLS; -- { serverError SYNTAX_ERROR }
|
||||||
|
SELECT number, grouping(number % 2, number) RESPECT NULLS AS gr FROM numbers(10) GROUP BY GROUPING SETS ((number), (number % 2)) SETTINGS force_grouping_standard_compatibility = 0; -- { serverError SYNTAX_ERROR }
|
||||||
|
Loading…
Reference in New Issue
Block a user