mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
dbms: implement range(n) function, add tests. [#METR-13613]
This commit is contained in:
parent
682bfb46a5
commit
ea9472312e
@ -17,6 +17,8 @@
|
||||
#include <DB/Functions/NumberTraits.h>
|
||||
#include <DB/Functions/FunctionsConditional.h>
|
||||
|
||||
#include <statdaemons/ext/range.hpp>
|
||||
|
||||
#include <unordered_map>
|
||||
|
||||
|
||||
@ -1226,6 +1228,7 @@ struct FunctionEmptyArray : public IFunction
|
||||
static const String name;
|
||||
static IFunction * create(const Context & context) { return new FunctionEmptyArray; }
|
||||
|
||||
private:
|
||||
String getName() const
|
||||
{
|
||||
return name;
|
||||
@ -1255,6 +1258,118 @@ struct FunctionEmptyArray : public IFunction
|
||||
template <typename DataType>
|
||||
const String FunctionEmptyArray<DataType>::name = FunctionEmptyArray::base_name + DataTypeToName<DataType>::get();
|
||||
|
||||
class FunctionRange : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "range";
|
||||
static IFunction * create(const Context &) { return new FunctionRange; }
|
||||
|
||||
private:
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception{
|
||||
"Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
|
||||
};
|
||||
|
||||
const auto arg = arguments.front().get();
|
||||
|
||||
if (!typeid_cast<const DataTypeUInt8 *>(arg) &&
|
||||
!typeid_cast<const DataTypeUInt16 *>(arg) &&
|
||||
!typeid_cast<const DataTypeUInt32 *>(arg) &
|
||||
!typeid_cast<const DataTypeUInt64 *>(arg))
|
||||
{
|
||||
throw Exception{
|
||||
"Illegal type " + arg->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||
};
|
||||
}
|
||||
|
||||
return new DataTypeArray{arg->clone()};
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool execute(Block & block, const IColumn * const arg, const size_t result) override
|
||||
{
|
||||
if (const auto in = typeid_cast<const ColumnVector<T> *>(arg))
|
||||
{
|
||||
const auto & in_data = in->getData();
|
||||
const auto total_values = std::accumulate(std::begin(in_data), std::end(in_data), std::size_t{},
|
||||
std::plus<std::size_t>{});
|
||||
|
||||
const auto data_col = new ColumnVector<T>{total_values};
|
||||
const auto out = new ColumnArray{
|
||||
data_col,
|
||||
new ColumnArray::ColumnOffsets_t{in->size()}
|
||||
};
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
auto & out_data = data_col->getData();
|
||||
auto & out_offsets = out->getOffsets();
|
||||
|
||||
IColumn::Offset_t offset{};
|
||||
for (const auto i : ext::range(0, in->size()))
|
||||
{
|
||||
std::copy(ext::make_range_iterator(T{}), ext::make_range_iterator(in_data[i]), &out_data[offset]);
|
||||
offset += in_data[i];
|
||||
out_offsets[i] = offset;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (const auto in = typeid_cast<const ColumnConst<T> *>(arg))
|
||||
{
|
||||
const auto & in_data = in->getData();
|
||||
const std::size_t total_values = in->size() * in_data;
|
||||
|
||||
const auto data_col = new ColumnVector<T>{total_values};
|
||||
const auto out = new ColumnArray{
|
||||
data_col,
|
||||
new ColumnArray::ColumnOffsets_t{in->size()}
|
||||
};
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
auto & out_data = data_col->getData();
|
||||
auto & out_offsets = out->getOffsets();
|
||||
|
||||
IColumn::Offset_t offset{};
|
||||
for (const auto i : ext::range(0, in->size()))
|
||||
{
|
||||
std::copy(ext::make_range_iterator(T{}), ext::make_range_iterator(in_data), &out_data[offset]);
|
||||
offset += in_data;
|
||||
out_offsets[i] = offset;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
|
||||
{
|
||||
const auto col = block.getByPosition(arguments[0]).column.get();
|
||||
|
||||
if (!execute<UInt8>(block, col, result) &&
|
||||
!execute<UInt16>(block, col, result) &&
|
||||
!execute<UInt32>(block, col, result) &&
|
||||
!execute<UInt64>(block, col, result))
|
||||
{
|
||||
throw Exception{
|
||||
"Illegal column " + col->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN
|
||||
};
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
struct NameHas { static constexpr auto name = "has"; };
|
||||
struct NameIndexOf { static constexpr auto name = "indexOf"; };
|
||||
|
@ -230,19 +230,17 @@ private:
|
||||
}
|
||||
|
||||
template <typename LeftType, typename RightType>
|
||||
bool executeRight(Block & block, const ColumnNumbers & arguments, const size_t result,
|
||||
const ColumnConst<LeftType> * const left_arg)
|
||||
bool executeRight(Block & block, const size_t result, const ColumnConst<LeftType> * const left_arg,
|
||||
const IColumn * const right_arg)
|
||||
{
|
||||
const auto arg = block.getByPosition(arguments[1]).column.get();
|
||||
|
||||
if (const auto right_arg = typeid_cast<const ColumnVector<RightType> *>(arg))
|
||||
if (const auto right_arg_typed = typeid_cast<const ColumnVector<RightType> *>(right_arg))
|
||||
{
|
||||
const auto dst = new ColumnVector<Float64>;
|
||||
block.getByPosition(result).column = dst;
|
||||
|
||||
LeftType left_src_data[Impl::rows_per_iteration];
|
||||
std::fill(std::begin(left_src_data), std::end(left_src_data), left_arg->getData());
|
||||
const auto & right_src_data = right_arg->getData();
|
||||
const auto & right_src_data = right_arg_typed->getData();
|
||||
const auto src_size = right_src_data.size();
|
||||
auto & dst_data = dst->getData();
|
||||
dst_data.resize(src_size);
|
||||
@ -267,10 +265,10 @@ private:
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (const auto right_arg = typeid_cast<const ColumnConst<RightType> *>(arg))
|
||||
else if (const auto right_arg_typed = typeid_cast<const ColumnConst<RightType> *>(right_arg))
|
||||
{
|
||||
const LeftType left_src[Impl::rows_per_iteration] { left_arg->getData() };
|
||||
const RightType right_src[Impl::rows_per_iteration] { right_arg->getData() };
|
||||
const RightType right_src[Impl::rows_per_iteration] { right_arg_typed->getData() };
|
||||
Float64 dst[Impl::rows_per_iteration];
|
||||
|
||||
Impl::execute(left_src, right_src, dst);
|
||||
@ -284,18 +282,16 @@ private:
|
||||
}
|
||||
|
||||
template <typename LeftType, typename RightType>
|
||||
bool executeRight(Block & block, const ColumnNumbers & arguments, const size_t result,
|
||||
const ColumnVector<LeftType> * const left_arg)
|
||||
bool executeRight(Block & block, const size_t result, const ColumnVector<LeftType> * const left_arg,
|
||||
const IColumn * const right_arg)
|
||||
{
|
||||
const auto arg = block.getByPosition(arguments[1]).column.get();
|
||||
|
||||
if (const auto right_arg = typeid_cast<const ColumnVector<RightType> *>(arg))
|
||||
if (const auto right_arg_typed = typeid_cast<const ColumnVector<RightType> *>(right_arg))
|
||||
{
|
||||
const auto dst = new ColumnVector<Float64>;
|
||||
block.getByPosition(result).column = dst;
|
||||
|
||||
const auto & left_src_data = left_arg->getData();
|
||||
const auto & right_src_data = right_arg->getData();
|
||||
const auto & right_src_data = right_arg_typed->getData();
|
||||
const auto src_size = left_src_data.size();
|
||||
auto & dst_data = dst->getData();
|
||||
dst_data.resize(src_size);
|
||||
@ -323,14 +319,14 @@ private:
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (const auto right_arg = typeid_cast<const ColumnConst<RightType> *>(arg))
|
||||
else if (const auto right_arg_typed = typeid_cast<const ColumnConst<RightType> *>(right_arg))
|
||||
{
|
||||
const auto dst = new ColumnVector<Float64>;
|
||||
block.getByPosition(result).column = dst;
|
||||
|
||||
const auto & left_src_data = left_arg->getData();
|
||||
RightType right_src_data[Impl::rows_per_iteration];
|
||||
std::fill(std::begin(right_src_data), std::end(right_src_data), right_arg->getData());
|
||||
std::fill(std::begin(right_src_data), std::end(right_src_data), right_arg_typed->getData());
|
||||
const auto src_size = left_src_data.size();
|
||||
auto & dst_data = dst->getData();
|
||||
dst_data.resize(src_size);
|
||||
@ -360,21 +356,23 @@ private:
|
||||
}
|
||||
|
||||
template <typename LeftType, template <typename> class LeftColumnType>
|
||||
bool executeLeftImpl(Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||
bool executeLeftImpl(Block & block, const ColumnNumbers & arguments, const size_t result,
|
||||
const IColumn * const left_arg)
|
||||
{
|
||||
if (const auto arg = typeid_cast<const LeftColumnType<LeftType> *>(
|
||||
block.getByPosition(arguments[0]).column.get()))
|
||||
if (const auto left_arg_typed = typeid_cast<const LeftColumnType<LeftType> *>(left_arg))
|
||||
{
|
||||
if (executeRight<LeftType, UInt8>(block, arguments, result, arg) ||
|
||||
executeRight<LeftType, UInt16>(block, arguments, result, arg) ||
|
||||
executeRight<LeftType, UInt32>(block, arguments, result, arg) ||
|
||||
executeRight<LeftType, UInt64>(block, arguments, result, arg) ||
|
||||
executeRight<LeftType, Int8>(block, arguments, result, arg) ||
|
||||
executeRight<LeftType, Int16>(block, arguments, result, arg) ||
|
||||
executeRight<LeftType, Int32>(block, arguments, result, arg) ||
|
||||
executeRight<LeftType, Int64>(block, arguments, result, arg) ||
|
||||
executeRight<LeftType, Float32>(block, arguments, result, arg) ||
|
||||
executeRight<LeftType, Float64>(block, arguments, result, arg))
|
||||
const auto right_arg = block.getByPosition(arguments[1]).column.get();
|
||||
|
||||
if (executeRight<LeftType, UInt8>(block, result, left_arg_typed, right_arg) ||
|
||||
executeRight<LeftType, UInt16>(block, result, left_arg_typed, right_arg) ||
|
||||
executeRight<LeftType, UInt32>(block, result, left_arg_typed, right_arg) ||
|
||||
executeRight<LeftType, UInt64>(block, result, left_arg_typed, right_arg) ||
|
||||
executeRight<LeftType, Int8>(block, result, left_arg_typed, right_arg) ||
|
||||
executeRight<LeftType, Int16>(block, result, left_arg_typed, right_arg) ||
|
||||
executeRight<LeftType, Int32>(block, result, left_arg_typed, right_arg) ||
|
||||
executeRight<LeftType, Int64>(block, result, left_arg_typed, right_arg) ||
|
||||
executeRight<LeftType, Float32>(block, result, left_arg_typed, right_arg) ||
|
||||
executeRight<LeftType, Float64>(block, result, left_arg_typed, right_arg))
|
||||
{
|
||||
return true;
|
||||
}
|
||||
@ -392,10 +390,11 @@ private:
|
||||
}
|
||||
|
||||
template <typename LeftType>
|
||||
bool executeLeft(Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||
bool executeLeft(Block & block, const ColumnNumbers & arguments, const size_t result,
|
||||
const IColumn * const left_arg)
|
||||
{
|
||||
if (executeLeftImpl<LeftType, ColumnVector>(block, arguments, result) ||
|
||||
executeLeftImpl<LeftType, ColumnConst>(block, arguments, result))
|
||||
if (executeLeftImpl<LeftType, ColumnVector>(block, arguments, result, left_arg) ||
|
||||
executeLeftImpl<LeftType, ColumnConst>(block, arguments, result, left_arg))
|
||||
return true;
|
||||
|
||||
return false;
|
||||
@ -403,19 +402,21 @@ private:
|
||||
|
||||
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
|
||||
{
|
||||
if (!executeLeft<UInt8>(block, arguments, result) &&
|
||||
!executeLeft<UInt16>(block, arguments, result) &&
|
||||
!executeLeft<UInt32>(block, arguments, result) &&
|
||||
!executeLeft<UInt64>(block, arguments, result) &&
|
||||
!executeLeft<Int8>(block, arguments, result) &&
|
||||
!executeLeft<Int16>(block, arguments, result) &&
|
||||
!executeLeft<Int32>(block, arguments, result) &&
|
||||
!executeLeft<Int64>(block, arguments, result) &&
|
||||
!executeLeft<Float32>(block, arguments, result) &&
|
||||
!executeLeft<Float64>(block, arguments, result))
|
||||
const auto left_arg = block.getByPosition(arguments[0]).column.get();
|
||||
|
||||
if (!executeLeft<UInt8>(block, arguments, result, left_arg) &&
|
||||
!executeLeft<UInt16>(block, arguments, result, left_arg) &&
|
||||
!executeLeft<UInt32>(block, arguments, result, left_arg) &&
|
||||
!executeLeft<UInt64>(block, arguments, result, left_arg) &&
|
||||
!executeLeft<Int8>(block, arguments, result, left_arg) &&
|
||||
!executeLeft<Int16>(block, arguments, result, left_arg) &&
|
||||
!executeLeft<Int32>(block, arguments, result, left_arg) &&
|
||||
!executeLeft<Int64>(block, arguments, result, left_arg) &&
|
||||
!executeLeft<Float32>(block, arguments, result, left_arg) &&
|
||||
!executeLeft<Float64>(block, arguments, result, left_arg))
|
||||
{
|
||||
throw Exception{
|
||||
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
|
||||
"Illegal column " + left_arg->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN
|
||||
};
|
||||
}
|
||||
|
@ -26,6 +26,7 @@ void registerFunctionsArray(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionEmptyArrayDate>();
|
||||
factory.registerFunction<FunctionEmptyArrayDateTime>();
|
||||
factory.registerFunction<FunctionEmptyArrayString>();
|
||||
factory.registerFunction<FunctionRange>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -0,0 +1,78 @@
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
96
dbms/tests/queries/0_stateless/00087_math_functions.sql
Normal file
96
dbms/tests/queries/0_stateless/00087_math_functions.sql
Normal file
@ -0,0 +1,96 @@
|
||||
select abs(0) = 0;
|
||||
select abs(1) = 1;
|
||||
select abs(1) = 1;
|
||||
select abs(0.0) = 0;
|
||||
select abs(1.0) = 1.0;
|
||||
select abs(-1.0) = 1.0;
|
||||
select abs(-128) = 128;
|
||||
select abs(127) = 127;
|
||||
select sum(abs(number - 10 as x) = (x < 0 ? -x : x)) / count() array join range(1000000) as number;
|
||||
|
||||
select square(0) = 0;
|
||||
select square(1) = 1;
|
||||
select square(2) = 4;
|
||||
select sum(square(x) = x * x) / count() array join range(1000000) as x;
|
||||
|
||||
select sqrt(0) = 0;
|
||||
select sqrt(1) = 1;
|
||||
select sqrt(4) = 2;
|
||||
select sum(sqrt(square(x)) = x) / count() array join range(1000000) as x;
|
||||
|
||||
select cbrt(0) = 0;
|
||||
select cbrt(1) = 1;
|
||||
select cbrt(8) = 2;
|
||||
select sum(abs(cbrt(x * square(x)) - x) < 1.0e-9) / count() array join range(1000000) as x;
|
||||
|
||||
select pow(1, 0) = 1;
|
||||
select pow(2, 0) = 1;
|
||||
select sum(pow(x, 0) = 1) / count() array join range(1000000) as x;
|
||||
select pow(1, 1) = 1;
|
||||
select pow(2, 1) = 2;
|
||||
select sum(abs(pow(x, 1) - x) < 1.0e-9) / count() array join range(1000000) as x;
|
||||
select sum(pow(x, 2) = square(x)) / count() array join range(10000) as x;
|
||||
|
||||
select tgamma(0) = inf;
|
||||
select tgamma(1) = 1;
|
||||
select tgamma(2) = 1;
|
||||
select tgamma(3) = 2;
|
||||
select tgamma(4) = 6;
|
||||
|
||||
select sum(abs(lgamma(x + 1) - log(tgamma(x + 1))) < 1.0e-9) / count() array join range(10) as x;
|
||||
|
||||
select abs(e() - arraySum(arrayMap(x -> 1 / tgamma(x + 1), range(13)))) < 1.0e-9;
|
||||
|
||||
select log(0) = -inf;
|
||||
select log(1) = 0;
|
||||
select log(e()) = 1;
|
||||
select log(exp(1)) = 1;
|
||||
select log(exp(2)) = 2;
|
||||
select sum(abs(log(exp(x)) - x) < 1.0e-9) / count() array join range(100) as x;
|
||||
|
||||
select exp2(-1) = 1/2;
|
||||
select exp2(0) = 1;
|
||||
select exp2(1) = 2;
|
||||
select exp2(2) = 4;
|
||||
select exp2(3) = 8;
|
||||
select sum(exp2(x) = pow(2, x)) / count() array join range(1000) as x;
|
||||
|
||||
select log2(0) = -inf;
|
||||
select log2(1) = 0;
|
||||
select log2(2) = 1;
|
||||
select log2(4) = 2;
|
||||
select sum(abs(log2(exp2(x)) - x) < 1.0e-9) / count() array join range(1000) as x;
|
||||
|
||||
select sin(0) = 0;
|
||||
select sin(pi() / 4) = 1 / sqrt(2);
|
||||
select sin(pi() / 2) = 1;
|
||||
select sin(3 * pi() / 2) = -1;
|
||||
select sum(sin(pi() / 2 + 2 * pi() * x) = 1) / count() array join range(1000000) as x;
|
||||
|
||||
select cos(0) = 1;
|
||||
select abs(cos(pi() / 4) - 1 / sqrt(2)) < 1.0e-9;
|
||||
select cos(pi() / 2) < 1.0e-9;
|
||||
select sum(abs(cos(2 * pi() * x)) - 1 < 1.0e-9) / count() array join range(1000000) as x;
|
||||
|
||||
select tan(0) = 0;
|
||||
select abs(tan(pi() / 4) - 1) < 1.0e-9;
|
||||
select sum(abs(tan(pi() / 4 + 2 * pi() * x) - 1) < 1.0e-8) / count() array join range(1000000) as x;
|
||||
|
||||
select asin(0) = 0;
|
||||
select asin(1) = pi() / 2;
|
||||
select asin(-1) = -pi() / 2;
|
||||
|
||||
select acos(0) = pi() / 2;
|
||||
select acos(1) = 0;
|
||||
select acos(-1) = pi();
|
||||
|
||||
select atan(0) = 0;
|
||||
select atan(1) = pi() / 4;
|
||||
|
||||
select erf(0) = 0;
|
||||
select erf(-10) = -1;
|
||||
select erf(10) = 1;
|
||||
|
||||
select erfc(0) = 1;
|
||||
select erfc(-10) = 2;
|
||||
select erfc(28) = 0;
|
Loading…
Reference in New Issue
Block a user