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

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


The following commit(s) were added to refs/heads/release-1.20 by this push:
     new ce29a69bcbe [FLINK-35498][table] Fix unexpected argument name 
conflicts when extracting method parameter names from UDF
ce29a69bcbe is described below

commit ce29a69bcbe384cdfaec7db98ce4447b760ce3a2
Author: Xuyang <xyzhong...@163.com>
AuthorDate: Thu Jul 4 21:13:48 2024 +0800

    [FLINK-35498][table] Fix unexpected argument name conflicts when extracting 
method parameter names from UDF
    
    This closes #25019
---
 .../table/types/extraction/ExtractionUtils.java    |  49 ++++++++-
 .../types/extraction/ExtractionUtilsTest.java      | 110 +++++++++++++++++++++
 .../planner/runtime/stream/sql/FunctionITCase.java |  43 ++++++++
 3 files changed, 198 insertions(+), 4 deletions(-)

diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/ExtractionUtils.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/ExtractionUtils.java
index 9854e62eb0c..5968b41fc84 100644
--- 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/ExtractionUtils.java
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/ExtractionUtils.java
@@ -19,6 +19,7 @@
 package org.apache.flink.table.types.extraction;
 
 import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.table.annotation.ArgumentHint;
 import org.apache.flink.table.api.DataTypes;
@@ -58,6 +59,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
+import java.util.TreeMap;
 import java.util.function.Function;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
@@ -747,7 +749,8 @@ public final class ExtractionUtils {
         return fieldNames;
     }
 
