[
https://issues.apache.org/jira/browse/FLINK-8689?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16394403#comment-16394403
]
ASF GitHub Bot commented on FLINK-8689:
---------------------------------------
Github user hequn8128 commented on a diff in the pull request:
https://github.com/apache/flink/pull/5555#discussion_r173644059
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/DistinctAccumulator.scala
---
@@ -0,0 +1,102 @@
+/*
+ * 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.lang.{Iterable => JIterable}
+import java.util.{Map => JMap}
+import org.apache.flink.table.api.dataview.MapView
+
+/**
+ * The base class for accumulator wrapper when applying distinct
aggregation.
+ * @param realAcc the actual accumulator which gets invoke after distinct
filter.
+ * @param mapView the [[MapView]] element used to store the distinct
filter hash map.
+ * @tparam E the element type for the distinct filter hash map.
+ * @tparam ACC the accumulator type for the realAcc.
+ */
+class DistinctAccumulator[E, ACC](var realAcc: ACC, var mapView:
MapView[E, Integer]) {
+ def this() {
+ this(null.asInstanceOf[ACC], new MapView[E, Integer]())
+ }
+
+ def this(realAcc: ACC) {
+ this(realAcc, new MapView[E, Integer]())
+ }
+
+ def getRealAcc: ACC = realAcc
+
+ def canEqual(a: Any): Boolean = a.isInstanceOf[DistinctAccumulator[E,
ACC]]
+
+ override def equals(that: Any): Boolean =
+ that match {
+ case that: DistinctAccumulator[E, ACC] => that.canEqual(this) &&
+ this.mapView == that.mapView
+ case _ => false
+ }
+
+ def add(element: E): Boolean = {
+ if (element != null) {
+ if (mapView.contains(element)) {
--- End diff --
Use mapView.get() directly and reuse the result. This can avoid to get
state twice.
> Add runtime support of distinct filter using MapView
> -----------------------------------------------------
>
> Key: FLINK-8689
> URL: https://issues.apache.org/jira/browse/FLINK-8689
> Project: Flink
> Issue Type: Sub-task
> Reporter: Rong Rong
> Assignee: Rong Rong
> Priority: Major
>
> This ticket should cover distinct aggregate function support to codegen for
> *AggregateCall*, where *isDistinct* fields is set to true.
> This can be verified using the following SQL, which is not currently
> producing correct results.
> {code:java}
> SELECT
> a,
> SUM(b) OVER (PARTITION BY a ORDER BY proctime ROWS BETWEEN 5 PRECEDING AND
> CURRENT ROW)
> FROM
> MyTable{code}
>
>
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)