ClickHouse/src/Functions/FunctionsStringHash.h

196 lines
8.1 KiB
C++
Raw Normal View History

2019-11-06 10:34:13 +00:00
#pragma once
#include <Columns/ColumnConst.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnVector.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypesNumber.h>
2020-12-25 15:19:30 +00:00
#include <DataTypes/DataTypeString.h>
2019-11-06 10:34:13 +00:00
#include <Functions/FunctionHelpers.h>
2021-05-17 07:30:42 +00:00
#include <Functions/IFunction.h>
#include <Interpreters/Context_fwd.h>
2019-11-06 10:34:13 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
2020-12-25 15:19:30 +00:00
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int ARGUMENT_OUT_OF_BOUND;
2019-11-06 10:34:13 +00:00
}
// FunctionStringHash
// Simhash: String -> UInt64
// Minhash: String -> (UInt64, UInt64)
2020-12-25 15:19:30 +00:00
template <typename Impl, typename Name, bool is_simhash, bool is_arg = false>
2019-11-06 10:34:13 +00:00
class FunctionsStringHash : public IFunction
{
public:
static constexpr auto name = Name::name;
2020-12-25 15:19:30 +00:00
static constexpr size_t default_shingle_size = 3;
static constexpr size_t default_num_hashes = 6;
2021-01-06 21:10:54 +00:00
static constexpr size_t max_shingle_size = 25;
static constexpr size_t max_num_hashes = 25;
2019-11-06 10:34:13 +00:00
2021-06-01 12:20:52 +00:00
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionsStringHash>(); }
2019-11-06 10:34:13 +00:00
String getName() const override { return name; }
2020-12-25 15:19:30 +00:00
size_t getNumberOfArguments() const override { return 0; }
bool isVariadic() const override { return true; }
2021-06-22 16:21:23 +00:00
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
2019-11-06 10:34:13 +00:00
2020-12-25 15:19:30 +00:00
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override
2019-11-06 10:34:13 +00:00
{
2020-12-25 15:19:30 +00:00
if constexpr (is_simhash)
return {1};
else
return {1, 2};
}
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.empty())
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} expect at least one argument", getName());
if (!isString(arguments[0].type))
2020-12-21 11:39:17 +00:00
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
2020-12-25 15:19:30 +00:00
"First argument of function {} must be String, got {}", getName(), arguments[0].type->getName());
size_t shingle_size = default_shingle_size;
if (arguments.size() > 1)
{
if (!isUnsignedInteger(arguments[1].type))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Second argument (shingle size) of function {} must be unsigned integer, got {}",
getName(), arguments[1].type->getName());
if (!arguments[1].column)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Second argument (shingle size) of function {} must be constant", getName());
shingle_size = arguments[1].column->getUInt(0);
}
size_t num_hashes = default_num_hashes;
if (arguments.size() > 2)
{
if constexpr (is_simhash)
throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION,
2020-12-29 10:16:22 +00:00
"Function {} expect no more than two arguments (text, shingle size), got {}",
2020-12-25 15:19:30 +00:00
getName(), arguments.size());
if (!isUnsignedInteger(arguments[2].type))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Third argument (num hashes) of function {} must be unsigned integer, got {}",
getName(), arguments[2].type->getName());
if (!arguments[2].column)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Third argument (num hashes) of function {} must be constant", getName());
num_hashes = arguments[2].column->getUInt(0);
}
if (arguments.size() > 3)
{
throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION,
2020-12-29 10:16:22 +00:00
"Function {} expect no more than three arguments (text, shingle size, num hashes), got {}",
2020-12-25 15:19:30 +00:00
getName(), arguments.size());
}
if (shingle_size == 0)
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Second argument (shingle size) of function {} cannot be zero", getName());
2020-12-25 15:19:30 +00:00
if (num_hashes == 0)
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Third argument (num hashes) of function {} cannot be zero", getName());
if (shingle_size > max_shingle_size)
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Second argument (shingle size) of function {} cannot be greater then {}", getName(), max_shingle_size);
if (num_hashes > max_num_hashes)
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Third argument (num hashes) of function {} cannot be greater then {}", getName(), max_num_hashes);
2020-12-21 11:39:17 +00:00
auto type = std::make_shared<DataTypeUInt64>();
2020-05-22 13:23:49 +00:00
if constexpr (is_simhash)
2020-12-21 11:39:17 +00:00
return type;
2020-12-25 15:19:30 +00:00
if constexpr (is_arg)
{
DataTypePtr string_type = std::make_shared<DataTypeString>();
DataTypes types(num_hashes, string_type);
auto tuple_type = std::make_shared<DataTypeTuple>(types);
return std::make_shared<DataTypeTuple>(DataTypes{tuple_type, tuple_type});
}
2020-12-21 11:39:17 +00:00
return std::make_shared<DataTypeTuple>(DataTypes{type, type});
2019-11-06 10:34:13 +00:00
}
2020-06-10 15:02:58 +00:00
bool useDefaultImplementationForConstants() const override { return true; }
2020-12-17 19:14:01 +00:00
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override
2019-11-06 10:34:13 +00:00
{
2020-12-17 19:14:01 +00:00
const ColumnPtr & column = arguments[0].column;
2020-12-21 11:39:17 +00:00
2020-12-25 15:19:30 +00:00
size_t shingle_size = default_shingle_size;
size_t num_hashes = default_num_hashes;
if (arguments.size() > 1)
shingle_size = arguments[1].column->getUInt(0);
if (arguments.size() > 2)
num_hashes = arguments[2].column->getUInt(0);
2020-05-22 13:23:49 +00:00
if constexpr (is_simhash)
2019-11-06 10:34:13 +00:00
{
2020-12-21 11:39:17 +00:00
auto col_res = ColumnVector<UInt64>::create();
auto & vec_res = col_res->getData();
2020-06-10 15:02:58 +00:00
vec_res.resize(column->size());
const ColumnString * col_str_vector = checkAndGetColumn<ColumnString>(&*column);
2020-12-25 15:19:30 +00:00
Impl::apply(col_str_vector->getChars(), col_str_vector->getOffsets(), shingle_size, vec_res);
2020-12-21 11:39:17 +00:00
return col_res;
2019-11-06 10:34:13 +00:00
}
2020-12-25 15:19:30 +00:00
else if constexpr (is_arg) // Min hash arg
{
MutableColumns min_columns(num_hashes);
MutableColumns max_columns(num_hashes);
for (size_t i = 0; i < num_hashes; ++i)
{
min_columns[i] = ColumnString::create();
max_columns[i] = ColumnString::create();
}
auto min_tuple = ColumnTuple::create(std::move(min_columns));
auto max_tuple = ColumnTuple::create(std::move(max_columns));
const ColumnString * col_str_vector = checkAndGetColumn<ColumnString>(&*column);
2020-12-25 17:02:19 +00:00
Impl::apply(col_str_vector->getChars(), col_str_vector->getOffsets(), shingle_size, num_hashes, nullptr, nullptr, min_tuple.get(), max_tuple.get());
2020-12-25 15:19:30 +00:00
MutableColumns tuple_columns;
tuple_columns.emplace_back(std::move(min_tuple));
tuple_columns.emplace_back(std::move(max_tuple));
return ColumnTuple::create(std::move(tuple_columns));
}
2019-11-06 10:34:13 +00:00
else // Min hash
{
2020-12-21 11:39:17 +00:00
auto col_h1 = ColumnVector<UInt64>::create();
auto col_h2 = ColumnVector<UInt64>::create();
auto & vec_h1 = col_h1->getData();
auto & vec_h2 = col_h2->getData();
2020-06-10 15:02:58 +00:00
vec_h1.resize(column->size());
vec_h2.resize(column->size());
const ColumnString * col_str_vector = checkAndGetColumn<ColumnString>(&*column);
2020-12-25 15:19:30 +00:00
Impl::apply(col_str_vector->getChars(), col_str_vector->getOffsets(), shingle_size, num_hashes, &vec_h1, &vec_h2, nullptr, nullptr);
2020-06-10 15:02:58 +00:00
MutableColumns tuple_columns;
tuple_columns.emplace_back(std::move(col_h1));
tuple_columns.emplace_back(std::move(col_h2));
2020-12-17 19:14:01 +00:00
return ColumnTuple::create(std::move(tuple_columns));
2019-11-06 10:34:13 +00:00
}
}
};
}