diff --git a/dbms/include/DB/Functions/FunctionsRound.h b/dbms/include/DB/Functions/FunctionsRound.h index 9e587edbbcf..4bffcf1cf11 100644 --- a/dbms/include/DB/Functions/FunctionsRound.h +++ b/dbms/include/DB/Functions/FunctionsRound.h @@ -1,7 +1,9 @@ #pragma once #include -#include // log2() +#include +#include +#include namespace DB @@ -11,6 +13,9 @@ namespace DB * roundToExp2 - вниз до ближайшей степени двойки; * roundDuration - вниз до ближайшего из: 0, 1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000; * roundAge - вниз до ближайшего из: 0, 18, 25, 35, 45. + * round(x, N) - арифметическое округление (N - сколько знаков после запятой оставить; 0 по умолчанию). + * ceil(x, N) - наименьшее число, которое не меньше x (N - сколько знаков после запятой оставить; 0 по умолчанию). + * floor(x, N) - наибольшее число, которое не больше x (N - сколько знаков после запятой оставить; 0 по умолчанию). */ template @@ -87,13 +92,366 @@ namespace DB } }; + template class Op, typename PowersTable> + struct FunctionApproximatingImpl + { + template + static inline U apply(U val, UInt8 precision, + typename std::enable_if::value>::type * = nullptr) + { + if (val == 0) + return val; + else + { + size_t power = PowersTable::values[precision]; + return Op::apply(val * power) / power; + } + } + + /// Для целых чисел ничего не надо делать. + template + static inline U apply(U val, UInt8 precision, + typename std::enable_if::value>::type * = nullptr) + { + return val; + } + }; + + template + struct PrecisionForType + { + template + static inline bool apply(const ColumnPtr & column, UInt8 & precision, + typename std::enable_if::value>::type * = nullptr) + { + using ColumnType = ColumnConst; + + const ColumnType * precision_col = typeid_cast(&*column); + if (precision_col == nullptr) + return false; + + U val = precision_col->getData(); + if (val < 0) + val = 0; + else if (val >= static_cast(std::numeric_limits::digits10)) + val = static_cast(std::numeric_limits::digits10); + + precision = static_cast(val); + + return true; + } + + /// Для целых чисел точность не имеет значения. + template + static inline bool apply(const ColumnPtr & column, UInt8 & precision, + typename std::enable_if::value>::type * = nullptr) + { + using ColumnType = ColumnConst; + + const ColumnType * precision_col = typeid_cast(&*column); + if (precision_col == nullptr) + return false; + + precision = 0; + + return true; + } + }; + + /// Следующий код генерирует во время сборки таблицу степеней числа 10. + +namespace +{ + /// Отдельные степени числа 10. + + template + struct PowerOf10 + { + static const size_t value = 10 * PowerOf10::value; + }; + + template<> + struct PowerOf10<0> + { + static const size_t value = 1; + }; +} + + /// Объявление и определение контейнера содержащего таблицу степеней числа 10. + + template + struct TableContainer + { + static const std::array values; + }; + + template + const std::array TableContainer::values = { TArgs... }; + + /// Генератор первых N степеней. + + template + struct FillArrayImpl + { + using result = typename FillArrayImpl::value, TArgs...>::result; + }; + + template + struct FillArrayImpl<0, TArgs...> + { + using result = TableContainer::value, TArgs...>; + }; + + template + struct FillArray + { + using result = typename FillArrayImpl::result; + }; + + /** Шаблон для функцией, которые вычисляют приближенное значение входного параметра + * типа (U)Int8/16/32/64 или Float32/64 и принимают дополнительный необязятельный + * параметр указывающий сколько знаков после запятой оставить (по умолчанию - 0). + * Op - функция (round/floor/ceil) + */ + template class Op, typename Name> + class FunctionApproximating : public IFunction + { + public: + static constexpr auto name = Name::name; + static IFunction * create(const Context & context) { return new FunctionApproximating; } + + private: + using PowersOf10 = FillArray::digits10 + 1>::result; + + private: + template + bool checkType(const IDataType * type) const + { + return typeid_cast(type) != nullptr; + } + + template + bool executeForType(Block & block, const ColumnNumbers & arguments, size_t result) + { + if (ColumnVector * col = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) + { + UInt8 precision = 0; + if (arguments.size() == 2) + precision = getPrecision(block.getByPosition(arguments[1]).column); + + ColumnVector * col_res = new ColumnVector; + block.getByPosition(result).column = col_res; + + typename ColumnVector::Container_t & vec_res = col_res->getData(); + vec_res.resize(col->getData().size()); + + const PODArray & a = col->getData(); + size_t size = a.size(); + for (size_t i = 0; i < size; ++i) + vec_res[i] = FunctionApproximatingImpl::apply(a[i], precision); + + return true; + } + else if (ColumnConst * col = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) + { + UInt8 precision = 0; + if (arguments.size() == 2) + precision = getPrecision(block.getByPosition(arguments[1]).column); + + T res = FunctionApproximatingImpl::apply(col->getData(), precision); + + ColumnConst * col_res = new ColumnConst(col->size(), res); + block.getByPosition(result).column = col_res; + + return true; + } + + return false; + } + + /// В зависимости от входного параметра, определить какая нужна точность + /// для результата. + template + UInt8 getPrecision(const ColumnPtr & column) + { + UInt8 precision = 0; + + if (!( PrecisionForType::apply(column, precision) + || PrecisionForType::apply(column, precision) + || PrecisionForType::apply(column, precision) + || PrecisionForType::apply(column, precision) + || PrecisionForType::apply(column, precision) + || PrecisionForType::apply(column, precision) + || PrecisionForType::apply(column, precision) + || PrecisionForType::apply(column, precision) + || PrecisionForType::apply(column, precision))) + { + throw Exception("Illegal column " + column->getName() + + " of second ('precision') argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } + + return precision; + } + + public: + /// Получить имя функции. + String getName() const override + { + return name; + } + + /// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. + DataTypePtr getReturnType(const DataTypes & arguments) const override + { + if ((arguments.size() < 1) || (arguments.size() > 2)) + 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); + + if (arguments.size() == 2) + { + const IDataType * type = &*arguments[1]; + if (!( checkType(type) + || checkType(type) + || checkType(type) + || checkType(type) + || checkType(type) + || checkType(type) + || checkType(type) + || checkType(type))) + { + throw Exception("Illegal type in second argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + } + + const IDataType * type = &*arguments[0]; + if (!type->behavesAsNumber()) + throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return arguments[0]; + } + + /// Выполнить функцию над блоком. + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override + { + if (!( executeForType(block, arguments, result) + || executeForType(block, arguments, result) + || executeForType(block, arguments, result) + || executeForType(block, arguments, result) + || executeForType(block, arguments, result) + || executeForType(block, arguments, result) + || executeForType(block, arguments, result) + || executeForType(block, arguments, result) + || executeForType(block, arguments, result) + || executeForType(block, arguments, result))) + { + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } + } + }; + +namespace +{ + /// Определение функцией для использования в шаблоне FunctionApproximating. + + template + struct RoundImpl + { + static inline T apply(T val) + { + return val; + } + }; + + template<> + struct RoundImpl + { + static inline DB::Float32 apply(DB::Float32 val) + { + return roundf(val); + } + }; + + template<> + struct RoundImpl + { + static inline DB::Float64 apply(DB::Float64 val) + { + return round(val); + } + }; + + template + struct CeilImpl + { + static inline T apply(T val) + { + return val; + } + }; + + template<> + struct CeilImpl + { + static inline DB::Float32 apply(DB::Float32 val) + { + return ceilf(val); + } + }; + + template<> + struct CeilImpl + { + static inline DB::Float64 apply(DB::Float64 val) + { + return ceil(val); + } + }; + + template + struct FloorImpl + { + static inline T apply(T val) + { + return val; + } + }; + + template<> + struct FloorImpl + { + static inline DB::Float32 apply(DB::Float32 val) + { + return floorf(val); + } + }; + + template<> + struct FloorImpl + { + static inline DB::Float64 apply(DB::Float64 val) + { + return floor(val); + } + }; +} struct NameRoundToExp2 { static constexpr auto name = "roundToExp2"; }; struct NameRoundDuration { static constexpr auto name = "roundDuration"; }; struct NameRoundAge { static constexpr auto name = "roundAge"; }; + struct NameRound { static constexpr auto name = "round"; }; + struct NameCeil { static constexpr auto name = "ceil"; }; + struct NameFloor { static constexpr auto name = "floor"; }; typedef FunctionUnaryArithmetic FunctionRoundToExp2; typedef FunctionUnaryArithmetic FunctionRoundDuration; typedef FunctionUnaryArithmetic FunctionRoundAge; - + typedef FunctionApproximating FunctionRound; + typedef FunctionApproximating FunctionCeil; + typedef FunctionApproximating FunctionFloor; } diff --git a/dbms/src/Functions/FunctionsRound.cpp b/dbms/src/Functions/FunctionsRound.cpp index bf40716a5c9..8ddbbe12ccc 100644 --- a/dbms/src/Functions/FunctionsRound.cpp +++ b/dbms/src/Functions/FunctionsRound.cpp @@ -9,6 +9,9 @@ void registerFunctionsRound(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); } }