support datetrunc and toStartOfxxx on date32

This commit is contained in:
liuneng 2024-08-26 16:47:59 +08:00
parent 674cddc969
commit 8ca4c872b8
6 changed files with 32 additions and 4 deletions

View File

@ -2,6 +2,7 @@
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeInterval.h>
#include <Formats/FormatSettings.h>
@ -43,6 +44,7 @@ public:
enum ResultType
{
Date,
Date32,
DateTime,
DateTime64,
};
@ -75,11 +77,11 @@ public:
bool second_argument_is_date = false;
auto check_second_argument = [&] {
if (!isDate(arguments[1].type) && !isDateTime(arguments[1].type) && !isDateTime64(arguments[1].type))
if (!isDateOrDate32(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 {}. "
"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 = isDateOrDate32(arguments[1].type);
if (second_argument_is_date && ((datepart_kind == IntervalKind::Kind::Hour)
|| (datepart_kind == IntervalKind::Kind::Minute) || (datepart_kind == IntervalKind::Kind::Second)))
@ -119,6 +121,8 @@ public:
if (result_type == ResultType::Date)
return std::make_shared<DataTypeDate>();
if (result_type == ResultType::Date32)
return std::make_shared<DataTypeDate32>();
else if (result_type == ResultType::DateTime)
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 1, false));
else

View File

@ -44,9 +44,9 @@ public:
auto check_first_argument = [&]
{
const DataTypePtr & type_arg1 = arguments[0].type;
if (!isDate(type_arg1) && !isDateTime(type_arg1) && !isDateTime64(type_arg1))
if (!isDateOrDate32(type_arg1) && !isDateTime(type_arg1) && !isDateTime64(type_arg1))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of 1st argument of function {}, expected a Date, DateTime or DateTime64",
"Illegal type {} of 1st argument of function {}, expected a Date, Date32, DateTime or DateTime64",
type_arg1->getName(), getName());
value_is_date = isDate(type_arg1);
};
@ -56,6 +56,7 @@ public:
enum class ResultType : uint8_t
{
Date,
Date32,
DateTime,
DateTime64
};
@ -128,6 +129,8 @@ public:
{
case ResultType::Date:
return std::make_shared<DataTypeDate>();
case ResultType::Date32:
return std::make_shared<DataTypeDate32>();
case ResultType::DateTime:
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false));
case ResultType::DateTime64:
@ -185,6 +188,12 @@ private:
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDate &>(time_column_type), *time_column_vec, interval_column, result_type, time_zone, input_rows_count);
}
else if (isDate32(time_column_type))
{
const auto * time_column_vec = checkAndGetColumn<ColumnDate32>(&time_column_col);
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDate32 &>(time_column_type), *time_column_vec, interval_column, result_type, time_zone, input_rows_count);
}
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for 1st argument of function {}, expected a Date, DateTime or DateTime64", getName());
}

View File

@ -1 +1,2 @@
1
2024-08-26 00:00:00

View File

@ -1 +1,2 @@
SELECT toStartOfDay(now()) = toDateTime(toDate(now()));
SELECT toStartOfDay(toDate32(now()));

View File

@ -1,11 +1,16 @@
2022-03-01 00:00:00
2022-03-01
2022-02-28
2022-02-28
2022-03-01 00:00:00
2022-03-01
2022-02-28
2022-02-28
2022-03-01 00:00:00
2022-03-01
2022-03-01 00:00:00
2022-03-01
2022-02-28
2022-02-28
2022-03-01 12:12:12.012000000
2022-03-01 12:12:12.012346

View File

@ -1,12 +1,17 @@
SELECT dateTrunc('DAY', toDateTime('2022-03-01 12:55:55'));
SELECT dateTrunc('MONTH', toDateTime64('2022-03-01 12:55:55', 2));
SELECT dateTrunc('WEEK', toDate('2022-03-01'));
SELECT dateTrunc('WEEK', toDate32('2022-03-01'));
SELECT dateTrunc('Day', toDateTime('2022-03-01 12:55:55'));
SELECT dateTrunc('Month', toDateTime64('2022-03-01 12:55:55', 2));
SELECT dateTrunc('Week', toDate('2022-03-01'));
SELECT dateTrunc('Week', toDate32('2022-03-01'));
SELECT dateTrunc('day', toDateTime('2022-03-01 12:55:55'));
SELECT dateTrunc('month', toDateTime64('2022-03-01 12:55:55', 2));
SELECT dateTrunc('day', toDate32('2022-03-01'));
SELECT dateTrunc('month', toDate32('2022-03-01'));
SELECT dateTrunc('week', toDate('2022-03-01'));
SELECT dateTrunc('week', toDate32('2022-03-01'));
SELECT dateTrunc('Nanosecond', toDateTime64('2022-03-01 12:12:12.0123', 3));
SELECT dateTrunc('MicroSecond', toDateTime64('2022-03-01 12:12:12.0123456', 7));
SELECT dateTrunc('MILLISECOND', toDateTime64('2022-03-01 12:12:12.012324251', 9));
@ -25,3 +30,6 @@ SELECT dateTrunc('MILLISECOND', toDateTime('2022-03-01')); -- { serverError ILL
SELECT dateTrunc('Nanosecond', toDate('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT dateTrunc('MicroSecond', toDate('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT dateTrunc('MILLISECOND', toDate('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT dateTrunc('Nanosecond', toDate32('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT dateTrunc('MicroSecond', toDate32('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT dateTrunc('MILLISECOND', toDate32('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }