lincoln-lil commented on code in PR #27821:
URL: https://github.com/apache/flink/pull/27821#discussion_r2988373591


##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/aggregate/AbstractBitmapOrWithRetractAggFunction.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.functions.aggregate;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.dataview.MapView;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.bitmap.Bitmap;
+import org.apache.flink.types.bitmap.RoaringBitmapData;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import javax.annotation.Nullable;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import static 
org.apache.flink.table.types.utils.DataTypeUtils.toInternalDataType;
+
+/** Abstract base class for BITMAP_OR_AGG and BITMAP_OR_CARDINALITY_AGG with 
retraction. */
+@Internal
+public abstract class AbstractBitmapOrWithRetractAggFunction<T>
+        extends BuiltInAggregateFunction<
+                T, 
AbstractBitmapOrWithRetractAggFunction.BitmapOrWithRetractAccumulator> {
+
+    private final transient DataType valueDataType;
+
+    public AbstractBitmapOrWithRetractAggFunction(LogicalType valueType) {
+        this.valueDataType = toInternalDataType(valueType);
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+    // Planning
+    // 
--------------------------------------------------------------------------------------------
+
+    @Override
+    public List<DataType> getArgumentDataTypes() {
+        return Collections.singletonList(valueDataType);
+    }
+
+    @Override
+    public DataType getAccumulatorDataType() {
+        return DataTypes.STRUCTURED(
+                BitmapOrWithRetractAccumulator.class,
+                DataTypes.FIELD("bitmapCount", DataTypes.INT().notNull()),
+                DataTypes.FIELD("bitmap", DataTypes.BITMAP().notNull()),
+                DataTypes.FIELD(
+                        "valueCount",
+                        MapView.newMapViewDataType(
+                                DataTypes.INT().notNull(), 
DataTypes.INT().notNull())));
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+    // Accumulator
+    // 
--------------------------------------------------------------------------------------------
+
+    /** Accumulator for BITMAP_OR_AGG and BITMAP_OR_CARDINALITY_AGG with 
retraction. */
+    public static class BitmapOrWithRetractAccumulator {
+
+        public int bitmapCount = 0;
+        // bitmap should reflect the actual data based on valueCount
+        public RoaringBitmapData bitmap = RoaringBitmapData.empty();
+        public MapView<Integer, Integer> valueCount = new MapView<>();
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj) {
+                return true;
+            }
+            if (obj == null || getClass() != obj.getClass()) {
+                return false;
+            }
+            BitmapOrWithRetractAccumulator that = 
(BitmapOrWithRetractAccumulator) obj;
+            return bitmapCount == that.bitmapCount
+                    && Objects.equals(bitmap, that.bitmap)
+                    && Objects.equals(valueCount, that.valueCount);
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(bitmapCount, bitmap, valueCount);
+        }
+    }
+
+    @Override
+    public BitmapOrWithRetractAccumulator createAccumulator() {
+        return new BitmapOrWithRetractAccumulator();
+    }
+
+    public void resetAccumulator(BitmapOrWithRetractAccumulator acc) {
+        acc.bitmapCount = 0;
+        acc.bitmap.clear();
+        acc.valueCount.clear();
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+    // Runtime
+    // 
--------------------------------------------------------------------------------------------
+
+    public void accumulate(BitmapOrWithRetractAccumulator acc, @Nullable 
Bitmap bitmap)
+            throws Exception {
+        if (bitmap == null) {
+            return;
+        }
+
+        acc.bitmapCount++;
+
+        RoaringBitmapData rbm32 = (RoaringBitmapData) bitmap;

Review Comment:
   ditto



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/aggregate/AbstractBitmapAndWithRetractAggFunction.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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.functions.aggregate;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.dataview.MapView;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.bitmap.Bitmap;
+import org.apache.flink.types.bitmap.RoaringBitmapData;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import javax.annotation.Nullable;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import static 
org.apache.flink.table.types.utils.DataTypeUtils.toInternalDataType;
+
+/** Abstract base class for BITMAP_AND_AGG and BITMAP_AND_CARDINALITY_AGG with 
retraction. */
+@Internal
+public abstract class AbstractBitmapAndWithRetractAggFunction<T>
+        extends BuiltInAggregateFunction<
+                T, 
AbstractBitmapAndWithRetractAggFunction.BitmapAndWithRetractAccumulator> {
+
+    private final transient DataType valueDataType;
+
+    public AbstractBitmapAndWithRetractAggFunction(LogicalType valueType) {
+        this.valueDataType = toInternalDataType(valueType);
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+    // Planning
+    // 
--------------------------------------------------------------------------------------------
+
+    @Override
+    public List<DataType> getArgumentDataTypes() {
+        return Collections.singletonList(valueDataType);
+    }
+
+    @Override
+    public DataType getAccumulatorDataType() {
+        return DataTypes.STRUCTURED(
+                BitmapAndWithRetractAccumulator.class,
+                DataTypes.FIELD("bitmapCount", DataTypes.INT().notNull()),
+                DataTypes.FIELD(
+                        "valueCount",
+                        MapView.newMapViewDataType(
+                                DataTypes.INT().notNull(), 
DataTypes.INT().notNull())));
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+    // Accumulator
+    // 
--------------------------------------------------------------------------------------------
+
+    /** Accumulator for BITMAP_AND_AGG and BITMAP_AND_CARDINALITY_AGG with 
retraction. */
+    public static class BitmapAndWithRetractAccumulator {
+
+        public int bitmapCount = 0;
+        public MapView<Integer, Integer> valueCount = new MapView<>();
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj) {
+                return true;
+            }
+            if (obj == null || getClass() != obj.getClass()) {
+                return false;
+            }
+            BitmapAndWithRetractAccumulator that = 
(BitmapAndWithRetractAccumulator) obj;
+            return bitmapCount == that.bitmapCount && 
Objects.equals(valueCount, that.valueCount);
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(bitmapCount, valueCount);
+        }
+    }
+
+    @Override
+    public BitmapAndWithRetractAccumulator createAccumulator() {
+        return new BitmapAndWithRetractAccumulator();
+    }
+
+    public void resetAccumulator(BitmapAndWithRetractAccumulator acc) {
+        acc.bitmapCount = 0;
+        acc.valueCount.clear();
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+    // Runtime
+    // 
--------------------------------------------------------------------------------------------
+
+    public void accumulate(BitmapAndWithRetractAccumulator acc, @Nullable 
Bitmap bitmap)
+            throws Exception {
+        if (bitmap == null) {
+            return;
+        }
+
+        acc.bitmapCount++;
+
+        RoaringBitmapData rbm32 = (RoaringBitmapData) bitmap;

Review Comment:
   Use `RoaringBitmapData.toRoaringBitmapData()` seems better?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to