#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; } /** 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 (!isString(arguments[0])) throw Exception("First argument for function " + getName() + " (unit) must be String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (!isDateOrDateTime(arguments[1])) throw Exception("Second argument for function " + getName() + " must be Date or DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (!isDateOrDateTime(arguments[2])) throw Exception("Third argument for function " + getName() + " must be Date or DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (arguments.size() == 4 && !isString(arguments[3])) 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, size_t input_rows_count) 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 = input_rows_count; 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_16 = checkAndGetColumn(&x)) dispatchForSecondColumn(*x_vec_16, y, timezone_x, timezone_y, result); else if (auto * x_vec_32 = checkAndGetColumn(&x)) dispatchForSecondColumn(*x_vec_32, y, timezone_x, timezone_y, result); else if (auto * x_const_16 = checkAndGetColumnConst(&x)) dispatchConstForSecondColumn(x_const_16->getValue(), y, timezone_x, timezone_y, result); else if (auto * x_const_32 = checkAndGetColumnConst(&x)) dispatchConstForSecondColumn(x_const_32->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_16 = checkAndGetColumn(&y)) vector_vector(x, *y_vec_16, timezone_x, timezone_y, result); else if (auto * y_vec_32 = checkAndGetColumn(&y)) vector_vector(x, *y_vec_32, timezone_x, timezone_y, result); else if (auto * y_const_16 = checkAndGetColumnConst(&y)) vector_constant(x, y_const_16->getValue(), timezone_x, timezone_y, result); else if (auto * y_const_32 = checkAndGetColumnConst(&y)) vector_constant(x, y_const_32->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_16 = checkAndGetColumn(&y)) constant_vector(x, *y_vec_16, timezone_x, timezone_y, result); else if (auto * y_vec_32 = checkAndGetColumn(&y)) constant_vector(x, *y_vec_32, 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)); } }; void registerFunctionDateDiff(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); } }