2018-02-23 21:22:52 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <Columns/ColumnConst.h>
|
2018-05-07 02:07:47 +00:00
|
|
|
#include <Columns/ColumnsNumber.h>
|
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
2018-02-23 21:22:52 +00:00
|
|
|
#include <Functions/FunctionHelpers.h>
|
2021-05-17 07:30:42 +00:00
|
|
|
#include <Functions/IFunction.h>
|
2018-05-07 02:07:47 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2021-04-10 23:33:54 +00:00
|
|
|
#include <Interpreters/Context_fwd.h>
|
2018-02-27 00:01:51 +00:00
|
|
|
|
2018-02-23 21:22:52 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2020-02-25 18:02:41 +00:00
|
|
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
2018-02-23 21:22:52 +00:00
|
|
|
extern const int BAD_ARGUMENTS;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
template <typename Impl>
|
|
|
|
class FunctionConsistentHashImpl : public IFunction
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
static constexpr auto name = Impl::name;
|
|
|
|
|
2021-06-01 12:20:52 +00:00
|
|
|
static FunctionPtr create(ContextPtr)
|
2018-05-07 02:07:47 +00:00
|
|
|
{
|
|
|
|
return std::make_shared<FunctionConsistentHashImpl<Impl>>();
|
2018-08-26 00:50:48 +00:00
|
|
|
}
|
2018-02-23 21:22:52 +00:00
|
|
|
|
2018-05-07 02:07:47 +00:00
|
|
|
String getName() const override
|
|
|
|
{
|
|
|
|
return name;
|
|
|
|
}
|
2018-02-23 21:22:52 +00:00
|
|
|
|
2018-05-07 02:07:47 +00:00
|
|
|
size_t getNumberOfArguments() const override
|
|
|
|
{
|
|
|
|
return 2;
|
|
|
|
}
|
2018-02-23 21:22:52 +00:00
|
|
|
|
2021-06-22 16:21:23 +00:00
|
|
|
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
|
2021-04-29 14:48:26 +00:00
|
|
|
|
2018-02-23 21:22:52 +00:00
|
|
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
|
|
|
{
|
2018-09-07 14:37:26 +00:00
|
|
|
if (!isInteger(arguments[0]))
|
2018-02-23 21:22:52 +00:00
|
|
|
throw Exception("Illegal type " + arguments[0]->getName() + " of the first argument of function " + getName(),
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
2018-02-26 01:27:33 +00:00
|
|
|
if (arguments[0]->getSizeOfValueInMemory() > sizeof(HashType))
|
|
|
|
throw Exception("Function " + getName() + " accepts " + std::to_string(sizeof(HashType) * 8) + "-bit integers at most"
|
2018-05-07 02:07:47 +00:00
|
|
|
+ ", got " + arguments[0]->getName(),
|
|
|
|
ErrorCodes::BAD_ARGUMENTS);
|
2018-02-26 01:27:33 +00:00
|
|
|
|
2018-09-07 14:37:26 +00:00
|
|
|
if (!isInteger(arguments[1]))
|
2018-02-23 21:22:52 +00:00
|
|
|
throw Exception("Illegal type " + arguments[1]->getName() + " of the second argument of function " + getName(),
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
|
|
|
return std::make_shared<DataTypeNumber<ResultType>>();
|
|
|
|
}
|
|
|
|
|
2018-05-07 02:07:47 +00:00
|
|
|
bool useDefaultImplementationForConstants() const override
|
|
|
|
{
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override
|
|
|
|
{
|
|
|
|
return {1};
|
|
|
|
}
|
2018-02-26 01:27:33 +00:00
|
|
|
|
2020-11-17 13:24:45 +00:00
|
|
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
2018-02-23 21:22:52 +00:00
|
|
|
{
|
2020-10-18 14:49:19 +00:00
|
|
|
if (isColumnConst(*arguments[1].column))
|
|
|
|
return executeConstBuckets(arguments);
|
2018-02-26 01:27:33 +00:00
|
|
|
else
|
2018-05-07 02:07:47 +00:00
|
|
|
throw Exception(
|
|
|
|
"The second argument of function " + getName() + " (number of buckets) must be constant", ErrorCodes::BAD_ARGUMENTS);
|
2018-02-26 01:27:33 +00:00
|
|
|
}
|
2018-02-23 21:22:52 +00:00
|
|
|
|
2018-02-26 01:27:33 +00:00
|
|
|
private:
|
|
|
|
using HashType = typename Impl::HashType;
|
|
|
|
using ResultType = typename Impl::ResultType;
|
2019-08-02 22:40:28 +00:00
|
|
|
using BucketsType = typename Impl::BucketsType;
|
2018-02-26 01:27:33 +00:00
|
|
|
|
|
|
|
template <typename T>
|
2020-07-21 13:58:07 +00:00
|
|
|
inline BucketsType checkBucketsRange(T buckets) const
|
2018-02-26 01:27:33 +00:00
|
|
|
{
|
|
|
|
if (unlikely(buckets <= 0))
|
2018-05-07 02:07:47 +00:00
|
|
|
throw Exception(
|
|
|
|
"The second argument of function " + getName() + " (number of buckets) must be positive number", ErrorCodes::BAD_ARGUMENTS);
|
2018-02-26 01:27:33 +00:00
|
|
|
|
2019-08-02 22:40:28 +00:00
|
|
|
if (unlikely(static_cast<UInt64>(buckets) > Impl::max_buckets))
|
2019-08-02 22:42:21 +00:00
|
|
|
throw Exception("The value of the second argument of function " + getName() + " (number of buckets) must not be greater than "
|
|
|
|
+ std::to_string(Impl::max_buckets), ErrorCodes::BAD_ARGUMENTS);
|
2018-02-23 21:22:52 +00:00
|
|
|
|
2018-02-26 01:27:33 +00:00
|
|
|
return static_cast<BucketsType>(buckets);
|
|
|
|
}
|
2018-02-23 21:22:52 +00:00
|
|
|
|
2020-11-17 13:24:45 +00:00
|
|
|
ColumnPtr executeConstBuckets(const ColumnsWithTypeAndName & arguments) const
|
2018-02-26 01:27:33 +00:00
|
|
|
{
|
2020-10-18 14:49:19 +00:00
|
|
|
Field buckets_field = (*arguments[1].column)[0];
|
2018-02-26 01:27:33 +00:00
|
|
|
BucketsType num_buckets;
|
2018-02-23 21:22:52 +00:00
|
|
|
|
|
|
|
if (buckets_field.getType() == Field::Types::Int64)
|
2018-02-26 01:27:33 +00:00
|
|
|
num_buckets = checkBucketsRange(buckets_field.get<Int64>());
|
2018-02-23 21:22:52 +00:00
|
|
|
else if (buckets_field.getType() == Field::Types::UInt64)
|
2018-02-26 01:27:33 +00:00
|
|
|
num_buckets = checkBucketsRange(buckets_field.get<UInt64>());
|
2018-02-23 21:22:52 +00:00
|
|
|
else
|
2021-09-06 15:59:46 +00:00
|
|
|
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
|
|
|
"Illegal type {} of the second argument of function {}",
|
|
|
|
buckets_field.getTypeName(), getName());
|
2018-02-23 21:22:52 +00:00
|
|
|
|
2020-10-18 14:49:19 +00:00
|
|
|
const auto & hash_col = arguments[0].column;
|
|
|
|
const IDataType * hash_type = arguments[0].type.get();
|
2018-02-26 01:27:33 +00:00
|
|
|
auto res_col = ColumnVector<ResultType>::create();
|
|
|
|
|
2018-09-07 14:37:26 +00:00
|
|
|
WhichDataType which(hash_type);
|
|
|
|
|
|
|
|
if (which.isUInt8())
|
2018-05-07 02:07:47 +00:00
|
|
|
executeType<UInt8>(hash_col, num_buckets, res_col.get());
|
2018-09-07 14:37:26 +00:00
|
|
|
else if (which.isUInt16())
|
2018-05-07 02:07:47 +00:00
|
|
|
executeType<UInt16>(hash_col, num_buckets, res_col.get());
|
2018-09-07 14:37:26 +00:00
|
|
|
else if (which.isUInt32())
|
2018-05-07 02:07:47 +00:00
|
|
|
executeType<UInt32>(hash_col, num_buckets, res_col.get());
|
2018-09-07 14:37:26 +00:00
|
|
|
else if (which.isUInt64())
|
2018-05-07 02:07:47 +00:00
|
|
|
executeType<UInt64>(hash_col, num_buckets, res_col.get());
|
2018-09-07 14:37:26 +00:00
|
|
|
else if (which.isInt8())
|
2018-05-07 02:07:47 +00:00
|
|
|
executeType<Int8>(hash_col, num_buckets, res_col.get());
|
2018-09-07 14:37:26 +00:00
|
|
|
else if (which.isInt16())
|
2018-05-07 02:07:47 +00:00
|
|
|
executeType<Int16>(hash_col, num_buckets, res_col.get());
|
2018-09-07 14:37:26 +00:00
|
|
|
else if (which.isInt32())
|
2018-05-07 02:07:47 +00:00
|
|
|
executeType<Int32>(hash_col, num_buckets, res_col.get());
|
2018-09-07 14:37:26 +00:00
|
|
|
else if (which.isInt64())
|
2018-05-07 02:07:47 +00:00
|
|
|
executeType<Int64>(hash_col, num_buckets, res_col.get());
|
2018-02-23 21:22:52 +00:00
|
|
|
else
|
|
|
|
throw Exception("Illegal type " + hash_type->getName() + " of the first argument of function " + getName(),
|
2018-05-07 02:07:47 +00:00
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2018-02-23 21:22:52 +00:00
|
|
|
|
2020-10-18 14:49:19 +00:00
|
|
|
return res_col;
|
2018-02-26 01:27:33 +00:00
|
|
|
}
|
2018-02-23 21:22:52 +00:00
|
|
|
|
2018-02-26 01:27:33 +00:00
|
|
|
template <typename CurrentHashType>
|
2020-07-21 13:58:07 +00:00
|
|
|
void executeType(const ColumnPtr & col_hash_ptr, BucketsType num_buckets, ColumnVector<ResultType> * col_result) const
|
2018-02-23 21:22:52 +00:00
|
|
|
{
|
2018-02-26 01:27:33 +00:00
|
|
|
auto col_hash = checkAndGetColumn<ColumnVector<CurrentHashType>>(col_hash_ptr.get());
|
2018-02-23 21:22:52 +00:00
|
|
|
if (!col_hash)
|
|
|
|
throw Exception("Illegal type of the first argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2018-05-07 02:07:47 +00:00
|
|
|
|
2018-02-26 01:27:33 +00:00
|
|
|
auto & vec_result = col_result->getData();
|
2018-02-23 21:22:52 +00:00
|
|
|
const auto & vec_hash = col_hash->getData();
|
|
|
|
|
|
|
|
size_t size = vec_hash.size();
|
|
|
|
vec_result.resize(size);
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
2018-02-26 01:27:33 +00:00
|
|
|
vec_result[i] = Impl::apply(static_cast<HashType>(vec_hash[i]), num_buckets);
|
2018-02-23 21:22:52 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
}
|