diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md
index 0364a610404..3f522eeb164 100644
--- a/docs/en/sql-reference/functions/date-time-functions.md
+++ b/docs/en/sql-reference/functions/date-time-functions.md
@@ -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.
diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp
index 29f8faa810c..8529643ee70 100644
--- a/src/Functions/toStartOfInterval.cpp
+++ b/src/Functions/toStartOfInterval.cpp
@@ -1,4 +1,5 @@
#include
+#include
#include
#include
#include
@@ -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
+struct Transform;
+
+template <>
+struct Transform
+{
+ 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
+{
+ 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
+{
+ 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
+{
+ 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
+{
+ static UInt32 execute(UInt16 d, Int64 days, const DateLUTImpl & time_zone, Int64)
+ {
+ return static_cast(time_zone.toStartOfDayInterval(ExtendedDayNum(d), days));
+ }
+
+ static UInt32 execute(Int32 d, Int64 days, const DateLUTImpl & time_zone, Int64)
+ {
+ return static_cast(time_zone.toStartOfDayInterval(ExtendedDayNum(d), days));
+ }
+
+ static UInt32 execute(UInt32 t, Int64 days, const DateLUTImpl & time_zone, Int64)
+ {
+ return static_cast(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
+{
+ 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
+{
+ 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
+{
+ 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
+{
+ 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(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(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
+{
+ 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(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(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
+{
+ 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(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(time_column_col);
auto scale = assert_cast(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(time_column_col);
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast(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(time_column_col);
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast(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(time_column_col);
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone);
@@ -292,7 +590,7 @@ private:
}
template
- 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(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_multiplier));
+ result_data[i] = static_cast(Transform::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(ToStartOfInterval::execute(td, num_units, time_zone, scale_multiplier));
+ result_data[i] = static_cast(Transform::execute(td, num_units, time_zone, scale_multiplier));
result_data[i] += scale_multiplier == 10 ? od : od / scale_multiplier;
}
diff --git a/tests/queries/0_stateless/02916_to_start_of_interval_origin_overload.reference b/tests/queries/0_stateless/02916_to_start_of_interval_origin_overload.reference
new file mode 100644
index 00000000000..7213925fb64
--- /dev/null
+++ b/tests/queries/0_stateless/02916_to_start_of_interval_origin_overload.reference
@@ -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
diff --git a/tests/queries/0_stateless/02916_to_start_of_interval_origin_overload.sql b/tests/queries/0_stateless/02916_to_start_of_interval_origin_overload.sql
new file mode 100644
index 00000000000..ce4c8f87811
--- /dev/null
+++ b/tests/queries/0_stateless/02916_to_start_of_interval_origin_overload.sql
@@ -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 }