Merge pull request #63406 from yohannj/fix_error_codes

Update ErrorCodes for functions using NUMBER_OF_ARGUMENTS_DOESNT_MATCH
This commit is contained in:
Yarik Briukhovetskyi 2024-05-08 15:12:00 +00:00 committed by GitHub
commit 767e32d566
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
37 changed files with 94 additions and 89 deletions

View File

@ -16,7 +16,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int AGGREGATE_FUNCTION_THROW;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
}
namespace
@ -116,7 +116,7 @@ void registerAggregateFunctionAggThrow(AggregateFunctionFactory & factory)
if (parameters.size() == 1)
throw_probability = parameters[0].safeGet<Float64>();
else if (parameters.size() > 1)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} cannot have more than one parameter", name);
throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} cannot have more than one parameter", name);
return std::make_shared<AggregateFunctionThrow>(argument_types, parameters, throw_probability);
});

View File

@ -27,6 +27,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
extern const int TOO_LARGE_ARRAY_SIZE;
extern const int CANNOT_CONVERT_TYPE;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
@ -74,7 +75,7 @@ public:
if (!params.empty())
{
if (params.size() > 2)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at most two parameters.", getName());
throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at most two parameters.", getName());
default_value = params[0];

View File

@ -22,7 +22,8 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
}
namespace
@ -34,12 +35,12 @@ namespace
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
if (parameters.size() > 4)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION,
"Aggregate function {} requires at most four parameters: "
"learning_rate, l2_regularization_coef, mini-batch size and weights_updater method", name);
if (argument_types.size() < 2)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION,
"Aggregate function {} requires at least two arguments: target and model's parameters", name);
for (size_t i = 0; i < argument_types.size(); ++i)

View File

@ -17,19 +17,19 @@
#include <boost/math/distributions/normal.hpp>
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
}
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
extern const int BAD_ARGUMENTS;
}
namespace
{
@ -141,7 +141,7 @@ public:
: IAggregateFunctionDataHelper<MannWhitneyData, AggregateFunctionMannWhitney> ({arguments}, {}, createResultType())
{
if (params.size() > 2)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} require two parameter or less", getName());
throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} require two parameter or less", getName());
if (params.empty())
{

View File

@ -14,7 +14,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NOT_IMPLEMENTED;
}
@ -118,7 +118,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);

View File

@ -12,7 +12,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -27,7 +27,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);

View File

@ -13,7 +13,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -29,7 +29,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);

View File

@ -13,7 +13,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -29,7 +29,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);

View File

@ -13,7 +13,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -28,7 +28,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);

View File

@ -13,7 +13,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -28,7 +28,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);

View File

@ -35,7 +35,7 @@ namespace ErrorCodes
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
}
@ -467,7 +467,7 @@ AggregateFunctionPtr createAggregateFunctionTopK(const std::string & name, const
if (!params.empty())
{
if (params.size() > 3)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION,
"Aggregate function '{}' requires three parameters or less", name);
threshold = applyVisitor(FieldVisitorConvertToNumber<UInt64>(), params[0]);

View File

@ -4,16 +4,16 @@
#include <AggregateFunctions/Moments.h>
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
}
namespace
{
@ -80,7 +80,7 @@ AggregateFunctionPtr createAggregateFunctionWelchTTest(
assertBinary(name, argument_types);
if (parameters.size() > 1)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires zero or one parameter.", name);
throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires zero or one parameter.", name);
if (!isNumber(argument_types[0]) || !isNumber(argument_types[1]))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Aggregate function {} only supports numerical types", name);

View File

@ -9,7 +9,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -26,7 +26,7 @@ public:
DataTypes transformArguments(const DataTypes & arguments) const override
{
if (arguments.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "-Array aggregate functions require at least one argument");
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Array aggregate functions require at least one argument");
DataTypes nested_arguments;
for (const auto & type : arguments)

View File

@ -18,7 +18,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -42,7 +42,7 @@ public:
, nested_func(nested), num_arguments(types.size())
{
if (num_arguments == 0)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} require at least one argument", getName());
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} require at least one argument", getName());
only_null_condition = types.back()->onlyNull();

View File

@ -11,7 +11,7 @@ namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
@ -35,7 +35,7 @@ public:
{
if (arguments.empty())
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION,
"Function {} needs at least one argument; passed {}.",
getName(),
arguments.size());

View File

@ -10,7 +10,7 @@ namespace DB
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
}
template <typename Name, typename Impl>
@ -37,7 +37,7 @@ struct MultiSearchFirstPositionImpl
{
// For performance of Volnitsky search, it is crucial to save only one byte for pattern number.
if (needles_arr.size() > std::numeric_limits<UInt8>::max())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION,
"Number of arguments for function {} doesn't match: passed {}, should be at most {}",
name, std::to_string(needles_arr.size()), std::to_string(std::numeric_limits<UInt8>::max()));

View File

@ -10,7 +10,7 @@ namespace DB
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
}
template <typename Name, typename Impl>
@ -37,7 +37,7 @@ struct MultiSearchImpl
{
// For performance of Volnitsky search, it is crucial to save only one byte for pattern number.
if (needles_arr.size() > std::numeric_limits<UInt8>::max())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION,
"Number of arguments for function {} doesn't match: passed {}, should be at most {}",
name, needles_arr.size(), std::to_string(std::numeric_limits<UInt8>::max()));

