mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-14 19:45:11 +00:00
Merge pull request #48851 from mauidude/date-trunc-return-date-time
date_trunc function to always return DateTime type
This commit is contained in:
commit
2aa8619534
@ -1,6 +1,6 @@
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeInterval.h>
|
||||
@ -39,51 +39,58 @@ public:
|
||||
{
|
||||
/// 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;
|
||||
auto check_first_argument = [&] {
|
||||
auto check_first_argument = [&]
|
||||
{
|
||||
const ColumnConst * datepart_column = checkAndGetColumnConst<ColumnString>(arguments[0].column.get());
|
||||
if (!datepart_column)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be constant string: "
|
||||
"name of datepart", getName());
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"First argument for function {} must be constant string: "
|
||||
"name of datepart",
|
||||
getName());
|
||||
|
||||
datepart_param = datepart_column->getValue<String>();
|
||||
if (datepart_param.empty())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "First argument (name of datepart) for function {} cannot be empty",
|
||||
getName());
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS, "First argument (name of datepart) for function {} cannot be empty", getName());
|
||||
|
||||
if (!IntervalKind::tryParseString(datepart_param, datepart_kind))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} doesn't look like datepart name in {}", datepart_param, getName());
|
||||
|
||||
result_type_is_date = (datepart_kind == IntervalKind::Year)
|
||||
|| (datepart_kind == IntervalKind::Quarter) || (datepart_kind == IntervalKind::Month)
|
||||
|| (datepart_kind == IntervalKind::Week);
|
||||
intermediate_type_is_date = (datepart_kind == IntervalKind::Year) || (datepart_kind == IntervalKind::Quarter)
|
||||
|| (datepart_kind == IntervalKind::Month) || (datepart_kind == IntervalKind::Week);
|
||||
};
|
||||
|
||||
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))
|
||||
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());
|
||||
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);
|
||||
|
||||
if (second_argument_is_date && ((datepart_kind == IntervalKind::Hour)
|
||||
|| (datepart_kind == IntervalKind::Minute) || (datepart_kind == IntervalKind::Second)))
|
||||
if (second_argument_is_date
|
||||
&& ((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());
|
||||
};
|
||||
|
||||
auto check_timezone_argument = [&] {
|
||||
auto check_timezone_argument = [&]
|
||||
{
|
||||
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",
|
||||
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);
|
||||
arguments[2].type->getName(),
|
||||
getName());
|
||||
};
|
||||
|
||||
if (arguments.size() == 2)
|
||||
@ -99,14 +106,13 @@ public:
|
||||
}
|
||||
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",
|
||||
getName(), arguments.size());
|
||||
getName(),
|
||||
arguments.size());
|
||||
}
|
||||
|
||||
if (result_type_is_date)
|
||||
return std::make_shared<DataTypeDate>();
|
||||
else
|
||||
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 1));
|
||||
}
|
||||
|
||||
@ -124,18 +130,31 @@ public:
|
||||
|
||||
auto to_start_of_interval = FunctionFactory::instance().get("toStartOfInterval", context);
|
||||
|
||||
ColumnPtr truncated_column;
|
||||
auto date_type = std::make_shared<DataTypeDate>();
|
||||
|
||||
if (arguments.size() == 2)
|
||||
return to_start_of_interval->build(temp_columns)->execute(temp_columns, result_type, input_rows_count);
|
||||
|
||||
temp_columns[2] = arguments[2];
|
||||
return to_start_of_interval->build(temp_columns)->execute(temp_columns, result_type, input_rows_count);
|
||||
}
|
||||
|
||||
bool hasInformationAboutMonotonicity() const override
|
||||
truncated_column = to_start_of_interval->build(temp_columns)
|
||||
->execute(temp_columns, intermediate_type_is_date ? date_type : result_type, input_rows_count);
|
||||
else
|
||||
{
|
||||
return true;
|
||||
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);
|
||||
}
|
||||
|
||||
if (!intermediate_type_is_date)
|
||||
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 { return true; }
|
||||
|
||||
Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override
|
||||
{
|
||||
return {.is_monotonic = true, .is_always_monotonic = true};
|
||||
@ -144,6 +163,7 @@ public:
|
||||
private:
|
||||
ContextPtr context;
|
||||
mutable IntervalKind::Kind datepart_kind = IntervalKind::Kind::Second;
|
||||
mutable bool intermediate_type_is_date = false;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -246,18 +246,18 @@ toUnixTimestamp
|
||||
1426415400
|
||||
1426415400
|
||||
date_trunc
|
||||
2019-01-01
|
||||
2020-01-01
|
||||
2020-01-01
|
||||
2019-10-01
|
||||
2020-01-01
|
||||
2020-01-01
|
||||
2019-12-01
|
||||
2020-01-01
|
||||
2020-01-01
|
||||
2019-12-30
|
||||
2019-12-30
|
||||
2019-12-30
|
||||
2019-01-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2019-10-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2019-12-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2019-12-30 00:00:00
|
||||
2019-12-30 00:00:00
|
||||
2019-12-30 00:00:00
|
||||
2019-12-31 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2020-01-02 00:00:00
|
||||
@ -270,18 +270,18 @@ date_trunc
|
||||
2019-12-31 20:11:22
|
||||
2020-01-01 12:11:22
|
||||
2020-01-02 05:11:22
|
||||
2019-01-01
|
||||
2020-01-01
|
||||
2020-01-01
|
||||
2019-10-01
|
||||
2020-01-01
|
||||
2020-01-01
|
||||
2019-12-01
|
||||
2020-01-01
|
||||
2020-01-01
|
||||
2019-12-30
|
||||
2019-12-30
|
||||
2019-12-30
|
||||
2019-01-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2019-10-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2019-12-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2019-12-30 00:00:00
|
||||
2019-12-30 00:00:00
|
||||
2019-12-30 00:00:00
|
||||
2019-12-31 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2020-01-02 00:00:00
|
||||
@ -294,8 +294,8 @@ date_trunc
|
||||
2019-12-31 20:11:22
|
||||
2020-01-01 12:11:22
|
||||
2020-01-02 05:11:22
|
||||
2020-01-01
|
||||
2020-01-01
|
||||
2020-01-01
|
||||
2019-12-30
|
||||
2020-01-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2019-12-30 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
|
@ -135,13 +135,13 @@ Code: 43
|
||||
------------------------------------------
|
||||
SELECT date_trunc(\'year\', N, \'Asia/Istanbul\')
|
||||
Code: 43
|
||||
"Date","2019-01-01"
|
||||
"Date","2019-01-01"
|
||||
"DateTime('Asia/Istanbul')","2019-01-01 00:00:00"
|
||||
"DateTime('Asia/Istanbul')","2019-01-01 00:00:00"
|
||||
------------------------------------------
|
||||
SELECT date_trunc(\'month\', N, \'Asia/Istanbul\')
|
||||
Code: 43
|
||||
"Date","2019-09-01"
|
||||
"Date","2019-09-01"
|
||||
"DateTime('Asia/Istanbul')","2019-09-01 00:00:00"
|
||||
"DateTime('Asia/Istanbul')","2019-09-01 00:00:00"
|
||||
------------------------------------------
|
||||
SELECT date_trunc(\'day\', N, \'Asia/Istanbul\')
|
||||
"DateTime('Asia/Istanbul')","2019-09-16 00:00:00"
|
||||
|
Loading…
Reference in New Issue
Block a user