2021-05-17 07:30:42 +00:00
|
|
|
#include <Functions/IFunction.h>
|
2018-09-09 20:57:54 +00:00
|
|
|
#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>
|
2019-11-06 10:19:08 +00:00
|
|
|
#include <DataTypes/getLeastSupertype.h>
|
2018-09-09 20:57:54 +00:00
|
|
|
#include <Columns/ColumnArray.h>
|
|
|
|
#include <Columns/ColumnVector.h>
|
2019-11-06 10:19:08 +00:00
|
|
|
#include <Interpreters/castColumn.h>
|
2021-07-14 08:31:08 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2018-09-09 21:15:40 +00:00
|
|
|
#include <numeric>
|
2022-11-27 12:41:34 +00:00
|
|
|
#include <vector>
|
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
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-03-31 00:54:40 +00:00
|
|
|
/** Generates array
|
|
|
|
* range(size): [0, size)
|
|
|
|
* range(start, end): [start, end)
|
|
|
|
* range(start, end, step): [start, end) with step increments.
|
|
|
|
*/
|
2018-09-09 20:57:54 +00:00
|
|
|
class FunctionRange : public IFunction
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
static constexpr auto name = "range";
|
2021-07-14 08:31:08 +00:00
|
|
|
|
|
|
|
const size_t max_elements;
|
|
|
|
static FunctionPtr create(ContextPtr context_) { return std::make_shared<FunctionRange>(std::move(context_)); }
|
|
|
|
explicit FunctionRange(ContextPtr context) : max_elements(context->getSettingsRef().function_range_max_elements_in_block) {}
|
2018-09-09 20:57:54 +00:00
|
|
|
|
|
|
|
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; }
|
2021-06-22 16:21:23 +00:00
|
|
|
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
|
2018-09-09 20:57:54 +00:00
|
|
|
|
|
|
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
|
|
|
{
|
2019-10-29 06:33:03 +00:00
|
|
|
if (arguments.size() > 3 || arguments.empty())
|
2019-10-29 03:03:47 +00:00
|
|
|
{
|
2021-03-31 00:54:40 +00:00
|
|
|
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
|
|
|
"Function {} needs 1..3 arguments; passed {}.",
|
|
|
|
getName(), arguments.size());
|
2019-10-29 03:03:47 +00:00
|
|
|
}
|
2018-09-09 20:57:54 +00:00
|
|
|
|
2019-10-29 03:03:47 +00:00
|
|
|
for (const auto & arg : arguments)
|
|
|
|
{
|
2022-11-27 12:41:34 +00:00
|
|
|
if (!isInteger(arg))
|
2019-10-29 03:03:47 +00:00
|
|
|
throw Exception{"Illegal type " + arg->getName() + " of argument of function " + getName(),
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
|
|
|
}
|
2019-11-06 10:19:08 +00:00
|
|
|
|
|
|
|
DataTypePtr common_type = getLeastSupertype(arguments);
|
|
|
|
return std::make_shared<DataTypeArray>(common_type);
|
2018-09-09 20:57:54 +00:00
|
|
|
}
|
|
|
|
|
2019-11-05 07:07:25 +00:00
|
|
|
template <typename T>
|
2020-10-19 21:21:10 +00:00
|
|
|
ColumnPtr executeInternal(const IColumn * arg) const
|
2019-11-05 07:07:25 +00:00
|
|
|
{
|
|
|
|
if (const auto in = checkAndGetColumn<ColumnVector<T>>(arg))
|
|
|
|
{
|
|
|
|
const auto & in_data = in->getData();
|
|
|
|
const auto total_values = std::accumulate(std::begin(in_data), std::end(in_data), size_t{},
|
2022-11-27 12:41:34 +00:00
|
|
|
[this] (const size_t lhs, const T rhs)
|
2019-11-05 07:07:25 +00:00
|
|
|
{
|
2022-11-27 12:41:34 +00:00
|
|
|
if (rhs < 0)
|
|
|
|
throw Exception{"A call to function " + getName() + " overflows, only support positive values when only end is provided",
|
|
|
|
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
|
|
|
|
|
2019-11-05 07:07:25 +00:00
|
|
|
const auto sum = lhs + rhs;
|
|
|
|
if (sum < lhs)
|
|
|
|
throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing",
|
|
|
|
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
|
|
|
|
|
|
|
|
return sum;
|
|
|
|
});
|
|
|
|
|
|
|
|
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(in->size());
|
|
|
|
|
|
|
|
auto & out_data = data_col->getData();
|
|
|
|
auto & out_offsets = offsets_col->getData();
|
|
|
|
|
|
|
|
IColumn::Offset offset{};
|
|
|
|
for (size_t row_idx = 0, rows = in->size(); row_idx < rows; ++row_idx)
|
|
|
|
{
|
2022-11-27 12:41:34 +00:00
|
|
|
for (T elem_idx = 0, elems = in_data[row_idx]; elem_idx < elems; ++elem_idx)
|
2022-10-07 10:46:45 +00:00
|
|
|
out_data[offset + elem_idx] = static_cast<T>(elem_idx);
|
2019-11-05 07:07:25 +00:00
|
|
|
|
|
|
|
offset += in_data[row_idx];
|
|
|
|
out_offsets[row_idx] = offset;
|
|
|
|
}
|
|
|
|
|
2020-10-19 21:21:10 +00:00
|
|
|
return ColumnArray::create(std::move(data_col), std::move(offsets_col));
|
2019-11-05 07:07:25 +00:00
|
|
|
}
|
|
|
|
else
|
2020-10-19 21:21:10 +00:00
|
|
|
return nullptr;
|
2019-11-05 07:07:25 +00:00
|
|
|
}
|
|
|
|
|
2019-11-06 10:19:08 +00:00
|
|
|
template <typename T>
|
2020-10-19 21:21:10 +00:00
|
|
|
ColumnPtr executeConstStartStep(
|
|
|
|
const IColumn * end_arg, const T start, const T step, const size_t input_rows_count) const
|
2019-11-06 10:19:08 +00:00
|
|
|
{
|
|
|
|
auto end_column = checkAndGetColumn<ColumnVector<T>>(end_arg);
|
|
|
|
if (!end_column)
|
2020-10-19 21:21:10 +00:00
|
|
|
return nullptr;
|
2019-11-06 10:19:08 +00:00
|
|
|
|
|
|
|
const auto & end_data = end_column->getData();
|
|
|
|
|
|
|
|
size_t total_values = 0;
|
|
|
|
size_t pre_values = 0;
|
2022-12-26 20:21:58 +00:00
|
|
|
PODArray<size_t> row_length(input_rows_count);
|
2019-11-06 10:19:08 +00:00
|
|
|
|
|
|
|
for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx)
|
|
|
|
{
|
2022-11-27 12:41:34 +00:00
|
|
|
if (step == 0)
|
2019-11-06 10:19:08 +00:00
|
|
|
throw Exception{"A call to function " + getName() + " overflows, the 3rd argument step can't be zero",
|
|
|
|
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
|
|
|
|
|
2022-11-27 12:41:34 +00:00
|
|
|
if (start < end_data[row_idx] && step > 0)
|
|
|
|
row_length[row_idx] = (static_cast<__int128_t>(end_data[row_idx]) - static_cast<__int128_t>(start) - 1) / static_cast<__int128_t>(step) + 1;
|
|
|
|
else if (start > end_data[row_idx] && step < 0)
|
|
|
|
row_length[row_idx] = (static_cast<__int128_t>(end_data[row_idx]) - static_cast<__int128_t>(start) + 1) / static_cast<__int128_t>(step) + 1;
|
2022-12-26 20:21:58 +00:00
|
|
|
else
|
|
|
|
row_length[row_idx] = 0;
|
2022-11-27 12:41:34 +00:00
|
|
|
|
|
|
|
pre_values += row_length[row_idx];
|
2019-11-06 10:19:08 +00:00
|
|
|
|
|
|
|
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)
|
|
|
|
{
|
2022-12-26 20:21:58 +00:00
|
|
|
for (size_t idx = 0; idx < row_length[row_idx]; ++idx)
|
|
|
|
{
|
2023-01-08 01:50:28 +00:00
|
|
|
out_data[offset] = static_cast<T>(start + idx * step);
|
2022-12-26 20:21:58 +00:00
|
|
|
++offset;
|
|
|
|
}
|
2019-11-06 10:19:08 +00:00
|
|
|
out_offsets[row_idx] = offset;
|
|
|
|
}
|
|
|
|
|
2020-10-19 21:21:10 +00:00
|
|
|
return ColumnArray::create(std::move(data_col), std::move(offsets_col));
|
2019-11-06 10:19:08 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
template <typename T>
|
2020-10-19 21:21:10 +00:00
|
|
|
ColumnPtr executeConstStep(
|
|
|
|
const IColumn * start_arg, const IColumn * end_arg, const T step, const size_t input_rows_count) const
|
2018-09-09 20:57:54 +00:00
|
|
|
{
|
2019-11-06 10:19:08 +00:00
|
|
|
auto start_column = checkAndGetColumn<ColumnVector<T>>(start_arg);
|
|
|
|
auto end_column = checkAndGetColumn<ColumnVector<T>>(end_arg);
|
|
|
|
if (!end_column || !start_column)
|
2020-10-19 21:21:10 +00:00
|
|
|
return nullptr;
|
2019-11-06 10:19:08 +00:00
|
|
|
|
|
|
|
const auto & start_data = start_column->getData();
|
|
|
|
const auto & end_data = end_column->getData();
|
|
|
|
|
|
|
|
size_t total_values = 0;
|
|
|
|
size_t pre_values = 0;
|
2022-12-26 20:21:58 +00:00
|
|
|
PODArray<size_t> row_length(input_rows_count);
|
2019-11-06 10:19:08 +00:00
|
|
|
|
|
|
|
for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx)
|
|
|
|
{
|
2022-11-27 12:41:34 +00:00
|
|
|
if (step == 0)
|
2019-11-06 10:19:08 +00:00
|
|
|
throw Exception{"A call to function " + getName() + " overflows, the 3rd argument step can't be zero",
|
|
|
|
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
|
|
|
|
|
2022-11-27 12:41:34 +00:00
|
|
|
if (start_data[row_idx] < end_data[row_idx] && step > 0)
|
|
|
|
row_length[row_idx] = (static_cast<__int128_t>(end_data[row_idx]) - static_cast<__int128_t>(start_data[row_idx]) - 1) / static_cast<__int128_t>(step) + 1;
|
|
|
|
else if (start_data[row_idx] > end_data[row_idx] && step < 0)
|
|
|
|
row_length[row_idx] = (static_cast<__int128_t>(end_data[row_idx]) - static_cast<__int128_t>(start_data[row_idx]) + 1) / static_cast<__int128_t>(step) + 1;
|
2022-12-26 20:21:58 +00:00
|
|
|
else
|
|
|
|
row_length[row_idx] = 0;
|
2022-11-27 12:41:34 +00:00
|
|
|
|
|
|
|
pre_values += row_length[row_idx];
|
2019-11-06 10:19:08 +00:00
|
|
|
|
|
|
|
if (pre_values < total_values)
|
|
|
|
throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing",
|
2022-11-27 12:41:34 +00:00
|
|
|
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
|
2019-11-06 10:19:08 +00:00
|
|
|
|
|
|
|
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)
|
|
|
|
{
|
2023-01-08 01:55:02 +00:00
|
|
|
for (size_t idx = 0; idx < row_length[row_idx]; ++idx)
|
|
|
|
{
|
|
|
|
out_data[offset] = static_cast<T>(start_data[row_idx] + idx * step);
|
|
|
|
++offset;
|
|
|
|
}
|
2019-11-06 10:19:08 +00:00
|
|
|
out_offsets[row_idx] = offset;
|
|
|
|
}
|
|
|
|
|
2020-10-19 21:21:10 +00:00
|
|
|
return ColumnArray::create(std::move(data_col), std::move(offsets_col));
|
2019-11-06 10:19:08 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
template <typename T>
|
2020-10-19 21:21:10 +00:00
|
|
|
ColumnPtr executeConstStart(
|
|
|
|
const IColumn * end_arg, const IColumn * step_arg, const T start, const size_t input_rows_count) const
|
2019-11-06 10:19:08 +00:00
|
|
|
{
|
|
|
|
auto end_column = checkAndGetColumn<ColumnVector<T>>(end_arg);
|
|
|
|
auto step_column = checkAndGetColumn<ColumnVector<T>>(step_arg);
|
|
|
|
if (!end_column || !step_column)
|
2020-10-19 21:21:10 +00:00
|
|
|
return nullptr;
|
2019-11-06 10:19:08 +00:00
|
|
|
|
|
|
|
const auto & end_data = end_column->getData();
|
|
|
|
const auto & step_data = step_column->getData();
|
|
|
|
|
|
|
|
size_t total_values = 0;
|
|
|
|
size_t pre_values = 0;
|
2022-12-26 20:21:58 +00:00
|
|
|
PODArray<size_t> row_length(input_rows_count);
|
2019-11-06 10:19:08 +00:00
|
|
|
|
|
|
|
for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx)
|
|
|
|
{
|
2022-11-27 12:41:34 +00:00
|
|
|
if (step_data[row_idx] == 0)
|
2019-11-06 10:19:08 +00:00
|
|
|
throw Exception{"A call to function " + getName() + " overflows, the 3rd argument step can't be zero",
|
2022-11-27 12:41:34 +00:00
|
|
|
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
|
|
|
|
|
|
|
|
if (start < end_data[row_idx] && step_data[row_idx] > 0)
|
|
|
|
row_length[row_idx] = (static_cast<__int128_t>(end_data[row_idx]) - static_cast<__int128_t>(start) - 1) / static_cast<__int128_t>(step_data[row_idx]) + 1;
|
|
|
|
else if (start > end_data[row_idx] && step_data[row_idx] < 0)
|
|
|
|
row_length[row_idx] = (static_cast<__int128_t>(end_data[row_idx]) - static_cast<__int128_t>(start) + 1) / static_cast<__int128_t>(step_data[row_idx]) + 1;
|
2022-12-26 20:21:58 +00:00
|
|
|
else
|
|
|
|
row_length[row_idx] = 0;
|
2019-11-06 10:19:08 +00:00
|
|
|
|
2022-11-27 12:41:34 +00:00
|
|
|
pre_values += row_length[row_idx];
|
2019-11-06 10:19:08 +00:00
|
|
|
|
|
|
|
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)
|
|
|
|
{
|
2023-01-08 01:50:28 +00:00
|
|
|
for (size_t idx = 0; idx < row_length[row_idx]; ++idx)
|
|
|
|
{
|
|
|
|
out_data[offset] = static_cast<T>(start + idx * step_data[row_idx]);
|
|
|
|
++offset;
|
|
|
|
}
|
2019-11-06 10:19:08 +00:00
|
|
|
out_offsets[row_idx] = offset;
|
|
|
|
}
|
|
|
|
|
2020-10-19 21:21:10 +00:00
|
|
|
return ColumnArray::create(std::move(data_col), std::move(offsets_col));
|
2019-11-06 10:19:08 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
template <typename T>
|
2020-10-19 21:21:10 +00:00
|
|
|
ColumnPtr executeGeneric(
|
|
|
|
const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t input_rows_count) const
|
2019-11-06 10:19:08 +00:00
|
|
|
{
|
|
|
|
auto start_column = checkAndGetColumn<ColumnVector<T>>(start_col);
|
|
|
|
auto end_column = checkAndGetColumn<ColumnVector<T>>(end_col);
|
|
|
|
auto step_column = checkAndGetColumn<ColumnVector<T>>(step_col);
|
2019-10-29 06:33:03 +00:00
|
|
|
|
|
|
|
if (!start_column || !end_column || !step_column)
|
2020-10-19 21:21:10 +00:00
|
|
|
return nullptr;
|
2018-09-09 20:57:54 +00:00
|
|
|
|
2019-10-29 06:33:03 +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;
|
2019-10-29 06:33:03 +00:00
|
|
|
size_t pre_values = 0;
|
2022-12-26 20:21:58 +00:00
|
|
|
PODArray<size_t> row_length(input_rows_count);
|
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
|
|
|
{
|
2022-11-27 12:41:34 +00:00
|
|
|
if (step_data[row_idx] == 0)
|
2023-01-08 01:55:02 +00:00
|
|
|
throw Exception{ErrorCodes::ARGUMENT_OUT_OF_BOUND,
|
|
|
|
"A call to function {} underflows, the 3rd argument step can't be less or equal to zero", getName()};
|
2022-11-27 12:41:34 +00:00
|
|
|
if (start_data[row_idx] < end_start[row_idx] && step_data[row_idx] > 0)
|
|
|
|
row_length[row_idx] = (static_cast<__int128_t>(end_start[row_idx]) - static_cast<__int128_t>(start_data[row_idx]) - 1) / static_cast<__int128_t>(step_data[row_idx]) + 1;
|
|
|
|
else if (start_data[row_idx] > end_start[row_idx] && step_data[row_idx] < 0)
|
|
|
|
row_length[row_idx] = (static_cast<__int128_t>(end_start[row_idx]) - static_cast<__int128_t>(start_data[row_idx]) + 1) / static_cast<__int128_t>(step_data[row_idx]) + 1;
|
2022-12-26 20:21:58 +00:00
|
|
|
else
|
|
|
|
row_length[row_idx] = 0;
|
2019-10-30 01:36:50 +00:00
|
|
|
|
2022-11-27 12:41:34 +00:00
|
|
|
pre_values += row_length[row_idx];
|
2019-10-29 06:33:03 +00:00
|
|
|
|
|
|
|
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
|
|
|
|
2019-11-06 10:19:08 +00:00
|
|
|
auto data_col = ColumnVector<T>::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
|
|
|
{
|
2023-01-08 01:55:02 +00:00
|
|
|
for (size_t idx = 0; idx < row_length[row_idx]; ++idx)
|
|
|
|
{
|
|
|
|
out_data[offset] = static_cast<T>(start_data[row_idx] + idx * step_data[row_idx]);
|
|
|
|
++offset;
|
|
|
|
}
|
2019-10-29 03:03:47 +00:00
|
|
|
out_offsets[row_idx] = offset;
|
2018-09-09 20:57:54 +00:00
|
|
|
}
|
|
|
|
|
2020-10-19 21:21:10 +00:00
|
|
|
return ColumnArray::create(std::move(data_col), std::move(offsets_col));
|
2019-10-29 03:03:47 +00:00
|
|
|
}
|
2019-10-29 06:33:03 +00:00
|
|
|
|
2020-11-17 13:24:45 +00:00
|
|
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
2019-10-29 06:33:03 +00:00
|
|
|
{
|
2021-03-31 00:54:40 +00:00
|
|
|
DataTypePtr elem_type = checkAndGetDataType<DataTypeArray>(result_type.get())->getNestedType();
|
|
|
|
WhichDataType which(elem_type);
|
|
|
|
|
2022-11-27 12:41:34 +00:00
|
|
|
if (!which.isNativeUInt() && !which.isNativeInt())
|
2021-03-31 00:54:40 +00:00
|
|
|
{
|
|
|
|
throw Exception{"Illegal columns of arguments of function " + getName()
|
2022-11-27 12:41:34 +00:00
|
|
|
+ ", the function only implemented for unsigned/signed integers up to 64 bit",
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN};
|
2021-03-31 00:54:40 +00:00
|
|
|
}
|
|
|
|
|
2020-10-19 21:21:10 +00:00
|
|
|
ColumnPtr res;
|
2019-10-29 05:01:45 +00:00
|
|
|
if (arguments.size() == 1)
|
|
|
|
{
|
2020-10-19 21:21:10 +00:00
|
|
|
const auto * col = arguments[0].column.get();
|
2022-11-27 12:41:34 +00:00
|
|
|
if (!((res = executeInternal<UInt8>(col)) || (res = executeInternal<UInt16>(col)) || (res = executeInternal<UInt32>(col))
|
|
|
|
|| (res = executeInternal<UInt64>(col)) || (res = executeInternal<Int8>(col)) || (res = executeInternal<Int16>(col))
|
|
|
|
|| (res = executeInternal<Int32>(col)) || (res = executeInternal<Int64>(col))))
|
2019-11-05 07:07:25 +00:00
|
|
|
{
|
|
|
|
throw Exception{"Illegal column " + col->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
|
|
|
}
|
2020-10-19 21:21:10 +00:00
|
|
|
return res;
|
2019-10-29 05:01:45 +00:00
|
|
|
}
|
|
|
|
|
2019-11-06 10:19:08 +00:00
|
|
|
Columns columns_holder(3);
|
2020-10-14 14:44:22 +00:00
|
|
|
ColumnRawPtrs column_ptrs(3);
|
2019-11-06 10:19:08 +00:00
|
|
|
|
2019-10-29 05:01:45 +00:00
|
|
|
for (size_t i = 0; i < arguments.size(); ++i)
|
|
|
|
{
|
2019-11-06 10:19:08 +00:00
|
|
|
if (i == 1)
|
2021-03-31 00:54:40 +00:00
|
|
|
columns_holder[i] = castColumn(arguments[i], elem_type)->convertToFullColumnIfConst();
|
2019-11-06 10:19:08 +00:00
|
|
|
else
|
2021-03-31 00:54:40 +00:00
|
|
|
columns_holder[i] = castColumn(arguments[i], elem_type);
|
2019-11-06 10:19:08 +00:00
|
|
|
|
2020-10-14 14:44:22 +00:00
|
|
|
column_ptrs[i] = columns_holder[i].get();
|
2019-10-29 05:01:45 +00:00
|
|
|
}
|
|
|
|
|
2021-03-31 00:54:40 +00:00
|
|
|
/// Step is one by default.
|
2019-11-05 07:07:25 +00:00
|
|
|
if (arguments.size() == 2)
|
2019-10-29 06:33:03 +00:00
|
|
|
{
|
2021-03-31 00:54:40 +00:00
|
|
|
/// Convert a column with constant 1 to the result type.
|
|
|
|
columns_holder[2] = castColumn(
|
|
|
|
{DataTypeUInt8().createColumnConst(input_rows_count, 1), std::make_shared<DataTypeUInt8>(), {}},
|
|
|
|
elem_type);
|
|
|
|
|
2020-10-14 14:44:22 +00:00
|
|
|
column_ptrs[2] = columns_holder[2].get();
|
2019-10-29 06:33:03 +00:00
|
|
|
}
|
|
|
|
|
2020-10-14 14:44:22 +00:00
|
|
|
bool is_start_const = isColumnConst(*column_ptrs[0]);
|
|
|
|
bool is_step_const = isColumnConst(*column_ptrs[2]);
|
2019-11-06 10:19:08 +00:00
|
|
|
if (is_start_const && is_step_const)
|
|
|
|
{
|
2022-11-27 12:41:34 +00:00
|
|
|
if (which.isNativeUInt())
|
2021-03-31 00:54:40 +00:00
|
|
|
{
|
2022-11-27 12:41:34 +00:00
|
|
|
UInt64 start = assert_cast<const ColumnConst &>(*column_ptrs[0]).getUInt(0);
|
|
|
|
UInt64 step = assert_cast<const ColumnConst &>(*column_ptrs[2]).getUInt(0);
|
|
|
|
|
|
|
|
if ((res = executeConstStartStep<UInt8>(column_ptrs[1], start, step, input_rows_count))
|
|
|
|
|| (res = executeConstStartStep<UInt16>(column_ptrs[1], start, step, input_rows_count))
|
|
|
|
|| (res = executeConstStartStep<UInt32>(
|
|
|
|
column_ptrs[1], static_cast<UInt32>(start), static_cast<UInt32>(step), input_rows_count))
|
|
|
|
|| (res = executeConstStartStep<UInt64>(column_ptrs[1], start, step, input_rows_count)))
|
|
|
|
{
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else if (which.isNativeInt())
|
|
|
|
{
|
|
|
|
Int64 start = assert_cast<const ColumnConst &>(*column_ptrs[0]).getInt(0);
|
|
|
|
Int64 step = assert_cast<const ColumnConst &>(*column_ptrs[2]).getInt(0);
|
|
|
|
|
|
|
|
if ((res = executeConstStartStep<Int8>(column_ptrs[1], start, step, input_rows_count))
|
|
|
|
|| (res = executeConstStartStep<Int16>(column_ptrs[1], start, step, input_rows_count))
|
|
|
|
|| (res = executeConstStartStep<Int32>(
|
|
|
|
column_ptrs[1], static_cast<Int32>(start), static_cast<Int32>(step), input_rows_count))
|
|
|
|
|| (res = executeConstStartStep<Int64>(column_ptrs[1], start, step, input_rows_count)))
|
|
|
|
{
|
|
|
|
}
|
2021-03-31 00:54:40 +00:00
|
|
|
}
|
2019-11-06 10:19:08 +00:00
|
|
|
}
|
|
|
|
else if (is_start_const && !is_step_const)
|
|
|
|
{
|
2022-11-27 12:41:34 +00:00
|
|
|
if (which.isNativeUInt())
|
|
|
|
{
|
|
|
|
UInt64 start = assert_cast<const ColumnConst &>(*column_ptrs[0]).getUInt(0);
|
2019-11-06 10:19:08 +00:00
|
|
|
|
2022-11-27 12:41:34 +00:00
|
|
|
if ((res = executeConstStart<UInt8>(column_ptrs[1], column_ptrs[2], start, input_rows_count))
|
|
|
|
|| (res = executeConstStart<UInt16>(column_ptrs[1], column_ptrs[2], start, input_rows_count))
|
|
|
|
|| (res = executeConstStart<UInt32>(column_ptrs[1], column_ptrs[2], static_cast<UInt32>(start), input_rows_count))
|
|
|
|
|| (res = executeConstStart<UInt64>(column_ptrs[1], column_ptrs[2], start, input_rows_count)))
|
|
|
|
{
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else if (which.isNativeInt())
|
2021-03-31 00:54:40 +00:00
|
|
|
{
|
2022-11-27 12:41:34 +00:00
|
|
|
Int64 start = assert_cast<const ColumnConst &>(*column_ptrs[0]).getInt(0);
|
|
|
|
|
|
|
|
if ((res = executeConstStart<Int8>(column_ptrs[1], column_ptrs[2], start, input_rows_count))
|
|
|
|
|| (res = executeConstStart<Int16>(column_ptrs[1], column_ptrs[2], start, input_rows_count))
|
|
|
|
|| (res = executeConstStart<Int32>(column_ptrs[1], column_ptrs[2], static_cast<Int32>(start), input_rows_count))
|
|
|
|
|| (res = executeConstStart<Int64>(column_ptrs[1], column_ptrs[2], start, input_rows_count)))
|
|
|
|
{
|
|
|
|
}
|
2021-03-31 00:54:40 +00:00
|
|
|
}
|
2019-11-06 10:19:08 +00:00
|
|
|
}
|
|
|
|
else if (!is_start_const && is_step_const)
|
|
|
|
{
|
2022-11-27 12:41:34 +00:00
|
|
|
if (which.isNativeUInt())
|
|
|
|
{
|
|
|
|
UInt64 step = assert_cast<const ColumnConst &>(*column_ptrs[2]).getUInt(0);
|
2019-11-06 10:19:08 +00:00
|
|
|
|
2022-11-27 12:41:34 +00:00
|
|
|
if ((res = executeConstStep<UInt8>(column_ptrs[0], column_ptrs[1], step, input_rows_count))
|
|
|
|
|| (res = executeConstStep<UInt16>(column_ptrs[0], column_ptrs[1], step, input_rows_count))
|
|
|
|
|| (res = executeConstStep<UInt32>(column_ptrs[0], column_ptrs[1], static_cast<UInt32>(step), input_rows_count))
|
|
|
|
|| (res = executeConstStep<UInt64>(column_ptrs[0], column_ptrs[1], step, input_rows_count)))
|
|
|
|
{
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else if (which.isNativeInt())
|
2021-03-31 00:54:40 +00:00
|
|
|
{
|
2022-11-27 12:41:34 +00:00
|
|
|
Int64 step = assert_cast<const ColumnConst &>(*column_ptrs[2]).getInt(0);
|
|
|
|
|
|
|
|
if ((res = executeConstStep<Int8>(column_ptrs[0], column_ptrs[1], step, input_rows_count))
|
|
|
|
|| (res = executeConstStep<Int16>(column_ptrs[0], column_ptrs[1], step, input_rows_count))
|
|
|
|
|| (res = executeConstStep<Int32>(column_ptrs[0], column_ptrs[1], static_cast<Int32>(step), input_rows_count))
|
|
|
|
|| (res = executeConstStep<Int64>(column_ptrs[0], column_ptrs[1], step, input_rows_count)))
|
|
|
|
{
|
|
|
|
}
|
2021-03-31 00:54:40 +00:00
|
|
|
}
|
2019-11-06 10:19:08 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2022-11-27 12:41:34 +00:00
|
|
|
if ((res = executeGeneric<UInt8>(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count))
|
|
|
|
|| (res = executeGeneric<UInt16>(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count))
|
|
|
|
|| (res = executeGeneric<UInt32>(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count))
|
|
|
|
|| (res = executeGeneric<UInt64>(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count))
|
|
|
|
|| (res = executeGeneric<Int8>(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count))
|
|
|
|
|| (res = executeGeneric<Int16>(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count))
|
|
|
|
|| (res = executeGeneric<Int32>(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count))
|
|
|
|
|| (res = executeGeneric<Int64>(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count)))
|
2021-03-31 00:54:40 +00:00
|
|
|
{
|
|
|
|
}
|
2019-11-06 10:19:08 +00:00
|
|
|
}
|
|
|
|
|
2020-10-19 21:21:10 +00:00
|
|
|
if (!res)
|
2018-09-09 20:57:54 +00:00
|
|
|
{
|
2020-10-14 14:44:22 +00:00
|
|
|
throw Exception{"Illegal columns " + column_ptrs[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
2018-09-09 20:57:54 +00:00
|
|
|
}
|
2020-10-19 21:21:10 +00:00
|
|
|
|
|
|
|
return res;
|
2018-09-09 20:57:54 +00:00
|
|
|
}
|
2019-10-29 03:03:47 +00:00
|
|
|
|
2018-09-09 20:57:54 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2022-07-04 07:01:39 +00:00
|
|
|
REGISTER_FUNCTION(Range)
|
2018-09-09 20:57:54 +00:00
|
|
|
{
|
|
|
|
factory.registerFunction<FunctionRange>();
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|