View File

@ -18,7 +18,7 @@ namespace DB
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int SIZES_OF_ARRAYS_DONT_MATCH;
@ -43,7 +43,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION,
"Number of arguments for function {} doesn't match: passed {}, should be at least 1.",
getName(), arguments.size());

View File

@ -59,7 +59,8 @@ namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int SIZES_OF_ARRAYS_DONT_MATCH;
}
@ -101,7 +102,7 @@ public:
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION,
"Number of arguments for function {} doesn't match: passed {}, should be at least 1.",
getName(), arguments.size());
@ -238,7 +239,7 @@ ColumnPtr FunctionArrayEnumerateRankedExtended<Derived>::executeImpl(
}
if (offsets_by_depth.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "No arrays passed to function {}", getName());
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No arrays passed to function {}", getName());
auto res_nested = ColumnUInt32::create();

View File

@ -14,7 +14,7 @@ namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int SIZES_OF_ARRAYS_DONT_MATCH;
extern const int TYPE_MISMATCH;
}
@ -41,7 +41,7 @@ public:
void getLambdaArgumentTypes(DataTypes & arguments) const override
{
if (arguments.size() < 3)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires as arguments a lambda function, at least one array and an accumulator", getName());
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} requires as arguments a lambda function, at least one array and an accumulator", getName());
DataTypes accumulator_and_array_types(arguments.size() - 1);
accumulator_and_array_types[0] = arguments.back();
@ -64,7 +64,7 @@ public:
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.size() < 3)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires as arguments a lambda function, at least one array and an accumulator", getName());
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} requires as arguments a lambda function, at least one array and an accumulator", getName());
const auto * lambda_function_type = checkAndGetDataType<DataTypeFunction>(arguments[0].type.get());
if (!lambda_function_type)

View File

@ -21,7 +21,7 @@ namespace DB
namespace ErrorCodes
{
extern const int SIZES_OF_ARRAYS_DONT_MATCH;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int BAD_ARGUMENTS;
@ -73,7 +73,7 @@ DataTypePtr FunctionArrayReduce::getReturnTypeImpl(const ColumnsWithTypeAndName
/// (possibly with parameters in parentheses, for example: "quantile(0.99)").
if (arguments.size() < 2)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION,
"Number of arguments for function {} doesn't match: passed {}, should be at least 2.",
getName(), arguments.size());

View File

@ -14,7 +14,7 @@ namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int SIZES_OF_ARRAYS_DONT_MATCH;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int ILLEGAL_COLUMN;
}
@ -39,7 +39,7 @@ public:
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION,
"Function {} needs at least one argument; passed {}." , getName(), arguments.size());
DataTypes arguments_types;

View File

@ -16,7 +16,7 @@ namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
}
using namespace GatherUtils;
@ -48,7 +48,7 @@ public:
{
if (arguments.size() < 2)
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION,
"Number of arguments for function {} doesn't match: passed {}, should be at least 2",
getName(),
arguments.size());
@ -225,7 +225,7 @@ public:
{
if (arguments.empty())
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION,
"Number of arguments for function {} doesn't match: passed {}, should be at least 1.",
getName(),
arguments.size());

View File

@ -17,7 +17,7 @@ namespace DB
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
}
class FunctionGenerateULID : public IFunction
@ -45,7 +45,7 @@ public:
{
if (arguments.size() > 1)
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION,
"Number of arguments for function {} doesn't match: passed {}, should be 0 or 1.",
getName(), arguments.size());

View File

@ -25,7 +25,7 @@ namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int ILLEGAL_COLUMN;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -53,7 +53,7 @@ namespace
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least one argument.", getName());
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} requires at least one argument.", getName());
for (const auto & arg : arguments)
if (!isString(arg.type))

View File

@ -18,9 +18,10 @@ namespace DB
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int SIZES_OF_ARRAYS_DONT_MATCH;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
}
namespace
@ -64,19 +65,19 @@ public:
{
size_t arguments_size = arguments.size();
if (arguments_size < 2)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION,
"Number of arguments for function {} doesn't match: passed {}, should be at least 2",
getName(),
arguments_size);
Names nested_names = extractNestedNames(arguments[0].column);
if (nested_names.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"First argument for function {} must be constant column with array of strings",
getName());
if (nested_names.size() != arguments_size - 1)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Size of nested names array for function {} does not match arrays arguments size. Actual {}. Expected {}",
getName(),
nested_names.size(),

View File

