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

lincoln lee commented on FLINK-27519:
-------------------------------------

[~paul8263] Current fix uses a unified renaming introduced by FLINK-22121.

But as you mentioned above, there's something wrong during the plan rewriting, 
and I spent some time to debug it, root cause is in calcite's 
`CalcRelSplitter`#execute, when split a project which contains over agg call 
into a overwindow and calc, the new overwindow node lost the original alias 
name, but semantically it doesn't mean it's wrong because the following new 
calc's output use the expected alias name. So if the nested over agg query 
works fine in calcite, we can keep the current renaming code. (encounters some 
unexpected building error in my local calcite project, will take some time to 
verify it later.)

> Fix duplicates names when there are multiple levels of over window aggregate
> ----------------------------------------------------------------------------
>
>                 Key: FLINK-27519
>                 URL: https://issues.apache.org/jira/browse/FLINK-27519
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / Planner
>    Affects Versions: 1.15.0
>            Reporter: Feng Jin
>            Assignee: lincoln lee
>            Priority: Major
>             Fix For: 2.0.0
>
>
> A similar  issue like 
> [FLINK-22121|https://issues.apache.org/jira/browse/FLINK-22121]
> And can be reproduced by adding this unit test 
> org.apache.flink.table.planner.plan.stream.sql.agg.GroupWindowTest#testWindowAggregateWithAnotherWindowAggregate
> {code:java}
> //代码占位符
> @Test
> def testWindowAggregateWithAnotherWindowAggregate(): Unit = {
>   val sql =
>     """
>       |SELECT CAST(pv AS INT) AS pv, CAST(uv AS INT) AS uv FROM (
>       |  SELECT *, count(distinct(c)) over (partition by a order by b desc) 
> AS uv
>       |  FROM (
>       |    SELECT *, count(*) over (partition by a, c order by b desc) AS pv
>       |    FROM MyTable
>       |  )
>       |)
>       |""".stripMargin
>   util.verifyExecPlan(sql)
> } {code}
> The error message : 
>  
>  
> {code:java}
> //代码占位符
> org.apache.flink.table.api.ValidationException: Field names must be unique. 
> Found duplicates: [w0$o0]    at 
> org.apache.flink.table.types.logical.RowType.validateFields(RowType.java:273)
>     at org.apache.flink.table.types.logical.RowType.<init>(RowType.java:158)
>     at org.apache.flink.table.types.logical.RowType.of(RowType.java:298)
>     at org.apache.flink.table.types.logical.RowType.of(RowType.java:290)
>     at 
> org.apache.flink.table.planner.calcite.FlinkTypeFactory$.toLogicalRowType(FlinkTypeFactory.scala:663)
>     at 
> org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregate.translateToExecNode(StreamPhysicalOverAggregate.scala:57)
>     at 
> org.apache.flink.table.planner.plan.nodes.exec.ExecNodeGraphGenerator.generate(ExecNodeGraphGenerator.java:74)
>     at 
> org.apache.flink.table.planner.plan.nodes.exec.ExecNodeGraphGenerator.generate(ExecNodeGraphGenerator.java:71)
>  {code}
>  
> I think we can add come logical in  FlinkLogicalOverAggregate  to avoid 
> duplicate names of  output rowType. 
>  
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to