提交 7ecc63fe 编写于 作者: A Alexey Milovidov

Functions for interval arithmetic on DateTime and Date: development [#CLICKHOUSE-2].

上级 7bdbd71d
...@@ -94,6 +94,8 @@ struct ConvertImpl ...@@ -94,6 +94,8 @@ struct ConvertImpl
*/ */
struct ToDateTimeImpl struct ToDateTimeImpl
{ {
static constexpr auto name = "toDateTime";
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
{ {
return time_zone.fromDayNum(DayNum_t(d)); return time_zone.fromDayNum(DayNum_t(d));
...@@ -101,7 +103,7 @@ struct ToDateTimeImpl ...@@ -101,7 +103,7 @@ struct ToDateTimeImpl
}; };
template <typename Name> struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name> template <typename Name> struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name>
: DateTimeTransformImpl<UInt16, UInt32, ToDateTimeImpl, Name> {}; : DateTimeTransformImpl<UInt16, UInt32, ToDateTimeImpl> {};
/// Implementation of toDate function. /// Implementation of toDate function.
...@@ -109,6 +111,8 @@ template <typename Name> struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name ...@@ -109,6 +111,8 @@ template <typename Name> struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name
template <typename FromType, typename ToType> template <typename FromType, typename ToType>
struct ToDateTransform32Or64 struct ToDateTransform32Or64
{ {
static constexpr auto name = "toDate";
static inline ToType execute(const FromType & from, const DateLUTImpl & time_zone) static inline ToType execute(const FromType & from, const DateLUTImpl & time_zone)
{ {
return (from < 0xFFFF) ? from : time_zone.toDayNum(from); return (from < 0xFFFF) ? from : time_zone.toDayNum(from);
...@@ -118,7 +122,7 @@ struct ToDateTransform32Or64 ...@@ -118,7 +122,7 @@ struct ToDateTransform32Or64
/** Conversion of DateTime to Date: throw off time component. /** Conversion of DateTime to Date: throw off time component.
*/ */
template <typename Name> struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name> template <typename Name> struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name>
: DateTimeTransformImpl<UInt32, UInt16, ToDateImpl, Name> {}; : DateTimeTransformImpl<UInt32, UInt16, ToDateImpl> {};
/** Special case of converting (U)Int32 or (U)Int64 (and also, for convenience, Float32, Float64) to Date. /** Special case of converting (U)Int32 or (U)Int64 (and also, for convenience, Float32, Float64) to Date.
* If number is less than 65536, then it is treated as DayNum, and if greater or equals, then as unix timestamp. * If number is less than 65536, then it is treated as DayNum, and if greater or equals, then as unix timestamp.
...@@ -128,17 +132,17 @@ template <typename Name> struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name ...@@ -128,17 +132,17 @@ template <typename Name> struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name
* (otherwise such usage would be frequent mistake). * (otherwise such usage would be frequent mistake).
*/ */
template <typename Name> struct ConvertImpl<DataTypeUInt32, DataTypeDate, Name> template <typename Name> struct ConvertImpl<DataTypeUInt32, DataTypeDate, Name>
: DateTimeTransformImpl<UInt32, UInt16, ToDateTransform32Or64<UInt32, UInt16>, Name> {}; : DateTimeTransformImpl<UInt32, UInt16, ToDateTransform32Or64<UInt32, UInt16>> {};
template <typename Name> struct ConvertImpl<DataTypeUInt64, DataTypeDate, Name> template <typename Name> struct ConvertImpl<DataTypeUInt64, DataTypeDate, Name>
: DateTimeTransformImpl<UInt64, UInt16, ToDateTransform32Or64<UInt64, UInt16>, Name> {}; : DateTimeTransformImpl<UInt64, UInt16, ToDateTransform32Or64<UInt64, UInt16>> {};
template <typename Name> struct ConvertImpl<DataTypeInt32, DataTypeDate, Name> template <typename Name> struct ConvertImpl<DataTypeInt32, DataTypeDate, Name>
: DateTimeTransformImpl<Int32, UInt16, ToDateTransform32Or64<Int32, UInt16>, Name> {}; : DateTimeTransformImpl<Int32, UInt16, ToDateTransform32Or64<Int32, UInt16>> {};
template <typename Name> struct ConvertImpl<DataTypeInt64, DataTypeDate, Name> template <typename Name> struct ConvertImpl<DataTypeInt64, DataTypeDate, Name>
: DateTimeTransformImpl<Int64, UInt16, ToDateTransform32Or64<Int64, UInt16>, Name> {}; : DateTimeTransformImpl<Int64, UInt16, ToDateTransform32Or64<Int64, UInt16>> {};
template <typename Name> struct ConvertImpl<DataTypeFloat32, DataTypeDate, Name> template <typename Name> struct ConvertImpl<DataTypeFloat32, DataTypeDate, Name>
: DateTimeTransformImpl<Float32, UInt16, ToDateTransform32Or64<Float32, UInt16>, Name> {}; : DateTimeTransformImpl<Float32, UInt16, ToDateTransform32Or64<Float32, UInt16>> {};
template <typename Name> struct ConvertImpl<DataTypeFloat64, DataTypeDate, Name> template <typename Name> struct ConvertImpl<DataTypeFloat64, DataTypeDate, Name>
: DateTimeTransformImpl<Float64, UInt16, ToDateTransform32Or64<Float64, UInt16>, Name> {}; : DateTimeTransformImpl<Float64, UInt16, ToDateTransform32Or64<Float64, UInt16>> {};
/** Transformation of numbers, dates, datetimes to strings: through formatting. /** Transformation of numbers, dates, datetimes to strings: through formatting.
......
...@@ -37,6 +37,13 @@ void registerFunctionsDateTime(FunctionFactory & factory) ...@@ -37,6 +37,13 @@ void registerFunctionsDateTime(FunctionFactory & factory)
factory.registerFunction<FunctionToYYYYMM>(); factory.registerFunction<FunctionToYYYYMM>();
factory.registerFunction<FunctionToYYYYMMDD>(); factory.registerFunction<FunctionToYYYYMMDD>();
factory.registerFunction<FunctionToYYYYMMDDhhmmss>(); factory.registerFunction<FunctionToYYYYMMDDhhmmss>();
factory.registerFunction<FunctionAddSeconds>();
factory.registerFunction<FunctionAddMinutes>();
factory.registerFunction<FunctionAddHours>();
factory.registerFunction<FunctionAddDays>();
factory.registerFunction<FunctionAddWeeks>();
factory.registerFunction<FunctionAddMonths>();
factory.registerFunction<FunctionAddYears>();
} }
} }
...@@ -85,6 +85,8 @@ struct ZeroTransform ...@@ -85,6 +85,8 @@ struct ZeroTransform
struct ToDateImpl struct ToDateImpl
{ {
static constexpr auto name = "toDate";
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return UInt16(time_zone.toDayNum(t)); return UInt16(time_zone.toDayNum(t));
...@@ -99,6 +101,8 @@ struct ToDateImpl ...@@ -99,6 +101,8 @@ struct ToDateImpl
struct ToStartOfDayImpl struct ToStartOfDayImpl
{ {
static constexpr auto name = "toStartOfDay";
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toDate(t); return time_zone.toDate(t);
...@@ -113,6 +117,8 @@ struct ToStartOfDayImpl ...@@ -113,6 +117,8 @@ struct ToStartOfDayImpl
struct ToMondayImpl struct ToMondayImpl
{ {
static constexpr auto name = "toMonday";
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t)); return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t));
...@@ -127,6 +133,8 @@ struct ToMondayImpl ...@@ -127,6 +133,8 @@ struct ToMondayImpl
struct ToStartOfMonthImpl struct ToStartOfMonthImpl
{ {
static constexpr auto name = "toStartOfMonth";
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toFirstDayNumOfMonth(time_zone.toDayNum(t)); return time_zone.toFirstDayNumOfMonth(time_zone.toDayNum(t));
...@@ -141,6 +149,8 @@ struct ToStartOfMonthImpl ...@@ -141,6 +149,8 @@ struct ToStartOfMonthImpl
struct ToStartOfQuarterImpl struct ToStartOfQuarterImpl
{ {
static constexpr auto name = "toStartOfQuarter";
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toFirstDayNumOfQuarter(time_zone.toDayNum(t)); return time_zone.toFirstDayNumOfQuarter(time_zone.toDayNum(t));
...@@ -155,6 +165,8 @@ struct ToStartOfQuarterImpl ...@@ -155,6 +165,8 @@ struct ToStartOfQuarterImpl
struct ToStartOfYearImpl struct ToStartOfYearImpl
{ {
static constexpr auto name = "toStartOfYear";
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toFirstDayNumOfYear(time_zone.toDayNum(t)); return time_zone.toFirstDayNumOfYear(time_zone.toDayNum(t));
...@@ -170,6 +182,8 @@ struct ToStartOfYearImpl ...@@ -170,6 +182,8 @@ struct ToStartOfYearImpl
struct ToTimeImpl struct ToTimeImpl
{ {
static constexpr auto name = "toTime";
/// When transforming to time, the date will be equated to 1970-01-02. /// When transforming to time, the date will be equated to 1970-01-02.
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
...@@ -186,6 +200,8 @@ struct ToTimeImpl ...@@ -186,6 +200,8 @@ struct ToTimeImpl
struct ToStartOfMinuteImpl struct ToStartOfMinuteImpl
{ {
static constexpr auto name = "toStartOfMinute";
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toStartOfMinute(t); return time_zone.toStartOfMinute(t);
...@@ -200,6 +216,8 @@ struct ToStartOfMinuteImpl ...@@ -200,6 +216,8 @@ struct ToStartOfMinuteImpl
struct ToStartOfFiveMinuteImpl struct ToStartOfFiveMinuteImpl
{ {
static constexpr auto name = "toStartOfFiveMinute";
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toStartOfFiveMinute(t); return time_zone.toStartOfFiveMinute(t);
...@@ -214,6 +232,8 @@ struct ToStartOfFiveMinuteImpl ...@@ -214,6 +232,8 @@ struct ToStartOfFiveMinuteImpl
struct ToStartOfHourImpl struct ToStartOfHourImpl
{ {
static constexpr auto name = "toStartOfHour";
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toStartOfHour(t); return time_zone.toStartOfHour(t);
...@@ -229,6 +249,8 @@ struct ToStartOfHourImpl ...@@ -229,6 +249,8 @@ struct ToStartOfHourImpl
struct ToYearImpl struct ToYearImpl
{ {
static constexpr auto name = "toYear";
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toYear(t); return time_zone.toYear(t);
...@@ -243,6 +265,8 @@ struct ToYearImpl ...@@ -243,6 +265,8 @@ struct ToYearImpl
struct ToMonthImpl struct ToMonthImpl
{ {
static constexpr auto name = "toMonth";
static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toMonth(t); return time_zone.toMonth(t);
...@@ -257,6 +281,8 @@ struct ToMonthImpl ...@@ -257,6 +281,8 @@ struct ToMonthImpl
struct ToDayOfMonthImpl struct ToDayOfMonthImpl
{ {
static constexpr auto name = "toDayOfMonth";
static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toDayOfMonth(t); return time_zone.toDayOfMonth(t);
...@@ -271,6 +297,8 @@ struct ToDayOfMonthImpl ...@@ -271,6 +297,8 @@ struct ToDayOfMonthImpl
struct ToDayOfWeekImpl struct ToDayOfWeekImpl
{ {
static constexpr auto name = "toDayOfWeek";
static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toDayOfWeek(t); return time_zone.toDayOfWeek(t);
...@@ -285,6 +313,8 @@ struct ToDayOfWeekImpl ...@@ -285,6 +313,8 @@ struct ToDayOfWeekImpl
struct ToHourImpl struct ToHourImpl
{ {
static constexpr auto name = "toHour";
static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toHour(t); return time_zone.toHour(t);
...@@ -300,6 +330,8 @@ struct ToHourImpl ...@@ -300,6 +330,8 @@ struct ToHourImpl
struct ToMinuteImpl struct ToMinuteImpl
{ {
static constexpr auto name = "toMinute";
static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toMinute(t); return time_zone.toMinute(t);
...@@ -314,6 +346,8 @@ struct ToMinuteImpl ...@@ -314,6 +346,8 @@ struct ToMinuteImpl
struct ToSecondImpl struct ToSecondImpl
{ {
static constexpr auto name = "toSecond";
static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toSecond(t); return time_zone.toSecond(t);
...@@ -328,6 +362,8 @@ struct ToSecondImpl ...@@ -328,6 +362,8 @@ struct ToSecondImpl
struct ToRelativeYearNumImpl struct ToRelativeYearNumImpl
{ {
static constexpr auto name = "toRelativeYearNum";
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toYear(t); return time_zone.toYear(t);
...@@ -342,6 +378,8 @@ struct ToRelativeYearNumImpl ...@@ -342,6 +378,8 @@ struct ToRelativeYearNumImpl
struct ToRelativeMonthNumImpl struct ToRelativeMonthNumImpl
{ {
static constexpr auto name = "toRelativeMonthNum";
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toRelativeMonthNum(t); return time_zone.toRelativeMonthNum(t);
...@@ -356,6 +394,8 @@ struct ToRelativeMonthNumImpl ...@@ -356,6 +394,8 @@ struct ToRelativeMonthNumImpl
struct ToRelativeWeekNumImpl struct ToRelativeWeekNumImpl
{ {
static constexpr auto name = "toRelativeWeekNum";
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toRelativeWeekNum(t); return time_zone.toRelativeWeekNum(t);
...@@ -370,6 +410,8 @@ struct ToRelativeWeekNumImpl ...@@ -370,6 +410,8 @@ struct ToRelativeWeekNumImpl
struct ToRelativeDayNumImpl struct ToRelativeDayNumImpl
{ {
static constexpr auto name = "toRelativeDayNum";
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toDayNum(t); return time_zone.toDayNum(t);
...@@ -385,6 +427,8 @@ struct ToRelativeDayNumImpl ...@@ -385,6 +427,8 @@ struct ToRelativeDayNumImpl
struct ToRelativeHourNumImpl struct ToRelativeHourNumImpl
{ {
static constexpr auto name = "toRelativeHourNum";
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toRelativeHourNum(t); return time_zone.toRelativeHourNum(t);
...@@ -399,6 +443,8 @@ struct ToRelativeHourNumImpl ...@@ -399,6 +443,8 @@ struct ToRelativeHourNumImpl
struct ToRelativeMinuteNumImpl struct ToRelativeMinuteNumImpl
{ {
static constexpr auto name = "toRelativeMinuteNum";
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toRelativeMinuteNum(t); return time_zone.toRelativeMinuteNum(t);
...@@ -413,6 +459,8 @@ struct ToRelativeMinuteNumImpl ...@@ -413,6 +459,8 @@ struct ToRelativeMinuteNumImpl
struct ToRelativeSecondNumImpl struct ToRelativeSecondNumImpl
{ {
static constexpr auto name = "toRelativeSecondNum";
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return t; return t;
...@@ -427,6 +475,8 @@ struct ToRelativeSecondNumImpl ...@@ -427,6 +475,8 @@ struct ToRelativeSecondNumImpl
struct ToYYYYMMImpl struct ToYYYYMMImpl
{ {
static constexpr auto name = "toYYYYMM";
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toNumYYYYMM(t); return time_zone.toNumYYYYMM(t);
...@@ -441,6 +491,8 @@ struct ToYYYYMMImpl ...@@ -441,6 +491,8 @@ struct ToYYYYMMImpl
struct ToYYYYMMDDImpl struct ToYYYYMMDDImpl
{ {
static constexpr auto name = "toYYYYMMDD";
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toNumYYYYMMDD(t); return time_zone.toNumYYYYMMDD(t);
...@@ -455,6 +507,8 @@ struct ToYYYYMMDDImpl ...@@ -455,6 +507,8 @@ struct ToYYYYMMDDImpl
struct ToYYYYMMDDhhmmssImpl struct ToYYYYMMDDhhmmssImpl
{ {
static constexpr auto name = "toYYYYMMDDhhmmss";
static inline UInt64 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt64 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toNumYYYYMMDDhhmmss(t); return time_zone.toNumYYYYMMDDhhmmss(t);
...@@ -479,24 +533,16 @@ struct Transformer ...@@ -479,24 +533,16 @@ struct Transformer
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
vec_to[i] = Transform::execute(vec_from[i], time_zone); vec_to[i] = Transform::execute(vec_from[i], time_zone);
} }
static void constant(const FromType & from, ToType & to, const DateLUTImpl & time_zone)
{
to = Transform::execute(from, time_zone);
}
}; };
template <typename FromType, typename ToType, typename Transform, typename Name> template <typename FromType, typename ToType, typename Transform>
struct DateTimeTransformImpl struct DateTimeTransformImpl
{ {
static void execute(Block & block, const ColumnNumbers & arguments, size_t result) static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
{ {
using Op = Transformer<FromType, ToType, Transform>; using Op = Transformer<FromType, ToType, Transform>;
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
const auto * sources = checkAndGetColumn<ColumnVector<FromType>>(source_col.get());
const ColumnConst * time_zone_column = nullptr; const ColumnConst * time_zone_column = nullptr;
if (arguments.size() == 2) if (arguments.size() == 2)
...@@ -505,7 +551,7 @@ struct DateTimeTransformImpl ...@@ -505,7 +551,7 @@ struct DateTimeTransformImpl
if (!time_zone_column) if (!time_zone_column)
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName() throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
+ " of second (time zone) argument of function " + Name::name + ", must be constant string", + " of second (time zone) argument of function " + Transform::name + ", must be constant string",
ErrorCodes::ILLEGAL_COLUMN); ErrorCodes::ILLEGAL_COLUMN);
} }
...@@ -513,7 +559,8 @@ struct DateTimeTransformImpl ...@@ -513,7 +559,8 @@ struct DateTimeTransformImpl
? DateLUT::instance(time_zone_column->getValue<String>()) ? DateLUT::instance(time_zone_column->getValue<String>())
: DateLUT::instance(); : DateLUT::instance();
if (sources) const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
if (const auto * sources = checkAndGetColumn<ColumnVector<FromType>>(source_col.get()))
{ {
auto col_to = std::make_shared<ColumnVector<ToType>>(); auto col_to = std::make_shared<ColumnVector<ToType>>();
block.getByPosition(result).column = col_to; block.getByPosition(result).column = col_to;
...@@ -522,18 +569,18 @@ struct DateTimeTransformImpl ...@@ -522,18 +569,18 @@ struct DateTimeTransformImpl
else else
{ {
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of first argument of function " + Name::name, + " of first argument of function " + Transform::name,
ErrorCodes::ILLEGAL_COLUMN); ErrorCodes::ILLEGAL_COLUMN);
} }
} }
}; };
template <typename ToDataType, typename Transform, typename Name> template <typename ToDataType, typename Transform>
class FunctionDateOrDateTimeToSomething : public IFunction class FunctionDateOrDateTimeToSomething : public IFunction
{ {
public: public:
static constexpr auto name = Name::name; static constexpr auto name = Transform::name;
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionDateOrDateTimeToSomething>(); }; static FunctionPtr create(const Context & context) { return std::make_shared<FunctionDateOrDateTimeToSomething>(); };
String getName() const override String getName() const override
...@@ -578,12 +625,12 @@ public: ...@@ -578,12 +625,12 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{ {
IDataType * from_type = block.getByPosition(arguments[0]).type.get(); const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
if (checkDataType<DataTypeDate>(from_type)) if (checkDataType<DataTypeDate>(from_type))
DateTimeTransformImpl<DataTypeDate::FieldType, typename ToDataType::FieldType, Transform, Name>::execute(block, arguments, result); DateTimeTransformImpl<DataTypeDate::FieldType, typename ToDataType::FieldType, Transform>::execute(block, arguments, result);
else if (checkDataType<DataTypeDateTime>(from_type)) else if (checkDataType<DataTypeDateTime>(from_type))
DateTimeTransformImpl<DataTypeDateTime::FieldType, typename ToDataType::FieldType, Transform, Name>::execute(block, arguments, result); DateTimeTransformImpl<DataTypeDateTime::FieldType, typename ToDataType::FieldType, Transform>::execute(block, arguments, result);
else else
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
...@@ -630,6 +677,272 @@ public: ...@@ -630,6 +677,272 @@ public:
}; };
struct AddSecondsImpl
{
static constexpr auto name = "addSeconds";
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
{
return t + delta;
}
static inline UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.fromDayNum(DayNum_t(d)) + delta;
}
};
struct AddMinutesImpl
{
static constexpr auto name = "addMinutes";
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
{
return t + delta * 60;
}
static inline UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.fromDayNum(DayNum_t(d)) + delta * 60;
}
};
struct AddHoursImpl
{
static constexpr auto name = "addHours";
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
{
return t + delta * 3600;
}
static inline UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.fromDayNum(DayNum_t(d)) + delta * 3600;
}
};
struct AddDaysImpl
{
static constexpr auto name = "addDays";
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.addDays(t, delta);
}
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
{
return d + delta;
}
};
struct AddWeeksImpl
{
static constexpr auto name = "addWeeks";
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.addWeeks(t, delta);
}
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
{
return d + delta * 7;
}
};
struct AddMonthsImpl
{
static constexpr auto name = "addMonths";
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.addMonths(t, delta);
}
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.addMonths(DayNum_t(d), delta);
}
};
struct AddYearsImpl
{
static constexpr auto name = "addYears";
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.addYears(t, delta);
}
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.addYears(DayNum_t(d), delta);
}
};
template <typename FromType, typename ToType, typename Transform>
struct Adder
{
static void vector_vector(const PaddedPODArray<FromType> & vec_from, PaddedPODArray<ToType> & vec_to, const IColumn & delta, const DateLUTImpl & time_zone)
{
size_t size = vec_from.size();
vec_to.resize(size);
for (size_t i = 0; i < size; ++i)
vec_to[i] = Transform::execute(vec_from[i], delta.getInt(i), time_zone);
}
static void vector_constant(const PaddedPODArray<FromType> & vec_from, PaddedPODArray<ToType> & vec_to, Int64 delta, const DateLUTImpl & time_zone)
{
size_t size = vec_from.size();
vec_to.resize(size);
for (size_t i = 0; i < size; ++i)
vec_to[i] = Transform::execute(vec_from[i], delta, time_zone);
}
static void constant_vector(const FromType & from, PaddedPODArray<ToType> & vec_to, const IColumn & delta, const DateLUTImpl & time_zone)
{
size_t size = delta.size();
vec_to.resize(size);
for (size_t i = 0; i < size; ++i)
vec_to[i] = Transform::execute(from, delta.getInt(i), time_zone);
}
};
template <typename FromType, typename Transform>
struct DateTimeAddIntervalImpl
{
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
{
using ToType = decltype(Transform::execute(FromType(), 0, std::declval<DateLUTImpl>()));
using Op = Adder<FromType, ToType, Transform>;
const ColumnConst * time_zone_column = nullptr;
if (arguments.size() == 3)
{
time_zone_column = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[2]).column.get());
if (!time_zone_column)
throw Exception("Illegal column " + block.getByPosition(arguments[2]).column->getName()
+ " of second (time zone) argument of function " + Transform::name + ", must be constant string",
ErrorCodes::ILLEGAL_COLUMN);
}
const DateLUTImpl & time_zone = time_zone_column
? DateLUT::instance(time_zone_column->getValue<String>())
: DateLUT::instance();
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
if (const auto * sources = checkAndGetColumn<ColumnVector<FromType>>(source_col.get()))
{
auto col_to = std::make_shared<ColumnVector<ToType>>();
block.getByPosition(result).column = col_to;
const IColumn & delta_column = *block.getByPosition(arguments[1]).column;
if (const auto * delta_const_column = typeid_cast<const ColumnConst *>(&delta_column))
Op::vector_constant(sources->getData(), col_to->getData(), delta_const_column->getValue<Int64>(), time_zone);
else
Op::vector_vector(sources->getData(), col_to->getData(), delta_column, time_zone);
}
else if (const auto * sources = checkAndGetColumnConst<ColumnVector<FromType>>(source_col.get()))
{
auto col_to = std::make_shared<ColumnVector<ToType>>();
block.getByPosition(result).column = col_to;
Op::constant_vector(sources->template getValue<FromType>(), col_to->getData(), *block.getByPosition(arguments[1]).column, time_zone);
}
else
{
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of first argument of function " + Transform::name,
ErrorCodes::ILLEGAL_COLUMN);
}
}
};
template <typename Transform>
class FunctionDateOrDateTimeAddInterval : public IFunction
{
public:
static constexpr auto name = Transform::name;
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionDateOrDateTimeAddInterval>(); };
String getName() const override
{
return name;
}
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 2 && arguments.size() != 3)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 2 or 3",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!arguments[1]->behavesAsNumber())
throw Exception("Second argument for function " + getName() + " (delta) must be number",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (arguments.size() == 2)
{
if (!checkDataType<DataTypeDate>(arguments[0].get())
&& !checkDataType<DataTypeDateTime>(arguments[0].get()))
throw Exception{
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName() +
". Should be a date or a date with time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
else
{
if (!checkDataType<DataTypeDateTime>(arguments[0].get())
|| !checkDataType<DataTypeString>(arguments[1].get()))
throw Exception{
"Function " + getName() + " supports 2 or 3 arguments. The 1st argument "
"must be of type Date or DateTime. The 2nd argument must be number. "
"The 3rd argument (optional) must be "
"a constant string with timezone name. The timezone argument is allowed "
"only when the 1st argument has the type DateTime",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
if (checkDataType<DataTypeDate>(arguments[0].get()))
return std::make_shared<typename std::conditional<std::is_same<
decltype(Transform::execute(DataTypeDate::FieldType(), 0, std::declval<DateLUTImpl>())), UInt16>::value,
DataTypeDate, DataTypeDateTime>::type>();
else
return std::make_shared<typename std::conditional<std::is_same<
decltype(Transform::execute(DataTypeDateTime::FieldType(), 0, std::declval<DateLUTImpl>())), UInt16>::value,
DataTypeDate, DataTypeDateTime>::type>();
}
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {2}; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
if (checkDataType<DataTypeDate>(from_type))
DateTimeAddIntervalImpl<DataTypeDate::FieldType, Transform>::execute(block, arguments, result);
else if (checkDataType<DataTypeDateTime>(from_type))
DateTimeAddIntervalImpl<DataTypeDateTime::FieldType, Transform>::execute(block, arguments, result);
else
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
};
/// Get the current time. (It is a constant, it is evaluated once for the entire query.) /// Get the current time. (It is a constant, it is evaluated once for the entire query.)
class FunctionNow : public IFunction class FunctionNow : public IFunction
{ {
...@@ -908,62 +1221,42 @@ public: ...@@ -908,62 +1221,42 @@ public:
}; };
struct NameToYear { static constexpr auto name = "toYear"; }; using FunctionToYear = FunctionDateOrDateTimeToSomething<DataTypeUInt16, ToYearImpl>;
struct NameToMonth { static constexpr auto name = "toMonth"; }; using FunctionToMonth = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToMonthImpl>;
struct NameToDayOfMonth { static constexpr auto name = "toDayOfMonth"; }; using FunctionToDayOfMonth = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToDayOfMonthImpl>;
struct NameToDayOfWeek { static constexpr auto name = "toDayOfWeek"; }; using FunctionToDayOfWeek = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToDayOfWeekImpl>;
struct NameToHour { static constexpr auto name = "toHour"; }; using FunctionToHour = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToHourImpl>;
struct NameToMinute { static constexpr auto name = "toMinute"; }; using FunctionToMinute = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToMinuteImpl>;
struct NameToSecond { static constexpr auto name = "toSecond"; }; using FunctionToSecond = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToSecondImpl>;
struct NameToStartOfDay { static constexpr auto name = "toStartOfDay"; }; using FunctionToStartOfDay = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfDayImpl>;
struct NameToMonday { static constexpr auto name = "toMonday"; }; using FunctionToMonday = FunctionDateOrDateTimeToSomething<DataTypeDate, ToMondayImpl>;
struct NameToStartOfMonth { static constexpr auto name = "toStartOfMonth"; }; using FunctionToStartOfMonth = FunctionDateOrDateTimeToSomething<DataTypeDate, ToStartOfMonthImpl>;
struct NameToStartOfQuarter { static constexpr auto name = "toStartOfQuarter"; }; using FunctionToStartOfQuarter = FunctionDateOrDateTimeToSomething<DataTypeDate, ToStartOfQuarterImpl>;
struct NameToStartOfYear { static constexpr auto name = "toStartOfYear"; }; using FunctionToStartOfYear = FunctionDateOrDateTimeToSomething<DataTypeDate, ToStartOfYearImpl>;
struct NameToStartOfMinute { static constexpr auto name = "toStartOfMinute"; }; using FunctionToStartOfMinute = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfMinuteImpl>;
struct NameToStartOfFiveMinute { static constexpr auto name = "toStartOfFiveMinute"; }; using FunctionToStartOfFiveMinute = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfFiveMinuteImpl>;
struct NameToStartOfHour { static constexpr auto name = "toStartOfHour"; }; using FunctionToStartOfHour = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfHourImpl>;
struct NameToTime { static constexpr auto name = "toTime"; }; using FunctionToTime = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToTimeImpl>;
struct NameToRelativeYearNum { static constexpr auto name = "toRelativeYearNum"; };
struct NameToRelativeMonthNum { static constexpr auto name = "toRelativeMonthNum"; }; using FunctionToRelativeYearNum = FunctionDateOrDateTimeToSomething<DataTypeUInt16, ToRelativeYearNumImpl>;
struct NameToRelativeWeekNum { static constexpr auto name = "toRelativeWeekNum"; }; using FunctionToRelativeMonthNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeMonthNumImpl>;
struct NameToRelativeDayNum { static constexpr auto name = "toRelativeDayNum"; }; using FunctionToRelativeWeekNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeWeekNumImpl>;
struct NameToRelativeHourNum { static constexpr auto name = "toRelativeHourNum"; }; using FunctionToRelativeDayNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeDayNumImpl>;
struct NameToRelativeMinuteNum { static constexpr auto name = "toRelativeMinuteNum"; };
struct NameToRelativeSecondNum { static constexpr auto name = "toRelativeSecondNum"; }; using FunctionToRelativeHourNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeHourNumImpl>;
struct NameToYYYYMM { static constexpr auto name = "toYYYYMM"; }; using FunctionToRelativeMinuteNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeMinuteNumImpl>;
struct NameToYYYYMMDD { static constexpr auto name = "toYYYYMMDD"; }; using FunctionToRelativeSecondNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeSecondNumImpl>;
struct NameToYYYYMMDDhhmmss { static constexpr auto name = "toYYYYMMDDhhmmss"; };
using FunctionToYYYYMM = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToYYYYMMImpl>;
using FunctionToYYYYMMDD = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToYYYYMMDDImpl>;
using FunctionToYear = FunctionDateOrDateTimeToSomething<DataTypeUInt16, ToYearImpl, NameToYear>; using FunctionToYYYYMMDDhhmmss = FunctionDateOrDateTimeToSomething<DataTypeUInt64, ToYYYYMMDDhhmmssImpl>;
using FunctionToMonth = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToMonthImpl, NameToMonth>;
using FunctionToDayOfMonth = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToDayOfMonthImpl, NameToDayOfMonth>; using FunctionAddSeconds = FunctionDateOrDateTimeAddInterval<AddSecondsImpl>;
using FunctionToDayOfWeek = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToDayOfWeekImpl, NameToDayOfWeek>; using FunctionAddMinutes = FunctionDateOrDateTimeAddInterval<AddMinutesImpl>;
using FunctionToHour = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToHourImpl, NameToHour>; using FunctionAddHours = FunctionDateOrDateTimeAddInterval<AddHoursImpl>;
using FunctionToMinute = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToMinuteImpl, NameToMinute>; using FunctionAddDays = FunctionDateOrDateTimeAddInterval<AddDaysImpl>;
using FunctionToSecond = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToSecondImpl, NameToSecond>; using FunctionAddWeeks = FunctionDateOrDateTimeAddInterval<AddWeeksImpl>;
using FunctionToStartOfDay = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfDayImpl, NameToStartOfDay>; using FunctionAddMonths = FunctionDateOrDateTimeAddInterval<AddMonthsImpl>;
using FunctionToMonday = FunctionDateOrDateTimeToSomething<DataTypeDate, ToMondayImpl, NameToMonday>; using FunctionAddYears = FunctionDateOrDateTimeAddInterval<AddYearsImpl>;
using FunctionToStartOfMonth = FunctionDateOrDateTimeToSomething<DataTypeDate, ToStartOfMonthImpl, NameToStartOfMonth>;
using FunctionToStartOfQuarter = FunctionDateOrDateTimeToSomething<DataTypeDate, ToStartOfQuarterImpl, NameToStartOfQuarter>;
using FunctionToStartOfYear = FunctionDateOrDateTimeToSomething<DataTypeDate, ToStartOfYearImpl, NameToStartOfYear>;
using FunctionToStartOfMinute = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfMinuteImpl, NameToStartOfMinute>;
using FunctionToStartOfFiveMinute = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfFiveMinuteImpl, NameToStartOfFiveMinute>;
using FunctionToStartOfHour = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfHourImpl, NameToStartOfHour>;
using FunctionToTime = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToTimeImpl, NameToTime>;
using FunctionToRelativeYearNum = FunctionDateOrDateTimeToSomething<DataTypeUInt16, ToRelativeYearNumImpl, NameToRelativeYearNum>;
using FunctionToRelativeMonthNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeMonthNumImpl, NameToRelativeMonthNum>;
using FunctionToRelativeWeekNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeWeekNumImpl, NameToRelativeWeekNum>;
using FunctionToRelativeDayNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeDayNumImpl, NameToRelativeDayNum>;
using FunctionToRelativeHourNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeHourNumImpl, NameToRelativeHourNum>;
using FunctionToRelativeMinuteNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeMinuteNumImpl, NameToRelativeMinuteNum>;
using FunctionToRelativeSecondNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeSecondNumImpl, NameToRelativeSecondNum>;
using FunctionToYYYYMM = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToYYYYMMImpl, NameToYYYYMM>;
using FunctionToYYYYMMDD = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToYYYYMMDDImpl, NameToYYYYMMDD>;
using FunctionToYYYYMMDDhhmmss = FunctionDateOrDateTimeToSomething<DataTypeUInt64, ToYYYYMMDDhhmmssImpl, NameToYYYYMMDDhhmmss>;
} }
...@@ -44,6 +44,10 @@ public: ...@@ -44,6 +44,10 @@ public:
UInt8 day_of_month; UInt8 day_of_month;
UInt8 day_of_week; UInt8 day_of_week;
/// Total number of days in current month. Actually we can use separate table that is independent of time zone.
/// But due to alignment, this field is totally zero cost.
UInt8 days_in_month;
/// For days, when offset from UTC was changed due to daylight saving time or permanent change, following values could be non zero. /// For days, when offset from UTC was changed due to daylight saving time or permanent change, following values could be non zero.
UInt16 time_at_offset_change; /// In seconds from beginning of the day. Assuming offset never changed close to the end of day (so, value < 65536). UInt16 time_at_offset_change; /// In seconds from beginning of the day. Assuming offset never changed close to the end of day (so, value < 65536).
Int16 amount_of_offset_change; /// Usually -3600 or 3600, but look at Lord Howe Island. Int16 amount_of_offset_change; /// Usually -3600 or 3600, but look at Lord Howe Island.
...@@ -217,13 +221,20 @@ public: ...@@ -217,13 +221,20 @@ public:
return lut[index - (lut[index].day_of_month - 1)].date; return lut[index - (lut[index].day_of_month - 1)].date;
} }
inline size_t daysInMonth(DayNum_t d) const
{
return lut[d].days_in_month;
}
inline size_t daysInMonth(time_t t) const inline size_t daysInMonth(time_t t) const
{ {
size_t today = findIndex(t); return find(t).days_in_month;
size_t start_of_month = today - (lut[today].day_of_month - 1); }
size_t next_month = start_of_month + 31;
size_t start_of_next_month = next_month - (lut[next_month].day_of_month - 1); inline size_t daysInMonth(short year, char month) const
return start_of_next_month - start_of_month; {
auto any_day_of_month = years_lut[year - DATE_LUT_MIN_YEAR] + 31 * (month - 1);
return lut[any_day_of_month].days_in_month;
} }
/** Round to start of day, then shift for specified amount of days. /** Round to start of day, then shift for specified amount of days.
...@@ -445,6 +456,122 @@ public: ...@@ -445,6 +456,122 @@ public:
num % 100); num % 100);
} }
/// Adding calendar intervals.
/// Implementation specific behaviour when delta is too big.
inline time_t addDays(time_t t, ssize_t delta) const
{
size_t index = findIndex(t);
time_t time_offset = toHour(t) * 3600 + toMinute(t) * 60 + toSecond(t);
index += delta;
if (time_offset >= lut[index].time_at_offset_change)
time_offset -= lut[index].amount_of_offset_change;
return lut[index].date + time_offset;
}
inline time_t addWeeks(time_t t, ssize_t delta) const
{
return addDays(t, delta * 7);
}
inline char saturateDayOfMonth(short year, char month, char day_of_month) const
{
if (likely(day_of_month <= 28))
return day_of_month;
auto days_in_month = daysInMonth(year, month);
if (day_of_month > days_in_month)
day_of_month = days_in_month;
return day_of_month;
}
/// If resulting month has less deys than source month, then saturation can happen.
/// Example: 31 Aug + 1 month = 30 Sep.
inline time_t addMonths(time_t t, ssize_t delta) const
{
size_t index = findIndex(t);
const Values & values = lut[index];
time_t time_offset = toHour(t) * 3600 + toMinute(t) * 60 + toSecond(t);
auto month = values.month + delta;
bool year_will_be_next = month > 12;
if (year_will_be_next)
month -= 12;
auto year = values.year + year_will_be_next;
auto day_of_month = saturateDayOfMonth(year, month, values.day_of_month);
DayNum_t result_day = makeDayNum(year, month, day_of_month);
if (time_offset >= lut[result_day].time_at_offset_change)
time_offset -= lut[result_day].amount_of_offset_change;
return lut[result_day].date + time_offset;
}
inline DayNum_t addMonths(DayNum_t d, ssize_t delta) const
{
const Values & values = lut[d];
auto month = values.month + delta;
bool year_will_be_next = month > 12;
if (year_will_be_next)
month -= 12;
auto year = values.year + year_will_be_next;
auto day_of_month = saturateDayOfMonth(year, month, values.day_of_month);
return makeDayNum(year, month, day_of_month);
}
/// Saturation can occur if 29 Feb is mapped to non-leap year.
inline time_t addYears(time_t t, ssize_t delta) const
{
size_t index = findIndex(t);
const Values & values = lut[index];
time_t time_offset = toHour(t) * 3600 + toMinute(t) * 60 + toSecond(t);
auto year = values.year + delta;
auto month = values.month;
auto day_of_month = values.day_of_month;
/// Saturation to 28 Feb can happen.
if (unlikely(day_of_month == 29 && month == 2))
day_of_month = saturateDayOfMonth(year, month, day_of_month);
DayNum_t result_day = makeDayNum(year, month, day_of_month);
if (time_offset >= lut[result_day].time_at_offset_change)
time_offset -= lut[result_day].amount_of_offset_change;
return lut[result_day].date + time_offset;
}
inline DayNum_t addYears(DayNum_t d, ssize_t delta) const
{
const Values & values = lut[d];
auto year = values.year + delta;
auto month = values.month;
auto day_of_month = values.day_of_month;
/// Saturation to 28 Feb can happen.
if (unlikely(day_of_month == 29 && month == 2))
day_of_month = saturateDayOfMonth(year, month, day_of_month);
return makeDayNum(year, month, day_of_month);
}
inline std::string timeToString(time_t t) const inline std::string timeToString(time_t t) const
{ {
const Values & values = find(t); const Values & values = find(t);
......
...@@ -64,6 +64,14 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_) ...@@ -64,6 +64,14 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_)
values.day_of_week = getDayOfWeek(date); values.day_of_week = getDayOfWeek(date);
values.date = start_of_day; values.date = start_of_day;
if (values.day_of_month == 1)
{
cctz::civil_month month(date);
values.days_in_month = cctz::civil_day(month + 1) - cctz::civil_day(month);
}
else
values.days_in_month = i != 0 ? lut[i - 1].days_in_month : 31;
values.time_at_offset_change = 0; values.time_at_offset_change = 0;
values.amount_of_offset_change = 0; values.amount_of_offset_change = 0;
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册