diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index bfb31d14a2813fee0f17e388e89779e7fc9d7456..bbb57f9839b5a585b49490ede127951f535f5cc7 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -94,6 +94,8 @@ struct ConvertImpl */ struct ToDateTimeImpl { + static constexpr auto name = "toDateTime"; + static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.fromDayNum(DayNum_t(d)); @@ -101,7 +103,7 @@ struct ToDateTimeImpl }; template struct ConvertImpl - : DateTimeTransformImpl {}; + : DateTimeTransformImpl {}; /// Implementation of toDate function. @@ -109,6 +111,8 @@ template struct ConvertImpl struct ToDateTransform32Or64 { + static constexpr auto name = "toDate"; + static inline ToType execute(const FromType & from, const DateLUTImpl & time_zone) { return (from < 0xFFFF) ? from : time_zone.toDayNum(from); @@ -118,7 +122,7 @@ struct ToDateTransform32Or64 /** Conversion of DateTime to Date: throw off time component. */ template struct ConvertImpl - : DateTimeTransformImpl {}; + : DateTimeTransformImpl {}; /** 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. @@ -128,17 +132,17 @@ template struct ConvertImpl struct ConvertImpl - : DateTimeTransformImpl, Name> {}; + : DateTimeTransformImpl> {}; template struct ConvertImpl - : DateTimeTransformImpl, Name> {}; + : DateTimeTransformImpl> {}; template struct ConvertImpl - : DateTimeTransformImpl, Name> {}; + : DateTimeTransformImpl> {}; template struct ConvertImpl - : DateTimeTransformImpl, Name> {}; + : DateTimeTransformImpl> {}; template struct ConvertImpl - : DateTimeTransformImpl, Name> {}; + : DateTimeTransformImpl> {}; template struct ConvertImpl - : DateTimeTransformImpl, Name> {}; + : DateTimeTransformImpl> {}; /** Transformation of numbers, dates, datetimes to strings: through formatting. diff --git a/dbms/src/Functions/FunctionsDateTime.cpp b/dbms/src/Functions/FunctionsDateTime.cpp index 21f93282a3eb29182c23650022bab14416796c6a..a56acfa8085f52e74a00f1c711a603552491af78 100644 --- a/dbms/src/Functions/FunctionsDateTime.cpp +++ b/dbms/src/Functions/FunctionsDateTime.cpp @@ -37,6 +37,13 @@ void registerFunctionsDateTime(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); } } diff --git a/dbms/src/Functions/FunctionsDateTime.h b/dbms/src/Functions/FunctionsDateTime.h index e1141c3973adb439f5199e2cdc58204bb76e6fb3..eec5eb7e3add7215cea3dae2da4fc03bb256810a 100644 --- a/dbms/src/Functions/FunctionsDateTime.h +++ b/dbms/src/Functions/FunctionsDateTime.h @@ -85,6 +85,8 @@ struct ZeroTransform struct ToDateImpl { + static constexpr auto name = "toDate"; + static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return UInt16(time_zone.toDayNum(t)); @@ -99,6 +101,8 @@ struct ToDateImpl struct ToStartOfDayImpl { + static constexpr auto name = "toStartOfDay"; + static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toDate(t); @@ -113,6 +117,8 @@ struct ToStartOfDayImpl struct ToMondayImpl { + static constexpr auto name = "toMonday"; + static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t)); @@ -127,6 +133,8 @@ struct ToMondayImpl struct ToStartOfMonthImpl { + static constexpr auto name = "toStartOfMonth"; + static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfMonth(time_zone.toDayNum(t)); @@ -141,6 +149,8 @@ struct ToStartOfMonthImpl struct ToStartOfQuarterImpl { + static constexpr auto name = "toStartOfQuarter"; + static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfQuarter(time_zone.toDayNum(t)); @@ -155,6 +165,8 @@ struct ToStartOfQuarterImpl struct ToStartOfYearImpl { + static constexpr auto name = "toStartOfYear"; + static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfYear(time_zone.toDayNum(t)); @@ -170,6 +182,8 @@ struct ToStartOfYearImpl struct ToTimeImpl { + static constexpr auto name = "toTime"; + /// When transforming to time, the date will be equated to 1970-01-02. static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { @@ -186,6 +200,8 @@ struct ToTimeImpl struct ToStartOfMinuteImpl { + static constexpr auto name = "toStartOfMinute"; + static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toStartOfMinute(t); @@ -200,6 +216,8 @@ struct ToStartOfMinuteImpl struct ToStartOfFiveMinuteImpl { + static constexpr auto name = "toStartOfFiveMinute"; + static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toStartOfFiveMinute(t); @@ -214,6 +232,8 @@ struct ToStartOfFiveMinuteImpl struct ToStartOfHourImpl { + static constexpr auto name = "toStartOfHour"; + static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toStartOfHour(t); @@ -229,6 +249,8 @@ struct ToStartOfHourImpl struct ToYearImpl { + static constexpr auto name = "toYear"; + static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toYear(t); @@ -243,6 +265,8 @@ struct ToYearImpl struct ToMonthImpl { + static constexpr auto name = "toMonth"; + static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toMonth(t); @@ -257,6 +281,8 @@ struct ToMonthImpl struct ToDayOfMonthImpl { + static constexpr auto name = "toDayOfMonth"; + static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toDayOfMonth(t); @@ -271,6 +297,8 @@ struct ToDayOfMonthImpl struct ToDayOfWeekImpl { + static constexpr auto name = "toDayOfWeek"; + static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toDayOfWeek(t); @@ -285,6 +313,8 @@ struct ToDayOfWeekImpl struct ToHourImpl { + static constexpr auto name = "toHour"; + static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toHour(t); @@ -300,6 +330,8 @@ struct ToHourImpl struct ToMinuteImpl { + static constexpr auto name = "toMinute"; + static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toMinute(t); @@ -314,6 +346,8 @@ struct ToMinuteImpl struct ToSecondImpl { + static constexpr auto name = "toSecond"; + static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toSecond(t); @@ -328,6 +362,8 @@ struct ToSecondImpl struct ToRelativeYearNumImpl { + static constexpr auto name = "toRelativeYearNum"; + static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toYear(t); @@ -342,6 +378,8 @@ struct ToRelativeYearNumImpl struct ToRelativeMonthNumImpl { + static constexpr auto name = "toRelativeMonthNum"; + static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toRelativeMonthNum(t); @@ -356,6 +394,8 @@ struct ToRelativeMonthNumImpl struct ToRelativeWeekNumImpl { + static constexpr auto name = "toRelativeWeekNum"; + static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toRelativeWeekNum(t); @@ -370,6 +410,8 @@ struct ToRelativeWeekNumImpl struct ToRelativeDayNumImpl { + static constexpr auto name = "toRelativeDayNum"; + static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toDayNum(t); @@ -385,6 +427,8 @@ struct ToRelativeDayNumImpl struct ToRelativeHourNumImpl { + static constexpr auto name = "toRelativeHourNum"; + static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toRelativeHourNum(t); @@ -399,6 +443,8 @@ struct ToRelativeHourNumImpl struct ToRelativeMinuteNumImpl { + static constexpr auto name = "toRelativeMinuteNum"; + static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toRelativeMinuteNum(t); @@ -413,6 +459,8 @@ struct ToRelativeMinuteNumImpl struct ToRelativeSecondNumImpl { + static constexpr auto name = "toRelativeSecondNum"; + static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return t; @@ -427,6 +475,8 @@ struct ToRelativeSecondNumImpl struct ToYYYYMMImpl { + static constexpr auto name = "toYYYYMM"; + static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toNumYYYYMM(t); @@ -441,6 +491,8 @@ struct ToYYYYMMImpl struct ToYYYYMMDDImpl { + static constexpr auto name = "toYYYYMMDD"; + static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toNumYYYYMMDD(t); @@ -455,6 +507,8 @@ struct ToYYYYMMDDImpl struct ToYYYYMMDDhhmmssImpl { + static constexpr auto name = "toYYYYMMDDhhmmss"; + static inline UInt64 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toNumYYYYMMDDhhmmss(t); @@ -479,24 +533,16 @@ struct Transformer for (size_t i = 0; i < size; ++i) 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 +template struct DateTimeTransformImpl { static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { using Op = Transformer; - const ColumnPtr source_col = block.getByPosition(arguments[0]).column; - const auto * sources = checkAndGetColumn>(source_col.get()); - const ColumnConst * time_zone_column = nullptr; if (arguments.size() == 2) @@ -505,7 +551,7 @@ struct DateTimeTransformImpl if (!time_zone_column) 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); } @@ -513,7 +559,8 @@ struct DateTimeTransformImpl ? DateLUT::instance(time_zone_column->getValue()) : DateLUT::instance(); - if (sources) + const ColumnPtr source_col = block.getByPosition(arguments[0]).column; + if (const auto * sources = checkAndGetColumn>(source_col.get())) { auto col_to = std::make_shared>(); block.getByPosition(result).column = col_to; @@ -522,18 +569,18 @@ struct DateTimeTransformImpl else { 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); } } }; -template +template class FunctionDateOrDateTimeToSomething : public IFunction { public: - static constexpr auto name = Name::name; + static constexpr auto name = Transform::name; static FunctionPtr create(const Context & context) { return std::make_shared(); }; String getName() const override @@ -578,12 +625,12 @@ public: 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(from_type)) - DateTimeTransformImpl::execute(block, arguments, result); + DateTimeTransformImpl::execute(block, arguments, result); else if (checkDataType(from_type)) - DateTimeTransformImpl::execute(block, arguments, result); + DateTimeTransformImpl::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); @@ -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 +struct Adder +{ + static void vector_vector(const PaddedPODArray & vec_from, PaddedPODArray & 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 & vec_from, PaddedPODArray & 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 & 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 +struct DateTimeAddIntervalImpl +{ + static void execute(Block & block, const ColumnNumbers & arguments, size_t result) + { + using ToType = decltype(Transform::execute(FromType(), 0, std::declval())); + using Op = Adder; + + const ColumnConst * time_zone_column = nullptr; + + if (arguments.size() == 3) + { + time_zone_column = checkAndGetColumnConst(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()) + : DateLUT::instance(); + + const ColumnPtr source_col = block.getByPosition(arguments[0]).column; + + if (const auto * sources = checkAndGetColumn>(source_col.get())) + { + auto col_to = std::make_shared>(); + block.getByPosition(result).column = col_to; + + const IColumn & delta_column = *block.getByPosition(arguments[1]).column; + + if (const auto * delta_const_column = typeid_cast(&delta_column)) + Op::vector_constant(sources->getData(), col_to->getData(), delta_const_column->getValue(), time_zone); + else + Op::vector_vector(sources->getData(), col_to->getData(), delta_column, time_zone); + } + else if (const auto * sources = checkAndGetColumnConst>(source_col.get())) + { + auto col_to = std::make_shared>(); + block.getByPosition(result).column = col_to; + Op::constant_vector(sources->template getValue(), 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 +class FunctionDateOrDateTimeAddInterval : public IFunction +{ +public: + static constexpr auto name = Transform::name; + static FunctionPtr create(const Context & context) { return std::make_shared(); }; + + 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(arguments[0].get()) + && !checkDataType(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(arguments[0].get()) + || !checkDataType(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(arguments[0].get())) + return std::make_shared())), UInt16>::value, + DataTypeDate, DataTypeDateTime>::type>(); + else + return std::make_shared())), 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(from_type)) + DateTimeAddIntervalImpl::execute(block, arguments, result); + else if (checkDataType(from_type)) + DateTimeAddIntervalImpl::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.) class FunctionNow : public IFunction { @@ -908,62 +1221,42 @@ public: }; -struct NameToYear { static constexpr auto name = "toYear"; }; -struct NameToMonth { static constexpr auto name = "toMonth"; }; -struct NameToDayOfMonth { static constexpr auto name = "toDayOfMonth"; }; -struct NameToDayOfWeek { static constexpr auto name = "toDayOfWeek"; }; -struct NameToHour { static constexpr auto name = "toHour"; }; -struct NameToMinute { static constexpr auto name = "toMinute"; }; -struct NameToSecond { static constexpr auto name = "toSecond"; }; -struct NameToStartOfDay { static constexpr auto name = "toStartOfDay"; }; -struct NameToMonday { static constexpr auto name = "toMonday"; }; -struct NameToStartOfMonth { static constexpr auto name = "toStartOfMonth"; }; -struct NameToStartOfQuarter { static constexpr auto name = "toStartOfQuarter"; }; -struct NameToStartOfYear { static constexpr auto name = "toStartOfYear"; }; -struct NameToStartOfMinute { static constexpr auto name = "toStartOfMinute"; }; -struct NameToStartOfFiveMinute { static constexpr auto name = "toStartOfFiveMinute"; }; -struct NameToStartOfHour { static constexpr auto name = "toStartOfHour"; }; -struct NameToTime { static constexpr auto name = "toTime"; }; -struct NameToRelativeYearNum { static constexpr auto name = "toRelativeYearNum"; }; -struct NameToRelativeMonthNum { static constexpr auto name = "toRelativeMonthNum"; }; -struct NameToRelativeWeekNum { static constexpr auto name = "toRelativeWeekNum"; }; -struct NameToRelativeDayNum { static constexpr auto name = "toRelativeDayNum"; }; -struct NameToRelativeHourNum { static constexpr auto name = "toRelativeHourNum"; }; -struct NameToRelativeMinuteNum { static constexpr auto name = "toRelativeMinuteNum"; }; -struct NameToRelativeSecondNum { static constexpr auto name = "toRelativeSecondNum"; }; -struct NameToYYYYMM { static constexpr auto name = "toYYYYMM"; }; -struct NameToYYYYMMDD { static constexpr auto name = "toYYYYMMDD"; }; -struct NameToYYYYMMDDhhmmss { static constexpr auto name = "toYYYYMMDDhhmmss"; }; - - -using FunctionToYear = FunctionDateOrDateTimeToSomething; -using FunctionToMonth = FunctionDateOrDateTimeToSomething; -using FunctionToDayOfMonth = FunctionDateOrDateTimeToSomething; -using FunctionToDayOfWeek = FunctionDateOrDateTimeToSomething; -using FunctionToHour = FunctionDateOrDateTimeToSomething; -using FunctionToMinute = FunctionDateOrDateTimeToSomething; -using FunctionToSecond = FunctionDateOrDateTimeToSomething; -using FunctionToStartOfDay = FunctionDateOrDateTimeToSomething; -using FunctionToMonday = FunctionDateOrDateTimeToSomething; -using FunctionToStartOfMonth = FunctionDateOrDateTimeToSomething; -using FunctionToStartOfQuarter = FunctionDateOrDateTimeToSomething; -using FunctionToStartOfYear = FunctionDateOrDateTimeToSomething; -using FunctionToStartOfMinute = FunctionDateOrDateTimeToSomething; -using FunctionToStartOfFiveMinute = FunctionDateOrDateTimeToSomething; -using FunctionToStartOfHour = FunctionDateOrDateTimeToSomething; -using FunctionToTime = FunctionDateOrDateTimeToSomething; - -using FunctionToRelativeYearNum = FunctionDateOrDateTimeToSomething; -using FunctionToRelativeMonthNum = FunctionDateOrDateTimeToSomething; -using FunctionToRelativeWeekNum = FunctionDateOrDateTimeToSomething; -using FunctionToRelativeDayNum = FunctionDateOrDateTimeToSomething; - -using FunctionToRelativeHourNum = FunctionDateOrDateTimeToSomething; -using FunctionToRelativeMinuteNum = FunctionDateOrDateTimeToSomething; -using FunctionToRelativeSecondNum = FunctionDateOrDateTimeToSomething; - -using FunctionToYYYYMM = FunctionDateOrDateTimeToSomething; -using FunctionToYYYYMMDD = FunctionDateOrDateTimeToSomething; -using FunctionToYYYYMMDDhhmmss = FunctionDateOrDateTimeToSomething; +using FunctionToYear = FunctionDateOrDateTimeToSomething; +using FunctionToMonth = FunctionDateOrDateTimeToSomething; +using FunctionToDayOfMonth = FunctionDateOrDateTimeToSomething; +using FunctionToDayOfWeek = FunctionDateOrDateTimeToSomething; +using FunctionToHour = FunctionDateOrDateTimeToSomething; +using FunctionToMinute = FunctionDateOrDateTimeToSomething; +using FunctionToSecond = FunctionDateOrDateTimeToSomething; +using FunctionToStartOfDay = FunctionDateOrDateTimeToSomething; +using FunctionToMonday = FunctionDateOrDateTimeToSomething; +using FunctionToStartOfMonth = FunctionDateOrDateTimeToSomething; +using FunctionToStartOfQuarter = FunctionDateOrDateTimeToSomething; +using FunctionToStartOfYear = FunctionDateOrDateTimeToSomething; +using FunctionToStartOfMinute = FunctionDateOrDateTimeToSomething; +using FunctionToStartOfFiveMinute = FunctionDateOrDateTimeToSomething; +using FunctionToStartOfHour = FunctionDateOrDateTimeToSomething; +using FunctionToTime = FunctionDateOrDateTimeToSomething; + +using FunctionToRelativeYearNum = FunctionDateOrDateTimeToSomething; +using FunctionToRelativeMonthNum = FunctionDateOrDateTimeToSomething; +using FunctionToRelativeWeekNum = FunctionDateOrDateTimeToSomething; +using FunctionToRelativeDayNum = FunctionDateOrDateTimeToSomething; + +using FunctionToRelativeHourNum = FunctionDateOrDateTimeToSomething; +using FunctionToRelativeMinuteNum = FunctionDateOrDateTimeToSomething; +using FunctionToRelativeSecondNum = FunctionDateOrDateTimeToSomething; + +using FunctionToYYYYMM = FunctionDateOrDateTimeToSomething; +using FunctionToYYYYMMDD = FunctionDateOrDateTimeToSomething; +using FunctionToYYYYMMDDhhmmss = FunctionDateOrDateTimeToSomething; + +using FunctionAddSeconds = FunctionDateOrDateTimeAddInterval; +using FunctionAddMinutes = FunctionDateOrDateTimeAddInterval; +using FunctionAddHours = FunctionDateOrDateTimeAddInterval; +using FunctionAddDays = FunctionDateOrDateTimeAddInterval; +using FunctionAddWeeks = FunctionDateOrDateTimeAddInterval; +using FunctionAddMonths = FunctionDateOrDateTimeAddInterval; +using FunctionAddYears = FunctionDateOrDateTimeAddInterval; } diff --git a/libs/libcommon/include/common/DateLUTImpl.h b/libs/libcommon/include/common/DateLUTImpl.h index 29f4f9ee6babe1400592ca8daa75ed41c91a589c..ae916459a4b6d84e2b5c243c43444e9266be7947 100644 --- a/libs/libcommon/include/common/DateLUTImpl.h +++ b/libs/libcommon/include/common/DateLUTImpl.h @@ -44,6 +44,10 @@ public: UInt8 day_of_month; 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. 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. @@ -217,13 +221,20 @@ public: 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 { - size_t today = findIndex(t); - 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); - return start_of_next_month - start_of_month; + return find(t).days_in_month; + } + + inline size_t daysInMonth(short year, char month) const + { + 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. @@ -445,6 +456,122 @@ public: 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 { const Values & values = find(t); diff --git a/libs/libcommon/src/DateLUTImpl.cpp b/libs/libcommon/src/DateLUTImpl.cpp index 14e0f019cb777b8ec7c8d140c5d8e5cf5d861ae4..07d887cf35affbcb24f818ef7a8f45d9ceb5b733 100644 --- a/libs/libcommon/src/DateLUTImpl.cpp +++ b/libs/libcommon/src/DateLUTImpl.cpp @@ -64,6 +64,14 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_) values.day_of_week = getDayOfWeek(date); 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.amount_of_offset_change = 0;