diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp
index 4d716637932..29f8faa810c 100644
--- a/src/Functions/toStartOfInterval.cpp
+++ b/src/Functions/toStartOfInterval.cpp
@@ -1,5 +1,7 @@
#include
#include
+#include
+#include
#include
#include
#include
@@ -21,300 +23,18 @@ namespace ErrorCodes
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ARGUMENT_OUT_OF_BOUND;
- extern const int DECIMAL_OVERFLOW;
}
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; }
@@ -472,9 +192,6 @@ public:
std::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];
@@ -485,23 +202,13 @@ public:
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 size_t time_zone_arg_num = (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;
}
- 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 dispatchForTimeColumn(
const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, const ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone) const
@@ -535,11 +242,10 @@ private:
if (time_column_vec)
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());
+ 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 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
@@ -547,16 +253,16 @@ private:
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());
- switch (interval_type->getKind())
+ switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case)
{
case IntervalKind::Nanosecond:
return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
@@ -601,17 +307,24 @@ 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.
+ if (origin_column.column == nullptr)
+ for (size_t i = 0; i != size; ++i)
+ result_data[i] = static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_multiplier));
+ else
+ {
+ UInt64 od = origin_column.column->get64(0);
+
+ for (size_t i = 0; i != size; ++i)
+ {
+ auto td = time_data[i];
+ if (od > size_t(td))
+ throw Exception(ErrorCodes::BAD_ARGUMENTS, "The origin must be before the end date/datetime");
+ td -= od;
+ result_data[i] = static_cast(ToStartOfInterval::execute(td, num_units, time_zone, scale_multiplier));
- 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] += scale_multiplier == 10 ? od : od / scale_multiplier;
+ }
+ }
return result_col;
}