diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index a351d7fdf30..04f065ecbdc 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -304,6 +305,301 @@ struct ToStartOfYearImpl }; +template +struct ToStartOfInterval; + +static constexpr auto TO_START_OF_INTERVAL_NAME = "toStartOfInterval"; + +template <> +struct ToStartOfInterval +{ + static UInt32 execute(UInt16, UInt64, const DateLUTImpl &, Int64) + { + throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME); + } + static UInt32 execute(Int32, UInt64, const DateLUTImpl &, Int64) + { + throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME); + } + static UInt32 execute(UInt32, UInt64, const DateLUTImpl &, Int64) + { + throwDateTimeIsNotSupported(TO_START_OF_INTERVAL_NAME); + } + static Int64 execute(Int64 t, UInt64 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 (t >= 0) [[likely]] + return t_nanoseconds / nanoseconds * nanoseconds; + else + return ((t_nanoseconds + 1) / nanoseconds - 1) * nanoseconds; + } + else + if (t >= 0) [[likely]] + return t / nanoseconds * nanoseconds; + else + return ((t + 1) / nanoseconds - 1) * nanoseconds; + } +}; + +template <> +struct ToStartOfInterval +{ + static UInt32 execute(UInt16, UInt64, const DateLUTImpl &, Int64) + { + throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME); + } + static UInt32 execute(Int32, UInt64, const DateLUTImpl &, Int64) + { + throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME); + } + static UInt32 execute(UInt32, UInt64, const DateLUTImpl &, Int64) + { + throwDateTimeIsNotSupported(TO_START_OF_INTERVAL_NAME); + } + static Int64 execute(Int64 t, UInt64 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 (t >= 0) [[likely]] + 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 (t >= 0) [[likely]] + return t / microseconds / scale_diff * microseconds; + else + return ((t + 1) / microseconds / scale_diff - 1) * microseconds; + } + else + if (t >= 0) [[likely]] + return t / microseconds * microseconds; + else + return ((t + 1) / microseconds - 1) * microseconds; + } +}; + +template <> +struct ToStartOfInterval +{ + static UInt32 execute(UInt16, UInt64, const DateLUTImpl &, Int64) + { + throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME); + } + static UInt32 execute(Int32, UInt64, const DateLUTImpl &, Int64) + { + throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME); + } + static UInt32 execute(UInt32, UInt64, const DateLUTImpl &, Int64) + { + throwDateTimeIsNotSupported(TO_START_OF_INTERVAL_NAME); + } + static Int64 execute(Int64 t, UInt64 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 (t >= 0) [[likely]] + 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 (t >= 0) [[likely]] + return t / milliseconds / scale_diff * milliseconds; + else + return ((t + 1) / milliseconds / scale_diff - 1) * milliseconds; + } + else + if (t >= 0) [[likely]] + return t / milliseconds * milliseconds; + else + return ((t + 1) / milliseconds - 1) * milliseconds; + } +}; + +template <> +struct ToStartOfInterval +{ + static UInt32 execute(UInt16, UInt64, const DateLUTImpl &, Int64) + { + throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME); + } + static UInt32 execute(Int32, UInt64, const DateLUTImpl &, Int64) + { + throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME); + } + static UInt32 execute(UInt32 t, UInt64 seconds, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfSecondInterval(t, seconds); + } + static Int64 execute(Int64 t, UInt64 seconds, const DateLUTImpl & time_zone, Int64 scale_multiplier) + { + return time_zone.toStartOfSecondInterval(t / scale_multiplier, seconds); + } +}; + +template <> +struct ToStartOfInterval +{ + static UInt32 execute(UInt16, UInt64, const DateLUTImpl &, Int64) + { + throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME); + } + static UInt32 execute(Int32, UInt64, const DateLUTImpl &, Int64) + { + throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME); + } + static UInt32 execute(UInt32 t, UInt64 minutes, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfMinuteInterval(t, minutes); + } + static Int64 execute(Int64 t, UInt64 minutes, const DateLUTImpl & time_zone, Int64 scale_multiplier) + { + return time_zone.toStartOfMinuteInterval(t / scale_multiplier, minutes); + } +}; + +template <> +struct ToStartOfInterval +{ + static UInt32 execute(UInt16, UInt64, const DateLUTImpl &, Int64) + { + throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME); + } + static UInt32 execute(Int32, UInt64, const DateLUTImpl &, Int64) + { + throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME); + } + static UInt32 execute(UInt32 t, UInt64 hours, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfHourInterval(t, hours); + } + static Int64 execute(Int64 t, UInt64 hours, const DateLUTImpl & time_zone, Int64 scale_multiplier) + { + return time_zone.toStartOfHourInterval(t / scale_multiplier, hours); + } +}; + +template <> +struct ToStartOfInterval +{ + static UInt32 execute(UInt16 d, UInt64 days, const DateLUTImpl & time_zone, Int64) + { + return static_cast(time_zone.toStartOfDayInterval(ExtendedDayNum(d), days)); + } + static UInt32 execute(Int32 d, UInt64 days, const DateLUTImpl & time_zone, Int64) + { + return static_cast(time_zone.toStartOfDayInterval(ExtendedDayNum(d), days)); + } + static UInt32 execute(UInt32 t, UInt64 days, const DateLUTImpl & time_zone, Int64) + { + return static_cast(time_zone.toStartOfDayInterval(time_zone.toDayNum(t), days)); + } + static Int64 execute(Int64 t, UInt64 days, const DateLUTImpl & time_zone, Int64 scale_multiplier) + { + return time_zone.toStartOfDayInterval(time_zone.toDayNum(t / scale_multiplier), days); + } +}; + +template <> +struct ToStartOfInterval +{ + static UInt16 execute(UInt16 d, UInt64 weeks, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfWeekInterval(DayNum(d), weeks); + } + static UInt16 execute(Int32 d, UInt64 weeks, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfWeekInterval(ExtendedDayNum(d), weeks); + } + static UInt16 execute(UInt32 t, UInt64 weeks, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t), weeks); + } + static UInt16 execute(Int64 t, UInt64 weeks, const DateLUTImpl & time_zone, Int64 scale_multiplier) + { + return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t / scale_multiplier), weeks); + } +}; + +template <> +struct ToStartOfInterval +{ + static UInt16 execute(UInt16 d, UInt64 months, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfMonthInterval(DayNum(d), months); + } + static UInt16 execute(Int32 d, UInt64 months, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfMonthInterval(ExtendedDayNum(d), months); + } + static UInt16 execute(UInt32 t, UInt64 months, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t), months); + } + static UInt16 execute(Int64 t, UInt64 months, const DateLUTImpl & time_zone, Int64 scale_multiplier) + { + return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t / scale_multiplier), months); + } +}; + +template <> +struct ToStartOfInterval +{ + static UInt16 execute(UInt16 d, UInt64 quarters, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfQuarterInterval(DayNum(d), quarters); + } + static UInt16 execute(Int32 d, UInt64 quarters, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfQuarterInterval(ExtendedDayNum(d), quarters); + } + static UInt16 execute(UInt32 t, UInt64 quarters, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t), quarters); + } + static UInt16 execute(Int64 t, UInt64 quarters, const DateLUTImpl & time_zone, Int64 scale_multiplier) + { + return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t / scale_multiplier), quarters); + } +}; + +template <> +struct ToStartOfInterval +{ + static UInt16 execute(UInt16 d, UInt64 years, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfYearInterval(DayNum(d), years); + } + static UInt16 execute(Int32 d, UInt64 years, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfYearInterval(ExtendedDayNum(d), years); + } + static UInt16 execute(UInt32 t, UInt64 years, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfYearInterval(time_zone.toDayNum(t), years); + } + static UInt16 execute(Int64 t, UInt64 years, const DateLUTImpl & time_zone, Int64 scale_multiplier) + { + return time_zone.toStartOfYearInterval(time_zone.toDayNum(t / scale_multiplier), years); + } +}; + + struct ToTimeImpl { /// When transforming to time, the date will be equated to 1970-01-02. diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 48bf88cb14c..2567b661496 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -27,300 +27,27 @@ namespace ErrorCodes 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 { public: static FunctionPtr create(ContextPtr) { return std::make_shared(); } - static constexpr auto name = function_name; + static constexpr auto name = "toStartOfInterval"; String getName() const override { return name; } bool isVariadic() const override { return true; } 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}; } + + bool hasInformationAboutMonotonicity() const override { return true; } + Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override + { + return { .is_monotonic = true, .is_always_monotonic = true }; + } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { @@ -336,22 +63,38 @@ public: const DataTypeInterval * interval_type = nullptr; bool result_type_is_date = false; bool result_type_is_datetime = false; + bool result_type_is_datetime_64 = false; auto check_interval_argument = [&] { interval_type = checkAndGetDataType(arguments[1].type.get()); if (!interval_type) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " "Should be an interval of time", arguments[1].type->getName(), getName()); - result_type_is_date = (interval_type->getKind() == IntervalKind::Year) - || (interval_type->getKind() == IntervalKind::Quarter) || (interval_type->getKind() == IntervalKind::Month) - || (interval_type->getKind() == IntervalKind::Week); - result_type_is_datetime = (interval_type->getKind() == IntervalKind::Day) || (interval_type->getKind() == IntervalKind::Hour) - || (interval_type->getKind() == IntervalKind::Minute) || (interval_type->getKind() == IntervalKind::Second); + switch (interval_type->getKind()) + { + case IntervalKind::Nanosecond: + case IntervalKind::Microsecond: + case IntervalKind::Millisecond: + result_type_is_datetime_64 = true; + break; + case IntervalKind::Second: + case IntervalKind::Minute: + case IntervalKind::Hour: + case IntervalKind::Day: + result_type_is_datetime = true; + break; + case IntervalKind::Week: + case IntervalKind::Month: + case IntervalKind::Quarter: + case IntervalKind::Year: + result_type_is_date = true; + break; + } }; auto check_timezone_argument = [&] { - if (!WhichDataType(arguments[2].type).isString()) + if (!isString(arguments[2].type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " "This argument is optional and must be a constant string with timezone name", arguments[2].type->getName(), getName()); @@ -384,7 +127,7 @@ public: return std::make_shared(); else if (result_type_is_datetime) return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); - else + else if (result_type_is_datetime_64) { auto scale = 0; @@ -398,80 +141,66 @@ public: return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); } + UNREACHABLE(); } - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } - 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 = dispatchForColumns(time_column, interval_column, result_type, time_zone); + auto result_column = dispatchForTimeColumn(time_column, interval_column, result_type, time_zone); return result_column; } - bool hasInformationAboutMonotonicity() const override - { - return true; - } - - Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override - { - return { .is_monotonic = true, .is_always_monotonic = true }; - } - private: - ColumnPtr dispatchForColumns( + ColumnPtr dispatchForTimeColumn( const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone) const { const auto & from_datatype = *time_column.type.get(); - const auto which_type = WhichDataType(from_datatype); - if (which_type.isDateTime64()) + if (isDateTime64(from_datatype)) { const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); auto scale = assert_cast(from_datatype).getScale(); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(from_datatype), *time_column_vec, interval_column, result_type, time_zone, scale); + return dispatchForIntervalColumn(assert_cast(from_datatype), *time_column_vec, interval_column, result_type, time_zone, scale); } - if (which_type.isDateTime()) + if (isDateTime(from_datatype)) { const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(from_datatype), *time_column_vec, interval_column, result_type, time_zone); + return dispatchForIntervalColumn(assert_cast(from_datatype), *time_column_vec, interval_column, result_type, time_zone); } - if (which_type.isDate()) + if (isDate(from_datatype)) { const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(from_datatype), *time_column_vec, interval_column, result_type, time_zone); + return dispatchForIntervalColumn(assert_cast(from_datatype), *time_column_vec, interval_column, result_type, time_zone); } - if (which_type.isDate32()) + if (isDate32(from_datatype)) { const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(from_datatype), *time_column_vec, interval_column, result_type, time_zone); + return dispatchForIntervalColumn(assert_cast(from_datatype), *time_column_vec, interval_column, result_type, time_zone); } - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for first argument of function {}. " - "Must contain dates or dates with time", getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for first argument of function {}. Must contain dates or dates with time", getName()); } - template + template ColumnPtr dispatchForIntervalColumn( - const FromDataType & from, const ColumnType & time_column, const ColumnWithTypeAndName & interval_column, + const TimeDataType & time_data_type, const TimeColumnType & time_column, const ColumnWithTypeAndName & interval_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale = 1) const { const auto * interval_type = checkAndGetDataType(interval_column.type.get()); if (!interval_type) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column for second argument of function {}, must be an interval of time.", getName()); + const auto * interval_column_const_int64 = checkAndGetColumnConst(interval_column.column.get()); if (!interval_column_const_int64) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Illegal column for second argument of function {}, must be a const interval of time.", - getName()); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column for second argument of function {}, must be a const interval of time.", getName()); + Int64 num_units = interval_column_const_int64->getValue(); if (num_units <= 0) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Value for second argument of function {} must be positive.", getName()); @@ -479,34 +208,34 @@ private: switch (interval_type->getKind()) { case IntervalKind::Nanosecond: - return execute(from, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); case IntervalKind::Microsecond: - return execute(from, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); case IntervalKind::Millisecond: - return execute(from, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); case IntervalKind::Second: - return execute(from, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); case IntervalKind::Minute: - return execute(from, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); case IntervalKind::Hour: - return execute(from, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); case IntervalKind::Day: - return execute(from, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); case IntervalKind::Week: - return execute(from, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); case IntervalKind::Month: - return execute(from, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); case IntervalKind::Quarter: - return execute(from, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); case IntervalKind::Year: - return execute(from, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); } UNREACHABLE(); } - template - ColumnPtr execute(const FromDataType &, const ColumnType & time_column_type, Int64 num_units, const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale) const + template + ColumnPtr execute(const TimeDataType &, const ColumnType & time_column_type, UInt64 num_units, const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale) const { using ToColumnType = typename ToDataType::ColumnType; using ToFieldType = typename ToDataType::FieldType; @@ -522,8 +251,7 @@ private: Int64 scale_multiplier = DecimalUtils::scaleMultiplier(scale); for (size_t i = 0; i != size; ++i) - result_data[i] = static_cast( - Transform::execute(time_data[i], num_units, time_zone, scale_multiplier)); + result_data[i] = static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_multiplier)); return result_col; }