[
https://issues.apache.org/jira/browse/FLINK-12249?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16885902#comment-16885902
]
Hequn Cheng commented on FLINK-12249:
-------------------------------------
[~sunjincheng121] Thanks for your suggestions. I think you make a good point
here.
The behavior of Calcite is similar to the common databases thus I think we
should fix in our side. One way to fix this problem is not to remove the window
group key during logical optimization, i.e., don't perform
{{.except(ImmutableBitSet.of(windowExprIdx))}} in
{{LogicalWindowAggregateRule}}. During physical optimization, we can then adapt
the current {{DataStreamGroupWindowAggregate}} and
{{DataSetWindowAggregateRule}} to remove the window group key field, as it is
meaningless for physical node.
Blink-planner also has the same problem.
> Type equivalence check fails for Window Aggregates
> --------------------------------------------------
>
> Key: FLINK-12249
> URL: https://issues.apache.org/jira/browse/FLINK-12249
> Project: Flink
> Issue Type: Bug
> Components: Table SQL / Legacy Planner, Tests
> Affects Versions: 1.9.0
> Reporter: Dawid Wysakowicz
> Assignee: Hequn Cheng
> Priority: Blocker
> Fix For: 1.9.0
>
>
> Creating Aggregate node fails in rules: {{LogicalWindowAggregateRule}} and
> {{ExtendedAggregateExtractProjectRule}} if the only grouping expression is a
> window and
> we compute aggregation on NON NULLABLE field.
> The root cause for that, is how return type inference strategies in calcite
> work and how we handle window aggregates. Take
> {{org.apache.calcite.sql.type.ReturnTypes#AGG_SUM}} as an example, based on
> {{groupCount}} it adjusts type nullability based on groupCount.
> Though we pass a false information as we strip down window aggregation from
> groupSet (in {{LogicalWindowAggregateRule}}).
> One can reproduce this problem also with a unit test like this:
> {code}
> @Test
> def testTumbleFunction2() = {
>
> val innerQuery =
> """
> |SELECT
> | CASE a WHEN 1 THEN 1 ELSE 99 END AS correct,
> | rowtime
> |FROM MyTable
> """.stripMargin
> val sql =
> "SELECT " +
> " SUM(correct) as cnt, " +
> " TUMBLE_START(rowtime, INTERVAL '15' MINUTE) as wStart " +
> s"FROM ($innerQuery) " +
> "GROUP BY TUMBLE(rowtime, INTERVAL '15' MINUTE)"
> val expected = ""
> streamUtil.verifySql(sql, expected)
> }
> {code}
> This causes e2e tests to fail:
> https://travis-ci.org/apache/flink/builds/521183361?utm_source=slack&utm_medium=notificationhttps://travis-ci.org/apache/flink/builds/521183361?utm_source=slack&utm_medium=notification
--
This message was sent by Atlassian JIRA
(v7.6.14#76016)