[ https://issues.apache.org/jira/browse/FLINK-5658?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15926462#comment-15926462 ]
ASF GitHub Bot commented on FLINK-5658: --------------------------------------- Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r106195270 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala --- @@ -0,0 +1,283 @@ +/* + * 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 java.io.{ByteArrayInputStream, ByteArrayOutputStream} +import java.util + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.{ProcessFunction} +import org.apache.flink.util.{Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.common.typeutils.base.StringSerializer +import org.apache.flink.api.java.functions.KeySelector +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.core.memory.{DataInputViewStreamWrapper, DataOutputViewStreamWrapper} +import org.apache.flink.runtime.state.{FunctionInitializationContext, FunctionSnapshotContext} +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.streaming.api.windowing.windows.TimeWindow +import org.apache.flink.table.functions.{Accumulator, AggregateFunction} + +import scala.collection.mutable.ArrayBuffer + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param interMediateType the intermediate row tye which the state saved + * @param keySelector the keyselector + * @param keyType the key type + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction[_]], + private val aggFields: Array[Int], + private val forwardedFieldCount: Int, + private val interMediateType: TypeInformation[Row], + private val keySelector: KeySelector[Row, Tuple], + private val keyType: TypeInformation[Tuple]) + extends ProcessFunction[Row, Row] + with CheckpointedFunction{ + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var state: MapState[TimeWindow, Row] = _ + private val aggregateWithIndex: Array[(AggregateFunction[_], Int)] = aggregates.zipWithIndex + + /** Sorted list per key for choose the recent result and the records need retraction **/ + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList[TimeWindow]] = + new java.util.HashMap[Tuple, java.util.LinkedList[TimeWindow]] + + /** For store timeSectionsMap **/ + private var timeSectionsState: ListState[String] = _ + private var inputKeySerializer: TypeSerializer[Tuple] = _ + private var timeSerializer: TypeSerializer[TimeWindow] = _ + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + interMediateType.createSerializer(getRuntimeContext.getExecutionConfig) + timeSerializer = new TimeWindow.Serializer + val stateDescriptor: MapStateDescriptor[TimeWindow, Row] = + new MapStateDescriptor[TimeWindow, Row]("rowtimeoverstate", timeSerializer, valueSerializer) + inputKeySerializer = keyType.createSerializer(getRuntimeContext.getExecutionConfig) + state = getRuntimeContext.getMapState[TimeWindow, Row](stateDescriptor) + } + + override def processElement( --- End diff -- To be honest, I think this design is a bit too complicated for the current state of the Table API / SQL. So far, we cannot handle retractions and a result cannot be updated once emitted. I propose the following design: ``` open() { // initalize a ListState[Tuple2[Long, Row]] to collect rows between watermarks // initialize a ValueState[Row] for the accumulators of the aggregation functions. } processElement(row: Row, ctx: Context, out: Collector) = { // check if row timestamp is before current watermark // if yes, return // else put the row in a ListState of Tuple2[Long, Row] (long is timestamp) and register a timer for rowTimestamp + 1 } // onTimer is called for each registered event-time timer when the watermark advances beyond the timer's timestamp. onTimer() { // get ListState and check size. // is size == 0, return // if size > 0 insert list into a priority queue sorted by timestamp // fetch the accumulator for the key (we hold one Row with accumulators as ValueState) // go over the priority queue, update the accumulator and emit the result. // clear list state and update accumulator } ``` With this design, processing each row is cheap (inserting into ListState) and we have to do work when a new watermark arrives. The multiple calls of `onTimer()`for each watermark should be OK. We ensure that data is correctly sorted but drop late data (just like the group windows at the moment). Further benefits are that we let Flink handle the key and state management. So no need for key extractors or implementing the `CheckpointedFunction` interface. > Add event time OVER RANGE BETWEEN UNBOUNDED PRECEDING aggregation to SQL > ------------------------------------------------------------------------ > > Key: FLINK-5658 > URL: https://issues.apache.org/jira/browse/FLINK-5658 > Project: Flink > Issue Type: Sub-task > Components: Table API & SQL > Reporter: Fabian Hueske > Assignee: Yuhong Hong > > The goal of this issue is to add support for OVER RANGE aggregations on event > 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 rowTime() RANGE BETWEEN UNBOUNDED > PRECEDING AND CURRENT ROW) AS sumB, > MIN(b) OVER (PARTITION BY c ORDER BY rowTime() RANGE BETWEEN UNBOUNDED > 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 rowTime() as parameter. rowTime() is a > parameterless scalar function that just indicates processing time mode. > - bounded PRECEDING is not supported (see FLINK-5655) > - 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)