[ https://issues.apache.org/jira/browse/FLINK-6228?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15977665#comment-15977665 ]
ASF GitHub Bot commented on FLINK-6228: --------------------------------------- Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3743#discussion_r112456197 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala --- @@ -810,6 +810,47 @@ class Table( new WindowedTable(this, window) } + /** + * Groups the records of a table by assigning them to windows defined by a time or row interval. + * + * For streaming tables of infinite size, grouping into windows is required to define finite + * groups on which over-based aggregates can be computed. + * + * Over window for batch tables are currently not supported. + * + * @param overWindows windows that specifies how elements are grouped. + * @return Over windowed table + */ + def window(overWindows: OverWindow*): OverWindowedTable = { + + if (tableEnv.isInstanceOf[BatchTableEnvironment]) { + throw TableException("Over window for batch tables are currently not supported.") + } else { + overWindows.foreach { overWindow => + val orderName = overWindow.orderBy.asInstanceOf[UnresolvedFieldReference].name --- End diff -- The other operators are validated later. Can you add this check to This check to `OverCall.validateInput()`? Please add tests to validate that the checks work. > 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)