Refactorings

This commit is contained in:
Robert Schulze 2023-09-29 16:52:48 +00:00
parent 3a2ca1f69c
commit 4b7ffa52c5
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
2 changed files with 359 additions and 335 deletions

View File

@ -5,6 +5,7 @@
#include <Common/Exception.h>
#include <Common/DateLUTImpl.h>
#include <Common/DateLUT.h>
#include <Common/IntervalKind.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnVector.h>
@ -304,6 +305,301 @@ struct ToStartOfYearImpl
};
template <IntervalKind::Kind unit>
struct ToStartOfInterval;
static constexpr auto TO_START_OF_INTERVAL_NAME = "toStartOfInterval";
template <>
struct ToStartOfInterval<IntervalKind::Nanosecond>
{
static UInt32 execute(UInt16, UInt64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(Int32, UInt64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32, UInt64, const DateLUTImpl &, Int64)
{
throwDateTimeIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static Int64 execute(Int64 t, UInt64 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 (t >= 0) [[likely]]
return t_nanoseconds / nanoseconds * nanoseconds;
else
return ((t_nanoseconds + 1) / nanoseconds - 1) * nanoseconds;
}
else
if (t >= 0) [[likely]]
return t / nanoseconds * nanoseconds;
else
return ((t + 1) / nanoseconds - 1) * nanoseconds;
}
};
template <>
struct ToStartOfInterval<IntervalKind::Microsecond>
{
static UInt32 execute(UInt16, UInt64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(Int32, UInt64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32, UInt64, const DateLUTImpl &, Int64)
{
throwDateTimeIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static Int64 execute(Int64 t, UInt64 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 (t >= 0) [[likely]]
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 (t >= 0) [[likely]]
return t / microseconds / scale_diff * microseconds;
else
return ((t + 1) / microseconds / scale_diff - 1) * microseconds;
}
else
if (t >= 0) [[likely]]
return t / microseconds * microseconds;
else
return ((t + 1) / microseconds - 1) * microseconds;
}
};
template <>
struct ToStartOfInterval<IntervalKind::Millisecond>
{
static UInt32 execute(UInt16, UInt64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(Int32, UInt64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32, UInt64, const DateLUTImpl &, Int64)
{
throwDateTimeIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static Int64 execute(Int64 t, UInt64 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 (t >= 0) [[likely]]
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 (t >= 0) [[likely]]
return t / milliseconds / scale_diff * milliseconds;
else
return ((t + 1) / milliseconds / scale_diff - 1) * milliseconds;
}
else
if (t >= 0) [[likely]]
return t / milliseconds * milliseconds;
else
return ((t + 1) / milliseconds - 1) * milliseconds;
}
};
template <>
struct ToStartOfInterval<IntervalKind::Second>
{
static UInt32 execute(UInt16, UInt64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(Int32, UInt64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32 t, UInt64 seconds, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfSecondInterval(t, seconds);
}
static Int64 execute(Int64 t, UInt64 seconds, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfSecondInterval(t / scale_multiplier, seconds);
}
};
template <>
struct ToStartOfInterval<IntervalKind::Minute>
{
static UInt32 execute(UInt16, UInt64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(Int32, UInt64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32 t, UInt64 minutes, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfMinuteInterval(t, minutes);
}
static Int64 execute(Int64 t, UInt64 minutes, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfMinuteInterval(t / scale_multiplier, minutes);
}
};
template <>
struct ToStartOfInterval<IntervalKind::Hour>
{
static UInt32 execute(UInt16, UInt64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(Int32, UInt64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32 t, UInt64 hours, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfHourInterval(t, hours);
}
static Int64 execute(Int64 t, UInt64 hours, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfHourInterval(t / scale_multiplier, hours);
}
};
template <>
struct ToStartOfInterval<IntervalKind::Day>
{
static UInt32 execute(UInt16 d, UInt64 days, const DateLUTImpl & time_zone, Int64)
{
return static_cast<UInt32>(time_zone.toStartOfDayInterval(ExtendedDayNum(d), days));
}
static UInt32 execute(Int32 d, UInt64 days, const DateLUTImpl & time_zone, Int64)
{
return static_cast<UInt32>(time_zone.toStartOfDayInterval(ExtendedDayNum(d), days));
}
static UInt32 execute(UInt32 t, UInt64 days, const DateLUTImpl & time_zone, Int64)
{
return static_cast<UInt32>(time_zone.toStartOfDayInterval(time_zone.toDayNum(t), days));
}
static Int64 execute(Int64 t, UInt64 days, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfDayInterval(time_zone.toDayNum(t / scale_multiplier), days);
}
};
template <>
struct ToStartOfInterval<IntervalKind::Week>
{
static UInt16 execute(UInt16 d, UInt64 weeks, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfWeekInterval(DayNum(d), weeks);
}
static UInt16 execute(Int32 d, UInt64 weeks, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfWeekInterval(ExtendedDayNum(d), weeks);
}
static UInt16 execute(UInt32 t, UInt64 weeks, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t), weeks);
}
static UInt16 execute(Int64 t, UInt64 weeks, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t / scale_multiplier), weeks);
}
};
template <>
struct ToStartOfInterval<IntervalKind::Month>
{
static UInt16 execute(UInt16 d, UInt64 months, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfMonthInterval(DayNum(d), months);
}
static UInt16 execute(Int32 d, UInt64 months, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfMonthInterval(ExtendedDayNum(d), months);
}
static UInt16 execute(UInt32 t, UInt64 months, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t), months);
}
static UInt16 execute(Int64 t, UInt64 months, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t / scale_multiplier), months);
}
};
template <>
struct ToStartOfInterval<IntervalKind::Quarter>
{
static UInt16 execute(UInt16 d, UInt64 quarters, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfQuarterInterval(DayNum(d), quarters);
}
static UInt16 execute(Int32 d, UInt64 quarters, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfQuarterInterval(ExtendedDayNum(d), quarters);
}
static UInt16 execute(UInt32 t, UInt64 quarters, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t), quarters);
}
static UInt16 execute(Int64 t, UInt64 quarters, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t / scale_multiplier), quarters);
}
};
template <>
struct ToStartOfInterval<IntervalKind::Year>
{
static UInt16 execute(UInt16 d, UInt64 years, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfYearInterval(DayNum(d), years);
}
static UInt16 execute(Int32 d, UInt64 years, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfYearInterval(ExtendedDayNum(d), years);
}
static UInt16 execute(UInt32 t, UInt64 years, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfYearInterval(time_zone.toDayNum(t), years);
}
static UInt16 execute(Int64 t, UInt64 years, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfYearInterval(time_zone.toDayNum(t / scale_multiplier), years);
}
};
struct ToTimeImpl
{
/// When transforming to time, the date will be equated to 1970-01-02.

View File

@ -27,300 +27,27 @@ namespace ErrorCodes
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; }
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}; }
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
{
@ -336,22 +63,38 @@ public:
const DataTypeInterval * interval_type = nullptr;
bool result_type_is_date = false;
bool result_type_is_datetime = false;
bool result_type_is_datetime_64 = false;
auto check_interval_argument = [&]
{
interval_type = checkAndGetDataType<DataTypeInterval>(arguments[1].type.get());
if (!interval_type)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. "
"Should be an interval of time", arguments[1].type->getName(), getName());
result_type_is_date = (interval_type->getKind() == IntervalKind::Year)
|| (interval_type->getKind() == IntervalKind::Quarter) || (interval_type->getKind() == IntervalKind::Month)
|| (interval_type->getKind() == IntervalKind::Week);
result_type_is_datetime = (interval_type->getKind() == IntervalKind::Day) || (interval_type->getKind() == IntervalKind::Hour)
|| (interval_type->getKind() == IntervalKind::Minute) || (interval_type->getKind() == IntervalKind::Second);
switch (interval_type->getKind())
{
case IntervalKind::Nanosecond:
case IntervalKind::Microsecond:
case IntervalKind::Millisecond:
result_type_is_datetime_64 = true;
break;
case IntervalKind::Second:
case IntervalKind::Minute:
case IntervalKind::Hour:
case IntervalKind::Day:
result_type_is_datetime = true;
break;
case IntervalKind::Week:
case IntervalKind::Month:
case IntervalKind::Quarter:
case IntervalKind::Year:
result_type_is_date = true;
break;
}
};
auto check_timezone_argument = [&]
{
if (!WhichDataType(arguments[2].type).isString())
if (!isString(arguments[2].type))
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",
arguments[2].type->getName(), getName());
@ -384,7 +127,7 @@ public:
return std::make_shared<DataTypeDate>();
else if (result_type_is_datetime)
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false));
else
else if (result_type_is_datetime_64)
{
auto scale = 0;
@ -398,80 +141,66 @@ public:
return std::make_shared<DataTypeDateTime64>(scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false));
}
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];
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);
auto result_column = dispatchForTimeColumn(time_column, interval_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 dispatchForColumns(
ColumnPtr dispatchForTimeColumn(
const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone) const
{
const auto & from_datatype = *time_column.type.get();
const auto which_type = WhichDataType(from_datatype);
if (which_type.isDateTime64())
if (isDateTime64(from_datatype))
{
const auto * time_column_vec = checkAndGetColumn<ColumnDateTime64>(time_column.column.get());
auto scale = assert_cast<const DataTypeDateTime64 &>(from_datatype).getScale();
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDateTime64&>(from_datatype), *time_column_vec, interval_column, result_type, time_zone, scale);
return dispatchForIntervalColumn(assert_cast<const DataTypeDateTime64 &>(from_datatype), *time_column_vec, interval_column, result_type, time_zone, scale);
}
if (which_type.isDateTime())
if (isDateTime(from_datatype))
{
const auto * time_column_vec = checkAndGetColumn<ColumnDateTime>(time_column.column.get());
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDateTime&>(from_datatype), *time_column_vec, interval_column, result_type, time_zone);
return dispatchForIntervalColumn(assert_cast<const DataTypeDateTime &>(from_datatype), *time_column_vec, interval_column, result_type, time_zone);
}
if (which_type.isDate())
if (isDate(from_datatype))
{
const auto * time_column_vec = checkAndGetColumn<ColumnDate>(time_column.column.get());
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDate&>(from_datatype), *time_column_vec, interval_column, result_type, time_zone);
return dispatchForIntervalColumn(assert_cast<const DataTypeDate &>(from_datatype), *time_column_vec, interval_column, result_type, time_zone);
}
if (which_type.isDate32())
if (isDate32(from_datatype))
{
const auto * time_column_vec = checkAndGetColumn<ColumnDate32>(time_column.column.get());
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDate32&>(from_datatype), *time_column_vec, interval_column, result_type, time_zone);
return dispatchForIntervalColumn(assert_cast<const DataTypeDate32 &>(from_datatype), *time_column_vec, interval_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 FromDataType & from, const ColumnType & time_column, const ColumnWithTypeAndName & interval_column,
const TimeDataType & time_data_type, const TimeColumnType & time_column, const ColumnWithTypeAndName & interval_column,
const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale = 1) const
{
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());
@ -479,34 +208,34 @@ private:
switch (interval_type->getKind())
{
case IntervalKind::Nanosecond:
return execute<FromDataType, DataTypeDateTime64, IntervalKind::Nanosecond>(from, time_column, num_units, result_type, time_zone, scale);
return execute<TimeDataType, DataTypeDateTime64, IntervalKind::Nanosecond>(time_data_type, time_column, num_units, result_type, time_zone, scale);
case IntervalKind::Microsecond:
return execute<FromDataType, DataTypeDateTime64, IntervalKind::Microsecond>(from, time_column, num_units, result_type, time_zone, scale);
return execute<TimeDataType, DataTypeDateTime64, IntervalKind::Microsecond>(time_data_type, time_column, num_units, result_type, time_zone, scale);
case IntervalKind::Millisecond:
return execute<FromDataType, DataTypeDateTime64, IntervalKind::Millisecond>(from, time_column, num_units, result_type, time_zone, scale);
return execute<TimeDataType, DataTypeDateTime64, IntervalKind::Millisecond>(time_data_type, time_column, num_units, result_type, time_zone, scale);
case IntervalKind::Second:
return execute<FromDataType, DataTypeDateTime, IntervalKind::Second>(from, time_column, num_units, result_type, time_zone, scale);
return execute<TimeDataType, DataTypeDateTime, IntervalKind::Second>(time_data_type, time_column, num_units, result_type, time_zone, scale);
case IntervalKind::Minute:
return execute<FromDataType, DataTypeDateTime, IntervalKind::Minute>(from, time_column, num_units, result_type, time_zone, scale);
return execute<TimeDataType, DataTypeDateTime, IntervalKind::Minute>(time_data_type, time_column, num_units, result_type, time_zone, scale);
case IntervalKind::Hour:
return execute<FromDataType, DataTypeDateTime, IntervalKind::Hour>(from, time_column, num_units, result_type, time_zone, scale);
return execute<TimeDataType, DataTypeDateTime, IntervalKind::Hour>(time_data_type, time_column, num_units, result_type, time_zone, scale);
case IntervalKind::Day:
return execute<FromDataType, DataTypeDateTime, IntervalKind::Day>(from, time_column, num_units, result_type, time_zone, scale);
return execute<TimeDataType, DataTypeDateTime, IntervalKind::Day>(time_data_type, time_column, num_units, result_type, time_zone, scale);
case IntervalKind::Week:
return execute<FromDataType, DataTypeDate, IntervalKind::Week>(from, time_column, num_units, result_type, time_zone, scale);
return execute<TimeDataType, DataTypeDate, IntervalKind::Week>(time_data_type, time_column, num_units, result_type, time_zone, scale);
case IntervalKind::Month:
return execute<FromDataType, DataTypeDate, IntervalKind::Month>(from, time_column, num_units, result_type, time_zone, scale);
return execute<TimeDataType, DataTypeDate, IntervalKind::Month>(time_data_type, time_column, num_units, result_type, time_zone, scale);
case IntervalKind::Quarter:
return execute<FromDataType, DataTypeDate, IntervalKind::Quarter>(from, time_column, num_units, result_type, time_zone, scale);
return execute<TimeDataType, DataTypeDate, IntervalKind::Quarter>(time_data_type, time_column, num_units, result_type, time_zone, scale);
case IntervalKind::Year:
return execute<FromDataType, DataTypeDate, IntervalKind::Year>(from, time_column, num_units, result_type, time_zone, scale);
return execute<TimeDataType, DataTypeDate, IntervalKind::Year>(time_data_type, time_column, num_units, result_type, time_zone, scale);
}
UNREACHABLE();
}
template <typename FromDataType, typename ToDataType, IntervalKind::Kind unit, typename ColumnType>
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 <typename TimeDataType, typename ToDataType, IntervalKind::Kind unit, typename ColumnType>
ColumnPtr execute(const TimeDataType &, const ColumnType & time_column_type, UInt64 num_units, const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale) const
{
using ToColumnType = typename ToDataType::ColumnType;
using ToFieldType = typename ToDataType::FieldType;
@ -522,8 +251,7 @@ private:
Int64 scale_multiplier = DecimalUtils::scaleMultiplier<DateTime64>(scale);
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] = static_cast<ToFieldType>(ToStartOfInterval<unit>::execute(time_data[i], num_units, time_zone, scale_multiplier));
return result_col;
}