[
https://issues.apache.org/jira/browse/FLINK-4691?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15551526#comment-15551526
]
ASF GitHub Bot commented on FLINK-4691:
---------------------------------------
Github user fhueske commented on a diff in the pull request:
https://github.com/apache/flink/pull/2562#discussion_r82151050
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateReduceCombineFunction.scala
---
@@ -0,0 +1,132 @@
+/*
+ * 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.api.table.runtime.aggregate
+
+import java.lang.Iterable
+
+import org.apache.flink.api.common.functions.{CombineFunction,
RichGroupReduceFunction}
+import org.apache.flink.api.table.Row
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.windowing.windows.Window
+import org.apache.flink.util.{Collector, Preconditions}
+
+import scala.collection.JavaConversions._
+
+
+/**
+ * It wraps the aggregate logic inside of
+ * [[org.apache.flink.api.java.operators.GroupReduceOperator]] and
+ * [[org.apache.flink.api.java.operators.GroupCombineOperator]]
+ *
+ * @param aggregates The aggregate functions.
+ * @param groupKeysMapping The index mapping of group keys between
intermediate aggregate Row
+ * and output Row.
+ * @param aggregateMapping The index mapping between aggregate function
list and aggregated value
+ * index in output Row.
+ */
+class AggregateReduceCombineFunction(
+ private val aggregates: Array[Aggregate[_ <: Any]],
+ private val groupKeysMapping: Array[(Int, Int)],
+ private val aggregateMapping: Array[(Int, Int)],
+ private val intermediateRowArity: Int)
+ extends RichGroupReduceFunction[Row, Row] with CombineFunction[Row,
Row] {
+
+ private var aggregateBuffer: Row = _
+ private var output: Row = _
+ private var aggContext: AggContext = _
+
+ /**
+ * Sets a new aggregation context used for [[Aggregate.evaluate()]].
+ */
+ def setAggContext(aggContext: AggContext): Unit = {
+ this.aggContext = aggContext
+ }
+
+ override def open(config: Configuration): Unit = {
+ Preconditions.checkNotNull(aggregates)
+ Preconditions.checkNotNull(groupKeysMapping)
+ val finalRowLength: Int = groupKeysMapping.length +
aggregateMapping.length
+ aggregateBuffer = new Row(intermediateRowArity)
+ output = new Row(finalRowLength)
+ aggContext = new AggContext
+ }
+
+ /**
+ * For grouped intermediate aggregate Rows, merge all of them into
aggregate buffer,
+ * calculate aggregated values output by aggregate buffer, and set them
into output
+ * Row based on the mapping relation between intermediate aggregate Row
and output Row.
+ *
+ * @param records Grouped intermediate aggregate Rows iterator.
+ * @param out The collector to hand results to.
+ *
+ */
+ override def reduce(records: Iterable[Row], out: Collector[Row]): Unit =
{
+
+ // Initiate intermediate aggregate value.
+ aggregates.foreach(_.initiate(aggregateBuffer))
+
+ // Merge intermediate aggregate value to buffer.
+ var last: Row = null
+ records.foreach((record) => {
+ aggregates.foreach(_.merge(record, aggregateBuffer))
+ last = record
+ })
+
+ // Set group keys value to final output.
+ groupKeysMapping.map {
--- End diff --
use `foreach` instead of `map` because `setField` returns `Unit`.
(can be fixed in `AggregateReduceGroupFunction` as well)
> Add group-windows for streaming tables
> ---------------------------------------
>
> Key: FLINK-4691
> URL: https://issues.apache.org/jira/browse/FLINK-4691
> Project: Flink
> Issue Type: Sub-task
> Components: Table API & SQL
> Reporter: Timo Walther
> Assignee: Timo Walther
>
> Add Tumble, Slide, Session group-windows for streaming tables as described in
> [FLIP-11|https://cwiki.apache.org/confluence/display/FLINK/FLIP-11%3A+Table+API+Stream+Aggregations].
>
> Implementation of group-windows on streaming tables. This includes
> implementing the API of group-windows, the logical validation for
> group-windows, and the definition of the “rowtime” and “systemtime” keywords.
> Group-windows on batch tables won’t be initially supported and will throw an
> exception.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)