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 00000000000000..40628ee2017fc7 --- /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 +#include + +#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 _matches; + int32_t _splits; + re2::RE2* _re2 = nullptr; + unsigned _number_of_subpatterns = 0; + + unsigned match(const char* subject, size_t subject_size, std::vector& matches, + unsigned limit) const; +}; + +unsigned RegexpSplit::match(const char* subject, size_t subject_size, std::vector& matches, + unsigned limit) const { + matches.clear(); + + if (limit == 0) { + return 0; + } + + limit = std::min(limit, _number_of_subpatterns + 1); + std::vector 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 +class SplitByRegexp : public IFunction { +public: + static constexpr auto name = "split_by_regexp"; + + static FunctionPtr create() { return std::make_shared(); } + + 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()); + return std::make_shared(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; + 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(*three_column).get_int(0); + const auto& src_column = assert_cast(*first_column); + const auto& pattern_column = assert_cast(*second_column); + + auto nullable_string_type = make_nullable(std::make_shared()); + 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(dest_nested_column); + auto& dest_column_string = + assert_cast(*(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_ptr = nullptr; + if (pattern_ref.size) { + re2_ptr = std::make_unique(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_ptr = nullptr; + auto pattern = pattern_column.get_data_at(row); + if (pattern.size) { + re2_ptr = std::make_unique(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_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(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(), std::make_shared()}; + } + + 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(), "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(), std::make_shared(), + std::make_shared()}; + } + 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>(); + factory.register_function>(); +} + +} // namespace doris::vectorized diff --git a/be/src/vec/functions/simple_function_factory.h b/be/src/vec/functions/simple_function_factory.h index 4a6ef81424387c..91ddb2ac1a0682 100644 --- a/be/src/vec/functions/simple_function_factory.h +++ b/be/src/vec/functions/simple_function_factory.h @@ -105,6 +105,7 @@ void register_function_tokenize(SimpleFunctionFactory& factory); void register_function_url(SimpleFunctionFactory& factory); void register_function_ip(SimpleFunctionFactory& factory); void register_function_multi_match(SimpleFunctionFactory& factory); +void register_function_split_by_regexp(SimpleFunctionFactory& factory); class SimpleFunctionFactory { using Creator = std::function; @@ -292,6 +293,7 @@ class SimpleFunctionFactory { register_function_ignore(instance); register_function_variant_element(instance); register_function_multi_match(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 f57840ffd76907..d6640d65109730 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 @@ -370,6 +370,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Sm4EncryptV2; 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; @@ -843,6 +844,7 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(Sm4EncryptV2.class, "sm4_encrypt_v2"), 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 00000000000000..8d1d0145d71047 --- /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.IntegerLiteral; +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 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 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 IntegerLiteral) + || (((IntegerLiteral) child(2)).getValue() < 0)) { + throw new AnalysisException("the third parameter of " + + getName() + " function must be a positive constant: " + toSql()); + } + } + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitSplitByRegexp(this, context); + } + + @Override + public List 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 12fd3f119b3105..3562e5d7347cc1 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 @@ -368,6 +368,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Sm4EncryptV2; 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; @@ -1830,6 +1831,10 @@ default R visitSplitByChar(SplitByChar splitByChar, C context) { 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 4bd3c1c4943043..7328355ce7afa4 100644 --- a/gensrc/script/doris_builtins_functions.py +++ b/gensrc/script/doris_builtins_functions.py @@ -1658,6 +1658,8 @@ [['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 00000000000000..588ad7fa5cb9d2 --- /dev/null +++ b/regression-test/data/query_p0/sql_functions/string_functions/test_split_by_regexp.out @@ -0,0 +1,52 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select1 -- +["a", "b", "c", "d", "e"] + +-- !select2 -- +["a", "bc", "de", "f"] + +-- !select3 -- +\N + +-- !select4 -- +\N + +-- !select5 -- +["abcde"] +["12553"] +[""] +[""] +[""] +["a1b1c1d"] +["", "", "", ""] +["a", "b", "c"] +["a", "b", "c", ""] +\N +["a", "b", "c", "12345", ""] + +-- !select6 -- +["d", "o", ",", "r", "i", "s"] +["d", "o", ",", "r", "i", "s"] +["d", "o", ",", "r", "i", "s"] +["do", "ris"] +["do,ris"] +["do,ris"] +["do", "ris"] +["do", "ris"] +["do", "ris"] +["do", "ris"] +["do", "ris"] + +-- !select7 -- +["a", "b", "c", "d", "e"] +["1", "2", "5", "5", "3"] +[] +[""] +[""] +["a", "b", "c", "d"] +["", "", "", ""] +["a", "b", "c"] +["a", "b", "c", ""] +\N +["a", "b", "c", "12345", ""] + 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 00000000000000..c9ace391b5bebb --- /dev/null +++ b/regression-test/suites/query_p0/sql_functions/string_functions/test_split_by_regexp.groovy @@ -0,0 +1,68 @@ +// 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;" +} +