done suggestings about logic

This commit is contained in:
yariks5s 2023-11-17 14:22:39 +00:00
parent aef1ea3df7
commit 188c469318

View File

@ -1,5 +1,7 @@
#include <base/arithmeticOverflow.h>
#include <Common/DateLUTImpl.h>
#include <DataTypes/DataTypesDecimal.h>
#include <Functions/FunctionHelpers.h>
#include <Columns/ColumnsDateTime.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeDate.h>
@ -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 <IntervalKind::Kind unit>
struct Transform;
template <>
struct Transform<IntervalKind::Year>
{
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<IntervalKind::Quarter>
{
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<IntervalKind::Month>
{
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<IntervalKind::Week>
{
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<IntervalKind::Day>
{
static UInt32 execute(UInt16 d, Int64 days, const DateLUTImpl & time_zone, Int64)
{
return static_cast<UInt32>(time_zone.toStartOfDayInterval(ExtendedDayNum(d), days));
}
static UInt32 execute(Int32 d, Int64 days, const DateLUTImpl & time_zone, Int64)
{
return static_cast<UInt32>(time_zone.toStartOfDayInterval(ExtendedDayNum(d), days));
}
static UInt32 execute(UInt32 t, Int64 days, const DateLUTImpl & time_zone, Int64)
{
return static_cast<UInt32>(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<IntervalKind::Hour>
{
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<IntervalKind::Minute>
{
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<IntervalKind::Second>
{
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<IntervalKind::Millisecond>
{
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<Int64>(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<Int64>(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<IntervalKind::Microsecond>
{
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<Int64>(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<Int64>(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<IntervalKind::Nanosecond>
{
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<Int64>(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<FunctionToStartOfInterval>(); }
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<const DataTypeDate32 &>(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 <typename ColumnType, typename FromDataType>
template <typename TimeColumnType, typename TimeDataType>
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<DataTypeInterval>(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<ColumnInt64>(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<Int64>();
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<TimeDataType, DataTypeDateTime64, IntervalKind::Nanosecond>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
@ -601,17 +307,24 @@ private:
Int64 scale_multiplier = DecimalUtils::scaleMultiplier<DateTime64>(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<ColumnType>(...)` 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<ToFieldType>(ToStartOfInterval<unit>::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<ToFieldType>(ToStartOfInterval<unit>::execute(td, num_units, time_zone, scale_multiplier));
for (size_t i = 0; i != size; ++i)
result_data[i] = static_cast<ToFieldType>(
Transform<unit>::execute(time_data[i], num_units, time_zone, scale_multiplier));
result_data[i] += scale_multiplier == 10 ? od : od / scale_multiplier;
}
}
return result_col;
}