#include #include #include #include #include #include #include #include #include #include #include #include namespace DB { namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ARGUMENT_OUT_OF_BOUND; } namespace { constexpr auto function_name = "toStartOfInterval"; template struct Transform; template <> struct Transform { static constexpr auto name = function_name; static UInt16 execute(UInt16 d, UInt64 years, const DateLUTImpl & time_zone) { return time_zone.toStartOfYearInterval(DayNum(d), years); } static UInt16 execute(Int32 d, UInt64 years, const DateLUTImpl & time_zone) { return time_zone.toStartOfYearInterval(ExtendedDayNum(d), years); } static UInt16 execute(UInt32 t, UInt64 years, const DateLUTImpl & time_zone) { return time_zone.toStartOfYearInterval(time_zone.toDayNum(t), years); } static UInt16 execute(Int64 t, UInt64 years, const DateLUTImpl & time_zone) { return time_zone.toStartOfYearInterval(time_zone.toDayNum(t), years); } }; template <> struct Transform { static constexpr auto name = function_name; static UInt16 execute(UInt16 d, UInt64 quarters, const DateLUTImpl & time_zone) { return time_zone.toStartOfQuarterInterval(DayNum(d), quarters); } static UInt16 execute(Int32 d, UInt64 quarters, const DateLUTImpl & time_zone) { return time_zone.toStartOfQuarterInterval(ExtendedDayNum(d), quarters); } static UInt16 execute(UInt32 t, UInt64 quarters, const DateLUTImpl & time_zone) { return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t), quarters); } static UInt16 execute(Int64 t, UInt64 quarters, const DateLUTImpl & time_zone) { return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t), quarters); } }; template <> struct Transform { static constexpr auto name = function_name; static UInt16 execute(UInt16 d, UInt64 months, const DateLUTImpl & time_zone) { return time_zone.toStartOfMonthInterval(DayNum(d), months); } static UInt16 execute(Int32 d, UInt64 months, const DateLUTImpl & time_zone) { return time_zone.toStartOfMonthInterval(ExtendedDayNum(d), months); } static UInt16 execute(UInt32 t, UInt64 months, const DateLUTImpl & time_zone) { return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t), months); } static UInt16 execute(Int64 t, UInt64 months, const DateLUTImpl & time_zone) { return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t), months); } }; template <> struct Transform { static constexpr auto name = function_name; static UInt16 execute(UInt16 d, UInt64 weeks, const DateLUTImpl & time_zone) { return time_zone.toStartOfWeekInterval(DayNum(d), weeks); } static UInt16 execute(Int32 d, UInt64 weeks, const DateLUTImpl & time_zone) { return time_zone.toStartOfWeekInterval(ExtendedDayNum(d), weeks); } static UInt16 execute(UInt32 t, UInt64 weeks, const DateLUTImpl & time_zone) { return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t), weeks); } static UInt16 execute(Int64 t, UInt64 weeks, const DateLUTImpl & time_zone) { return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t), weeks); } }; template <> struct Transform { static constexpr auto name = function_name; static UInt32 execute(UInt16 d, UInt64 days, const DateLUTImpl & time_zone) { return time_zone.toStartOfDayInterval(ExtendedDayNum(d), days); } static UInt32 execute(Int32 d, UInt64 days, const DateLUTImpl & time_zone) { return time_zone.toStartOfDayInterval(ExtendedDayNum(d), days); } static UInt32 execute(UInt32 t, UInt64 days, const DateLUTImpl & time_zone) { return time_zone.toStartOfDayInterval(time_zone.toDayNum(t), days); } static UInt32 execute(Int64 t, UInt64 days, const DateLUTImpl & time_zone) { return time_zone.toStartOfDayInterval(time_zone.toDayNum(t), days); } }; template <> struct Transform { static constexpr auto name = function_name; static UInt32 execute(UInt16, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); } static UInt32 execute(Int32, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); } static UInt32 execute(UInt32 t, UInt64 hours, const DateLUTImpl & time_zone) { return time_zone.toStartOfHourInterval(t, hours); } static UInt32 execute(Int64 t, UInt64 hours, const DateLUTImpl & time_zone) { return time_zone.toStartOfHourInterval(t, hours); } }; template <> struct Transform { static constexpr auto name = function_name; static UInt32 execute(UInt16, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); } static UInt32 execute(Int32, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); } static UInt32 execute(UInt32 t, UInt64 minutes, const DateLUTImpl & time_zone) { return time_zone.toStartOfMinuteInterval(t, minutes); } static UInt32 execute(Int64 t, UInt64 minutes, const DateLUTImpl & time_zone) { return time_zone.toStartOfMinuteInterval(t, minutes); } }; template <> struct Transform { static constexpr auto name = function_name; static UInt32 execute(UInt16, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); } static UInt32 execute(Int32, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); } static UInt32 execute(UInt32 t, UInt64 seconds, const DateLUTImpl & time_zone) { return time_zone.toStartOfSecondInterval(t, seconds); } static Int64 execute(Int64 t, UInt64 seconds, const DateLUTImpl & time_zone) { return time_zone.toStartOfSecondInterval(t, seconds); } }; class FunctionToStartOfInterval : public IFunction { public: static FunctionPtr create(ContextPtr) { return std::make_shared(); } static constexpr auto name = function_name; 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; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { bool first_argument_is_date = false; auto check_first_argument = [&] { if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) throw Exception( "Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() + ". Should be a date or a date with time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); first_argument_is_date = isDate(arguments[0].type); }; const DataTypeInterval * interval_type = nullptr; bool result_type_is_date = false; auto check_interval_argument = [&] { interval_type = checkAndGetDataType(arguments[1].type.get()); if (!interval_type) throw Exception( "Illegal type " + arguments[1].type->getName() + " of argument of function " + getName() + ". Should be an interval of time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); result_type_is_date = (interval_type->getKind() == IntervalKind::Year) || (interval_type->getKind() == IntervalKind::Quarter) || (interval_type->getKind() == IntervalKind::Month) || (interval_type->getKind() == IntervalKind::Week); }; auto check_timezone_argument = [&] { if (!WhichDataType(arguments[2].type).isString()) throw Exception( "Illegal type " + arguments[2].type->getName() + " of argument of function " + getName() + ". This argument is optional and must be a constant string with timezone name", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (first_argument_is_date && result_type_is_date) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The timezone argument of function {} with interval type {} is allowed only when the 1st argument " "has the type DateTime", getName(), interval_type->getKind().toString()); }; if (arguments.size() == 2) { check_first_argument(); check_interval_argument(); } else if (arguments.size() == 3) { check_first_argument(); check_interval_argument(); check_timezone_argument(); } else { throw Exception( "Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + ", should be 2 or 3", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } if (result_type_is_date) return std::make_shared(); else return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0)); } bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, 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, 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( const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, const DateLUTImpl & time_zone) const { const auto & from_datatype = *time_column.type.get(); const auto which_type = WhichDataType(from_datatype); if (which_type.isDateTime()) { 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, time_zone); } if (which_type.isDate()) { 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, time_zone); } if (which_type.isDate32()) { 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, time_zone); } if (which_type.isDateTime64()) { 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, time_zone); } throw Exception( "Illegal column for first argument of function " + getName() + ". Must contain dates or dates with time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } template ColumnPtr dispatchForIntervalColumn( const FromDataType & from, const ColumnType & time_column, const ColumnWithTypeAndName & interval_column, const DateLUTImpl & time_zone) const { const auto * interval_type = checkAndGetDataType(interval_column.type.get()); if (!interval_type) throw Exception( "Illegal column for second argument of function " + getName() + ", must be an interval of time.", ErrorCodes::ILLEGAL_COLUMN); const auto * interval_column_const_int64 = checkAndGetColumnConst(interval_column.column.get()); if (!interval_column_const_int64) throw Exception( "Illegal column for second argument of function " + getName() + ", must be a const interval of time.", ErrorCodes::ILLEGAL_COLUMN); Int64 num_units = interval_column_const_int64->getValue(); if (num_units <= 0) throw Exception("Value for second argument of function " + getName() + " must be positive.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); switch (interval_type->getKind()) { case IntervalKind::Second: return execute(from, time_column, num_units, time_zone); case IntervalKind::Minute: return execute(from, time_column, num_units, time_zone); case IntervalKind::Hour: return execute(from, time_column, num_units, time_zone); case IntervalKind::Day: return execute(from, time_column, num_units, time_zone); case IntervalKind::Week: return execute(from, time_column, num_units, time_zone); case IntervalKind::Month: return execute(from, time_column, num_units, time_zone); case IntervalKind::Quarter: return execute(from, time_column, num_units, time_zone); case IntervalKind::Year: return execute(from, time_column, num_units, time_zone); } __builtin_unreachable(); } template ColumnPtr execute(const FromDataType & from_datatype, const ColumnType & time_column, UInt64 num_units, const DateLUTImpl & time_zone) const { const auto & time_data = time_column.getData(); size_t size = time_column.size(); auto result = ColumnVector::create(); auto & result_data = result->getData(); result_data.resize(size); if constexpr (std::is_same_v) { const auto transform = TransformDateTime64>{from_datatype.getScale()}; for (size_t i = 0; i != size; ++i) result_data[i] = transform.execute(time_data[i], num_units, time_zone); } else { for (size_t i = 0; i != size; ++i) result_data[i] = Transform::execute(time_data[i], num_units, time_zone); } return result; } }; } void registerFunctionToStartOfInterval(FunctionFactory & factory) { factory.registerFunction(); } }