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

ASF GitHub Bot commented on FLINK-5654:
---------------------------------------

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

    https://github.com/apache/flink/pull/3590#discussion_r107444057
  
    --- 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
    +    
    +    while(continue && iter.hasNext())
    +    {
    +      var currentElement:Tuple2[Long,Row]= iter.next()  
    +      if(currentElement._1<time_boundary){
    +        iter.remove()
    --- End diff --
    
    @sunjincheng121 had a good idea, IMO. 
    
    He's using a `MapState[Long, List[Row]]` to store the Rows. Since the map 
is not ordered, we have to go through the keys (the Long values) to find the 
keys of the Rows that need to be retracted. This is again O(n) but we only have 
to deserialize `Long` which is very efficient. Once the keys of the Rows that 
need to be retracted, we get exactly those Rows and do not need to touch any of 
the other Rows. When updating the state, we only update the keys that we 
changed and do not touch the others.
    
    What do you think, @rtudoran?


> Add processing time OVER RANGE BETWEEN x PRECEDING aggregation to SQL
> ---------------------------------------------------------------------
>
>                 Key: FLINK-5654
>                 URL: https://issues.apache.org/jira/browse/FLINK-5654
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API & SQL
>            Reporter: Fabian Hueske
>            Assignee: radu
>
> The goal of this issue is to add support for OVER RANGE 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() RANGE BETWEEN INTERVAL '1' 
> HOUR PRECEDING AND CURRENT ROW) AS sumB,
>   MIN(b) OVER (PARTITION BY c ORDER BY procTime() RANGE BETWEEN INTERVAL '1' 
> HOUR 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-5657)
> - 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