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

ASF GitHub Bot commented on FLINK-6228:
---------------------------------------

Github user sunjincheng121 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3743#discussion_r112620324
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala
 ---
    @@ -49,6 +57,128 @@ case class Call(functionName: String, args: 
Seq[Expression]) extends Expression
     }
     
     /**
    +  * Over expression for calcite over transform.
    +  *
    +  * @param agg             over-agg expression
    +  * @param aggAlias        agg alias for following `select()` clause.
    +  * @param overWindowAlias over window alias
    +  * @param overWindow      over window
    +  */
    +case class OverCall(
    --- End diff --
    
    Can we check `partitionBy` as following(In `toRexNode `method):
    ```
    overWindow.partitionBy.foreach {
          x =>
            val partitionKey = 
relBuilder.field(x.asInstanceOf[UnresolvedFieldReference].name)
            if (!FlinkTypeFactory.toTypeInfo(partitionKey.getType).isKeyType) {
              ValidationException(
                s"expression $partitionKey cannot be used as a partition key 
expression " +
                "because it's not a valid key type which must be hashable and 
comparable")
            }
    ```
    Because It's not work(can not resolve the partition key field) when I add 
`partitionBy` into children, and add check logic into `validateInput` method.
    `def children: Seq[Expression] = Seq(agg) ++ overWindow.partitionBy`
    Is there something wrong in the code ?



> Integrating the OVER windows in the Table API
> ---------------------------------------------
>
>                 Key: FLINK-6228
>                 URL: https://issues.apache.org/jira/browse/FLINK-6228
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API & SQL
>            Reporter: sunjincheng
>            Assignee: sunjincheng
>
> Syntax:
> {code}
> table
>    .overWindows(
>     (Rows|Range [ partitionBy value_expression , ... [ n ]] [ orderBy 
> order_by_expression] 
>       (preceding  
> UNBOUNDED|value_specification.(rows|milli|second|minute|hour|day|month|year)|CURRENTROW)
>      [following 
> UNBOUNDED|value_specification.(rows|milli|second|minute|hour|day|month|year)|CURRENTROW]
>     as alias,...[n])
>    )
>   .select( [col1,...[n]], (agg(col1) OVER overWindowAlias, … [n])
> {code}
> Implement restrictions:
> * All OVER clauses in the same SELECT clause must be exactly the same.
> * The PARTITION BY clause is optional (no partitioning results in single 
> threaded execution).
> * The ORDER BY Before the 
> [FLINK-5884|https://issues.apache.org/jira/browse/FLINK-5884]  implementation 
> orderBy may only have ‘rowtime/’proctime(for stream)/‘specific-time-field(for 
> batch).
> * FOLLOWING is not supported.
> User interface design document [See | 
> https://docs.google.com/document/d/13Z-Ovx3jwtmzkSweJyGkMy751BouhuJ29Y1CTNZt2DI/edit#]



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to