Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3585#discussion_r107432524
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowsClauseBoundedOverProcessFunction.scala
 ---
    @@ -0,0 +1,207 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.java.typeutils.RowTypeInfo
    +import org.apache.flink.configuration.Configuration
    +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}
    +
    +/**
    +  * Process Function for ROWS clause event-time bounded OVER window
    +  *
    +  * @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  the count of forwarded fields.
    +  * @param aggregationStateType the row type info of aggregation
    +  * @param precedingOffset      the preceding offset
    +  */
    +class RowsClauseBoundedOverProcessFunction(
    +    private val aggregates: Array[AggregateFunction[_]],
    +    private val aggFields: Array[Int],
    +    private val forwardedFieldCount: Int,
    +    private val aggregationStateType: RowTypeInfo,
    +    private val precedingOffset: Int)
    +  extends ProcessFunction[Row, Row] {
    +
    +  Preconditions.checkNotNull(aggregates)
    +  Preconditions.checkNotNull(aggFields)
    +  Preconditions.checkArgument(aggregates.length == aggFields.length)
    +  Preconditions.checkNotNull(forwardedFieldCount)
    +  Preconditions.checkNotNull(aggregationStateType)
    +  Preconditions.checkNotNull(precedingOffset)
    +
    +  private var output: Row = _
    +
    +  // the state which keeps the last triggering timestamp
    +  private var lastTriggeringTsState: ValueState[Long] = _
    +
    +  // the state which keeps the count of data
    +  private var dataCountState: ValueState[Long] = null
    +
    +  // the state which used to materialize the accumulator for incremental 
calculation
    +  private var accumulatorState: ValueState[Row] = _
    +
    +  // the state which keeps all the data that are not expired.
    +  // The first element (as the mapState key) of the tuple is the time 
stamp. Per each time stamp,
    +  // the second element of tuple is a list that contains the entire data 
of all the rows belonging
    +  // to this time stamp.
    +  private var dataState: MapState[Long, JList[Row]] = _
    +
    +  override def open(config: Configuration) {
    +
    +    output = new Row(forwardedFieldCount + aggregates.length)
    +
    +
    +    val lastTriggeringTsDescriptor: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("lastTriggeringTsState", 
classOf[Long])
    +    lastTriggeringTsState = 
getRuntimeContext.getState(lastTriggeringTsDescriptor)
    +
    +    val dataCountStateDescriptor =
    +      new ValueStateDescriptor[Long]("dataCountState", classOf[Long])
    +    dataCountState = getRuntimeContext.getState(dataCountStateDescriptor)
    +
    +    val accumulatorStateDescriptor =
    +      new ValueStateDescriptor[Row]("accumulatorState", 
aggregationStateType)
    +    accumulatorState = 
getRuntimeContext.getState(accumulatorStateDescriptor)
    +
    +    val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
    +      new MapStateDescriptor[Long, JList[Row]](
    +        "dataState",
    +        classOf[Long],
    +        classOf[JList[Row]])
    +
    +    dataState = getRuntimeContext.getMapState(mapStateDescriptor)
    +
    +  }
    +
    +  override def processElement(
    +    input: Row,
    +    ctx: ProcessFunction[Row, Row]#Context,
    +    out: Collector[Row]): Unit = {
    +
    +    // triggering timestamp for trigger calculation
    +    val triggeringTs = ctx.timestamp
    +
    +    val lastTriggeringTs = lastTriggeringTsState.value
    +    // check if the data is expired, if not, save the data and register 
event time timer
    +    if (triggeringTs > lastTriggeringTs && triggeringTs > 
ctx.timerService.currentWatermark) {
    +      if (dataState.contains(triggeringTs)) {
    +        val data = dataState.get(triggeringTs)
    +        data.add(input)
    +        dataState.put(triggeringTs, data)
    +      } else {
    +        val data = new ArrayList[Row]
    +        data.add(input)
    +        dataState.put(triggeringTs, data)
    +        // register event time timer
    +        ctx.timerService.registerEventTimeTimer(triggeringTs)
    --- End diff --
    
    In event-time timers are only triggered when a new watermark is received. 
    
    Your implementation relies on the fact that the timers are called in the 
correct order (which I would hope to be true, but haven't checked). 
    
    If we register a timer for `currentWatermark + 1`, only one timer will be 
registered and called so we have to process all records in that call. If we 
follow your approach, we have multiple timers which will be called one after 
the other (but at the same logical time when a watermark is received). In each 
call, we need to do similar work which might make this approach less efficient. 
    
    For example, if we register only a single timer, we could sort the key 
fields of the MapState just once and split them into values to retract and to 
emit, where we have to go multiple time through the keys if we register 
multiple timers.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to