Merge pull request #13888 from vladimir-golovchenko/add-date-trunc-function

Added date_trunc function
This commit is contained in:
alexey-milovidov 2020-08-27 02:12:27 +03:00 committed by GitHub
commit 0f706c01ca
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 386 additions and 0 deletions

View File

@ -339,6 +339,22 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d
└────────────┴───────────┴───────────┴───────────┘
```
## date_trunc(datepart, time\_or\_data\[, time\_zone\]), dateTrunc(datepart, time\_or\_data\[, time\_zone\]) {#date_trunc}
Truncates a date or date with time based on the specified datepart, such as
- `second`
- `minute`
- `hour`
- `day`
- `week`
- `month`
- `quarter`
- `year`
```sql
SELECT date_trunc('hour', now())
```
## now {#now}
Accepts zero arguments and returns the current time at one of the moments of request execution.

View File

@ -4,6 +4,7 @@
/format_schemas
/metadata_dropped
/preprocessed_configs
/shadow
/tmp
/user_files

View File

@ -176,4 +176,50 @@ const char * IntervalKind::toNameOfFunctionExtractTimePart() const
}
__builtin_unreachable();
}
bool IntervalKind::tryParseString(const std::string & kind, IntervalKind::Kind & result)
{
if ("second" == kind)
{
result = IntervalKind::Second;
return true;
}
if ("minute" == kind)
{
result = IntervalKind::Minute;
return true;
}
if ("hour" == kind)
{
result = IntervalKind::Hour;
return true;
}
if ("day" == kind)
{
result = IntervalKind::Day;
return true;
}
if ("week" == kind)
{
result = IntervalKind::Week;
return true;
}
if ("month" == kind)
{
result = IntervalKind::Month;
return true;
}
if ("quarter" == kind)
{
result = IntervalKind::Quarter;
return true;
}
if ("year" == kind)
{
result = IntervalKind::Year;
return true;
}
return false;
}
}

View File

@ -52,5 +52,10 @@ struct IntervalKind
/// For example, `IntervalKind{IntervalKind::Day}.getExtractTimePartFunctionName()`
/// returns "toDayOfMonth".
const char * toNameOfFunctionExtractTimePart() const;
/// Converts the string representation of an interval kind to its IntervalKind equivalent.
/// Returns false if the conversion unsucceeded.
/// For example, `IntervalKind::tryParseString('second', result)` returns `result` equals `IntervalKind::Kind::Second`.
static bool tryParseString(const std::string & kind, IntervalKind::Kind & result);
};
}

View File

@ -0,0 +1,171 @@
#include <Columns/ColumnConst.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeInterval.h>
#include <Functions/DateTimeTransforms.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int BAD_ARGUMENTS;
}
class FunctionDateTrunc : public IFunction
{
public:
static constexpr auto name = "date_trunc";
explicit FunctionDateTrunc(const Context & context_) : context(context_) {}
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionDateTrunc>(context); }
String getName() const override { return name; }
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
/// The first argument is a constant string with the name of datepart.
auto result_type_is_date = false;
String datepart_param;
auto check_first_argument = [&] {
const ColumnConst * datepart_column = checkAndGetColumnConst<ColumnString>(arguments[0].column.get());
if (!datepart_column)
throw Exception("First argument for function " + getName() + " must be constant string: name of datepart",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
datepart_param = datepart_column->getValue<String>();
if (datepart_param.empty())
throw Exception("First argument (name of datepart) for function " + getName() + " cannot be empty",
ErrorCodes::BAD_ARGUMENTS);
if (!IntervalKind::tryParseString(datepart_param, datepart_kind))
throw Exception(datepart_param + " doesn't look like datepart name in " + getName(),
ErrorCodes::BAD_ARGUMENTS);
result_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 = [&] {
if (!isDateOrDateTime(arguments[1].type))
throw Exception(
"Illegal type " + arguments[1].type->getName() + " of 2nd argument of function " + getName()
+ ". Should be a date or a date with time",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
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)))
throw Exception("Illegal type Date of argument for function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
};
auto check_timezone_argument = [&] {
if (!WhichDataType(arguments[2].type).isString())
throw Exception(
"Illegal type " + arguments[2].type->getName() + " of argument of function " + getName()
+ ". This argument is optional and must be a constant string with timezone name",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (second_argument_is_date && result_type_is_date)
throw Exception(
"The timezone argument of function " + getName() + " with datepart '" + datepart_param
+ "' is allowed only when the 2nd argument has the type DateTime",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
};
if (arguments.size() == 2)
{
check_first_argument();
check_second_argument();
}
else if (arguments.size() == 3)
{
check_first_argument();
check_second_argument();
check_timezone_argument();
}
else
{
throw Exception(
"Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size())
+ ", should be 2 or 3",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
if (result_type_is_date)
return std::make_shared<DataTypeDate>();
else
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 1));
}
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 2}; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
Block temp_block = block;
const UInt16 interval_value = 1;
const ColumnPtr interval_column = ColumnConst::create(ColumnInt64::create(1, interval_value), input_rows_count);
const size_t interval_pos = temp_block.columns();
temp_block.insert({interval_column, std::make_shared<DataTypeInterval>(datepart_kind), ""});
if (arguments.size() == 2)
{
auto to_start_of_interval = FunctionFactory::instance().get("toStartOfInterval", context)->build(
{temp_block.getByPosition(arguments[1]), temp_block.getByPosition(interval_pos)});
to_start_of_interval->execute(temp_block, {arguments[1], interval_pos}, result, input_rows_count);
}
else
{
auto to_start_of_interval = FunctionFactory::instance().get("toStartOfInterval", context)->build(
{temp_block.getByPosition(arguments[1]), temp_block.getByPosition(interval_pos),
temp_block.getByPosition(arguments[2])});
to_start_of_interval->execute(temp_block, {arguments[1], interval_pos, arguments[2]}, result, input_rows_count);
}
block.getByPosition(result).column = std::move(temp_block.getByPosition(result).column);
}
bool hasInformationAboutMonotonicity() const override
{
return true;
}
Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override
{
return { true, true, true };
}
private:
const Context & context;
mutable IntervalKind::Kind datepart_kind = IntervalKind::Kind::Second;
};
void registerFunctionDateTrunc(FunctionFactory & factory)
{
factory.registerFunction<FunctionDateTrunc>(FunctionFactory::CaseInsensitive);
/// Compatibility alias.
factory.registerAlias("dateTrunc", FunctionDateTrunc::name);
}
}

View File

@ -65,6 +65,7 @@ void registerFunctionSubtractYears(FunctionFactory &);
void registerFunctionDateDiff(FunctionFactory &);
void registerFunctionToTimeZone(FunctionFactory &);
void registerFunctionFormatDateTime(FunctionFactory &);
void registerFunctionDateTrunc(FunctionFactory &);
void registerFunctionsDateTime(FunctionFactory & factory)
{
@ -130,6 +131,7 @@ void registerFunctionsDateTime(FunctionFactory & factory)
registerFunctionDateDiff(factory);
registerFunctionToTimeZone(factory);
registerFunctionFormatDateTime(factory);
registerFunctionDateTrunc(factory);
}
}

View File

@ -136,6 +136,7 @@ SRCS(
currentDatabase.cpp
currentUser.cpp
dateDiff.cpp
date_trunc.cpp
defaultValueOfArgumentType.cpp
defaultValueOfTypeName.cpp
demange.cpp

View File

@ -124,4 +124,5 @@
<query>SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDate('2017-01-01') + number % 1000 + rand() % 10 AS t, {date_transform}(t))</query>
<query>SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, {binary_function}(t, 1))</query>
<query>SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, toStartOfInterval(t, INTERVAL 1 month))</query>
<query>SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, date_trunc('month', t))</query>
</test>

View File

@ -37,4 +37,5 @@
<query>SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDate('2017-01-01') + number % 1000 + rand() % 10 AS t, {date_transform}(t))</query>
<query>SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, {binary_function}(t, 1))</query>
<query>SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, toStartOfInterval(t, INTERVAL 1 month))</query>
<query>SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, date_trunc('month', t))</query>
</test>

View File

@ -226,6 +226,7 @@ toDate
2014-09-30
2014-10-01
2014-09-30
2014-09-30
toString
2015-07-15 13:30:00
2015-07-15 12:30:00
@ -243,3 +244,57 @@ toUnixTimestamp
1426415400
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-12-31 00:00:00
2020-01-01 00:00:00
2020-01-02 00:00:00
2019-12-31 20:00:00
2020-01-01 12:00:00
2020-01-02 05:00:00
2019-12-31 20:11:00
2020-01-01 12:11:00
2020-01-02 05:11:00
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-12-31 00:00:00
2020-01-01 00:00:00
2020-01-02 00:00:00
2019-12-31 20:00:00
2020-01-01 12:00:00
2020-01-02 05:00:00
2019-12-31 20:11:00
2020-01-01 12:11:00
2020-01-02 05:11:00
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

View File

@ -321,6 +321,8 @@ SELECT toDate(1412106600, 'Europe/London');
SELECT toDate(1412106600, 'Asia/Tokyo');
SELECT toDate(1412106600, 'Pacific/Pitcairn');
SELECT toDate(16343);
/* toString */
SELECT 'toString';
@ -344,3 +346,63 @@ SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Europe/Paris'), 'Europe
SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Europe/London'), 'Europe/London');
SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Asia/Tokyo'), 'Asia/Tokyo');
SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Pacific/Pitcairn'), 'Pacific/Pitcairn');
/* date_trunc */
SELECT 'date_trunc';
SELECT date_trunc('year', toDateTime('2020-01-01 04:11:22', 'Europe/London'), 'America/Vancouver');
SELECT date_trunc('year', toDateTime('2020-01-01 12:11:22', 'Europe/London'), 'Europe/London');
SELECT date_trunc('year', toDateTime('2020-01-01 20:11:22', 'Europe/London'), 'Asia/Tokyo');
SELECT date_trunc('quarter', toDateTime('2020-01-01 04:11:22', 'Europe/London'), 'America/Vancouver');
SELECT date_trunc('quarter', toDateTime('2020-01-01 12:11:22', 'Europe/London'), 'Europe/London');
SELECT date_trunc('quarter', toDateTime('2020-01-01 20:11:22', 'Europe/London'), 'Asia/Tokyo');
SELECT date_trunc('month', toDateTime('2020-01-01 04:11:22', 'Europe/London'), 'America/Vancouver');
SELECT date_trunc('month', toDateTime('2020-01-01 12:11:22', 'Europe/London'), 'Europe/London');
SELECT date_trunc('month', toDateTime('2020-01-01 20:11:22', 'Europe/London'), 'Asia/Tokyo');
SELECT date_trunc('week', toDateTime('2020-01-01 04:11:22', 'Europe/London'), 'America/Vancouver');
SELECT date_trunc('week', toDateTime('2020-01-01 12:11:22', 'Europe/London'), 'Europe/London');
SELECT date_trunc('week', toDateTime('2020-01-01 20:11:22', 'Europe/London'), 'Asia/Tokyo');
SELECT date_trunc('day', toDateTime('2020-01-01 04:11:22', 'Europe/London'), 'America/Vancouver');
SELECT date_trunc('day', toDateTime('2020-01-01 12:11:22', 'Europe/London'), 'Europe/London');
SELECT date_trunc('day', toDateTime('2020-01-01 20:11:22', 'Europe/London'), 'Asia/Tokyo');
SELECT date_trunc('hour', toDateTime('2020-01-01 04:11:22', 'Europe/London'), 'America/Vancouver');
SELECT date_trunc('hour', toDateTime('2020-01-01 12:11:22', 'Europe/London'), 'Europe/London');
SELECT date_trunc('hour', toDateTime('2020-01-01 20:11:22', 'Europe/London'), 'Asia/Tokyo');
SELECT date_trunc('minute', toDateTime('2020-01-01 04:11:22', 'Europe/London'), 'America/Vancouver');
SELECT date_trunc('minute', toDateTime('2020-01-01 12:11:22', 'Europe/London'), 'Europe/London');
SELECT date_trunc('minute', toDateTime('2020-01-01 20:11:22', 'Europe/London'), 'Asia/Tokyo');
SELECT date_trunc('second', toDateTime('2020-01-01 04:11:22', 'Europe/London'), 'America/Vancouver');
SELECT date_trunc('second', toDateTime('2020-01-01 12:11:22', 'Europe/London'), 'Europe/London');
SELECT date_trunc('second', toDateTime('2020-01-01 20:11:22', 'Europe/London'), 'Asia/Tokyo');
SELECT date_trunc('year', toDateTime64('2020-01-01 04:11:22.123', 3, 'Europe/London'), 'America/Vancouver');
SELECT date_trunc('year', toDateTime64('2020-01-01 12:11:22.123', 3, 'Europe/London'), 'Europe/London');
SELECT date_trunc('year', toDateTime64('2020-01-01 20:11:22.123', 3, 'Europe/London'), 'Asia/Tokyo');
SELECT date_trunc('quarter', toDateTime64('2020-01-01 04:11:22.123', 3, 'Europe/London'), 'America/Vancouver');
SELECT date_trunc('quarter', toDateTime64('2020-01-01 12:11:22.123', 3, 'Europe/London'), 'Europe/London');
SELECT date_trunc('quarter', toDateTime64('2020-01-01 20:11:22.123', 3, 'Europe/London'), 'Asia/Tokyo');
SELECT date_trunc('month', toDateTime64('2020-01-01 04:11:22.123', 3, 'Europe/London'), 'America/Vancouver');
SELECT date_trunc('month', toDateTime64('2020-01-01 12:11:22.123', 3, 'Europe/London'), 'Europe/London');
SELECT date_trunc('month', toDateTime64('2020-01-01 20:11:22.123', 3, 'Europe/London'), 'Asia/Tokyo');
SELECT date_trunc('week', toDateTime64('2020-01-01 04:11:22.123', 3, 'Europe/London'), 'America/Vancouver');
SELECT date_trunc('week', toDateTime64('2020-01-01 12:11:22.123', 3, 'Europe/London'), 'Europe/London');
SELECT date_trunc('week', toDateTime64('2020-01-01 20:11:22.123', 3, 'Europe/London'), 'Asia/Tokyo');
SELECT date_trunc('day', toDateTime64('2020-01-01 04:11:22.123', 3, 'Europe/London'), 'America/Vancouver');
SELECT date_trunc('day', toDateTime64('2020-01-01 12:11:22.123', 3, 'Europe/London'), 'Europe/London');
SELECT date_trunc('day', toDateTime64('2020-01-01 20:11:22.123', 3, 'Europe/London'), 'Asia/Tokyo');
SELECT date_trunc('hour', toDateTime64('2020-01-01 04:11:22.123', 3, 'Europe/London'), 'America/Vancouver');
SELECT date_trunc('hour', toDateTime64('2020-01-01 12:11:22.123', 3, 'Europe/London'), 'Europe/London');
SELECT date_trunc('hour', toDateTime64('2020-01-01 20:11:22.123', 3, 'Europe/London'), 'Asia/Tokyo');
SELECT date_trunc('minute', toDateTime64('2020-01-01 04:11:22.123', 3, 'Europe/London'), 'America/Vancouver');
SELECT date_trunc('minute', toDateTime64('2020-01-01 12:11:22.123', 3, 'Europe/London'), 'Europe/London');
SELECT date_trunc('minute', toDateTime64('2020-01-01 20:11:22.123', 3, 'Europe/London'), 'Asia/Tokyo');
SELECT date_trunc('second', toDateTime64('2020-01-01 04:11:22.123', 3, 'Europe/London'), 'America/Vancouver');
SELECT date_trunc('second', toDateTime64('2020-01-01 12:11:22.123', 3, 'Europe/London'), 'Europe/London');
SELECT date_trunc('second', toDateTime64('2020-01-01 20:11:22.123', 3, 'Europe/London'), 'Asia/Tokyo');
SELECT date_trunc('year', toDate('2020-01-01', 'Europe/London'));
SELECT date_trunc('quarter', toDate('2020-01-01', 'Europe/London'));
SELECT date_trunc('month', toDate('2020-01-01', 'Europe/London'));
SELECT date_trunc('week', toDate('2020-01-01', 'Europe/London'));
SELECT date_trunc('day', toDate('2020-01-01', 'Europe/London'), 'America/Vancouver');

View File

@ -35,6 +35,10 @@ toStartOfInterval(N, INTERVAL 1 year)
toStartOfInterval(N, INTERVAL 1 month)
toStartOfInterval(N, INTERVAL 1 day)
toStartOfInterval(N, INTERVAL 15 minute)
date_trunc('year', N)
date_trunc('month', N)
date_trunc('day', N)
date_trunc('minute', N)
toTime(N)
toRelativeYearNum(N)
toRelativeQuarterNum(N)

View File

@ -143,6 +143,27 @@ Code: 43: Illegal type Date of argument for function toStartOfInterval.
"DateTime('Europe/Minsk')","2019-09-16 19:15:00"
"DateTime('Europe/Minsk')","2019-09-16 19:15:00"
------------------------------------------
SELECT date_trunc(\'year\', N)
"Date","2019-01-01"
"Date","2019-01-01"
"Date","2019-01-01"
------------------------------------------
SELECT date_trunc(\'month\', N)
"Date","2019-09-01"
"Date","2019-09-01"
"Date","2019-09-01"
------------------------------------------
SELECT date_trunc(\'day\', N)
"DateTime","2019-09-16 00:00:00"
"DateTime('Europe/Minsk')","2019-09-16 00:00:00"
"DateTime('Europe/Minsk')","2019-09-16 00:00:00"
------------------------------------------
SELECT date_trunc(\'minute\', N)
Code: 43: Illegal type Date of argument for function date_trunc.
"DateTime('Europe/Minsk')","2019-09-16 19:20:00"
"DateTime('Europe/Minsk')","2019-09-16 19:20:00"
------------------------------------------
SELECT toTime(N)
Code: 43: Illegal type Date of argument for function toTime.