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

ASF GitHub Bot commented on FLINK-7309:
---------------------------------------

Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4479#discussion_r132690201
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala
 ---
    @@ -1172,6 +1172,13 @@ abstract class CodeGenerator(
         val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldType)
         val defaultValue = primitiveDefaultValue(fieldType)
     
    +    // the initial value for result.
    +    val initValue = fieldType match {
    --- End diff --
    
    I had to fix the same issue for FLINK-7337 in PR #4488: 
https://github.com/apache/flink/pull/4488/files#diff-e05ff53adcc1407715fe09572ee092e0L1196
    
    I don't think we need this `initValue`. Especially, we should not introduce 
a special case for `String`. The problem with `String` was a buggy 
implementation of `concat` that did not respect the `isNull` field but relied 
on `resultTerm == null`. I fix this issue in my PR so, we can avoid the special 
case for `String` here.



> NullPointerException in CodeGenUtils.timePointToInternalCode() generated code
> -----------------------------------------------------------------------------
>
>                 Key: FLINK-7309
>                 URL: https://issues.apache.org/jira/browse/FLINK-7309
>             Project: Flink
>          Issue Type: Bug
>          Components: Local Runtime, Table API & SQL
>    Affects Versions: 1.3.1
>            Reporter: Liangliang Chen
>            Priority: Critical
>
> The code generated by CodeGenUtils.timePointToInternalCode() will cause a 
> NullPointerException when SQL table field type is `TIMESTAMP` and the field 
> value is `null`.
> Example for reproduce:
> {code}
> object StreamSQLExample {
>   def main(args: Array[String]): Unit = {
>     val env = StreamExecutionEnvironment.getExecutionEnvironment
>     val tEnv = TableEnvironment.getTableEnvironment(env)
>     // null field value
>     val orderA: DataStream[Order] = env.fromCollection(Seq(
>       Order(null, "beer", 3)))
>       
>     tEnv.registerDataStream("OrderA", orderA, 'ts, 'product, 'amount)
>     val result = tEnv.sql("SELECT * FROM OrderA")
>     result.toAppendStream[Order].print()
>     
>     env.execute()
>   }
>   case class Order(ts: Timestamp, product: String, amount: Int)
> }
> {code}
> In the above example, timePointToInternalCode() will generated some 
> statements like this:
> {code}
> ...
>           long result$1 = 
> org.apache.calcite.runtime.SqlFunctions.toLong((java.sql.Timestamp) in1.ts());
>           boolean isNull$2 = (java.sql.Timestamp) in1.ts() == null;
> ...
> {code}
> so, the NPE will happen when in1.ts() is null.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to