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

2017-08-04 Thread Liangliang Chen (JIRA)

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

Liangliang Chen commented on FLINK-7309:


@yestinchen ???

> 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)


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

2017-08-04 Thread Liangliang Chen (JIRA)

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

Liangliang Chen edited comment on FLINK-7309 at 8/4/17 9:32 AM:


hi, [~twalthr], I'm not very familiar with Scala-Lang, so I write a test 
program in 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 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 result = tEnv.toAppendStream(table, Row.class);
result.print();

env.execute();
  }
}
{code}

I use a {{Row}} type in this example and the exception occurs again.


was (Author: llchen):
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 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 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)


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

2017-08-03 Thread Liangliang Chen (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-7309?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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 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 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)


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

2017-08-03 Thread Liangliang Chen (JIRA)

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

Liangliang Chen commented on FLINK-7309:


Thanks [~fhueske], thank you for the quick reply, and thank you [~twalthr] to 
confirm it. I'm glad to fix it and I will open a pull request later!

> 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)


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

2017-08-03 Thread Liangliang Chen (JIRA)

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

Liangliang Chen commented on FLINK-7309:


hi, [~fhueske], Can you help to review this?

I think we can fix it by added null check if confirm that it is a bug.

> 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)


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

2017-08-03 Thread Liangliang Chen (JIRA)

 [ 
https://issues.apache.org/jira/browse/FLINK-7309?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Liangliang Chen updated FLINK-7309:
---
Component/s: Local Runtime

> 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)


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

2017-07-31 Thread Liangliang Chen (JIRA)

 [ 
https://issues.apache.org/jira/browse/FLINK-7309?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Liangliang Chen updated FLINK-7309:
---
Description: 
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.

  was:
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.


> 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:
> {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)


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

2017-07-31 Thread Liangliang Chen (JIRA)
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)