diff --git a/src/Functions/age.cpp b/src/Functions/age.cpp new file mode 100644 index 00000000000..31852777fa5 --- /dev/null +++ b/src/Functions/age.cpp @@ -0,0 +1,303 @@ +#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; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; + extern const int BAD_ARGUMENTS; +} + +namespace +{ + +/** age('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 FunctionAge : public IFunction +{ +private: + mutable Int32 UnitDivisor; +public: + static constexpr auto name = "age"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override + { + return name; + } + + bool isVariadic() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) 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(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 3 or 4", + getName(), arguments.size()); + + if (!isString(arguments[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "First argument for function {} (unit) must be String", + getName()); + + if (!isDate(arguments[1]) && !isDate32(arguments[1]) && !isDateTime(arguments[1]) && !isDateTime64(arguments[1])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Second argument for function {} must be Date, Date32, DateTime or DateTime64", + getName()); + + if (!isDate(arguments[2]) && !isDate32(arguments[2]) && !isDateTime(arguments[2]) && !isDateTime64(arguments[2])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Third argument for function {} must be Date, Date32, DateTime or DateTime64", + getName() + ); + + if (arguments.size() == 4 && !isString(arguments[3])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Fourth argument for function {} (timezone) must be String", + getName()); + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 3}; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const auto * unit_column = checkAndGetColumnConst(arguments[0].column.get()); + if (!unit_column) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "First argument for function {} must be constant String", + getName()); + + String unit = Poco::toLower(unit_column->getValue()); + + const IColumn & x = *arguments[1].column; + const IColumn & y = *arguments[2].column; + + size_t rows = input_rows_count; + auto res = ColumnInt64::create(rows); + + const auto & timezone_x = extractTimeZoneFromFunctionArguments(arguments, 3, 1); + const auto & timezone_y = extractTimeZoneFromFunctionArguments(arguments, 3, 2); + + if (unit == "year" || unit == "yy" || unit == "yyyy") + UnitDivisor = IntervalKind(IntervalKind::Year).toAvgSeconds(); + else if (unit == "quarter" || unit == "qq" || unit == "q") + UnitDivisor = IntervalKind(IntervalKind::Quarter).toAvgSeconds(); + else if (unit == "month" || unit == "mm" || unit == "m") + UnitDivisor = IntervalKind(IntervalKind::Month).toAvgSeconds(); + else if (unit == "week" || unit == "wk" || unit == "ww") + UnitDivisor = IntervalKind(IntervalKind::Week).toAvgSeconds(); + else if (unit == "day" || unit == "dd" || unit == "d") + UnitDivisor = IntervalKind(IntervalKind::Day).toAvgSeconds(); + else if (unit == "hour" || unit == "hh" || unit == "h") + UnitDivisor = IntervalKind(IntervalKind::Hour).toAvgSeconds(); + else if (unit == "minute" || unit == "mi" || unit == "n") + UnitDivisor = IntervalKind(IntervalKind::Minute).toAvgSeconds(); + else if (unit == "second" || unit == "ss" || unit == "s") + UnitDivisor = 1; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Function {} does not support '{}' unit", getName(), unit); + + dispatchForColumns>(x, y, timezone_x, timezone_y, res->getData()); + + return res; + } + +private: + + template + void dispatchForColumns( + const IColumn & x, const IColumn & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + if (const auto * x_vec_16 = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_16, y, timezone_x, timezone_y, result); + else if (const auto * x_vec_32 = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_32, y, timezone_x, timezone_y, result); + else if (const auto * x_vec_32_s = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_32_s, y, timezone_x, timezone_y, result); + else if (const auto * x_vec_64 = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_64, y, timezone_x, timezone_y, result); + else if (const auto * x_const_16 = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_16->getValue(), y, timezone_x, timezone_y, result); + else if (const auto * x_const_32 = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_32->getValue(), y, timezone_x, timezone_y, result); + else if (const auto * x_const_32_s = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_32_s->getValue(), y, timezone_x, timezone_y, result); + else if (const auto * x_const_64 = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_64->getValue>(), y, timezone_x, timezone_y, result); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column for first argument of function {}, must be Date, Date32, DateTime or DateTime64", + getName()); + } + + template + void dispatchForSecondColumn( + const LeftColumnType & x, const IColumn & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + if (const auto * y_vec_16 = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_16, timezone_x, timezone_y, result); + else if (const auto * y_vec_32 = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_32, timezone_x, timezone_y, result); + else if (const auto * y_vec_32_s = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_32_s, timezone_x, timezone_y, result); + else if (const auto * y_vec_64 = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_64, timezone_x, timezone_y, result); + else if (const auto * y_const_16 = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_16->getValue(), timezone_x, timezone_y, result); + else if (const auto * y_const_32 = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_32->getValue(), timezone_x, timezone_y, result); + else if (const auto * y_const_32_s = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_32_s->getValue(), timezone_x, timezone_y, result); + else if (const auto * y_const_64 = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_64->getValue>(), timezone_x, timezone_y, result); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column for second argument of function {}, must be Date, Date32, DateTime or DateTime64", + getName()); + } + + template + void dispatchConstForSecondColumn( + T1 x, const IColumn & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + if (const auto * y_vec_16 = checkAndGetColumn(&y)) + constantVector(x, *y_vec_16, timezone_x, timezone_y, result); + else if (const auto * y_vec_32 = checkAndGetColumn(&y)) + constantVector(x, *y_vec_32, timezone_x, timezone_y, result); + else if (const auto * y_vec_32_s = checkAndGetColumn(&y)) + constantVector(x, *y_vec_32_s, timezone_x, timezone_y, result); + else if (const auto * y_vec_64 = checkAndGetColumn(&y)) + constantVector(x, *y_vec_64, timezone_x, timezone_y, result); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column for second argument of function {}, must be Date, Date32, DateTime or DateTime64", + getName()); + } + + template + void vectorVector( + const LeftColumnType & x, const RightColumnType & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + const auto & x_data = x.getData(); + const auto & y_data = y.getData(); + + const auto transform_x = TransformDateTime64(getScale(x)); + const auto transform_y = TransformDateTime64(getScale(y)); + for (size_t i = 0, size = x.size(); i < size; ++i) + result[i] = calculate(transform_x, transform_y, x_data[i], y_data[i], timezone_x, timezone_y); + } + + template + void vectorConstant( + const LeftColumnType & x, T2 y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + const auto & x_data = x.getData(); + const auto transform_x = TransformDateTime64(getScale(x)); + const auto transform_y = TransformDateTime64(getScale(y)); + const auto y_value = stripDecimalFieldValue(y); + + for (size_t i = 0, size = x.size(); i < size; ++i) + result[i] = calculate(transform_x, transform_y, x_data[i], y_value, timezone_x, timezone_y); + } + + template + void constantVector( + T1 x, const RightColumnType & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + const auto & y_data = y.getData(); + const auto transform_x = TransformDateTime64(getScale(x)); + const auto transform_y = TransformDateTime64(getScale(y)); + const auto x_value = stripDecimalFieldValue(x); + + for (size_t i = 0, size = y.size(); i < size; ++i) + result[i] = calculate(transform_x, transform_y, x_value, y_data[i], timezone_x, timezone_y); + } + + template + Int64 calculate(const TransformX & transform_x, const TransformY & transform_y, T1 x, T2 y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y) const + { + Int64 res_y = static_cast(transform_y.execute(y, timezone_y)); + Int64 res_x = static_cast(transform_x.execute(x, timezone_x)); + return (res_y - res_x) / UnitDivisor; + // return static_cast(transform_y.execute(y, timezone_y)) + // - static_cast(transform_x.execute(x, timezone_x)); + } + + template + static UInt32 getScale(const T & v) + { + if constexpr (std::is_same_v) + return v.getScale(); + else if constexpr (std::is_same_v>) + return v.getScale(); + + return 0; + } + template + static auto stripDecimalFieldValue(T && v) + { + if constexpr (std::is_same_v, DecimalField>) + return v.getValue(); + else + return v; + } +}; + +} + +REGISTER_FUNCTION(Age) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); +} + +}