improve range to support start,step args

This commit is contained in:
sundy-li 2019-10-29 14:33:03 +08:00
parent a653f50432
commit 66ffec32f7
2 changed files with 61 additions and 33 deletions

View File

@ -35,9 +35,9 @@ private:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() > 2 || arguments.empty())
if (arguments.size() > 3 || arguments.empty())
{
throw Exception{"Function " + getName() + " needs [1-2] argument; passed "
throw Exception{"Function " + getName() + " needs [1-3] argument; passed "
+ std::to_string(arguments.size()) + ".",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
}
@ -48,37 +48,47 @@ private:
throw Exception{"Illegal type " + arg->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
return std::make_shared<DataTypeArray>(arguments.back());
return std::make_shared<DataTypeArray>(arguments.size() == 3 ? arguments[1] : arguments.back());
}
template <typename T, typename E>
bool executeInternal(Block & block, const IColumn * start_col, const IColumn * end_col, const size_t result)
template <typename Start, typename End, typename Step>
bool executeStartEndStep(Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t result)
{
static constexpr size_t max_elements = 100'000'000;
auto start_column = checkAndGetColumn<ColumnVector<T>>(start_col);
auto end_column = checkAndGetColumn<ColumnVector<E>>(end_col);
if (!start_column || !end_column)
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)
{
return false;
}
const auto & in_start_data = start_column->getData();
const auto & in_end_data = end_column->getData();
const auto & start_data = start_column->getData();
const auto & end_start = end_column->getData();
const auto & step_data = step_column->getData();
size_t total_values = 0;
size_t pre_values = 0;
for (size_t row_idx = 0, rows = end_column->size(); row_idx < rows; ++row_idx)
{
total_values += in_start_data[row_idx] >= in_end_data[row_idx] ? 0 : (in_end_data[row_idx] -in_start_data[row_idx]);
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;
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};
}
auto data_col = ColumnVector<E>::create(total_values);
auto data_col = ColumnVector<End>::create(total_values);
auto offsets_col = ColumnArray::ColumnOffsets::create(end_column->size());
auto & out_data = data_col->getData();
@ -87,7 +97,7 @@ private:
IColumn::Offset offset{};
for (size_t row_idx = 0, rows = end_column->size(); row_idx < rows; ++row_idx)
{
for (size_t st = in_start_data[row_idx], ed = in_end_data[row_idx]; st < ed; ++st)
for (size_t st = start_data[row_idx], ed = end_start[row_idx]; st < ed; st += step_data[row_idx])
out_data[offset++] = st;
out_offsets[row_idx] = offset;
@ -96,23 +106,33 @@ private:
block.getByPosition(result).column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
return true;
}
template <typename T>
bool executeStartInternal(Block & block, const IColumn * start_col, const IColumn * end_col, const size_t result)
template <typename Start, typename End>
bool executeStartEnd(Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t result)
{
return executeInternal<T, UInt8>(block, start_col, end_col, result)
|| executeInternal<T, UInt16>(block, start_col, end_col, result)
|| executeInternal<T, UInt32>(block, start_col, end_col, result)
|| executeInternal<T, UInt64>(block, start_col, end_col, result);
return executeStartEndStep<Start, End, UInt8>(block, start_col, end_col, step_col,result)
|| executeStartEndStep<Start, End, UInt16>(block, start_col, end_col, step_col,result)
|| executeStartEndStep<Start, End, UInt32>(block, start_col, end_col, step_col,result)
|| executeStartEndStep<Start, End, UInt64>(block, start_col, end_col, step_col,result);
}
template <typename Start>
bool executeStart(Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t result)
{
return executeStartEnd<Start, UInt8>(block, start_col, end_col, step_col, result)
|| executeStartEnd<Start, UInt16>(block, start_col, end_col, step_col, result)
|| executeStartEnd<Start, UInt32>(block, start_col, end_col, step_col, result)
|| executeStartEnd<Start, UInt64>(block, start_col, end_col, step_col, result);
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override
{
Columns columns_holder(2);
ColumnRawPtrs columns(2);
{
Columns columns_holder(3);
ColumnRawPtrs columns(3);
size_t idx = 0;
size_t rows = block.getByPosition(arguments[0]).column.get()->size();
// for start column, default to 0
if (arguments.size() == 1)
{
columns_holder[idx] = std::move(DataTypeUInt8().createColumnConst(rows, 0)->convertToFullColumnIfConst());
@ -127,12 +147,19 @@ private:
idx ++;
}
if (!executeStartInternal<UInt8>(block, columns[0], columns[1], result) &&
!executeStartInternal<UInt16>(block, columns[0], columns[1], result) &&
!executeStartInternal<UInt32>(block, columns[0], columns[1], result) &&
!executeStartInternal<UInt64>(block, columns[0], columns[1], result))
// for step column, defaults to 1
if (arguments.size() <= 2)
{
throw Exception{"Illegal column " + columns[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
columns_holder[idx] = std::move(DataTypeUInt8().createColumnConst(rows, 1)->convertToFullColumnIfConst());
columns[idx] = columns_holder[idx].get();
}
if (!executeStart<UInt8>(block, columns[0], columns[1], columns[2], result) &&
!executeStart<UInt16>(block, columns[0], columns[1], columns[2], result) &&
!executeStart<UInt32>(block, columns[0], columns[1], columns[2], result) &&
!executeStart<UInt64>(block, columns[0], columns[1], columns[2], result))
{
throw Exception{"Illegal columns " + columns[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
}
}

View File

@ -34,10 +34,11 @@ Accepts zero arguments and returns an empty array of the appropriate type.
Accepts an empty array and returns a one-element array that is equal to the default value.
## range([start,] end)
Returns an array of numbers from start to end-1.
If the argument `start` is not specified, defaults to 0.
## range(end) or range(start, end [, step])
Returns an array of numbers from start to end-1 by step.
If the argument `start` is not specified, defaults to 0.
If the argument `step` is not specified, default to 1.
It behaviors almost like pythonic `range`. But the difference is that all the arguments type must be `UInt` numbers.
Just in case, an exception is thrown if arrays with a total length of more than 100,000,000 elements are created in a data block.
## array(x1, ...), operator \[x1, ...\]