[
https://issues.apache.org/jira/browse/FLINK-5653?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15946692#comment-15946692
]
ASF GitHub Bot commented on FLINK-5653:
---------------------------------------
Github user sunjincheng121 commented on a diff in the pull request:
https://github.com/apache/flink/pull/3574#discussion_r108611842
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala
---
@@ -0,0 +1,157 @@
+/*
+ * 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.configuration.Configuration
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.table.functions.{ Accumulator, AggregateFunction }
+import scala.collection.mutable.Queue
+import org.apache.flink.api.common.state.ListStateDescriptor
+import org.apache.flink.api.common.state.ListState
+import org.apache.flink.api.common.typeinfo.TypeHint
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import java.util.{ ArrayList, LinkedList, List => JList }
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+
+class BoundedProcessingOverRowProcessFunction(
+ private val aggregates: Array[AggregateFunction[_]],
+ private val aggFields: Array[Int],
+ private val bufferSize: Int,
+ private val forwardedFieldCount: Int,
+ private val aggregatesTypeInfo: RowTypeInfo,
+ private val inputType: TypeInformation[Row])
+ extends ProcessFunction[Row, Row] {
+
+ Preconditions.checkNotNull(aggregates)
+ Preconditions.checkNotNull(aggFields)
+ Preconditions.checkArgument(aggregates.length == aggFields.length)
+ Preconditions.checkArgument(bufferSize > 0)
+
+ private var accumulators: Row = _
+ private var output: Row = _
+ private var accumulatorState: ValueState[Row] = _
+ private var rowMapState: MapState[Long, JList[Row]] = _
+
+ override def open(config: Configuration) {
+
+ output = new Row(forwardedFieldCount + aggregates.length)
+ // We keep the elements received in a list state
+ // together with the ingestion time in the operator
+ val rowListTypeInfo: TypeInformation[JList[Row]] =
+ new
ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]]
+
+ val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
+ new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState",
+ BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]],
rowListTypeInfo)
+
+ rowMapState = getRuntimeContext.getMapState(mapStateDescriptor)
+
+ val stateDescriptor: ValueStateDescriptor[Row] =
+ new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo)
+
+ accumulatorState = getRuntimeContext.getState(stateDescriptor)
+ }
+
+ override def processElement(
+ input: Row,
+ ctx: ProcessFunction[Row, Row]#Context,
+ out: Collector[Row]): Unit = {
+
+ val currentTime = ctx.timerService().currentProcessingTime()
+ var i = 0
+
+ var accumulators = accumulatorState.value()
+ // initialize state for the first processed element
+ if(accumulators == null){
+ accumulators = new Row(aggregates.length)
+ while (i < aggregates.length) {
+ accumulators.setField(i, aggregates(i).createAccumulator())
+ i += 1
+ }
+ }
+
+ val keyIter = rowMapState.keys.iterator
+ var oldestTimeStamp = currentTime
+ var toRetract: JList[Row] = null
+ var currentKeyTime: Long = 0L
+ i = 0
+ while(keyIter.hasNext){
+ currentKeyTime = keyIter.next
+ i += rowMapState.get(currentKeyTime).size()
+ if(currentKeyTime <= oldestTimeStamp){
+ oldestTimeStamp = currentKeyTime
+ toRetract = rowMapState.get(currentKeyTime)
--- End diff --
We only need move `toRetract = rowMapState.get(currentKeyTime)` to loop
out, then `toRetract = rowMapState.get(oldestTimeStamp)`.
> Add processing time OVER ROWS BETWEEN x PRECEDING aggregation to SQL
> --------------------------------------------------------------------
>
> Key: FLINK-5653
> URL: https://issues.apache.org/jira/browse/FLINK-5653
> Project: Flink
> Issue Type: Sub-task
> Components: Table API & SQL
> Reporter: Fabian Hueske
> Assignee: Stefano Bortoli
>
> The goal of this issue is to add support for OVER ROWS aggregations on
> processing time streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT
> a,
> SUM(b) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 PRECEDING
> AND CURRENT ROW) AS sumB,
> MIN(b) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 PRECEDING
> AND CURRENT ROW) AS minB
> FROM myStream
> {code}
> The following restrictions should initially apply:
> - 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 clause may only have procTime() as parameter. procTime() is a
> parameterless scalar function that just indicates processing time mode.
> - UNBOUNDED PRECEDING is not supported (see FLINK-5656)
> - FOLLOWING is not supported.
> The restrictions will be resolved in follow up issues. If we find that some
> of the restrictions are trivial to address, we can add the functionality in
> this issue as well.
> This issue includes:
> - Design of the DataStream operator to compute OVER ROW aggregates
> - Translation from Calcite's RelNode representation (LogicalProject with
> RexOver expression).
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)