#pragma once #include #include #include #include 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 по умолчанию). * * Значение параметра N: * - N > 0: округлять до числа с N десятичными знаками после запятой * - N < 0: окурглять до целого числа с N нулевыми знаками * - N = 0: округлять до целого числа */ template struct RoundToExp2Impl { typedef A ResultType; static inline A apply(A x) { return x <= 0 ? static_cast(0) : (static_cast(1) << static_cast(log2(static_cast(x)))); } }; template<> struct RoundToExp2Impl { typedef Float32 ResultType; static inline Float32 apply(Float32 x) { return static_cast(x < 1 ? 0. : pow(2., floor(log2(x)))); } }; template<> struct RoundToExp2Impl { typedef Float64 ResultType; static inline Float64 apply(Float64 x) { return x < 1 ? 0. : pow(2., floor(log2(x))); } }; template struct RoundDurationImpl { typedef UInt16 ResultType; static inline ResultType apply(A x) { return x < 1 ? 0 : (x < 10 ? 1 : (x < 30 ? 10 : (x < 60 ? 30 : (x < 120 ? 60 : (x < 180 ? 120 : (x < 240 ? 180 : (x < 300 ? 240 : (x < 600 ? 300 : (x < 1200 ? 600 : (x < 1800 ? 1200 : (x < 3600 ? 1800 : (x < 7200 ? 3600 : (x < 18000 ? 7200 : (x < 36000 ? 18000 : 36000)))))))))))))); } }; template struct RoundAgeImpl { typedef UInt8 ResultType; static inline ResultType apply(A x) { return x < 18 ? 0 : (x < 25 ? 18 : (x < 35 ? 25 : (x < 45 ? 35 : 45))); } }; /** Быстрое вычисление остатка от деления для применения к округлению целых чисел. * Без проверки, потому что делитель всегда положительный. */ template struct FastModulo; template struct FastModulo::value>::type> { private: template struct Extend; template struct Extend::value || std::is_same::value>::type> { using Type = Int64; }; template struct Extend::value || std::is_same::value>::type> { using Type = UInt64; }; template struct Extend::value && (sizeof(InputType) >= 4)>::type> { using Type = InputType; }; using U = typename Extend::Type; public: using Divisor = std::pair >; static inline Divisor prepare(size_t b) { return std::make_pair(b, libdivide::divider(b)); } static inline T compute(T a, const Divisor & divisor) { U val = static_cast(a); U rem = val - (val / divisor.second) * static_cast(divisor.first); return static_cast(rem); } }; /** Этот параметр контролирует поведение функций округления. */ enum ScaleMode { PositiveScale, // округлять до числа с N десятичными знаками после запятой NegativeScale, // окурглять до целого числа с N нулевыми знаками ZeroScale, // округлять до целого числа NullScale // возвращать нулевое значение }; /** Реализация низкоуровневых функций округления для целочисленных значений. */ template struct IntegerRoundingComputation; template struct IntegerRoundingComputation::value && ((scale_mode == PositiveScale) || (scale_mode == ZeroScale))>::type> { using Divisor = int; static inline Divisor prepare(size_t scale) { return 0; } static inline T compute(T in, const Divisor & scale) { return in; } }; template struct IntegerRoundingComputation::value>::type> { using Op = FastModulo; using Divisor = typename Op::Divisor; static inline Divisor prepare(size_t scale) { return Op::prepare(scale); } static inline T compute(T in, const Divisor & scale) { T factor = (in < 0) ? -1 : 1; in *= factor; T rem = Op::compute(in, scale); in -= rem; T res; if ((2 * rem) < static_cast(scale.first)) res = in; else res = in + scale.first; return factor * res; } }; template struct IntegerRoundingComputation::value>::type> { using Op = FastModulo; using Divisor = typename Op::Divisor; static inline Divisor prepare(size_t scale) { return Op::prepare(scale); } static inline T compute(T in, const Divisor & scale) { T factor = (in < 0) ? -1 : 1; in *= factor; T rem = Op::compute(in, scale); T res = in - rem + scale.first; return factor * res; } }; template struct IntegerRoundingComputation::value>::type> { using Op = FastModulo; using Divisor = typename Op::Divisor; static inline Divisor prepare(size_t scale) { return Op::prepare(scale); } static inline T compute(T in, const Divisor & scale) { T factor = (in < 0) ? -1 : 1; in *= factor; T rem = Op::compute(in, scale); T res = in - rem; return factor * res; } }; template class BaseFloatRoundingComputation; template<> class BaseFloatRoundingComputation { public: using Scale = __m128; static const size_t data_count = 4; protected: /// Предотвратить появление отрицательных нолей определённых в стандарте IEEE-754. static inline void normalize(__m128 & val, const __m128 & mask) { __m128 mask1 = _mm_cmpeq_ps(val, getZero()); __m128 mask2 = _mm_and_ps(mask, mask1); mask2 = _mm_cmpeq_ps(mask2, getZero()); mask2 = _mm_min_ps(mask2, getTwo()); mask2 = _mm_sub_ps(mask2, getOne()); val = _mm_mul_ps(val, mask2); } static inline const __m128 & getZero() { static const __m128 zero = _mm_set1_ps(0.0); return zero; } static inline const __m128 & getOne() { static const __m128 one = _mm_set1_ps(1.0); return one; } static inline const __m128 & getTwo() { static const __m128 two = _mm_set1_ps(2.0); return two; } }; template<> class BaseFloatRoundingComputation { public: using Scale = __m128d; static const size_t data_count = 2; protected: /// Предотвратить появление отрицательных нолей определённых в стандарте IEEE-754. static inline void normalize(__m128d & val, const __m128d & mask) { __m128d mask1 = _mm_cmpeq_pd(val, getZero()); __m128d mask2 = _mm_and_pd(mask, mask1); mask2 = _mm_cmpeq_pd(mask2, getZero()); mask2 = _mm_min_pd(mask2, getTwo()); mask2 = _mm_sub_pd(mask2, getOne()); val = _mm_mul_pd(val, mask2); } static inline const __m128d & getZero() { static const __m128d zero = _mm_set1_pd(0.0); return zero; } static inline const __m128d & getOne() { static const __m128d one = _mm_set1_pd(1.0); return one; } static inline const __m128d & getTwo() { static const __m128d two = _mm_set1_pd(2.0); return two; } }; /** Реализация низкоуровневых функций округления для значений с плавающей точкой. */ template class FloatRoundingComputation; template class FloatRoundingComputation : public BaseFloatRoundingComputation { public: static inline void prepare(size_t scale, Scale & mm_scale) { Float32 fscale = static_cast(scale); mm_scale = _mm_load1_ps(&fscale); } static inline void compute(const Float32 * __restrict in, const Scale & scale, Float32 * __restrict out) { __m128 val = _mm_loadu_ps(in); __m128 mask = _mm_cmplt_ps(val, getZero()); /// Алгоритм округления. val = _mm_mul_ps(val, scale); val = _mm_round_ps(val, rounding_mode); val = _mm_div_ps(val, scale); normalize(val, mask); _mm_storeu_ps(out, val); } }; template class FloatRoundingComputation : public BaseFloatRoundingComputation { public: static inline void prepare(size_t scale, Scale & mm_scale) { Float32 fscale = static_cast(scale); mm_scale = _mm_load1_ps(&fscale); } static inline void compute(const Float32 * __restrict in, const Scale & scale, Float32 * __restrict out) { __m128 val = _mm_loadu_ps(in); __m128 mask = _mm_cmplt_ps(val, getZero()); /// Превратить отрицательные значения в положительные. __m128 factor = _mm_cmpge_ps(val, getZero()); factor = _mm_min_ps(factor, getTwo()); factor = _mm_sub_ps(factor, getOne()); val = _mm_mul_ps(val, factor); /// Алгоритм округления. val = _mm_div_ps(val, scale); __m128 res = _mm_cmpge_ps(val, getOneTenth()); val = _mm_round_ps(val, rounding_mode); val = _mm_mul_ps(val, scale); val = _mm_and_ps(val, res); /// Вернуть настоящие знаки всех значений. val = _mm_mul_ps(val, factor); normalize(val, mask); _mm_storeu_ps(out, val); } private: static inline const __m128 & getOneTenth() { static const __m128 one_tenth = _mm_set1_ps(0.1); return one_tenth; } }; template class FloatRoundingComputation : public BaseFloatRoundingComputation { public: static inline void prepare(size_t scale, Scale & mm_scale) { } static inline void compute(const Float32 * __restrict in, const Scale & scale, Float32 * __restrict out) { __m128 val = _mm_loadu_ps(in); __m128 mask = _mm_cmplt_ps(val, getZero()); val = _mm_round_ps(val, rounding_mode); normalize(val, mask); _mm_storeu_ps(out, val); } }; template class FloatRoundingComputation : public BaseFloatRoundingComputation { public: static inline void prepare(size_t scale, Scale & mm_scale) { Float64 fscale = static_cast(scale); mm_scale = _mm_load1_pd(&fscale); } static inline void compute(const Float64 * __restrict in, const Scale & scale, Float64 * __restrict out) { __m128d val = _mm_loadu_pd(in); __m128d mask = _mm_cmplt_pd(val, getZero()); /// Алгоритм округления. val = _mm_mul_pd(val, scale); val = _mm_round_pd(val, rounding_mode); val = _mm_div_pd(val, scale); normalize(val, mask); _mm_storeu_pd(out, val); } }; template class FloatRoundingComputation : public BaseFloatRoundingComputation { public: static inline void prepare(size_t scale, Scale & mm_scale) { Float64 fscale = static_cast(scale); mm_scale = _mm_load1_pd(&fscale); } static inline void compute(const Float64 * __restrict in, const Scale & scale, Float64 * __restrict out) { __m128d val = _mm_loadu_pd(in); __m128d mask = _mm_cmplt_pd(val, getZero()); /// Превратить отрицательные значения в положительные. __m128d factor = _mm_cmpge_pd(val, getZero()); factor = _mm_min_pd(factor, getTwo()); factor = _mm_sub_pd(factor, getOne()); val = _mm_mul_pd(val, factor); /// Алгоритм округления. val = _mm_div_pd(val, scale); __m128d res = _mm_cmpge_pd(val, getOneTenth()); val = _mm_round_pd(val, rounding_mode); val = _mm_mul_pd(val, scale); val = _mm_and_pd(val, res); /// Вернуть настоящие знаки всех значений. val = _mm_mul_pd(val, factor); normalize(val, mask); _mm_storeu_pd(out, val); } private: static inline const __m128d & getOneTenth() { static const __m128d one_tenth = _mm_set1_pd(0.1); return one_tenth; } }; template class FloatRoundingComputation : public BaseFloatRoundingComputation { public: static inline void prepare(size_t scale, Scale & mm_scale) { } static inline void compute(const Float64 * __restrict in, const Scale & scale, Float64 * __restrict out) { __m128d val = _mm_loadu_pd(in); __m128d mask = _mm_cmplt_pd(val, getZero()); val = _mm_round_pd(val, rounding_mode); normalize(val, mask); _mm_storeu_pd(out, val); } }; /** Реализация высокоуровневых функций округления. */ template struct FunctionRoundingImpl; /** Реализация высокоуровневых функций округления для целочисленных значений. */ template struct FunctionRoundingImpl::value && (scale_mode != NullScale)>::type> { private: using Op = IntegerRoundingComputation; public: static inline void apply(const PODArray & in, size_t scale, typename ColumnVector::Container_t & out) { auto divisor = Op::prepare(scale); const T* begin_in = &in[0]; const T* end_in = begin_in + in.size(); T* __restrict p_out = &out[0]; for (const T* __restrict p_in = begin_in; p_in != end_in; ++p_in) { *p_out = Op::compute(*p_in, divisor); ++p_out; } } static inline T apply(T val, size_t scale) { auto divisor = Op::prepare(scale); return Op::compute(val, divisor); } }; /** Реализация высокоуровневых функций округления для значений с плавающей точкой. */ template struct FunctionRoundingImpl::value && (scale_mode != NullScale)>::type> { private: using Op = FloatRoundingComputation; using Data = std::array; using Scale = typename Op::Scale; public: static inline void apply(const PODArray & in, size_t scale, typename ColumnVector::Container_t & out) { Scale mm_scale; Op::prepare(scale, mm_scale); const size_t data_count = std::tuple_size(); const T* begin_in = &in[0]; const T* end_in = begin_in + in.size(); T* begin_out = &out[0]; const T* end_out = begin_out + out.size(); const T* limit = end_in - (data_count - 1); const T* __restrict p_in = begin_in; T* __restrict p_out = begin_out; for (; p_in < limit; p_in += data_count) { Op::compute(p_in, mm_scale, p_out); p_out += data_count; } if (p_in < end_in) { Data tmp{0}; T* begin_tmp = &tmp[0]; const T* end_tmp = begin_tmp + data_count; for (T* __restrict p_tmp = begin_tmp; (p_tmp != end_tmp) && (p_in != end_in); ++p_tmp) { *p_tmp = *p_in; ++p_in; } Data res; const T* begin_res = &res[0]; const T* end_res = begin_res + data_count; Op::compute(reinterpret_cast(&tmp), mm_scale, reinterpret_cast(&res)); for (const T* __restrict p_res = begin_res; (p_res != end_res) && (p_out != end_out); ++p_res) { *p_out = *p_res; ++p_out; } } } static inline T apply(T val, size_t scale) { if (val == 0) return val; else { Scale mm_scale; Op::prepare(scale, mm_scale); Data tmp{0}; tmp[0] = val; Data res; Op::compute(reinterpret_cast(&tmp), mm_scale, reinterpret_cast(&res)); return res[0]; } } }; /** Реализация высокоуровневых функций округления в том случае, когда возвращается нулевое значение. */ template struct FunctionRoundingImpl::type> { public: static inline void apply(const PODArray & in, size_t scale, typename ColumnVector::Container_t & out) { ::memset(reinterpret_cast(&out[0]), 0, in.size() * sizeof(T)); } static inline T apply(T val, size_t scale) { return 0; } }; /// Следующий код генерирует во время сборки таблицу степеней числа 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; }; /** Этот шаблон определяет точность, которую используют функции round/ceil/floor, * затем преобразовывает её в значение, которое можно использовать в операциях * умножения и деления. Поэтому оно называется масштабом. */ template struct ScaleForRightType; template struct ScaleForRightType::value && std::is_signed::value>::type> { static inline bool apply(const ColumnPtr & column, ScaleMode & scale_mode, size_t & scale) { using PowersOf10 = typename FillArray::digits10 + 1>::result; using ColumnType = ColumnConst; const ColumnType * precision_col = typeid_cast(&*column); if (precision_col == nullptr) return false; U val = precision_col->getData(); if (val < 0) { if (val < -static_cast(std::numeric_limits::digits10)) { scale_mode = NullScale; scale = 1; } else { scale_mode = NegativeScale; scale = PowersOf10::values[-val]; } } else if (val == 0) { scale_mode = ZeroScale; scale = 1; } else { scale_mode = PositiveScale; if (val > std::numeric_limits::digits10) val = static_cast(std::numeric_limits::digits10); scale = PowersOf10::values[val]; } return true; } }; template struct ScaleForRightType::value && std::is_unsigned::value>::type> { static inline bool apply(const ColumnPtr & column, ScaleMode & scale_mode, size_t & scale) { using PowersOf10 = typename FillArray::digits10 + 1>::result; using ColumnType = ColumnConst; const ColumnType * precision_col = typeid_cast(&*column); if (precision_col == nullptr) return false; U val = precision_col->getData(); if (val == 0) { scale_mode = ZeroScale; scale = 1; } else { scale_mode = PositiveScale; if (val > static_cast(std::numeric_limits::digits10)) val = static_cast(std::numeric_limits::digits10); scale = PowersOf10::values[val]; } return true; } }; template struct ScaleForRightType::value && std::is_signed::value>::type> { static inline bool apply(const ColumnPtr & column, ScaleMode & scale_mode, size_t & scale) { using PowersOf10 = typename FillArray::digits10 + 1>::result; using ColumnType = ColumnConst; const ColumnType * precision_col = typeid_cast(&*column); if (precision_col == nullptr) return false; U val = precision_col->getData(); if (val < 0) { if (val < -std::numeric_limits::digits10) { scale_mode = NullScale; scale = 1; } else { scale_mode = NegativeScale; scale = PowersOf10::values[-val]; } } else { scale_mode = ZeroScale; scale = 1; } return true; } }; template struct ScaleForRightType::value && std::is_unsigned::value>::type> { static inline bool apply(const ColumnPtr & column, ScaleMode & scale_mode, size_t & scale) { using ColumnType = ColumnConst; const ColumnType * precision_col = typeid_cast(&*column); if (precision_col == nullptr) return false; scale_mode = ZeroScale; scale = 1; return true; } }; /** Превратить параметр точности в масштаб. */ template struct ScaleForLeftType { static inline void apply(const ColumnPtr & column, ScaleMode & scale_mode, size_t & scale) { if (!( ScaleForRightType::apply(column, scale_mode, scale) || ScaleForRightType::apply(column, scale_mode, scale) || ScaleForRightType::apply(column, scale_mode, scale) || ScaleForRightType::apply(column, scale_mode, scale) || ScaleForRightType::apply(column, scale_mode, scale) || ScaleForRightType::apply(column, scale_mode, scale) || ScaleForRightType::apply(column, scale_mode, scale) || ScaleForRightType::apply(column, scale_mode, scale) || ScaleForRightType::apply(column, scale_mode, scale) || ScaleForRightType::apply(column, scale_mode, scale) || ScaleForRightType::apply(column, scale_mode, scale))) { throw Exception("Internal error", ErrorCodes::LOGICAL_ERROR); } } }; /** Главный шаблон применяющий функцию округления к значению или столбцу. */ template struct Cruncher { using Op = FunctionRoundingImpl; static inline void apply(Block & block, ColumnVector * col, const ColumnNumbers & arguments, size_t result, size_t scale) { 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()); Op::apply(col->getData(), scale, vec_res); } static inline void apply(Block & block, ColumnConst * col, const ColumnNumbers & arguments, size_t result, size_t scale) { T res = Op::apply(col->getData(), scale); ColumnConst * col_res = new ColumnConst(col->size(), res); block.getByPosition(result).column = col_res; } }; /** Выбрать подходящий алгоритм обработки в зависимости от масштаба. */ template class U, int rounding_mode> struct Dispatcher { static inline void apply(Block & block, U * col, const ColumnNumbers & arguments, size_t result) { ScaleMode scale_mode; size_t scale; if (arguments.size() == 2) ScaleForLeftType::apply(block.getByPosition(arguments[1]).column, scale_mode, scale); else { scale_mode = ZeroScale; scale = 1; } if (scale_mode == PositiveScale) Cruncher::apply(block, col, arguments, result, scale); else if (scale_mode == ZeroScale) Cruncher::apply(block, col, arguments, result, scale); else if (scale_mode == NegativeScale) Cruncher::apply(block, col, arguments, result, scale); else if (scale_mode == NullScale) Cruncher::apply(block, col, arguments, result, scale); else throw Exception("Illegal operation", ErrorCodes::LOGICAL_ERROR); } }; /** Шаблон для функций, которые округляют значение входного параметра типа * (U)Int8/16/32/64 или Float32/64, и принимают дополнительный необязятельный * параметр (по умолчанию - 0). */ template class FunctionRounding : public IFunction { public: static constexpr auto name = Name::name; static IFunction * create(const Context & context) { return new FunctionRounding; } 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)) { Dispatcher::apply(block, col, arguments, result); return true; } else if (ColumnConst * col = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) { Dispatcher::apply(block, col, arguments, result); return true; } else return false; } 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) || 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); } } }; 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 FunctionRounding FunctionRound; typedef FunctionRounding FunctionCeil; typedef FunctionRounding FunctionFloor; }