This is an automated email from the ASF dual-hosted git repository.

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/branch-2.1 by this push:
     new 9877a088341 [feature](function) support ngram_search function #38226 
(#40893)
9877a088341 is described below

commit 9877a088341edee678f6d9bec294648685d352d8
Author: Mryange <59914473+mrya...@users.noreply.github.com>
AuthorDate: Sat Sep 21 20:34:44 2024 +0800

    [feature](function) support ngram_search function #38226 (#40893)
    
    https://github.com/apache/doris/pull/38226
    mysql [test]>select ngram_search('123456789' , '12345' , 3);
    +---------------------------------------+
    | ngram_search('123456789', '12345', 3) |
    +---------------------------------------+
    |                                   0.6 |
    +---------------------------------------+
    1 row in set (0.01 sec)
    
    mysql [test]>select ngram_search("abababab","babababa",2);
    +-----------------------------------------+
    | ngram_search('abababab', 'babababa', 2) |
    +-----------------------------------------+
    |                                       1 |
    +-----------------------------------------+
    1 row in set (0.01 sec)
    ```
    
    doc https://github.com/apache/doris-website/pull/899
    
    ## Proposed changes
    
    Issue Number: close #xxx
    
    <!--Describe your changes.-->
---
 be/src/vec/columns/column_const.h                  |  10 ++
 be/src/vec/functions/function_string.cpp           |   1 +
 be/src/vec/functions/function_string.h             | 126 +++++++++++++++++++++
 .../doris/catalog/BuiltinScalarFunctions.java      |   2 +
 .../expressions/functions/scalar/NgramSearch.java  |  78 +++++++++++++
 .../expressions/visitor/ScalarFunctionVisitor.java |   5 +
 .../string_functions/test_string_function.out      | Bin 4244 -> 4589 bytes
 .../string_functions/test_string_function.groovy   |  45 +++++---
 8 files changed, 250 insertions(+), 17 deletions(-)

diff --git a/be/src/vec/columns/column_const.h 
b/be/src/vec/columns/column_const.h
index 14fa5d6b191..c3f571399bf 100644
--- a/be/src/vec/columns/column_const.h
+++ b/be/src/vec/columns/column_const.h
@@ -68,6 +68,16 @@ T index_check_const(T arg, bool constancy) noexcept {
     return constancy ? 0 : arg;
 }
 
+template <bool is_const, typename T>
+    requires std::is_integral_v<T>
+constexpr T index_check_const(T arg) noexcept {
+    if constexpr (is_const) {
+        return 0;
+    } else {
+        return arg;
+    }
+}
+
 /*
  * @return first : data_column_ptr for ColumnConst, itself otherwise.
  *         second : whether it's ColumnConst.
diff --git a/be/src/vec/functions/function_string.cpp 
b/be/src/vec/functions/function_string.cpp
index 673889a33e5..27462e36a02 100644
--- a/be/src/vec/functions/function_string.cpp
+++ b/be/src/vec/functions/function_string.cpp
@@ -1172,6 +1172,7 @@ void register_function_string(SimpleFunctionFactory& 
factory) {
     factory.register_function<FunctionSubReplace<SubReplaceThreeImpl>>();
     factory.register_function<FunctionSubReplace<SubReplaceFourImpl>>();
     factory.register_function<FunctionStrcmp>();
+    factory.register_function<FunctionNgramSearch>();
 
     /// @TEMPORARY: for be_exec_version=3
     
factory.register_alternative_function<FunctionSubstringOld<Substr3ImplOld>>();
diff --git a/be/src/vec/functions/function_string.h 
b/be/src/vec/functions/function_string.h
index c05e7deb56b..661fcaf56f6 100644
--- a/be/src/vec/functions/function_string.h
+++ b/be/src/vec/functions/function_string.h
@@ -57,6 +57,7 @@
 #include "vec/columns/column.h"
 #include "vec/columns/column_const.h"
 #include "vec/columns/column_vector.h"
+#include "vec/common/hash_table/phmap_fwd_decl.h"
 #include "vec/common/int_exp.h"
 #include "vec/common/memcmp_small.h"
 #include "vec/common/memcpy_small.h"
@@ -4245,4 +4246,129 @@ private:
 #endif
     }
 };
+
+class FunctionNgramSearch : public IFunction {
+public:
+    static constexpr auto name = "ngram_search";
+    static FunctionPtr create() { return 
std::make_shared<FunctionNgramSearch>(); }
+    String get_name() const override { return name; }
+    size_t get_number_of_arguments() const override { return 3; }
+    DataTypePtr get_return_type_impl(const DataTypes& arguments) const 
override {
+        return std::make_shared<DataTypeFloat64>();
+    }
+
+    // ngram_search(text,pattern,gram_num)
+    Status execute_impl(FunctionContext* context, Block& block, const 
ColumnNumbers& arguments,
+                        size_t result, size_t input_rows_count) const override 
{
+        CHECK_EQ(arguments.size(), 3);
+        auto col_res = ColumnFloat64::create();
+        bool col_const[3];
+        ColumnPtr argument_columns[3];
+        for (int i = 0; i < 3; ++i) {
+            std::tie(argument_columns[i], col_const[i]) =
+                    
unpack_if_const(block.get_by_position(arguments[i]).column);
+        }
+        // There is no need to check if the 2-th,3-th parameters are const 
here because fe has already checked them.
+        auto pattern = assert_cast<const 
ColumnString*>(argument_columns[1].get())->get_data_at(0);
+        auto gram_num = assert_cast<const 
ColumnInt32*>(argument_columns[2].get())->get_element(0);
+        const auto* text_col = assert_cast<const 
ColumnString*>(argument_columns[0].get());
+
+        if (col_const[0]) {
+            _execute_impl<true>(text_col, pattern, gram_num, *col_res, 
input_rows_count);
+        } else {
+            _execute_impl<false>(text_col, pattern, gram_num, *col_res, 
input_rows_count);
+        }
+
+        block.replace_by_position(result, std::move(col_res));
+        return Status::OK();
+    }
+
+private:
+    using NgramMap = phmap::flat_hash_map<uint32_t, uint8_t>;
+    // In the map, the key is the CRC32 hash result of a substring in the 
string,
+    // and the value indicates whether this hash is found in the text or 
pattern.
+    constexpr static auto not_found = 0b00;
+    constexpr static auto found_in_pattern = 0b01;
+    constexpr static auto found_in_text = 0b10;
+    constexpr static auto found_in_pattern_and_text = 0b11;
+
+    uint32_t sub_str_hash(const char* data, int32_t length) const {
+        constexpr static uint32_t seed = 0;
+        return HashUtil::crc_hash(data, length, seed);
+    }
+
+    template <bool column_const>
+    void _execute_impl(const ColumnString* text_col, StringRef& pattern, int 
gram_num,
+                       ColumnFloat64& res, size_t size) const {
+        auto& res_data = res.get_data();
+        res_data.resize_fill(size, 0);
+        // If the length of the pattern is less than gram_num, return 0.
+        if (pattern.size < gram_num) {
+            return;
+        }
+
+        // Build a map by pattern string, which will be used repeatedly in the 
following loop.
+        NgramMap pattern_map;
+        int pattern_count = get_pattern_set(pattern_map, pattern, gram_num);
+        // Each time a loop is executed, the map will be modified, so it needs 
to be restored afterward.
+        std::vector<uint32_t> restore_map;
+
+        for (int i = 0; i < size; i++) {
+            auto text = 
text_col->get_data_at(index_check_const<column_const>(i));
+            if (text.size < gram_num) {
+                // If the length of the text is less than gram_num, return 0.
+                continue;
+            }
+            restore_map.reserve(text.size);
+            auto [text_count, intersection_count] =
+                    get_text_set(text, gram_num, pattern_map, restore_map);
+
+            // 2 * |Intersection| / (|text substr set| + |pattern substr set|)
+            res_data[i] = 2.0 * intersection_count / (text_count + 
pattern_count);
+        }
+    }
+
+    size_t get_pattern_set(NgramMap& pattern_map, StringRef& pattern, int 
gram_num) const {
+        size_t pattern_count = 0;
+        for (int i = 0; i + gram_num <= pattern.size; i++) {
+            uint32_t cur_hash = sub_str_hash(pattern.data + i, gram_num);
+            if (!pattern_map.contains(cur_hash)) {
+                pattern_map[cur_hash] = found_in_pattern;
+                pattern_count++;
+            }
+        }
+        return pattern_count;
+    }
+
+    pair<size_t, size_t> get_text_set(StringRef& text, int gram_num, NgramMap& 
pattern_map,
+                                      std::vector<uint32_t>& restore_map) 
const {
+        restore_map.clear();
+        //intersection_count indicates a substring both in pattern and text.
+        size_t text_count = 0, intersection_count = 0;
+        for (int i = 0; i + gram_num <= text.size; i++) {
+            uint32_t cur_hash = sub_str_hash(text.data + i, gram_num);
+            auto& val = pattern_map[cur_hash];
+            if (val == not_found) {
+                val ^= found_in_text;
+                DCHECK(val == found_in_text);
+                // only found in text
+                text_count++;
+                restore_map.push_back(cur_hash);
+            } else if (val == found_in_pattern) {
+                val ^= found_in_text;
+                DCHECK(val == found_in_pattern_and_text);
+                // found in text and pattern
+                text_count++;
+                intersection_count++;
+                restore_map.push_back(cur_hash);
+            }
+        }
+        // Restore the pattern_map.
+        for (auto& restore_hash : restore_map) {
+            pattern_map[restore_hash] ^= found_in_text;
+        }
+
+        return {text_count, intersection_count};
+    }
+};
 } // namespace doris::vectorized
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
index ded6f764e0f..6dcf0f39aa5 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
@@ -305,6 +305,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MultiSearchAl
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MurmurHash332;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MurmurHash364;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Negative;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.NgramSearch;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.NonNullable;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.NotNullOrEmpty;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Now;
@@ -762,6 +763,7 @@ public class BuiltinScalarFunctions implements 
FunctionHelper {
             scalar(Negative.class, "negative"),
             scalar(NonNullable.class, "non_nullable"),
             scalar(NotNullOrEmpty.class, "not_null_or_empty"),
+            scalar(NgramSearch.class, "ngram_search"),
             scalar(Now.class, "now", "current_timestamp", "localtime", 
"localtimestamp"),
             scalar(Nullable.class, "nullable"),
             scalar(NullIf.class, "nullif"),
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/NgramSearch.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/NgramSearch.java
new file mode 100644
index 00000000000..8ac713c6a09
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/NgramSearch.java
@@ -0,0 +1,78 @@
+// 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.
+
+package org.apache.doris.nereids.trees.expressions.functions.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.exceptions.AnalysisException;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import 
org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.DoubleType;
+import org.apache.doris.nereids.types.IntegerType;
+import org.apache.doris.nereids.types.StringType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * ScalarFunction 'NgramSearch'.
+ */
+public class NgramSearch extends ScalarFunction
+        implements ExplicitlyCastableSignature, PropagateNullable {
+
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            
FunctionSignature.ret(DoubleType.INSTANCE).args(StringType.INSTANCE, 
StringType.INSTANCE,
+                    IntegerType.INSTANCE));
+
+    /**
+     * constructor with 3 argument.
+     */
+    public NgramSearch(Expression arg0, Expression arg1, Expression arg2) {
+        super("ngram_search", arg0, arg1, arg2);
+        if (!(arg1.isConstant())) {
+            throw new AnalysisException(
+                    "ngram_search(text,pattern,gram_num): pattern support 
const value only.");
+        }
+        if (!(arg2.isConstant())) {
+            throw new AnalysisException(
+                    "ngram_search(text,pattern,gram_num): gram_num support 
const value only.");
+        }
+    }
+
+    /**
+     * withChildren.
+     */
+    @Override
+    public NgramSearch withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 3);
+        return new NgramSearch(children.get(0), children.get(1), 
children.get(2));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitNgramSearch(this, context);
+    }
+}
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
index bfca1a11585..1c765f9a6e3 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
@@ -306,6 +306,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MultiSearchAl
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MurmurHash332;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MurmurHash364;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Negative;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.NgramSearch;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.NotNullOrEmpty;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Now;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.NullIf;
@@ -1567,6 +1568,10 @@ public interface ScalarFunctionVisitor<R, C> {
         return visitScalarFunction(negative, context);
     }
 
+    default R visitNgramSearch(NgramSearch ngramSearch, C context) {
+        return visitScalarFunction(ngramSearch, context);
+    }
+
     default R visitNotNullOrEmpty(NotNullOrEmpty notNullOrEmpty, C context) {
         return visitScalarFunction(notNullOrEmpty, context);
     }
diff --git 
a/regression-test/data/query_p0/sql_functions/string_functions/test_string_function.out
 
b/regression-test/data/query_p0/sql_functions/string_functions/test_string_function.out
index 0aae3a22064..a64ba474599 100644
Binary files 
a/regression-test/data/query_p0/sql_functions/string_functions/test_string_function.out
 and 
b/regression-test/data/query_p0/sql_functions/string_functions/test_string_function.out
 differ
diff --git 
a/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy
 
b/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy
index ea6001d1399..eb38c3e0fa8 100644
--- 
a/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy
+++ 
b/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy
@@ -362,22 +362,33 @@ suite("test_string_function", "arrow_flight_sql") {
     qt_strcmp2 """ select strcmp('abc', 'abc'); """
     qt_strcmp3 """ select strcmp('abcd', 'abc'); """
 
-    sql "SELECT random_bytes(7);"
-    qt_sql_random_bytes "SELECT random_bytes(null);"
-    test {
-        sql " select random_bytes(-1); "
-        exception "argument -1 of function random_bytes at row 0 was invalid"
-    }
-    def some_result = sql """ SELECT random_bytes(10) a FROM numbers("number" 
= "10") """
-    assertTrue(some_result[0][0] != some_result[1][0], "${some_result[0][0]} 
should different with ${some_result[1][0]}")
-    sql "select random_bytes(k1) from test_function_char;"
 
-    explain {
-        sql("""select/*+SET_VAR(enable_fold_constant_by_be=true)*/ 
random_bytes(10) from numbers("number" = "10");""")
-        contains "final projections: random_bytes(10)"
-    }
-    explain {
-        sql("""select/*+SET_VAR(enable_fold_constant_by_be=true)*/ random(10) 
from numbers("number" = "10");""")
-        contains "final projections: random(10)"
-    }
+    sql "drop table if exists test_function_ngram_search;";
+    sql """ create table test_function_ngram_search (
+        k1 int not null,
+        s string null 
+    ) distributed by hash (k1) buckets 1
+    properties ("replication_num"="1");
+    """
+
+    sql """  insert into test_function_ngram_search 
values(1,"fffhhhkkkk"),(2,"abc1313131"),(3,'1313131') ,(4,'abc') , (5,null)"""
+
+    qt_ngram_search1 """ select k1, ngram_search(s,'abc1313131',3) as x , s 
from test_function_ngram_search order by x ;"""
+
+    qt_ngram_search2 """select ngram_search('abc','abc1313131',3); """
+    qt_ngram_search3 """select ngram_search('abc1313131','abc1313131',3); """
+    qt_ngram_search3 """select ngram_search('1313131','abc1313131',3); """
+    
+
+    sql "drop table if exists test_function_ngram_search;";
+    sql """ create table test_function_ngram_search (
+        k1 int not null,
+        s string not null 
+    ) distributed by hash (k1) buckets 1
+    properties ("replication_num"="1");
+    """
+
+    sql """  insert into test_function_ngram_search 
values(1,"fffhhhkkkk"),(2,"abc1313131"),(3,'1313131') ,(4,'abc') """
+
+    qt_ngram_search1_not_null """ select k1, ngram_search(s,'abc1313131',3) as 
x , s from test_function_ngram_search order by x ;"""
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org
For additional commands, e-mail: commits-h...@doris.apache.org

Reply via email to