2019-08-12 14:10:29 +00:00
|
|
|
#include <Columns/ColumnConst.h>
|
2019-08-13 13:20:32 +00:00
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
2019-08-12 14:10:29 +00:00
|
|
|
#include <DataTypes/getLeastSupertype.h>
|
|
|
|
#include <Functions/FunctionFactory.h>
|
2021-05-17 07:30:42 +00:00
|
|
|
#include <Functions/IFunction.h>
|
2019-08-12 14:10:29 +00:00
|
|
|
#include <IO/WriteHelpers.h>
|
|
|
|
#include <Interpreters/castColumn.h>
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
2020-06-28 20:43:26 +00:00
|
|
|
extern const int ARGUMENT_OUT_OF_BOUND;
|
2019-08-12 14:10:29 +00:00
|
|
|
}
|
|
|
|
|
2020-09-07 18:00:37 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
2019-08-12 14:10:29 +00:00
|
|
|
// Implements function, giving value for column within range of given
|
|
|
|
// Example:
|
|
|
|
// | c1 |
|
|
|
|
// | 10 |
|
|
|
|
// | 20 |
|
2019-08-23 01:31:04 +00:00
|
|
|
// SELECT c1, neighbor(c1, 1) as c2:
|
2019-08-12 14:10:29 +00:00
|
|
|
// | c1 | c2 |
|
|
|
|
// | 10 | 20 |
|
|
|
|
// | 20 | 0 |
|
2019-08-23 01:31:04 +00:00
|
|
|
class FunctionNeighbor : public IFunction
|
2019-08-12 14:10:29 +00:00
|
|
|
{
|
|
|
|
public:
|
2019-08-23 01:31:04 +00:00
|
|
|
static constexpr auto name = "neighbor";
|
2021-06-01 12:20:52 +00:00
|
|
|
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionNeighbor>(); }
|
2019-08-12 14:10:29 +00:00
|
|
|
|
|
|
|
/// Get the name of the function.
|
|
|
|
String getName() const override { return name; }
|
|
|
|
|
|
|
|
size_t getNumberOfArguments() const override { return 0; }
|
|
|
|
|
|
|
|
bool isVariadic() const override { return true; }
|
|
|
|
|
2020-01-06 04:18:56 +00:00
|
|
|
bool isStateful() const override { return true; }
|
|
|
|
|
2019-08-12 14:10:29 +00:00
|
|
|
bool isDeterministic() const override { return false; }
|
|
|
|
|
|
|
|
bool isDeterministicInScopeOfQuery() const override { return false; }
|
|
|
|
|
2021-06-22 16:21:23 +00:00
|
|
|
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
2021-04-29 14:48:26 +00:00
|
|
|
|
2019-08-14 12:09:51 +00:00
|
|
|
bool useDefaultImplementationForNulls() const override { return false; }
|
|
|
|
|
2019-08-23 03:00:06 +00:00
|
|
|
bool useDefaultImplementationForConstants() const override { return false; }
|
|
|
|
|
2021-01-27 13:26:16 +00:00
|
|
|
/// We do not use default implementation for LowCardinality because this is not a pure function.
|
|
|
|
/// If used, optimization for LC may execute function only for dictionary, which gives wrong result.
|
2021-01-26 10:31:29 +00:00
|
|
|
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
|
|
|
|
|
2019-08-12 14:10:29 +00:00
|
|
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
|
|
|
{
|
|
|
|
size_t number_of_arguments = arguments.size();
|
|
|
|
|
|
|
|
if (number_of_arguments < 2 || number_of_arguments > 3)
|
2023-01-17 16:39:07 +00:00
|
|
|
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
|
|
|
"Number of arguments for function {} doesn't match: passed {}, should be from 2 to 3",
|
2023-01-23 21:13:58 +00:00
|
|
|
getName(), number_of_arguments);
|
2019-08-12 14:10:29 +00:00
|
|
|
|
2019-08-14 11:32:03 +00:00
|
|
|
// second argument must be an integer
|
2019-08-12 14:10:29 +00:00
|
|
|
if (!isInteger(arguments[1]))
|
2023-01-17 16:39:07 +00:00
|
|
|
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
|
|
|
"Illegal type {} of second argument of function {} - should be an integer", arguments[1]->getName(), getName());
|
2019-08-14 12:09:51 +00:00
|
|
|
else if (arguments[1]->isNullable())
|
2023-01-17 16:39:07 +00:00
|
|
|
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
|
|
|
"Illegal type {} of second argument of function {} - can not be Nullable", arguments[1]->getName(), getName());
|
2019-08-12 14:10:29 +00:00
|
|
|
|
|
|
|
// check that default value column has supertype with first argument
|
|
|
|
if (number_of_arguments == 3)
|
2021-06-12 15:10:25 +00:00
|
|
|
return getLeastSupertype(DataTypes{arguments[0], arguments[2]});
|
2019-08-12 14:10:29 +00:00
|
|
|
|
|
|
|
return arguments[0];
|
|
|
|
}
|
|
|
|
|
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-08-12 14:10:29 +00:00
|
|
|
{
|
2020-10-19 15:27:41 +00:00
|
|
|
const ColumnWithTypeAndName & source_elem = arguments[0];
|
|
|
|
const ColumnWithTypeAndName & offset_elem = arguments[1];
|
2019-08-23 03:00:06 +00:00
|
|
|
bool has_defaults = arguments.size() == 3;
|
2019-08-12 14:10:29 +00:00
|
|
|
|
2020-04-14 21:05:45 +00:00
|
|
|
ColumnPtr source_column_casted = castColumn(source_elem, result_type);
|
2019-08-23 03:00:06 +00:00
|
|
|
ColumnPtr offset_column = offset_elem.column;
|
2019-08-23 02:02:14 +00:00
|
|
|
|
2019-08-23 03:00:06 +00:00
|
|
|
ColumnPtr default_column_casted;
|
|
|
|
if (has_defaults)
|
2019-08-13 13:20:32 +00:00
|
|
|
{
|
2020-10-19 15:27:41 +00:00
|
|
|
const ColumnWithTypeAndName & default_elem = arguments[2];
|
2020-04-14 21:05:45 +00:00
|
|
|
default_column_casted = castColumn(default_elem, result_type);
|
2019-08-12 14:10:29 +00:00
|
|
|
}
|
|
|
|
|
2019-08-23 03:00:06 +00:00
|
|
|
bool source_is_constant = isColumnConst(*source_column_casted);
|
|
|
|
bool offset_is_constant = isColumnConst(*offset_column);
|
2019-08-23 02:02:14 +00:00
|
|
|
|
2019-08-23 03:00:06 +00:00
|
|
|
bool default_is_constant = false;
|
|
|
|
if (has_defaults)
|
|
|
|
default_is_constant = isColumnConst(*default_column_casted);
|
2019-08-23 02:02:14 +00:00
|
|
|
|
2019-08-23 03:00:06 +00:00
|
|
|
if (source_is_constant)
|
|
|
|
source_column_casted = assert_cast<const ColumnConst &>(*source_column_casted).getDataColumnPtr();
|
|
|
|
if (offset_is_constant)
|
|
|
|
offset_column = assert_cast<const ColumnConst &>(*offset_column).getDataColumnPtr();
|
|
|
|
if (default_is_constant)
|
|
|
|
default_column_casted = assert_cast<const ColumnConst &>(*default_column_casted).getDataColumnPtr();
|
2019-08-12 14:10:29 +00:00
|
|
|
|
2019-08-23 04:25:32 +00:00
|
|
|
if (offset_is_constant)
|
2019-08-13 13:20:32 +00:00
|
|
|
{
|
2019-08-23 04:25:32 +00:00
|
|
|
/// Optimization for the case when we can copy many values at once.
|
|
|
|
|
|
|
|
Int64 offset = offset_column->getInt(0);
|
|
|
|
|
2020-06-28 20:43:26 +00:00
|
|
|
/// Protection from possible overflow.
|
|
|
|
if (unlikely(offset > (1 << 30) || offset < -(1 << 30)))
|
|
|
|
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Too large offset: {} in function {}", offset, getName());
|
|
|
|
|
2019-08-23 04:25:32 +00:00
|
|
|
auto result_column = result_type->createColumn();
|
|
|
|
|
|
|
|
auto insert_range_from = [&](bool is_const, const ColumnPtr & src, Int64 begin, Int64 size)
|
|
|
|
{
|
|
|
|
/// Saturation of bounds.
|
|
|
|
if (begin < 0)
|
|
|
|
{
|
|
|
|
size += begin;
|
|
|
|
begin = 0;
|
|
|
|
}
|
|
|
|
if (size <= 0)
|
|
|
|
return;
|
2022-04-18 08:18:31 +00:00
|
|
|
if (size > static_cast<Int64>(input_rows_count))
|
2019-08-23 04:25:32 +00:00
|
|
|
size = input_rows_count;
|
|
|
|
|
|
|
|
if (!src)
|
|
|
|
{
|
|
|
|
for (Int64 i = 0; i < size; ++i)
|
|
|
|
result_column->insertDefault();
|
|
|
|
}
|
|
|
|
else if (is_const)
|
|
|
|
{
|
|
|
|
for (Int64 i = 0; i < size; ++i)
|
|
|
|
result_column->insertFrom(*src, 0);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
result_column->insertRangeFrom(*src, begin, size);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
if (offset == 0)
|
|
|
|
{
|
|
|
|
/// Degenerate case, just copy source column as is.
|
2020-10-19 15:27:41 +00:00
|
|
|
return source_is_constant
|
2020-06-28 20:43:26 +00:00
|
|
|
? ColumnConst::create(source_column_casted, input_rows_count)
|
|
|
|
: source_column_casted;
|
2019-08-23 04:25:32 +00:00
|
|
|
}
|
|
|
|
else if (offset > 0)
|
|
|
|
{
|
2022-04-18 08:18:31 +00:00
|
|
|
insert_range_from(source_is_constant, source_column_casted, offset, static_cast<Int64>(input_rows_count) - offset);
|
|
|
|
insert_range_from(default_is_constant, default_column_casted, static_cast<Int64>(input_rows_count) - offset, offset);
|
2020-10-19 15:27:41 +00:00
|
|
|
return result_column;
|
2019-08-23 04:25:32 +00:00
|
|
|
}
|
2019-08-13 13:20:32 +00:00
|
|
|
else
|
2019-08-23 04:25:32 +00:00
|
|
|
{
|
|
|
|
insert_range_from(default_is_constant, default_column_casted, 0, -offset);
|
2022-04-18 08:18:31 +00:00
|
|
|
insert_range_from(source_is_constant, source_column_casted, 0, static_cast<Int64>(input_rows_count) + offset);
|
2020-10-19 15:27:41 +00:00
|
|
|
return result_column;
|
2019-08-23 04:25:32 +00:00
|
|
|
}
|
2019-08-12 14:10:29 +00:00
|
|
|
}
|
2019-08-23 04:25:32 +00:00
|
|
|
else
|
|
|
|
{
|
|
|
|
auto result_column = result_type->createColumn();
|
2019-08-23 03:00:06 +00:00
|
|
|
|
2019-08-23 04:25:32 +00:00
|
|
|
for (size_t row = 0; row < input_rows_count; ++row)
|
|
|
|
{
|
2021-05-08 14:43:03 +00:00
|
|
|
Int64 offset = offset_column->getInt(row);
|
2020-06-28 20:43:26 +00:00
|
|
|
|
|
|
|
/// Protection from possible overflow.
|
|
|
|
if (unlikely(offset > (1 << 30) || offset < -(1 << 30)))
|
|
|
|
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Too large offset: {} in function {}", offset, getName());
|
|
|
|
|
|
|
|
Int64 src_idx = row + offset;
|
2019-08-23 04:25:32 +00:00
|
|
|
|
2022-04-18 08:18:31 +00:00
|
|
|
if (src_idx >= 0 && src_idx < static_cast<Int64>(input_rows_count))
|
2019-08-23 04:25:32 +00:00
|
|
|
result_column->insertFrom(*source_column_casted, source_is_constant ? 0 : src_idx);
|
|
|
|
else if (has_defaults)
|
|
|
|
result_column->insertFrom(*default_column_casted, default_is_constant ? 0 : row);
|
|
|
|
else
|
|
|
|
result_column->insertDefault();
|
|
|
|
}
|
|
|
|
|
2020-10-19 15:27:41 +00:00
|
|
|
return result_column;
|
2019-08-23 04:25:32 +00:00
|
|
|
}
|
2019-08-12 14:10:29 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2020-09-07 18:00:37 +00:00
|
|
|
}
|
|
|
|
|
2022-07-04 07:01:39 +00:00
|
|
|
REGISTER_FUNCTION(Neighbor)
|
2019-08-12 14:10:29 +00:00
|
|
|
{
|
2019-08-23 01:31:04 +00:00
|
|
|
factory.registerFunction<FunctionNeighbor>();
|
2019-08-12 14:10:29 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|