mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
args to supertype
This commit is contained in:
parent
71ec49cc2f
commit
76b75dc0e8
@ -3,8 +3,10 @@
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <numeric>
|
||||
|
||||
|
||||
@ -25,9 +27,11 @@ class FunctionRange : public IFunction
|
||||
public:
|
||||
static constexpr auto name = "range";
|
||||
static constexpr size_t max_elements = 100'000'000;
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionRange>(); }
|
||||
static FunctionPtr create(const Context & context_) { return std::make_shared<FunctionRange>(context_); }
|
||||
FunctionRange(const Context & context_) : context(context_) {}
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
String getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
@ -49,7 +53,9 @@ private:
|
||||
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());
|
||||
|
||||
DataTypePtr common_type = getLeastSupertype(arguments);
|
||||
return std::make_shared<DataTypeArray>(common_type);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
@ -97,12 +103,175 @@ private:
|
||||
return false;
|
||||
}
|
||||
|
||||
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 input_rows_count, const size_t result)
|
||||
template <typename T>
|
||||
bool executeConstStartStep(Block & block, const IColumn * end_arg, const T start, const T step, const size_t input_rows_count, const size_t result)
|
||||
{
|
||||
auto start_column = checkAndGetColumn<ColumnVector<Start>>(start_col);
|
||||
auto end_column = checkAndGetColumn<ColumnVector<End>>(end_col);
|
||||
auto step_column = checkAndGetColumn<ColumnVector<Step>>(step_col);
|
||||
auto end_column = checkAndGetColumn<ColumnVector<T>>(end_arg);
|
||||
if (!end_column)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
const auto & end_data = end_column->getData();
|
||||
|
||||
size_t total_values = 0;
|
||||
size_t pre_values = 0;
|
||||
|
||||
for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx)
|
||||
{
|
||||
if (start < end_data[row_idx] && step == 0)
|
||||
throw Exception{"A call to function " + getName() + " overflows, the 3rd argument step can't be zero",
|
||||
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
|
||||
|
||||
pre_values += start >= end_data[row_idx] ? 0
|
||||
: (end_data[row_idx] - start - 1) / step + 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<T>::create(total_values);
|
||||
auto offsets_col = ColumnArray::ColumnOffsets::create(end_column->size());
|
||||
|
||||
auto & out_data = data_col->getData();
|
||||
auto & out_offsets = offsets_col->getData();
|
||||
|
||||
IColumn::Offset offset{};
|
||||
for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx)
|
||||
{
|
||||
for (size_t st = start, ed = end_data[row_idx]; st < ed; st += step)
|
||||
out_data[offset++] = st;
|
||||
|
||||
out_offsets[row_idx] = offset;
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool executeConstStep(Block & block, const IColumn * start_arg, const IColumn * end_arg, const T step, const size_t input_rows_count, const size_t result)
|
||||
{
|
||||
auto start_column = checkAndGetColumn<ColumnVector<T>>(start_arg);
|
||||
auto end_column = checkAndGetColumn<ColumnVector<T>>(end_arg);
|
||||
if (!end_column || !start_column)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
const auto & start_data = start_column->getData();
|
||||
const auto & end_data = end_column->getData();
|
||||
|
||||
size_t total_values = 0;
|
||||
size_t pre_values = 0;
|
||||
|
||||
for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx)
|
||||
{
|
||||
if (start_data[row_idx] < end_data[row_idx] && step == 0)
|
||||
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_data[row_idx] ? 0
|
||||
: (end_data[row_idx] - start_data[row_idx] - 1) / step + 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<T>::create(total_values);
|
||||
auto offsets_col = ColumnArray::ColumnOffsets::create(end_column->size());
|
||||
|
||||
auto & out_data = data_col->getData();
|
||||
auto & out_offsets = offsets_col->getData();
|
||||
|
||||
IColumn::Offset offset{};
|
||||
for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx)
|
||||
{
|
||||
for (size_t st = start_data[row_idx], ed = end_data[row_idx]; st < ed; st += step)
|
||||
out_data[offset++] = st;
|
||||
|
||||
out_offsets[row_idx] = offset;
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool executeConstStart(Block & block, const IColumn * end_arg, const IColumn * step_arg, const T start, const size_t input_rows_count, const size_t result)
|
||||
{
|
||||
auto end_column = checkAndGetColumn<ColumnVector<T>>(end_arg);
|
||||
auto step_column = checkAndGetColumn<ColumnVector<T>>(step_arg);
|
||||
if (!end_column || !step_column)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
const auto & end_data = 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; row_idx < input_rows_count; ++row_idx)
|
||||
{
|
||||
if (start < end_data[row_idx] && step_data[row_idx] == 0)
|
||||
throw Exception{"A call to function " + getName() + " overflows, the 3rd argument step can't be zero",
|
||||
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
|
||||
|
||||
pre_values += start >= end_data[row_idx] ? 0
|
||||
: (end_data[row_idx] - start - 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<T>::create(total_values);
|
||||
auto offsets_col = ColumnArray::ColumnOffsets::create(end_column->size());
|
||||
|
||||
auto & out_data = data_col->getData();
|
||||
auto & out_offsets = offsets_col->getData();
|
||||
|
||||
IColumn::Offset offset{};
|
||||
for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx)
|
||||
{
|
||||
for (size_t st = start, ed = end_data[row_idx]; st < ed; st += step_data[row_idx])
|
||||
out_data[offset++] = st;
|
||||
|
||||
out_offsets[row_idx] = offset;
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool executeGeneric(Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t input_rows_count, const size_t result)
|
||||
{
|
||||
auto start_column = checkAndGetColumn<ColumnVector<T>>(start_col);
|
||||
auto end_column = checkAndGetColumn<ColumnVector<T>>(end_col);
|
||||
auto step_column = checkAndGetColumn<ColumnVector<T>>(step_col);
|
||||
|
||||
if (!start_column || !end_column || !step_column)
|
||||
{
|
||||
@ -136,7 +305,7 @@ private:
|
||||
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
|
||||
}
|
||||
|
||||
auto data_col = ColumnVector<End>::create(total_values);
|
||||
auto data_col = ColumnVector<T>::create(total_values);
|
||||
auto offsets_col = ColumnArray::ColumnOffsets::create(end_column->size());
|
||||
|
||||
auto & out_data = data_col->getData();
|
||||
@ -155,29 +324,8 @@ private:
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename Start, typename End>
|
||||
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)
|
||||
{
|
||||
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>
|
||||
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)
|
||||
{
|
||||
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);
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
Columns columns_holder(3);
|
||||
ColumnRawPtrs columns(3);
|
||||
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
const auto col = block.getByPosition(arguments[0]).column.get();
|
||||
@ -191,23 +339,68 @@ private:
|
||||
return;
|
||||
}
|
||||
|
||||
Columns columns_holder(3);
|
||||
ColumnRawPtrs columns(3);
|
||||
|
||||
const auto return_type = checkAndGetDataType<DataTypeArray>(block.getByPosition(result).type.get())->getNestedType();
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
columns_holder[i] = block.getByPosition(arguments[i]).column->convertToFullColumnIfConst();
|
||||
if (i == 1)
|
||||
columns_holder[i] = castColumn(block.getByPosition(arguments[i]), return_type, context)->convertToFullColumnIfConst();
|
||||
else
|
||||
columns_holder[i] = castColumn(block.getByPosition(arguments[i]), return_type, context);
|
||||
|
||||
columns[i] = columns_holder[i].get();
|
||||
}
|
||||
|
||||
// for step column, defaults to 1
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
columns_holder[2] = DataTypeUInt8().createColumnConst(input_rows_count, 1)->convertToFullColumnIfConst();
|
||||
columns_holder[2] = return_type->createColumnConst(input_rows_count, 1);
|
||||
columns[2] = columns_holder[2].get();
|
||||
}
|
||||
|
||||
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))
|
||||
bool is_start_const = isColumnConst(*columns[0]);
|
||||
bool is_step_const = isColumnConst(*columns[2]);
|
||||
bool ok;
|
||||
if (is_start_const && is_step_const)
|
||||
{
|
||||
UInt64 start = assert_cast<const ColumnConst &>(*columns[0]).getUInt(0);
|
||||
UInt64 step = assert_cast<const ColumnConst &>(*columns[2]).getUInt(0);
|
||||
|
||||
ok = executeConstStartStep<UInt8>(block, columns[1], start, step, input_rows_count, result) ||
|
||||
executeConstStartStep<UInt16>(block, columns[1], start, step, input_rows_count, result) ||
|
||||
executeConstStartStep<UInt32>(block, columns[1], start, step, input_rows_count, result) ||
|
||||
executeConstStartStep<UInt64>(block, columns[1], start, step, input_rows_count, result);
|
||||
}
|
||||
else if (is_start_const && !is_step_const)
|
||||
{
|
||||
UInt64 start = assert_cast<const ColumnConst &>(*columns[0]).getUInt(0);
|
||||
|
||||
ok = executeConstStart<UInt8>(block, columns[1], columns[2], start, input_rows_count, result) ||
|
||||
executeConstStart<UInt16>(block, columns[1], columns[2], start, input_rows_count, result) ||
|
||||
executeConstStart<UInt32>(block, columns[1], columns[2], start, input_rows_count, result) ||
|
||||
executeConstStart<UInt64>(block, columns[1], columns[2], start, input_rows_count, result);
|
||||
}
|
||||
else if (!is_start_const && is_step_const)
|
||||
{
|
||||
UInt64 step = assert_cast<const ColumnConst &>(*columns[2]).getUInt(0);
|
||||
|
||||
ok = executeConstStep<UInt8>(block, columns[0], columns[1], step, input_rows_count, result) ||
|
||||
executeConstStep<UInt16>(block, columns[0], columns[1], step, input_rows_count, result) ||
|
||||
executeConstStep<UInt32>(block, columns[0], columns[1], step, input_rows_count, result) ||
|
||||
executeConstStep<UInt64>(block, columns[0], columns[1], step, input_rows_count, result);
|
||||
}
|
||||
else
|
||||
{
|
||||
ok = executeGeneric<UInt8>(block, columns[0], columns[1], columns[2], input_rows_count, result) ||
|
||||
executeGeneric<UInt16>(block, columns[0], columns[1], columns[2], input_rows_count, result) ||
|
||||
executeGeneric<UInt32>(block, columns[0], columns[1], columns[2], input_rows_count, result) ||
|
||||
executeGeneric<UInt64>(block, columns[0], columns[1], columns[2], input_rows_count, result);
|
||||
}
|
||||
|
||||
if (!ok)
|
||||
{
|
||||
throw Exception{"Illegal columns " + columns[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
|
@ -35,6 +35,7 @@ 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(end), 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, defaults to 1.
|
||||
|
Loading…
Reference in New Issue
Block a user