Github user rtudoran commented on a diff in the pull request:
https://github.com/apache/flink/pull/3590#discussion_r107402759
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedProcessingOverProcessFunction.scala
---
@@ -0,0 +1,141 @@
+/*
+ * 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.state.{ListState, ListStateDescriptor}
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.runtime.state.{FunctionInitializationContext,
FunctionSnapshotContext}
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.{Accumulator, AggregateFunction}
+import org.apache.flink.types.Row
+import org.apache.flink.util.{Collector, Preconditions}
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+
+/**
+ * Process Function used for the aggregate in partitioned bounded windows
in
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param aggregates the list of all
[[org.apache.flink.table.functions.AggregateFunction]]
+ * used for this aggregation
+ * @param aggFields the position (in the input Row) of the input value
for each aggregate
+ * @param forwardedFieldCount Is used to indicate fields in the current
element to forward
+ * @param rowTypeInfo Is used to indicate the field schema
+ * @param time_boundary Is used to indicate the processing time boundaries
+ */
+class ProcTimeBoundedProcessingOverProcessFunction(
+ private val aggregates: Array[AggregateFunction[_]],
+ private val aggFields: Array[Int],
+ private val forwardedFieldCount: Int,
+ private val rowTypeInfo: RowTypeInfo,
+ private val time_boundary: Long)
+ extends ProcessFunction[Row, Row] {
+
+ Preconditions.checkNotNull(aggregates)
+ Preconditions.checkNotNull(aggFields)
+ Preconditions.checkArgument(aggregates.length == aggFields.length)
+
+ private var accumulators: Row = _
+ private var output: Row = _
+ private var windowBuffer: ListState[Tuple2[Long,Row]] = null
+ private var state: ValueState[Row] = _
+
+
+ override def open(config: Configuration) {
+ output = new Row(forwardedFieldCount + aggregates.length)
+
+ accumulators = new Row(aggregates.length)
+ var i = 0
+ while (i < aggregates.length) {
+ accumulators.setField(i, aggregates(i).createAccumulator())
+ i += 1
+ }
+
+ // We keep the elements received in a list state
+ // together with the ingestion time in the operator
+ val bufferDescriptor: ListStateDescriptor[Tuple2[Long,Row]] =
+ new ListStateDescriptor[Tuple2[Long,Row]]("windowBufferState",
classOf[Tuple2[Long,Row]])
+ windowBuffer = getRuntimeContext.getListState(bufferDescriptor)
+
+ val stateDescriptor: ValueStateDescriptor[Row] =
+ new ValueStateDescriptor[Row]("overState", classOf[Row] ,
accumulators)
+ state = getRuntimeContext.getState(stateDescriptor)
+ }
+
+ override def processElement(
+ input: Row,
+ ctx: ProcessFunction[Row, Row]#Context,
+ out: Collector[Row]): Unit = {
+
+ var current_time = System.currentTimeMillis()
+ //buffer the event incoming event
+ windowBuffer.add(new Tuple2(
+ current_time,
+ input))
+
+ var i = 0
+
+ var accumulators = state.value()
+
+ //set the fields of the last event to carry on with the aggregates
+ i = 0
+ while (i < forwardedFieldCount) {
+ output.setField(i, input.getField(i))
+ i += 1
+ }
+
+ //update the elements to be removed and retract them from aggregators
+ var iter = windowBuffer.get.iterator()
+ var continue:Boolean = true
--- End diff --
@fhueske 2 points:
-If we traverse the whole list of all elements - than what is the
difference compared to having the window as before and going through the whole
list to aggregate?...complexity would be the same...
-There is no need to go through the whole list. The list is sorted based on
processing time (i.e. based on incoming order of events). As we bound things
based on time it means that we only need to go through the oldest elements
until we find one which is still within the scope of the window. When we find
this we can stop the search as we ensured we have only the right elements
remaining in the buffer.
Considering that the reason to switch from window to process function was
to reduce the number of operations - i would say we need to keep
this...otherwise it is basically the same and in the case i would ask you to
merge the window implementation
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---