ClickHouse/src/Functions/neighbor.cpp

195 lines
7.2 KiB
C++
Raw Normal View History

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>
#include <Functions/IFunctionImpl.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;
}
// 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";
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionNeighbor>(context); }
2019-08-12 14:10:29 +00:00
2020-03-18 03:27:32 +00:00
explicit FunctionNeighbor(const Context & context_) : context(context_) {}
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; }
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; }
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)
throw Exception(
"Number of arguments for function " + getName() + " doesn't match: passed " + toString(number_of_arguments)
+ ", should be from 2 to 3",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
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]))
throw Exception(
2019-08-14 12:09:51 +00:00
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + " - should be an integer",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
else if (arguments[1]->isNullable())
throw Exception(
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + " - can not be Nullable",
2019-08-12 14:10:29 +00:00
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
// check that default value column has supertype with first argument
if (number_of_arguments == 3)
2019-08-23 03:00:06 +00:00
return getLeastSupertype({arguments[0], arguments[2]});
2019-08-12 14:10:29 +00:00
return arguments[0];
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
2019-08-23 02:02:14 +00:00
const DataTypePtr & result_type = block.getByPosition(result).type;
2019-08-12 14:10:29 +00:00
2019-08-23 03:00:06 +00:00
const ColumnWithTypeAndName & source_elem = block.getByPosition(arguments[0]);
const ColumnWithTypeAndName & offset_elem = block.getByPosition(arguments[1]);
bool has_defaults = arguments.size() == 3;
2019-08-12 14:10:29 +00:00
2019-08-23 03:00:06 +00:00
ColumnPtr source_column_casted = castColumn(source_elem, result_type, context);
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
{
2019-08-23 03:00:06 +00:00
const ColumnWithTypeAndName & default_elem = block.getByPosition(arguments[2]);
default_column_casted = castColumn(default_elem, result_type, context);
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);
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;
if (size > Int64(input_rows_count))
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.
2019-08-23 04:34:59 +00:00
block.getByPosition(result).column = source_is_constant ? ColumnConst::create(source_column_casted, input_rows_count) : source_column_casted;
2019-08-23 04:25:32 +00:00
}
else if (offset > 0)
{
insert_range_from(source_is_constant, source_column_casted, offset, Int64(input_rows_count) - offset);
insert_range_from(default_is_constant, default_column_casted, Int64(input_rows_count) - offset, offset);
block.getByPosition(result).column = std::move(result_column);
}
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);
insert_range_from(source_is_constant, source_column_casted, 0, Int64(input_rows_count) + offset);
block.getByPosition(result).column = std::move(result_column);
}
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)
{
Int64 src_idx = row + offset_column->getInt(offset_is_constant ? 0 : row);
if (src_idx >= 0 && src_idx < Int64(input_rows_count))
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();
}
block.getByPosition(result).column = std::move(result_column);
}
2019-08-12 14:10:29 +00:00
}
2019-08-13 13:20:32 +00:00
2019-08-12 14:10:29 +00:00
private:
const Context & context;
};
2019-08-23 01:31:04 +00:00
void registerFunctionNeighbor(FunctionFactory & factory)
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
}
}