[ 
https://issues.apache.org/jira/browse/FLINK-5658?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15938734#comment-15938734
 ] 

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_r107712950
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala
 ---
    @@ -0,0 +1,200 @@
    +/*
    + * 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, LinkedList, List => JList}
    +
    +import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
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.java.typeutils.{ListTypeInfo}
    +import org.apache.flink.streaming.api.operators.TimestampedCollector
    +import org.apache.flink.table.functions.{Accumulator, AggregateFunction}
    +
    +
    +/**
    +  * 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 inputType the input row tye which the state saved
    +  *
    +  */
    +class UnboundedEventTimeOverProcessFunction(
    +    private val aggregates: Array[AggregateFunction[_]],
    +    private val aggFields: Array[Int],
    +    private val forwardedFieldCount: Int,
    +    private val intermediateType: TypeInformation[Row],
    +    private val inputType: TypeInformation[Row])
    +  extends ProcessFunction[Row, Row]{
    +
    +  Preconditions.checkNotNull(aggregates)
    +  Preconditions.checkNotNull(aggFields)
    +  Preconditions.checkArgument(aggregates.length == aggFields.length)
    +
    +  private var output: Row = _
    +  private var accumulatorState: ValueState[Row] = _
    +  private var rowMapState: MapState[Long, JList[Row]] = _
    +  private var sortList: LinkedList[Long] = _
    +
    +
    +  override def open(config: Configuration) {
    +    output = new Row(forwardedFieldCount + aggregates.length)
    +    sortList = new LinkedList[Long]()
    +
    +    val stateDescriptor: ValueStateDescriptor[Row] =
    +      new ValueStateDescriptor[Row]("accumulatorstate", intermediateType)
    +    accumulatorState = getRuntimeContext.getState[Row](stateDescriptor)
    +
    +    val rowListTypeInfo: TypeInformation[JList[Row]] =
    +      new 
ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]]
    +    val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
    +      new MapStateDescriptor[Long, JList[Row]]("rowmapstate",
    +        BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 
rowListTypeInfo)
    +    rowMapState = getRuntimeContext.getMapState(mapStateDescriptor)
    +  }
    +
    +  /**
    +    * Process one element from the input stream, not emit the output
    +    *
    +    * @param input The input value.
    +    * @param ctx   The ctx to register timer or get current time
    +    * @param out   The collector for returning result values.
    +    *
    +    */
    +  override def processElement(
    +     input: Row,
    +     ctx:  ProcessFunction[Row, Row]#Context,
    +     out: Collector[Row]): Unit = {
    +
    +    // discard later record
    +    if (ctx.timestamp() >= ctx.timerService().currentWatermark()) {
    +      // ensure every key just register on timer
    +      
ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1)
    +
    +      val rowList =
    +        if (rowMapState.contains(ctx.timestamp)) 
rowMapState.get(ctx.timestamp)
    +        else new ArrayList[Row]()
    +      rowList.add(input)
    +      rowMapState.put(ctx.timestamp, rowList)
    +    }
    +  }
    +
    +  /**
    +    * Called when a timer set fires, sort current records according the 
timestamp
    +    * and emit the output
    +    *
    +    * @param timestamp The timestamp of the firing timer.
    +    * @param ctx       The ctx to register timer or get current time
    +    * @param out       The collector for returning result values.
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: ProcessFunction[Row, Row]#OnTimerContext,
    +      out: Collector[Row]): Unit = {
    +
    +    
Preconditions.checkArgument(out.isInstanceOf[TimestampedCollector[Row]])
    +    val collector = out.asInstanceOf[TimestampedCollector[Row]]
    +
    +    val mapIter = rowMapState.iterator
    --- End diff --
    
    we should use `rowMapState.keys` to only retrieve the keys. 
    If we iterate over the entries, also the values will be deserialized.


> 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)

Reply via email to