[
https://issues.apache.org/jira/browse/FLINK-20765?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Rui Li updated FLINK-20765:
---------------------------
Description:
Suppose we have a function that takes a BOOLEAN NOT NULL parameter:
{code}
/**
* A function that takes BOOLEAN NOT NULL.
*/
public static class BoolEcho extends ScalarFunction {
public Boolean eval(@DataTypeHint("BOOLEAN NOT NULL") Boolean
b) {
return b;
}
}
{code}
Then the following test case will fail:
{code}
CREATE TABLE SourceTable(x INT NOT NULL,y INT) WITH ('connector' =
'COLLECTION');
SELECT BoolEcho(x=1 and y is null) FROM SourceTable;
{code}
with exception:
{noformat}
org.apache.flink.table.planner.codegen.CodeGenException: Mismatch of function's
argument data type 'BOOLEAN NOT NULL' and actual argument type 'BOOLEAN'.
at
org.apache.flink.table.planner.codegen.calls.BridgingFunctionGenUtil$$anonfun$verifyArgumentTypes$1.apply(BridgingFunctionGenUtil.scala:323)
at
org.apache.flink.table.planner.codegen.calls.BridgingFunctionGenUtil$$anonfun$verifyArgumentTypes$1.apply(BridgingFunctionGenUtil.scala:320)
at
scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
at
org.apache.flink.table.planner.codegen.calls.BridgingFunctionGenUtil$.verifyArgumentTypes(BridgingFunctionGenUtil.scala:320)
at
org.apache.flink.table.planner.codegen.calls.BridgingFunctionGenUtil$.generateFunctionAwareCallWithDataType(BridgingFunctionGenUtil.scala:95)
at
org.apache.flink.table.planner.codegen.calls.BridgingFunctionGenUtil$.generateFunctionAwareCall(BridgingFunctionGenUtil.scala:65)
at
org.apache.flink.table.planner.codegen.calls.BridgingSqlFunctionCallGen.generate(BridgingSqlFunctionCallGen.scala:62)
at
org.apache.flink.table.planner.codegen.ExprCodeGenerator.generateCallExpression(ExprCodeGenerator.scala:832)
at
org.apache.flink.table.planner.codegen.ExprCodeGenerator.visitCall(ExprCodeGenerator.scala:529)
at
org.apache.flink.table.planner.codegen.ExprCodeGenerator.visitCall(ExprCodeGenerator.scala:56)
at org.apache.calcite.rex.RexCall.accept(RexCall.java:174)
at
org.apache.flink.table.planner.codegen.ExprCodeGenerator.generateExpression(ExprCodeGenerator.scala:155)
{noformat}
> ScalarOperatorGens doesn't set proper nullability for result type of
> generated expressions
> ------------------------------------------------------------------------------------------
>
> Key: FLINK-20765
> URL: https://issues.apache.org/jira/browse/FLINK-20765
> Project: Flink
> Issue Type: Bug
> Components: Table SQL / Planner
> Reporter: Rui Li
> Priority: Major
>
> Suppose we have a function that takes a BOOLEAN NOT NULL parameter:
> {code}
> /**
> * A function that takes BOOLEAN NOT NULL.
> */
> public static class BoolEcho extends ScalarFunction {
> public Boolean eval(@DataTypeHint("BOOLEAN NOT NULL") Boolean
> b) {
> return b;
> }
> }
> {code}
> Then the following test case will fail:
> {code}
> CREATE TABLE SourceTable(x INT NOT NULL,y INT) WITH ('connector' =
> 'COLLECTION');
> SELECT BoolEcho(x=1 and y is null) FROM SourceTable;
> {code}
> with exception:
> {noformat}
> org.apache.flink.table.planner.codegen.CodeGenException: Mismatch of
> function's argument data type 'BOOLEAN NOT NULL' and actual argument type
> 'BOOLEAN'.
> at
> org.apache.flink.table.planner.codegen.calls.BridgingFunctionGenUtil$$anonfun$verifyArgumentTypes$1.apply(BridgingFunctionGenUtil.scala:323)
> at
> org.apache.flink.table.planner.codegen.calls.BridgingFunctionGenUtil$$anonfun$verifyArgumentTypes$1.apply(BridgingFunctionGenUtil.scala:320)
> at
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
> at
> org.apache.flink.table.planner.codegen.calls.BridgingFunctionGenUtil$.verifyArgumentTypes(BridgingFunctionGenUtil.scala:320)
> at
> org.apache.flink.table.planner.codegen.calls.BridgingFunctionGenUtil$.generateFunctionAwareCallWithDataType(BridgingFunctionGenUtil.scala:95)
> at
> org.apache.flink.table.planner.codegen.calls.BridgingFunctionGenUtil$.generateFunctionAwareCall(BridgingFunctionGenUtil.scala:65)
> at
> org.apache.flink.table.planner.codegen.calls.BridgingSqlFunctionCallGen.generate(BridgingSqlFunctionCallGen.scala:62)
> at
> org.apache.flink.table.planner.codegen.ExprCodeGenerator.generateCallExpression(ExprCodeGenerator.scala:832)
> at
> org.apache.flink.table.planner.codegen.ExprCodeGenerator.visitCall(ExprCodeGenerator.scala:529)
> at
> org.apache.flink.table.planner.codegen.ExprCodeGenerator.visitCall(ExprCodeGenerator.scala:56)
> at org.apache.calcite.rex.RexCall.accept(RexCall.java:174)
> at
> org.apache.flink.table.planner.codegen.ExprCodeGenerator.generateExpression(ExprCodeGenerator.scala:155)
> {noformat}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)