mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 17:41:59 +00:00
Use C++20 format() syntax for exceptions
This commit is contained in:
parent
9d84d3de89
commit
e0099d52de
@ -70,9 +70,9 @@ public:
|
||||
TransformDateTime64<Transform>{assert_cast<const DataTypeDateTime64 *>(from_type)->getScale()});
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0].type->getName() + " of argument of function " + this->getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of argument of function {}",
|
||||
arguments[0].type->getName(), this->getName());
|
||||
}
|
||||
|
||||
};
|
||||
|
@ -45,9 +45,9 @@ public:
|
||||
TransformDateTime64<Transform>{assert_cast<const DataTypeDateTime64 *>(from_type)->getScale()});
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0].type->getName() + " of argument of function " + this->getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of argument of function {}",
|
||||
arguments[0].type->getName(), this->getName());
|
||||
}
|
||||
|
||||
};
|
||||
|
@ -37,9 +37,9 @@ public:
|
||||
/// only validate the time_zone part if the number of arguments is 2.
|
||||
if ((which.isDateTime() || which.isDateTime64()) && arguments.size() == 2
|
||||
&& extractTimeZoneNameFromFunctionArguments(arguments, 1, 0).empty())
|
||||
throw Exception(
|
||||
"Function " + this->getName() + " supports a 2nd argument (optional) that must be non-empty and be a valid time zone",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Function {} supports a 2nd argument (optional) that must be a valid time zone",
|
||||
this->getName());
|
||||
|
||||
if ((which.isDate32() || which.isDateTime64()) && enable_extended_results_for_datetime_functions)
|
||||
return std::make_shared<DataTypeDate32>();
|
||||
@ -74,8 +74,9 @@ public:
|
||||
return DateTimeTransformImpl<DataTypeDateTime64, DataTypeDate, decltype(transformer)>::execute(arguments, result_type, input_rows_count, transformer);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal type " + arguments[0].type->getName() + " of argument of function " + this->getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of argument of function {}",
|
||||
arguments[0].type->getName(), this->getName());
|
||||
}
|
||||
|
||||
};
|
||||
|
@ -39,9 +39,9 @@ public:
|
||||
/// If the time zone is specified but empty, throw an exception.
|
||||
/// only validate the time_zone part if the number of arguments is 2.
|
||||
if (arguments.size() == 2 && time_zone.empty())
|
||||
throw Exception(
|
||||
"Function " + this->getName() + " supports a 2nd argument (optional) that must be non-empty and be a valid time zone",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Function {} supports a 2nd argument (optional) that must be a valid time zone",
|
||||
this->getName());
|
||||
|
||||
if ((which.isDate32() || which.isDateTime64()) && enable_extended_results_for_datetime_functions)
|
||||
{
|
||||
@ -83,8 +83,9 @@ public:
|
||||
return DateTimeTransformImpl<DataTypeDateTime64, DataTypeDateTime, decltype(transformer)>::execute(arguments, result_type, input_rows_count, transformer);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal type " + arguments[0].type->getName() + " of argument of function " + this->getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of argument of function {}",
|
||||
arguments[0].type->getName(), this->getName());
|
||||
}
|
||||
|
||||
};
|
||||
|
@ -28,9 +28,9 @@ public:
|
||||
/// only validate the time_zone part if the number of arguments is 2. This is mainly
|
||||
/// to accommodate functions like toStartOfDay(today()), toStartOfDay(yesterday()) etc.
|
||||
if (arguments.size() == 2 && time_zone.empty())
|
||||
throw Exception(
|
||||
"Function " + this->getName() + " supports a 2nd argument (optional) that must be non-empty and be a valid time zone",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Function {} supports a 2nd argument (optional) that must be a valid time zone",
|
||||
this->getName());
|
||||
return std::make_shared<ToDataType>(time_zone);
|
||||
}
|
||||
if constexpr (std::is_same_v<ToDataType, DataTypeDateTime64>)
|
||||
@ -78,8 +78,9 @@ public:
|
||||
return DateTimeTransformImpl<DataTypeDateTime64, ToDataType, decltype(transformer)>::execute(arguments, result_type, input_rows_count, transformer);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal type " + arguments[0].type->getName() + " of argument of function " + this->getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of argument of function {}",
|
||||
arguments[0].type->getName(), this->getName());
|
||||
}
|
||||
|
||||
};
|
||||
|
@ -70,51 +70,44 @@ protected:
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
|
||||
+ ". Must be Date, Date32, DateTime or DateTime64.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of argument of function {}. Must be Date, Date32, DateTime or DateTime64.",
|
||||
arguments[0].type->getName(), getName());
|
||||
}
|
||||
else if (arguments.size() == 2)
|
||||
{
|
||||
if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0].type->getName() + " of 1st argument of function " + getName()
|
||||
+ ". Must be Date, Date32, DateTime or DateTime64.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of 1st argument of function {}. Must be Date, Date32, DateTime or DateTime64.",
|
||||
arguments[0].type->getName(), getName());
|
||||
if (!isUInt8(arguments[1].type))
|
||||
throw Exception(
|
||||
"Illegal type of 2nd (optional) argument of function " + getName()
|
||||
+ ". Must be constant UInt8 (week mode).",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type of 2nd (optional) argument of function {}. Must be constant UInt8 (week mode).",
|
||||
getName());
|
||||
}
|
||||
else if (arguments.size() == 3)
|
||||
{
|
||||
if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
|
||||
+ ". Must be Date, Date32, DateTime or DateTime64",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of argument of function {}. Must be Date, Date32, DateTime or DateTime64",
|
||||
arguments[0].type->getName(), getName());
|
||||
if (!isUInt8(arguments[1].type))
|
||||
throw Exception(
|
||||
"Illegal type of 2nd (optional) argument of function " + getName()
|
||||
+ ". Must be constant UInt8 (week mode).",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type of 2nd (optional) argument of function {}. Must be constant UInt8 (week mode).",
|
||||
getName());
|
||||
if (!isString(arguments[2].type))
|
||||
throw Exception(
|
||||
"Illegal type of 3rd (optional) argument of function " + getName()
|
||||
+ ". Must be constant string (timezone name).",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type of 3rd (optional) argument of function {}. Must be constant string (timezone name).",
|
||||
getName());
|
||||
if ((isDate(arguments[0].type) || isDate32(arguments[0].type)) && is_result_type_date_or_date32)
|
||||
throw Exception(
|
||||
"The timezone argument of function " + getName() + " is allowed only when the 1st argument is DateTime or DateTime64.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"The timezone argument of function {} is allowed only when the 1st argument is DateTime or DateTime64.",
|
||||
getName());
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size())
|
||||
+ ", expected 1, 2 or 3.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Number of arguments for function {} doesn't match: passed {}, expected 1, 2 or 3.",
|
||||
getName(), arguments.size());
|
||||
}
|
||||
|
||||
};
|
||||
|
@ -85,33 +85,30 @@ protected:
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
if (!isDateOrDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
|
||||
+ ". Should be Date, Date32, DateTime or DateTime64",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of argument of function {}. Should be Date, Date32, DateTime or DateTime64",
|
||||
arguments[0].type->getName(), getName());
|
||||
}
|
||||
else if (arguments.size() == 2)
|
||||
{
|
||||
if (!isDateOrDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
|
||||
+ ". Should be Date, Date32, DateTime or DateTime64",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of argument of function {}. Should be Date, Date32, DateTime or DateTime64",
|
||||
arguments[0].type->getName(), getName());
|
||||
if (!isString(arguments[1].type))
|
||||
throw Exception(
|
||||
"Function " + getName() + " supports 1 or 2 arguments. The optional 2nd argument must be "
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Function {} supports 1 or 2 arguments. The optional 2nd argument must be "
|
||||
"a constant string with a timezone name",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
getName());
|
||||
if (isDateOrDate32(arguments[0].type) && is_result_type_date_or_date32)
|
||||
throw Exception(
|
||||
"The timezone argument of function " + getName() + " is allowed only when the 1st argument has the type DateTime or DateTime64",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"The timezone argument of function {} is allowed only when the 1st argument has the type DateTime or DateTime64",
|
||||
getName());
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size())
|
||||
+ ", should be 1 or 2",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Number of arguments for function {} doesn't match: passed {}, should be 1 or 2",
|
||||
getName(), arguments.size());
|
||||
}
|
||||
};
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user