[
https://issues.apache.org/jira/browse/FLINK-6228?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15975987#comment-15975987
]
ASF GitHub Bot commented on FLINK-6228:
---------------------------------------
GitHub user sunjincheng121 opened a pull request:
https://github.com/apache/flink/pull/3743
[FLINK-6228][table] Integrating the OVER windows in the Table API (st…
In this PR I had integrating the OVER windows in the Table API,
Implementation of the syntax and use examples are as follows:
* Syntax:
```
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])
```
* examples:
```
// Rows clause
table
.window(Over partitionBy 'c orderBy 'rowTime preceding 2.rows as 'w1)
.select(
'c,
'b.count over 'w1 as 'countB,
'e.sum over 'w1 as 'sumE)
// Range clause
table
.window(Over partitionBy 'c orderBy 'rowTime preceding 2.milli as 'w1)
.select(
'c,
'b.count over 'w1 as 'countB,
'e.sum over 'w1 as 'sumE)
```
* More detail Info :
https://docs.google.com/document/d/13Z-Ovx3jwtmzkSweJyGkMy751BouhuJ29Y1CTNZt2DI/edit#
NOTE: The documentation of the OVER tableAPI not included in this PR.
- [x] General
- The pull request references the related JIRA issue
("[FLINK-6228][table] Integrating the OVER windows in the Table API")
- The pull request addresses only one issue
- Each commit in the PR has a meaningful commit message (including the
JIRA id)
- [ ] Documentation
- Documentation has been added for new functionality
- Old documentation affected by the pull request has been updated
- JavaDoc for public methods has been added
- [x] Tests & Build
- Functionality added by the pull request is covered by tests
- `mvn clean verify` has been executed successfully locally or a Travis
build has passed
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/sunjincheng121/flink FLINK-6228-PR
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/flink/pull/3743.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #3743
----
commit 03d4153be93d505cdb47d174e5aafe10eb93a45f
Author: sunjincheng121 <[email protected]>
Date: 2017-04-13T09:36:18Z
[FLINK-6228][table] Integrating the OVER windows in the Table API (stream)
----
> 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)