[
https://issues.apache.org/jira/browse/FLINK-7309?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16112834#comment-16112834
]
Liangliang Chen commented on FLINK-7309:
----------------------------------------
hi, [~twalthr], I'm not very familiar with Scala-Lang, so I rewrite a test
example with Java as below:
{code}
public class TestNullSQL {
public static void main(String[] args) throws Exception {
// set up execution environment
StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
StreamTableEnvironment tEnv =
StreamTableEnvironment.getTableEnvironment(env);
TypeInformation<?>[] types = {BasicTypeInfo.INT_TYPE_INFO,
SqlTimeTypeInfo.TIMESTAMP};
String names[] = {"id", "ts"};
RowTypeInfo typeInfo = new RowTypeInfo(types, names);
// we assign a null value here!!
DataStream<Row> input = env.fromElements(Row.of(1001,
null)).returns(typeInfo);
tEnv.registerDataStream("test_table", input);
Table table = tEnv.sql("SELECT id, ts FROM test_table");
DataStream<Row> result = tEnv.toAppendStream(table, Row.class);
result.print();
env.execute();
}
}
{code}
I use a row type in this example and the exception will still happens.The Row
data type supports an arbitrary number of fields and fields with
{quote}null{quote} values, so I think the generated code has some problems. And
what do you think about?
> 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)