mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Merge pull request #63406 from yohannj/fix_error_codes
Update ErrorCodes for functions using NUMBER_OF_ARGUMENTS_DOESNT_MATCH
This commit is contained in:
commit
767e32d566
@ -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);
|
||||
});
|
||||
|
@ -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];
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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())
|
||||
{
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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]);
|
||||
|
@ -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);
|
||||
|
@ -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)
|
||||
|
@ -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();
|
||||
|
||||
|
@ -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());
|
||||
|
@ -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()));
|
||||
|
||||
|
@ -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()));
|
||||
|
||||
|
@ -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());
|
||||
|
||||
|
@ -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();
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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());
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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());
|
||||
|
@ -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());
|
||||
|
||||
|
@ -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))
|
||||
|
@ -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(),
|
||||
|
@ -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())
|
||||
{
|
||||
|
@ -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))
|
||||
{
|
||||
|
@ -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());
|
||||
|
@ -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:
|
||||
|
@ -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;
|
||||
|
@ -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 }
|
||||
|
@ -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 }
|
||||
|
@ -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 }
|
||||
|
||||
|
@ -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}
|
||||
|
@ -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 }
|
||||
|
@ -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 }
|
||||
|
Loading…
Reference in New Issue
Block a user