[ 
https://issues.apache.org/jira/browse/BEAM-7724?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16891330#comment-16891330
 ] 

Rui Wang commented on BEAM-7724:
--------------------------------

I remember the code path you used had an issue that if you define two eval, 
only one will be registered (that code path does not allow function 
overloading).

You can reproduce what this JIRA by:

You can first add the following piece of code to BuiltinStringFunctions.java


{code:java}
  @UDF(
      funcName = "TESTFN",
      parameterArray = {TypeName.STRING},
      returnType = TypeName.INT64)
  public Long leng(String str) {
    if (str == null) {
      return null;
    }
    return (long) str.length();
  }

  @UDF(
      funcName = "TESTFN",
      parameterArray = {TypeName.BYTES},
      returnType = TypeName.INT64)
  public Long leng(byte[] bytes) {
    if (bytes == null) {
      return null;
    }
    return (long) bytes.length;
  }
{code}

Then as a unit test, run this:


{code:java}
    Schema inputSchema =
        Schema.builder().addNullableField("c3", FieldType.STRING).build();

    PCollection<Row> inputTable =
        pipeline.apply(
            Create.of(Row.withSchema(inputSchema).addValue(null).build())
                .withRowSchema(inputSchema));

    PCollection<Row> outputStream =
        inputTable.apply(SqlTransform.query("select TESTFN(CAST(NULL AS 
VARCHAR)) from PCOLLECTION"));

    pipeline.run();
{code}

After that, you should be able to see the critical exception message:


{code:java}
Could not compile CalcFn: {
  c.output(org.apache.beam.sdk.values.Row.withSchema(outputSchema).addValue(new 
org.apache.beam.sdk.extensions.sql.impl.udf.BuiltinStringFunctions().leng(null)).build());
}

java.lang.RuntimeException: Could not compile CalcFn: {
  c.output(org.apache.beam.sdk.values.Row.withSchema(outputSchema).addValue(new 
org.apache.beam.sdk.extensions.sql.impl.udf.BuiltinStringFunctions().leng(null)).build());
}
{code}


This shows what this JIRA talks about:

If there are two UDF that will be both registered (note that there is an 
auto-registration code path to register such UDF), and if two UDF is actually 
function overloading, which is  Long leng(byte[] bytes) and  Long leng(String 
str), Janio will fail to execute it when parameter is a null as a constant.


The resolution is to make sure [generated 
code|https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java#L190]
 generate something like BuiltinStringFunctions().leng((String)null))




> Codegen should cast(null) to a type to match exact function signature
> ---------------------------------------------------------------------
>
>                 Key: BEAM-7724
>                 URL: https://issues.apache.org/jira/browse/BEAM-7724
>             Project: Beam
>          Issue Type: Improvement
>          Components: dsl-sql
>            Reporter: Rui Wang
>            Assignee: sridhar Reddy
>            Priority: Major
>
> If there are two function signatures for the same function name, when input 
> parameter is null, Janino will throw exception due to vagueness:
> A(String)
> A(Integer)
> Janino does not know how to match A(null).  



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

Reply via email to