Skip to content

Commit

Permalink
Merge pull request #37013 from mnutt/hashid
Browse files Browse the repository at this point in the history
Add hashid support
  • Loading branch information
yakov-olkhovskiy committed May 21, 2022
2 parents ec2a1c8 + 23dbf1b commit d878f19
Show file tree
Hide file tree
Showing 12 changed files with 222 additions and 0 deletions.
4 changes: 4 additions & 0 deletions .gitmodules
Expand Up @@ -268,3 +268,7 @@
[submodule "contrib/eigen"]
path = contrib/eigen
url = https://github.com/eigen-mirror/eigen
[submodule "contrib/hashidsxx"]
path = contrib/hashidsxx
url = https://github.com/schoentoon/hashidsxx.git

1 change: 1 addition & 0 deletions contrib/CMakeLists.txt
Expand Up @@ -140,6 +140,7 @@ add_contrib (libpq-cmake libpq)
add_contrib (nuraft-cmake NuRaft)
add_contrib (fast_float-cmake fast_float)
add_contrib (datasketches-cpp-cmake datasketches-cpp)
add_contrib (hashidsxx-cmake hashidsxx)

option(ENABLE_NLP "Enable NLP functions support" ${ENABLE_LIBRARIES})
if (ENABLE_NLP)
Expand Down
1 change: 1 addition & 0 deletions contrib/hashidsxx
Submodule hashidsxx added at 783f69
14 changes: 14 additions & 0 deletions contrib/hashidsxx-cmake/CMakeLists.txt
@@ -0,0 +1,14 @@
set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/hashidsxx")

set (SRCS
"${LIBRARY_DIR}/hashids.cpp"
)

set (HDRS
"${LIBRARY_DIR}/hashids.h"
)

add_library(_hashidsxx ${SRCS} ${HDRS})
target_include_directories(_hashidsxx SYSTEM PUBLIC "${LIBRARY_DIR}")

add_library(ch_contrib::hashidsxx ALIAS _hashidsxx)
1 change: 1 addition & 0 deletions docker/test/fasttest/run.sh
Expand Up @@ -178,6 +178,7 @@ function clone_submodules
contrib/replxx
contrib/wyhash
contrib/eigen
contrib/hashidsxx
)

git submodule sync
Expand Down
1 change: 1 addition & 0 deletions src/Core/Settings.h
Expand Up @@ -586,6 +586,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
/** Experimental functions */ \
M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \
M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \
M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions (hashid, etc)", 0) \
M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \
M(String, insert_deduplication_token, "", "If not empty, used for duplicate detection instead of data digest", 0) \
M(Bool, count_distinct_optimization, false, "Rewrite count distinct to subquery of group by", 0) \
Expand Down
1 change: 1 addition & 0 deletions src/Functions/CMakeLists.txt
Expand Up @@ -23,6 +23,7 @@ target_link_libraries(clickhouse_functions
dbms
ch_contrib::metrohash
ch_contrib::murmurhash
ch_contrib::hashidsxx

PRIVATE
ch_contrib::zlib
Expand Down
12 changes: 12 additions & 0 deletions src/Functions/FunctionHashID.cpp
@@ -0,0 +1,12 @@
#include "FunctionHashID.h"
#include <Functions/FunctionFactory.h>

namespace DB
{

void registerFunctionHashID(FunctionFactory & factory)
{
factory.registerFunction<FunctionHashID>();
}

}
169 changes: 169 additions & 0 deletions src/Functions/FunctionHashID.h
@@ -0,0 +1,169 @@
#pragma once

#include <Common/config.h>

#include <hashids.h>

#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <Interpreters/Context.h>

#include <functional>
#include <initializer_list>

namespace DB
{

namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int SUPPORT_IS_DISABLED;
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
}

// hashid(string, salt)
class FunctionHashID : public IFunction
{
public:
static constexpr auto name = "hashid";

static FunctionPtr create(ContextPtr context)
{
if (!context->getSettingsRef().allow_experimental_hash_functions)
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
"Hashing function '{}' is experimental. Set `allow_experimental_hash_functions` setting to enable it", name);

return std::make_shared<FunctionHashID>();
}

String getName() const override { return name; }

size_t getNumberOfArguments() const override { return 0; }

bool isVariadic() const override { return true; }

bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }

DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.size() < 1)
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} expects at least one argument", getName());

const auto & id_col = arguments[0];
if (!isUnsignedInteger(id_col.type))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"First argument of function {} must be unsigned integer, got {}",
getName(),
arguments[0].type->getName());

if (arguments.size() > 1)
{
const auto & hash_col = arguments[1];
if (!isString(hash_col.type) || !isColumnConst(*hash_col.column.get()))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Second argument of function {} must be String, got {}",
getName(),
arguments[1].type->getName());
}

