style check, docs, tests and modified logic

This commit is contained in:
yariks5s 2023-11-17 17:35:04 +00:00
parent 188c469318
commit 6605a375b6
4 changed files with 335 additions and 5 deletions

View File

@ -590,7 +590,7 @@ Rounds down a date with time to the start of the ten-minute interval.
Rounds down the date with time to the start of the fifteen-minute interval.
## toStartOfInterval(time_or_data, INTERVAL x unit \[, time_zone\])
## toStartOfInterval(time_or_data, INTERVAL x unit \[, origin_time_or_data\] \[, time_zone\])
This is a generalization of other functions named `toStartOf*`. For example,
`toStartOfInterval(t, INTERVAL 1 year)` returns the same as `toStartOfYear(t)`,
@ -598,6 +598,11 @@ This is a generalization of other functions named `toStartOf*`. For example,
`toStartOfInterval(t, INTERVAL 1 day)` returns the same as `toStartOfDay(t)`,
`toStartOfInterval(t, INTERVAL 15 minute)` returns the same as `toStartOfFifteenMinutes(t)` etc.
Also it has an overload including `origin_time_or_data` column which emulates TimescaleDB's `time_bucket()` function, respectively PostgreSQL's `date_bin()` function:
``` SQL
SELECT toStartOfInterval(toDateTime('2023-01-01 14:45:00'), toIntervalMinute(1), toDateTime('2023-01-01 14:35:30'));
```
## toTime
Converts a date with time to a certain fixed date, while preserving the time.

View File