-    private static @Nullable List<String> extractExecutableNames(Executable 
executable) {
+    @VisibleForTesting
+    static @Nullable List<String> extractExecutableNames(Executable 
executable) {
         final int offset;
         if (!Modifier.isStatic(executable.getModifiers())) {
             // remove "this" as first parameter
@@ -824,6 +827,40 @@ public final class ExtractionUtils {
      *   <localVar:index=2 , name=otherLocal2 , desc=J, sig=null, start=L1, 
end=L2>
      * }
      * }</pre>
+     *
+     * <p>If a constructor or method has multiple identical local variables 
that are not initialized
+     * like:
+     *
+     * <pre>{@code
+     * String localVariable;
+     * if (generic == null) {
+     *     localVariable = "null";
+     * } else if (generic < 0) {
+     *     localVariable = "negative";
+     * } else if (generic > 0) {
+     *     localVariable = "positive";
+     * } else {
+     *     localVariable = "zero";
+     * }
+     * }</pre>
+     *
+     * <p>Its local variable table is as follows:
+     *
+     * <pre>{@code
+     * Start  Length  Slot     Name           Signature
+     * 7       3       2     localVariable   Ljava/lang/String;
+     * 22      3       2     localVariable   Ljava/lang/String;
+     * 37      3       2     localVariable   Ljava/lang/String;
+     * 0      69       0     this            ...;
+     * 0      69       1     generic         Ljava/lang/Long;
+     * 43     26       2     localVariable   Ljava/lang/String;
+     * }</pre>
+     *
+     * <p>The method parameters are always at the head in the 'slot' list.
+     *
+     * <p>NOTE: the first parameter may be "this" if the function is not 
static. See more at <a
+     * 
href="https://docs.oracle.com/javase/specs/jvms/se8/html/jvms-3.html";>3.6. 
Receiving
+     * Arguments</a>
      */
     private static class ParameterExtractor extends ClassVisitor {
 
@@ -831,7 +868,7 @@ public final class ExtractionUtils {
 
         private final String methodDescriptor;
 
-        private final List<String> parameterNames = new ArrayList<>();
+        private final Map<Integer, String> parameterNamesWithIndex = new 
TreeMap<>();
 
         ParameterExtractor(Constructor<?> constructor) {
             super(OPCODE);
@@ -844,7 +881,11 @@ public final class ExtractionUtils {
         }
 
         List<String> getParameterNames() {
-            return parameterNames;
+            // method parameters are always at the head in the 'index' list
+            // NOTE: the first parameter may be "this" if the function is not 
static
+            // See more at Chapter "3.6. Receiving Arguments" in
+            // https://docs.oracle.com/javase/specs/jvms/se8/html/jvms-3.html
+            return new ArrayList<>(parameterNamesWithIndex.values());
         }
 
         @Override
@@ -860,7 +901,7 @@ public final class ExtractionUtils {
                             Label start,
                             Label end,
                             int index) {
-                        parameterNames.add(name);
+                        parameterNamesWithIndex.put(index, name);
                     }
                 };
             }
diff --git 
a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/ExtractionUtilsTest.java
 
b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/ExtractionUtilsTest.java
index 80b12fb9a95..c8149345fa9 100644
--- 
a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/ExtractionUtilsTest.java
+++ 
b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/ExtractionUtilsTest.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.table.types.extraction;
 
+import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableList;
+
 import org.junit.jupiter.api.Test;
 
 import java.lang.reflect.GenericArrayType;
@@ -93,6 +95,41 @@ public class ExtractionUtilsTest {
         
assertThat(innerFuture.getActualTypeArguments()[0]).isEqualTo(Long.class);
     }
 
+    @Test
+    void testExtractExecutableNamesWithMultiLocalVariableBlocks() {
+        List<String> expectedParameterNames =
+                ImmutableList.of("generic", "genericFuture", 
"listOfGenericFuture", "array");
+
+        // test the local variable is not initialized at first
+        List<Method> methods =
+                ExtractionUtils.collectMethods(
+                        MultiLocalVariableWithoutInitializationClass.class, 
"method");
+        Method method = methods.get(0);
+        List<String> parameterNames = 
ExtractionUtils.extractExecutableNames(method);
+        assertThat(parameterNames).isEqualTo(expectedParameterNames);
+
+        // test the local variable is initialized at first
+        methods =
+                ExtractionUtils.collectMethods(
+                        MultiLocalVariableBlocksWithInitializationClass.class, 
"method");
+        method = methods.get(0);
+        parameterNames = ExtractionUtils.extractExecutableNames(method);
+        assertThat(parameterNames).isEqualTo(expectedParameterNames);
+    }
+
+    @Test
+    void testExtractExecutableNamesWithParameterNameShadowed() {
+        List<String> expectedParameterNames =
+                ImmutableList.of(
+                        "generic", "result", "genericFuture", 
"listOfGenericFuture", "array");
+        // test the local variable is not initialized at first
+        List<Method> methods =
+                
ExtractionUtils.collectMethods(ParameterNameShadowedClass.class, "method");
+        Method method = methods.get(0);
+        List<String> parameterNames = 
ExtractionUtils.extractExecutableNames(method);
+        assertThat(parameterNames).isEqualTo(expectedParameterNames);
+    }
+
     /** Test function. */
     public static class ClassBase<T> {
 
@@ -114,4 +151,77 @@ public class ExtractionUtilsTest {
 
     /** Test function. */
     public static class FutureClass extends 
ClassBase2<CompletableFuture<Long>> {}
+
+    /**
+     * A test function that contains multi local variable blocks without 
initialization at first.
+     */
+    public static class MultiLocalVariableWithoutInitializationClass extends 
ClassBase<Long> {
+
+        @Override
+        public void method(
+                Long generic,
+                CompletableFuture<Long> genericFuture,
+                List<CompletableFuture<Long>> listOfGenericFuture,
+                Long[] array) {
+            // don't initialize the local variable
+            String localVariable;
+
+            if (generic == null) {
+                localVariable = "null";
+            } else if (generic < 0) {
+                localVariable = "negative";
+            } else if (generic > 0) {
+                localVariable = "positive";
+            } else {
+                localVariable = "zero";
+            }
+
+            // use the local variable
+            System.err.println("localVariable: " + localVariable);
+        }
+    }
+
+    /** A test function that contains multi local variable blocks with 
initialization at first. */
+    public static class MultiLocalVariableBlocksWithInitializationClass 
extends ClassBase<Long> {
+
+        @Override
+        public void method(
+                Long generic,
+                CompletableFuture<Long> genericFuture,
+                List<CompletableFuture<Long>> listOfGenericFuture,
+                Long[] array) {
+            // initialize the local variable
+            String localVariable = "";
+
+            if (generic == null) {
+                localVariable = "null";
+            } else if (generic < 0) {
+                localVariable = "negative";
+            } else if (generic > 0) {
+                localVariable = "positive";
+            } else {
+                localVariable = "zero";
+            }
+
+            // use the local variable
+            System.err.println("localVariable: " + localVariable);
+        }
+    }
+
+    /**
+     * A test function where one function parameter has the same name as a 
class member variable
+     * within another complex function parameter.
+     */
+    public static class ParameterNameShadowedClass {
+
+        @SuppressWarnings("unused")
+        public void method(
+                Long generic,
+                // this `result` has the same name as the class member 
variable in
+                // `CompletableFuture`
+                Object result,
+                CompletableFuture<Long> genericFuture,
+                List<CompletableFuture<Long>> listOfGenericFuture,
+                Long[] array) {}
+    }
 }
diff --git 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java
 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java
index f0623717681..199f2266022 100644
--- 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java
+++ 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java
@@ -1542,6 +1542,26 @@ public class FunctionITCase extends StreamingTestBase {
                 "drop function lowerUdf");
     }
 
+    @Test
+    void testUdfWithMultiLocalVariables() {
+        List<Row> sourceData = Arrays.asList(Row.of(1L, 2L), Row.of(2L, 3L));
+        TestCollectionTableFactory.reset();
+        TestCollectionTableFactory.initData(sourceData);
+
+        tEnv().executeSql(
+                        "CREATE TABLE SourceTable(x BIGINT, y BIGINT) WITH 
('connector' = 'COLLECTION')");
+        tEnv().executeSql(
+                        "CREATE FUNCTION MultiLocalVariables AS '"
+                                + MultiLocalVariableBlocksClass.class.getName()
+                                + "'");
+
+        List<Row> actualRows =
+                CollectionUtil.iteratorToList(
+                        tEnv().executeSql("SELECT MultiLocalVariables(x, y) 
FROM SourceTable")
+                                .collect());
+        assertThat(actualRows).isEqualTo(Arrays.asList(Row.of(2L), 
Row.of(6L)));
+    }
+
     // 
--------------------------------------------------------------------------------------------
     // Test functions
     // 
--------------------------------------------------------------------------------------------
@@ -2139,6 +2159,29 @@ public class FunctionITCase extends StreamingTestBase {
         }
     }
 
+    /** A function that contains a local variable with multi blocks. */
+    public static class MultiLocalVariableBlocksClass extends ScalarFunction {
+
+        public Long eval(Long a, Long b) {
+            long localVariable;
+            if (a == null) {
+                // block 1
+                localVariable = 0;
+            } else if (a == 0) {
+                // block 2
+                localVariable = -1;
+            } else if (b < 1) {
+                // block 3
+                localVariable = -1L * a;
+            } else {
+                // block 4
+                localVariable = a;
+            }
+
+            return localVariable * Optional.ofNullable(b).orElse(0L);
+        }
+    }
+
     private interface FunctionCreator {
         void createFunction(TableEnvironment environment);
     }

Reply via email to