diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 0a3a409ea9f..4d716637932 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -321,6 +321,11 @@ 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, 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 }; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { @@ -329,7 +334,7 @@ public: { const DataTypePtr & type_arg1 = arguments[0].type; if (!isDate(type_arg1) && !isDateTime(type_arg1) && !isDateTime64(type_arg1)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 1st argument of function {}. " "Should be a date or a date with time", type_arg1->getName(), getName()); value_is_date = isDate(type_arg1); }; @@ -347,7 +352,7 @@ public: const DataTypePtr & type_arg2 = arguments[1].type; interval_type = checkAndGetDataType(type_arg2.get()); if (!interval_type) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 2nd argument of function {}. " "Should be an interval of time", type_arg2->getName(), getName()); switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case) { @@ -371,18 +376,48 @@ public: } }; + enum class ThirdArgument + { + IsTimezone, + IsOrigin + }; + ThirdArgument third_argument; /// valid only if 3rd argument is given 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 argument of function {}. " - "This argument is optional and must be a constant string with timezone name", + if (isString(type_arg3)) + { + third_argument = ThirdArgument::IsTimezone; + 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 the type DateTime or DateTime64", + getName(), interval_type->getKind().toString()); + } + else if (isDateOrDate32OrDateTimeOrDateTime64(type_arg3)) + third_argument = ThirdArgument::IsOrigin; + 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()); + + }; + + auto check_fourth_argument = [&] + { + if (third_argument != ThirdArgument::IsOrigin) /// 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 the type DateTime or DateTime64", - getName(), interval_type->getKind().toString()); + "The 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()); }; if (arguments.size() == 2) @@ -396,10 +431,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 {}, should be 2, 3 or 4", getName(), arguments.size()); } @@ -408,7 +450,10 @@ public: case ResultType::Date: return std::make_shared(); case ResultType::DateTime: - return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); + { + const size_t time_zone_arg_num = (arguments.size() == 2 || (arguments.size() == 3 && third_argument == ThirdArgument::IsTimezone)) ? 2 : 3; + return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, time_zone_arg_num, 0, false)); + } case ResultType::DateTime64: { UInt32 scale = 0; @@ -419,7 +464,8 @@ public: else if (interval_type->getKind() == IntervalKind::Millisecond) scale = 3; - return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); + const size_t time_zone_arg_num = (arguments.size() == 2 || (arguments.size() == 3 && third_argument == ThirdArgument::IsTimezone)) ? 2 : 3; + return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, time_zone_arg_num, 0, false)); } } @@ -433,8 +479,16 @@ public: { 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); + + ColumnWithTypeAndName origin_column; + const bool has_origin_arg = (arguments.size() == 3 && isDateOrDate32OrDateTimeOrDateTime64(arguments[2].type)) || arguments.size() == 4; + if (has_origin_arg) + origin_column = arguments[2]; + + const size_t time_zone_arg_num = (arguments.size() == 2 || (arguments.size() == 3 && isString(arguments[2].type))) ? 2 : 3; + const auto & time_zone = extractTimeZoneFromFunctionArguments(arguments, time_zone_arg_num, 0); + + auto result_column = dispatchForTimeColumn(time_column, interval_column, origin_column, result_type, time_zone); return result_column; } @@ -449,8 +503,8 @@ public: } private: - ColumnPtr dispatchForColumns( - const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone) const + ColumnPtr dispatchForTimeColumn( + 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(); @@ -461,25 +515,25 @@ private: 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, scale); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone, 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); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); } else 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); + 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); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_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()); @@ -487,7 +541,7 @@ private: 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 ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale = 1) const { const auto * interval_type = checkAndGetDataType(interval_column.type.get()); @@ -505,34 +559,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, origin_column, 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, origin_column, 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, origin_column, 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, origin_column, 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, origin_column, 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, origin_column, 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, origin_column, 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, origin_column, 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, origin_column, 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, origin_column, 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, origin_column, result_type, time_zone, scale); } std::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, Int64 num_units, [[maybe_unused]] 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; @@ -547,6 +601,14 @@ private: Int64 scale_multiplier = DecimalUtils::scaleMultiplier(scale); + /// TODO: This part is missing. origin_column is either {} (<-- to check, you could do `origin_column.column == nullptr`) or not {} + /// In the former case, we can execute below existing code. + /// In the latter case, we need to read the actual origin value. As per `getArgumentsThatAreAlwaysConstant()` (see above), we + /// can be sure that origin_column is a `ColumnConst`. The second assumption we can reasonable make is that it has the same + /// type (Date/Date32/DateTime/DateTime64) as the time column (1st argument). Since the method we are in is already + /// templatized on the data type of the time column, we can use `checkAndGetColumnConst(...)` to cast + /// `origin_column.column` to a const column and then read the (const) value from it, and proceed with the calculations. + for (size_t i = 0; i != size; ++i) result_data[i] = static_cast( Transform::execute(time_data[i], num_units, time_zone, scale_multiplier));