#pragma once #include #include #include #include #include #include #include #include #include #include "IFunctionOld.h" #include #include #include #include #include #include #include #include #ifdef __SSE4_1__ #include #else #include #endif namespace DB { namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ARGUMENT_OUT_OF_BOUND; extern const int ILLEGAL_COLUMN; extern const int BAD_ARGUMENTS; extern const int CANNOT_SET_ROUNDING_MODE; } /** Rounding Functions: * round(x, N) - rounding to nearest (N = 0 by default). Use banker's rounding for floating point numbers. * roundBankers(x, N) - rounding to nearest (N = 0 by default). Use banker's rounding for all numbers. * floor(x, N) is the largest number <= x (N = 0 by default). * ceil(x, N) is the smallest number >= x (N = 0 by default). * trunc(x, N) - is the largest by absolute value number that is not greater than x by absolute value (N = 0 by default). * * The value of the parameter N (scale): * - N > 0: round to the number with N decimal places after the decimal point * - N < 0: round to an integer with N zero characters * - N = 0: round to an integer * * Type of the result is the type of argument. * For integer arguments, when passing negative scale, overflow can occur. * In that case, the behavior is implementation specific. */ /** This parameter controls the behavior of the rounding functions. */ enum class ScaleMode { Positive, // round to a number with N decimal places after the decimal point Negative, // round to an integer with N zero characters Zero, // round to an integer }; enum class RoundingMode { #ifdef __SSE4_1__ Round = _MM_FROUND_TO_NEAREST_INT | _MM_FROUND_NO_EXC, Floor = _MM_FROUND_TO_NEG_INF | _MM_FROUND_NO_EXC, Ceil = _MM_FROUND_TO_POS_INF | _MM_FROUND_NO_EXC, Trunc = _MM_FROUND_TO_ZERO | _MM_FROUND_NO_EXC, #else Round = 8, /// Values are correspond to above just in case. Floor = 9, Ceil = 10, Trunc = 11, #endif }; enum class TieBreakingMode { Auto, // use banker's rounding for floating point numbers, round up otherwise Bankers, // use banker's rounding }; /// For N, no more than the number of digits in the largest type. using Scale = Int16; /** Rounding functions for integer values. */ template struct IntegerRoundingComputation { static const size_t data_count = 1; static size_t prepare(size_t scale) { return scale; } /// Integer overflow is Ok. static ALWAYS_INLINE_NO_SANITIZE_UNDEFINED T computeImpl(T x, T scale) { switch (rounding_mode) { case RoundingMode::Trunc: { return x / scale * scale; } case RoundingMode::Floor: { if (x < 0) x -= scale - 1; return x / scale * scale; } case RoundingMode::Ceil: { if (x >= 0) x += scale - 1; return x / scale * scale; } case RoundingMode::Round: { if (x < 0) x -= scale; switch (tie_breaking_mode) { case TieBreakingMode::Auto: x = (x + scale / 2) / scale * scale; break; case TieBreakingMode::Bankers: { T quotient = (x + scale / 2) / scale; if (quotient * scale == x + scale / 2) // round half to even x = ((quotient + (x < 0)) & ~1) * scale; else // round the others as usual x = quotient * scale; break; } } return x; } } __builtin_unreachable(); } static ALWAYS_INLINE T compute(T x, T scale) { switch (scale_mode) { case ScaleMode::Zero: return x; case ScaleMode::Positive: return x; case ScaleMode::Negative: return computeImpl(x, scale); } __builtin_unreachable(); } static ALWAYS_INLINE void compute(const T * __restrict in, size_t scale, T * __restrict out) { if (sizeof(T) <= sizeof(scale) && scale > size_t(std::numeric_limits::max())) *out = 0; else *out = compute(*in, scale); } }; #ifdef __SSE4_1__ template class BaseFloatRoundingComputation; template <> class BaseFloatRoundingComputation { public: using ScalarType = Float32; using VectorType = __m128; static const size_t data_count = 4; static VectorType load(const ScalarType * in) { return _mm_loadu_ps(in); } static VectorType load1(const ScalarType in) { return _mm_load1_ps(&in); } static void store(ScalarType * out, VectorType val) { _mm_storeu_ps(out, val);} static VectorType multiply(VectorType val, VectorType scale) { return _mm_mul_ps(val, scale); } static VectorType divide(VectorType val, VectorType scale) { return _mm_div_ps(val, scale); } template static VectorType apply(VectorType val) { return _mm_round_ps(val, int(mode)); } static VectorType prepare(size_t scale) { return load1(scale); } }; template <> class BaseFloatRoundingComputation { public: using ScalarType = Float64; using VectorType = __m128d; static const size_t data_count = 2; static VectorType load(const ScalarType * in) { return _mm_loadu_pd(in); } static VectorType load1(const ScalarType in) { return _mm_load1_pd(&in); } static void store(ScalarType * out, VectorType val) { _mm_storeu_pd(out, val);} static VectorType multiply(VectorType val, VectorType scale) { return _mm_mul_pd(val, scale); } static VectorType divide(VectorType val, VectorType scale) { return _mm_div_pd(val, scale); } template static VectorType apply(VectorType val) { return _mm_round_pd(val, int(mode)); } static VectorType prepare(size_t scale) { return load1(scale); } }; #else /// Implementation for ARM. Not vectorized. inline float roundWithMode(float x, RoundingMode mode) { switch (mode) { case RoundingMode::Round: return nearbyintf(x); case RoundingMode::Floor: return floorf(x); case RoundingMode::Ceil: return ceilf(x); case RoundingMode::Trunc: return truncf(x); } __builtin_unreachable(); } inline double roundWithMode(double x, RoundingMode mode) { switch (mode) { case RoundingMode::Round: return nearbyint(x); case RoundingMode::Floor: return floor(x); case RoundingMode::Ceil: return ceil(x); case RoundingMode::Trunc: return trunc(x); } __builtin_unreachable(); } template class BaseFloatRoundingComputation { public: using ScalarType = T; using VectorType = T; static const size_t data_count = 1; static VectorType load(const ScalarType * in) { return *in; } static VectorType load1(const ScalarType in) { return in; } static VectorType store(ScalarType * out, ScalarType val) { return *out = val;} static VectorType multiply(VectorType val, VectorType scale) { return val * scale; } static VectorType divide(VectorType val, VectorType scale) { return val / scale; } template static VectorType apply(VectorType val) { return roundWithMode(val, mode); } static VectorType prepare(size_t scale) { return load1(scale); } }; #endif /** Implementation of low-level round-off functions for floating-point values. */ template class FloatRoundingComputation : public BaseFloatRoundingComputation { using Base = BaseFloatRoundingComputation; public: static inline void compute(const T * __restrict in, const typename Base::VectorType & scale, T * __restrict out) { auto val = Base::load(in); if (scale_mode == ScaleMode::Positive) val = Base::multiply(val, scale); else if (scale_mode == ScaleMode::Negative) val = Base::divide(val, scale); val = Base::template apply(val); if (scale_mode == ScaleMode::Positive) val = Base::divide(val, scale); else if (scale_mode == ScaleMode::Negative) val = Base::multiply(val, scale); Base::store(out, val); } }; /** Implementing high-level rounding functions. */ template struct FloatRoundingImpl { private: static_assert(!IsDecimalNumber); using Op = FloatRoundingComputation; using Data = std::array; using ColumnType = std::conditional_t, ColumnDecimal, ColumnVector>; using Container = typename ColumnType::Container; public: static NO_INLINE void apply(const Container & in, size_t scale, Container & out) { auto mm_scale = Op::prepare(scale); const size_t data_count = std::tuple_size(); const T* end_in = in.data() + in.size(); const T* limit = in.data() + in.size() / data_count * data_count; const T* __restrict p_in = in.data(); T* __restrict p_out = out.data(); while (p_in < limit) { Op::compute(p_in, mm_scale, p_out); p_in += data_count; p_out += data_count; } if (p_in < end_in) { Data tmp_src{{}}; Data tmp_dst; size_t tail_size_bytes = (end_in - p_in) * sizeof(*p_in); memcpy(&tmp_src, p_in, tail_size_bytes); Op::compute(reinterpret_cast(&tmp_src), mm_scale, reinterpret_cast(&tmp_dst)); memcpy(p_out, &tmp_dst, tail_size_bytes); } } }; template struct IntegerRoundingImpl { private: using Op = IntegerRoundingComputation; using Container = typename ColumnVector::Container; public: template static NO_INLINE void applyImpl(const Container & in, Container & out) { const T * end_in = in.data() + in.size(); const T * __restrict p_in = in.data(); T * __restrict p_out = out.data(); while (p_in < end_in) { Op::compute(p_in, scale, p_out); ++p_in; ++p_out; } } static NO_INLINE void apply(const Container & in, size_t scale, Container & out) { /// Manual function cloning for compiler to generate integer division by constant. switch (scale) { case 1ULL: return applyImpl<1ULL>(in, out); case 10ULL: return applyImpl<10ULL>(in, out); case 100ULL: return applyImpl<100ULL>(in, out); case 1000ULL: return applyImpl<1000ULL>(in, out); case 10000ULL: return applyImpl<10000ULL>(in, out); case 100000ULL: return applyImpl<100000ULL>(in, out); case 1000000ULL: return applyImpl<1000000ULL>(in, out); case 10000000ULL: return applyImpl<10000000ULL>(in, out); case 100000000ULL: return applyImpl<100000000ULL>(in, out); case 1000000000ULL: return applyImpl<1000000000ULL>(in, out); case 10000000000ULL: return applyImpl<10000000000ULL>(in, out); case 100000000000ULL: return applyImpl<100000000000ULL>(in, out); case 1000000000000ULL: return applyImpl<1000000000000ULL>(in, out); case 10000000000000ULL: return applyImpl<10000000000000ULL>(in, out); case 100000000000000ULL: return applyImpl<100000000000000ULL>(in, out); case 1000000000000000ULL: return applyImpl<1000000000000000ULL>(in, out); case 10000000000000000ULL: return applyImpl<10000000000000000ULL>(in, out); case 100000000000000000ULL: return applyImpl<100000000000000000ULL>(in, out); case 1000000000000000000ULL: return applyImpl<1000000000000000000ULL>(in, out); case 10000000000000000000ULL: return applyImpl<10000000000000000000ULL>(in, out); default: throw Exception("Unexpected 'scale' parameter passed to function", ErrorCodes::BAD_ARGUMENTS); } } }; template class DecimalRoundingImpl { private: static_assert(IsDecimalNumber); using NativeType = typename T::NativeType; using Op = IntegerRoundingComputation; using Container = typename ColumnDecimal::Container; public: static NO_INLINE void apply(const Container & in, Container & out, Scale scale_arg) { scale_arg = in.getScale() - scale_arg; if (scale_arg > 0) { size_t scale = intExp10(scale_arg); const NativeType * __restrict p_in = reinterpret_cast(in.data()); const NativeType * end_in = reinterpret_cast(in.data()) + in.size(); NativeType * __restrict p_out = reinterpret_cast(out.data()); while (p_in < end_in) { Op::compute(p_in, scale, p_out); ++p_in; ++p_out; } } else { memcpy(out.data(), in.data(), in.size() * sizeof(T)); } } }; /** Select the appropriate processing algorithm depending on the scale. */ template class Dispatcher { template using FunctionRoundingImpl = std::conditional_t, FloatRoundingImpl, IntegerRoundingImpl>; static ColumnPtr apply(const ColumnVector * col, Scale scale_arg) { auto col_res = ColumnVector::create(); typename ColumnVector::Container & vec_res = col_res->getData(); vec_res.resize(col->getData().size()); if (!vec_res.empty()) { if (scale_arg == 0) { size_t scale = 1; FunctionRoundingImpl::apply(col->getData(), scale, vec_res); } else if (scale_arg > 0) { size_t scale = intExp10(scale_arg); FunctionRoundingImpl::apply(col->getData(), scale, vec_res); } else { size_t scale = intExp10(-scale_arg); FunctionRoundingImpl::apply(col->getData(), scale, vec_res); } } return col_res; } static ColumnPtr apply(const ColumnDecimal * col, Scale scale_arg) { const typename ColumnDecimal::Container & vec_src = col->getData(); auto col_res = ColumnDecimal::create(vec_src.size(), vec_src.getScale()); auto & vec_res = col_res->getData(); if (!vec_res.empty()) DecimalRoundingImpl::apply(col->getData(), vec_res, scale_arg); return col_res; } public: static ColumnPtr apply(const IColumn * column, Scale scale_arg) { if constexpr (is_arithmetic_v) return apply(checkAndGetColumn>(column), scale_arg); else if constexpr (IsDecimalNumber) return apply(checkAndGetColumn>(column), scale_arg); } }; /** A template for functions that round the value of an input parameter of type * (U)Int8/16/32/64, Float32/64 or Decimal32/64/128, and accept an additional optional parameter (default is 0). */ template class FunctionRounding : public IFunction { public: static constexpr auto name = Name::name; static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } /// Get result types by argument types. If the function does not apply to these arguments, throw an exception. DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if ((arguments.empty()) || (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); for (const auto & type : arguments) if (!isNumber(type)) throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return arguments[0]; } static Scale getScaleArg(const ColumnsWithTypeAndName & arguments) { if (arguments.size() == 2) { const IColumn & scale_column = *arguments[1].column; if (!isColumnConst(scale_column)) throw Exception("Scale argument for rounding functions must be constant", ErrorCodes::ILLEGAL_COLUMN); Field scale_field = assert_cast(scale_column).getField(); if (scale_field.getType() != Field::Types::UInt64 && scale_field.getType() != Field::Types::Int64) throw Exception("Scale argument for rounding functions must have integer type", ErrorCodes::ILLEGAL_COLUMN); Int64 scale64 = scale_field.get(); if (scale64 > std::numeric_limits::max() || scale64 < std::numeric_limits::min()) throw Exception("Scale argument for rounding function is too large", ErrorCodes::ARGUMENT_OUT_OF_BOUND); return scale64; } return 0; } bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override { const ColumnWithTypeAndName & column = arguments[0]; Scale scale_arg = getScaleArg(arguments); ColumnPtr res; auto call = [&](const auto & types) -> bool { using Types = std::decay_t; using DataType = typename Types::LeftType; if constexpr (IsDataTypeNumber || IsDataTypeDecimal) { using FieldType = typename DataType::FieldType; res = Dispatcher::apply(column.column.get(), scale_arg); return true; } return false; }; #if !defined(__SSE4_1__) /// In case of "nearbyint" function is used, we should ensure the expected rounding mode for the Banker's rounding. /// Actually it is by default. But we will set it just in case. if constexpr (rounding_mode == RoundingMode::Round) if (0 != fesetround(FE_TONEAREST)) throw Exception("Cannot set floating point rounding mode", ErrorCodes::CANNOT_SET_ROUNDING_MODE); #endif if (!callOnIndexAndDataType(column.type->getTypeId(), call)) { throw Exception("Illegal column " + column.name + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } return res; } bool hasInformationAboutMonotonicity() const override { return true; } Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override { return { true, true, true }; } }; /** Rounds down to a number within explicitly specified array. * If the value is less than the minimal bound - returns the minimal bound. */ class FunctionRoundDown : public IFunction { public: static constexpr auto name = "roundDown"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } bool isVariadic() const override { return false; } size_t getNumberOfArguments() const override { return 2; } bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { const DataTypePtr & type_x = arguments[0]; if (!isNumber(type_x)) throw Exception{"Unsupported type " + type_x->getName() + " of first argument of function " + getName() + ", must be numeric type.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; const DataTypeArray * type_arr = checkAndGetDataType(arguments[1].get()); if (!type_arr) throw Exception{"Second argument of function " + getName() + ", must be array of boundaries to round to.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; const auto type_arr_nested = type_arr->getNestedType(); if (!isNumber(type_arr_nested)) { throw Exception{"Elements of array of second argument of function " + getName() + " must be numeric type.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } return getLeastSupertype({type_x, type_arr_nested}); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t) const override { auto in_column = arguments[0].column; const auto & in_type = arguments[0].type; auto array_column = arguments[1].column; const auto & array_type = arguments[1].type; const auto & return_type = result_type; auto column_result = return_type->createColumn(); auto * out = column_result.get(); if (!in_type->equals(*return_type)) in_column = castColumn(arguments[0], return_type); if (!array_type->equals(*return_type)) array_column = castColumn(arguments[1], std::make_shared(return_type)); const auto * in = in_column.get(); auto boundaries = typeid_cast(*array_column).getValue(); size_t num_boundaries = boundaries.size(); if (!num_boundaries) throw Exception("Empty array is illegal for boundaries in " + getName() + " function", ErrorCodes::BAD_ARGUMENTS); if (!executeNum(in, out, boundaries) && !executeNum(in, out, boundaries) && !executeNum(in, out, boundaries) && !executeNum(in, out, boundaries) && !executeNum(in, out, boundaries) && !executeNum(in, out, boundaries) && !executeNum(in, out, boundaries) && !executeNum(in, out, boundaries) && !executeNum(in, out, boundaries) && !executeNum(in, out, boundaries) && !executeDecimal(in, out, boundaries) && !executeDecimal(in, out, boundaries) && !executeDecimal(in, out, boundaries)) { throw Exception{"Illegal column " + in->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } return column_result; } private: template bool executeNum(const IColumn * in_untyped, IColumn * out_untyped, const Array & boundaries) const { const auto in = checkAndGetColumn>(in_untyped); auto out = typeid_cast *>(out_untyped); if (!in || !out) return false; executeImplNumToNum(in->getData(), out->getData(), boundaries); return true; } template bool executeDecimal(const IColumn * in_untyped, IColumn * out_untyped, const Array & boundaries) const { const auto in = checkAndGetColumn>(in_untyped); auto out = typeid_cast *>(out_untyped); if (!in || !out) return false; executeImplNumToNum(in->getData(), out->getData(), boundaries); return true; } template void NO_INLINE executeImplNumToNum(const Container & src, Container & dst, const Array & boundaries) const { using ValueType = typename Container::value_type; std::vector boundary_values(boundaries.size()); for (size_t i = 0; i < boundaries.size(); ++i) boundary_values[i] = boundaries[i].get(); std::sort(boundary_values.begin(), boundary_values.end()); boundary_values.erase(std::unique(boundary_values.begin(), boundary_values.end()), boundary_values.end()); size_t size = src.size(); dst.resize(size); if (boundary_values.size() < 32) /// Just a guess { /// Linear search with value on previous iteration as a hint. /// Not optimal if the size of list is large and distribution of values is uniform random. auto begin = boundary_values.begin(); auto end = boundary_values.end(); auto it = begin + (end - begin) / 2; for (size_t i = 0; i < size; ++i) { auto value = src[i]; if (*it < value) { while (it != end && *it <= value) ++it; if (it != begin) --it; } else { while (*it > value && it != begin) --it; } dst[i] = *it; } } else { for (size_t i = 0; i < size; ++i) { auto it = std::upper_bound(boundary_values.begin(), boundary_values.end(), src[i]); if (it == boundary_values.end()) { dst[i] = boundary_values.back(); } else if (it == boundary_values.begin()) { dst[i] = boundary_values.front(); } else { dst[i] = *(it - 1); } } } } }; struct NameRound { static constexpr auto name = "round"; }; struct NameRoundBankers { static constexpr auto name = "roundBankers"; }; struct NameCeil { static constexpr auto name = "ceil"; }; struct NameFloor { static constexpr auto name = "floor"; }; struct NameTrunc { static constexpr auto name = "trunc"; }; using FunctionRound = FunctionRounding; using FunctionRoundBankers = FunctionRounding; using FunctionFloor = FunctionRounding; using FunctionCeil = FunctionRounding; using FunctionTrunc = FunctionRounding; }