Update ErrorCodes for functions using NUMBER_OF_ARGUMENTS_DOESNT_MATCH

This commit is contained in:
Yohann Jardin 2024-05-06 11:35:48 +02:00
parent 2c6d0c69ab
commit ab7d457c51
31 changed files with 73 additions and 68 deletions

View File

@ -16,7 +16,7 @@ struct Settings;
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int AGGREGATE_FUNCTION_THROW; extern const int AGGREGATE_FUNCTION_THROW;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
} }
namespace namespace
@ -116,7 +116,7 @@ void registerAggregateFunctionAggThrow(AggregateFunctionFactory & factory)
if (parameters.size() == 1) if (parameters.size() == 1)
throw_probability = parameters[0].safeGet<Float64>(); throw_probability = parameters[0].safeGet<Float64>();
else if (parameters.size() > 1) 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); return std::make_shared<AggregateFunctionThrow>(argument_types, parameters, throw_probability);
}); });

View File

@ -27,6 +27,7 @@ struct Settings;
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; 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 TOO_LARGE_ARRAY_SIZE;
extern const int CANNOT_CONVERT_TYPE; extern const int CANNOT_CONVERT_TYPE;
extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_TYPE_OF_ARGUMENT;
@ -74,7 +75,7 @@ public:
if (!params.empty()) if (!params.empty())
{ {
if (params.size() > 2) 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]; default_value = params[0];

View File

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

View File

@ -21,7 +21,7 @@ namespace ErrorCodes
{ {
extern const int NOT_IMPLEMENTED; extern const int NOT_IMPLEMENTED;
extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
} }
@ -141,7 +141,7 @@ public:
: IAggregateFunctionDataHelper<MannWhitneyData, AggregateFunctionMannWhitney> ({arguments}, {}, createResultType()) : IAggregateFunctionDataHelper<MannWhitneyData, AggregateFunctionMannWhitney> ({arguments}, {}, createResultType())
{ {
if (params.size() > 2) 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()) if (params.empty())
{ {

View File

@ -14,7 +14,7 @@ struct Settings;
namespace ErrorCodes 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_TYPE_OF_ARGUMENT;
extern const int NOT_IMPLEMENTED; extern const int NOT_IMPLEMENTED;
} }
@ -118,7 +118,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{ {
if (argument_types.empty()) 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]; const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type); WhichDataType which(argument_type);

View File

@ -12,7 +12,7 @@ struct Settings;
namespace ErrorCodes 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_TYPE_OF_ARGUMENT;
} }
@ -27,7 +27,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{ {
if (argument_types.empty()) 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]; const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type); WhichDataType which(argument_type);

View File

@ -13,7 +13,7 @@ struct Settings;
namespace ErrorCodes 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_TYPE_OF_ARGUMENT;
} }
@ -29,7 +29,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{ {
if (argument_types.empty()) 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]; const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type); WhichDataType which(argument_type);

View File

@ -13,7 +13,7 @@ struct Settings;
namespace ErrorCodes 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_TYPE_OF_ARGUMENT;
} }
@ -29,7 +29,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{ {
if (argument_types.empty()) 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]; const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type); WhichDataType which(argument_type);

View File

@ -13,7 +13,7 @@ struct Settings;
namespace ErrorCodes 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_TYPE_OF_ARGUMENT;
} }
@ -28,7 +28,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{ {
if (argument_types.empty()) 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]; const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type); WhichDataType which(argument_type);

View File

@ -13,7 +13,7 @@ struct Settings;
namespace ErrorCodes 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_TYPE_OF_ARGUMENT;
} }
@ -28,7 +28,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{ {
if (argument_types.empty()) 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]; const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type); WhichDataType which(argument_type);

View File

@ -35,7 +35,7 @@ namespace ErrorCodes
extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR; 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.empty())
{ {
if (params.size() > 3) 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); "Aggregate function '{}' requires three parameters or less", name);
threshold = applyVisitor(FieldVisitorConvertToNumber<UInt64>(), params[0]); threshold = applyVisitor(FieldVisitorConvertToNumber<UInt64>(), params[0]);

View File

@ -7,7 +7,7 @@
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
} }
namespace DB namespace DB
@ -80,7 +80,7 @@ AggregateFunctionPtr createAggregateFunctionWelchTTest(
assertBinary(name, argument_types); assertBinary(name, argument_types);
if (parameters.size() > 1) 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])) if (!isNumber(argument_types[0]) || !isNumber(argument_types[1]))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Aggregate function {} only supports numerical types", name); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Aggregate function {} only supports numerical types", name);

