Feature implementation. [#METR-15210]

This commit is contained in:
Alexey Arno 2015-05-13 01:42:11 +03:00
parent 3cea0c83bb
commit 02191fcfe3

View File

@ -142,31 +142,123 @@ namespace DB
}
};
template<typename A, typename B>
template<typename A>
struct RoundImpl
{
typedef A ResultType;
template <typename Result = ResultType>
static inline Result apply(A a, B b)
static inline A apply(A a, Int8 scale)
{
auto vb = typename NumberTraits::ToInteger<B>::Type(b);
if (scale < 0)
scale = 0;
if (vb < 0)
vb = 0;
size_t power = (scale < static_cast<Int8>(powers_count)) ? powers_of_10::value[scale] : pow(10, scale);
return static_cast<A>(round(a * power) / power);
}
};
size_t scale = (vb < static_cast<int>(powers_count)) ? powers_of_10::value[vb] : pow(10, vb);
return static_cast<Result>(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 <typename T0>
bool executeType(Block & block, const ColumnNumbers & arguments, Int8 scale, size_t result)
{
if (ColumnVector<T0> * col = typeid_cast<ColumnVector<T0> *>(&*block.getByPosition(arguments[0]).column))
{
ColumnVector<T0> * col_res = new ColumnVector<T0>;
block.getByPosition(result).column = col_res;
typename ColumnVector<T0>::Container_t & vec_res = col_res->getData();
vec_res.resize(col->getData().size());
const PODArray<T0> & a = col->getData();
size_t size = a.size();
for (size_t i = 0; i < size; ++i)
vec_res[i] = RoundImpl<T0>::apply(a[i], scale);
return true;
}
else if (ColumnConst<T0> * col = typeid_cast<ColumnConst<T0> *>(&*block.getByPosition(arguments[0]).column))
{
T0 res = RoundImpl<T0>::apply(col->getData(), scale);
ColumnConst<T0> * col_res = new ColumnConst<T0>(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<Int8>::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<Int8> * scale_col = typeid_cast<const ColumnConstInt8 *>(&*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<UInt8>(block, arguments, scale, result)
|| executeType<UInt16>(block, arguments, scale, result)
|| executeType<UInt32>(block, arguments, scale, result)
|| executeType<UInt64>(block, arguments, scale, result)
|| executeType<Int8>(block, arguments, scale, result)
|| executeType<Int16>(block, arguments, scale, result)
|| executeType<Int32>(block, arguments, scale, result)
|| executeType<Int64>(block, arguments, scale, result)
|| executeType<Float32>(block, arguments, scale, result)
|| executeType<Float64>(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<RoundToExp2Impl, NameRoundToExp2> FunctionRoundToExp2;
typedef FunctionUnaryArithmetic<RoundDurationImpl, NameRoundDuration> FunctionRoundDuration;
typedef FunctionUnaryArithmetic<RoundAgeImpl, NameRoundAge> FunctionRoundAge;
typedef FunctionBinaryArithmetic<RoundImpl, NameRound> FunctionRound;
}