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

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

Github user shaoxuan-wang commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3470#discussion_r104301147
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/MaxAggFunctionWithRetract.scala
 ---
    @@ -0,0 +1,205 @@
    +/*
    + * 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.functions.aggfunctions
    +
    +import java.math.BigDecimal
    +import java.util.{HashMap => JHashMap, List => JList}
    +
    +import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
    +import org.apache.flink.api.java.tuple.{Tuple3 => JTuple3}
    +import org.apache.flink.api.java.typeutils.{MapTypeInfo, TupleTypeInfo}
    +import org.apache.flink.table.api.TableException
    +import org.apache.flink.table.functions.{Accumulator, AggregateFunction}
    +
    +/** The initial accumulator for Max with retraction aggregate function */
    +class MaxWithRetractAccumulator[T] extends JTuple3[T, Long, JHashMap[T, 
Long]] with Accumulator
    +
    +/**
    +  * Base class for built-in Max with retraction aggregate function
    +  *
    +  * @tparam T the type for the aggregation result
    +  */
    +abstract class MaxWithRetractAggFunction[T](implicit ord: Ordering[T]) 
extends AggregateFunction[T] {
    +
    +  override def createAccumulator(): Accumulator = {
    +    val acc = new MaxWithRetractAccumulator[T]
    +    acc.f0 = getInitValue //max
    +    acc.f1 = 0L //total count
    +    acc.f2 = new JHashMap[T, Long]() //store the count for each value
    +    acc
    +  }
    +
    +  override def accumulate(accumulator: Accumulator, value: Any): Unit = {
    +    if (value != null) {
    +      val v = value.asInstanceOf[T]
    +      val a = accumulator.asInstanceOf[MaxWithRetractAccumulator[T]]
    +
    +      if (a.f1 == 0 || (ord.compare(a.f0, v) < 0)) {
    +        a.f0 = v
    +      }
    +
    +      a.f1 += 1L
    +
    +      if (!a.f2.containsKey(v)) {
    +        a.f2.put(v, 1L)
    +      } else {
    +        var count = a.f2.get(v)
    +        count += 1L
    +        a.f2.put(v, count)
    +      }
    +    }
    +  }
    +
    +  override def retract(accumulator: Accumulator, value: Any): Unit = {
    +    if (value != null) {
    +      val v = value.asInstanceOf[T]
    +      val a = accumulator.asInstanceOf[MaxWithRetractAccumulator[T]]
    +
    +      a.f1 -= 1L
    +
    +      if (!a.f2.containsKey(v)) {
    +        throw TableException("unexpected retract message")
    +      } else {
    +        var count = a.f2.get(v)
    +        count -= 1L
    +        if (count == 0) {
    +          //remove the key v from the map if the number of appearance of 
the value v is 0
    +          a.f2.remove(v)
    +          //if the total count is 0, we could just simply set the f0(max) 
to the initial value
    +          if (a.f1 == 0) {
    +            a.f0 = getInitValue
    +            return
    +          }
    +          //if v is the current max value, we have to iterate the map to 
find the 2nd biggest
    +          // value to replace v as the max value
    +          if (v == a.f0) {
    +            val iterator = a.f2.keySet().iterator()
    +            var key = iterator.next()
    +            a.f0 = key
    +            while (iterator.hasNext()) {
    +              key = iterator.next()
    +              if (ord.compare(a.f0, key) < 0) {
    +                a.f0 = key
    +              }
    +            }
    +          }
    +        } else {
    +          a.f2.put(v, count)
    +        }
    +      }
    +    }
    +  }
    +
    +  override def getValue(accumulator: Accumulator): T = {
    +    val a = accumulator.asInstanceOf[MaxWithRetractAccumulator[T]]
    +    if (a.f1 != 0) {
    +      a.f0
    +    } else {
    +      null.asInstanceOf[T]
    +    }
    +  }
    +
    +  override def merge(accumulators: JList[Accumulator]): Accumulator = {
    +    val ret = accumulators.get(0)
    +    var i: Int = 1
    +    while (i < accumulators.size()) {
    +      val a = 
accumulators.get(i).asInstanceOf[MaxWithRetractAccumulator[T]]
    +      if (a.f1 != 0) {
    +        accumulate(ret.asInstanceOf[MaxWithRetractAccumulator[T]], a.f0)
    --- End diff --
    
    Ah, good catch!  We should merge the hashmap as well, retraction is an 
important feature for dataStrream. We might want to always turn the retraction 
mechanism on. So merge the hashmap is a much-have, as session window and future 
dataStream local-global agg will leverage merge method.


> Add retract method into the aggregateFunction
> ---------------------------------------------
>
>                 Key: FLINK-5956
>                 URL: https://issues.apache.org/jira/browse/FLINK-5956
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API & SQL
>            Reporter: Shaoxuan Wang
>            Assignee: Shaoxuan Wang
>
> Retraction method is help for processing updated message. It will also very 
> helpful for window Aggregation. This PR will first add retraction methods 
> into the aggregateFunctions, such that on-going over window Aggregation can 
> get benefit from it.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to