Use C++20 format() syntax for exceptions

This commit is contained in:
Roman Vasin 2022-09-29 14:07:02 +00:00
parent 9d84d3de89
commit e0099d52de
7 changed files with 63 additions and 70 deletions

View File

@ -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());
}
};

View File

@ -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());
}
};

View File

@ -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());
}
};

View File

@ -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());
}
};

View File

@ -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());
}
};

View File

@ -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());
}
};

View File

@ -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());
}
};