[
https://issues.apache.org/jira/browse/FLINK-5047?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15874713#comment-15874713
]
ASF GitHub Bot commented on FLINK-5047:
---------------------------------------
GitHub user twalthr opened a pull request:
https://github.com/apache/flink/pull/3364
[FLINK-5047] [table] Add sliding group-windows for batch tables
This PR implements sliding group-windows. It covers the following cases:
- Grouped Slide count-windows with incremental aggregates
- Grouped and All Slide time-windows with incremental and non-incremental
aggregates
All windows support the overlapping and non-overlapping case. Slide windows
are pre-tumbled if possible. This PR also fixes some bugs.
If the general design is ok, I will also implement the missing
non-incremental aggregates for count-windows. And add a bit of documentation.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/twalthr/flink FLINK-5047
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/flink/pull/3364.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 #3364
----
commit adcedd91e76e73457740816f691ccf64f2e2e38b
Author: twalthr <[email protected]>
Date: 2017-01-18T15:56:02Z
[FLINK-5047] [table] Add sliding group-windows for batch tables
----
> Add sliding group-windows for batch tables
> ------------------------------------------
>
> Key: FLINK-5047
> URL: https://issues.apache.org/jira/browse/FLINK-5047
> Project: Flink
> Issue Type: Sub-task
> Components: Table API & SQL
> Reporter: Jark Wu
> Assignee: Timo Walther
>
> Add Slide group-windows for batch tables as described in
> [FLIP-11|https://cwiki.apache.org/confluence/display/FLINK/FLIP-11%3A+Table+API+Stream+Aggregations].
> There are two ways to implement sliding windows for batch:
> 1. replicate the output in order to assign keys for overlapping windows. This
> is probably the more straight-forward implementation and supports any
> aggregation function but blows up the data volume.
> 2. if the aggregation functions are combinable / pre-aggregatable, we can
> also find the largest tumbling window size from which the sliding windows can
> be assembled. This is basically the technique used to express sliding windows
> with plain SQL (GROUP BY + OVER clauses). For a sliding window Slide(10
> minutes, 2 minutes) this would mean to first compute aggregates of
> non-overlapping (tumbling) 2 minute windows and assembling consecutively 5 of
> these into a sliding window (could be done in a MapPartition with sorted
> input). The implementation could be done as an optimizer rule to split the
> sliding aggregate into a tumbling aggregate and a SQL WINDOW operator. Maybe
> it makes sense to implement the WINDOW clause first and reuse this for
> sliding windows.
> 3. There is also a third, hybrid solution: Doing the pre-aggregation on the
> largest non-overlapping windows (as in 2) and replicating these results and
> processing those as in the 1) approach. The benefits of this is that it a) is
> based on the implementation that supports non-combinable aggregates (which is
> required in any case) and b) that it does not require the implementation of
> the SQL WINDOW operator. Internally, this can be implemented again as an
> optimizer rule that translates the SlidingWindow into a pre-aggregating
> TublingWindow and a final SlidingWindow (with replication).
> see FLINK-4692 for more discussion
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)