View File

@ -9,7 +9,7 @@ struct Settings;
namespace ErrorCodes 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_TYPE_OF_ARGUMENT;
} }
@ -26,7 +26,7 @@ public:
DataTypes transformArguments(const DataTypes & arguments) const override DataTypes transformArguments(const DataTypes & arguments) const override
{ {
if (arguments.empty()) 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; DataTypes nested_arguments;
for (const auto & type : arguments) for (const auto & type : arguments)

View File

@ -18,7 +18,7 @@ struct Settings;
namespace ErrorCodes 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_TYPE_OF_ARGUMENT;
} }
@ -42,7 +42,7 @@ public:
, nested_func(nested), num_arguments(types.size()) , nested_func(nested), num_arguments(types.size())
{ {
if (num_arguments == 0) 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(); only_null_condition = types.back()->onlyNull();

View File

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

View File

@ -10,7 +10,7 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
} }
template <typename Name, typename Impl> 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. // 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()) 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 {}", "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())); 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 namespace ErrorCodes
{ {
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
} }
template <typename Name, typename Impl> 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. // 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()) 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 {}", "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())); name, needles_arr.size(), std::to_string(std::numeric_limits<UInt8>::max()));

View File

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

View File

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

View File

@ -14,7 +14,7 @@ namespace ErrorCodes
{ {
extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT; 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 SIZES_OF_ARRAYS_DONT_MATCH;
extern const int TYPE_MISMATCH; extern const int TYPE_MISMATCH;
} }
@ -41,7 +41,7 @@ public:
void getLambdaArgumentTypes(DataTypes & arguments) const override void getLambdaArgumentTypes(DataTypes & arguments) const override
{ {
if (arguments.size() < 3) 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); DataTypes accumulator_and_array_types(arguments.size() - 1);
accumulator_and_array_types[0] = arguments.back(); accumulator_and_array_types[0] = arguments.back();
@ -64,7 +64,7 @@ public:
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{ {
if (arguments.size() < 3) 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()); const auto * lambda_function_type = checkAndGetDataType<DataTypeFunction>(arguments[0].type.get());
if (!lambda_function_type) if (!lambda_function_type)

View File

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

View File

@ -14,7 +14,7 @@ namespace ErrorCodes
{ {
extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int SIZES_OF_ARRAYS_DONT_MATCH; 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_COLUMN;
} }
@ -39,7 +39,7 @@ public:
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{ {
if (arguments.empty()) 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()); "Function {} needs at least one argument; passed {}." , getName(), arguments.size());
DataTypes arguments_types; DataTypes arguments_types;

View File

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

View File

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

View File

@ -25,7 +25,7 @@ namespace ErrorCodes
{ {
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
extern const int ILLEGAL_COLUMN; 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; extern const int ILLEGAL_TYPE_OF_ARGUMENT;
} }
@ -53,7 +53,7 @@ namespace
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{ {
if (arguments.empty()) 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) for (const auto & arg : arguments)
if (!isString(arg.type)) if (!isString(arg.type))

View File

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

View File

@ -18,7 +18,7 @@ namespace ErrorCodes
{ {
extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int CANNOT_CLOCK_GETTIME; extern const int CANNOT_CLOCK_GETTIME;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
} }
namespace namespace
@ -128,7 +128,7 @@ public:
if (arguments.size() > 2) 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()) if (!arguments.empty())
{ {

View File

@ -12,7 +12,7 @@ namespace DB
namespace ErrorCodes 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; extern const int ILLEGAL_TYPE_OF_ARGUMENT;
} }
@ -63,7 +63,7 @@ public:
{ {
if (arguments.size() > 1) 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)) if (arguments.size() == 1 && !isStringOrFixedString(arguments[0].type))
{ {

View File

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

View File

@ -37,7 +37,7 @@ namespace DB
{ {
namespace ErrorCodes 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 BAD_ARGUMENTS;
extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_COLUMN;
@ -87,7 +87,7 @@ public:
{ {
if (arguments.size() < 2) 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: /** 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 BAD_ARGUMENTS;
extern const int LOGICAL_ERROR; 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; 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; ASTs & args = args_func.at(0)->children;
if (args.empty()) 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>(); const auto & literal = args[0]->as<const ASTLiteral>();
String value; String value;