Merge pull request #48851 from mauidude/date-trunc-return-date-time

date_trunc function to always return DateTime type
This commit is contained in:
Nikolay Degterinsky 2023-05-02 16:43:20 +02:00 committed by GitHub
commit 2aa8619534
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 91 additions and 71 deletions

View File

@ -1,6 +1,6 @@
#include <Columns/ColumnConst.h> #include <Columns/ColumnConst.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeDate.h> #include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h> #include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeInterval.h> #include <DataTypes/DataTypeInterval.h>
@ -25,7 +25,7 @@ class FunctionDateTrunc : public IFunction
public: public:
static constexpr auto name = "dateTrunc"; static constexpr auto name = "dateTrunc";
explicit FunctionDateTrunc(ContextPtr context_) : context(context_) {} explicit FunctionDateTrunc(ContextPtr context_) : context(context_) { }
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionDateTrunc>(context); } static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionDateTrunc>(context); }
@ -39,51 +39,58 @@ public:
{ {
/// The first argument is a constant string with the name of datepart. /// The first argument is a constant string with the name of datepart.
auto result_type_is_date = false; intermediate_type_is_date = false;
String datepart_param; String datepart_param;
auto check_first_argument = [&] { auto check_first_argument = [&]
{
const ColumnConst * datepart_column = checkAndGetColumnConst<ColumnString>(arguments[0].column.get()); const ColumnConst * datepart_column = checkAndGetColumnConst<ColumnString>(arguments[0].column.get());
if (!datepart_column) if (!datepart_column)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be constant string: " throw Exception(
"name of datepart", getName()); ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"First argument for function {} must be constant string: "
"name of datepart",
getName());
datepart_param = datepart_column->getValue<String>(); datepart_param = datepart_column->getValue<String>();
if (datepart_param.empty()) if (datepart_param.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "First argument (name of datepart) for function {} cannot be empty", throw Exception(
getName()); ErrorCodes::BAD_ARGUMENTS, "First argument (name of datepart) for function {} cannot be empty", getName());
if (!IntervalKind::tryParseString(datepart_param, datepart_kind)) if (!IntervalKind::tryParseString(datepart_param, datepart_kind))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} doesn't look like datepart name in {}", datepart_param, getName()); throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} doesn't look like datepart name in {}", datepart_param, getName());
result_type_is_date = (datepart_kind == IntervalKind::Year) intermediate_type_is_date = (datepart_kind == IntervalKind::Year) || (datepart_kind == IntervalKind::Quarter)
|| (datepart_kind == IntervalKind::Quarter) || (datepart_kind == IntervalKind::Month) || (datepart_kind == IntervalKind::Month) || (datepart_kind == IntervalKind::Week);
|| (datepart_kind == IntervalKind::Week);
}; };
bool second_argument_is_date = false; bool second_argument_is_date = false;
auto check_second_argument = [&] { auto check_second_argument = [&]
{
if (!isDate(arguments[1].type) && !isDateTime(arguments[1].type) && !isDateTime64(arguments[1].type)) if (!isDate(arguments[1].type) && !isDateTime(arguments[1].type) && !isDateTime64(arguments[1].type))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 2nd argument of function {}. " throw Exception(
"Should be a date or a date with time", arguments[1].type->getName(), getName()); ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of 2nd argument of function {}. "
"Should be a date or a date with time",
arguments[1].type->getName(),
getName());
second_argument_is_date = isDate(arguments[1].type); second_argument_is_date = isDate(arguments[1].type);
if (second_argument_is_date && ((datepart_kind == IntervalKind::Hour) if (second_argument_is_date
|| (datepart_kind == IntervalKind::Minute) || (datepart_kind == IntervalKind::Second))) && ((datepart_kind == IntervalKind::Hour) || (datepart_kind == IntervalKind::Minute)
|| (datepart_kind == IntervalKind::Second)))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type Date of argument for function {}", getName()); throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type Date of argument for function {}", getName());
}; };
auto check_timezone_argument = [&] { auto check_timezone_argument = [&]
{
if (!WhichDataType(arguments[2].type).isString()) if (!WhichDataType(arguments[2].type).isString())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument of function {}. "
"This argument is optional and must be a constant string with timezone name", "This argument is optional and must be a constant string with timezone name",
arguments[2].type->getName(), getName()); arguments[2].type->getName(),
getName());
if (second_argument_is_date && result_type_is_date)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"The timezone argument of function {} with datepart '{}' "
"is allowed only when the 2nd argument has the type DateTime",
getName(), datepart_param);
}; };
if (arguments.size() == 2) if (arguments.size() == 2)
@ -99,15 +106,14 @@ public:
} }
else else
{ {
throw Exception(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 2 or 3", "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3",
getName(), arguments.size()); getName(),
arguments.size());
} }
if (result_type_is_date) return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 1));
return std::make_shared<DataTypeDate>();
else
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 1));
} }
bool useDefaultImplementationForConstants() const override { return true; } bool useDefaultImplementationForConstants() const override { return true; }
@ -124,26 +130,40 @@ public:
auto to_start_of_interval = FunctionFactory::instance().get("toStartOfInterval", context); auto to_start_of_interval = FunctionFactory::instance().get("toStartOfInterval", context);
ColumnPtr truncated_column;
auto date_type = std::make_shared<DataTypeDate>();
if (arguments.size() == 2) if (arguments.size() == 2)
return to_start_of_interval->build(temp_columns)->execute(temp_columns, result_type, input_rows_count); truncated_column = to_start_of_interval->build(temp_columns)
->execute(temp_columns, intermediate_type_is_date ? date_type : result_type, input_rows_count);
else
{
temp_columns[2] = arguments[2];
truncated_column = to_start_of_interval->build(temp_columns)
->execute(temp_columns, intermediate_type_is_date ? date_type : result_type, input_rows_count);
}
temp_columns[2] = arguments[2]; if (!intermediate_type_is_date)
return to_start_of_interval->build(temp_columns)->execute(temp_columns, result_type, input_rows_count); return truncated_column;
ColumnsWithTypeAndName temp_truncated_column(1);
temp_truncated_column[0] = {truncated_column, date_type, ""};
auto to_date_time_or_default = FunctionFactory::instance().get("toDateTime", context);
return to_date_time_or_default->build(temp_truncated_column)->execute(temp_truncated_column, result_type, input_rows_count);
} }
bool hasInformationAboutMonotonicity() const override bool hasInformationAboutMonotonicity() const override { return true; }
{
return true;
}
Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override
{ {
return { .is_monotonic = true, .is_always_monotonic = true }; return {.is_monotonic = true, .is_always_monotonic = true};
} }
private: private:
ContextPtr context; ContextPtr context;
mutable IntervalKind::Kind datepart_kind = IntervalKind::Kind::Second; mutable IntervalKind::Kind datepart_kind = IntervalKind::Kind::Second;
mutable bool intermediate_type_is_date = false;
}; };
} }

