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 c788a71905ee8..4db958749eca1 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 @@ -956,22 +956,31 @@ private static ClassReader getClassReader(Class cls) { *

NOTE: the first parameter may be "this" if the function is not static. See more at 3.6. Receiving * Arguments + * + *

The visitor matches methods by both name and descriptor to avoid visiting synthetic + * methods generated by the compiler for lambda expressions (e.g. {@code lambda$eval$0}). These + * synthetic methods may share the same descriptor as the target method and would otherwise + * corrupt the extracted parameter names. */ private static class ParameterExtractor extends ClassVisitor { private static final int OPCODE = Opcodes.ASM9; + private final String methodName; + private final String methodDescriptor; private final Map parameterNamesWithIndex = new TreeMap<>(); ParameterExtractor(Constructor constructor) { super(OPCODE); + methodName = ""; methodDescriptor = getConstructorDescriptor(constructor); } ParameterExtractor(Method method) { super(OPCODE); + methodName = method.getName(); methodDescriptor = getMethodDescriptor(method); } @@ -986,7 +995,7 @@ List getParameterNames() { @Override public MethodVisitor visitMethod( int access, String name, String descriptor, String signature, String[] exceptions) { - if (descriptor.equals(methodDescriptor)) { + if (name.equals(methodName) && descriptor.equals(methodDescriptor)) { return new MethodVisitor(OPCODE) { @Override public void visitLocalVariable( 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 431f0c23d91ef..5229729a7d262 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 @@ -30,6 +30,7 @@ import java.lang.reflect.Type; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.function.Supplier; import static org.assertj.core.api.AssertionsForClassTypes.assertThat; @@ -241,4 +242,26 @@ public void method( List> listOfGenericFuture, Long[] array) {} } + + /** + * Verifies that {@link ExtractionUtils#extractExecutableNames} returns correct parameter names + * when a method contains a lambda that captures its own parameters, producing a synthetic + * method with the same bytecode descriptor. + */ + @Test + void testExtractExecutableNamesWithLambdaCapture() { + Method method = ExtractionUtils.collectMethods(LambdaCaptureClass.class, "eval").get(0); + assertThat(ExtractionUtils.extractExecutableNames(method)) + .isEqualTo(ImmutableList.of("id", "field")); + } + + /** A single-method class where the lambda captures all parameters of the enclosing method. */ + public static class LambdaCaptureClass { + + @SuppressWarnings("unused") + public String eval(Long id, String field) { + Supplier supplier = () -> String.valueOf(id) + field; + return supplier.get(); + } + } } diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/TypeInferenceExtractorTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/TypeInferenceExtractorTest.java index 4d7334fd82266..fe99447138e98 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/TypeInferenceExtractorTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/TypeInferenceExtractorTest.java @@ -230,6 +230,17 @@ private static Stream functionSpecs() { TypeStrategies.explicit( DataTypes.BIGINT().notNull().bridgedTo(long.class))), // --- + // test eval method with lambda capture whose synthetic method shares the + // same bytecode descriptor, previously causing "Argument name conflict" + TestSpec.forScalarFunction( + "ScalarFunctionWithLambdaCapture", + ScalarFunctionWithLambdaCapture.class) + .expectStaticArgument( + StaticArgument.scalar("id", DataTypes.BIGINT(), false)) + .expectStaticArgument( + StaticArgument.scalar("field", DataTypes.STRING(), false)) + .expectOutput(TypeStrategies.explicit(DataTypes.STRING())), + // --- // test overloaded arguments extraction async TestSpec.forAsyncScalarFunction(OverloadedFunctionAsync.class) .expectOutputMapping( @@ -1544,6 +1555,13 @@ public long eval(String s) { } } + private static class ScalarFunctionWithLambdaCapture extends ScalarFunction { + public String eval(Long id, String field) { + Supplier supplier = () -> String.valueOf(id) + field; + return supplier.get(); + } + } + private static class VarArgFunction extends ScalarFunction { public String eval(int i, int... more) { return null;