You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Liangliang Chen (JIRA)" <ji...@apache.org> on 2017/07/31 13:17:00 UTC

[jira] [Created] (FLINK-7309) NullPointerException in CodeGenUtils.timePointToInternalCode() generated code

Liangliang Chen created FLINK-7309:
--------------------------------------

             Summary: NullPointerException in CodeGenUtils.timePointToInternalCode() generated code
                 Key: FLINK-7309
                 URL: https://issues.apache.org/jira/browse/FLINK-7309
             Project: Flink
          Issue Type: Bug
          Components: 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:
{quote}
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)
}
{quote}

In the above example, timePointToInternalCode() will generated some statements like this:
{quote}
...
          long result$1 = org.apache.calcite.runtime.SqlFunctions.toLong((java.sql.Timestamp) in1.ts());
          boolean isNull$2 = (java.sql.Timestamp) in1.ts() == null;
...
{quote}

so, the NPE will happen when in1.ts() is null.



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