View File

@ -246,18 +246,18 @@ toUnixTimestamp
1426415400 1426415400
1426415400 1426415400
date_trunc date_trunc
2019-01-01 2019-01-01 00:00:00
2020-01-01 2020-01-01 00:00:00
2020-01-01 2020-01-01 00:00:00
2019-10-01 2019-10-01 00:00:00
2020-01-01 2020-01-01 00:00:00
2020-01-01 2020-01-01 00:00:00
2019-12-01 2019-12-01 00:00:00
2020-01-01 2020-01-01 00:00:00
2020-01-01 2020-01-01 00:00:00
2019-12-30 2019-12-30 00:00:00
2019-12-30 2019-12-30 00:00:00
2019-12-30 2019-12-30 00:00:00
2019-12-31 00:00:00 2019-12-31 00:00:00
2020-01-01 00:00:00 2020-01-01 00:00:00
2020-01-02 00:00:00 2020-01-02 00:00:00
@ -270,18 +270,18 @@ date_trunc
2019-12-31 20:11:22 2019-12-31 20:11:22
2020-01-01 12:11:22 2020-01-01 12:11:22
2020-01-02 05:11:22 2020-01-02 05:11:22
2019-01-01 2019-01-01 00:00:00
2020-01-01 2020-01-01 00:00:00
2020-01-01 2020-01-01 00:00:00
2019-10-01 2019-10-01 00:00:00
2020-01-01 2020-01-01 00:00:00
2020-01-01 2020-01-01 00:00:00
2019-12-01 2019-12-01 00:00:00
2020-01-01 2020-01-01 00:00:00
2020-01-01 2020-01-01 00:00:00
2019-12-30 2019-12-30 00:00:00
2019-12-30 2019-12-30 00:00:00
2019-12-30 2019-12-30 00:00:00
2019-12-31 00:00:00 2019-12-31 00:00:00
2020-01-01 00:00:00 2020-01-01 00:00:00
2020-01-02 00:00:00 2020-01-02 00:00:00
@ -294,8 +294,8 @@ date_trunc
2019-12-31 20:11:22 2019-12-31 20:11:22
2020-01-01 12:11:22 2020-01-01 12:11:22
2020-01-02 05:11:22 2020-01-02 05:11:22
2020-01-01 2020-01-01 00:00:00
2020-01-01 2020-01-01 00:00:00
2020-01-01 2020-01-01 00:00:00
2019-12-30 2019-12-30 00:00:00
2020-01-01 00:00:00 2020-01-01 00:00:00

View File

@ -135,13 +135,13 @@ Code: 43
------------------------------------------ ------------------------------------------
SELECT date_trunc(\'year\', N, \'Asia/Istanbul\') SELECT date_trunc(\'year\', N, \'Asia/Istanbul\')
Code: 43 Code: 43
"Date","2019-01-01" "DateTime('Asia/Istanbul')","2019-01-01 00:00:00"
"Date","2019-01-01" "DateTime('Asia/Istanbul')","2019-01-01 00:00:00"
------------------------------------------ ------------------------------------------
SELECT date_trunc(\'month\', N, \'Asia/Istanbul\') SELECT date_trunc(\'month\', N, \'Asia/Istanbul\')
Code: 43 Code: 43
"Date","2019-09-01" "DateTime('Asia/Istanbul')","2019-09-01 00:00:00"
"Date","2019-09-01" "DateTime('Asia/Istanbul')","2019-09-01 00:00:00"
------------------------------------------ ------------------------------------------
SELECT date_trunc(\'day\', N, \'Asia/Istanbul\') SELECT date_trunc(\'day\', N, \'Asia/Istanbul\')
"DateTime('Asia/Istanbul')","2019-09-16 00:00:00" "DateTime('Asia/Istanbul')","2019-09-16 00:00:00"