ClickHouse/dbms/src/Functions/array/range.cpp

178 lines
7.4 KiB
C++
Raw Normal View History

2018-09-09 20:57:54 +00:00
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
2019-10-29 03:03:47 +00:00
#include <DataTypes/DataTypesNumber.h>
2018-09-09 20:57:54 +00:00
#include <DataTypes/DataTypeArray.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnVector.h>
2018-09-09 21:15:40 +00:00
#include <numeric>
2018-09-09 20:57:54 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
2019-10-29 03:03:47 +00:00
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
2018-09-09 20:57:54 +00:00
}
class FunctionRange : public IFunction
{
public:
static constexpr auto name = "range";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionRange>(); }
private:
String getName() const override { return name; }
2019-10-29 03:03:47 +00:00
size_t getNumberOfArguments() const override { return 0; }
bool isVariadic() const override { return true; }
2018-09-09 20:57:54 +00:00
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() > 3 || arguments.empty())
2019-10-29 03:03:47 +00:00
{
2019-10-30 20:39:29 +00:00
throw Exception{"Function " + getName() + " needs 1..3 arguments; passed "
2019-10-29 03:03:47 +00:00
+ std::to_string(arguments.size()) + ".",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
}
2018-09-09 20:57:54 +00:00
2019-10-29 03:03:47 +00:00
for (const auto & arg : arguments)
{
if (!isUnsignedInteger(arg))
throw Exception{"Illegal type " + arg->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
return std::make_shared<DataTypeArray>(arguments.size() == 3 ? arguments[1] : arguments.back());
2018-09-09 20:57:54 +00:00
}
template <typename Start, typename End, typename Step>
2019-10-31 04:59:50 +00:00
bool executeStartEndStep(Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t input_rows_count, const size_t result)
2018-09-09 20:57:54 +00:00
{
static constexpr size_t max_elements = 100'000'000;
auto start_column = checkAndGetColumn<ColumnVector<Start>>(start_col);
auto end_column = checkAndGetColumn<ColumnVector<End>>(end_col);
auto step_column = checkAndGetColumn<ColumnVector<Step>>(step_col);
if (!start_column || !end_column || !step_column)
2018-09-09 20:57:54 +00:00
{
2019-10-29 03:03:47 +00:00
return false;
}
2018-09-09 20:57:54 +00:00
const auto & start_data = start_column->getData();
const auto & end_start = end_column->getData();
const auto & step_data = step_column->getData();
2019-10-29 03:03:47 +00:00
size_t total_values = 0;
size_t pre_values = 0;
2019-10-29 03:03:47 +00:00
2019-10-31 04:59:50 +00:00
for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx)
2019-10-29 03:03:47 +00:00
{
2019-10-30 02:06:30 +00:00
if (start_data[row_idx] < end_start[row_idx] && step_data[row_idx] == 0)
2019-10-30 01:36:50 +00:00
throw Exception{"A call to function " + getName() + " overflows, the 3rd argument step can't be zero",
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
pre_values += start_data[row_idx] >= end_start[row_idx] ? 0
: (end_start[row_idx] -start_data[row_idx] - 1) / (step_data[row_idx]) + 1;
if (pre_values < total_values)
throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing",
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
total_values = pre_values;
2018-09-09 20:57:54 +00:00
if (total_values > max_elements)
throw Exception{"A call to function " + getName() + " would produce " + std::to_string(total_values) +
" array elements, which is greater than the allowed maximum of " + std::to_string(max_elements),
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
2019-10-29 03:03:47 +00:00
}
2018-09-09 20:57:54 +00:00
auto data_col = ColumnVector<End>::create(total_values);
2019-10-29 03:03:47 +00:00
auto offsets_col = ColumnArray::ColumnOffsets::create(end_column->size());
2018-09-09 20:57:54 +00:00
2019-10-29 03:03:47 +00:00
auto & out_data = data_col->getData();
auto & out_offsets = offsets_col->getData();
2018-09-09 20:57:54 +00:00
2019-10-29 03:03:47 +00:00
IColumn::Offset offset{};
2019-10-31 04:59:50 +00:00
for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx)
2019-10-29 03:03:47 +00:00
{
for (size_t st = start_data[row_idx], ed = end_start[row_idx]; st < ed; st += step_data[row_idx])
2019-10-29 03:03:47 +00:00
out_data[offset++] = st;
2018-09-09 20:57:54 +00:00
2019-10-29 03:03:47 +00:00
out_offsets[row_idx] = offset;
2018-09-09 20:57:54 +00:00
}
2019-10-29 03:03:47 +00:00
block.getByPosition(result).column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
return true;
}
template <typename Start, typename End>
2019-10-31 04:59:50 +00:00
bool executeStartEnd(Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t input_rows_count, const size_t result)
{
2019-10-31 04:59:50 +00:00
return executeStartEndStep<Start, End, UInt8>(block, start_col, end_col, step_col, input_rows_count, result)
|| executeStartEndStep<Start, End, UInt16>(block, start_col, end_col, step_col, input_rows_count, result)
|| executeStartEndStep<Start, End, UInt32>(block, start_col, end_col, step_col, input_rows_count, result)
|| executeStartEndStep<Start, End, UInt64>(block, start_col, end_col, step_col, input_rows_count, result);
}
template <typename Start>
2019-10-31 04:59:50 +00:00
bool executeStart(Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t input_rows_count, const size_t result)
2018-09-09 20:57:54 +00:00
{
2019-10-31 04:59:50 +00:00
return executeStartEnd<Start, UInt8>(block, start_col, end_col, step_col, input_rows_count, result)
|| executeStartEnd<Start, UInt16>(block, start_col, end_col, step_col, input_rows_count, result)
|| executeStartEnd<Start, UInt32>(block, start_col, end_col, step_col, input_rows_count, result)
|| executeStartEnd<Start, UInt64>(block, start_col, end_col, step_col, input_rows_count, result);
2019-10-29 03:03:47 +00:00
}
2018-09-09 20:57:54 +00:00
2019-10-31 04:59:50 +00:00
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
Columns columns_holder(3);
ColumnRawPtrs columns(3);
2019-10-29 05:01:45 +00:00
size_t idx = 0;
// for start column, default to 0
2019-10-29 05:01:45 +00:00
if (arguments.size() == 1)
{
2019-10-31 04:59:50 +00:00
columns_holder[idx] = DataTypeUInt8().createColumnConst(input_rows_count, 0)->convertToFullColumnIfConst();
2019-10-29 05:01:45 +00:00
columns[idx] = columns_holder[idx].get();
2019-10-31 04:59:50 +00:00
++idx;
2019-10-29 05:01:45 +00:00
}
for (size_t i = 0; i < arguments.size(); ++i)
{
2019-10-29 11:20:35 +00:00
columns_holder[idx] = block.getByPosition(arguments[i]).column->convertToFullColumnIfConst();
2019-10-29 05:01:45 +00:00
columns[idx] = columns_holder[idx].get();
2019-10-31 04:59:50 +00:00
++idx;
2019-10-29 05:01:45 +00:00
}
// for step column, defaults to 1
if (arguments.size() <= 2)
{
2019-10-31 04:59:50 +00:00
columns_holder[idx] = DataTypeUInt8().createColumnConst(input_rows_count, 1)->convertToFullColumnIfConst();
columns[idx] = columns_holder[idx].get();
}
2019-10-31 04:59:50 +00:00
if (!executeStart<UInt8 >(block, columns[0], columns[1], columns[2], input_rows_count, result) &&
!executeStart<UInt16>(block, columns[0], columns[1], columns[2], input_rows_count, result) &&
!executeStart<UInt32>(block, columns[0], columns[1], columns[2], input_rows_count, result) &&
!executeStart<UInt64>(block, columns[0], columns[1], columns[2], input_rows_count, result))
2018-09-09 20:57:54 +00:00
{
throw Exception{"Illegal columns " + columns[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
2018-09-09 20:57:54 +00:00
}
}
2019-10-29 03:03:47 +00:00
2018-09-09 20:57:54 +00:00
};
void registerFunctionRange(FunctionFactory & factory)
{
factory.registerFunction<FunctionRange>();
}
}