diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index b65fb3d7e95..31f7c31f04c 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1617,8 +1617,19 @@ The calculation is performed relative to specific points in time: If unit `WEEK` was specified, `toStartOfInterval` assumes that weeks start on Monday. Note that this behavior is different from that of function `toStartOfWeek` in which weeks start by default on Sunday. -**See Also** +**Syntax** +```sql +toStartOfInterval(value, INTERVAL x unit[, time_zone]) +toStartOfInterval(value, INTERVAL x unit[, origin[, time_zone]]) +``` + +The second overload emulates TimescaleDB's `time_bucket()` function, respectively PostgreSQL's `date_bin()` function, e.g. + +``` SQL +SELECT toStartOfInterval(toDateTime('2023-01-01 14:45:00'), INTERVAL 1 MINUTE, toDateTime('2023-01-01 14:35:30')); +``` +**See Also** - [date_trunc](#date_trunc) ## toTime @@ -3884,19 +3895,29 @@ Result: └───────────────────────────────────────────────────────────────────────┘ ``` -## timeSlots(StartTime, Duration,\[, Size\]) +## timeSlots For a time interval starting at ‘StartTime’ and continuing for ‘Duration’ seconds, it returns an array of moments in time, consisting of points from this interval rounded down to the ‘Size’ in seconds. ‘Size’ is an optional parameter set to 1800 (30 minutes) by default. This is necessary, for example, when searching for pageviews in the corresponding session. Accepts DateTime and DateTime64 as ’StartTime’ argument. For DateTime, ’Duration’ and ’Size’ arguments must be `UInt32`. For ’DateTime64’ they must be `Decimal64`. Returns an array of DateTime/DateTime64 (return type matches the type of ’StartTime’). For DateTime64, the return value's scale can differ from the scale of ’StartTime’ --- the highest scale among all given arguments is taken. -Example: +**Syntax** + +```sql +timeSlots(StartTime, Duration,\[, Size\]) +``` + +**Example** + ```sql SELECT timeSlots(toDateTime('2012-01-01 12:20:00'), toUInt32(600)); SELECT timeSlots(toDateTime('1980-12-12 21:01:02', 'UTC'), toUInt32(600), 299); SELECT timeSlots(toDateTime64('1980-12-12 21:01:02.1234', 4, 'UTC'), toDecimal64(600.1, 1), toDecimal64(299, 0)); ``` + +Result: + ``` text ┌─timeSlots(toDateTime('2012-01-01 12:20:00'), toUInt32(600))─┐ │ ['2012-01-01 12:00:00','2012-01-01 12:30:00'] │ diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index dfb4b76e5e2..d6d533f16ed 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -492,7 +492,7 @@ struct ToStartOfInterval { throwDateTimeIsNotSupported(TO_START_OF_INTERVAL_NAME); } - static Int64 execute(Int64 t, Int64 nanoseconds, const DateLUTImpl &, Int64 scale_multiplier) + static Int64 execute(Int64 t, Int64 nanoseconds, const DateLUTImpl &, Int64 scale_multiplier, Int64 /*origin*/ = 0) { if (scale_multiplier < 1000000000) { @@ -527,7 +527,7 @@ struct ToStartOfInterval { throwDateTimeIsNotSupported(TO_START_OF_INTERVAL_NAME); } - static Int64 execute(Int64 t, Int64 microseconds, const DateLUTImpl &, Int64 scale_multiplier) + static Int64 execute(Int64 t, Int64 microseconds, const DateLUTImpl &, Int64 scale_multiplier, Int64 /*origin*/ = 0) { if (scale_multiplier < 1000000) { @@ -570,7 +570,7 @@ struct ToStartOfInterval { throwDateTimeIsNotSupported(TO_START_OF_INTERVAL_NAME); } - static Int64 execute(Int64 t, Int64 milliseconds, const DateLUTImpl &, Int64 scale_multiplier) + static Int64 execute(Int64 t, Int64 milliseconds, const DateLUTImpl &, Int64 scale_multiplier, Int64 /*origin*/ = 0) { if (scale_multiplier < 1000) { @@ -613,7 +613,7 @@ struct ToStartOfInterval { return time_zone.toStartOfSecondInterval(t, seconds); } - static Int64 execute(Int64 t, Int64 seconds, const DateLUTImpl & time_zone, Int64 scale_multiplier) + static Int64 execute(Int64 t, Int64 seconds, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 /*origin*/ = 0) { return time_zone.toStartOfSecondInterval(t / scale_multiplier, seconds); } @@ -634,7 +634,7 @@ struct ToStartOfInterval { return time_zone.toStartOfMinuteInterval(t, minutes); } - static Int64 execute(Int64 t, Int64 minutes, const DateLUTImpl & time_zone, Int64 scale_multiplier) + static Int64 execute(Int64 t, Int64 minutes, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 /*origin*/ = 0) { return time_zone.toStartOfMinuteInterval(t / scale_multiplier, minutes); } @@ -655,7 +655,7 @@ struct ToStartOfInterval { return time_zone.toStartOfHourInterval(t, hours); } - static Int64 execute(Int64 t, Int64 hours, const DateLUTImpl & time_zone, Int64 scale_multiplier) + static Int64 execute(Int64 t, Int64 hours, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 /*origin*/ = 0) { return time_zone.toStartOfHourInterval(t / scale_multiplier, hours); } @@ -676,7 +676,7 @@ struct ToStartOfInterval { 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) + static Int64 execute(Int64 t, Int64 days, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 /*origin*/ = 0) { return time_zone.toStartOfDayInterval(time_zone.toDayNum(t / scale_multiplier), days); } @@ -697,9 +697,13 @@ struct ToStartOfInterval { return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t), weeks); } - static UInt16 execute(Int64 t, Int64 weeks, const DateLUTImpl & time_zone, Int64 scale_multiplier) + static Int64 execute(Int64 t, Int64 weeks, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 origin = 0) { - return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t / scale_multiplier), weeks); + if (origin == 0) + return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t / scale_multiplier), weeks); + else + return ToStartOfInterval::execute(t, weeks * 7, time_zone, scale_multiplier, origin); + } }; @@ -718,9 +722,23 @@ struct ToStartOfInterval { return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t), months); } - static UInt16 execute(Int64 t, Int64 months, const DateLUTImpl & time_zone, Int64 scale_multiplier) + static Int64 execute(Int64 t, Int64 months, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 origin = 0) { - return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t / scale_multiplier), months); + const Int64 scaled_time = t / scale_multiplier; + if (origin == 0) + return time_zone.toStartOfMonthInterval(time_zone.toDayNum(scaled_time), months); + else + { + const Int64 scaled_origin = origin / scale_multiplier; + const Int64 days = time_zone.toDayOfMonth(scaled_time + scaled_origin) - time_zone.toDayOfMonth(scaled_origin); + Int64 months_to_add = time_zone.toMonth(scaled_time + scaled_origin) - time_zone.toMonth(scaled_origin); + const Int64 years = time_zone.toYear(scaled_time + scaled_origin) - time_zone.toYear(scaled_origin); + months_to_add = days < 0 ? months_to_add - 1 : months_to_add; + months_to_add += years * 12; + Int64 month_multiplier = (months_to_add / months) * months; + + return (time_zone.addMonths(time_zone.toDate(scaled_origin), month_multiplier) - time_zone.toDate(scaled_origin)); + } } }; @@ -739,9 +757,12 @@ struct ToStartOfInterval { return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t), quarters); } - static UInt16 execute(Int64 t, Int64 quarters, const DateLUTImpl & time_zone, Int64 scale_multiplier) + static Int64 execute(Int64 t, Int64 quarters, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 origin = 0) { - return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t / scale_multiplier), quarters); + if (origin == 0) + return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t / scale_multiplier), quarters); + else + return ToStartOfInterval::execute(t, quarters * 3, time_zone, scale_multiplier, origin); } }; @@ -760,9 +781,12 @@ struct ToStartOfInterval { return time_zone.toStartOfYearInterval(time_zone.toDayNum(t), years); } - static UInt16 execute(Int64 t, Int64 years, const DateLUTImpl & time_zone, Int64 scale_multiplier) + static Int64 execute(Int64 t, Int64 years, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 origin = 0) { - return time_zone.toStartOfYearInterval(time_zone.toDayNum(t / scale_multiplier), years); + if (origin == 0) + return time_zone.toStartOfYearInterval(time_zone.toDayNum(t / scale_multiplier), years); + else + return ToStartOfInterval::execute(t, years * 12, time_zone, scale_multiplier, origin); } }; diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 709f5f86d80..6573fef6634 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -10,21 +10,31 @@ #include #include #include +#include namespace DB { namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ARGUMENT_OUT_OF_BOUND; + extern const int BAD_ARGUMENTS; extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int ARGUMENT_OUT_OF_BOUND; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } class FunctionToStartOfInterval : public IFunction { +private: + enum class Overload + { + Default, /// toStartOfInterval(time, interval) or toStartOfInterval(time, interval, timezone) + Origin /// toStartOfInterval(time, interval, origin) or toStartOfInterval(time, interval, origin, timezone) + }; + mutable Overload overload; + public: static FunctionPtr create(ContextPtr) { return std::make_shared(); } @@ -34,7 +44,7 @@ public: size_t getNumberOfArguments() const override { return 0; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2, 3}; } bool hasInformationAboutMonotonicity() const override { return true; } Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override { return { .is_monotonic = true, .is_always_monotonic = true }; } @@ -72,6 +82,9 @@ public: "Illegal type {} of 2nd argument of function {}, expected a time interval", type_arg2->getName(), getName()); + overload = Overload::Default; + + /// Determine result type for default overload (no origin) switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case) { case IntervalKind::Kind::Nanosecond: @@ -97,13 +110,49 @@ public: auto check_third_argument = [&] { const DataTypePtr & type_arg3 = arguments[2].type; - if (!isString(type_arg3)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of 3rd argument of function {}, expected a constant timezone string", + if (isString(type_arg3)) + { + if (value_is_date && result_type == ResultType::Date) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "A timezone argument of function {} with interval type {} is allowed only when the 1st argument has the type DateTime or DateTime64", + getName(), interval_type->getKind().toString()); + } + else if (isDateOrDate32OrDateTimeOrDateTime64(type_arg3)) + { + overload = Overload::Origin; + const DataTypePtr & type_arg1 = arguments[0].type; + if (isDate(type_arg1) && isDate(type_arg3)) + result_type = ResultType::Date; + else if (isDate32(type_arg1) && isDate32(type_arg3)) + result_type = ResultType::Date32; + else if (isDateTime(type_arg1) && isDateTime(type_arg3)) + result_type = ResultType::DateTime; + else if (isDateTime64(type_arg1) && isDateTime64(type_arg3)) + result_type = ResultType::DateTime64; + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName()); + } + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 3rd argument of function {}. " + "This argument is optional and must be a constant String with timezone name or a Date/Date32/DateTime/DateTime64 with a constant origin", type_arg3->getName(), getName()); - if (value_is_date && result_type == ResultType::Date) /// weird why this is && instead of || but too afraid to change it + }; + + auto check_fourth_argument = [&] + { + if (overload != Overload::Origin) /// sanity check + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 3rd argument of function {}. " + "The third argument must a Date/Date32/DateTime/DateTime64 with a constant origin", + arguments[2].type->getName(), getName()); + + const DataTypePtr & type_arg4 = arguments[3].type; + if (!isString(type_arg4)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 4th argument of function {}. " + "This argument is optional and must be a constant String with timezone name", + type_arg4->getName(), getName()); + if (value_is_date && result_type == ResultType::Date) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "The timezone argument of function {} with interval type {} is allowed only when the 1st argument has type DateTime or DateTimt64", + "A timezone argument of function {} with interval type {} is allowed only when the 1st argument has the type DateTime or DateTime64", getName(), interval_type->getKind().toString()); }; @@ -118,10 +167,17 @@ public: check_second_argument(); check_third_argument(); } + else if (arguments.size() == 4) + { + check_first_argument(); + check_second_argument(); + check_third_argument(); + check_fourth_argument(); + } else { throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3", + "Number of arguments for function {} doesn't match: passed {}, must be 2, 3 or 4", getName(), arguments.size()); } @@ -132,10 +188,19 @@ public: case ResultType::Date32: return std::make_shared(); case ResultType::DateTime: - return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); + { + const size_t time_zone_arg_num = (overload == Overload::Default) ? 2 : 3; + return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, time_zone_arg_num, 0, false)); + } case ResultType::DateTime64: { UInt32 scale = 0; + if (isDateTime64(arguments[0].type) && overload == Overload::Origin) + { + scale = assert_cast(*arguments[0].type.get()).getScale(); + if (assert_cast(*arguments[2].type.get()).getScale() != scale) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same scale", getName()); + } if (interval_type->getKind() == IntervalKind::Kind::Nanosecond) scale = 9; else if (interval_type->getKind() == IntervalKind::Kind::Microsecond) @@ -143,69 +208,103 @@ public: else if (interval_type->getKind() == IntervalKind::Kind::Millisecond) scale = 3; - return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); + const size_t time_zone_arg_num = (overload == Overload::Default) ? 2 : 3; + return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, time_zone_arg_num, 0, false)); } } std::unreachable(); } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /* input_rows_count */) const override { const auto & time_column = arguments[0]; const auto & interval_column = arguments[1]; - const auto & time_zone = extractTimeZoneFromFunctionArguments(arguments, 2, 0); - auto result_column = dispatchForTimeColumn(time_column, interval_column, result_type, time_zone, input_rows_count); + + ColumnWithTypeAndName origin_column; + if (overload == Overload::Origin) + origin_column = arguments[2]; + + const size_t time_zone_arg_num = (overload == Overload::Default) ? 2 : 3; + const auto & time_zone = extractTimeZoneFromFunctionArguments(arguments, time_zone_arg_num, 0); + + ColumnPtr result_column; + if (isDate(result_type)) + result_column = dispatchForTimeColumn(time_column, interval_column, origin_column, result_type, time_zone); + else if (isDate32(result_type)) + result_column = dispatchForTimeColumn(time_column, interval_column, origin_column, result_type, time_zone); + else if (isDateTime(result_type)) + result_column = dispatchForTimeColumn(time_column, interval_column, origin_column, result_type, time_zone); + else if (isDateTime64(result_type)) + result_column = dispatchForTimeColumn(time_column, interval_column, origin_column, result_type, time_zone); return result_column; } private: + template ColumnPtr dispatchForTimeColumn( - const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, - const DataTypePtr & result_type, const DateLUTImpl & time_zone, - size_t input_rows_count) const + const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, const ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone) const { const auto & time_column_type = *time_column.type.get(); const auto & time_column_col = *time_column.column.get(); - if (isDateTime64(time_column_type)) - { - const auto * time_column_vec = checkAndGetColumn(&time_column_col); - auto scale = assert_cast(time_column_type).getScale(); - - if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone, input_rows_count, scale); - } - else if (isDateTime(time_column_type)) - { - 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, result_type, time_zone, input_rows_count); - } - else if (isDate(time_column_type)) + if (isDate(time_column_type)) { 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, result_type, time_zone, input_rows_count); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); } else if (isDate32(time_column_type)) { 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, result_type, time_zone, input_rows_count); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); + } + else if (isDateTime(time_column_type)) + { + 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 (isDateTime64(time_column_type)) + { + const auto * time_column_vec = checkAndGetColumn(&time_column_col); + auto scale = assert_cast(time_column_type).getScale(); + + if (time_column_vec) + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone, scale); } throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for 1st argument of function {}, expected a Date, Date32, DateTime or DateTime64", getName()); } - template + template ColumnPtr dispatchForIntervalColumn( - const TimeDataType & time_data_type, const TimeColumnType & time_column, const ColumnWithTypeAndName & interval_column, - const DataTypePtr & result_type, const DateLUTImpl & time_zone, size_t input_rows_count, UInt16 scale = 1) const + const TimeDataType & time_data_type, const TimeColumnType & time_column, const ColumnWithTypeAndName & interval_column, const ColumnWithTypeAndName & origin_column, + const DataTypePtr & result_type, const DateLUTImpl & time_zone, UInt16 scale = 1) const { const auto * interval_type = checkAndGetDataType(interval_column.type.get()); if (!interval_type) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column for 2nd argument of function {}, must be a time interval", getName()); + switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case) + { + case IntervalKind::Kind::Nanosecond: + case IntervalKind::Kind::Microsecond: + case IntervalKind::Kind::Millisecond: + if (isDateOrDate32(time_data_type) || isDateTime(time_data_type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal interval kind for argument data type {}", isDate(time_data_type) ? "Date" : "DateTime"); + break; + case IntervalKind::Kind::Second: + case IntervalKind::Kind::Minute: + case IntervalKind::Kind::Hour: + if (isDateOrDate32(time_data_type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal interval kind for argument data type Date"); + break; + default: + break; + } + const auto * interval_column_const_int64 = checkAndGetColumnConst(interval_column.column.get()); if (!interval_column_const_int64) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column for 2nd argument of function {}, must be a const time interval", getName()); @@ -217,51 +316,102 @@ private: switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case) { case IntervalKind::Kind::Nanosecond: - return execute(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Kind::Microsecond: - return execute(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Kind::Millisecond: - return execute(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Kind::Second: - return execute(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Kind::Minute: - return execute(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Kind::Hour: - return execute(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Kind::Day: - return execute(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Kind::Week: - return execute(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Kind::Month: - return execute(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Kind::Quarter: - return execute(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Kind::Year: - return execute(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); } std::unreachable(); } - template - ColumnPtr execute( - const TimeDataType &, const TimeColumnType & time_column_type, Int64 num_units, - const DataTypePtr & result_type, const DateLUTImpl & time_zone, size_t input_rows_count, UInt16 scale) const + template + ColumnPtr execute(const TimeDataType &, const TimeColumnType & time_column_type, Int64 num_units, const ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone, UInt16 scale) const { using ResultColumnType = typename ResultDataType::ColumnType; - using ResultFieldType = typename ResultDataType::FieldType; const auto & time_data = time_column_type.getData(); + size_t size = time_data.size(); auto result_col = result_type->createColumn(); auto * col_to = assert_cast(result_col.get()); auto & result_data = col_to->getData(); - result_data.resize(input_rows_count); + result_data.resize(size); Int64 scale_multiplier = DecimalUtils::scaleMultiplier(scale); - for (size_t i = 0; i != input_rows_count; ++i) - result_data[i] = static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_multiplier)); + if (origin_column.column) // Overload: Origin + { + const bool is_small_interval = (unit == IntervalKind::Kind::Nanosecond || unit == IntervalKind::Kind::Microsecond || unit == IntervalKind::Kind::Millisecond); + const bool is_result_date = isDateOrDate32(result_type); + + Int64 result_scale = scale_multiplier; + Int64 origin_scale = 1; + + if (isDateTime64(result_type)) /// We have origin scale only in case if arguments are DateTime64. + origin_scale = assert_cast(*origin_column.type).getScaleMultiplier(); + else if (!is_small_interval) /// In case of large interval and arguments are not DateTime64, we should not have scale in result. + result_scale = 1; + + if (is_small_interval) + result_scale = assert_cast(*result_type).getScaleMultiplier(); + + /// In case if we have a difference between time arguments and Interval, we need to calculate the difference between them + /// to get the right precision for the result. In case of large intervals, we should not have scale difference. + Int64 scale_diff = is_small_interval ? std::max(result_scale / origin_scale, origin_scale / result_scale) : 1; + + static constexpr Int64 SECONDS_PER_DAY = 86'400; + + UInt64 origin = origin_column.column->get64(0); + for (size_t i = 0; i != size; ++i) + { + UInt64 time_arg = time_data[i]; + if (origin > static_cast(time_arg)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The origin must be before the end date / date with time"); + + if (is_result_date) /// All internal calculations of ToStartOfInterval<...> expect arguments to be seconds or milli-, micro-, nanoseconds. + { + time_arg *= SECONDS_PER_DAY; + origin *= SECONDS_PER_DAY; + } + + Int64 offset = ToStartOfInterval::execute(time_arg - origin, num_units, time_zone, result_scale, origin); + + /// In case if arguments are DateTime64 with large interval, we should apply scale on it. + offset *= (!is_small_interval) ? result_scale : 1; + + if (is_result_date) /// Convert back to date after calculations. + { + offset /= SECONDS_PER_DAY; + origin /= SECONDS_PER_DAY; + } + + result_data[i] = 0; + result_data[i] += (result_scale < origin_scale) ? (origin + offset) / scale_diff : (origin + offset) * scale_diff; + } + } + else // Overload: Default + { + for (size_t i = 0; i != size; ++i) + result_data[i] = static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_multiplier)); + } return result_col; } diff --git a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference new file mode 100644 index 00000000000..f0afdf03963 --- /dev/null +++ b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference @@ -0,0 +1,59 @@ +-- Negative tests +Time and origin as Date +2023-02-01 +2023-08-01 +2023-10-08 +2023-10-08 +2023-10-09 +Time and origin as Date32 +2023-02-01 +2023-08-01 +2023-10-08 +2023-10-08 +2023-10-09 +Time and origin as DateTime +2023-02-01 09:08:07 +2023-08-01 09:08:07 +2023-10-08 09:08:07 +2023-10-08 09:08:07 +2023-10-09 09:08:07 +2023-10-09 10:10:07 +2023-10-09 10:11:07 +2023-10-09 10:11:12 +Time and origin as DateTime64(9) +2023-02-01 09:08:07.123456789 +2023-08-01 09:08:07.123456789 +2023-09-10 09:08:07.123456789 +2023-10-08 09:08:07.123456789 +2023-10-09 09:08:07.123456789 +2023-10-09 10:10:07.123456789 +2023-10-09 10:11:11.123456789 +2023-10-09 10:11:12.123456789 +2023-10-09 10:11:12.987 +2023-10-09 10:11:12.987654 +2023-10-09 10:11:12.987654321 +Time and origin as DateTime64(3) +2023-02-01 09:08:07.123 +2023-08-01 09:08:07.123 +2023-10-08 09:08:07.123 +2023-10-08 09:08:07.123 +2023-10-09 09:08:07.123 +2023-10-09 10:10:07.123 +2023-10-09 10:11:11.123 +2023-10-09 10:11:12.123 +2023-10-09 10:11:12.987 +2023-10-09 10:11:12.987000 +2023-10-09 10:11:12.987000000 +Non-const arguments +2023-03-01 16:55:00.00 +2023-02-01 16:55:00.00 +2023-03-01 16:55:00.00 +2023-02-01 16:55:00.00 +2023-03-01 16:55:00.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-02-01 16:44:30.00 diff --git a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql new file mode 100644 index 00000000000..b03ccae31d9 --- /dev/null +++ b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql @@ -0,0 +1,95 @@ +set session_timezone = 'UTC'; + +SELECT '-- Negative tests'; + +-- time and origin arguments must have the same type +SELECT toStartOfInterval(toDate('2023-01-02 14:45:50'), toIntervalSecond(5), toDate32('2023-01-02 14:44:30')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDate('2023-01-02 14:45:50'), toIntervalMillisecond(12), toDateTime('2023-01-02 14:44:30')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDate32('2023-01-02 14:45:50'), toIntervalHour(5), toDate('2023-01-02 14:44:30')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), toDateTime64('2023-01-02 14:44:30', 2)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50', 2), toIntervalMinute(1), toDate('2023-01-02 14:44:30')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +-- the origin must be before the time +SELECT toStartOfInterval(toDateTime('2023-01-02 14:42:50'), toIntervalMinute(1), toDateTime('2023-01-02 14:44:30')); -- { serverError BAD_ARGUMENTS } + +-- the origin must be constant +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')) from numbers(1); -- { serverError ILLEGAL_COLUMN } +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalHour(1), materialize(toDateTime('2023-01-02 14:44:30')), 'Europe/Amsterdam'); -- { serverError ILLEGAL_COLUMN } + +-- with 4 arguments, the 3rd one must not be a string or an integer +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), 'Europe/Amsterdam', 'Europe/Amsterdam'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), 5, 'Europe/Amsterdam'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +-- too many arguments +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), toDateTime('2020-01-02 14:44:30'), 'Europe/Amsterdam', 5); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +SELECT 'Time and origin as Date'; +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalYear(1), toDate('2022-02-01')); +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalQuarter(1), toDate('2022-02-01')); +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalMonth(1), toDate('2023-09-08')); +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalWeek(1), toDate('2023-10-01')); +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalDay(1), toDate('2023-10-08')); +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalHour(1), toDate('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalMinute(1), toDate('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalSecond(1), toDate('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalMillisecond(1), toDate('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalMicrosecond(1), toDate('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalNanosecond(1), toDate('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT 'Time and origin as Date32'; +SELECT toStartOfInterval(toDate32('2023-10-09'), toIntervalYear(1), toDate32('2022-02-01')); +SELECT toStartOfInterval(toDate32('2023-10-09'), toIntervalQuarter(1), toDate32('2022-02-01')); +SELECT toStartOfInterval(toDate32('2023-10-09'), toIntervalMonth(1), toDate32('2023-09-08')); +SELECT toStartOfInterval(toDate32('2023-10-09'), toIntervalWeek(1), toDate32('2023-10-01')); +SELECT toStartOfInterval(toDate32('2023-10-09'), toIntervalDay(1), toDate32('2023-10-08')); +SELECT toStartOfInterval(toDate32('2023-10-09'), toIntervalHour(1), toDate32('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDate32('2023-10-09'), toIntervalMinute(1), toDate32('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDate32('2023-10-09'), toIntervalSecond(1), toDate32('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDate32('2023-10-09'), toIntervalMillisecond(1), toDate32('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDate32('2023-10-09'), toIntervalMicrosecond(1), toDate32('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDate32('2023-10-09'), toIntervalNanosecond(1), toDate32('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT 'Time and origin as DateTime'; +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalYear(1), toDateTime('2022-02-01 09:08:07')); +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalQuarter(1), toDateTime('2022-02-01 09:08:07')); +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMonth(1), toDateTime('2023-09-08 09:08:07')); +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalWeek(1), toDateTime('2023-10-01 09:08:07')); +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalDay(1), toDateTime('2023-10-08 09:08:07')); +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalHour(1), toDateTime('2023-10-09 09:10:07')); +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMinute(1), toDateTime('2023-10-09 09:10:07')); +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalSecond(1), toDateTime('2023-10-09 09:10:07')); +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMillisecond(1), toDateTime('2023-10-09 10:11:12')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMicrosecond(1), toDateTime('2023-10-09 10:11:12')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalNanosecond(1), toDateTime('2023-10-09 10:11:12')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT 'Time and origin as DateTime64(9)'; +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalYear(1), toDateTime64('2022-02-01 09:08:07.123456789', 9)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalQuarter(1), toDateTime64('2022-02-01 09:08:07.123456789', 9)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMonth(1), toDateTime64('2023-09-10 09:08:07.123456789', 9)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalWeek(1), toDateTime64('2023-10-01 09:08:07.123456789', 9)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalDay(1), toDateTime64('2023-10-08 09:08:07.123456789', 9)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalHour(1), toDateTime64('2023-10-09 09:10:07.123456789', 9)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMinute(1), toDateTime64('2023-10-09 09:10:11.123456789', 9)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalSecond(1), toDateTime64('2023-10-09 10:11:10.123456789', 9)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMillisecond(1), toDateTime64('2023-10-09 10:11:12.123456789', 9)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMicrosecond(1), toDateTime64('2023-10-09 10:11:12.123456789', 9)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalNanosecond(1), toDateTime64('2023-10-09 10:11:12.123456789', 9)); + +SELECT 'Time and origin as DateTime64(3)'; +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalYear(1), toDateTime64('2022-02-01 09:08:07.123', 3)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalQuarter(1), toDateTime64('2022-02-01 09:08:07.123', 3)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalMonth(1), toDateTime64('2023-09-08 09:08:07.123', 3)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalWeek(1), toDateTime64('2023-10-01 09:08:07.123', 3)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalDay(1), toDateTime64('2023-10-08 09:08:07.123', 3)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalHour(1), toDateTime64('2023-10-09 09:10:07.123', 3)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalMinute(1), toDateTime64('2023-10-09 10:10:11.123', 3)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalSecond(1), toDateTime64('2023-10-09 10:11:10.123', 3)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalMillisecond(1), toDateTime64('2023-10-09 10:11:12.123', 3)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalMicrosecond(1), toDateTime64('2023-10-09 10:11:12.123', 3)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalNanosecond(1), toDateTime64('2023-10-09 10:11:12.123', 3)); + +SELECT 'Non-const arguments'; +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'), toIntervalHour(1), toDateTime('2023-01-01 13:55:00'), 'Europe/Amsterdam') from numbers(5); +SELECT toStartOfInterval(materialize(toDateTime('2023-01-02 14:45:50')), toIntervalHour(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); +SELECT toStartOfInterval(materialize(toDateTime64('2023-02-01 15:45:50', 2)), toIntervalHour(1), toDateTime64('2023-01-02 14:44:30', 2), 'Europe/Amsterdam'); diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index d10db5f0d3d..0e08a5f8540 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -975,6 +975,7 @@ ThreadPoolRemoteFSReaderThreads ThreadPoolRemoteFSReaderThreadsActive ThreadsActive ThreadsInOvercommitTracker +TimescaleDB's TimeSeries Timeunit TinyLog