@ -1,4 +1,5 @@
#include <base/arithmeticOverflow.h>
#include <Common/Exception.h>
#include <Common/DateLUTImpl.h>
#include <DataTypes/DataTypesDecimal.h>
#include <Functions/FunctionHelpers.h>
@ -23,11 +24,292 @@ namespace ErrorCodes
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int BAD_ARGUMENTS;
}
namespace
{
constexpr auto function_name = "toStartOfInterval";
template <IntervalKind::Kind unit>
struct Transform;
template <>
struct Transform<IntervalKind::Year>
{
static UInt16 execute(UInt16 d, Int64 years, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfYearInterval(DayNum(d), years);
}
static UInt16 execute(Int32 d, Int64 years, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfYearInterval(ExtendedDayNum(d), years);
}
static UInt16 execute(UInt32 t, Int64 years, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfYearInterval(time_zone.toDayNum(t), years);
}
static UInt16 execute(Int64 t, Int64 years, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfYearInterval(time_zone.toDayNum(t / scale_multiplier), years);
}
};
template <>
struct Transform<IntervalKind::Quarter>
{
static UInt16 execute(UInt16 d, Int64 quarters, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfQuarterInterval(DayNum(d), quarters);
}
static UInt16 execute(Int32 d, Int64 quarters, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfQuarterInterval(ExtendedDayNum(d), quarters);
}
static UInt16 execute(UInt32 t, Int64 quarters, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t), quarters);
}
static UInt16 execute(Int64 t, Int64 quarters, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t / scale_multiplier), quarters);
}
};
template <>
struct Transform<IntervalKind::Month>
{
static UInt16 execute(UInt16 d, Int64 months, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfMonthInterval(DayNum(d), months);
}
static UInt16 execute(Int32 d, Int64 months, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfMonthInterval(ExtendedDayNum(d), months);
}
static UInt16 execute(UInt32 t, Int64 months, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t), months);
}
static UInt16 execute(Int64 t, Int64 months, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t / scale_multiplier), months);
}
};
template <>
struct Transform<IntervalKind::Week>
{
static UInt16 execute(UInt16 d, Int64 weeks, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfWeekInterval(DayNum(d), weeks);
}
static UInt16 execute(Int32 d, Int64 weeks, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfWeekInterval(ExtendedDayNum(d), weeks);
}
static UInt16 execute(UInt32 t, Int64 weeks, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t), weeks);
}
static UInt16 execute(Int64 t, Int64 weeks, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t / scale_multiplier), weeks);
}
};
template <>
struct Transform<IntervalKind::Day>
{
static UInt32 execute(UInt16 d, Int64 days, const DateLUTImpl & time_zone, Int64)
{
return static_cast<UInt32>(time_zone.toStartOfDayInterval(ExtendedDayNum(d), days));
}
static UInt32 execute(Int32 d, Int64 days, const DateLUTImpl & time_zone, Int64)
{
return static_cast<UInt32>(time_zone.toStartOfDayInterval(ExtendedDayNum(d), days));
}
static UInt32 execute(UInt32 t, Int64 days, const DateLUTImpl & time_zone, Int64)
{
return static_cast<UInt32>(time_zone.toStartOfDayInterval(time_zone.toDayNum(t), days));
}
static Int64 execute(Int64 t, Int64 days, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfDayInterval(time_zone.toDayNum(t / scale_multiplier), days);
}
};
template <>
struct Transform<IntervalKind::Hour>
{
static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); }
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); }
static UInt32 execute(UInt32 t, Int64 hours, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfHourInterval(t, hours);
}
static Int64 execute(Int64 t, Int64 hours, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfHourInterval(t / scale_multiplier, hours);
}
};
template <>
struct Transform<IntervalKind::Minute>
{
static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); }
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); }
static UInt32 execute(UInt32 t, Int64 minutes, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfMinuteInterval(t, minutes);
}
static Int64 execute(Int64 t, Int64 minutes, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfMinuteInterval(t / scale_multiplier, minutes);
}
};
template <>
struct Transform<IntervalKind::Second>
{
static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); }
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); }
static UInt32 execute(UInt32 t, Int64 seconds, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfSecondInterval(t, seconds);
}
static Int64 execute(Int64 t, Int64 seconds, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfSecondInterval(t / scale_multiplier, seconds);
}
};
template <>
struct Transform<IntervalKind::Millisecond>
{
static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); }
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); }
static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64) { throwDateTimeIsNotSupported(function_name); }
static Int64 execute(Int64 t, Int64 milliseconds, const DateLUTImpl &, Int64 scale_multiplier)
{
if (scale_multiplier < 1000)
{
Int64 t_milliseconds = 0;
if (common::mulOverflow(t, static_cast<Int64>(1000) / scale_multiplier, t_milliseconds))
throw DB::Exception(ErrorCodes::DECIMAL_OVERFLOW, "Numeric overflow");
if (likely(t >= 0))
return t_milliseconds / milliseconds * milliseconds;
else
return ((t_milliseconds + 1) / milliseconds - 1) * milliseconds;
}
else if (scale_multiplier > 1000)
{
Int64 scale_diff = scale_multiplier / static_cast<Int64>(1000);
if (likely(t >= 0))
return t / milliseconds / scale_diff * milliseconds;
else
return ((t + 1) / milliseconds / scale_diff - 1) * milliseconds;
}
else
if (likely(t >= 0))
return t / milliseconds * milliseconds;
else
return ((t + 1) / milliseconds - 1) * milliseconds;
}
};
template <>
struct Transform<IntervalKind::Microsecond>
{
static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); }
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); }
static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64) { throwDateTimeIsNotSupported(function_name); }
static Int64 execute(Int64 t, Int64 microseconds, const DateLUTImpl &, Int64 scale_multiplier)
{
if (scale_multiplier < 1000000)
{
Int64 t_microseconds = 0;
if (common::mulOverflow(t, static_cast<Int64>(1000000) / scale_multiplier, t_microseconds))
throw DB::Exception(ErrorCodes::DECIMAL_OVERFLOW, "Numeric overflow");
if (likely(t >= 0))
return t_microseconds / microseconds * microseconds;
else
return ((t_microseconds + 1) / microseconds - 1) * microseconds;
}
else if (scale_multiplier > 1000000)
{
Int64 scale_diff = scale_multiplier / static_cast<Int64>(1000000);
if (likely(t >= 0))
return t / microseconds / scale_diff * microseconds;
else
return ((t + 1) / microseconds / scale_diff - 1) * microseconds;
}
else
if (likely(t >= 0))
return t / microseconds * microseconds;
else
return ((t + 1) / microseconds - 1) * microseconds;
}
};
template <>
struct Transform<IntervalKind::Nanosecond>
{
static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); }
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); }
static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64) { throwDateTimeIsNotSupported(function_name); }
static Int64 execute(Int64 t, Int64 nanoseconds, const DateLUTImpl &, Int64 scale_multiplier)
{
if (scale_multiplier < 1000000000)
{
Int64 t_nanoseconds = 0;
if (common::mulOverflow(t, (static_cast<Int64>(1000000000) / scale_multiplier), t_nanoseconds))
throw DB::Exception(ErrorCodes::DECIMAL_OVERFLOW, "Numeric overflow");
if (likely(t >= 0))
return t_nanoseconds / nanoseconds * nanoseconds;
else
return ((t_nanoseconds + 1) / nanoseconds - 1) * nanoseconds;
}
else
if (likely(t >= 0))
return t / nanoseconds * nanoseconds;
else
return ((t + 1) / nanoseconds - 1) * nanoseconds;
}
};
class FunctionToStartOfInterval : public IFunction
{
@ -218,6 +500,10 @@ private:
if (isDateTime64(time_column_type))
{
if (origin_column.column != nullptr)
if (!isDateTime64(origin_column.type.get()))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName());
const auto * time_column_vec = checkAndGetColumn<ColumnDateTime64>(time_column_col);
auto scale = assert_cast<const DataTypeDateTime64 &>(time_column_type).getScale();
@ -226,18 +512,30 @@ private:
}
else if (isDateTime(time_column_type))
{
if (origin_column.column != nullptr)
if (!isDateTime(origin_column.type.get()))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName());
const auto * time_column_vec = checkAndGetColumn<ColumnDateTime>(time_column_col);
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDateTime &>(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone);
}
else if (isDate(time_column_type))
{
if (origin_column.column != nullptr)
if (!isDate(origin_column.type.get()))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName());
const auto * time_column_vec = checkAndGetColumn<ColumnDate>(time_column_col);
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDate &>(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone);
}
else if (isDate32(time_column_type))
{
if (origin_column.column != nullptr)
if (!isDate32(origin_column.type.get()))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName());
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, origin_column, result_type, time_zone);
@ -292,7 +590,7 @@ private:
}
template <typename TimeDataType, typename ToDataType, IntervalKind::Kind unit, typename ColumnType>
ColumnPtr execute(const TimeDataType &, const ColumnType & time_column_type, Int64 num_units, [[maybe_unused]] const ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale) const
ColumnPtr execute(const TimeDataType &, const ColumnType & time_column_type, Int64 num_units, const ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale) const
{
using ToColumnType = typename ToDataType::ColumnType;
using ToFieldType = typename ToDataType::FieldType;
@ -309,18 +607,18 @@ private:
if (origin_column.column == nullptr)
for (size_t i = 0; i != size; ++i)
result_data[i] = static_cast<ToFieldType>(ToStartOfInterval<unit>::execute(time_data[i], num_units, time_zone, scale_multiplier));
result_data[i] = static_cast<ToFieldType>(Transform<unit>::execute(time_data[i], num_units, time_zone, scale_multiplier));
else
{
UInt64 od = origin_column.column->get64(0);
for (size_t i = 0; i != size; ++i)
{
auto td = time_data[i];
if (od > size_t(td))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The origin must be before the end date/datetime");
td -= od;
result_data[i] = static_cast<ToFieldType>(ToStartOfInterval<unit>::execute(td, num_units, time_zone, scale_multiplier));
result_data[i] = static_cast<ToFieldType>(Transform<unit>::execute(td, num_units, time_zone, scale_multiplier));
result_data[i] += scale_multiplier == 10 ? od : od / scale_multiplier;
}

View File

@ -0,0 +1,15 @@
2023-03-01 16:55:00
2023-02-01 16:55:00
2023-03-01 16:55:00
2023-02-01 16:55:00
2023-03-01 16:55:00
2023-03-01 16:55:00
2023-02-01 16:55:00
2023-03-01 16:55:00
2023-02-01 16:55:00
2023-03-01 16:55:00
2023-01-02 15:44:30
2023-01-02 15:44:30
2023-01-02 14:45:30
2023-01-02 14:45:30
2023-01-02

View File

@ -0,0 +1,12 @@
set session_timezone = 'UTC';
SELECT toStartOfInterval(number % 2 == 0 ? toDateTime64('2023-03-01 15:55:00', 2) : toDateTime64('2023-02-01 15:55:00', 2), toIntervalMinute(1), toDateTime64('2023-01-01 13:55:00', 2), 'Europe/Amsterdam') from numbers(5);
SELECT toStartOfInterval(number % 2 == 0 ? toDateTime('2023-03-01 15:55:00') : toDateTime('2023-02-01 15:55:00'), toIntervalMinute(1), toDateTime('2023-01-01 13:55:00'), 'Europe/Amsterdam') from numbers(5);
SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalHour(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam');
SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50', 2), toIntervalHour(1), toDateTime64('2023-01-02 14:44:30', 2), 'Europe/Amsterdam');
SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50', 2), toIntervalMinute(1), toDateTime64('2023-01-02 14:44:30', 2));
SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), toDateTime('2023-01-02 14:44:30'));
SELECT toStartOfInterval(toDate('2023-01-02 14:45:50'), toIntervalWeek(1), toDate('2023-01-02 14:44:30'));
SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), toDateTime64('2023-01-02 14:44:30', 2)); -- { serverError 43 }
SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50', 2), toIntervalMinute(1), toDate('2023-01-02 14:44:30')); -- { serverError 43 }
SELECT toStartOfInterval(toDateTime('2023-01-02 14:42:50'), toIntervalMinute(1), toDateTime('2023-01-02 14:44:30')); -- { serverError 36 }
SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), number % 2 == 0 ? toDateTime('2023-02-01 15:55:00') : toDateTime('2023-01-01 15:55:00'), 'Europe/Amsterdam') from numbers(1); -- { serverError 44 }