[
https://issues.apache.org/jira/browse/FLINK-5047?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15898176#comment-15898176
]
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_r104511932
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideTimeWindowAggFlatMapFunction.scala
---
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.aggregate
+
+import org.apache.flink.api.common.functions.RichFlatMapFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow
+import org.apache.flink.types.Row
+import org.apache.flink.util.{Collector, Preconditions}
+
+
+/**
+ * It is used for sliding windows on batch for time-windows. It takes a
prepared input row,
+ * aligns the window start, and replicates or omits records for different
panes of a sliding
+ * window. It is used for non-incremental aggregations.
+ *
+ * @param aggregates aggregate functions
+ * @param groupingKeysLength number of grouping keys
+ * @param windowSize window size of the sliding window
+ * @param windowSlide window slide of the sliding window
+ * @param returnType return type of this function
+ */
+class DataSetSlideTimeWindowAggFlatMapFunction(
+ private val aggregates: Array[Aggregate[_]],
+ private val groupingKeysLength: Int,
+ private val timeFieldPos: Int,
+ private val windowSize: Long,
+ private val windowSlide: Long,
+ @transient private val returnType: TypeInformation[Row])
+ extends RichFlatMapFunction[Row, Row]
+ with ResultTypeQueryable[Row] {
+
+ private var aggregateBuffer: Row = _
+ private var outWindowStartIndex: Int = _
+
+ override def open(config: Configuration) {
+ Preconditions.checkNotNull(aggregates)
+ // add one field to store window start
+ val partialRowLength = groupingKeysLength +
+ aggregates.map(_.intermediateDataType.length).sum + 1
+ aggregateBuffer = new Row(partialRowLength)
+ outWindowStartIndex = partialRowLength - 1
+ }
+
+ override def flatMap(record: Row, out: Collector[Row]): Unit = {
+ val windowStart = record.getField(timeFieldPos).asInstanceOf[Long]
+
+ // adopted from SlidingEventTimeWindows.assignWindows
+ var start: Long = TimeWindow.getWindowStartWithOffset(windowStart, 0,
windowSlide)
+
+ // skip preparing output if it is not necessary
+ if (start > windowStart - windowSize) {
+
+ // prepare output
+ for (i <- aggregates.indices) {
--- End diff --
Isn't this just copying data from record to the aggregateBuffer?
Doesn't the input record have the same schema as the output record? Isn't
is sufficient emit the input record multiple times with adapted
`outWindowStartIndex`?
> 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)