Feature implementation. [#METR-15210]

This commit is contained in:
Alexey Arno 2015-05-13 16:40:01 +03:00
parent 533b69ef4c
commit af04e3e49d

View File

@ -1,8 +1,9 @@
#pragma once
#include <DB/Functions/FunctionsArithmetic.h>
#include <cmath> // log2()
#include <boost/concept_check.hpp>
#include <cmath>
#include <type_traits>
#include <array>
namespace
{
@ -28,11 +29,11 @@ struct PowerOf10<0>
template <size_t... TArgs>
struct TablePowersOf10
{
static const size_t value[sizeof...(TArgs)];
static const std::array<size_t, sizeof...(TArgs)> value;
};
template <size_t... TArgs>
const size_t TablePowersOf10<TArgs...>::value[sizeof...(TArgs)] = { TArgs... };
const std::array<size_t, sizeof...(TArgs)> TablePowersOf10<TArgs...>::value = { TArgs... };
/// Сгенерить первые N степеней.
@ -54,8 +55,7 @@ struct FillArray
using result = typename FillArrayImpl<N-1>::result;
};
static const size_t powers_count = 16;
using powers_of_10 = FillArray<powers_count>::result;
using powers_of_10 = FillArray<16>::result;
}
@ -145,44 +145,110 @@ namespace DB
}
};
template<typename T>
struct RoundImpl
{
static inline double apply(double x)
static inline T apply(T val)
{
return round(x);
throw Exception("Invalid invokation", ErrorCodes::LOGICAL_ERROR);
}
};
template<>
struct RoundImpl<Float32>
{
static inline Float32 apply(Float32 val)
{
return roundf(val);
}
};
template<>
struct RoundImpl<Float64>
{
static inline Float64 apply(Float64 val)
{
return round(val);
}
};
template<typename T>
struct CeilImpl
{
static inline double apply(double x)
static inline T apply(T val)
{
return ceil(x);
throw Exception("Invalid invokation", ErrorCodes::LOGICAL_ERROR);
}
};
template<>
struct CeilImpl<Float32>
{
static inline Float32 apply(Float32 val)
{
return ceilf(val);
}
};
template<>
struct CeilImpl<Float64>
{
static inline Float64 apply(Float64 val)
{
return ceil(val);
}
};
template<typename T>
struct FloorImpl
{
static inline double apply(double x)
static inline T apply(T val)
{
return floor(x);
throw Exception("Invalid invokation", ErrorCodes::LOGICAL_ERROR);
}
};
template<typename A, typename Op>
template<>
struct FloorImpl<Float32>
{
static inline Float32 apply(Float32 val)
{
return floorf(val);
}
};
template<>
struct FloorImpl<Float64>
{
static inline Float64 apply(Float64 val)
{
return floor(val);
}
};
template<typename A, template<typename> class Op, typename PowersTable>
struct FunctionApproximatingImpl
{
static inline A apply(A a, Int8 scale)
template <typename A2 = A>
static inline A2 apply(A2 a, UInt8 scale, typename std::enable_if<std::is_floating_point<A2>::value>::type * = nullptr)
{
if (scale < 0)
scale = 0;
if (a == 0)
return a;
else
{
size_t power = PowersTable::value[scale];
return Op<A2>::apply(a * power) / power;
}
}
size_t power = (scale < static_cast<Int8>(powers_count)) ? powers_of_10::value[scale] : pow(10, scale);
return static_cast<A>(Op::apply(a * power) / power);
template <typename A2 = A>
static inline A2 apply(A2 a, UInt8 scale, typename std::enable_if<std::is_integral<A2>::value>::type * = nullptr)
{
return a;
}
};
template<typename Op, typename Name>
template<template<typename> class Op, typename PowersTable, typename Name>
class FunctionApproximating : public IFunction
{
public:
@ -190,7 +256,13 @@ namespace DB
static IFunction * create(const Context & context) { return new FunctionApproximating; }
private:
template <typename T0>
template<typename T>
bool checkType(const IDataType * type) const
{
return typeid_cast<const T *>(type) != nullptr;
}
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))
@ -204,13 +276,13 @@ namespace DB
const PODArray<T0> & a = col->getData();
size_t size = a.size();
for (size_t i = 0; i < size; ++i)
vec_res[i] = FunctionApproximatingImpl<T0, Op>::apply(a[i], scale);
vec_res[i] = FunctionApproximatingImpl<T0, Op, PowersTable>::apply(a[i], scale);
return true;
}
else if (ColumnConst<T0> * col = typeid_cast<ColumnConst<T0> *>(&*block.getByPosition(arguments[0]).column))
{
T0 res = FunctionApproximatingImpl<T0, Op>::apply(col->getData(), scale);
T0 res = FunctionApproximatingImpl<T0, Op, PowersTable>::apply(col->getData(), scale);
ColumnConst<T0> * col_res = new ColumnConst<T0>(col->size(), res);
block.getByPosition(result).column = col_res;
@ -221,6 +293,48 @@ namespace DB
return false;
}
template<typename T>
bool getScaleForType(const ColumnPtr & column, UInt8 & scale)
{
using ColumnType = ColumnConst<T>;
const ColumnType * scale_col = typeid_cast<const ColumnType *>(&*column);
if (scale_col == nullptr)
return false;
T val = scale_col->getData();
if (std::is_signed<T>::value && (val < 0))
val = 0;
else if (val >= static_cast<T>(PowersTable::value.size()))
val = static_cast<T>(PowersTable::value.size()) - 1;
scale = static_cast<UInt8>(val);
return true;
}
UInt8 getScale(const ColumnPtr & column)
{
UInt8 scale = 0;
if (!( getScaleForType<UInt8>(column, scale)
|| getScaleForType<UInt16>(column, scale)
|| getScaleForType<UInt16>(column, scale)
|| getScaleForType<UInt32>(column, scale)
|| getScaleForType<UInt64>(column, scale)
|| getScaleForType<Int8>(column, scale)
|| getScaleForType<Int16>(column, scale)
|| getScaleForType<Int32>(column, scale)
|| getScaleForType<Int64>(column, scale)))
{
throw Exception("Illegal column " + column->getName()
+ " of second ('scale') argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
return scale;
}
public:
/// Получить имя функции.
String getName() const override
@ -236,8 +350,21 @@ namespace DB
+ 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);
if (arguments.size() == 2)
{
const IDataType * type = &*arguments[1];
if (!( checkType<DataTypeUInt8>(type)
|| checkType<DataTypeUInt16>(type)
|| checkType<DataTypeUInt32>(type)
|| checkType<DataTypeUInt64>(type)
|| checkType<DataTypeInt8>(type)
|| checkType<DataTypeInt16>(type)
|| checkType<DataTypeInt32>(type)
|| checkType<DataTypeInt64>(type)))
{
throw Exception("Illegal type in second argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
const IDataType * type = &*arguments[0];
if (!type->behavesAsNumber())
@ -252,19 +379,9 @@ namespace DB
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
{
Int8 scale = 0;
UInt8 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();
}
scale = getScale(block.getByPosition(arguments[1]).column);
if (!( executeType<UInt8>(block, arguments, scale, result)
|| executeType<UInt16>(block, arguments, scale, result)
@ -292,7 +409,7 @@ namespace DB
typedef FunctionUnaryArithmetic<RoundToExp2Impl, NameRoundToExp2> FunctionRoundToExp2;
typedef FunctionUnaryArithmetic<RoundDurationImpl, NameRoundDuration> FunctionRoundDuration;
typedef FunctionUnaryArithmetic<RoundAgeImpl, NameRoundAge> FunctionRoundAge;
typedef FunctionApproximating<RoundImpl, NameRound> FunctionRound;
typedef FunctionApproximating<CeilImpl, NameCeil> FunctionCeil;
typedef FunctionApproximating<FloorImpl, NameFloor> FunctionFloor;
typedef FunctionApproximating<RoundImpl, powers_of_10, NameRound> FunctionRound;
typedef FunctionApproximating<CeilImpl, powers_of_10, NameCeil> FunctionCeil;
typedef FunctionApproximating<FloorImpl, powers_of_10, NameFloor> FunctionFloor;
}