You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2020/07/29 07:46:28 UTC

[GitHub] [incubator-pinot] kishoreg opened a new pull request #5766: Adding distinct count support based on bitmap

kishoreg opened a new pull request #5766:
URL: https://github.com/apache/incubator-pinot/pull/5766


   ## Description
   
   We currently use IntHashSet for distinct count. This new version uses bitmap to store the distinct id's which is more efficient. This is also a better serialize/deserialize.
   
   `select distinctCountBitmap(columnName) from T`
   
   TODO:
   - test cases
   - ensure that star tree works
   
   ## Upgrade Notes
   Does this PR prevent a zero down-time upgrade? (Assume upgrade order: Controller, Broker, Server, Minion)
   * [ ] Yes (Please label as **<code>backward-incompat</code>**, and complete the section below on Release Notes)
   
   Does this PR fix a zero-downtime upgrade introduced earlier?
   * [ ] Yes (Please label this as **<code>backward-incompat</code>**, and complete the section below on Release Notes)
   
   Does this PR otherwise need attention when creating release notes? Things to consider:
   - New configuration options
   - Deprecation of configurations
   - Signature changes to public methods/interfaces
   - New plugins added or old plugins removed
   * [ ] Yes (Please label this PR as **<code>release-notes</code>** and complete the section on Release Notes)
   ## Release Notes
   If you have tagged this as either backward-incompat or release-notes,
   you MUST add text here that you would like to see appear in release notes of the
   next release.
   
   If you have a series of commits adding or enabling a feature, then
   add this section only in final commit that marks the feature completed.
   Refer to earlier release notes to see examples of text
   
   ## Documentation
   If you have introduced a new feature or configuration, please add it to the documentation as well.
   See https://docs.pinot.apache.org/developers/developers-and-contributors/update-document
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5766: Adding distinct count support based on bitmap

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5766:
URL: https://github.com/apache/incubator-pinot/pull/5766#discussion_r463706260



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/DistinctCountBitmapAggregationFunction.java
##########
@@ -0,0 +1,409 @@
+/**
+ * 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.pinot.core.query.aggregation.function;
+
+import java.util.Map;
+import org.apache.pinot.common.function.AggregationFunctionType;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.core.common.BlockValSet;
+import org.apache.pinot.core.common.ObjectSerDeUtils;
+import org.apache.pinot.core.query.aggregation.AggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.ObjectAggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.GroupByResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.ObjectGroupByResultHolder;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.segment.index.readers.Dictionary;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.roaringbitmap.PeekableIntIterator;
+import org.roaringbitmap.RoaringBitmap;
+
+
+public class DistinctCountBitmapAggregationFunction extends BaseSingleInputAggregationFunction<RoaringBitmap, Integer> {

Review comment:
       javadoc would be good. 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5766: Adding distinct count support based on bitmap

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5766:
URL: https://github.com/apache/incubator-pinot/pull/5766#discussion_r463873001



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/DistinctCountBitmapAggregationFunction.java
##########
@@ -0,0 +1,409 @@
+/**
+ * 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.pinot.core.query.aggregation.function;
+
+import java.util.Map;
+import org.apache.pinot.common.function.AggregationFunctionType;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.core.common.BlockValSet;
+import org.apache.pinot.core.common.ObjectSerDeUtils;
+import org.apache.pinot.core.query.aggregation.AggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.ObjectAggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.GroupByResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.ObjectGroupByResultHolder;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.segment.index.readers.Dictionary;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.roaringbitmap.PeekableIntIterator;
+import org.roaringbitmap.RoaringBitmap;
+
+
+public class DistinctCountBitmapAggregationFunction extends BaseSingleInputAggregationFunction<RoaringBitmap, Integer> {

Review comment:
       Added




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5766: Adding distinct count support based on bitmap

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5766:
URL: https://github.com/apache/incubator-pinot/pull/5766#discussion_r463885786



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/DistinctCountBitmapAggregationFunction.java
##########
@@ -0,0 +1,409 @@
+/**
+ * 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.pinot.core.query.aggregation.function;
+
+import java.util.Map;
+import org.apache.pinot.common.function.AggregationFunctionType;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.core.common.BlockValSet;
+import org.apache.pinot.core.common.ObjectSerDeUtils;
+import org.apache.pinot.core.query.aggregation.AggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.ObjectAggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.GroupByResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.ObjectGroupByResultHolder;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.segment.index.readers.Dictionary;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.roaringbitmap.PeekableIntIterator;
+import org.roaringbitmap.RoaringBitmap;
+
+
+public class DistinctCountBitmapAggregationFunction extends BaseSingleInputAggregationFunction<RoaringBitmap, Integer> {
+  protected Dictionary _dictionary;
+
+  public DistinctCountBitmapAggregationFunction(ExpressionContext expression) {
+    super(expression);
+  }
+
+  @Override
+  public AggregationFunctionType getType() {
+    return AggregationFunctionType.DISTINCTCOUNTBITMAP;
+  }
+
+  @Override
+  public void accept(AggregationFunctionVisitorBase visitor) {
+    visitor.visit(this);
+  }
+
+  @Override
+  public AggregationResultHolder createAggregationResultHolder() {
+    return new ObjectAggregationResultHolder();
+  }
+
+  @Override
+  public GroupByResultHolder createGroupByResultHolder(int initialCapacity, int maxCapacity) {
+    return new ObjectGroupByResultHolder(initialCapacity, maxCapacity);
+  }
+
+  @Override
+  public void aggregate(int length, AggregationResultHolder aggregationResultHolder,
+      Map<ExpressionContext, BlockValSet> blockValSetMap) {
+    BlockValSet blockValSet = blockValSetMap.get(_expression);
+
+    // Treat BYTES value as serialized RoaringBitmap
+    DataType valueType = blockValSet.getValueType();
+    if (valueType == DataType.BYTES) {
+      byte[][] bytesValues = blockValSet.getBytesValuesSV();
+      RoaringBitmap bitmap = aggregationResultHolder.getResult();
+      if (bitmap != null) {
+        for (int i = 0; i < length; i++) {
+          bitmap.or(ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[i]));
+        }
+      } else {
+        bitmap = ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[0]);
+        aggregationResultHolder.setValue(bitmap);
+        for (int i = 1; i < length; i++) {
+          bitmap.or(ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[i]));
+        }
+      }
+      return;
+    }
+
+    // For dictionary-encoded expression, store dictionary ids into the bitmap
+    RoaringBitmap bitmap = getBitmap(aggregationResultHolder);
+    Dictionary dictionary = blockValSet.getDictionary();
+    if (dictionary != null) {
+      _dictionary = dictionary;
+      int[] dictIds = blockValSet.getDictionaryIdsSV();
+      bitmap.addN(dictIds, 0, length);
+      return;
+    }
+
+    // For non-dictionary-encoded expression, store hash code of the values into the bitmap
+    switch (valueType) {
+      case INT:
+        int[] intValues = blockValSet.getIntValuesSV();
+        bitmap.addN(intValues, 0, length);
+        break;
+      case LONG:
+        long[] longValues = blockValSet.getLongValuesSV();
+        for (int i = 0; i < length; i++) {
+          bitmap.add(Long.hashCode(longValues[i]));
+        }
+        break;
+      case FLOAT:
+        float[] floatValues = blockValSet.getFloatValuesSV();
+        for (int i = 0; i < length; i++) {
+          bitmap.add(Float.hashCode(floatValues[i]));
+        }
+        break;
+      case DOUBLE:
+        double[] doubleValues = blockValSet.getDoubleValuesSV();
+        for (int i = 0; i < length; i++) {
+          bitmap.add(Double.hashCode(doubleValues[i]));
+        }
+        break;
+      case STRING:
+        String[] stringValues = blockValSet.getStringValuesSV();
+        for (int i = 0; i < length; i++) {
+          bitmap.add(stringValues[i].hashCode());
+        }
+        break;
+      default:
+        throw new IllegalStateException(
+            "Illegal data type for DISTINCT_COUNT_BITMAP aggregation function: " + valueType);
+    }
+  }
+
+  @Override
+  public void aggregateGroupBySV(int length, int[] groupKeyArray, GroupByResultHolder groupByResultHolder,
+      Map<ExpressionContext, BlockValSet> blockValSetMap) {
+    BlockValSet blockValSet = blockValSetMap.get(_expression);
+
+    // Treat BYTES value as serialized RoaringBitmap
+    DataType valueType = blockValSet.getValueType();
+    if (valueType == DataType.BYTES) {
+      byte[][] bytesValues = blockValSet.getBytesValuesSV();
+      for (int i = 0; i < length; i++) {
+        RoaringBitmap value = ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[i]);
+        int groupKey = groupKeyArray[i];
+        RoaringBitmap bitmap = groupByResultHolder.getResult(groupKey);
+        if (bitmap != null) {
+          bitmap.or(value);
+        } else {
+          groupByResultHolder.setValueForKey(groupKey, value);
+        }
+      }
+      return;
+    }
+
+    // For dictionary-encoded expression, store dictionary ids into the bitmap
+    Dictionary dictionary = blockValSet.getDictionary();
+    if (dictionary != null) {
+      _dictionary = dictionary;
+      int[] dictIds = blockValSet.getDictionaryIdsSV();
+      for (int i = 0; i < length; i++) {
+        getBitmap(groupByResultHolder, groupKeyArray[i]).add(dictIds[i]);
+      }
+      return;
+    }
+
+    // For non-dictionary-encoded expression, store hash code of the values into the bitmap
+    switch (valueType) {
+      case INT:
+        int[] intValues = blockValSet.getIntValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(intValues[i]);
+        }
+        break;
+      case LONG:
+        long[] longValues = blockValSet.getLongValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(Long.hashCode(longValues[i]));
+        }
+        break;
+      case FLOAT:
+        float[] floatValues = blockValSet.getFloatValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(Float.hashCode(floatValues[i]));
+        }
+        break;
+      case DOUBLE:
+        double[] doubleValues = blockValSet.getDoubleValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(Double.hashCode(doubleValues[i]));
+        }
+        break;
+      case STRING:
+        String[] stringValues = blockValSet.getStringValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(stringValues[i].hashCode());
+        }
+        break;
+      default:
+        throw new IllegalStateException(
+            "Illegal data type for DISTINCT_COUNT_BITMAP aggregation function: " + valueType);
+    }
+  }
+
+  @Override
+  public void aggregateGroupByMV(int length, int[][] groupKeysArray, GroupByResultHolder groupByResultHolder,
+      Map<ExpressionContext, BlockValSet> blockValSetMap) {
+    BlockValSet blockValSet = blockValSetMap.get(_expression);
+
+    // Treat BYTES value as serialized RoaringBitmap
+    DataType valueType = blockValSet.getValueType();
+    if (valueType == DataType.BYTES) {
+      byte[][] bytesValues = blockValSet.getBytesValuesSV();
+      for (int i = 0; i < length; i++) {
+        RoaringBitmap value = ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[i]);
+        for (int groupKey : groupKeysArray[i]) {
+          RoaringBitmap bitmap = groupByResultHolder.getResult(groupKey);
+          if (bitmap != null) {
+            bitmap.or(value);
+          } else {
+            // Clone a bitmap for the group
+            groupByResultHolder.setValueForKey(groupKey, value.clone());
+          }
+        }
+      }
+      return;
+    }
+
+    // For dictionary-encoded expression, store dictionary ids into the bitmap
+    Dictionary dictionary = blockValSet.getDictionary();

Review comment:
       Did a performance comparison on IntOpenHashSet vs RoaringBitmap vs BitSet, and find that the performance: BitSet > RoaringBitmap >> IntOpenHashSet. So here we use RoaringBitmap for good performance + good compression.
   




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5766: Adding distinct count support based on bitmap

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5766:
URL: https://github.com/apache/incubator-pinot/pull/5766#discussion_r463707260



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/DistinctCountBitmapAggregationFunction.java
##########
@@ -0,0 +1,409 @@
+/**
+ * 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.pinot.core.query.aggregation.function;
+
+import java.util.Map;
+import org.apache.pinot.common.function.AggregationFunctionType;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.core.common.BlockValSet;
+import org.apache.pinot.core.common.ObjectSerDeUtils;
+import org.apache.pinot.core.query.aggregation.AggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.ObjectAggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.GroupByResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.ObjectGroupByResultHolder;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.segment.index.readers.Dictionary;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.roaringbitmap.PeekableIntIterator;
+import org.roaringbitmap.RoaringBitmap;
+
+
+public class DistinctCountBitmapAggregationFunction extends BaseSingleInputAggregationFunction<RoaringBitmap, Integer> {
+  protected Dictionary _dictionary;
+
+  public DistinctCountBitmapAggregationFunction(ExpressionContext expression) {
+    super(expression);
+  }
+
+  @Override
+  public AggregationFunctionType getType() {
+    return AggregationFunctionType.DISTINCTCOUNTBITMAP;
+  }
+
+  @Override
+  public void accept(AggregationFunctionVisitorBase visitor) {
+    visitor.visit(this);
+  }
+
+  @Override
+  public AggregationResultHolder createAggregationResultHolder() {
+    return new ObjectAggregationResultHolder();
+  }
+
+  @Override
+  public GroupByResultHolder createGroupByResultHolder(int initialCapacity, int maxCapacity) {
+    return new ObjectGroupByResultHolder(initialCapacity, maxCapacity);
+  }
+
+  @Override
+  public void aggregate(int length, AggregationResultHolder aggregationResultHolder,
+      Map<ExpressionContext, BlockValSet> blockValSetMap) {
+    BlockValSet blockValSet = blockValSetMap.get(_expression);
+
+    // Treat BYTES value as serialized RoaringBitmap
+    DataType valueType = blockValSet.getValueType();
+    if (valueType == DataType.BYTES) {
+      byte[][] bytesValues = blockValSet.getBytesValuesSV();
+      RoaringBitmap bitmap = aggregationResultHolder.getResult();
+      if (bitmap != null) {
+        for (int i = 0; i < length; i++) {
+          bitmap.or(ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[i]));
+        }
+      } else {
+        bitmap = ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[0]);
+        aggregationResultHolder.setValue(bitmap);
+        for (int i = 1; i < length; i++) {
+          bitmap.or(ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[i]));
+        }
+      }
+      return;
+    }
+
+    // For dictionary-encoded expression, store dictionary ids into the bitmap
+    RoaringBitmap bitmap = getBitmap(aggregationResultHolder);
+    Dictionary dictionary = blockValSet.getDictionary();
+    if (dictionary != null) {
+      _dictionary = dictionary;
+      int[] dictIds = blockValSet.getDictionaryIdsSV();
+      bitmap.addN(dictIds, 0, length);
+      return;
+    }
+
+    // For non-dictionary-encoded expression, store hash code of the values into the bitmap
+    switch (valueType) {
+      case INT:
+        int[] intValues = blockValSet.getIntValuesSV();
+        bitmap.addN(intValues, 0, length);
+        break;
+      case LONG:
+        long[] longValues = blockValSet.getLongValuesSV();
+        for (int i = 0; i < length; i++) {
+          bitmap.add(Long.hashCode(longValues[i]));
+        }
+        break;
+      case FLOAT:
+        float[] floatValues = blockValSet.getFloatValuesSV();
+        for (int i = 0; i < length; i++) {
+          bitmap.add(Float.hashCode(floatValues[i]));
+        }
+        break;
+      case DOUBLE:
+        double[] doubleValues = blockValSet.getDoubleValuesSV();
+        for (int i = 0; i < length; i++) {
+          bitmap.add(Double.hashCode(doubleValues[i]));
+        }
+        break;
+      case STRING:
+        String[] stringValues = blockValSet.getStringValuesSV();
+        for (int i = 0; i < length; i++) {
+          bitmap.add(stringValues[i].hashCode());
+        }
+        break;
+      default:
+        throw new IllegalStateException(
+            "Illegal data type for DISTINCT_COUNT_BITMAP aggregation function: " + valueType);
+    }
+  }
+
+  @Override
+  public void aggregateGroupBySV(int length, int[] groupKeyArray, GroupByResultHolder groupByResultHolder,
+      Map<ExpressionContext, BlockValSet> blockValSetMap) {
+    BlockValSet blockValSet = blockValSetMap.get(_expression);
+
+    // Treat BYTES value as serialized RoaringBitmap
+    DataType valueType = blockValSet.getValueType();
+    if (valueType == DataType.BYTES) {
+      byte[][] bytesValues = blockValSet.getBytesValuesSV();
+      for (int i = 0; i < length; i++) {
+        RoaringBitmap value = ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[i]);
+        int groupKey = groupKeyArray[i];
+        RoaringBitmap bitmap = groupByResultHolder.getResult(groupKey);
+        if (bitmap != null) {
+          bitmap.or(value);
+        } else {
+          groupByResultHolder.setValueForKey(groupKey, value);
+        }
+      }
+      return;
+    }
+
+    // For dictionary-encoded expression, store dictionary ids into the bitmap
+    Dictionary dictionary = blockValSet.getDictionary();
+    if (dictionary != null) {
+      _dictionary = dictionary;
+      int[] dictIds = blockValSet.getDictionaryIdsSV();
+      for (int i = 0; i < length; i++) {
+        getBitmap(groupByResultHolder, groupKeyArray[i]).add(dictIds[i]);
+      }
+      return;
+    }
+
+    // For non-dictionary-encoded expression, store hash code of the values into the bitmap
+    switch (valueType) {
+      case INT:
+        int[] intValues = blockValSet.getIntValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(intValues[i]);
+        }
+        break;
+      case LONG:
+        long[] longValues = blockValSet.getLongValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(Long.hashCode(longValues[i]));
+        }
+        break;
+      case FLOAT:
+        float[] floatValues = blockValSet.getFloatValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(Float.hashCode(floatValues[i]));
+        }
+        break;
+      case DOUBLE:
+        double[] doubleValues = blockValSet.getDoubleValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(Double.hashCode(doubleValues[i]));
+        }
+        break;
+      case STRING:
+        String[] stringValues = blockValSet.getStringValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(stringValues[i].hashCode());
+        }
+        break;
+      default:
+        throw new IllegalStateException(
+            "Illegal data type for DISTINCT_COUNT_BITMAP aggregation function: " + valueType);
+    }
+  }
+
+  @Override
+  public void aggregateGroupByMV(int length, int[][] groupKeysArray, GroupByResultHolder groupByResultHolder,
+      Map<ExpressionContext, BlockValSet> blockValSetMap) {
+    BlockValSet blockValSet = blockValSetMap.get(_expression);
+
+    // Treat BYTES value as serialized RoaringBitmap
+    DataType valueType = blockValSet.getValueType();
+    if (valueType == DataType.BYTES) {
+      byte[][] bytesValues = blockValSet.getBytesValuesSV();
+      for (int i = 0; i < length; i++) {
+        RoaringBitmap value = ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[i]);
+        for (int groupKey : groupKeysArray[i]) {
+          RoaringBitmap bitmap = groupByResultHolder.getResult(groupKey);
+          if (bitmap != null) {
+            bitmap.or(value);
+          } else {
+            // Clone a bitmap for the group
+            groupByResultHolder.setValueForKey(groupKey, value.clone());
+          }
+        }
+      }
+      return;
+    }
+
+    // For dictionary-encoded expression, store dictionary ids into the bitmap
+    Dictionary dictionary = blockValSet.getDictionary();

Review comment:
       So DistinctCount stores dictionaryId in IntHashSet (with your recent change) and here we store them in a bitmap. I am guessing this is more space efficient?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on pull request #5766: Adding distinct count support based on bitmap

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on pull request #5766:
URL: https://github.com/apache/incubator-pinot/pull/5766#issuecomment-667437954


   @siddharthteotia Addressed all the comments


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] siddharthteotia commented on pull request #5766: Adding distinct count support based on bitmap

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on pull request #5766:
URL: https://github.com/apache/incubator-pinot/pull/5766#issuecomment-665457815


   This is fantastic. Sorry, by mistake approved it. Will review by tomorrow


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5766: Adding distinct count support based on bitmap

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5766:
URL: https://github.com/apache/incubator-pinot/pull/5766#discussion_r463885786



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/DistinctCountBitmapAggregationFunction.java
##########
@@ -0,0 +1,409 @@
+/**
+ * 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.pinot.core.query.aggregation.function;
+
+import java.util.Map;
+import org.apache.pinot.common.function.AggregationFunctionType;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.core.common.BlockValSet;
+import org.apache.pinot.core.common.ObjectSerDeUtils;
+import org.apache.pinot.core.query.aggregation.AggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.ObjectAggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.GroupByResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.ObjectGroupByResultHolder;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.segment.index.readers.Dictionary;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.roaringbitmap.PeekableIntIterator;
+import org.roaringbitmap.RoaringBitmap;
+
+
+public class DistinctCountBitmapAggregationFunction extends BaseSingleInputAggregationFunction<RoaringBitmap, Integer> {
+  protected Dictionary _dictionary;
+
+  public DistinctCountBitmapAggregationFunction(ExpressionContext expression) {
+    super(expression);
+  }
+
+  @Override
+  public AggregationFunctionType getType() {
+    return AggregationFunctionType.DISTINCTCOUNTBITMAP;
+  }
+
+  @Override
+  public void accept(AggregationFunctionVisitorBase visitor) {
+    visitor.visit(this);
+  }
+
+  @Override
+  public AggregationResultHolder createAggregationResultHolder() {
+    return new ObjectAggregationResultHolder();
+  }
+
+  @Override
+  public GroupByResultHolder createGroupByResultHolder(int initialCapacity, int maxCapacity) {
+    return new ObjectGroupByResultHolder(initialCapacity, maxCapacity);
+  }
+
+  @Override
+  public void aggregate(int length, AggregationResultHolder aggregationResultHolder,
+      Map<ExpressionContext, BlockValSet> blockValSetMap) {
+    BlockValSet blockValSet = blockValSetMap.get(_expression);
+
+    // Treat BYTES value as serialized RoaringBitmap
+    DataType valueType = blockValSet.getValueType();
+    if (valueType == DataType.BYTES) {
+      byte[][] bytesValues = blockValSet.getBytesValuesSV();
+      RoaringBitmap bitmap = aggregationResultHolder.getResult();
+      if (bitmap != null) {
+        for (int i = 0; i < length; i++) {
+          bitmap.or(ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[i]));
+        }
+      } else {
+        bitmap = ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[0]);
+        aggregationResultHolder.setValue(bitmap);
+        for (int i = 1; i < length; i++) {
+          bitmap.or(ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[i]));
+        }
+      }
+      return;
+    }
+
+    // For dictionary-encoded expression, store dictionary ids into the bitmap
+    RoaringBitmap bitmap = getBitmap(aggregationResultHolder);
+    Dictionary dictionary = blockValSet.getDictionary();
+    if (dictionary != null) {
+      _dictionary = dictionary;
+      int[] dictIds = blockValSet.getDictionaryIdsSV();
+      bitmap.addN(dictIds, 0, length);
+      return;
+    }
+
+    // For non-dictionary-encoded expression, store hash code of the values into the bitmap
+    switch (valueType) {
+      case INT:
+        int[] intValues = blockValSet.getIntValuesSV();
+        bitmap.addN(intValues, 0, length);
+        break;
+      case LONG:
+        long[] longValues = blockValSet.getLongValuesSV();
+        for (int i = 0; i < length; i++) {
+          bitmap.add(Long.hashCode(longValues[i]));
+        }
+        break;
+      case FLOAT:
+        float[] floatValues = blockValSet.getFloatValuesSV();
+        for (int i = 0; i < length; i++) {
+          bitmap.add(Float.hashCode(floatValues[i]));
+        }
+        break;
+      case DOUBLE:
+        double[] doubleValues = blockValSet.getDoubleValuesSV();
+        for (int i = 0; i < length; i++) {
+          bitmap.add(Double.hashCode(doubleValues[i]));
+        }
+        break;
+      case STRING:
+        String[] stringValues = blockValSet.getStringValuesSV();
+        for (int i = 0; i < length; i++) {
+          bitmap.add(stringValues[i].hashCode());
+        }
+        break;
+      default:
+        throw new IllegalStateException(
+            "Illegal data type for DISTINCT_COUNT_BITMAP aggregation function: " + valueType);
+    }
+  }
+
+  @Override
+  public void aggregateGroupBySV(int length, int[] groupKeyArray, GroupByResultHolder groupByResultHolder,
+      Map<ExpressionContext, BlockValSet> blockValSetMap) {
+    BlockValSet blockValSet = blockValSetMap.get(_expression);
+
+    // Treat BYTES value as serialized RoaringBitmap
+    DataType valueType = blockValSet.getValueType();
+    if (valueType == DataType.BYTES) {
+      byte[][] bytesValues = blockValSet.getBytesValuesSV();
+      for (int i = 0; i < length; i++) {
+        RoaringBitmap value = ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[i]);
+        int groupKey = groupKeyArray[i];
+        RoaringBitmap bitmap = groupByResultHolder.getResult(groupKey);
+        if (bitmap != null) {
+          bitmap.or(value);
+        } else {
+          groupByResultHolder.setValueForKey(groupKey, value);
+        }
+      }
+      return;
+    }
+
+    // For dictionary-encoded expression, store dictionary ids into the bitmap
+    Dictionary dictionary = blockValSet.getDictionary();
+    if (dictionary != null) {
+      _dictionary = dictionary;
+      int[] dictIds = blockValSet.getDictionaryIdsSV();
+      for (int i = 0; i < length; i++) {
+        getBitmap(groupByResultHolder, groupKeyArray[i]).add(dictIds[i]);
+      }
+      return;
+    }
+
+    // For non-dictionary-encoded expression, store hash code of the values into the bitmap
+    switch (valueType) {
+      case INT:
+        int[] intValues = blockValSet.getIntValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(intValues[i]);
+        }
+        break;
+      case LONG:
+        long[] longValues = blockValSet.getLongValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(Long.hashCode(longValues[i]));
+        }
+        break;
+      case FLOAT:
+        float[] floatValues = blockValSet.getFloatValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(Float.hashCode(floatValues[i]));
+        }
+        break;
+      case DOUBLE:
+        double[] doubleValues = blockValSet.getDoubleValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(Double.hashCode(doubleValues[i]));
+        }
+        break;
+      case STRING:
+        String[] stringValues = blockValSet.getStringValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(stringValues[i].hashCode());
+        }
+        break;
+      default:
+        throw new IllegalStateException(
+            "Illegal data type for DISTINCT_COUNT_BITMAP aggregation function: " + valueType);
+    }
+  }
+
+  @Override
+  public void aggregateGroupByMV(int length, int[][] groupKeysArray, GroupByResultHolder groupByResultHolder,
+      Map<ExpressionContext, BlockValSet> blockValSetMap) {
+    BlockValSet blockValSet = blockValSetMap.get(_expression);
+
+    // Treat BYTES value as serialized RoaringBitmap
+    DataType valueType = blockValSet.getValueType();
+    if (valueType == DataType.BYTES) {
+      byte[][] bytesValues = blockValSet.getBytesValuesSV();
+      for (int i = 0; i < length; i++) {
+        RoaringBitmap value = ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[i]);
+        for (int groupKey : groupKeysArray[i]) {
+          RoaringBitmap bitmap = groupByResultHolder.getResult(groupKey);
+          if (bitmap != null) {
+            bitmap.or(value);
+          } else {
+            // Clone a bitmap for the group
+            groupByResultHolder.setValueForKey(groupKey, value.clone());
+          }
+        }
+      }
+      return;
+    }
+
+    // For dictionary-encoded expression, store dictionary ids into the bitmap
+    Dictionary dictionary = blockValSet.getDictionary();

Review comment:
       Did a performance comparison on IntOpenHashSet vs RoaringBitmap vs BitSet, and find that BitSet has the best performance (almost 10x faster on adding values). Will change the function to use the BitSet. Will file a separate PR for DistinctCount to use BitSet




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5766: Adding distinct count support based on bitmap

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5766:
URL: https://github.com/apache/incubator-pinot/pull/5766#discussion_r463708558



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/DistinctCountBitmapAggregationFunction.java
##########
@@ -0,0 +1,409 @@
+/**
+ * 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.pinot.core.query.aggregation.function;
+
+import java.util.Map;
+import org.apache.pinot.common.function.AggregationFunctionType;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.core.common.BlockValSet;
+import org.apache.pinot.core.common.ObjectSerDeUtils;
+import org.apache.pinot.core.query.aggregation.AggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.ObjectAggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.GroupByResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.ObjectGroupByResultHolder;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.segment.index.readers.Dictionary;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.roaringbitmap.PeekableIntIterator;
+import org.roaringbitmap.RoaringBitmap;
+
+
+public class DistinctCountBitmapAggregationFunction extends BaseSingleInputAggregationFunction<RoaringBitmap, Integer> {
+  protected Dictionary _dictionary;
+
+  public DistinctCountBitmapAggregationFunction(ExpressionContext expression) {
+    super(expression);
+  }
+
+  @Override
+  public AggregationFunctionType getType() {
+    return AggregationFunctionType.DISTINCTCOUNTBITMAP;
+  }
+
+  @Override
+  public void accept(AggregationFunctionVisitorBase visitor) {
+    visitor.visit(this);
+  }
+
+  @Override
+  public AggregationResultHolder createAggregationResultHolder() {
+    return new ObjectAggregationResultHolder();
+  }
+
+  @Override
+  public GroupByResultHolder createGroupByResultHolder(int initialCapacity, int maxCapacity) {
+    return new ObjectGroupByResultHolder(initialCapacity, maxCapacity);
+  }
+
+  @Override
+  public void aggregate(int length, AggregationResultHolder aggregationResultHolder,
+      Map<ExpressionContext, BlockValSet> blockValSetMap) {
+    BlockValSet blockValSet = blockValSetMap.get(_expression);
+
+    // Treat BYTES value as serialized RoaringBitmap
+    DataType valueType = blockValSet.getValueType();
+    if (valueType == DataType.BYTES) {
+      byte[][] bytesValues = blockValSet.getBytesValuesSV();
+      RoaringBitmap bitmap = aggregationResultHolder.getResult();
+      if (bitmap != null) {
+        for (int i = 0; i < length; i++) {
+          bitmap.or(ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[i]));
+        }
+      } else {
+        bitmap = ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[0]);
+        aggregationResultHolder.setValue(bitmap);
+        for (int i = 1; i < length; i++) {
+          bitmap.or(ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[i]));
+        }
+      }
+      return;
+    }
+
+    // For dictionary-encoded expression, store dictionary ids into the bitmap
+    RoaringBitmap bitmap = getBitmap(aggregationResultHolder);
+    Dictionary dictionary = blockValSet.getDictionary();
+    if (dictionary != null) {
+      _dictionary = dictionary;
+      int[] dictIds = blockValSet.getDictionaryIdsSV();
+      bitmap.addN(dictIds, 0, length);
+      return;
+    }
+
+    // For non-dictionary-encoded expression, store hash code of the values into the bitmap
+    switch (valueType) {
+      case INT:
+        int[] intValues = blockValSet.getIntValuesSV();
+        bitmap.addN(intValues, 0, length);
+        break;
+      case LONG:
+        long[] longValues = blockValSet.getLongValuesSV();
+        for (int i = 0; i < length; i++) {
+          bitmap.add(Long.hashCode(longValues[i]));
+        }
+        break;
+      case FLOAT:
+        float[] floatValues = blockValSet.getFloatValuesSV();
+        for (int i = 0; i < length; i++) {
+          bitmap.add(Float.hashCode(floatValues[i]));
+        }
+        break;
+      case DOUBLE:
+        double[] doubleValues = blockValSet.getDoubleValuesSV();
+        for (int i = 0; i < length; i++) {
+          bitmap.add(Double.hashCode(doubleValues[i]));
+        }
+        break;
+      case STRING:
+        String[] stringValues = blockValSet.getStringValuesSV();
+        for (int i = 0; i < length; i++) {
+          bitmap.add(stringValues[i].hashCode());
+        }
+        break;
+      default:
+        throw new IllegalStateException(
+            "Illegal data type for DISTINCT_COUNT_BITMAP aggregation function: " + valueType);
+    }
+  }
+
+  @Override
+  public void aggregateGroupBySV(int length, int[] groupKeyArray, GroupByResultHolder groupByResultHolder,
+      Map<ExpressionContext, BlockValSet> blockValSetMap) {
+    BlockValSet blockValSet = blockValSetMap.get(_expression);
+
+    // Treat BYTES value as serialized RoaringBitmap
+    DataType valueType = blockValSet.getValueType();
+    if (valueType == DataType.BYTES) {
+      byte[][] bytesValues = blockValSet.getBytesValuesSV();
+      for (int i = 0; i < length; i++) {
+        RoaringBitmap value = ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[i]);
+        int groupKey = groupKeyArray[i];
+        RoaringBitmap bitmap = groupByResultHolder.getResult(groupKey);
+        if (bitmap != null) {
+          bitmap.or(value);
+        } else {
+          groupByResultHolder.setValueForKey(groupKey, value);
+        }
+      }
+      return;
+    }
+
+    // For dictionary-encoded expression, store dictionary ids into the bitmap
+    Dictionary dictionary = blockValSet.getDictionary();
+    if (dictionary != null) {
+      _dictionary = dictionary;
+      int[] dictIds = blockValSet.getDictionaryIdsSV();
+      for (int i = 0; i < length; i++) {
+        getBitmap(groupByResultHolder, groupKeyArray[i]).add(dictIds[i]);
+      }
+      return;
+    }
+
+    // For non-dictionary-encoded expression, store hash code of the values into the bitmap
+    switch (valueType) {
+      case INT:
+        int[] intValues = blockValSet.getIntValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(intValues[i]);
+        }
+        break;
+      case LONG:
+        long[] longValues = blockValSet.getLongValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(Long.hashCode(longValues[i]));
+        }
+        break;
+      case FLOAT:
+        float[] floatValues = blockValSet.getFloatValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(Float.hashCode(floatValues[i]));
+        }
+        break;
+      case DOUBLE:
+        double[] doubleValues = blockValSet.getDoubleValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(Double.hashCode(doubleValues[i]));
+        }
+        break;
+      case STRING:
+        String[] stringValues = blockValSet.getStringValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(stringValues[i].hashCode());
+        }
+        break;
+      default:
+        throw new IllegalStateException(
+            "Illegal data type for DISTINCT_COUNT_BITMAP aggregation function: " + valueType);
+    }
+  }
+
+  @Override
+  public void aggregateGroupByMV(int length, int[][] groupKeysArray, GroupByResultHolder groupByResultHolder,
+      Map<ExpressionContext, BlockValSet> blockValSetMap) {
+    BlockValSet blockValSet = blockValSetMap.get(_expression);
+
+    // Treat BYTES value as serialized RoaringBitmap
+    DataType valueType = blockValSet.getValueType();
+    if (valueType == DataType.BYTES) {
+      byte[][] bytesValues = blockValSet.getBytesValuesSV();
+      for (int i = 0; i < length; i++) {
+        RoaringBitmap value = ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[i]);
+        for (int groupKey : groupKeysArray[i]) {
+          RoaringBitmap bitmap = groupByResultHolder.getResult(groupKey);
+          if (bitmap != null) {
+            bitmap.or(value);
+          } else {
+            // Clone a bitmap for the group
+            groupByResultHolder.setValueForKey(groupKey, value.clone());
+          }
+        }
+      }
+      return;
+    }
+
+    // For dictionary-encoded expression, store dictionary ids into the bitmap
+    Dictionary dictionary = blockValSet.getDictionary();
+    if (dictionary != null) {
+      _dictionary = dictionary;
+      int[] dictIds = blockValSet.getDictionaryIdsSV();
+      for (int i = 0; i < length; i++) {
+        setValueForGroupKeys(groupByResultHolder, groupKeysArray[i], dictIds[i]);
+      }
+      return;
+    }
+
+    // For non-dictionary-encoded expression, store hash code of the values into the bitmap
+    switch (valueType) {
+      case INT:
+        int[] intValues = blockValSet.getIntValuesSV();
+        for (int i = 0; i < length; i++) {
+          setValueForGroupKeys(groupByResultHolder, groupKeysArray[i], intValues[i]);
+        }
+        break;
+      case LONG:
+        long[] longValues = blockValSet.getLongValuesSV();
+        for (int i = 0; i < length; i++) {
+          setValueForGroupKeys(groupByResultHolder, groupKeysArray[i], Long.hashCode(longValues[i]));
+        }
+        break;
+      case FLOAT:
+        float[] floatValues = blockValSet.getFloatValuesSV();
+        for (int i = 0; i < length; i++) {
+          setValueForGroupKeys(groupByResultHolder, groupKeysArray[i], Float.hashCode(floatValues[i]));
+        }
+        break;
+      case DOUBLE:
+        double[] doubleValues = blockValSet.getDoubleValuesSV();
+        for (int i = 0; i < length; i++) {
+          setValueForGroupKeys(groupByResultHolder, groupKeysArray[i], Double.hashCode(doubleValues[i]));
+        }
+        break;
+      case STRING:
+        String[] stringValues = blockValSet.getStringValuesSV();
+        for (int i = 0; i < length; i++) {
+          setValueForGroupKeys(groupByResultHolder, groupKeysArray[i], stringValues[i].hashCode());
+        }
+        break;
+      default:
+        throw new IllegalStateException(
+            "Illegal data type for DISTINCT_COUNT_BITMAP aggregation function: " + valueType);
+    }
+  }
+
+  @Override
+  public RoaringBitmap extractAggregationResult(AggregationResultHolder aggregationResultHolder) {
+    RoaringBitmap bitmap = aggregationResultHolder.getResult();
+    if (bitmap == null) {
+      return new RoaringBitmap();
+    }
+
+    if (_dictionary != null) {
+      // For dictionary-encoded expression, convert dictionary ids to hash code of the values
+      return convertToValueBitmap(bitmap, _dictionary);
+    } else {
+      // For serialized RoaringBitmap and non-dictionary-encoded expression, directly return the bitmap
+      return bitmap;
+    }
+  }
+
+  @Override
+  public RoaringBitmap extractGroupByResult(GroupByResultHolder groupByResultHolder, int groupKey) {
+    RoaringBitmap bitmap = groupByResultHolder.getResult(groupKey);
+    if (bitmap == null) {
+      return new RoaringBitmap();
+    }
+
+    if (_dictionary != null) {
+      // For dictionary-encoded expression, convert dictionary ids to hash code of the values
+      return convertToValueBitmap(bitmap, _dictionary);
+    } else {
+      // For serialized RoaringBitmap and non-dictionary-encoded expression, directly return the bitmap
+      return bitmap;
+    }
+  }
+
+  @Override
+  public RoaringBitmap merge(RoaringBitmap intermediateResult1, RoaringBitmap intermediateResult2) {
+    intermediateResult1.or(intermediateResult2);
+    return intermediateResult1;
+  }
+
+  @Override
+  public boolean isIntermediateResultComparable() {
+    return false;
+  }
+
+  @Override
+  public ColumnDataType getIntermediateResultColumnType() {
+    return ColumnDataType.OBJECT;
+  }
+
+  @Override
+  public ColumnDataType getFinalResultColumnType() {
+    return ColumnDataType.INT;
+  }
+
+  @Override
+  public Integer extractFinalResult(RoaringBitmap intermediateResult) {
+    return intermediateResult.getCardinality();
+  }
+
+  /**
+   * Returns the bitmap from the result holder or creates a new one if it does not exist.
+   */
+  protected static RoaringBitmap getBitmap(AggregationResultHolder aggregationResultHolder) {
+    RoaringBitmap bitmap = aggregationResultHolder.getResult();
+    if (bitmap == null) {
+      bitmap = new RoaringBitmap();
+      aggregationResultHolder.setValue(bitmap);
+    }
+    return bitmap;
+  }
+
+  /**
+   * Returns the bitmap for the given group key or creates a new one if it does not exist.
+   */
+  protected static RoaringBitmap getBitmap(GroupByResultHolder groupByResultHolder, int groupKey) {
+    RoaringBitmap bitmap = groupByResultHolder.getResult(groupKey);
+    if (bitmap == null) {
+      bitmap = new RoaringBitmap();
+      groupByResultHolder.setValueForKey(groupKey, bitmap);
+    }
+    return bitmap;
+  }
+
+  /**
+   * Helper method to set value for the given group keys into the result holder.
+   */
+  private void setValueForGroupKeys(GroupByResultHolder groupByResultHolder, int[] groupKeys, int value) {
+    for (int groupKey : groupKeys) {
+      getBitmap(groupByResultHolder, groupKey).add(value);
+    }
+  }
+
+  /**
+   * Helper method to read dictionary and convert dictionary ids to hash code of the values for dictionary-encoded
+   * expression.
+   */
+  private static RoaringBitmap convertToValueBitmap(RoaringBitmap dictIdBitmap, Dictionary dictionary) {

Review comment:
       This doesn't really look like a good use of bitmap for space efficiency purpose. We are storing the hashcodes of real values in the bitmap for result extraction
   Since the range of hashcodes could be arbitrary (as opposed to 0 .. N-1 for dictionaryIds), I wonder if bitmap will end up consuming more space here?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/DistinctCountBitmapMVAggregationFunction.java
##########
@@ -0,0 +1,256 @@
+/**
+ * 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.pinot.core.query.aggregation.function;
+
+import java.util.Map;
+import org.apache.pinot.common.function.AggregationFunctionType;
+import org.apache.pinot.core.common.BlockValSet;
+import org.apache.pinot.core.query.aggregation.AggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.GroupByResultHolder;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.segment.index.readers.Dictionary;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.roaringbitmap.RoaringBitmap;
+
+
+public class DistinctCountBitmapMVAggregationFunction extends DistinctCountBitmapAggregationFunction {

Review comment:
       Javadoc would be good




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5766: Adding distinct count support based on bitmap

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5766:
URL: https://github.com/apache/incubator-pinot/pull/5766#discussion_r463886202



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/DistinctCountBitmapAggregationFunction.java
##########
@@ -0,0 +1,409 @@
+/**
+ * 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.pinot.core.query.aggregation.function;
+
+import java.util.Map;
+import org.apache.pinot.common.function.AggregationFunctionType;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.core.common.BlockValSet;
+import org.apache.pinot.core.common.ObjectSerDeUtils;
+import org.apache.pinot.core.query.aggregation.AggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.ObjectAggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.GroupByResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.ObjectGroupByResultHolder;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.segment.index.readers.Dictionary;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.roaringbitmap.PeekableIntIterator;
+import org.roaringbitmap.RoaringBitmap;
+
+
+public class DistinctCountBitmapAggregationFunction extends BaseSingleInputAggregationFunction<RoaringBitmap, Integer> {
+  protected Dictionary _dictionary;
+
+  public DistinctCountBitmapAggregationFunction(ExpressionContext expression) {
+    super(expression);
+  }
+
+  @Override
+  public AggregationFunctionType getType() {
+    return AggregationFunctionType.DISTINCTCOUNTBITMAP;
+  }
+
+  @Override
+  public void accept(AggregationFunctionVisitorBase visitor) {
+    visitor.visit(this);
+  }
+
+  @Override
+  public AggregationResultHolder createAggregationResultHolder() {
+    return new ObjectAggregationResultHolder();
+  }
+
+  @Override
+  public GroupByResultHolder createGroupByResultHolder(int initialCapacity, int maxCapacity) {
+    return new ObjectGroupByResultHolder(initialCapacity, maxCapacity);
+  }
+
+  @Override
+  public void aggregate(int length, AggregationResultHolder aggregationResultHolder,
+      Map<ExpressionContext, BlockValSet> blockValSetMap) {
+    BlockValSet blockValSet = blockValSetMap.get(_expression);
+
+    // Treat BYTES value as serialized RoaringBitmap
+    DataType valueType = blockValSet.getValueType();
+    if (valueType == DataType.BYTES) {
+      byte[][] bytesValues = blockValSet.getBytesValuesSV();
+      RoaringBitmap bitmap = aggregationResultHolder.getResult();
+      if (bitmap != null) {
+        for (int i = 0; i < length; i++) {
+          bitmap.or(ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[i]));
+        }
+      } else {
+        bitmap = ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[0]);
+        aggregationResultHolder.setValue(bitmap);
+        for (int i = 1; i < length; i++) {
+          bitmap.or(ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[i]));
+        }
+      }
+      return;
+    }
+
+    // For dictionary-encoded expression, store dictionary ids into the bitmap
+    RoaringBitmap bitmap = getBitmap(aggregationResultHolder);
+    Dictionary dictionary = blockValSet.getDictionary();
+    if (dictionary != null) {
+      _dictionary = dictionary;
+      int[] dictIds = blockValSet.getDictionaryIdsSV();
+      bitmap.addN(dictIds, 0, length);
+      return;
+    }
+
+    // For non-dictionary-encoded expression, store hash code of the values into the bitmap
+    switch (valueType) {
+      case INT:
+        int[] intValues = blockValSet.getIntValuesSV();
+        bitmap.addN(intValues, 0, length);
+        break;
+      case LONG:
+        long[] longValues = blockValSet.getLongValuesSV();
+        for (int i = 0; i < length; i++) {
+          bitmap.add(Long.hashCode(longValues[i]));
+        }
+        break;
+      case FLOAT:
+        float[] floatValues = blockValSet.getFloatValuesSV();
+        for (int i = 0; i < length; i++) {
+          bitmap.add(Float.hashCode(floatValues[i]));
+        }
+        break;
+      case DOUBLE:
+        double[] doubleValues = blockValSet.getDoubleValuesSV();
+        for (int i = 0; i < length; i++) {
+          bitmap.add(Double.hashCode(doubleValues[i]));
+        }
+        break;
+      case STRING:
+        String[] stringValues = blockValSet.getStringValuesSV();
+        for (int i = 0; i < length; i++) {
+          bitmap.add(stringValues[i].hashCode());
+        }
+        break;
+      default:
+        throw new IllegalStateException(
+            "Illegal data type for DISTINCT_COUNT_BITMAP aggregation function: " + valueType);
+    }
+  }
+
+  @Override
+  public void aggregateGroupBySV(int length, int[] groupKeyArray, GroupByResultHolder groupByResultHolder,
+      Map<ExpressionContext, BlockValSet> blockValSetMap) {
+    BlockValSet blockValSet = blockValSetMap.get(_expression);
+
+    // Treat BYTES value as serialized RoaringBitmap
+    DataType valueType = blockValSet.getValueType();
+    if (valueType == DataType.BYTES) {
+      byte[][] bytesValues = blockValSet.getBytesValuesSV();
+      for (int i = 0; i < length; i++) {
+        RoaringBitmap value = ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[i]);
+        int groupKey = groupKeyArray[i];
+        RoaringBitmap bitmap = groupByResultHolder.getResult(groupKey);
+        if (bitmap != null) {
+          bitmap.or(value);
+        } else {
+          groupByResultHolder.setValueForKey(groupKey, value);
+        }
+      }
+      return;
+    }
+
+    // For dictionary-encoded expression, store dictionary ids into the bitmap
+    Dictionary dictionary = blockValSet.getDictionary();
+    if (dictionary != null) {
+      _dictionary = dictionary;
+      int[] dictIds = blockValSet.getDictionaryIdsSV();
+      for (int i = 0; i < length; i++) {
+        getBitmap(groupByResultHolder, groupKeyArray[i]).add(dictIds[i]);
+      }
+      return;
+    }
+
+    // For non-dictionary-encoded expression, store hash code of the values into the bitmap
+    switch (valueType) {
+      case INT:
+        int[] intValues = blockValSet.getIntValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(intValues[i]);
+        }
+        break;
+      case LONG:
+        long[] longValues = blockValSet.getLongValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(Long.hashCode(longValues[i]));
+        }
+        break;
+      case FLOAT:
+        float[] floatValues = blockValSet.getFloatValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(Float.hashCode(floatValues[i]));
+        }
+        break;
+      case DOUBLE:
+        double[] doubleValues = blockValSet.getDoubleValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(Double.hashCode(doubleValues[i]));
+        }
+        break;
+      case STRING:
+        String[] stringValues = blockValSet.getStringValuesSV();
+        for (int i = 0; i < length; i++) {
+          getBitmap(groupByResultHolder, groupKeyArray[i]).add(stringValues[i].hashCode());
+        }
+        break;
+      default:
+        throw new IllegalStateException(
+            "Illegal data type for DISTINCT_COUNT_BITMAP aggregation function: " + valueType);
+    }
+  }
+
+  @Override
+  public void aggregateGroupByMV(int length, int[][] groupKeysArray, GroupByResultHolder groupByResultHolder,
+      Map<ExpressionContext, BlockValSet> blockValSetMap) {
+    BlockValSet blockValSet = blockValSetMap.get(_expression);
+
+    // Treat BYTES value as serialized RoaringBitmap
+    DataType valueType = blockValSet.getValueType();
+    if (valueType == DataType.BYTES) {
+      byte[][] bytesValues = blockValSet.getBytesValuesSV();
+      for (int i = 0; i < length; i++) {
+        RoaringBitmap value = ObjectSerDeUtils.ROARING_BITMAP_SER_DE.deserialize(bytesValues[i]);
+        for (int groupKey : groupKeysArray[i]) {
+          RoaringBitmap bitmap = groupByResultHolder.getResult(groupKey);
+          if (bitmap != null) {
+            bitmap.or(value);
+          } else {
+            // Clone a bitmap for the group
+            groupByResultHolder.setValueForKey(groupKey, value.clone());
+          }
+        }
+      }
+      return;
+    }
+
+    // For dictionary-encoded expression, store dictionary ids into the bitmap
+    Dictionary dictionary = blockValSet.getDictionary();
+    if (dictionary != null) {
+      _dictionary = dictionary;
+      int[] dictIds = blockValSet.getDictionaryIdsSV();
+      for (int i = 0; i < length; i++) {
+        setValueForGroupKeys(groupByResultHolder, groupKeysArray[i], dictIds[i]);
+      }
+      return;
+    }
+
+    // For non-dictionary-encoded expression, store hash code of the values into the bitmap
+    switch (valueType) {
+      case INT:
+        int[] intValues = blockValSet.getIntValuesSV();
+        for (int i = 0; i < length; i++) {
+          setValueForGroupKeys(groupByResultHolder, groupKeysArray[i], intValues[i]);
+        }
+        break;
+      case LONG:
+        long[] longValues = blockValSet.getLongValuesSV();
+        for (int i = 0; i < length; i++) {
+          setValueForGroupKeys(groupByResultHolder, groupKeysArray[i], Long.hashCode(longValues[i]));
+        }
+        break;
+      case FLOAT:
+        float[] floatValues = blockValSet.getFloatValuesSV();
+        for (int i = 0; i < length; i++) {
+          setValueForGroupKeys(groupByResultHolder, groupKeysArray[i], Float.hashCode(floatValues[i]));
+        }
+        break;
+      case DOUBLE:
+        double[] doubleValues = blockValSet.getDoubleValuesSV();
+        for (int i = 0; i < length; i++) {
+          setValueForGroupKeys(groupByResultHolder, groupKeysArray[i], Double.hashCode(doubleValues[i]));
+        }
+        break;
+      case STRING:
+        String[] stringValues = blockValSet.getStringValuesSV();
+        for (int i = 0; i < length; i++) {
+          setValueForGroupKeys(groupByResultHolder, groupKeysArray[i], stringValues[i].hashCode());
+        }
+        break;
+      default:
+        throw new IllegalStateException(
+            "Illegal data type for DISTINCT_COUNT_BITMAP aggregation function: " + valueType);
+    }
+  }
+
+  @Override
+  public RoaringBitmap extractAggregationResult(AggregationResultHolder aggregationResultHolder) {
+    RoaringBitmap bitmap = aggregationResultHolder.getResult();
+    if (bitmap == null) {
+      return new RoaringBitmap();
+    }
+
+    if (_dictionary != null) {
+      // For dictionary-encoded expression, convert dictionary ids to hash code of the values
+      return convertToValueBitmap(bitmap, _dictionary);
+    } else {
+      // For serialized RoaringBitmap and non-dictionary-encoded expression, directly return the bitmap
+      return bitmap;
+    }
+  }
+
+  @Override
+  public RoaringBitmap extractGroupByResult(GroupByResultHolder groupByResultHolder, int groupKey) {
+    RoaringBitmap bitmap = groupByResultHolder.getResult(groupKey);
+    if (bitmap == null) {
+      return new RoaringBitmap();
+    }
+
+    if (_dictionary != null) {
+      // For dictionary-encoded expression, convert dictionary ids to hash code of the values
+      return convertToValueBitmap(bitmap, _dictionary);
+    } else {
+      // For serialized RoaringBitmap and non-dictionary-encoded expression, directly return the bitmap
+      return bitmap;
+    }
+  }
+
+  @Override
+  public RoaringBitmap merge(RoaringBitmap intermediateResult1, RoaringBitmap intermediateResult2) {
+    intermediateResult1.or(intermediateResult2);
+    return intermediateResult1;
+  }
+
+  @Override
+  public boolean isIntermediateResultComparable() {
+    return false;
+  }
+
+  @Override
+  public ColumnDataType getIntermediateResultColumnType() {
+    return ColumnDataType.OBJECT;
+  }
+
+  @Override
+  public ColumnDataType getFinalResultColumnType() {
+    return ColumnDataType.INT;
+  }
+
+  @Override
+  public Integer extractFinalResult(RoaringBitmap intermediateResult) {
+    return intermediateResult.getCardinality();
+  }
+
+  /**
+   * Returns the bitmap from the result holder or creates a new one if it does not exist.
+   */
+  protected static RoaringBitmap getBitmap(AggregationResultHolder aggregationResultHolder) {
+    RoaringBitmap bitmap = aggregationResultHolder.getResult();
+    if (bitmap == null) {
+      bitmap = new RoaringBitmap();
+      aggregationResultHolder.setValue(bitmap);
+    }
+    return bitmap;
+  }
+
+  /**
+   * Returns the bitmap for the given group key or creates a new one if it does not exist.
+   */
+  protected static RoaringBitmap getBitmap(GroupByResultHolder groupByResultHolder, int groupKey) {
+    RoaringBitmap bitmap = groupByResultHolder.getResult(groupKey);
+    if (bitmap == null) {
+      bitmap = new RoaringBitmap();
+      groupByResultHolder.setValueForKey(groupKey, bitmap);
+    }
+    return bitmap;
+  }
+
+  /**
+   * Helper method to set value for the given group keys into the result holder.
+   */
+  private void setValueForGroupKeys(GroupByResultHolder groupByResultHolder, int[] groupKeys, int value) {
+    for (int groupKey : groupKeys) {
+      getBitmap(groupByResultHolder, groupKey).add(value);
+    }
+  }
+
+  /**
+   * Helper method to read dictionary and convert dictionary ids to hash code of the values for dictionary-encoded
+   * expression.
+   */
+  private static RoaringBitmap convertToValueBitmap(RoaringBitmap dictIdBitmap, Dictionary dictionary) {

Review comment:
       That is exactly the purpose of using RoaringBitmap, which is the well compressed version of the BitSet. The serialized RoaringBitmap size is much smaller than the serialized IntOpenHashSet.

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/DistinctCountBitmapMVAggregationFunction.java
##########
@@ -0,0 +1,256 @@
+/**
+ * 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.pinot.core.query.aggregation.function;
+
+import java.util.Map;
+import org.apache.pinot.common.function.AggregationFunctionType;
+import org.apache.pinot.core.common.BlockValSet;
+import org.apache.pinot.core.query.aggregation.AggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.GroupByResultHolder;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.segment.index.readers.Dictionary;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.roaringbitmap.RoaringBitmap;
+
+
+public class DistinctCountBitmapMVAggregationFunction extends DistinctCountBitmapAggregationFunction {

Review comment:
       Added




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5766: Adding distinct count support based on bitmap

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5766:
URL: https://github.com/apache/incubator-pinot/pull/5766#discussion_r463709116



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/startree/v2/DistinctCountBitmapStarTreeV2Test.java
##########
@@ -0,0 +1,51 @@
+/**
+ * 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.pinot.core.startree.v2;

Review comment:
       Do we not need to add end-to-end query execution test for the new function without star tree as well




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang merged pull request #5766: Adding distinct count support based on bitmap

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang merged pull request #5766:
URL: https://github.com/apache/incubator-pinot/pull/5766


   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5766: Adding distinct count support based on bitmap

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5766:
URL: https://github.com/apache/incubator-pinot/pull/5766#discussion_r463886290



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/startree/v2/DistinctCountBitmapStarTreeV2Test.java
##########
@@ -0,0 +1,51 @@
+/**
+ * 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.pinot.core.startree.v2;

Review comment:
       Good point, will add




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org