@ -18,7 +18,7 @@ namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int CANNOT_CLOCK_GETTIME;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
}
namespace
@ -128,7 +128,7 @@ public:
if (arguments.size() > 2)
{
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Arguments size of function {} should be 0, or 1, or 2", getName());
throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Arguments size of function {} should be 0, or 1, or 2", getName());
}
if (!arguments.empty())
{

View File

@ -12,7 +12,7 @@ namespace DB
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -63,7 +63,7 @@ public:
{
if (arguments.size() > 1)
{
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Arguments size of function {} should be 0 or 1", getName());
throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Arguments size of function {} should be 0 or 1", getName());
}
if (arguments.size() == 1 && !isStringOrFixedString(arguments[0].type))
{

View File

@ -11,7 +11,8 @@ namespace DB
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int BAD_ARGUMENTS;
}
@ -117,14 +118,14 @@ namespace
{
if (arguments.empty())
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION,
"Number of arguments for function {} doesn't match: passed {}, should be 1.",
getName(),
arguments.size());
if (arguments.size() > 1)
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION,
"Number of arguments for function {} doesn't match: passed {}, should be 1.",
getName(),
arguments.size());

View File

@ -37,7 +37,7 @@ namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int BAD_ARGUMENTS;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
@ -87,7 +87,7 @@ public:
{
if (arguments.size() < 2)
{
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least 2 arguments", getName());
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} requires at least 2 arguments", getName());
}
/** We allow function invocation in one of the following forms:

View File

@ -25,7 +25,7 @@ namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
}
@ -112,7 +112,7 @@ void TableFunctionValues::parseArguments(const ASTPtr & ast_function, ContextPtr
ASTs & args = args_func.at(0)->children;
if (args.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' requires at least 1 argument", getName());
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Table function '{}' requires at least 1 argument", getName());
const auto & literal = args[0]->as<const ASTLiteral>();
String value;

View File

@ -93,4 +93,4 @@ SELECT concat(materialize(NULL :: Nullable(UInt64)));
SELECT CONCAT('Testing the ', 'alias');
SELECT concat(); -- { serverError 42 }
SELECT concat(); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION }

View File

@ -152,7 +152,7 @@ DROP TABLE arrays_test;
select '---------BAD';
SELECT arrayEnumerateUniqRanked(); -- { serverError 42 }
SELECT arrayEnumerateUniqRanked(); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION }
SELECT arrayEnumerateUniqRanked([]);
SELECT arrayEnumerateUniqRanked(1); -- { serverError 36 }
SELECT arrayEnumerateUniqRanked(2,[]); -- { serverError 36 }

View File

@ -2,7 +2,7 @@ SELECT arrayZip(['a', 'b', 'c'], ['d', 'e', 'f']);
SELECT arrayZip(['a', 'b', 'c'], ['d', 'e', 'f'], ['g', 'h', 'i']);
SELECT arrayZip(); -- { serverError 42 }
SELECT arrayZip(); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION }
SELECT arrayZip('a', 'b', 'c'); -- { serverError 43 }

View File

@ -11,8 +11,8 @@ SELECT parseTimeDelta('1s - 1ms : 1μs ; 1ns');
SELECT parseTimeDelta('1.11s1.11ms1.11us1.11ns');
-- invalid expressions
SELECT parseTimeDelta(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH}
SELECT parseTimeDelta('1yr', 1); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH}
SELECT parseTimeDelta(); -- {serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION}
SELECT parseTimeDelta('1yr', 1); -- {serverError TOO_MANY_ARGUMENTS_FOR_FUNCTION}
SELECT parseTimeDelta(1); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT}
SELECT parseTimeDelta(' '); -- {serverError BAD_ARGUMENTS}
SELECT parseTimeDelta('-1yr'); -- {serverError BAD_ARGUMENTS}

View File

@ -1,4 +1,4 @@
SELECT count() FROM (SELECT DISTINCT nowInBlock(), nowInBlock('Pacific/Pitcairn') FROM system.numbers LIMIT 2);
SELECT nowInBlock(1); -- { serverError 43 }
SELECT nowInBlock(NULL) IS NULL;
SELECT nowInBlock('UTC', 'UTC'); -- { serverError 42 }
SELECT nowInBlock('UTC', 'UTC'); -- { serverError TOO_MANY_ARGUMENTS_FOR_FUNCTION }

View File

@ -1,7 +1,7 @@
SELECT '-- Negative tests';
SELECT arrayFold(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT arrayFold(1); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT arrayFold(1, toUInt64(0)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT arrayFold(); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION }
SELECT arrayFold(1); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION }
SELECT arrayFold(1, toUInt64(0)); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION }
SELECT arrayFold(1, emptyArrayUInt64(), toUInt64(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayFold( acc,x -> x, emptyArrayString(), toInt8(0)); -- { serverError TYPE_MISMATCH }
SELECT arrayFold( acc,x -> x, 'not an array', toUInt8(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }