#pragma once #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include namespace DB { namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } /** Functions for working with date and time. * * toYear, toMonth, toDayOfMonth, toDayOfWeek, toHour, toMinute, toSecond, * toMonday, toStartOfMonth, toStartOfYear, toStartOfMinute, toStartOfFiveMinute, toStartOfFifteenMinutes * toStartOfHour, toTime, * now, today, yesterday * TODO: makeDate, makeDateTime * * (toDate - located in FunctionConversion.h file) * * Return types: * toYear -> UInt16 * toMonth, toDayOfMonth, toDayOfWeek, toHour, toMinute, toSecond -> UInt8 * toMonday, toStartOfMonth, toStartOfYear -> Date * toStartOfMinute, toStartOfHour, toTime, now -> DateTime * * And also: * * timeSlot(EventTime) * - rounds the time to half an hour. * * timeSlots(StartTime, Duration) * - for the time interval beginning at `StartTime` and continuing `Duration` seconds, * returns an array of time points, consisting of rounding down to half an hour of points from this interval. * For example, timeSlots(toDateTime('2012-01-01 12:20:00'), 600) = [toDateTime('2012-01-01 12:00:00'), toDateTime('2012-01-01 12:30:00')]. * This is necessary to search for hits that are part of the corresponding visit. */ /// Determine working timezone either from optional argument with time zone name or from time zone in DateTime type of argument. std::string extractTimeZoneNameFromFunctionArguments(const ColumnsWithTypeAndName & arguments, size_t time_zone_arg_num, size_t datetime_arg_num); const DateLUTImpl & extractTimeZoneFromFunctionArguments(Block & block, const ColumnNumbers & arguments, size_t time_zone_arg_num, size_t datetime_arg_num); #define TIME_SLOT_SIZE 1800 /** Transformations. * Represents two functions - from datetime (UInt32) and from date (UInt16). * * Also, the "factor transformation" F is defined for the T transformation. * This is a transformation of F such that its value identifies the region of monotonicity for T * (for a fixed value of F, the transformation T is monotonic). * * Or, figuratively, if T is similar to taking the remainder of division, then F is similar to division. * * Example: for transformation T "get the day number in the month" (2015-02-03 -> 3), * factor-transformation F is "round to the nearest month" (2015-02-03 -> 2015-02-01). */ /// This factor transformation will say that the function is monotone everywhere. struct ZeroTransform { static inline UInt16 execute(UInt32, const DateLUTImpl &) { return 0; } static inline UInt16 execute(UInt16, const DateLUTImpl &) { return 0; } }; struct ToDateImpl { static constexpr auto name = "toDate"; static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return UInt16(time_zone.toDayNum(t)); } static inline UInt16 execute(UInt16 d, const DateLUTImpl &) { return d; } using FactorTransform = ZeroTransform; }; struct ToStartOfDayImpl { static constexpr auto name = "toStartOfDay"; static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toDate(t); } static inline UInt32 execute(UInt16, const DateLUTImpl &) { throw Exception("Illegal type Date of argument for function toStartOfDay", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } using FactorTransform = ZeroTransform; }; 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)); } static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfWeek(DayNum_t(d)); } using FactorTransform = ZeroTransform; }; 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)); } static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfMonth(DayNum_t(d)); } using FactorTransform = ZeroTransform; }; 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)); } static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfQuarter(DayNum_t(d)); } using FactorTransform = ZeroTransform; }; 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)); } static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfYear(DayNum_t(d)); } using FactorTransform = ZeroTransform; }; 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) { return time_zone.toTime(t) + 86400; } static inline UInt32 execute(UInt16, const DateLUTImpl &) { throw Exception("Illegal type Date of argument for function toTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } using FactorTransform = ToDateImpl; }; struct ToStartOfMinuteImpl { static constexpr auto name = "toStartOfMinute"; static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toStartOfMinute(t); } static inline UInt32 execute(UInt16, const DateLUTImpl &) { throw Exception("Illegal type Date of argument for function toStartOfMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } using FactorTransform = ZeroTransform; }; struct ToStartOfFiveMinuteImpl { static constexpr auto name = "toStartOfFiveMinute"; static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toStartOfFiveMinute(t); } static inline UInt32 execute(UInt16, const DateLUTImpl &) { throw Exception("Illegal type Date of argument for function toStartOfFiveMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } using FactorTransform = ZeroTransform; }; struct ToStartOfFifteenMinutesImpl { static constexpr auto name = "toStartOfFifteenMinutes"; static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toStartOfFifteenMinutes(t); } static inline UInt32 execute(UInt16, const DateLUTImpl &) { throw Exception("Illegal type Date of argument for function toStartOfFifteenMinutes", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } using FactorTransform = ZeroTransform; }; struct ToStartOfHourImpl { static constexpr auto name = "toStartOfHour"; static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toStartOfHour(t); } static inline UInt32 execute(UInt16, const DateLUTImpl &) { throw Exception("Illegal type Date of argument for function toStartOfHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } using FactorTransform = ZeroTransform; }; struct ToYearImpl { static constexpr auto name = "toYear"; static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toYear(t); } static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toYear(DayNum_t(d)); } using FactorTransform = ZeroTransform; }; struct ToQuarterImpl { static constexpr auto name = "toQuarter"; static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toQuarter(t); } static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toQuarter(DayNum_t(d)); } using FactorTransform = ToStartOfYearImpl; }; struct ToMonthImpl { static constexpr auto name = "toMonth"; static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toMonth(t); } static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toMonth(DayNum_t(d)); } using FactorTransform = ToStartOfYearImpl; }; struct ToDayOfMonthImpl { static constexpr auto name = "toDayOfMonth"; static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toDayOfMonth(t); } static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toDayOfMonth(DayNum_t(d)); } using FactorTransform = ToStartOfMonthImpl; }; struct ToDayOfWeekImpl { static constexpr auto name = "toDayOfWeek"; static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toDayOfWeek(t); } static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toDayOfWeek(DayNum_t(d)); } using FactorTransform = ToMondayImpl; }; struct ToHourImpl { static constexpr auto name = "toHour"; static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toHour(t); } static inline UInt8 execute(UInt16, const DateLUTImpl &) { throw Exception("Illegal type Date of argument for function toHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } using FactorTransform = ToDateImpl; }; struct ToMinuteImpl { static constexpr auto name = "toMinute"; static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toMinute(t); } static inline UInt8 execute(UInt16, const DateLUTImpl &) { throw Exception("Illegal type Date of argument for function toMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } using FactorTransform = ToStartOfHourImpl; }; struct ToSecondImpl { static constexpr auto name = "toSecond"; static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toSecond(t); } static inline UInt8 execute(UInt16, const DateLUTImpl &) { throw Exception("Illegal type Date of argument for function toSecond", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } using FactorTransform = ToStartOfMinuteImpl; }; struct ToRelativeYearNumImpl { static constexpr auto name = "toRelativeYearNum"; static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toYear(t); } static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toYear(DayNum_t(d)); } using FactorTransform = ZeroTransform; }; struct ToRelativeQuarterNumImpl { static constexpr auto name = "toRelativeQuarterNum"; static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toRelativeQuarterNum(t); } static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toRelativeQuarterNum(DayNum_t(d)); } using FactorTransform = ZeroTransform; }; struct ToRelativeMonthNumImpl { static constexpr auto name = "toRelativeMonthNum"; static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toRelativeMonthNum(t); } static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toRelativeMonthNum(DayNum_t(d)); } using FactorTransform = ZeroTransform; }; struct ToRelativeWeekNumImpl { static constexpr auto name = "toRelativeWeekNum"; static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toRelativeWeekNum(t); } static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toRelativeWeekNum(DayNum_t(d)); } using FactorTransform = ZeroTransform; }; struct ToRelativeDayNumImpl { static constexpr auto name = "toRelativeDayNum"; static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toDayNum(t); } static inline UInt16 execute(UInt16 d, const DateLUTImpl &) { return static_cast(d); } using FactorTransform = ZeroTransform; }; struct ToRelativeHourNumImpl { static constexpr auto name = "toRelativeHourNum"; static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toRelativeHourNum(t); } static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toRelativeHourNum(DayNum_t(d)); } using FactorTransform = ZeroTransform; }; struct ToRelativeMinuteNumImpl { static constexpr auto name = "toRelativeMinuteNum"; static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toRelativeMinuteNum(t); } static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toRelativeMinuteNum(DayNum_t(d)); } using FactorTransform = ZeroTransform; }; struct ToRelativeSecondNumImpl { static constexpr auto name = "toRelativeSecondNum"; static inline UInt32 execute(UInt32 t, const DateLUTImpl &) { return t; } static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.fromDayNum(DayNum_t(d)); } using FactorTransform = ZeroTransform; }; struct ToYYYYMMImpl { static constexpr auto name = "toYYYYMM"; static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toNumYYYYMM(t); } static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toNumYYYYMM(static_cast(d)); } using FactorTransform = ZeroTransform; }; struct ToYYYYMMDDImpl { static constexpr auto name = "toYYYYMMDD"; static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toNumYYYYMMDD(t); } static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toNumYYYYMMDD(static_cast(d)); } using FactorTransform = ZeroTransform; }; struct ToYYYYMMDDhhmmssImpl { static constexpr auto name = "toYYYYMMDDhhmmss"; static inline UInt64 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toNumYYYYMMDDhhmmss(t); } static inline UInt64 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toNumYYYYMMDDhhmmss(time_zone.toDate(static_cast(d))); } using FactorTransform = ZeroTransform; }; template struct Transformer { static void vector(const PaddedPODArray & vec_from, PaddedPODArray & vec_to, 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], time_zone); } }; template struct DateTimeTransformImpl { static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { using Op = Transformer; const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block, arguments, 1, 0); const ColumnPtr source_col = block.getByPosition(arguments[0]).column; if (const auto * sources = checkAndGetColumn>(source_col.get())) { auto col_to = ColumnVector::create(); Op::vector(sources->getData(), col_to->getData(), time_zone); block.getByPosition(result).column = std::move(col_to); } else { throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + Transform::name, ErrorCodes::ILLEGAL_COLUMN); } } }; template class FunctionDateOrDateTimeToSomething : public IFunction { public: static constexpr auto name = Transform::name; static FunctionPtr create(const Context &) { return std::make_shared(); }; String getName() const override { return name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } void getReturnTypeAndPrerequisitesImpl( const ColumnsWithTypeAndName & arguments, DataTypePtr & out_return_type, std::vector &) override { if (arguments.size() == 1) { if (!arguments[0].type->isDateOrDateTime()) throw Exception{ "Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() + ". Should be a date or a date with time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } else if (arguments.size() == 2) { if (!checkDataType(arguments[0].type.get()) || !checkDataType(arguments[1].type.get())) throw Exception{ "Function " + getName() + " supports 1 or 2 arguments. The 1st argument " "must be of type Date or DateTime. The 2nd 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}; } else throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + ", should be 1 or 2", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); /// For DateTime, if time zone is specified, attach it to type. if (std::is_same_v) out_return_type = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0)); else out_return_type = std::make_shared(); } bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } 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)) DateTimeTransformImpl::execute(block, arguments, result); else if (checkDataType(from_type)) 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); } bool hasInformationAboutMonotonicity() const override { return true; } Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override { IFunction::Monotonicity is_monotonic { true }; IFunction::Monotonicity is_not_monotonic; if (std::is_same_v) { is_monotonic.is_always_monotonic = true; return is_monotonic; } /// This method is called only if the function has one argument. Therefore, we do not care about the non-local time zone. const DateLUTImpl & date_lut = DateLUT::instance(); if (left.isNull() || right.isNull()) return is_not_monotonic; /// The function is monotonous on the [left, right] segment, if the factor transformation returns the same values for them. if (checkAndGetDataType(&type)) { return Transform::FactorTransform::execute(UInt16(left.get()), date_lut) == Transform::FactorTransform::execute(UInt16(right.get()), date_lut) ? is_monotonic : is_not_monotonic; } else { return Transform::FactorTransform::execute(UInt32(left.get()), date_lut) == Transform::FactorTransform::execute(UInt32(right.get()), date_lut) ? is_monotonic : is_not_monotonic; } } }; struct AddSecondsImpl { static constexpr auto name = "addSeconds"; static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &) { 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 &) { 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 &) { 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 &) { 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 &) { 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 SubtractIntervalImpl { static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone) { return Transform::execute(t, -delta, time_zone); } static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone) { return Transform::execute(d, -delta, time_zone); } }; struct SubtractSecondsImpl : SubtractIntervalImpl { static constexpr auto name = "subtractSeconds"; }; struct SubtractMinutesImpl : SubtractIntervalImpl { static constexpr auto name = "subtractMinutes"; }; struct SubtractHoursImpl : SubtractIntervalImpl { static constexpr auto name = "subtractHours"; }; struct SubtractDaysImpl : SubtractIntervalImpl { static constexpr auto name = "subtractDays"; }; struct SubtractWeeksImpl : SubtractIntervalImpl { static constexpr auto name = "subtractWeeks"; }; struct SubtractMonthsImpl : SubtractIntervalImpl { static constexpr auto name = "subtractMonths"; }; struct SubtractYearsImpl : SubtractIntervalImpl { static constexpr auto name = "subtractYears"; }; 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 DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block, arguments, 2, 0); const ColumnPtr source_col = block.getByPosition(arguments[0]).column; if (const auto * sources = checkAndGetColumn>(source_col.get())) { auto col_to = ColumnVector::create(); 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->getField().get(), time_zone); else Op::vector_vector(sources->getData(), col_to->getData(), delta_column, time_zone); block.getByPosition(result).column = std::move(col_to); } else if (const auto * sources = checkAndGetColumnConst>(source_col.get())) { auto col_to = ColumnVector::create(); Op::constant_vector(sources->template getValue(), col_to->getData(), *block.getByPosition(arguments[1]).column, time_zone); block.getByPosition(result).column = std::move(col_to); } 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 &) { return std::make_shared(); }; String getName() const override { return name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } void getReturnTypeAndPrerequisitesImpl( const ColumnsWithTypeAndName & arguments, DataTypePtr & out_return_type, std::vector &) 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].type->isNumber()) throw Exception("Second argument for function " + getName() + " (delta) must be number", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (arguments.size() == 2) { if (!arguments[0].type->isDateOrDateTime()) throw Exception{ "Illegal type " + arguments[0].type->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].type.get()) || !checkDataType(arguments[2].type.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].type.get())) { if (std::is_same_v())), UInt16>) out_return_type = std::make_shared(); else out_return_type = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0)); } else { if (std::is_same_v())), UInt16>) out_return_type = std::make_shared(); else out_return_type = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0)); } } 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); } }; /** dateDiff('unit', t1, t2, [timezone]) * t1 and t2 can be Date or DateTime * * If timezone is specified, it applied to both arguments. * If not, timezones from datatypes t1 and t2 are used. * If that timezones are not the same, the result is unspecified. * * Timezone matters because days can have different length. */ class FunctionDateDiff : public IFunction { public: static constexpr auto name = "dateDiff"; static FunctionPtr create(const 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() != 3 && arguments.size() != 4) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + ", should be 3 or 4", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); if (!arguments[0]->isString()) throw Exception("First argument for function " + getName() + " (unit) must be String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (!arguments[1]->isDateOrDateTime()) throw Exception("Second argument for function " + getName() + " must be Date or DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (!arguments[2]->isDateOrDateTime()) throw Exception("Third argument for function " + getName() + " must be Date or DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (arguments.size() == 4 && !arguments[3]->isString()) throw Exception("Fourth argument for function " + getName() + " (timezone) must be String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_shared(); } bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 3}; } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override { auto * unit_column = checkAndGetColumnConst(block.getByPosition(arguments[0]).column.get()); if (!unit_column) throw Exception("First argument for function " + getName() + " must be constant String", ErrorCodes::ILLEGAL_COLUMN); String unit = Poco::toLower(unit_column->getValue()); const IColumn & x = *block.getByPosition(arguments[1]).column; const IColumn & y = *block.getByPosition(arguments[2]).column; size_t rows = block.rows(); auto res = ColumnInt64::create(rows); const DateLUTImpl & timezone_x = extractTimeZoneFromFunctionArguments(block, arguments, 3, 1); const DateLUTImpl & timezone_y = extractTimeZoneFromFunctionArguments(block, arguments, 3, 2); if (unit == "year" || unit == "yy" || unit == "yyyy") dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "quarter" || unit == "qq" || unit == "q") dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "month" || unit == "mm" || unit == "m") dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "week" || unit == "wk" || unit == "ww") dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "day" || unit == "dd" || unit == "d") dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "hour" || unit == "hh") dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "minute" || unit == "mi" || unit == "n") dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "second" || unit == "ss" || unit == "s") dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else throw Exception("Function " + getName() + " does not support '" + unit + "' unit", ErrorCodes::BAD_ARGUMENTS); block.getByPosition(result).column = std::move(res); } private: template void dispatchForColumns( const IColumn & x, const IColumn & y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, ColumnInt64::Container & result) { if (auto * x_vec = checkAndGetColumn(&x)) dispatchForSecondColumn(*x_vec, y, timezone_x, timezone_y, result); else if (auto * x_vec = checkAndGetColumn(&x)) dispatchForSecondColumn(*x_vec, y, timezone_x, timezone_y, result); else if (auto * x_const = checkAndGetColumnConst(&x)) dispatchConstForSecondColumn(x_const->getValue(), y, timezone_x, timezone_y, result); else if (auto * x_const = checkAndGetColumnConst(&x)) dispatchConstForSecondColumn(x_const->getValue(), y, timezone_x, timezone_y, result); else throw Exception("Illegal column for first argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN); } template void dispatchForSecondColumn( const ColumnVector & x, const IColumn & y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, ColumnInt64::Container & result) { if (auto * y_vec = checkAndGetColumn(&y)) vector_vector(x, *y_vec, timezone_x, timezone_y, result); else if (auto * y_vec = checkAndGetColumn(&y)) vector_vector(x, *y_vec, timezone_x, timezone_y, result); else if (auto * y_const = checkAndGetColumnConst(&y)) vector_constant(x, y_const->getValue(), timezone_x, timezone_y, result); else if (auto * y_const = checkAndGetColumnConst(&y)) vector_constant(x, y_const->getValue(), timezone_x, timezone_y, result); else throw Exception("Illegal column for second argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN); } template void dispatchConstForSecondColumn( T1 x, const IColumn & y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, ColumnInt64::Container & result) { if (auto * y_vec = checkAndGetColumn(&y)) constant_vector(x, *y_vec, timezone_x, timezone_y, result); else if (auto * y_vec = checkAndGetColumn(&y)) constant_vector(x, *y_vec, timezone_x, timezone_y, result); else throw Exception("Illegal column for second argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN); } template void vector_vector( const ColumnVector & x, const ColumnVector & y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, ColumnInt64::Container & result) { const auto & x_data = x.getData(); const auto & y_data = y.getData(); for (size_t i = 0, size = x.size(); i < size; ++i) result[i] = calculate(x_data[i], y_data[i], timezone_x, timezone_y); } template void vector_constant( const ColumnVector & x, T2 y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, ColumnInt64::Container & result) { const auto & x_data = x.getData(); for (size_t i = 0, size = x.size(); i < size; ++i) result[i] = calculate(x_data[i], y, timezone_x, timezone_y); } template void constant_vector( T1 x, const ColumnVector & y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, ColumnInt64::Container & result) { const auto & y_data = y.getData(); for (size_t i = 0, size = y.size(); i < size; ++i) result[i] = calculate(x, y_data[i], timezone_x, timezone_y); } template Int64 calculate(T1 x, T2 y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y) { return Int64(Transform::execute(y, timezone_y)) - Int64(Transform::execute(x, timezone_x)); } }; /// Get the current time. (It is a constant, it is evaluated once for the entire query.) class FunctionNow : public IFunction { public: static constexpr auto name = "now"; static FunctionPtr create(const Context &) { return std::make_shared(); }; String getName() const override { return name; } size_t getNumberOfArguments() const override { return 0; } DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override { return std::make_shared(); } void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override { block.getByPosition(result).column = DataTypeUInt32().createColumnConst( block.rows(), static_cast(time(nullptr))); } }; class FunctionToday : public IFunction { public: static constexpr auto name = "today"; static FunctionPtr create(const Context &) { return std::make_shared(); }; String getName() const override { return name; } size_t getNumberOfArguments() const override { return 0; } DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override { return std::make_shared(); } void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override { block.getByPosition(result).column = DataTypeUInt16().createColumnConst( block.rows(), UInt64(DateLUT::instance().toDayNum(time(nullptr)))); } }; class FunctionYesterday : public IFunction { public: static constexpr auto name = "yesterday"; static FunctionPtr create(const Context &) { return std::make_shared(); }; String getName() const override { return name; } size_t getNumberOfArguments() const override { return 0; } DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override { return std::make_shared(); } void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override { block.getByPosition(result).column = DataTypeUInt16().createColumnConst( block.rows(), UInt64(DateLUT::instance().toDayNum(time(nullptr)) - 1)); } }; /// Just changes time zone information for data type. The calculation is free. class FunctionToTimeZone : public IFunction { public: static constexpr auto name = "toTimeZone"; static FunctionPtr create(const Context &) { return std::make_shared(); }; String getName() const override { return name; } size_t getNumberOfArguments() const override { return 2; } void getReturnTypeAndPrerequisitesImpl( const ColumnsWithTypeAndName & arguments, DataTypePtr & out_return_type, std::vector &) override { if (arguments.size() != 2) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + ", should be 2", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); if (!checkDataType(arguments[0].type.get())) throw Exception{ "Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() + ". Should be DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; String time_zone_name = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0); out_return_type = std::make_shared(time_zone_name); } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override { block.getByPosition(result).column = block.getByPosition(arguments[0]).column; } }; class FunctionTimeSlot : public IFunction { public: static constexpr auto name = "timeSlot"; static FunctionPtr create(const Context &) { return std::make_shared(); }; String getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (!checkDataType(arguments[0].get())) throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be DateTime.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_shared(); } bool useDefaultImplementationForConstants() const override { return true; } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override { if (const ColumnUInt32 * times = typeid_cast(block.getByPosition(arguments[0]).column.get())) { auto res = ColumnUInt32::create(); ColumnUInt32::Container & res_vec = res->getData(); const ColumnUInt32::Container & vec = times->getData(); size_t size = vec.size(); res_vec.resize(size); for (size_t i = 0; i < size; ++i) res_vec[i] = vec[i] / TIME_SLOT_SIZE * TIME_SLOT_SIZE; block.getByPosition(result).column = std::move(res); } else throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } }; template struct TimeSlotsImpl { static void vector_vector( const PaddedPODArray & starts, const PaddedPODArray & durations, PaddedPODArray & result_values, ColumnArray::Offsets & result_offsets) { size_t size = starts.size(); result_offsets.resize(size); result_values.reserve(size); ColumnArray::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { for (UInt32 value = starts[i] / TIME_SLOT_SIZE; value <= (starts[i] + durations[i]) / TIME_SLOT_SIZE; ++value) { result_values.push_back(value * TIME_SLOT_SIZE); ++current_offset; } result_offsets[i] = current_offset; } } static void vector_constant( const PaddedPODArray & starts, DurationType duration, PaddedPODArray & result_values, ColumnArray::Offsets & result_offsets) { size_t size = starts.size(); result_offsets.resize(size); result_values.reserve(size); ColumnArray::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { for (UInt32 value = starts[i] / TIME_SLOT_SIZE; value <= (starts[i] + duration) / TIME_SLOT_SIZE; ++value) { result_values.push_back(value * TIME_SLOT_SIZE); ++current_offset; } result_offsets[i] = current_offset; } } static void constant_vector( UInt32 start, const PaddedPODArray & durations, PaddedPODArray & result_values, ColumnArray::Offsets & result_offsets) { size_t size = durations.size(); result_offsets.resize(size); result_values.reserve(size); ColumnArray::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { for (UInt32 value = start / TIME_SLOT_SIZE; value <= (start + durations[i]) / TIME_SLOT_SIZE; ++value) { result_values.push_back(value * TIME_SLOT_SIZE); ++current_offset; } result_offsets[i] = current_offset; } } static void constant_constant( UInt32 start, DurationType duration, Array & result) { for (UInt32 value = start / TIME_SLOT_SIZE; value <= (start + duration) / TIME_SLOT_SIZE; ++value) result.push_back(static_cast(value * TIME_SLOT_SIZE)); } }; class FunctionTimeSlots : public IFunction { public: static constexpr auto name = "timeSlots"; static FunctionPtr create(const Context &) { return std::make_shared(); }; String getName() const override { return name; } size_t getNumberOfArguments() const override { return 2; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (!checkDataType(arguments[0].get())) throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be DateTime.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (!checkDataType(arguments[1].get())) throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ". Must be UInt32.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_shared(std::make_shared()); } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override { auto starts = checkAndGetColumn(block.getByPosition(arguments[0]).column.get()); auto const_starts = checkAndGetColumnConst(block.getByPosition(arguments[0]).column.get()); auto durations = checkAndGetColumn(block.getByPosition(arguments[1]).column.get()); auto const_durations = checkAndGetColumnConst(block.getByPosition(arguments[1]).column.get()); auto res = ColumnArray::create(ColumnUInt32::create()); ColumnUInt32::Container & res_values = typeid_cast(res->getData()).getData(); if (starts && durations) { TimeSlotsImpl::vector_vector(starts->getData(), durations->getData(), res_values, res->getOffsets()); block.getByPosition(result).column = std::move(res); } else if (starts && const_durations) { TimeSlotsImpl::vector_constant(starts->getData(), const_durations->getValue(), res_values, res->getOffsets()); block.getByPosition(result).column = std::move(res); } else if (const_starts && durations) { TimeSlotsImpl::constant_vector(const_starts->getValue(), durations->getData(), res_values, res->getOffsets()); block.getByPosition(result).column = std::move(res); } else if (const_starts && const_durations) { Array const_res; TimeSlotsImpl::constant_constant(const_starts->getValue(), const_durations->getValue(), const_res); block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(block.rows(), const_res); } else throw Exception("Illegal columns " + block.getByPosition(arguments[0]).column->getName() + ", " + block.getByPosition(arguments[1]).column->getName() + " of arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } }; using FunctionToYear = FunctionDateOrDateTimeToSomething; using FunctionToQuarter = 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 FunctionToStartOfFifteenMinutes = FunctionDateOrDateTimeToSomething; using FunctionToStartOfHour = FunctionDateOrDateTimeToSomething; using FunctionToTime = FunctionDateOrDateTimeToSomething; using FunctionToRelativeYearNum = FunctionDateOrDateTimeToSomething; using FunctionToRelativeQuarterNum = 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; using FunctionSubtractSeconds = FunctionDateOrDateTimeAddInterval; using FunctionSubtractMinutes = FunctionDateOrDateTimeAddInterval; using FunctionSubtractHours = FunctionDateOrDateTimeAddInterval; using FunctionSubtractDays = FunctionDateOrDateTimeAddInterval; using FunctionSubtractWeeks = FunctionDateOrDateTimeAddInterval; using FunctionSubtractMonths = FunctionDateOrDateTimeAddInterval; using FunctionSubtractYears = FunctionDateOrDateTimeAddInterval; }