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

lincoln pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 697b4b26010c6e7014bc48f8b1bf1ad5ee1ca55e
Author: dylanhz <[email protected]>
AuthorDate: Fri Aug 16 10:38:09 2024 +0800

    [FLINK-35931][table-planner] Use a separate test class for the regexp 
family of functions
---
 .../planner/functions/RegexpFunctionsITCase.java   | 53 ++++++++++++++++++++++
 .../planner/functions/StringFunctionsITCase.java   | 25 +---------
 2 files changed, 54 insertions(+), 24 deletions(-)

diff --git 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/RegexpFunctionsITCase.java
 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/RegexpFunctionsITCase.java
new file mode 100644
index 00000000000..ab66e6e4030
--- /dev/null
+++ 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/RegexpFunctionsITCase.java
@@ -0,0 +1,53 @@
+/*
+ * 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.flink.table.planner.functions;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.api.Expressions.$;
+import static org.apache.flink.table.api.Expressions.call;
+
+/** Test regex functions correct behaviour. */
+class RegexpFunctionsITCase extends BuiltInFunctionTestBase {
+
+    @Override
+    Stream<TestSetSpec> getTestSetSpecs() {
+        return Stream.of(regexpExtractTestCases()).flatMap(s -> s);
+    }
+
+    private Stream<TestSetSpec> regexpExtractTestCases() {
+        return Stream.of(
+                TestSetSpec.forFunction(
+                                BuiltInFunctionDefinitions.REGEXP_EXTRACT, 
"Check return type")
+                        .onFieldsWithData("22", "ABC")
+                        .testResult(
+                                call("regexpExtract", $("f0"), "[A-Z]+"),
+                                "REGEXP_EXTRACT(f0,'[A-Z]+')",
+                                null,
+                                DataTypes.STRING().nullable())
+                        .testResult(
+                                call("regexpExtract", $("f1"), "[A-Z]+"),
+                                "REGEXP_EXTRACT(f1, '[A-Z]+')",
+                                "ABC",
+                                DataTypes.STRING().nullable()));
+    }
+}
diff --git 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/StringFunctionsITCase.java
 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/StringFunctionsITCase.java
index d1baff469e4..08a390b0897 100644
--- 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/StringFunctionsITCase.java
+++ 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/StringFunctionsITCase.java
@@ -31,7 +31,6 @@ import java.time.temporal.ChronoUnit;
 import java.util.stream.Stream;
 
 import static org.apache.flink.table.api.Expressions.$;
-import static org.apache.flink.table.api.Expressions.call;
 import static org.apache.flink.table.api.Expressions.lit;
 
 /** Test String functions correct behaviour. */
@@ -39,12 +38,7 @@ class StringFunctionsITCase extends BuiltInFunctionTestBase {
 
     @Override
     Stream<TestSetSpec> getTestSetSpecs() {
-        return Stream.of(
-                        bTrimTestCases(),
-                        eltTestCases(),
-                        printfTestCases(),
-                        regexpExtractTestCases(),
-                        translateTestCases())
+        return Stream.of(bTrimTestCases(), eltTestCases(), printfTestCases(), 
translateTestCases())
                 .flatMap(s -> s);
     }
 
@@ -316,23 +310,6 @@ class StringFunctionsITCase extends 
BuiltInFunctionTestBase {
                                         + "PRINTF(format <CHARACTER_STRING>, 
obj <ANY>...)"));
     }
 
-    private Stream<TestSetSpec> regexpExtractTestCases() {
-        return Stream.of(
-                TestSetSpec.forFunction(
-                                BuiltInFunctionDefinitions.REGEXP_EXTRACT, 
"Check return type")
-                        .onFieldsWithData("22", "ABC")
-                        .testResult(
-                                call("regexpExtract", $("f0"), "[A-Z]+"),
-                                "REGEXP_EXTRACT(f0,'[A-Z]+')",
-                                null,
-                                DataTypes.STRING().nullable())
-                        .testResult(
-                                call("regexpExtract", $("f1"), "[A-Z]+"),
-                                "REGEXP_EXTRACT(f1, '[A-Z]+')",
-                                "ABC",
-                                DataTypes.STRING().nullable()));
-    }
-
     private Stream<TestSetSpec> translateTestCases() {
         return Stream.of(
                 TestSetSpec.forFunction(BuiltInFunctionDefinitions.TRANSLATE)

Reply via email to