lincoln-lil commented on code in PR #25137:
URL: https://github.com/apache/flink/pull/25137#discussion_r1724562351


##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/RegexpExtractAllFunction.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.runtime.functions.scalar;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.SpecializedFunction;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+
+import static 
org.apache.flink.table.runtime.functions.SqlFunctionUtils.getRegexpMatcher;
+
+/** Implementation of {@link BuiltInFunctionDefinitions#REGEXP_EXTRACT_ALL}. */
+@Internal
+public class RegexpExtractAllFunction extends BuiltInScalarFunction {
+
+    public RegexpExtractAllFunction(SpecializedFunction.SpecializedContext 
context) {
+        super(BuiltInFunctionDefinitions.REGEXP_EXTRACT_ALL, context);
+    }
+
+    public @Nullable ArrayData eval(@Nullable StringData str, @Nullable 
StringData regex) {
+        return eval(str, regex, 1);
+    }
+
+    public @Nullable ArrayData eval(
+            @Nullable StringData str, @Nullable StringData regex, @Nullable 
Number extractIndex) {
+        if (extractIndex == null) {
+            return null;
+        }
+
+        Matcher matcher = getRegexpMatcher(str, regex);
+        if (matcher == null) {
+            return null;
+        }
+
+        long groupIndex = extractIndex.longValue();
+        if (groupIndex < 0 || matcher.groupCount() < groupIndex) {

Review Comment:
   We can check `if (groupIndex < 0)` before matching to avoid unnecessary cost



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/SqlFunctionUtils.java:
##########
@@ -64,7 +68,7 @@ public class SqlFunctionUtils {
 
     private static final Logger LOG = 
LoggerFactory.getLogger(SqlFunctionUtils.class);
 
-    private static final ThreadLocalCache<String, Pattern> 
REGEXP_PATTERN_CACHE =
+    public static final ThreadLocalCache<String, Pattern> REGEXP_PATTERN_CACHE 
=

Review Comment:
   nit:  we can keep it `private` since the reusable method added.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to