mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
fix comments
This commit is contained in:
parent
b618fe03bf
commit
699b9d4026
@ -57,8 +57,15 @@ namespace
|
||||
Null
|
||||
};
|
||||
|
||||
enum class ReturnType: uint8_t
|
||||
{
|
||||
DateTime,
|
||||
DateTime64
|
||||
};
|
||||
|
||||
constexpr Int32 minYear = 1970;
|
||||
constexpr Int32 maxYear = 2106;
|
||||
constexpr Int32 maxPrecisionOfDateTime64 = 6;
|
||||
|
||||
const std::unordered_map<String, std::pair<String, Int32>> dayOfWeekMap{
|
||||
{"mon", {"day", 1}},
|
||||
@ -570,8 +577,8 @@ namespace
|
||||
}
|
||||
};
|
||||
|
||||
/// _FUNC_(str[, format, timezone])
|
||||
template <typename Name, ParseSyntax parse_syntax, ErrorHandling error_handling, bool parseDateTime64 = false>
|
||||
/// _FUNC_(str[scale, format, timezone])
|
||||
template <typename Name, ParseSyntax parse_syntax, ReturnType return_type, ErrorHandling error_handling>
|
||||
class FunctionParseDateTimeImpl : public IFunction
|
||||
{
|
||||
public:
|
||||
@ -602,79 +609,112 @@ namespace
|
||||
{"time", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isString), nullptr, "String"}
|
||||
};
|
||||
|
||||
FunctionArgumentDescriptors optional_args{
|
||||
{"format", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isString), nullptr, "String"},
|
||||
{"timezone", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isString), &isColumnConst, "const String"}
|
||||
};
|
||||
|
||||
FunctionArgumentDescriptors optional_args;
|
||||
if constexpr (return_type == ReturnType::DateTime64)
|
||||
{
|
||||
optional_args = {
|
||||
{"precision or format", static_cast<FunctionArgumentDescriptor::TypeValidator>([](const IDataType & data_type) -> bool {
|
||||
return isUInt(data_type) || isString(data_type);
|
||||
}), nullptr, "Number or String"},
|
||||
{"format", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isString), nullptr, "String"},
|
||||
{"timezone", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isString), &isColumnConst, "const String"}
|
||||
};
|
||||
}
|
||||
else
|
||||
optional_args = {
|
||||
{"format", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isString), nullptr, "String"},
|
||||
{"timezone", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isString), &isColumnConst, "const String"}
|
||||
};
|
||||
validateFunctionArguments(*this, arguments, mandatory_args, optional_args);
|
||||
|
||||
String time_zone_name = getTimeZone(arguments).getTimeZone();
|
||||
DataTypePtr date_type = nullptr;
|
||||
if constexpr (parseDateTime64)
|
||||
DataTypePtr data_type;
|
||||
if constexpr (return_type == ReturnType::DateTime64)
|
||||
{
|
||||
String format = getFormat(arguments);
|
||||
std::vector<Instruction> instructions = parseFormat(format);
|
||||
UInt32 scale = 0;
|
||||
if (!instructions.empty())
|
||||
if (arguments.size() == 1)
|
||||
return std::make_shared<DataTypeDateTime64>(0, time_zone_name);
|
||||
else
|
||||
{
|
||||
for (const auto & ins : instructions)
|
||||
UInt32 precision = 0;
|
||||
if (isUInt(arguments[1].type))
|
||||
{
|
||||
if (scale > 0)
|
||||
break;
|
||||
const String fragment = ins.getFragment();
|
||||
const auto * col_precision = checkAndGetColumnConst<ColumnUInt8>(arguments[1].column.get());
|
||||
if (col_precision)
|
||||
precision = col_precision->getValue<UInt8>();
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"The input precision 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);
|
||||
std::vector<Instruction> instructions = parseFormat(format);
|
||||
for (const auto & instruction : instructions)
|
||||
{
|
||||
const String & fragment = instruction.getFragment();
|
||||
UInt32 val = 0;
|
||||
for (char ch : fragment)
|
||||
{
|
||||
if (ch != 'S')
|
||||
{
|
||||
scale = 0;
|
||||
val = 0;
|
||||
break;
|
||||
}
|
||||
else
|
||||
scale++;
|
||||
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)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"The precision of input format string {} not equals the given precision value {}.",
|
||||
format,
|
||||
precision);
|
||||
else if (precision == 0 && val != 0)
|
||||
precision = val;
|
||||
}
|
||||
if (precision > maxPrecisionOfDateTime64)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"The precision of the input format string {} exceed the max precision value {}.",
|
||||
format,
|
||||
maxPrecisionOfDateTime64);
|
||||
data_type = std::make_shared<DataTypeDateTime64>(precision, time_zone_name);
|
||||
}
|
||||
date_type = std::make_shared<DataTypeDateTime64>(scale, time_zone_name);
|
||||
}
|
||||
else
|
||||
date_type = std::make_shared<DataTypeDateTime>(time_zone_name);
|
||||
data_type = std::make_shared<DataTypeDateTime>(time_zone_name);
|
||||
if (error_handling == ErrorHandling::Null)
|
||||
return std::make_shared<DataTypeNullable>(date_type);
|
||||
return date_type;
|
||||
return std::make_shared<DataTypeNullable>(data_type);
|
||||
return data_type;
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||
{
|
||||
ColumnUInt8::MutablePtr col_null_map;
|
||||
DataTypePtr non_null_result_type;
|
||||
if constexpr (error_handling == ErrorHandling::Null)
|
||||
col_null_map = ColumnUInt8::create(input_rows_count, 0);
|
||||
if constexpr (parseDateTime64)
|
||||
non_null_result_type = removeNullable(result_type);
|
||||
else
|
||||
non_null_result_type = result_type;
|
||||
|
||||
if constexpr (return_type == ReturnType::DateTime64)
|
||||
{
|
||||
const DataTypeDateTime64 * datatime64_type = checkAndGetDataType<DataTypeDateTime64>(removeNullable(result_type).get());
|
||||
auto col_res = ColumnDateTime64::create(input_rows_count, datatime64_type->getScale());
|
||||
PaddedPODArray<DataTypeDateTime64::FieldType> & res_data = col_res->getData();
|
||||
executeImpl2<DataTypeDateTime64::FieldType>(arguments, result_type, input_rows_count, res_data, col_null_map);
|
||||
if constexpr (error_handling == ErrorHandling::Null)
|
||||
return ColumnNullable::create(std::move(col_res), std::move(col_null_map));
|
||||
else
|
||||
return col_res;
|
||||
const auto * datatime64_type = checkAndGetDataType<DataTypeDateTime64>(non_null_result_type.get());
|
||||
MutableColumnPtr col_res = ColumnDateTime64::create(input_rows_count, datatime64_type->getScale());
|
||||
ColumnDateTime64 * col_datetime64 = assert_cast<ColumnDateTime64 *>(col_res.get());
|
||||
return executeImpl2<DataTypeDateTime64::FieldType>(arguments, result_type, input_rows_count, col_res, col_datetime64->getData());
|
||||
}
|
||||
else
|
||||
{
|
||||
auto col_res = ColumnDateTime::create(input_rows_count);
|
||||
PaddedPODArray<DataTypeDateTime::FieldType> & res_data = col_res->getData();
|
||||
executeImpl2<DataTypeDateTime::FieldType>(arguments, result_type, input_rows_count, res_data, col_null_map);
|
||||
if constexpr (error_handling == ErrorHandling::Null)
|
||||
return ColumnNullable::create(std::move(col_res), std::move(col_null_map));
|
||||
else
|
||||
return col_res;
|
||||
MutableColumnPtr col_res = ColumnDateTime::create(input_rows_count);
|
||||
ColumnDateTime * col_datetime = assert_cast<ColumnDateTime *>(col_res.get());
|
||||
return executeImpl2<DataTypeDateTime::FieldType>(arguments, result_type, input_rows_count, col_res, col_datetime->getData());
|
||||
}
|
||||
}
|
||||
|
||||
template<typename T>
|
||||
void executeImpl2(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count,
|
||||
PaddedPODArray<T> & res_data, ColumnUInt8::MutablePtr & col_null_map) const
|
||||
ColumnPtr executeImpl2(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count,
|
||||
MutableColumnPtr & col_res, PaddedPODArray<T> & res_data) const
|
||||
{
|
||||
const auto * col_str = checkAndGetColumn<ColumnString>(arguments[0].column.get());
|
||||
if (!col_str)
|
||||
@ -683,8 +723,21 @@ namespace
|
||||
"Illegal column {} of first ('str') argument of function {}. Must be string.",
|
||||
arguments[0].column->getName(),
|
||||
getName());
|
||||
|
||||
ColumnUInt8::MutablePtr col_null_map;
|
||||
if constexpr (error_handling == ErrorHandling::Null)
|
||||
col_null_map = ColumnUInt8::create(input_rows_count, 0);
|
||||
|
||||
String format = getFormat(arguments);
|
||||
Int64 multiplier = 0;
|
||||
UInt32 precision = 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);
|
||||
}
|
||||
|
||||
String format = getFormat(arguments, precision);
|
||||
const auto & time_zone = getTimeZone(arguments);
|
||||
std::vector<Instruction> instructions = parseFormat(format);
|
||||
|
||||
@ -733,8 +786,8 @@ namespace
|
||||
|
||||
Int64OrError result = 0;
|
||||
|
||||
/// Ensure all input was consumed
|
||||
if (!parseDateTime64 && cur < end)
|
||||
/// Ensure all input was consumed when the return type is `DateTime`.
|
||||
if (return_type == ReturnType::DateTime && cur < end)
|
||||
{
|
||||
result = tl::unexpected(ErrorCodeAndMessage(
|
||||
ErrorCodes::CANNOT_PARSE_DATETIME,
|
||||
@ -747,12 +800,8 @@ namespace
|
||||
{
|
||||
if (result = datetime.buildDateTime(time_zone); result.has_value())
|
||||
{
|
||||
if constexpr (parseDateTime64)
|
||||
{
|
||||
const DataTypeDateTime64 * datatime64_type = checkAndGetDataType<DataTypeDateTime64>(removeNullable(result_type).get());
|
||||
Int64 multiplier = DecimalUtils::scaleMultiplier<DateTime64>(datatime64_type->getScale());
|
||||
if constexpr (return_type == ReturnType::DateTime64)
|
||||
res_data[i] = static_cast<Int64>(*result) * multiplier + datetime.microsecond;
|
||||
}
|
||||
else
|
||||
res_data[i] = static_cast<UInt32>(*result);
|
||||
}
|
||||
@ -777,6 +826,10 @@ namespace
|
||||
}
|
||||
}
|
||||
}
|
||||
if constexpr (error_handling == ErrorHandling::Null)
|
||||
return ColumnNullable::create(std::move(col_res), std::move(col_null_map));
|
||||
else
|
||||
return std::move(col_res);
|
||||
}
|
||||
|
||||
|
||||
@ -808,7 +861,7 @@ namespace
|
||||
explicit Instruction(const String & literal_) : literal(literal_), fragment("LITERAL") { }
|
||||
explicit Instruction(String && literal_) : literal(std::move(literal_)), fragment("LITERAL") { }
|
||||
|
||||
String getFragment() const { return fragment; }
|
||||
const String & getFragment() const { return fragment; }
|
||||
|
||||
/// For debug
|
||||
[[maybe_unused]] String toString() const
|
||||
@ -1695,7 +1748,7 @@ namespace
|
||||
}
|
||||
|
||||
[[nodiscard]]
|
||||
static PosOrError jodaMicroSecondOfSecond(size_t repetitions, Pos cur, Pos end, const String & fragment, DateTime<error_handling> & date)
|
||||
static PosOrError jodaMicrosecondOfSecond(size_t repetitions, Pos cur, Pos end, const String & fragment, DateTime<error_handling> & date)
|
||||
{
|
||||
Int32 microsecond;
|
||||
ASSIGN_RESULT_OR_RETURN_ERROR(cur, (readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2uz), fragment, microsecond)))
|
||||
@ -1704,25 +1757,25 @@ namespace
|
||||
}
|
||||
|
||||
[[nodiscard]]
|
||||
static PosOrError jodaTimezoneId(size_t, Pos cur, Pos end, const String &, DateTime<error_handling> & date)
|
||||
static PosOrError jodaTimezone(size_t, Pos cur, Pos end, const String &, DateTime<error_handling> & date)
|
||||
{
|
||||
String dateTimeZone;
|
||||
String read_time_zone;
|
||||
while (cur <= end)
|
||||
{
|
||||
dateTimeZone += *cur;
|
||||
read_time_zone += *cur;
|
||||
++cur;
|
||||
}
|
||||
const DateLUTImpl & date_time_zone = DateLUT::instance(dateTimeZone);
|
||||
const DateLUTImpl & date_time_zone = DateLUT::instance(read_time_zone);
|
||||
const auto result = date.buildDateTime(date_time_zone);
|
||||
if (result.has_value())
|
||||
{
|
||||
const auto timezoneOffset = date_time_zone.timezoneOffset(*result);
|
||||
const DateLUTImpl::Time timezone_offset = date_time_zone.timezoneOffset(*result);
|
||||
date.has_time_zone_offset = true;
|
||||
date.time_zone_offset = timezoneOffset;
|
||||
date.time_zone_offset = timezone_offset;
|
||||
return cur;
|
||||
}
|
||||
else
|
||||
RETURN_ERROR(ErrorCodes::CANNOT_PARSE_DATETIME, "Unable to build date time from timezone {}", dateTimeZone)
|
||||
RETURN_ERROR(ErrorCodes::CANNOT_PARSE_DATETIME, "Unable to build date time from timezone {}", read_time_zone)
|
||||
}
|
||||
|
||||
[[nodiscard]]
|
||||
@ -1745,8 +1798,22 @@ namespace
|
||||
|
||||
Int32 hour;
|
||||
ASSIGN_RESULT_OR_RETURN_ERROR(cur, (readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2uz), fragment, hour)))
|
||||
if (hour < 0 || hour > 23)
|
||||
RETURN_ERROR(
|
||||
ErrorCodes::CANNOT_PARSE_DATETIME,
|
||||
"Unable to parse fragment {} from {} because of the hour of datetime not in range [0, 23]: {}",
|
||||
fragment,
|
||||
std::string_view(cur, end - cur),
|
||||
std::string_view(cur, 1))
|
||||
Int32 minute;
|
||||
ASSIGN_RESULT_OR_RETURN_ERROR(cur, (readNumberWithVariableLength(cur, end, false, false, false, repetitions, std::max(repetitions, 2uz), fragment, minute)))
|
||||
if (minute < 0 || minute > 59)
|
||||
RETURN_ERROR(
|
||||
ErrorCodes::CANNOT_PARSE_DATETIME,
|
||||
"Unable to parse fragment {} from {} because of the minute of datetime not in range [0, 59]: {}",
|
||||
fragment,
|
||||
std::string_view(cur, end - cur),
|
||||
std::string_view(cur, 1))
|
||||
date.has_time_zone_offset = true;
|
||||
date.time_zone_offset = sign * (hour * 3600 + minute * 60);
|
||||
return cur;
|
||||
@ -2133,10 +2200,10 @@ namespace
|
||||
instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaSecondOfMinute, repetitions));
|
||||
break;
|
||||
case 'S':
|
||||
instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaMicroSecondOfSecond, repetitions));
|
||||
instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaMicrosecondOfSecond, repetitions));
|
||||
break;
|
||||
case 'z':
|
||||
instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaTimezoneId, repetitions));
|
||||
instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaTimezone, repetitions));
|
||||
break;
|
||||
case 'Z':
|
||||
instructions.emplace_back(ACTION_ARGS_WITH_BIND(Instruction::jodaTimezoneOffset, repetitions));
|
||||
@ -2156,26 +2223,45 @@ namespace
|
||||
}
|
||||
|
||||
|
||||
String getFormat(const ColumnsWithTypeAndName & arguments) const
|
||||
String getFormat(const ColumnsWithTypeAndName & arguments, UInt32 precision) const
|
||||
{
|
||||
if (arguments.size() == 1)
|
||||
size_t format_arg_index = 1;
|
||||
if constexpr (return_type == ReturnType::DateTime64)
|
||||
{
|
||||
if constexpr (parse_syntax == ParseSyntax::MySQL)
|
||||
return "%Y-%m-%d %H:%i:%s";
|
||||
/// When parse `DateTime64` like `parseDateTime64[InJodaSyntax][OrZero/OrNull]('2024-11-05 12:22.22.123', 3), then the format is treated
|
||||
/// as default value `yyyy-MM-dd HH:mm:ss`.
|
||||
/// When parse `DateTime64` like `parseDateTime64[InJodaSyntax][OrZero/OrNull]('2024-11-05 12:22:22.123', 'yyyy-MM-dd HH:mm:ss.SSS')`,
|
||||
/// then the second argument is the format.
|
||||
/// When parse `DateTime64` like `parseDateTime64[InJodaSyntax][OrZero/OrNull]('2024-11-05 12:22:22.123', 3, 'yyyy-MM-dd HH:mm:ss.SSS')`,
|
||||
/// then the third argument is the format.
|
||||
if (arguments.size() > 1 && isString(removeNullable(arguments[1].type)))
|
||||
format_arg_index = 1;
|
||||
else
|
||||
return "yyyy-MM-dd HH:mm:ss";
|
||||
format_arg_index = 2;
|
||||
}
|
||||
|
||||
if (arguments.size() <= format_arg_index)
|
||||
{
|
||||
String format;
|
||||
if constexpr (parse_syntax == ParseSyntax::MySQL)
|
||||
format = "%Y-%m-%d %H:%i:%s";
|
||||
else
|
||||
format = "yyyy-MM-dd HH:mm:ss";
|
||||
if (precision > 0)
|
||||
format += "." + String(precision, 'S');
|
||||
return format;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!arguments[1].column || !isColumnConst(*arguments[1].column))
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Argument at index {} for function {} must be constant", 1, getName());
|
||||
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[1].column.get());
|
||||
const auto * col_format = checkAndGetColumnConst<ColumnString>(arguments[format_arg_index].column.get());
|
||||
if (!col_format)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal column {} of second ('format') argument of function {}. Must be constant string.",
|
||||
arguments[1].column->getName(),
|
||||
arguments[format_arg_index].column->getName(),
|
||||
getName());
|
||||
return col_format->getValue<String>();
|
||||
}
|
||||
@ -2183,15 +2269,19 @@ namespace
|
||||
|
||||
const DateLUTImpl & getTimeZone(const ColumnsWithTypeAndName & arguments) const
|
||||
{
|
||||
if (arguments.size() < 3)
|
||||
size_t timezone_arg_index = 2;
|
||||
if constexpr (return_type == ReturnType::DateTime64)
|
||||
timezone_arg_index = 3;
|
||||
|
||||
if (arguments.size() <= timezone_arg_index)
|
||||
return DateLUT::instance();
|
||||
|
||||
const auto * col = checkAndGetColumnConst<ColumnString>(arguments[2].column.get());
|
||||
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.",
|
||||
arguments[2].column->getName(),
|
||||
arguments[timezone_arg_index].column->getName(),
|
||||
getName());
|
||||
|
||||
String time_zone = col->getValue<String>();
|
||||
@ -2229,6 +2319,21 @@ namespace
|
||||
static constexpr auto name = "parseDateTimeInJodaSyntaxOrNull";
|
||||
};
|
||||
|
||||
struct NameParseDateTime64
|
||||
{
|
||||
static constexpr auto name = "parseDateTime64";
|
||||
};
|
||||
|
||||
struct NameParseDateTime64OrZero
|
||||
{
|
||||
static constexpr auto name = "parseDateTime64OrZero";
|
||||
};
|
||||
|
||||
struct NameParseDateTime64OrNull
|
||||
{
|
||||
static constexpr auto name = "parseDateTime64OrNull";
|
||||
};
|
||||
|
||||
struct NameParseDateTime64InJodaSyntax
|
||||
{
|
||||
static constexpr auto name = "parseDateTime64InJodaSyntax";
|
||||
@ -2244,15 +2349,18 @@ namespace
|
||||
static constexpr auto name = "parseDateTime64InJodaSyntaxOrNull";
|
||||
};
|
||||
|
||||
using FunctionParseDateTime = FunctionParseDateTimeImpl<NameParseDateTime, ParseSyntax::MySQL, ErrorHandling::Exception>;
|
||||
using FunctionParseDateTimeOrZero = FunctionParseDateTimeImpl<NameParseDateTimeOrZero, ParseSyntax::MySQL, ErrorHandling::Zero>;
|
||||
using FunctionParseDateTimeOrNull = FunctionParseDateTimeImpl<NameParseDateTimeOrNull, ParseSyntax::MySQL, ErrorHandling::Null>;
|
||||
using FunctionParseDateTimeInJodaSyntax = FunctionParseDateTimeImpl<NameParseDateTimeInJodaSyntax, ParseSyntax::Joda, ErrorHandling::Exception>;
|
||||
using FunctionParseDateTimeInJodaSyntaxOrZero = FunctionParseDateTimeImpl<NameParseDateTimeInJodaSyntaxOrZero, ParseSyntax::Joda, ErrorHandling::Zero>;
|
||||
using FunctionParseDateTimeInJodaSyntaxOrNull = FunctionParseDateTimeImpl<NameParseDateTimeInJodaSyntaxOrNull, ParseSyntax::Joda, ErrorHandling::Null>;
|
||||
using FunctionParseDateTime64InJodaSyntax = FunctionParseDateTimeImpl<NameParseDateTime64InJodaSyntax, ParseSyntax::Joda, ErrorHandling::Exception, true>;
|
||||
using FunctionParseDateTime64InJodaSyntaxOrZero = FunctionParseDateTimeImpl<NameParseDateTime64InJodaSyntaxOrZero, ParseSyntax::Joda, ErrorHandling::Zero, true>;
|
||||
using FunctionParseDateTime64InJodaSyntaxOrNull = FunctionParseDateTimeImpl<NameParseDateTime64InJodaSyntaxOrNull, ParseSyntax::Joda, ErrorHandling::Null, true>;
|
||||
using FunctionParseDateTime = FunctionParseDateTimeImpl<NameParseDateTime, ParseSyntax::MySQL, ReturnType::DateTime, ErrorHandling::Exception>;
|
||||
using FunctionParseDateTimeOrZero = FunctionParseDateTimeImpl<NameParseDateTimeOrZero, ParseSyntax::MySQL, ReturnType::DateTime, ErrorHandling::Zero>;
|
||||
using FunctionParseDateTimeOrNull = FunctionParseDateTimeImpl<NameParseDateTimeOrNull, ParseSyntax::MySQL, ReturnType::DateTime, ErrorHandling::Null>;
|
||||
using FunctionParseDateTime64 = FunctionParseDateTimeImpl<NameParseDateTime64, ParseSyntax::MySQL, ReturnType::DateTime64, ErrorHandling::Exception>;
|
||||
using FunctionParseDateTime64OrZero = FunctionParseDateTimeImpl<NameParseDateTime64OrZero, ParseSyntax::MySQL, ReturnType::DateTime64, ErrorHandling::Zero>;
|
||||
using FunctionParseDateTime64OrNull = FunctionParseDateTimeImpl<NameParseDateTime64OrNull, ParseSyntax::MySQL, ReturnType::DateTime64, ErrorHandling::Null>;
|
||||
using FunctionParseDateTimeInJodaSyntax = FunctionParseDateTimeImpl<NameParseDateTimeInJodaSyntax, ParseSyntax::Joda, ReturnType::DateTime, ErrorHandling::Exception>;
|
||||
using FunctionParseDateTimeInJodaSyntaxOrZero = FunctionParseDateTimeImpl<NameParseDateTimeInJodaSyntaxOrZero, ParseSyntax::Joda, ReturnType::DateTime, ErrorHandling::Zero>;
|
||||
using FunctionParseDateTimeInJodaSyntaxOrNull = FunctionParseDateTimeImpl<NameParseDateTimeInJodaSyntaxOrNull, ParseSyntax::Joda, ReturnType::DateTime, ErrorHandling::Null>;
|
||||
using FunctionParseDateTime64InJodaSyntax = FunctionParseDateTimeImpl<NameParseDateTime64InJodaSyntax, ParseSyntax::Joda, ReturnType::DateTime64, ErrorHandling::Exception>;
|
||||
using FunctionParseDateTime64InJodaSyntaxOrZero = FunctionParseDateTimeImpl<NameParseDateTime64InJodaSyntaxOrZero, ParseSyntax::Joda, ReturnType::DateTime64, ErrorHandling::Zero>;
|
||||
using FunctionParseDateTime64InJodaSyntaxOrNull = FunctionParseDateTimeImpl<NameParseDateTime64InJodaSyntaxOrNull, ParseSyntax::Joda, ReturnType::DateTime64, ErrorHandling::Null>;
|
||||
}
|
||||
|
||||
REGISTER_FUNCTION(ParseDateTime)
|
||||
@ -2262,6 +2370,9 @@ REGISTER_FUNCTION(ParseDateTime)
|
||||
factory.registerFunction<FunctionParseDateTimeOrZero>();
|
||||
factory.registerFunction<FunctionParseDateTimeOrNull>();
|
||||
factory.registerAlias("str_to_date", FunctionParseDateTimeOrNull::name, FunctionFactory::Case::Insensitive);
|
||||
factory.registerFunction<FunctionParseDateTime64>();
|
||||
factory.registerFunction<FunctionParseDateTime64OrZero>();
|
||||
factory.registerFunction<FunctionParseDateTime64OrNull>();
|
||||
|
||||
factory.registerFunction<FunctionParseDateTimeInJodaSyntax>();
|
||||
factory.registerFunction<FunctionParseDateTimeInJodaSyntaxOrZero>();
|
||||
|
Loading…
Reference in New Issue
Block a user