mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
add parseDateTime64 functions
This commit is contained in:
parent
699b9d4026
commit
1c74206bf2
@ -193,6 +193,7 @@ namespace
|
||||
Int32 minute = 0; /// range [0, 59]
|
||||
Int32 second = 0; /// range [0, 59]
|
||||
Int32 microsecond = 0; /// range [0, 999999]
|
||||
UInt32 scale = 0; /// The microsecond scale of DateTime64.
|
||||
|
||||
bool is_am = true; /// If is_hour_of_half_day = true and is_am = false (i.e. pm) then add 12 hours to the result DateTime
|
||||
bool hour_starts_at_1 = false; /// Whether the hour is clockhour
|
||||
@ -221,6 +222,7 @@ namespace
|
||||
minute = 0;
|
||||
second = 0;
|
||||
microsecond = 0;
|
||||
scale = 0;
|
||||
|
||||
is_am = true;
|
||||
hour_starts_at_1 = false;
|
||||
@ -599,7 +601,7 @@ namespace
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
||||
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2, 3}; }
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
@ -613,9 +615,9 @@ namespace
|
||||
if constexpr (return_type == ReturnType::DateTime64)
|
||||
{
|
||||
optional_args = {
|
||||
{"precision or format", static_cast<FunctionArgumentDescriptor::TypeValidator>([](const IDataType & data_type) -> bool {
|
||||
{"scale/format", static_cast<FunctionArgumentDescriptor::TypeValidator>([](const IDataType & data_type) -> bool {
|
||||
return isUInt(data_type) || isString(data_type);
|
||||
}), nullptr, "Number or String"},
|
||||
}), nullptr, "UInt or String"},
|
||||
{"format", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isString), nullptr, "String"},
|
||||
{"timezone", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isString), &isColumnConst, "const String"}
|
||||
};
|
||||
@ -631,24 +633,34 @@ namespace
|
||||
DataTypePtr data_type;
|
||||
if constexpr (return_type == ReturnType::DateTime64)
|
||||
{
|
||||
UInt32 scale = 0;
|
||||
if (arguments.size() == 1)
|
||||
return std::make_shared<DataTypeDateTime64>(0, time_zone_name);
|
||||
{
|
||||
/// In MySQL parse syntax, the scale of microseond is 6.
|
||||
if constexpr (parse_syntax == ParseSyntax::MySQL)
|
||||
scale = 6;
|
||||
}
|
||||
else
|
||||
{
|
||||
UInt32 precision = 0;
|
||||
if (isUInt(arguments[1].type))
|
||||
{
|
||||
const auto * col_precision = checkAndGetColumnConst<ColumnUInt8>(arguments[1].column.get());
|
||||
if (col_precision)
|
||||
precision = col_precision->getValue<UInt8>();
|
||||
const auto * col_scale = checkAndGetColumnConst<ColumnUInt8>(arguments[1].column.get());
|
||||
if (col_scale)
|
||||
scale = col_scale->getValue<UInt8>();
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"The input precision value may exceed the max value of `DateTime64`: {}.",
|
||||
"The input scale value may exceed the max value of `DateTime64`: {}.",
|
||||
maxPrecisionOfDateTime64);
|
||||
}
|
||||
/// Construct the return type `DataTypDateTime64` with precision and time zone name. The precision value can be specified or be extracted
|
||||
/// from the format string by computing how many 'S' characters are contained in the format's micorsceond fragment.
|
||||
String format = getFormat(arguments, precision);
|
||||
else
|
||||
{
|
||||
if constexpr (parse_syntax == ParseSyntax::MySQL)
|
||||
scale = 6;
|
||||
}
|
||||
|
||||
/// Construct the return type `DataTypDateTime64` with scale and time zone name. The scale value can be specified or be extracted
|
||||
/// from the format string by c how many 'S' characters are contained in the format's micorsceond fragment.
|
||||
String format = getFormat(arguments, scale);
|
||||
std::vector<Instruction> instructions = parseFormat(format);
|
||||
for (const auto & instruction : instructions)
|
||||
{
|
||||
@ -664,26 +676,27 @@ namespace
|
||||
else
|
||||
val++;
|
||||
}
|
||||
/// If the precision is already specified by the second parameter, but it not equals the value that extract from the format string,
|
||||
/// then we should throw an exception; If the precision is not specified, then we set its value as the extracted one.
|
||||
if (val != 0 && precision != 0 && val != precision)
|
||||
/// If the scale is already specified by the second argument, but it not equals the value that extract from the format string,
|
||||
/// then we should throw an exception; If the scale is not specified, then we should set its value as the extracted one.
|
||||
if (val != 0 && scale != 0 && val != scale)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"The precision of input format string {} not equals the given precision value {}.",
|
||||
"The scale of input format string {} not equals the given scale value {}.",
|
||||
format,
|
||||
precision);
|
||||
else if (precision == 0 && val != 0)
|
||||
precision = val;
|
||||
scale);
|
||||
else if (scale == 0 && val != 0)
|
||||
scale = val;
|
||||
}
|
||||
if (precision > maxPrecisionOfDateTime64)
|
||||
if (scale > maxPrecisionOfDateTime64)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"The precision of the input format string {} exceed the max precision value {}.",
|
||||
"The scale of the input format string {} exceed the max scale value {}.",
|
||||
format,
|
||||
maxPrecisionOfDateTime64);
|
||||
data_type = std::make_shared<DataTypeDateTime64>(precision, time_zone_name);
|
||||
}
|
||||
data_type = std::make_shared<DataTypeDateTime64>(scale, time_zone_name);
|
||||
}
|
||||
else
|
||||
data_type = std::make_shared<DataTypeDateTime>(time_zone_name);
|
||||
|
||||
if (error_handling == ErrorHandling::Null)
|
||||
return std::make_shared<DataTypeNullable>(data_type);
|
||||
return data_type;
|
||||
@ -729,15 +742,15 @@ namespace
|
||||
col_null_map = ColumnUInt8::create(input_rows_count, 0);
|
||||
|
||||
Int64 multiplier = 0;
|
||||
UInt32 precision = 0;
|
||||
UInt32 scale = 0;
|
||||
if constexpr (return_type == ReturnType::DateTime64)
|
||||
{
|
||||
const DataTypeDateTime64 * datatime64_type = checkAndGetDataType<DataTypeDateTime64>(removeNullable(result_type).get());
|
||||
precision = datatime64_type->getScale();
|
||||
multiplier = DecimalUtils::scaleMultiplier<DateTime64>(precision);
|
||||
scale = datatime64_type->getScale();
|
||||
multiplier = DecimalUtils::scaleMultiplier<DateTime64>(scale);
|
||||
}
|
||||
|
||||
String format = getFormat(arguments, precision);
|
||||
const String format = getFormat(arguments, scale);
|
||||
const auto & time_zone = getTimeZone(arguments);
|
||||
std::vector<Instruction> instructions = parseFormat(format);
|
||||
|
||||
@ -746,6 +759,9 @@ namespace
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
{
|
||||
datetime.reset();
|
||||
if constexpr (return_type == ReturnType::DateTime64)
|
||||
datetime.scale = scale;
|
||||
|
||||
StringRef str_ref = col_str->getDataAt(i);
|
||||
Pos cur = str_ref.data;
|
||||
Pos end = str_ref.data + str_ref.size;
|
||||
@ -787,7 +803,7 @@ namespace
|
||||
Int64OrError result = 0;
|
||||
|
||||
/// Ensure all input was consumed when the return type is `DateTime`.
|
||||
if (return_type == ReturnType::DateTime && cur < end)
|
||||
if (cur < end)
|
||||
{
|
||||
result = tl::unexpected(ErrorCodeAndMessage(
|
||||
ErrorCodes::CANNOT_PARSE_DATETIME,
|
||||
@ -938,6 +954,28 @@ namespace
|
||||
return cur;
|
||||
}
|
||||
|
||||
template<typename T, NeedCheckSpace need_check_space>
|
||||
[[nodiscard]]
|
||||
static PosOrError readNumber6(Pos cur, Pos end, [[maybe_unused]] const String & fragment, T & res)
|
||||
{
|
||||
if constexpr (need_check_space == NeedCheckSpace::Yes)
|
||||
RETURN_ERROR_IF_FAILED(checkSpace(cur, end, 6, "readNumber6 requires size >= 6", fragment))
|
||||
|
||||
res = (*cur - '0');
|
||||
++cur;
|
||||
res = res * 10 + (*cur - '0');
|
||||
++cur;
|
||||
res = res * 10 + (*cur - '0');
|
||||
++cur;
|
||||
res = res * 10 + (*cur - '0');
|
||||
++cur;
|
||||
res = res * 10 + (*cur - '0');
|
||||
++cur;
|
||||
res = res * 10 + (*cur - '0');
|
||||
++cur;
|
||||
return cur;
|
||||
}
|
||||
|
||||
[[nodiscard]]
|
||||
static VoidOrError checkSpace(Pos cur, Pos end, size_t len, const String & msg, const String & fragment)
|
||||
{
|
||||
@ -1358,13 +1396,18 @@ namespace
|
||||
}
|
||||
|
||||
[[nodiscard]]
|
||||
static PosOrError mysqlMicrosecond(Pos cur, Pos end, const String & fragment, DateTime<error_handling> & /*date*/)
|
||||
static PosOrError mysqlMicrosecond(Pos cur, Pos end, const String & fragment, DateTime<error_handling> & date)
|
||||
{
|
||||
RETURN_ERROR_IF_FAILED(checkSpace(cur, end, 6, "mysqlMicrosecond requires size >= 6", fragment))
|
||||
|
||||
for (size_t i = 0; i < 6; ++i)
|
||||
ASSIGN_RESULT_OR_RETURN_ERROR(cur, (assertNumber<NeedCheckSpace::No>(cur, end, fragment)))
|
||||
|
||||
if (date.scale != 6)
|
||||
RETURN_ERROR(
|
||||
ErrorCodes::CANNOT_PARSE_DATETIME,
|
||||
"Unable to parse fragment {} from {} because of the microsecond's scale {} is not 6",
|
||||
fragment,
|
||||
std::string_view(cur, end - cur),
|
||||
std::to_string(date.scale))
|
||||
Int32 microsecond = 0;
|
||||
ASSIGN_RESULT_OR_RETURN_ERROR(cur, (readNumber6<Int32, NeedCheckSpace::Yes>(cur, end, fragment, microsecond)))
|
||||
RETURN_ERROR_IF_FAILED(date.setMicrosecond(microsecond))
|
||||
return cur;
|
||||
}
|
||||
|
||||
@ -1775,7 +1818,7 @@ namespace
|
||||
return cur;
|
||||
}
|
||||
else
|
||||
RETURN_ERROR(ErrorCodes::CANNOT_PARSE_DATETIME, "Unable to build date time from timezone {}", read_time_zone)
|
||||
RETURN_ERROR(ErrorCodes::CANNOT_PARSE_DATETIME, "Unable to parse date time from timezone {}", read_time_zone)
|
||||
}
|
||||
|
||||
[[nodiscard]]
|
||||
@ -2223,7 +2266,7 @@ namespace
|
||||
}
|
||||
|
||||
|
||||
String getFormat(const ColumnsWithTypeAndName & arguments, UInt32 precision) const
|
||||
String getFormat(const ColumnsWithTypeAndName & arguments, UInt32 scale) const
|
||||
{
|
||||
size_t format_arg_index = 1;
|
||||
if constexpr (return_type == ReturnType::DateTime64)
|
||||
@ -2247,15 +2290,17 @@ namespace
|
||||
format = "%Y-%m-%d %H:%i:%s";
|
||||
else
|
||||
format = "yyyy-MM-dd HH:mm:ss";
|
||||
if (precision > 0)
|
||||
format += "." + String(precision, 'S');
|
||||
if (scale > 0)
|
||||
{
|
||||
if constexpr (parse_syntax == ParseSyntax::MySQL)
|
||||
format += ".%f";
|
||||
else
|
||||
format += "." + String(scale, 'S');
|
||||
}
|
||||
return format;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!arguments[format_arg_index].column || !isColumnConst(*arguments[format_arg_index].column))
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Argument at index {} for function {} must be constant", format_arg_index, getName());
|
||||
|
||||
const auto * col_format = checkAndGetColumnConst<ColumnString>(arguments[format_arg_index].column.get());
|
||||
if (!col_format)
|
||||
throw Exception(
|
||||
@ -2269,18 +2314,24 @@ namespace
|
||||
|
||||
const DateLUTImpl & getTimeZone(const ColumnsWithTypeAndName & arguments) const
|
||||
{
|
||||
size_t timezone_arg_index = 2;
|
||||
if constexpr (return_type == ReturnType::DateTime64)
|
||||
timezone_arg_index = 3;
|
||||
|
||||
if (arguments.size() <= timezone_arg_index)
|
||||
if (arguments.size() < 3)
|
||||
return DateLUT::instance();
|
||||
|
||||
else if constexpr (return_type == ReturnType::DateTime64)
|
||||
{
|
||||
/// If the return type is DateTime64, and the second argument is UInt type for scale, then it has 2 reasonable situations:
|
||||
/// the first like parseDateTime64[InJodaSyntax][OrZero/OrNull]('2024-11-07 17:27.30.123456', 6, '%Y-%m-%d %H:%i:%s.%f', 'Etc/GMT+8')
|
||||
/// the second like parseDateTime64[InJodaSyntax][OrZero/OrNull]('2024-11-07 17:27.30.123456', 6, '%Y-%m-%d %H:%i:%s.%f'). And for the
|
||||
/// first one, we should return the last argument as its timezone, and for the second one, we should return the default time zone as
|
||||
/// `DateLUT::instance()`.
|
||||
if (isUInt(arguments[1].type) && arguments.size() < 4)
|
||||
return DateLUT::instance();
|
||||
}
|
||||
size_t timezone_arg_index = arguments.size() - 1;
|
||||
const auto * col = checkAndGetColumnConst<ColumnString>(arguments[timezone_arg_index].column.get());
|
||||
if (!col)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal column {} of third ('timezone') argument of function {}. Must be constant String.",
|
||||
"Illegal column {} of ('timezone') argument of function {}. Must be constant String.",
|
||||
arguments[timezone_arg_index].column->getName(),
|
||||
getName());
|
||||
|
||||
|
17
tests/queries/0_stateless/03252_parse_datetime64.reference
Normal file
17
tests/queries/0_stateless/03252_parse_datetime64.reference
Normal file
@ -0,0 +1,17 @@
|
||||
2024-10-09 10:30:10.123456
|
||||
2024-10-09 10:30:10.123456 2024-10-09 10:30:10.123456
|
||||
2024-10-09 10:30:10.123456 2024-10-09 10:30:10.123456
|
||||
2024-10-09 10:30:10.123456
|
||||
1970-01-01 08:00:00.000000
|
||||
1970-01-01 08:00:00.000
|
||||
1970-01-01 08:00:00.000
|
||||
2024-10-09 10:30:10.123456 2024-10-09 10:30:10.123456
|
||||
2024-10-09 10:30:10.123456 2024-10-09 10:30:10.123456
|
||||
1970-01-01 08:00:00.000
|
||||
2024-10-09 10:30:10.123456
|
||||
\N
|
||||
\N
|
||||
\N
|
||||
2024-10-09 10:30:10.123456 2024-10-09 10:30:10.123456
|
||||
2024-10-09 10:30:10.123456 2024-10-09 10:30:10.123456
|
||||
\N
|
32
tests/queries/0_stateless/03252_parse_datetime64.sql
Normal file
32
tests/queries/0_stateless/03252_parse_datetime64.sql
Normal file
@ -0,0 +1,32 @@
|
||||
set session_timezone = 'Asia/Shanghai';
|
||||
|
||||
select parseDateTime64('2024-10-09 10:30:10.123456');
|
||||
select parseDateTime64('2024-10-09 10:30:10.123'); -- { serverError NOT_ENOUGH_SPACE }
|
||||
select parseDateTime64('2024-10-09 10:30:10', 3); -- { serverError NOT_ENOUGH_SPACE }
|
||||
select parseDateTime64('2024-10-09 10:30:10.', 3); -- { serverError CANNOT_PARSE_DATETIME }
|
||||
select parseDateTime64('2024-10-09 10:30:10', -3); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT}
|
||||
select parseDateTime64('2024-10-09 10:30:10', 9); -- { serverError BAD_ARGUMENTS }
|
||||
select parseDateTime64('2024-10-09 10:30:10.123456', 6), parseDateTime64('2024-10-09 10:30:10.123456', '%Y-%m-%d %H:%i:%s.%f');
|
||||
select parseDateTime64('2024-10-09 10:30:10.123456', 6, '%Y-%m-%d %H:%i:%s.%f'), parseDateTime64('2024-10-09 10:30:10.123456', 6, '%Y-%m-%d %H:%i:%s.%f', 'Etc/GMT-7');
|
||||
select parseDateTime64('2024-10-09 10:30:10.123', 3, '%Y-%m-%d %H:%i:%s.%f'); -- { serverError CANNOT_PARSE_DATETIME }
|
||||
select parseDateTime64('2024-10-09 10:30:10.123', 6, '%Y-%m-%d %H:%i:%s.%f'); -- { serverError NOT_ENOUGH_SPACE }
|
||||
|
||||
select parseDateTime64OrZero('2024-10-09 10:30:10.123456');
|
||||
select parseDateTime64OrZero('2024-10-09 10:30:10.123');
|
||||
select parseDateTime64OrZero('2024-10-09 10:30:10', 3);
|
||||
select parseDateTime64OrZero('2024-10-09 10:30:10.', 3);
|
||||
select parseDateTime64OrZero('2024-10-09 10:30:10', -3); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT}
|
||||
select parseDateTime64OrZero('2024-10-09 10:30:10', 9); -- { serverError BAD_ARGUMENTS }
|
||||
select parseDateTime64OrZero('2024-10-09 10:30:10.123456', 6), parseDateTime64OrZero('2024-10-09 10:30:10.123456', '%Y-%m-%d %H:%i:%s.%f');
|
||||
select parseDateTime64OrZero('2024-10-09 10:30:10.123456', 6, '%Y-%m-%d %H:%i:%s.%f'), parseDateTime64OrZero('2024-10-09 10:30:10.123456', 6, '%Y-%m-%d %H:%i:%s.%f', 'Etc/GMT-7');
|
||||
select parseDateTime64OrZero('2024-10-09 10:30:10.123', 3, '%Y-%m-%d %H:%i:%s.%f');
|
||||
|
||||
select parseDateTime64OrNull('2024-10-09 10:30:10.123456');
|
||||
select parseDateTime64OrNull('2024-10-09 10:30:10.123');
|
||||
select parseDateTime64OrNull('2024-10-09 10:30:10', 3);
|
||||
select parseDateTime64OrNull('2024-10-09 10:30:10.', 3);
|
||||
select parseDateTime64OrNull('2024-10-09 10:30:10', -3); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT}
|
||||
select parseDateTime64OrNull('2024-10-09 10:30:10', 9); -- { serverError BAD_ARGUMENTS }
|
||||
select parseDateTime64OrNull('2024-10-09 10:30:10.123456', 6), parseDateTime64OrZero('2024-10-09 10:30:10.123456', '%Y-%m-%d %H:%i:%s.%f');
|
||||
select parseDateTime64OrNull('2024-10-09 10:30:10.123456', 6, '%Y-%m-%d %H:%i:%s.%f'), parseDateTime64OrNull('2024-10-09 10:30:10.123456', 6, '%Y-%m-%d %H:%i:%s.%f', 'Etc/GMT-7');;
|
||||
select parseDateTime64OrNull('2024-10-09 10:30:10.123', 3, '%Y-%m-%d %H:%i:%s.%f');
|
@ -1,14 +1,26 @@
|
||||
2024-10-09 10:30:10.123
|
||||
2024-10-09 10:30:10.123456
|
||||
2024-10-10 02:30:10.123456
|
||||
2024-10-09 10:30:10.123 2024-10-09 10:30:10.000123
|
||||
2024-10-09 10:30:10.123 2024-10-09 10:30:10.000123
|
||||
2024-10-09 10:30:10.123 2024-10-09 10:30:10.000123
|
||||
2024-10-10 02:42:10.123456
|
||||
2024-10-10 01:30:10.123456
|
||||
2024-10-09 10:30:10.123
|
||||
2024-10-09 10:30:10.123456
|
||||
1970-01-01 08:00:00.000000000
|
||||
2024-10-10 02:30:10.123456
|
||||
2024-10-10 01:30:10.123456
|
||||
2024-10-09 10:30:10.123
|
||||
2024-10-09 10:30:10.123456
|
||||
1970-01-01 08:00:00.000
|
||||
1970-01-01 08:00:00.000
|
||||
2024-10-09 10:30:10.123 2024-10-09 10:30:10.000123
|
||||
2024-10-09 10:30:10.123 2024-10-09 10:30:10.000123
|
||||
2024-10-09 10:30:10.123 2024-10-09 10:30:10.000123
|
||||
2024-10-10 02:42:10.123456
|
||||
1970-01-01 08:00:00.000000
|
||||
2024-10-10 01:30:10.123456
|
||||
2024-10-10 01:30:10.123456
|
||||
1970-01-01 08:00:00.000000
|
||||
\N
|
||||
\N
|
||||
2024-10-09 10:30:10.123 2024-10-09 10:30:10.000123
|
||||
2024-10-09 10:30:10.123 2024-10-09 10:30:10.000123
|
||||
2024-10-09 10:30:10.123 2024-10-09 10:30:10.000123
|
||||
2024-10-10 02:42:10.123456
|
||||
\N
|
||||
2024-10-10 02:30:10.123456
|
||||
2024-10-10 01:30:10.123456
|
||||
2024-10-10 01:30:10.123456
|
||||
\N
|
||||
|
@ -1,19 +1,49 @@
|
||||
set session_timezone = 'Asia/Shanghai';
|
||||
|
||||
select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123', 'yyyy-MM-dd HH:mm:ss.SSS');
|
||||
select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS');
|
||||
select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123456789', 'yyyy-MM-dd HH:mm:ss.SSSSSSSSS'); -- { serverError CANNOT_PARSE_DATETIME }
|
||||
select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123456-0800', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ');
|
||||
select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123456America/Los_Angeles', 'yyyy-MM-dd HH:mm:ss.SSSSSSz');
|
||||
select parseDateTime64InJodaSyntax('2024-10-09 10:30:10', 3); -- { serverError NOT_ENOUGH_SPACE }
|
||||
select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.', 3); -- { serverError CANNOT_PARSE_DATETIME }
|
||||
select parseDateTime64InJodaSyntax('2024-10-09 10:30:10', -3); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT}
|
||||
select parseDateTime64InJodaSyntax('2024-10-09 10:30:10', 9); -- { serverError BAD_ARGUMENTS }
|
||||
select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123', 3), parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123', 6);
|
||||
select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123', 'yyyy-MM-dd HH:mm:ss.SSS'), parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123', 'yyyy-MM-dd HH:mm:ss.SSSSSS');
|
||||
select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123', 3, 'yyyy-MM-dd HH:mm:ss.SSS'), parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123', 6, 'yyyy-MM-dd HH:mm:ss.SSSSSS');
|
||||
select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123', 3, 'yyyy-MM-dd HH:mm:ss.SSSS'); -- { serverError BAD_ARGUMENTS }
|
||||
select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123456-0812', 6, 'yyyy-MM-dd HH:mm:ss.SSSSSSZ');
|
||||
select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123456-08123', 6, 'yyyy-MM-dd HH:mm:ss.SSSSSSZZZ'); -- {serverError CANNOT_PARSE_DATETIME}
|
||||
select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123456America/Los_Angeles', 6, 'yyyy-MM-dd HH:mm:ss.SSSSSSz');
|
||||
select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123456America/Los_Angeles', 6, 'yyyy-MM-dd HH:mm:ss.SSSSSSzzz');
|
||||
-- incorrect timezone offset and timezone
|
||||
select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123456-8000', 6, 'yyyy-MM-dd HH:mm:ss.SSSSSSZ'); -- { serverError CANNOT_PARSE_DATETIME }
|
||||
select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123456ABCD', 6, 'yyyy-MM-dd HH:mm:ss.SSSSSSz'); -- { serverError BAD_ARGUMENTS }
|
||||
|
||||
select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123', 'yyyy-MM-dd HH:mm:ss.SSS');
|
||||
select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS');
|
||||
select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123456789', 'yyyy-MM-dd HH:mm:ss.SSSSSSSSS');
|
||||
select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123456-0800', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ');
|
||||
select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123456America/Los_Angeles', 'yyyy-MM-dd HH:mm:ss.SSSSSSz');
|
||||
select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10', 3);
|
||||
select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.', 3);
|
||||
select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10', -3); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT}
|
||||
select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10', 9); -- { serverError BAD_ARGUMENTS }
|
||||
select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123', 3), parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123', 6);
|
||||
select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123', 'yyyy-MM-dd HH:mm:ss.SSS'), parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123', 'yyyy-MM-dd HH:mm:ss.SSSSSS');
|
||||
select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123', 3, 'yyyy-MM-dd HH:mm:ss.SSS'), parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123', 6, 'yyyy-MM-dd HH:mm:ss.SSSSSS');
|
||||
select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123', 3, 'yyyy-MM-dd HH:mm:ss.SSSS'); -- { serverError BAD_ARGUMENTS }
|
||||
select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123456-0812', 6, 'yyyy-MM-dd HH:mm:ss.SSSSSSZ');
|
||||
select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123456-08123', 6, 'yyyy-MM-dd HH:mm:ss.SSSSSSZZZ');
|
||||
select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123456America/Los_Angeles', 6, 'yyyy-MM-dd HH:mm:ss.SSSSSSz');
|
||||
select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123456America/Los_Angeles', 6, 'yyyy-MM-dd HH:mm:ss.SSSSSSzzz');
|
||||
-- incorrect timezone offset and timezone
|
||||
select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123456-8000', 6, 'yyyy-MM-dd HH:mm:ss.SSSSSSZ');
|
||||
select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123456ABCD', 6, 'yyyy-MM-dd HH:mm:ss.SSSSSSz'); -- { serverError BAD_ARGUMENTS }
|
||||
|
||||
select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123', 'yyyy-MM-dd HH:mm:ss.SSS');
|
||||
select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS');
|
||||
select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123456789', 'yyyy-MM-dd HH:mm:ss.SSSSSSSSS');
|
||||
select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123456-0800', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ');
|
||||
select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123456America/Los_Angeles', 'yyyy-MM-dd HH:mm:ss.SSSSSSz');
|
||||
select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10', 3);
|
||||
select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.', 3);
|
||||
select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10', -3); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT}
|
||||
select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10', 9); -- { serverError BAD_ARGUMENTS }
|
||||
select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123', 3), parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123', 6);
|
||||
select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123', 'yyyy-MM-dd HH:mm:ss.SSS'), parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123', 'yyyy-MM-dd HH:mm:ss.SSSSSS');
|
||||
select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123', 3, 'yyyy-MM-dd HH:mm:ss.SSS'), parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123', 6, 'yyyy-MM-dd HH:mm:ss.SSSSSS');
|
||||
select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123', 3, 'yyyy-MM-dd HH:mm:ss.SSSS'); -- { serverError BAD_ARGUMENTS }
|
||||
select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123456-0812', 6, 'yyyy-MM-dd HH:mm:ss.SSSSSSZ');
|
||||
select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123456-08123', 6, 'yyyy-MM-dd HH:mm:ss.SSSSSSZZZ');
|
||||
select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123456America/Los_Angeles', 6, 'yyyy-MM-dd HH:mm:ss.SSSSSSz');
|
||||
select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123456America/Los_Angeles', 6, 'yyyy-MM-dd HH:mm:ss.SSSSSSzzz');
|
||||
-- incorrect timezone offset and timezone
|
||||
select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123456-8000', 6, 'yyyy-MM-dd HH:mm:ss.SSSSSSZ');
|
||||
select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123456ABCD', 6, 'yyyy-MM-dd HH:mm:ss.SSSSSSz'); -- { serverError BAD_ARGUMENTS }
|
Loading…
Reference in New Issue
Block a user