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

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


The following commit(s) were added to refs/heads/branch-3.0 by this push:
     new d499904f407 [cherry-pick](branch-30) support split_by_regexp (#38259) 
(#47676) (#50272)
d499904f407 is described below

commit d499904f4073db892a57213861eb5a5f17582a0b
Author: zhangstar333 <[email protected]>
AuthorDate: Wed Jun 18 12:24:03 2025 +0800

    [cherry-pick](branch-30) support split_by_regexp (#38259) (#47676) (#50272)
    
    bp (#38259) (#47676)
    also need pick https://github.com/apache/doris/pull/51293
    
    ---------
    
    Co-authored-by: James <[email protected]>
---
 be/src/vec/functions/function_split_by_regexp.cpp  | 378 +++++++++++++++++++++
 be/src/vec/functions/simple_function_factory.h     |   2 +
 .../doris/catalog/BuiltinScalarFunctions.java      |   2 +
 .../functions/scalar/SplitByRegexp.java            |  97 ++++++
 .../expressions/visitor/ScalarFunctionVisitor.java |   5 +
 gensrc/script/doris_builtins_functions.py          |   2 +
 .../string_functions/test_split_by_regexp.out      | Bin 0 -> 987 bytes
 .../string_functions/test_split_by_regexp.groovy   |  74 ++++
 8 files changed, 560 insertions(+)

diff --git a/be/src/vec/functions/function_split_by_regexp.cpp 
b/be/src/vec/functions/function_split_by_regexp.cpp
new file mode 100644
index 00000000000..40628ee2017
--- /dev/null
+++ b/be/src/vec/functions/function_split_by_regexp.cpp
@@ -0,0 +1,378 @@
+// 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 <fmt/format.h>
+#include <glog/logging.h>
+
+#include "common/status.h"
+#include "vec/columns/column_array.h"
+#include "vec/columns/column_const.h"
+#include "vec/core/types.h"
+#include "vec/data_types/data_type_array.h"
+#include "vec/data_types/data_type_number.h"
+#include "vec/data_types/data_type_string.h"
+#include "vec/functions/function.h"
+#include "vec/functions/function_string.h"
+#include "vec/functions/simple_function_factory.h"
+
+namespace doris::vectorized {
+
+struct Match {
+    std::string::size_type offset;
+    std::string::size_type length;
+};
+
+class RegexpSplit {
+public:
+    void init(re2::RE2* re2, int32_t max_splits);
+    void set(const char* pos, const char* end);
+    bool get(const char*& token_begin, const char*& token_end);
+
+private:
+    const char* _pos;
+    const char* _end;
+
+    std::int32_t _max_splits = 0;
+    std::vector<Match> _matches;
+    int32_t _splits;
+    re2::RE2* _re2 = nullptr;
+    unsigned _number_of_subpatterns = 0;
+
+    unsigned match(const char* subject, size_t subject_size, 
std::vector<Match>& matches,
+                   unsigned limit) const;
+};
+
+unsigned RegexpSplit::match(const char* subject, size_t subject_size, 
std::vector<Match>& matches,
+                            unsigned limit) const {
+    matches.clear();
+
+    if (limit == 0) {
+        return 0;
+    }
+
+    limit = std::min(limit, _number_of_subpatterns + 1);
+    std::vector<re2::StringPiece> pieces(limit);
+
+    if (!_re2->Match({subject, subject_size}, 0, subject_size, 
re2::RE2::UNANCHORED, pieces.data(),
+                     limit)) {
+        return 0;
+    } else {
+        matches.resize(limit);
+        for (size_t i = 0; i < limit; ++i) {
+            if (pieces[i].empty()) {
+                matches[i].offset = std::string::npos;
+                matches[i].length = 0;
+            } else {
+                matches[i].offset = pieces[i].data() - subject;
+                matches[i].length = pieces[i].length();
+            }
+        }
+        return limit;
+    }
+}
+
+void RegexpSplit::init(re2::RE2* re2, int32_t max_splits) {
+    _max_splits = max_splits;
+    _re2 = re2;
+    if (_re2) {
+        _number_of_subpatterns = _re2->NumberOfCapturingGroups();
+    }
+}
+
+// Called for each next string.
+void RegexpSplit::set(const char* pos, const char* end) {
+    _pos = pos;
+    _end = end;
+    _splits = 0;
+}
+
+// Get the next token, if any, or return false.
+bool RegexpSplit::get(const char*& token_begin, const char*& token_end) {
+    if (!_re2) {
+        if (_pos == _end) {
+            return false;
+        }
+
+        token_begin = _pos;
+        if (_max_splits != -1) {
+            if (_splits == _max_splits - 1) {
+                token_end = _end;
+                _pos = _end;
+                return true;
+            }
+        }
+
+        _pos += 1;
+        token_end = _pos;
+        ++_splits;
+    } else {
+        if (!_pos || _pos > _end) {
+            return false;
+        }
+
+        token_begin = _pos;
+        if (_max_splits != -1) {
+            if (_splits == _max_splits - 1) {
+                token_end = _end;
+                _pos = nullptr;
+                return true;
+            }
+        }
+
+        if (!match(_pos, _end - _pos, _matches, _number_of_subpatterns + 1) ||
+            !_matches[0].length) {
+            token_end = _end;
+            _pos = _end + 1;
+        } else {
+            token_end = _pos + _matches[0].offset;
+            _pos = token_end + _matches[0].length;
+            ++_splits;
+        }
+    }
+
+    return true;
+}
+
+template <typename Impl>
+class SplitByRegexp : public IFunction {
+public:
+    static constexpr auto name = "split_by_regexp";
+
+    static FunctionPtr create() { return std::make_shared<SplitByRegexp>(); }
+
+    String get_name() const override { return name; }
+
+    size_t get_number_of_arguments() const override {
+        return get_variadic_argument_types_impl().size();
+    }
+
+    bool is_variadic() const override { return true; }
+
+    DataTypes get_variadic_argument_types_impl() const override {
+        return Impl::get_variadic_argument_types();
+    }
+
+    DataTypePtr get_return_type_impl(const DataTypes& arguments) const 
override {
+        DCHECK(is_string(arguments[0]))
+                << "first argument for function: " << name << " should be 
string"
+                << " and arguments[0] is " << arguments[0]->get_name();
+        DCHECK(is_string(arguments[1]))
+                << "second argument for function: " << name << " should be 
string"
+                << " and arguments[1] is " << arguments[1]->get_name();
+        auto nullable_string_type = 
make_nullable(std::make_shared<DataTypeString>());
+        return std::make_shared<DataTypeArray>(nullable_string_type);
+    }
+
+    Status execute_impl(FunctionContext* context, Block& block, const 
ColumnNumbers& arguments,
+                        size_t result, size_t input_rows_count) const override 
{
+        return Impl::execute_impl(context, block, arguments, result, 
input_rows_count);
+    }
+};
+
+struct ExecuteImpl {
+    using NullMapType = PaddedPODArray<UInt8>;
+    static Status execute_impl(FunctionContext* context, Block& block,
+                               const ColumnNumbers& arguments, size_t result,
+                               size_t input_rows_count) {
+        const auto& [first_column, left_const] =
+                unpack_if_const(block.get_by_position(arguments[0]).column);
+        const auto& [second_column, right_const] =
+                unpack_if_const(block.get_by_position(arguments[1]).column);
+        const auto& [three_column, three_is_const] =
+                unpack_if_const(block.get_by_position(arguments[2]).column);
+        auto limit_value = assert_cast<const 
ColumnInt32&>(*three_column).get_int(0);
+        const auto& src_column = assert_cast<const 
ColumnString&>(*first_column);
+        const auto& pattern_column = assert_cast<const 
ColumnString&>(*second_column);
+
+        auto nullable_string_type = 
make_nullable(std::make_shared<DataTypeString>());
+        auto dest_column_ptr = 
ColumnArray::create(nullable_string_type->create_column(),
+                                                   
ColumnArray::ColumnOffsets::create());
+        IColumn* dest_nested_column = &dest_column_ptr->get_data();
+        auto& dest_offsets = dest_column_ptr->get_offsets();
+        DCHECK(dest_nested_column != nullptr);
+
+        NullMapType* dest_nested_null_map = nullptr;
+        auto* dest_nullable_col = 
assert_cast<ColumnNullable*>(dest_nested_column);
+        auto& dest_column_string =
+                
assert_cast<ColumnString&>(*(dest_nullable_col->get_nested_column_ptr()));
+        dest_nested_null_map = 
&dest_nullable_col->get_null_map_column().get_data();
+        RE2::Options opts;
+        opts.set_never_nl(false);
+        opts.set_dot_nl(true);
+        // split_by_regexp(ColumnString, "xxx")
+        if (right_const) {
+            RETURN_IF_ERROR(_execute_constant_pattern(
+                    src_column, pattern_column.get_data_at(0), 
dest_column_string, dest_offsets,
+                    dest_nested_null_map, limit_value, input_rows_count, 
&opts));
+        } else if (left_const) {
+            // split_by_regexp("xxx", ColumnString)
+            _execute_constant_src_string(src_column.get_data_at(0), 
pattern_column,
+                                         dest_column_string, dest_offsets, 
dest_nested_null_map,
+                                         limit_value, input_rows_count, &opts);
+        } else {
+            // split_by_regexp(ColumnString, ColumnString)
+            _execute_vector_vector(src_column, pattern_column, 
dest_column_string, dest_offsets,
+                                   dest_nested_null_map, limit_value, 
input_rows_count, &opts);
+        }
+
+        block.replace_by_position(result, std::move(dest_column_ptr));
+        return Status::OK();
+    }
+
+private:
+    static Status _execute_constant_pattern(const ColumnString& 
src_column_string,
+                                            const StringRef& pattern_ref,
+                                            ColumnString& dest_column_string,
+                                            ColumnArray::Offsets64& 
dest_offsets,
+                                            NullMapType* dest_nested_null_map, 
Int64 limit_value,
+                                            size_t input_rows_count, 
RE2::Options* opts) {
+        const char* token_begin = nullptr;
+        const char* token_end = nullptr;
+        UInt64 index = 0;
+        std::unique_ptr<re2::RE2> re2_ptr = nullptr;
+        if (pattern_ref.size) {
+            re2_ptr = std::make_unique<re2::RE2>(pattern_ref.to_string_view(), 
*opts);
+        }
+        if (!re2_ptr->ok()) {
+            return Status::RuntimeError("Invalid pattern: {}", 
pattern_ref.debug_string());
+        }
+        RegexpSplit RegexpSplit;
+        RegexpSplit.init(re2_ptr.get(), limit_value);
+        for (int row = 0; row < input_rows_count; ++row) {
+            auto str_data = src_column_string.get_data_at(row);
+            RegexpSplit.set(str_data.begin(), str_data.end());
+            while (RegexpSplit.get(token_begin, token_end)) {
+                size_t token_size = token_end - token_begin;
+                dest_column_string.insert_data(token_begin, token_size);
+                dest_nested_null_map->push_back(false);
+                index += 1;
+            }
+            dest_offsets.push_back(index);
+        }
+        return Status::OK();
+    }
+
+    static void _execute_constant_src_string(const StringRef& str_ref,
+                                             const ColumnString& 
pattern_column,
+                                             ColumnString& dest_column_string,
+                                             ColumnArray::Offsets64& 
dest_offsets,
+                                             NullMapType* 
dest_nested_null_map, Int64 limit_value,
+                                             size_t input_rows_count, 
RE2::Options* opts) {
+        const char* token_begin = nullptr;
+        const char* token_end = nullptr;
+        UInt64 index = 0;
+        RegexpSplit RegexpSplit;
+
+        for (int row = 0; row < input_rows_count; ++row) {
+            std::unique_ptr<re2::RE2> re2_ptr = nullptr;
+            auto pattern = pattern_column.get_data_at(row);
+            if (pattern.size) {
+                re2_ptr = std::make_unique<re2::RE2>(pattern.to_string_view(), 
*opts);
+                if (!re2_ptr->ok()) {
+                    dest_column_string.insert_default();
+                    dest_nested_null_map->push_back(true);
+                    index += 1;
+                    dest_offsets.push_back(index);
+                    continue;
+                }
+            }
+
+            RegexpSplit.init(re2_ptr.get(), limit_value);
+            RegexpSplit.set(str_ref.begin(), str_ref.end());
+            while (RegexpSplit.get(token_begin, token_end)) {
+                size_t token_size = token_end - token_begin;
+                dest_column_string.insert_data(token_begin, token_size);
+                dest_nested_null_map->push_back(false);
+                index += 1;
+            }
+            dest_offsets.push_back(index);
+        }
+    }
+
+    static void _execute_vector_vector(const ColumnString& src_column_string,
+                                       const ColumnString& pattern_column,
+                                       ColumnString& dest_column_string,
+                                       ColumnArray::Offsets64& dest_offsets,
+                                       NullMapType* dest_nested_null_map, 
Int64 limit_value,
+                                       size_t input_rows_count, RE2::Options* 
opts) {
+        const char* token_begin = nullptr;
+        const char* token_end = nullptr;
+        UInt64 index = 0;
+        RegexpSplit RegexpSplit;
+
+        for (int row = 0; row < input_rows_count; ++row) {
+            std::unique_ptr<re2::RE2> re2_ptr = nullptr;
+            auto str_data = src_column_string.get_data_at(row);
+            auto pattern = pattern_column.get_data_at(row);
+            if (pattern.size) {
+                re2_ptr = std::make_unique<re2::RE2>(pattern.to_string_view(), 
*opts);
+                if (!re2_ptr->ok()) {
+                    dest_column_string.insert_default();
+                    dest_nested_null_map->push_back(true);
+                    index += 1;
+                    dest_offsets.push_back(index);
+                    continue;
+                }
+            }
+            RegexpSplit.init(re2_ptr.get(), limit_value);
+            RegexpSplit.set(str_data.begin(), str_data.end());
+            while (RegexpSplit.get(token_begin, token_end)) {
+                size_t token_size = token_end - token_begin;
+                dest_column_string.insert_data(token_begin, token_size);
+                dest_nested_null_map->push_back(false);
+                index += 1;
+            }
+            dest_offsets.push_back(index);
+        }
+    }
+};
+
+struct TwoArgumentImpl {
+    static DataTypes get_variadic_argument_types() {
+        return {std::make_shared<DataTypeString>(), 
std::make_shared<DataTypeString>()};
+    }
+
+    static Status execute_impl(FunctionContext* context, Block& block,
+                               const ColumnNumbers& arguments, size_t result,
+                               size_t input_rows_count) {
+        DCHECK_EQ(arguments.size(), 2);
+        auto max_limit = ColumnConst::create(ColumnInt32::create(1, -1), 
input_rows_count);
+        block.insert({std::move(max_limit), std::make_shared<DataTypeInt32>(), 
"max_limit"});
+        ColumnNumbers temp_arguments = {arguments[0], arguments[1], 
block.columns() - 1};
+        return ExecuteImpl::execute_impl(context, block, temp_arguments, 
result, input_rows_count);
+    }
+};
+
+struct ThreeArgumentImpl {
+    static DataTypes get_variadic_argument_types() {
+        return {std::make_shared<DataTypeString>(), 
std::make_shared<DataTypeString>(),
+                std::make_shared<DataTypeInt32>()};
+    }
+    static Status execute_impl(FunctionContext* context, Block& block,
+                               const ColumnNumbers& arguments, size_t result,
+                               size_t input_rows_count) {
+        DCHECK_EQ(arguments.size(), 3);
+        return ExecuteImpl::execute_impl(context, block, arguments, result, 
input_rows_count);
+    }
+};
+
+void register_function_split_by_regexp(SimpleFunctionFactory& factory) {
+    factory.register_function<SplitByRegexp<TwoArgumentImpl>>();
+    factory.register_function<SplitByRegexp<ThreeArgumentImpl>>();
+}
+
+} // namespace doris::vectorized
diff --git a/be/src/vec/functions/simple_function_factory.h 
b/be/src/vec/functions/simple_function_factory.h
index 7619858153c..727cf98cda3 100644
--- a/be/src/vec/functions/simple_function_factory.h
+++ b/be/src/vec/functions/simple_function_factory.h
@@ -109,6 +109,7 @@ void register_function_url(SimpleFunctionFactory& factory);
 void register_function_ip(SimpleFunctionFactory& factory);
 void register_function_multi_match(SimpleFunctionFactory& factory);
 void register_function_assert_true(SimpleFunctionFactory& factory);
+void register_function_split_by_regexp(SimpleFunctionFactory& factory);
 
 class SimpleFunctionFactory {
     using Creator = std::function<FunctionBuilderPtr()>;
@@ -313,6 +314,7 @@ public:
             register_function_variant_element(instance);
             register_function_multi_match(instance);
             register_function_assert_true(instance);
+            register_function_split_by_regexp(instance);
         });
         return instance;
     }
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 1c1a820952f..7d9314cb74f 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
@@ -385,6 +385,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.Sm4Decrypt;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Sm4Encrypt;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Space;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.SplitByChar;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.SplitByRegexp;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.SplitByString;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.SplitPart;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Sqrt;
@@ -881,6 +882,7 @@ public class BuiltinScalarFunctions implements 
FunctionHelper {
             scalar(Sm4Encrypt.class, "sm4_encrypt"),
             scalar(Space.class, "space"),
             scalar(SplitByChar.class, "split_by_char"),
+            scalar(SplitByRegexp.class, "split_by_regexp"),
             scalar(SplitByString.class, "split_by_string"),
             scalar(SplitPart.class, "split_part"),
             scalar(Sqrt.class, "sqrt"),
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SplitByRegexp.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SplitByRegexp.java
new file mode 100644
index 00000000000..a72ed434cc3
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SplitByRegexp.java
@@ -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.
+
+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.literal.IntegerLikeLiteral;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.ArrayType;
+import org.apache.doris.nereids.types.IntegerType;
+import org.apache.doris.nereids.types.StringType;
+import org.apache.doris.nereids.types.VarcharType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * ScalarFunction 'split_by_regexp'. This class is generated by
+ * GenerateFunction.
+ */
+public class SplitByRegexp extends ScalarFunction
+        implements ExplicitlyCastableSignature, PropagateNullable {
+
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(ArrayType.of(VarcharType.SYSTEM_DEFAULT))
+                    .args(StringType.INSTANCE, StringType.INSTANCE),
+            FunctionSignature.ret(ArrayType.of(VarcharType.SYSTEM_DEFAULT))
+                    .args(StringType.INSTANCE, StringType.INSTANCE, 
IntegerType.INSTANCE));
+
+    /**
+     * constructor with 2 arguments.
+     */
+    public SplitByRegexp(Expression arg0, Expression arg1) {
+        super("split_by_regexp", arg0, arg1);
+    }
+
+    /**
+     * constructor with 3 arguments.
+     */
+    public SplitByRegexp(Expression arg0, Expression arg1, Expression arg2) {
+        super("split_by_regexp", arg0, arg1, arg2);
+    }
+
+    /**
+     * withChildren.
+     */
+    @Override
+    public SplitByRegexp withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 2 || children.size() == 
3);
+        if (children.size() == 2) {
+            return new SplitByRegexp(children.get(0), children.get(1));
+        } else {
+            return new SplitByRegexp(children.get(0), children.get(1), 
children.get(2));
+        }
+    }
+
+    @Override
+    public void checkLegalityBeforeTypeCoercion() {
+        if (children().size() == 3) {
+            if (!child(2).isConstant() || !(child(2) instanceof 
IntegerLikeLiteral)
+                    || (((IntegerLikeLiteral) child(2)).getIntValue() < 0)) {
+                throw new AnalysisException("the third parameter of "
+                        + getName() + " function must be a positive constant: 
" + toSql());
+            }
+        }
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitSplitByRegexp(this, context);
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+}
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 b061e2f8d6a..f85ce2fbed3 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
@@ -383,6 +383,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.Sm4Decrypt;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Sm4Encrypt;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Space;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.SplitByChar;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.SplitByRegexp;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.SplitByString;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.SplitPart;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Sqrt;
@@ -1912,6 +1913,10 @@ public interface ScalarFunctionVisitor<R, C> {
         return visitScalarFunction(splitByChar, context);
     }
 
+    default R visitSplitByRegexp(SplitByRegexp splitByRegexp, C context) {
+        return visitScalarFunction(splitByRegexp, context);
+    }
+
     default R visitSplitByString(SplitByString splitByString, C context) {
         return visitScalarFunction(splitByString, context);
     }
diff --git a/gensrc/script/doris_builtins_functions.py 
b/gensrc/script/doris_builtins_functions.py
index 99c17f7dfc8..3f4614dc337 100644
--- a/gensrc/script/doris_builtins_functions.py
+++ b/gensrc/script/doris_builtins_functions.py
@@ -1659,6 +1659,8 @@ visible_functions = {
         [['money_format'], 'VARCHAR', ['DECIMAL64'], ''],
         [['money_format'], 'VARCHAR', ['DECIMAL128'], ''],
         [['split_by_string'],'ARRAY_VARCHAR',['STRING','STRING'], ''],
+        [['split_by_regexp'],'ARRAY_VARCHAR',['STRING','STRING'], ''],
+        [['split_by_regexp'],'ARRAY_VARCHAR',['STRING','STRING', 'INT'], ''],
         [['split_part'], 'VARCHAR', ['VARCHAR', 'VARCHAR', 'INT'], 
'ALWAYS_NULLABLE'],
         [['substring_index'], 'VARCHAR', ['VARCHAR', 'VARCHAR', 'INT'], 
'DEPEND_ON_ARGUMENT'],
         [['extract_url_parameter'], 'VARCHAR', ['VARCHAR', 'VARCHAR'], ''],
diff --git 
a/regression-test/data/query_p0/sql_functions/string_functions/test_split_by_regexp.out
 
b/regression-test/data/query_p0/sql_functions/string_functions/test_split_by_regexp.out
new file mode 100644
index 00000000000..1fb99f58ab1
Binary files /dev/null and 
b/regression-test/data/query_p0/sql_functions/string_functions/test_split_by_regexp.out
 differ
diff --git 
a/regression-test/suites/query_p0/sql_functions/string_functions/test_split_by_regexp.groovy
 
b/regression-test/suites/query_p0/sql_functions/string_functions/test_split_by_regexp.groovy
new file mode 100644
index 00000000000..4b9719068e6
--- /dev/null
+++ 
b/regression-test/suites/query_p0/sql_functions/string_functions/test_split_by_regexp.groovy
@@ -0,0 +1,74 @@
+// 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.
+
+suite("test_split_by_regexp") {
+    qt_select1 "select split_by_regexp('abcde','');"
+    qt_select2 "select split_by_regexp('a12bc23de345f','\\\\d+');"
+    qt_select3 "select split_by_regexp('a12bc23de345f',NULL);"
+    qt_select4 "select split_by_regexp(NULL, 'a12bc23de345f');"
+
+    def tableName1 = "test_split_by_regexp"
+
+    sql """DROP TABLE IF EXISTS ${tableName1}"""
+    sql """ 
+            CREATE TABLE IF NOT EXISTS ${tableName1} (
+              `k1` int(11) NULL COMMENT "",
+              `v1` varchar(20) NULL COMMENT "",
+              `v2` varchar(1) NOT NULL COMMENT ""
+            ) ENGINE=OLAP
+            DUPLICATE KEY(`k1`)
+            DISTRIBUTED BY HASH(`k1`) BUCKETS 1
+            PROPERTIES (
+            "replication_allocation" = "tag.location.default: 1",
+            "storage_format" = "V2"
+            )
+        """
+    sql """ INSERT INTO ${tableName1} VALUES(1, 'abcde', '') """
+    sql """ INSERT INTO ${tableName1} VALUES(2, '12553', '') """
+    sql """ INSERT INTO ${tableName1} VALUES(3, '', '') """
+    sql """ INSERT INTO ${tableName1} VALUES(4, '', ',') """
+    sql """ INSERT INTO ${tableName1} VALUES(5, '', 'a') """
+    sql """ INSERT INTO ${tableName1} VALUES(6, 'a1b1c1d', '1') """
+    sql """ INSERT INTO ${tableName1} VALUES(7, ',,,', ',') """
+    sql """ INSERT INTO ${tableName1} VALUES(8, 'a,b,c', ',') """
+    sql """ INSERT INTO ${tableName1} VALUES(9, 'a,b,c,', ',') """
+    sql """ INSERT INTO ${tableName1} VALUES(10, null, ',') """
+    sql """ INSERT INTO ${tableName1} VALUES(11, 'a,b,c,12345,', ',') """
+
+    test {
+        sql " select split_by_regexp(NULL, 'a12bc23de345f', k1) from 
test_split_by_regexp"
+        exception "function must be a positive constant"
+    }
+    test {
+        sql " select split_by_regexp(NULL, 'a12bc23de345f', -10) from 
test_split_by_regexp"
+        exception "function must be a positive constant"
+    }
+    test {
+        sql " select split_by_regexp(NULL, 'a12bc23de345f', 1 + 2) from 
test_split_by_regexp"
+        exception "function must be a positive constant"
+    }
+    qt_select5 "select split_by_regexp(v1, ',') from test_split_by_regexp 
order by k1;"
+    qt_select6 "select split_by_regexp('do,ris', v2) from test_split_by_regexp 
order by k1;"
+    qt_select7 "select split_by_regexp(v1, v2) from test_split_by_regexp order 
by k1;"
+    qt_select8 "select split_by_regexp('aa,bbb,cccc', ',', 1);"
+    qt_select9 "select split_by_regexp('aa,bbb,cccc', ',', 2);"
+    qt_select10 "select split_by_regexp('aa,bbb,cccc', ',', 3);"
+    qt_select11 "select split_by_regexp('aa,bbb,cccc', ',', 4);"
+    qt_select12 "select split_by_regexp('aa,bbb,cccc', ',', 100000000);"
+    qt_select13 "select split_by_regexp('aa,bbb,cccc', ',', 10000000000000);"
+}
+


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to