From 02191fcfe332560de157e41df30ca86ee058080b Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Wed, 13 May 2015 01:42:11 +0300 Subject: [PATCH] Feature implementation. [#METR-15210] --- dbms/include/DB/Functions/FunctionsRound.h | 116 ++++++++++++++++++--- 1 file changed, 104 insertions(+), 12 deletions(-) diff --git a/dbms/include/DB/Functions/FunctionsRound.h b/dbms/include/DB/Functions/FunctionsRound.h index 34b236f3b55..31e1ed35e78 100644 --- a/dbms/include/DB/Functions/FunctionsRound.h +++ b/dbms/include/DB/Functions/FunctionsRound.h @@ -142,31 +142,123 @@ namespace DB } }; - template + template struct RoundImpl { - typedef A ResultType; - - template - static inline Result apply(A a, B b) + static inline A apply(A a, Int8 scale) { - auto vb = typename NumberTraits::ToInteger::Type(b); + if (scale < 0) + scale = 0; - if (vb < 0) - vb = 0; + size_t power = (scale < static_cast(powers_count)) ? powers_of_10::value[scale] : pow(10, scale); + return static_cast(round(a * power) / power); + } + }; - size_t scale = (vb < static_cast(powers_count)) ? powers_of_10::value[vb] : pow(10, vb); - return static_cast(round(a * scale) / scale); + class FunctionRound : public IFunction + { + public: + static constexpr auto name = "round"; + static IFunction * create(const Context & context) { return new FunctionRound; } + + private: + template + bool executeType(Block & block, const ColumnNumbers & arguments, Int8 scale, size_t result) + { + if (ColumnVector * col = typeid_cast *>(&*block.getByPosition(arguments[0]).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] = RoundImpl::apply(a[i], scale); + + return true; + } + else if (ColumnConst * col = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) + { + T0 res = RoundImpl::apply(col->getData(), scale); + + ColumnConst * col_res = new ColumnConst(col->size(), res); + block.getByPosition(result).column = col_res; + + return true; + } + + 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.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if ((arguments.size() == 2) && (arguments[1]->getName() != TypeName::get())) + throw Exception("Illegal type in second argument", 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]; + + DataTypePtr result; + } + + /// Выполнить функцию над блоком. + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override + { + Int8 scale = 0; + if (arguments.size() == 2) + { + const ColumnConst * scale_col = typeid_cast(&*block.getByPosition(arguments[1]).column); + + if (scale_col == nullptr) + throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName() + + " of second ('scale') argument of function " + name + + ". Must be constant int8.", + ErrorCodes::ILLEGAL_COLUMN); + + scale = scale_col->getData(); + } + + if (!( executeType(block, arguments, scale, result) + || executeType(block, arguments, scale, result) + || executeType(block, arguments, scale, result) + || executeType(block, arguments, scale, result) + || executeType(block, arguments, scale, result) + || executeType(block, arguments, scale, result) + || executeType(block, arguments, scale, result) + || executeType(block, arguments, scale, result) + || executeType(block, arguments, scale, result) + || executeType(block, arguments, scale, 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"; }; typedef FunctionUnaryArithmetic FunctionRoundToExp2; typedef FunctionUnaryArithmetic FunctionRoundDuration; typedef FunctionUnaryArithmetic FunctionRoundAge; - typedef FunctionBinaryArithmetic FunctionRound; }