lincoln lee created FLINK-35498:
-----------------------------------

             Summary: Unexpected argument name conflict error when do extract 
method params from udf
                 Key: FLINK-35498
                 URL: https://issues.apache.org/jira/browse/FLINK-35498
             Project: Flink
          Issue Type: Bug
          Components: Table SQL / Planner
    Affects Versions: 1.19.0, 1.20.0
            Reporter: lincoln lee
            Assignee: xuyang


Follow the steps to reproduce the error:

test case:

{code}
util.addTemporarySystemFunction("myudf", new TestXyz)
util.tableEnv.explainSql("select myudf(f1, f2) from t")
{code}

 

udf: TestXyz 

{code}
public class TestXyz extends ScalarFunction {
public String eval(String s1, String s2) {
String localV1;

if (s1 == null) {
if (s2 != null) {
localV1 = s2;
} else {
localV1 = s2 + s1;
}
} else {
if ("xx".equals(s2)) {
localV1 = s1.length() >= s2.length() ? s1 : s2;
} else {
localV1 = s1;
}
}
if (s1 == null) {
return s2 + localV1;
}
if (s2 == null) {
return s1;
}
return s1.length() >= s2.length() ? s1 + localV1 : s2;
}
}
{code}

 

error stack:

{code}

Caused by: org.apache.flink.table.api.ValidationException: Unable to extract a 
type inference from method:
public java.lang.String 
org.apache.flink.table.planner.runtime.utils.TestXyz.eval(java.lang.String,java.lang.String)
    at 
org.apache.flink.table.types.extraction.ExtractionUtils.extractionError(ExtractionUtils.java:362)
    at 
org.apache.flink.table.types.extraction.BaseMappingExtractor.extractResultMappings(BaseMappingExtractor.java:154)
    at 
org.apache.flink.table.types.extraction.BaseMappingExtractor.extractOutputMapping(BaseMappingExtractor.java:100)
    ... 53 more
Caused by: org.apache.flink.table.api.ValidationException: Argument name 
conflict, there are at least two argument names that are the same.
    at 
org.apache.flink.table.types.extraction.ExtractionUtils.extractionError(ExtractionUtils.java:362)
    at 
org.apache.flink.table.types.extraction.ExtractionUtils.extractionError(ExtractionUtils.java:357)
    at 
org.apache.flink.table.types.extraction.FunctionSignatureTemplate.of(FunctionSignatureTemplate.java:73)
    at 
org.apache.flink.table.types.extraction.BaseMappingExtractor.lambda$createParameterSignatureExtraction$9(BaseMappingExtractor.java:381)
    at 
org.apache.flink.table.types.extraction.BaseMappingExtractor.putExtractedResultMappings(BaseMappingExtractor.java:298)
    at 
org.apache.flink.table.types.extraction.BaseMappingExtractor.collectMethodMappings(BaseMappingExtractor.java:244)
    at 
org.apache.flink.table.types.extraction.BaseMappingExtractor.extractResultMappings(BaseMappingExtractor.java:137)
    ... 54 more

{code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to