Skip to content

Commit

Permalink
Support hll function: hll_empty(), hll_hash(), hll_cardinality() (apa…
Browse files Browse the repository at this point in the history
  • Loading branch information
HappenLee committed Aug 10, 2021
1 parent 344f75d commit c8d6c94
Show file tree
Hide file tree
Showing 16 changed files with 463 additions and 16 deletions.
8 changes: 7 additions & 1 deletion be/src/exprs/hll_function.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,10 @@ using doris_udf::StringVal;
void HllFunctions::init() {}

StringVal HllFunctions::hll_hash(FunctionContext* ctx, const StringVal& input) {
return AnyValUtil::from_string_temp(ctx, hll_hash(input));
}

std::string HllFunctions::hll_hash(const StringVal& input) {
HyperLogLog hll;
if (!input.is_null) {
uint64_t hash_value = HashUtil::murmur_hash64A(input.ptr, input.len, HashUtil::MURMUR_SEED);
Expand All @@ -37,14 +41,16 @@ StringVal HllFunctions::hll_hash(FunctionContext* ctx, const StringVal& input) {
std::string buf;
buf.resize(hll.max_serialized_size());
buf.resize(hll.serialize((uint8_t*)buf.c_str()));
return AnyValUtil::from_string_temp(ctx, buf);

return buf;
}

void HllFunctions::hll_init(FunctionContext*, StringVal* dst) {
dst->is_null = false;
dst->len = sizeof(HyperLogLog);
dst->ptr = (uint8_t*)new HyperLogLog();
}

StringVal HllFunctions::hll_empty(FunctionContext* ctx) {
return AnyValUtil::from_string_temp(ctx, HyperLogLog::empty());
}
Expand Down
3 changes: 3 additions & 0 deletions be/src/exprs/hll_function.h
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
#ifndef DORIS_BE_SRC_QUERY_EXPRS_HLL_FUNCTION_H
#define DORIS_BE_SRC_QUERY_EXPRS_HLL_FUNCTION_H

#include <string>
#include "udf/udf.h"

namespace doris {
Expand All @@ -26,6 +27,8 @@ class HllFunctions {
public:
static void init();
static StringVal hll_hash(FunctionContext* ctx, const StringVal& dest_base);
static std::string hll_hash(const StringVal& dest_base);

static StringVal hll_empty(FunctionContext* ctx);
static void hll_init(FunctionContext*, StringVal* dst);

Expand Down
3 changes: 3 additions & 0 deletions be/src/vec/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,9 @@ set(VEC_FILES
functions/function_helpers.cpp
functions/functions_logical.cpp
functions/function_cast.cpp
functions/hll_cardinality.cpp
functions/hll_empty.cpp
functions/hll_hash.cpp
functions/plus.cpp
functions/modulo.cpp
functions/multiply.cpp
Expand Down
1 change: 1 addition & 0 deletions be/src/vec/data_types/data_type_factory.hpp
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ class DataTypeFactory {
instance.regist_data_type("Int64", DataTypePtr(std::make_shared<DataTypeInt64>()));
instance.regist_data_type("Float32", DataTypePtr(std::make_shared<DataTypeFloat32>()));
instance.regist_data_type("Float64", DataTypePtr(std::make_shared<DataTypeFloat64>()));
instance.regist_data_type("String", DataTypePtr(std::make_shared<DataTypeString>()));
});
return instance;
}
Expand Down
1 change: 1 addition & 0 deletions be/src/vec/functions/abs.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ struct AbsImpl {
struct NameAbs {
static constexpr auto name = "abs";
};

using FunctionAbs = FunctionUnaryArithmetic<AbsImpl, NameAbs, false>;

//template <> struct FunctionUnaryArithmeticMonotonicity<NameAbs>
Expand Down
4 changes: 2 additions & 2 deletions be/src/vec/functions/function_cast.h
Original file line number Diff line number Diff line change
Expand Up @@ -953,8 +953,8 @@ class FunctionCast final : public IFunctionBase {
if (callOnIndexAndDataType<void>(to_type->getTypeId(), make_default_wrapper)) return ret;

switch (to_type->getTypeId()) {
// case TypeIndex::String:
// return createStringWrapper(from_type);
// case TypeIndex::String:
// return createStringWrapper(from_type);
// case TypeIndex::FixedString:
// return createFixedStringWrapper(from_type, checkAndGetDataType<DataTypeFixedString>(to_type.get())->getN());

Expand Down
32 changes: 32 additions & 0 deletions be/src/vec/functions/function_const.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
#pragma once

#include "vec/data_types/data_types_number.h"
#include "vec/columns/columns_number.h"
#include "vec/functions/function.h"


namespace doris::vectorized {

template<typename Impl>
class FunctionConst : public IFunction {
public:
static constexpr auto name = Impl::name;

static FunctionPtr create() { return std::make_shared<FunctionConst>(); }

private:
String getName() const override { return name; }

size_t getNumberOfArguments() const override { return 0; }

DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override {
return Impl::get_return_type();
}

void executeImpl(Block &block, const ColumnNumbers &, size_t result, size_t input_rows_count) override {
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(input_rows_count,
Impl::init_value());
}
};

}
117 changes: 117 additions & 0 deletions be/src/vec/functions/function_string_or_array_to_t.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,117 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

#include "vec/data_types/data_type_string.h"
#include "vec/data_types/data_types_number.h"
#include "vec/functions/function.h"
#include "vec/functions/function_helpers.h"
#include "vec/columns/column_vector.h"
#include "vec/columns/column_string.h"
//#include<vec/columns/column_fixed_string.h>
//#include<vec/columns/column_array.h>

namespace doris::vectorized
{

namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}


template <typename Impl, typename Name, typename ResultType>
class FunctionStringOrArrayToT : public IFunction
{
public:
static constexpr auto name = Name::name;
// static FunctionPtr create(const Context &)
static FunctionPtr create() {
return std::make_shared<FunctionStringOrArrayToT>();
}

String getName() const override
{
return name;
}

size_t getNumberOfArguments() const override
{
return 1;
}

DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isStringOrFixedString(arguments[0])
&& !isArray(arguments[0]))
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

return std::make_shared<DataTypeNumber<ResultType>>();
}

bool useDefaultImplementationForConstants() const override { return true; }

void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnPtr column = block.getByPosition(arguments[0]).column;
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
{
auto col_res = ColumnVector<ResultType>::create();

typename ColumnVector<ResultType>::Container & vec_res = col_res->getData();
vec_res.resize(col->size());
Impl::vector(col->getChars(), col->getOffsets(), vec_res);

block.getByPosition(result).column = std::move(col_res);
}
// else if (const ColumnFixedString * col_fixed = checkAndGetColumn<ColumnFixedString>(column.get()))
// {
// if (Impl::is_fixed_to_constant)
// {
// ResultType res = 0;
// Impl::vector_fixed_to_constant(col_fixed->getChars(), col_fixed->getN(), res);
//
// block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(col_fixed->size(), toField(res));
// }
// else
// {
// auto col_res = ColumnVector<ResultType>::create();
//
// typename ColumnVector<ResultType>::Container & vec_res = col_res->getData();
// vec_res.resize(col_fixed->size());
// Impl::vector_fixed_to_vector(col_fixed->getChars(), col_fixed->getN(), vec_res);
//
// block.getByPosition(result).column = std::move(col_res);
// }
// }
// else if (const ColumnArray * col_arr = checkAndGetColumn<ColumnArray>(column.get()))
// {
// auto col_res = ColumnVector<ResultType>::create();
//
// typename ColumnVector<ResultType>::Container & vec_res = col_res->getData();
// vec_res.resize(col_arr->size());
// Impl::array(col_arr->getOffsets(), vec_res);
//
// block.getByPosition(result).column = std::move(col_res);
// }
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
};

}
97 changes: 97 additions & 0 deletions be/src/vec/functions/function_string_to_string.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,97 @@

// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

#include "vec/data_types/data_type_string.h"
#include "vec/data_types/data_types_number.h"
#include "vec/functions/function.h"
#include "vec/functions/function_helpers.h"
#include "vec/columns/column_vector.h"
#include "vec/columns/column_string.h"
//#include<vec/columns/column_fixed_string.h>
//#include<vec/columns/column_array.h>

namespace doris::vectorized
{

namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}


template <typename Impl, typename Name, bool is_injective = false>
class FunctionStringToString : public IFunction
{
public:
static constexpr auto name = Name::name;
// static FunctionPtr create(const Context &)
static FunctionPtr create()
{
return std::make_shared<FunctionStringToString>();
}

String getName() const override
{
return name;
}

size_t getNumberOfArguments() const override
{
return 1;
}

bool isInjective(const Block &) override
{
return is_injective;
}

DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isStringOrFixedString(arguments[0]))
throw Exception(
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

return arguments[0];
}

bool useDefaultImplementationForConstants() const override { return true; }

void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnPtr column = block.getByPosition(arguments[0]).column;
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
{
auto col_res = ColumnString::create();
Impl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets());
block.getByPosition(result).column = std::move(col_res);
}
// else if (const ColumnFixedString * col_fixed = checkAndGetColumn<ColumnFixedString>(column.get()))
// {
// auto col_res = ColumnFixedString::create(col_fixed->getN());
// Impl::vector_fixed(col_fixed->getChars(), col_fixed->getN(), col_res->getChars());
// block.getByPosition(result).column = std::move(col_res);
// }
else
throw Exception(
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
};

}
49 changes: 49 additions & 0 deletions be/src/vec/functions/hll_cardinality.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,49 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

//#include <Functions/FunctionFactory.h>
#include "exprs/hll_function.h"
#include "udf/udf.h"

#include "vec/data_types/number_traits.h"
#include "vec/functions/function_string_or_array_to_t.h"
#include "vec/functions/simple_function_factory.h"

namespace doris::vectorized {

struct HLLCardinalityImpl {
static void vector(const ColumnString::Chars& data, const ColumnString::Offsets& offsets, PaddedPODArray<Int64>& res) {
auto size = res.size();
for (int i = 0; i < size; ++i) {
auto val = HllFunctions::hll_cardinality(nullptr,
StringVal((uint8_t*)&data[offsets[i - 1]], offsets[i] - offsets[i - 1] - 1));
res[i] = val.val;
}
}
};

struct NameHLLCardinality {
static constexpr auto name = "hll_cardinality";
};

using FunctionHLLCardinality = FunctionStringOrArrayToT<HLLCardinalityImpl, NameHLLCardinality, Int64>;

void registerFunctionHLLCardinality(SimpleFunctionFactory& factory) {
factory.registerFunction<FunctionHLLCardinality>();
}

}
Loading

0 comments on commit c8d6c94

Please sign in to comment.