[ https://issues.apache.org/jira/browse/BEAM-7724?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16891266#comment-16891266 ]
sridhar Reddy edited comment on BEAM-7724 at 7/31/19 8:08 PM: -------------------------------------------------------------- [~amaliujia] I tested this issue a little bit and found out that the problem may lie in the order of methods and it is possible that currently only one UDF works/accepted(I need to debug further) and only the first method gets accepted. It seems to work fine with null as input. Ex: public static class SFunction implements BeamSqlUdf { public static int eval (Double a) { if (a==null) return 444; else return 14; } public static int eval (Integer a) { if (a==null)) return 444; else return a*2; } } Schema type = Schema.builder().addInt32Field("c1").addStringField("c2").addNullableField("c3", Schema.FieldType.DOUBLE).build(); the following works fine even when c3 is null ---- PCollection<Row> outputStream = inputTable.apply(SqlTransform.query("select c1, c2, eval(c3) from PCOLLECTION where c1 = 81 ") .registerUdf("eval",SFunction.class)); ------ The following doesn't work unless eval(Integer a) method comes first. ------------ PCollection<Row> outputStream = inputTable.apply(SqlTransform.query("select eval(c1), c2, c3 from PCOLLECTION where c1 = 81 ") .registerUdf("eval",SFunction.class)); ---- Let me know your thoughts. was (Author: sridharg): [~amaliujia] I tested this issue a little bit and found out that the problem may lie in the order of methods and it is possible that currently only one UDF works/accepted(I need to debug further) and only the first method gets accepted. It seems to work fine with null as input. Ex: public static class SFunction implements BeamSqlUdf { public static int eval (Double a) { if (a==null) return 444; else return 14; } public static int eval (Integer a) { if (a.equals(null)) return 444; else return a*2; } } Schema type = Schema.builder().addInt32Field("c1").addStringField("c2").addNullableField("c3", Schema.FieldType.DOUBLE).build(); the following works fine even when c3 is null ---- PCollection<Row> outputStream = inputTable.apply(SqlTransform.query("select c1, c2, eval(c3) from PCOLLECTION where c1 = 81 ") .registerUdf("eval",SFunction.class)); ------ The following doesn't work unless eval(Integer a) method comes first. ------------ PCollection<Row> outputStream = inputTable.apply(SqlTransform.query("select eval(c1), c2, c3 from PCOLLECTION where c1 = 81 ") .registerUdf("eval",SFunction.class)); ----- Let me know your thoughts. > 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)