From c8d6c94d24dc1a57cccf7b0cba6d091bd0dda719 Mon Sep 17 00:00:00 2001 From: HappenLee Date: Sun, 16 May 2021 21:53:45 -0500 Subject: [PATCH] Support hll function: hll_empty(), hll_hash(), hll_cardinality() (#36) --- be/src/exprs/hll_function.cpp | 8 +- be/src/exprs/hll_function.h | 3 + be/src/vec/CMakeLists.txt | 3 + be/src/vec/data_types/data_type_factory.hpp | 1 + be/src/vec/functions/abs.cpp | 1 + be/src/vec/functions/function_cast.h | 4 +- be/src/vec/functions/function_const.h | 32 +++++ .../functions/function_string_or_array_to_t.h | 117 ++++++++++++++++++ .../vec/functions/function_string_to_string.h | 97 +++++++++++++++ be/src/vec/functions/hll_cardinality.cpp | 49 ++++++++ be/src/vec/functions/hll_empty.cpp | 43 +++++++ be/src/vec/functions/hll_hash.cpp | 63 ++++++++++ .../vec/functions/simple_function_factory.h | 6 + .../org/apache/doris/catalog/FunctionSet.java | 17 +++ gensrc/script/doris_builtins_functions.py | 24 ++-- gensrc/script/gen_builtins_functions.py | 11 +- 16 files changed, 463 insertions(+), 16 deletions(-) create mode 100644 be/src/vec/functions/function_const.h create mode 100644 be/src/vec/functions/function_string_or_array_to_t.h create mode 100644 be/src/vec/functions/function_string_to_string.h create mode 100644 be/src/vec/functions/hll_cardinality.cpp create mode 100644 be/src/vec/functions/hll_empty.cpp create mode 100644 be/src/vec/functions/hll_hash.cpp diff --git a/be/src/exprs/hll_function.cpp b/be/src/exprs/hll_function.cpp index fe33c11830770f..af8b3e162a17aa 100644 --- a/be/src/exprs/hll_function.cpp +++ b/be/src/exprs/hll_function.cpp @@ -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); @@ -37,7 +41,8 @@ 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) { @@ -45,6 +50,7 @@ void HllFunctions::hll_init(FunctionContext*, StringVal* dst) { dst->len = sizeof(HyperLogLog); dst->ptr = (uint8_t*)new HyperLogLog(); } + StringVal HllFunctions::hll_empty(FunctionContext* ctx) { return AnyValUtil::from_string_temp(ctx, HyperLogLog::empty()); } diff --git a/be/src/exprs/hll_function.h b/be/src/exprs/hll_function.h index b1d8e2cf381151..a8f7ab0b65b95a 100644 --- a/be/src/exprs/hll_function.h +++ b/be/src/exprs/hll_function.h @@ -18,6 +18,7 @@ #ifndef DORIS_BE_SRC_QUERY_EXPRS_HLL_FUNCTION_H #define DORIS_BE_SRC_QUERY_EXPRS_HLL_FUNCTION_H +#include #include "udf/udf.h" namespace doris { @@ -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); diff --git a/be/src/vec/CMakeLists.txt b/be/src/vec/CMakeLists.txt index 54f4b2eaa2d176..6529fa68f79bae 100644 --- a/be/src/vec/CMakeLists.txt +++ b/be/src/vec/CMakeLists.txt @@ -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 diff --git a/be/src/vec/data_types/data_type_factory.hpp b/be/src/vec/data_types/data_type_factory.hpp index 6ceb7c8da9b7da..29a36a50423df0 100644 --- a/be/src/vec/data_types/data_type_factory.hpp +++ b/be/src/vec/data_types/data_type_factory.hpp @@ -46,6 +46,7 @@ class DataTypeFactory { instance.regist_data_type("Int64", DataTypePtr(std::make_shared())); instance.regist_data_type("Float32", DataTypePtr(std::make_shared())); instance.regist_data_type("Float64", DataTypePtr(std::make_shared())); + instance.regist_data_type("String", DataTypePtr(std::make_shared())); }); return instance; } diff --git a/be/src/vec/functions/abs.cpp b/be/src/vec/functions/abs.cpp index dfea3c43f0d52c..d138c7619879fe 100644 --- a/be/src/vec/functions/abs.cpp +++ b/be/src/vec/functions/abs.cpp @@ -47,6 +47,7 @@ struct AbsImpl { struct NameAbs { static constexpr auto name = "abs"; }; + using FunctionAbs = FunctionUnaryArithmetic; //template <> struct FunctionUnaryArithmeticMonotonicity diff --git a/be/src/vec/functions/function_cast.h b/be/src/vec/functions/function_cast.h index 406087886b6e9b..c24a025aae4200 100644 --- a/be/src/vec/functions/function_cast.h +++ b/be/src/vec/functions/function_cast.h @@ -953,8 +953,8 @@ class FunctionCast final : public IFunctionBase { if (callOnIndexAndDataType(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(to_type.get())->getN()); diff --git a/be/src/vec/functions/function_const.h b/be/src/vec/functions/function_const.h new file mode 100644 index 00000000000000..d956b35a673ecb --- /dev/null +++ b/be/src/vec/functions/function_const.h @@ -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 +class FunctionConst : public IFunction { +public: + static constexpr auto name = Impl::name; + + static FunctionPtr create() { return std::make_shared(); } + +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()); + } +}; + +} diff --git a/be/src/vec/functions/function_string_or_array_to_t.h b/be/src/vec/functions/function_string_or_array_to_t.h new file mode 100644 index 00000000000000..571795eb410d49 --- /dev/null +++ b/be/src/vec/functions/function_string_or_array_to_t.h @@ -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 +//#include + +namespace doris::vectorized +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +template +class FunctionStringOrArrayToT : public IFunction +{ +public: + static constexpr auto name = Name::name; +// static FunctionPtr create(const Context &) + static FunctionPtr create() { + return std::make_shared(); + } + + 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>(); + } + + 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(column.get())) + { + auto col_res = ColumnVector::create(); + + typename ColumnVector::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(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::create(); +// +// typename ColumnVector::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(column.get())) +// { +// auto col_res = ColumnVector::create(); +// +// typename ColumnVector::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); + } +}; + +} \ No newline at end of file diff --git a/be/src/vec/functions/function_string_to_string.h b/be/src/vec/functions/function_string_to_string.h new file mode 100644 index 00000000000000..75d202edc4534b --- /dev/null +++ b/be/src/vec/functions/function_string_to_string.h @@ -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 +//#include + +namespace doris::vectorized +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +template +class FunctionStringToString : public IFunction +{ +public: + static constexpr auto name = Name::name; +// static FunctionPtr create(const Context &) + static FunctionPtr create() + { + return std::make_shared(); + } + + 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(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(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); + } +}; + +} \ No newline at end of file diff --git a/be/src/vec/functions/hll_cardinality.cpp b/be/src/vec/functions/hll_cardinality.cpp new file mode 100644 index 00000000000000..260f85e4a37e96 --- /dev/null +++ b/be/src/vec/functions/hll_cardinality.cpp @@ -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 +#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& 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; + +void registerFunctionHLLCardinality(SimpleFunctionFactory& factory) { + factory.registerFunction(); +} + +} diff --git a/be/src/vec/functions/hll_empty.cpp b/be/src/vec/functions/hll_empty.cpp new file mode 100644 index 00000000000000..9359d4f8b5b3b3 --- /dev/null +++ b/be/src/vec/functions/hll_empty.cpp @@ -0,0 +1,43 @@ +// 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 +#include "exprs/hll_function.h" +#include "olap/hll.h" + +#include "vec/data_types/data_type_string.h" +#include "vec/functions/function_const.h" +#include "vec/functions/simple_function_factory.h" + +namespace doris::vectorized { + +struct HLLEmptyImpl { + static constexpr auto name = "hll_empty"; + static auto get_return_type() { return std::make_shared();} + static Field init_value() { + auto hll = HyperLogLog::empty(); + return {hll.c_str(), hll.size()}; + } +}; + +using FunctionHLLEmpty = FunctionConst; + +void registerFunctionHLLEmpty(SimpleFunctionFactory& factory) { + factory.registerFunction(); +} + +} \ No newline at end of file diff --git a/be/src/vec/functions/hll_hash.cpp b/be/src/vec/functions/hll_hash.cpp new file mode 100644 index 00000000000000..6938dc148cc376 --- /dev/null +++ b/be/src/vec/functions/hll_hash.cpp @@ -0,0 +1,63 @@ + +// 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 +#include "exprs/hll_function.h" +#include "udf/udf.h" + +#include "vec/functions/function_string_to_string.h" +#include "vec/functions/simple_function_factory.h" + +namespace doris::vectorized { + +struct HLLHashImpl { + static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { + size_t size = offsets.size(); + res_offsets.resize(size); + res_data.reserve(data.size()); + + size_t prev_offset = 0; + size_t res_offset = 0; + + for (size_t i = 0; i < size; ++i) { + auto hash_string = HllFunctions::hll_hash(StringVal((uint8_t*)(&data[prev_offset]), + offsets[i] - prev_offset - 1)); + + res_data.resize(res_data.size() + hash_string.length() + 1); + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], hash_string.c_str(), hash_string.length()); + res_offset += hash_string.length() + 1; + res_data[res_offset - 1] = 0; + + res_offsets[i] = res_offset; + prev_offset = offsets[i]; + } + } +}; + +struct NameHLLHash { + static constexpr auto name = "hll_hash"; +}; + +using FunctionHLLHash = FunctionStringToString; + +void registerFunctionHLLHash(SimpleFunctionFactory& factory) { + factory.registerFunction(); +} + +} \ No newline at end of file diff --git a/be/src/vec/functions/simple_function_factory.h b/be/src/vec/functions/simple_function_factory.h index f889be03c9f630..dc7edf32be059b 100644 --- a/be/src/vec/functions/simple_function_factory.h +++ b/be/src/vec/functions/simple_function_factory.h @@ -27,6 +27,9 @@ class SimpleFunctionFactory; void registerFunctionComparison(SimpleFunctionFactory& factory); void registerFunctionAbs(SimpleFunctionFactory& factory); +void registerFunctionHLLCardinality(SimpleFunctionFactory& factory); +void registerFunctionHLLEmpty(SimpleFunctionFactory& factory); +void registerFunctionHLLHash(SimpleFunctionFactory& factory); void registerFunctionLogical(SimpleFunctionFactory& factory); void registerFunctionCast(SimpleFunctionFactory& factory); void registerFunctionPlus(SimpleFunctionFactory& factory); @@ -73,6 +76,9 @@ class SimpleFunctionFactory { static SimpleFunctionFactory instance; std::call_once(oc, [&]() { registerFunctionAbs(instance); + registerFunctionHLLCardinality(instance); + registerFunctionHLLEmpty(instance); + registerFunctionHLLHash(instance); registerFunctionComparison(instance); registerFunctionLogical(instance); registerFunctionCast(instance); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java index 4c43a1d5a9e32d..1040ac7b070343 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java @@ -990,6 +990,11 @@ public void addScalarBuiltin(String fnName, String symbol, boolean userVisible, addScalarBuiltin(fnName, symbol, userVisible, null, null, varArgs, retType, args); } + public void addScalarAndVectorizedBuiltin(String fnName, String symbol, boolean userVisible, + boolean varArgs, PrimitiveType retType, PrimitiveType ... args) { + addScalarAndVectorizedBuiltin(fnName, symbol, userVisible, null, null, varArgs, retType, args); + } + /** * Add a builtin with the specified name and signatures to this db. */ @@ -1005,6 +1010,18 @@ public void addScalarBuiltin(String fnName, String symbol, boolean userVisible, symbol, prepareFnSymbol, closeFnSymbol, userVisible)); } + private void addScalarAndVectorizedBuiltin(String fnName, String symbol, boolean userVisible, + String prepareFnSymbol, String closeFnSymbol, boolean varArgs, + PrimitiveType retType, PrimitiveType ... args) { + ArrayList argsType = new ArrayList(); + for (PrimitiveType type : args) { + argsType.add(Type.fromPrimitiveType(type)); + } + addBuiltinBothScalaAndVectorized(ScalarFunction.createBuiltin( + fnName, argsType, varArgs, Type.fromPrimitiveType(retType), + symbol, prepareFnSymbol, closeFnSymbol, userVisible)); + } + /** * Adds a builtin to this database. The function must not already exist. */ diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py index 7979224b50b349..accdcf80e5cc3a 100755 --- a/gensrc/script/doris_builtins_functions.py +++ b/gensrc/script/doris_builtins_functions.py @@ -405,21 +405,23 @@ '_ZN5doris13MathFunctions1eEPN9doris_udf15FunctionContextE'], [['abs'], 'DOUBLE', ['DOUBLE'], - '_ZN5doris13MathFunctions3absEPN9doris_udf15FunctionContextERKNS1_9DoubleValE'], + '_ZN5doris13MathFunctions3absEPN9doris_udf15FunctionContextERKNS1_9DoubleValE', 'vec'], [['abs'], 'FLOAT', ['FLOAT'], - '_ZN5doris13MathFunctions3absEPN9doris_udf15FunctionContextERKNS1_8FloatValE'], + '_ZN5doris13MathFunctions3absEPN9doris_udf15FunctionContextERKNS1_8FloatValE', 'vec'], [['abs'], 'LARGEINT', ['LARGEINT'], - '_ZN5doris13MathFunctions3absEPN9doris_udf15FunctionContextERKNS1_11LargeIntValE'], + '_ZN5doris13MathFunctions3absEPN9doris_udf15FunctionContextERKNS1_11LargeIntValE', 'vec'], [['abs'], 'LARGEINT', ['BIGINT'], - '_ZN5doris13MathFunctions3absEPN9doris_udf15FunctionContextERKNS1_9BigIntValE'], + '_ZN5doris13MathFunctions3absEPN9doris_udf15FunctionContextERKNS1_9BigIntValE', 'vec'], [['abs'], 'INT', ['SMALLINT'], - '_ZN5doris13MathFunctions3absEPN9doris_udf15FunctionContextERKNS1_11SmallIntValE'], + '_ZN5doris13MathFunctions3absEPN9doris_udf15FunctionContextERKNS1_11SmallIntValE', 'vec'], [['abs'], 'BIGINT', ['INT'], - '_ZN5doris13MathFunctions3absEPN9doris_udf15FunctionContextERKNS1_6IntValE'], + '_ZN5doris13MathFunctions3absEPN9doris_udf15FunctionContextERKNS1_6IntValE', 'vec'], [['abs'], 'SMALLINT', ['TINYINT'], - '_ZN5doris13MathFunctions3absEPN9doris_udf15FunctionContextERKNS1_10TinyIntValE'], + '_ZN5doris13MathFunctions3absEPN9doris_udf15FunctionContextERKNS1_10TinyIntValE', 'vec'], + [['abs'], 'DECIMAL', ['DECIMAL'], + '_ZN5doris13MathFunctions3absEPN9doris_udf15FunctionContextERKNS1_10DecimalValE', 'vec'], [['abs'], 'DECIMALV2', ['DECIMALV2'], - '_ZN5doris13MathFunctions3absEPN9doris_udf15FunctionContextERKNS1_12DecimalV2ValE'], + '_ZN5doris13MathFunctions3absEPN9doris_udf15FunctionContextERKNS1_12DecimalV2ValE', 'vec'], [['sign'], 'FLOAT', ['DOUBLE'], '_ZN5doris13MathFunctions4signEPN9doris_udf15FunctionContextERKNS1_9DoubleValE'], @@ -788,11 +790,11 @@ #hll function [['hll_cardinality'], 'BIGINT', ['VARCHAR'], - '_ZN5doris12HllFunctions15hll_cardinalityEPN9doris_udf15FunctionContextERKNS1_9StringValE'], + '_ZN5doris12HllFunctions15hll_cardinalityEPN9doris_udf15FunctionContextERKNS1_9StringValE', 'vec'], [['hll_hash'], 'HLL', ['VARCHAR'], - '_ZN5doris12HllFunctions8hll_hashEPN9doris_udf15FunctionContextERKNS1_9StringValE'], + '_ZN5doris12HllFunctions8hll_hashEPN9doris_udf15FunctionContextERKNS1_9StringValE', 'vec'], [['hll_empty'], 'HLL', [], - '_ZN5doris12HllFunctions9hll_emptyEPN9doris_udf15FunctionContextE'], + '_ZN5doris12HllFunctions9hll_emptyEPN9doris_udf15FunctionContextE', 'vec'], #bitmap function diff --git a/gensrc/script/gen_builtins_functions.py b/gensrc/script/gen_builtins_functions.py index 7e45eed53cc2c7..47ebc42140757b 100755 --- a/gensrc/script/gen_builtins_functions.py +++ b/gensrc/script/gen_builtins_functions.py @@ -67,7 +67,10 @@ def add_function(fn_meta_data, user_visible): entry["args"] = fn_meta_data[2] entry["symbol"] = fn_meta_data[3] if len(fn_meta_data) >= 5: - entry["prepare"] = fn_meta_data[4] + if fn_meta_data[4] != "vec": + entry["prepare"] = fn_meta_data[4] + else: + entry["vec"] = True if len(fn_meta_data) >= 6: entry["close"] = fn_meta_data[5] entry["user_visible"] = user_visible @@ -114,7 +117,11 @@ def generate_fe_registry_init(filename): for entry in meta_data_entries: for name in entry["sql_names"]: java_output = generate_fe_entry(entry, name) - java_registry_file.write(" functionSet.addScalarBuiltin(%s);\n" % java_output) + if ("vec" not in entry): + java_registry_file.write(" functionSet.addScalarBuiltin(%s);\n" % java_output) + else: + java_registry_file.write(" functionSet.addScalarAndVectorizedBuiltin(%s);\n" % java_output) + java_registry_file.write("\n")