mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-09 17:14:47 +00:00
done suggestings about logic
This commit is contained in:
parent
aef1ea3df7
commit
188c469318
@ -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;
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user