Merge pull request #48471 from jrdi/16414-totimezone-throw

toTimeZone function throw an error when no constant string is provided
This commit is contained in:
Alexander Tokmakov 2023-04-13 17:57:37 +03:00 committed by GitHub
commit 0fbc9585f1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 26 additions and 12 deletions

View File

@ -68,8 +68,7 @@ public:
String timezone;
if (arguments.size() == 2)
{
if (arguments[1].column)
timezone = extractTimeZoneNameFromColumn(*arguments[1].column);
timezone = extractTimeZoneNameFromColumn(arguments[1].column.get(), arguments[1].name);
if (timezone.empty())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,

View File

@ -245,7 +245,8 @@ private:
{
if (additional_argument_index < arguments.size())
{
time_zone = extractTimeZoneNameFromColumn(*arguments[additional_argument_index].column);
time_zone = extractTimeZoneNameFromColumn(arguments[additional_argument_index].column.get(),
arguments[additional_argument_index].name);
++additional_argument_index;
}
}

View File

@ -17,14 +17,14 @@ namespace ErrorCodes
}
std::string extractTimeZoneNameFromColumn(const IColumn & column)
std::string extractTimeZoneNameFromColumn(const IColumn * column, const String & column_name)
{
const ColumnConst * time_zone_column = checkAndGetColumnConst<ColumnString>(&column);
const ColumnConst * time_zone_column = checkAndGetColumnConst<ColumnString>(column);
if (!time_zone_column)
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
"Illegal column {} of time zone argument of function, must be constant string",
column.getName());
"Illegal column {} of time zone argument of function, must be a constant string",
column_name);
return time_zone_column->getValue<String>();
}
@ -33,9 +33,9 @@ std::string extractTimeZoneNameFromColumn(const IColumn & column)
std::string extractTimeZoneNameFromFunctionArguments(const ColumnsWithTypeAndName & arguments, size_t time_zone_arg_num, size_t datetime_arg_num)
{
/// Explicit time zone may be passed in last argument.
if (arguments.size() == time_zone_arg_num + 1 && arguments[time_zone_arg_num].column)
if (arguments.size() == time_zone_arg_num + 1)
{
return extractTimeZoneNameFromColumn(*arguments[time_zone_arg_num].column);
return extractTimeZoneNameFromColumn(arguments[time_zone_arg_num].column.get(), arguments[time_zone_arg_num].name);
}
else
{
@ -57,7 +57,7 @@ const DateLUTImpl & extractTimeZoneFromFunctionArguments(const ColumnsWithTypeAn
{
if (arguments.size() == time_zone_arg_num + 1)
{
std::string time_zone = extractTimeZoneNameFromColumn(*arguments[time_zone_arg_num].column);
std::string time_zone = extractTimeZoneNameFromColumn(arguments[time_zone_arg_num].column.get(), arguments[time_zone_arg_num].name);
if (time_zone.empty())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Provided time zone must be non-empty and be a valid time zone");
return DateLUT::instance(time_zone);

View File

@ -12,7 +12,7 @@ namespace DB
class Block;
std::string extractTimeZoneNameFromColumn(const IColumn & column);
std::string extractTimeZoneNameFromColumn(const IColumn * column, const String & column_name);
/// Determine working timezone either from optional argument with time zone name or from time zone in DateTime type of argument.
/// Returns empty string if default time zone should be used.

View File

@ -8,7 +8,6 @@
#include <IO/WriteHelpers.h>
#include <Common/assert_cast.h>
namespace DB
{
namespace ErrorCodes
@ -100,6 +99,7 @@ public:
"Should be DateTime or DateTime64", arguments[0].type->getName(), getName());
String time_zone_name = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0);
if (which_type.isDateTime())
return std::make_shared<DataTypeDateTime>(time_zone_name);

View File

@ -20,3 +20,17 @@ SELECT toTimeZone(toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul'), 'Asia/Kolk
SELECT toString(toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul'));
SELECT toString(toTimeZone(toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul'), 'Asia/Kolkata'));
SELECT toString(toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul'), 'Asia/Kolkata');
SELECT toTimeZone(dt, tz) FROM (
SELECT toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul') AS dt, arrayJoin(['Asia/Kolkata', 'UTC']) AS tz
); -- { serverError ILLEGAL_COLUMN }
SELECT materialize('Asia/Kolkata') t, toTimeZone(toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul'), t); -- { serverError ILLEGAL_COLUMN }
CREATE TEMPORARY TABLE tmp AS SELECT arrayJoin(['Europe/Istanbul', 'Asia/Istanbul']);
SELECT toTimeZone(now(), (*,).1) FROM tmp; -- { serverError ILLEGAL_COLUMN }
SELECT now((*,).1) FROM tmp; -- { serverError ILLEGAL_COLUMN }
SELECT now64(1, (*,).1) FROM tmp; -- { serverError ILLEGAL_COLUMN }
SELECT toStartOfInterval(now(), INTERVAL 3 HOUR, (*,).1) FROM tmp; -- { serverError ILLEGAL_COLUMN }
SELECT snowflakeToDateTime(toInt64(123), (*,).1) FROM tmp; -- { serverError ILLEGAL_COLUMN }
SELECT toUnixTimestamp(now(), (*,).1) FROM tmp; -- { serverError ILLEGAL_COLUMN }
SELECT toDateTimeOrDefault('2023-04-12 16:43:32', (*,).1, now()) FROM tmp; -- { serverError ILLEGAL_COLUMN }