Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 5EB35200C40 for ; Wed, 8 Mar 2017 15:33:44 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 5D88F160B88; Wed, 8 Mar 2017 14:33:44 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 7CAC9160B86 for ; Wed, 8 Mar 2017 15:33:43 +0100 (CET) Received: (qmail 93578 invoked by uid 500); 8 Mar 2017 14:33:42 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 93465 invoked by uid 99); 8 Mar 2017 14:33:42 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 08 Mar 2017 14:33:42 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id 1B26DC0BF8 for ; Wed, 8 Mar 2017 14:33:42 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 0.651 X-Spam-Level: X-Spam-Status: No, score=0.651 tagged_above=-999 required=6.31 tests=[RP_MATCHES_RCVD=-0.001, SPF_NEUTRAL=0.652] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id EYEH5A7XSiwM for ; Wed, 8 Mar 2017 14:33:40 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTP id 5186F60DBE for ; Wed, 8 Mar 2017 14:33:40 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 622D2E0045 for ; Wed, 8 Mar 2017 14:33:39 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 73C4B24363 for ; Wed, 8 Mar 2017 14:33:38 +0000 (UTC) Date: Wed, 8 Mar 2017 14:33:38 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (FLINK-5047) Add sliding group-windows for batch tables MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Wed, 08 Mar 2017 14:33:44 -0000 [ https://issues.apache.org/jira/browse/FLINK-5047?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15901337#comment-15901337 ] ASF GitHub Bot commented on FLINK-5047: --------------------------------------- Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3364#discussion_r104925837 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala --- @@ -186,6 +200,130 @@ object AggregateUtil { } /** + * Create a [[org.apache.flink.api.common.functions.GroupReduceFunction]] that prepares for + * partial aggregates of sliding windows (time and count-windows). + * It requires a prepared input (with intermediate aggregate fields and aligned rowtime for + * pre-tumbling in case of time-windows), pre-aggregates (pre-tumbles) rows, aligns the + * window-start, and replicates or omits records for different panes of a sliding window. + * + * The output of the function contains the grouping keys, the intermediate aggregate values of + * all aggregate function and the aligned window start. Window start must not be a timestamp, + * but can also be a count value for count-windows. + * + * The output is stored in Row by the following format: + * + * {{{ + * avg(x) aggOffsetInRow = 2 count(z) aggOffsetInRow = 5 + * | | + * v v + * +---------+---------+--------+--------+--------+--------+-------------+ + * |groupKey1|groupKey2| sum1 | count1 | sum2 | count2 | windowStart | + * +---------+---------+--------+--------+--------+--------+-------------+ + * ^ ^ + * | | + * sum(y) aggOffsetInRow = 4 window start for pane mapping + * }}} + * + * NOTE: this function is only used for sliding windows with partial aggregates on batch tables. + */ + def createDataSetSlideWindowPrepareGroupReduceFunction( + window: LogicalWindow, + namedAggregates: Seq[CalcitePair[AggregateCall, String]], + groupings: Array[Int], + inputType: RelDataType, + isParserCaseSensitive: Boolean) + : RichGroupReduceFunction[Row, Row] = { + + val aggregates = transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false)._2 + + val returnType: RowTypeInfo = createDataSetAggregateBufferDataType( + groupings, + aggregates, + inputType, + Some(Array(BasicTypeInfo.LONG_TYPE_INFO))) + + window match { + case EventTimeSlidingGroupWindow(_, _, size, slide) if isTimeInterval(size.resultType) => + // sliding time-window + // for partial aggregations + new DataSetSlideTimeWindowAggReduceCombineFunction( + aggregates, + groupings.length, + returnType.getArity - 1, + asLong(size), + asLong(slide), + returnType) + + case _ => + throw new UnsupportedOperationException(s"$window is currently not supported on batch.") + } + } + + /** + * Create a [[org.apache.flink.api.common.functions.FlatMapFunction]] that prepares for + * non-incremental aggregates of sliding windows (time-windows). + * + * It requires a prepared input (with intermediate aggregate fields), aligns the + * window-start, and replicates or omits records for different panes of a sliding window. + * + * The output of the function contains the grouping keys, the intermediate aggregate values of + * all aggregate function and the aligned window start. + * + * The output is stored in Row by the following format: + * + * {{{ + * avg(x) aggOffsetInRow = 2 count(z) aggOffsetInRow = 5 + * | | + * v v + * +---------+---------+--------+--------+--------+--------+-------------+ + * |groupKey1|groupKey2| sum1 | count1 | sum2 | count2 | windowStart | + * +---------+---------+--------+--------+--------+--------+-------------+ + * ^ ^ + * | | + * sum(y) aggOffsetInRow = 4 window start for pane mapping + * }}} + * + * NOTE: this function is only used for time-based sliding windows on batch tables. + */ + def createDataSetSlideWindowPrepareFlatMapFunction( + window: LogicalWindow, + namedAggregates: Seq[CalcitePair[AggregateCall, String]], + groupings: Array[Int], + inputType: RelDataType, + isParserCaseSensitive: Boolean) + : FlatMapFunction[Row, Row] = { + + val aggregates = transformToAggregateFunctions( --- End diff -- not needed because input type = output type > 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)