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

[jira] [Comment Edited] (FLINK-7177) DataSetAggregateWithNullValuesRule fails creating null literal for non-nullable type

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

Fabian Hueske edited comment on FLINK-7177 at 7/14/17 10:26 AM:
----------------------------------------------------------------

I looked into this issue, the problem is caused by the {{DataSetAggregateWithNullValuesRule}}, which tries to create a {{null}} literal for a non-nullable data type (field {{a}} of type {{RelRecordType}} in the given example).
The purpose of the rule is to compute correct results for non-grouped aggregates if the input table is empty. For that it unions the input table with another table that has a single record with null literals. However, creating a null literal for a non-nullable data type fails.


was (Author: fhueske):
I looked into this issue, the problem is caused by the {{DataSetAggregateWithNullValuesRule}}, which try to create a {{null}} literal for a non-nullable data type (a {{RelRecordType}}, field {{a}} in the given example).
The purpose of the rule is to compute correct results for non-grouped aggregates if the input table is empty. For that it unions the input table with another table that has a single record with null literals. However, creating a null literal for a non-nullable data type fails.

> DataSetAggregateWithNullValuesRule fails creating null literal for non-nullable type
> ------------------------------------------------------------------------------------
>
>                 Key: FLINK-7177
>                 URL: https://issues.apache.org/jira/browse/FLINK-7177
>             Project: Flink
>          Issue Type: Bug
>          Components: Table API & SQL
>    Affects Versions: 1.3.1
>            Reporter: Rong Rong
>
> For example:
> {code:title=flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/AggregationsITCase.scala|borderStyle=solid}
> @Test
>   def testTableAggregationWithMultipleTableAPI(): Unit = {
>     val env = ExecutionEnvironment.getExecutionEnvironment
>     val tEnv = TableEnvironment.getTableEnvironment(env, config)
>     val inputTable = CollectionDataSets.getSmallNestedTupleDataSet(env).toTable(tEnv, 'a, 'b)
>     tEnv.registerDataSet("MyTable", inputTable)
>     val result = tEnv.scan("MyTable")
>       .where('a.get("_1") > 0)
>       .select('a.get("_1").avg, 'a.get("_2").sum, 'b.count)
>     val expected = "2,6,3"
>     val results = result.toDataSet[Row].collect()
>     TestBaseUtils.compareResultAsText(results.asJava, expected)
>   }
> {code}
> Details can be found in: https://github.com/apache/flink/compare/master...walterddr:bug_report_sql_query_result_consume_by_table_api



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