if (arguments.size() > 2)
{
const auto & min_length_col = arguments[2];
if (!isUInt8(min_length_col.type) || !isColumnConst(*min_length_col.column.get()))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Third argument of function {} must be UInt8, got {}",
getName(),
arguments[2].type->getName());
}

if (arguments.size() > 3)
{
const auto & alphabet_col = arguments[3];
if (!isString(alphabet_col.type) || !isColumnConst(*alphabet_col.column.get()))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Fourth argument of function {} must be String, got {}",
getName(),
arguments[3].type->getName());
}

if (arguments.size() > 4)
{
throw Exception(
ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION,
"Function {} expect no more than four arguments (integer, salt, min_length, optional_alphabet), got {}",
getName(),
arguments.size());
}

return std::make_shared<DataTypeString>();
}

ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const auto & numcolumn = arguments[0].column;

if (checkAndGetColumn<ColumnUInt8>(numcolumn.get()) || checkAndGetColumn<ColumnUInt16>(numcolumn.get())
|| checkAndGetColumn<ColumnUInt32>(numcolumn.get()) || checkAndGetColumn<ColumnUInt64>(numcolumn.get())
|| checkAndGetColumnConst<ColumnUInt8>(numcolumn.get()) || checkAndGetColumnConst<ColumnUInt16>(numcolumn.get())
|| checkAndGetColumnConst<ColumnUInt32>(numcolumn.get()) || checkAndGetColumnConst<ColumnUInt64>(numcolumn.get()))
{
std::string salt;
UInt8 minLength = 0;
std::string alphabet;

if (arguments.size() >= 4)
{
const auto & alphabetcolumn = arguments[3].column;
if (auto alpha_col = checkAndGetColumnConst<ColumnString>(alphabetcolumn.get()))
{
alphabet = alpha_col->getValue<String>();
if (alphabet.find('\0') != std::string::npos)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Custom alphabet must not contain null character");
}
}
else
alphabet.assign(DEFAULT_ALPHABET);

if (arguments.size() >= 3)
{
const auto & minlengthcolumn = arguments[2].column;
if (auto min_length_col = checkAndGetColumnConst<ColumnUInt8>(minlengthcolumn.get()))
minLength = min_length_col->getValue<UInt8>();
}

if (arguments.size() >= 2)
{
const auto & saltcolumn = arguments[1].column;
if (auto salt_col = checkAndGetColumnConst<ColumnString>(saltcolumn.get()))
salt = salt_col->getValue<String>();
}

hashidsxx::Hashids hash(salt, minLength, alphabet);

auto col_res = ColumnString::create();

for (size_t i = 0; i < input_rows_count; ++i)
{
col_res->insert(hash.encode({numcolumn->getUInt(i)}));
}

return col_res;
}
else
throw Exception(
"Illegal column " + arguments[0].column->getName() + " of first argument of function hashid", ErrorCodes::ILLEGAL_COLUMN);
}
};

}
2 changes: 2 additions & 0 deletions src/Functions/registerFunctions.cpp
Expand Up @@ -24,6 +24,7 @@ void registerFunctionsEmbeddedDictionaries(FunctionFactory &);
void registerFunctionsExternalDictionaries(FunctionFactory &);
void registerFunctionsExternalModels(FunctionFactory &);
void registerFunctionsFormatting(FunctionFactory &);
void registerFunctionHashID(FunctionFactory &);
void registerFunctionsHashing(FunctionFactory &);
void registerFunctionsHigherOrder(FunctionFactory &);
void registerFunctionsLogical(FunctionFactory &);
Expand Down Expand Up @@ -137,6 +138,7 @@ void registerFunctions()
#endif
registerFunctionTid(factory);
registerFunctionLogTrace(factory);
registerFunctionHashID(factory);
}

}
11 changes: 11 additions & 0 deletions tests/queries/0_stateless/02293_hashid.reference
@@ -0,0 +1,11 @@
0 gY
1 jR
2 k5
3 l5
4 mO
0 pbgkmdljlpjoapne
1 akemglnjepjpodba
2 obmgndljgajpkeao
3 dldokmpjpgjgeanb
4 nkdlpgajngjnobme
YQrvD5XGvbx
5 changes: 5 additions & 0 deletions tests/queries/0_stateless/02293_hashid.sql
@@ -0,0 +1,5 @@
SET allow_experimental_hash_functions = 1;

select number, hashid(number) from system.numbers limit 5;
select number, hashid(number, 's3cr3t', 16, 'abcdefghijklmnop') from system.numbers limit 5;
select hashid(1234567890123456, 's3cr3t');

0 comments on commit d878f19

Please sign in to comment.