You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "cbalci (via GitHub)" <gi...@apache.org> on 2023/07/13 16:24:22 UTC

[GitHub] [pinot] cbalci opened a new pull request, #11098: Introduce FrequentStringsSketch and FrequentLongsSketch aggregation functions

cbalci opened a new pull request, #11098:
URL: https://github.com/apache/pinot/pull/11098

   Introducing two approximate aggregation function `FrequentStringsSketch` and `FrequentLongsSketch` for estimating the frequencies of items a dataset in a memory efficient way. Functions are based on the [Apache Datasketches](https://datasketches.apache.org/docs/Frequency/FrequencySketchesOverview.html) library.
   
   **Signature:**
   ```
   FREQUENTLONGSSKETCH(col, maxMapSize=256) -> Base64 encoded sketch object
   FREQUENTSTRINGSSKETCH(col, maxMapSize=256) -> Base64 encoded sketch object
   ```
   
   **Example usage:**
   ```
   select FREQUENTSTRINGSSKETCH(AirlineID, 16) from airlineStats
   ```
   
   | frequentstringssketch(AirlineID)      |
   | ----------- |
   | BAEKAwMAAAADAAAAAA...      |
   
   Which can be used, for example in Java as:
   ```java
   byte[] byteArr = Base64.getDecoder().decode(encodedSketch);
   ItemsSketch<String> sketch = ItemsSketch.getInstance(Memory.wrap(byteArr), new ArrayOfStringsSerDe());
   
   ItemsSketch.Row[] items = sketch.getFrequentItems(ErrorType.NO_FALSE_NEGATIVES);
   for (int i = 0; i < items.length; i++) {
     ItemsSketch.Row item = items[i];
     System.out.printf("Airline: %s, Frequency: %d %n", item.getItem(), item.getEstimate());
   }
   ```
   
   **Testing:**
   Basic aggregation and group by query tests are included in the PR.
   
   **Design:**
   This is a part of a larger effort to improve Datasketches support for Pinot as discussed in [this document](https://docs.google.com/document/d/1ctmKVRi67lpO6x1RYKDvDYf05EZx2Vbs2OnUudYP-bU/edit#heading=h.nctch2wugvub). Feel free to add design related comments on the document as well.
   
   
   `feature` `performance`


-- 
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: commits-unsubscribe@pinot.apache.org

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] [pinot] chenboat commented on a diff in pull request #11098: Introduce FrequentStringsSketch and FrequentLongsSketch aggregation functions

Posted by "chenboat (via GitHub)" <gi...@apache.org>.
chenboat commented on code in PR #11098:
URL: https://github.com/apache/pinot/pull/11098#discussion_r1281307520


##########
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/FrequentLongsSketchAggregationFunction.java:
##########
@@ -0,0 +1,251 @@
+/**
+ * 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 com.google.common.base.Preconditions;
+import java.util.List;
+import java.util.Map;
+import org.apache.datasketches.frequencies.LongsSketch;
+import org.apache.datasketches.memory.Memory;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.BlockValSet;
+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.segment.local.customobject.SerializedFrequentLongsSketch;
+import org.apache.pinot.segment.spi.AggregationFunctionType;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * <p>
+ *  {@code FrequentLongsSketchAggregationFunction} provides an approximate FrequentItems aggregation function based on
+ *  <a href="https://datasketches.apache.org/docs/Frequency/FrequentItemsOverview.html">Apache DataSketches library</a>.
+ *  It is memory efficient compared to exact counting.
+ * </p>
+ * <p>
+ *   The function takes an INT or LONG column as input and returns a Base64 encoded sketch object which can be
+ *   deserialized and used to estimate the frequency of items in the dataset (how many times they appear).
+ * </p>
+ * <p><b>FREQUENT_STRINGS_SKETCH(col, maxMapSize=256)</b></p>
+ * <p>E.g.:</p>
+ * <ul>
+ *   <li><b>FREQUENT_LONGS_SKETCH(col)</b></li>
+ *   <li><b>FREQUENT_LONGS_SKETCH(col, 1024)</b></li>
+ * </ul>
+ *
+ * <p>
+ *   If the column type is BYTES, the aggregation function will assume it is a serialized FrequentItems data sketch
+ *   of type `LongsSketch`and will attempt to deserialize it for merging with other sketch objects.
+ * </p>
+ *
+ * <p>
+ *   There is a variation of the function (<b>FREQUENT_STRINGS_SKETCH</b>) which accepts STRING type input columns.
+ * </p>
+ */
+public class FrequentLongsSketchAggregationFunction
+    extends BaseSingleInputAggregationFunction<LongsSketch, Comparable<?>> {
+  protected static final int DEFAULT_MAX_MAP_SIZE = 256;
+
+  protected int _maxMapSize;
+
+  public FrequentLongsSketchAggregationFunction(List<ExpressionContext> arguments) {
+    super(arguments.get(0));
+    int numArguments = arguments.size();
+    Preconditions.checkArgument(numArguments == 1 || numArguments == 2,
+        "Expecting 1 or 2 arguments for FrequentLongsSketch function: FREQUENTITEMSSKETCH(column, maxMapSize");
+    _maxMapSize = numArguments == 2 ? arguments.get(1).getLiteral().getIntValue() : DEFAULT_MAX_MAP_SIZE;
+  }
+
+  @Override
+  public AggregationFunctionType getType() {
+    return AggregationFunctionType.FREQUENTLONGSSKETCH;
+  }
+
+  @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 valueSet = blockValSetMap.get(_expression);
+    FieldSpec.DataType valueType = valueSet.getValueType();
+
+    LongsSketch sketch = getOrCreateSketch(aggregationResultHolder);
+
+    if (valueType == FieldSpec.DataType.BYTES) {

Review Comment:
   NIT: can we use switch-case here?



-- 
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: commits-unsubscribe@pinot.apache.org

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] [pinot] codecov-commenter commented on pull request #11098: Introduce FrequentStringsSketch and FrequentLongsSketch aggregation functions

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #11098:
URL: https://github.com/apache/pinot/pull/11098#issuecomment-1634672632

   ## [Codecov](https://app.codecov.io/gh/apache/pinot/pull/11098?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   > Merging [#11098](https://app.codecov.io/gh/apache/pinot/pull/11098?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (af24e08) into [master](https://app.codecov.io/gh/apache/pinot/commit/20bef67a561f4d78b72a12970547b5ca5306b8df?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (20bef67) will **decrease** coverage by `0.01%`.
   > The diff coverage is `0.00%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #11098      +/-   ##
   ==========================================
   - Coverage    0.11%    0.11%   -0.01%     
   ==========================================
     Files        2203     2207       +4     
     Lines      118144   118359     +215     
     Branches    17877    17920      +43     
   ==========================================
     Hits          137      137              
   - Misses     117987   118202     +215     
     Partials       20       20              
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1temurin11 | `?` | |
   | integration1temurin17 | `0.00% <0.00%> (ø)` | |
   | integration1temurin20 | `?` | |
   | integration2temurin11 | `?` | |
   | integration2temurin17 | `?` | |
   | integration2temurin20 | `?` | |
   | unittests1temurin11 | `0.00% <0.00%> (ø)` | |
   | unittests1temurin17 | `0.00% <0.00%> (ø)` | |
   | unittests1temurin20 | `?` | |
   | unittests2temurin11 | `0.11% <0.00%> (-0.01%)` | :arrow_down: |
   | unittests2temurin17 | `0.11% <0.00%> (-0.01%)` | :arrow_down: |
   | unittests2temurin20 | `0.11% <0.00%> (-0.01%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://app.codecov.io/gh/apache/pinot/pull/11098?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Coverage Δ | |
   |---|---|---|
   | [...org/apache/pinot/core/common/ObjectSerDeUtils.java](https://app.codecov.io/gh/apache/pinot/pull/11098?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9jb21tb24vT2JqZWN0U2VyRGVVdGlscy5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...gregation/function/AggregationFunctionFactory.java](https://app.codecov.io/gh/apache/pinot/pull/11098?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9hZ2dyZWdhdGlvbi9mdW5jdGlvbi9BZ2dyZWdhdGlvbkZ1bmN0aW9uRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...nction/FrequentLongsSketchAggregationFunction.java](https://app.codecov.io/gh/apache/pinot/pull/11098?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9hZ2dyZWdhdGlvbi9mdW5jdGlvbi9GcmVxdWVudExvbmdzU2tldGNoQWdncmVnYXRpb25GdW5jdGlvbi5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...tion/FrequentStringsSketchAggregationFunction.java](https://app.codecov.io/gh/apache/pinot/pull/11098?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9hZ2dyZWdhdGlvbi9mdW5jdGlvbi9GcmVxdWVudFN0cmluZ3NTa2V0Y2hBZ2dyZWdhdGlvbkZ1bmN0aW9uLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...al/customobject/SerializedFrequentLongsSketch.java](https://app.codecov.io/gh/apache/pinot/pull/11098?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9jdXN0b21vYmplY3QvU2VyaWFsaXplZEZyZXF1ZW50TG9uZ3NTa2V0Y2guamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [.../customobject/SerializedFrequentStringsSketch.java](https://app.codecov.io/gh/apache/pinot/pull/11098?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9jdXN0b21vYmplY3QvU2VyaWFsaXplZEZyZXF1ZW50U3RyaW5nc1NrZXRjaC5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...che/pinot/segment/spi/AggregationFunctionType.java](https://app.codecov.io/gh/apache/pinot/pull/11098?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3Qtc2VnbWVudC1zcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3NlZ21lbnQvc3BpL0FnZ3JlZ2F0aW9uRnVuY3Rpb25UeXBlLmphdmE=) | `0.00% <0.00%> (ø)` | |
   
   ... and [3 files with indirect coverage changes](https://app.codecov.io/gh/apache/pinot/pull/11098/indirect-changes?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
   


-- 
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: commits-unsubscribe@pinot.apache.org

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] [pinot] cbalci commented on a diff in pull request #11098: Introduce FrequentStringsSketch and FrequentLongsSketch aggregation functions

Posted by "cbalci (via GitHub)" <gi...@apache.org>.
cbalci commented on code in PR #11098:
URL: https://github.com/apache/pinot/pull/11098#discussion_r1290848973


##########
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/FrequentLongsSketchAggregationFunction.java:
##########
@@ -0,0 +1,251 @@
+/**
+ * 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 com.google.common.base.Preconditions;
+import java.util.List;
+import java.util.Map;
+import org.apache.datasketches.frequencies.LongsSketch;
+import org.apache.datasketches.memory.Memory;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.BlockValSet;
+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.segment.local.customobject.SerializedFrequentLongsSketch;
+import org.apache.pinot.segment.spi.AggregationFunctionType;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * <p>
+ *  {@code FrequentLongsSketchAggregationFunction} provides an approximate FrequentItems aggregation function based on
+ *  <a href="https://datasketches.apache.org/docs/Frequency/FrequentItemsOverview.html">Apache DataSketches library</a>.
+ *  It is memory efficient compared to exact counting.
+ * </p>
+ * <p>
+ *   The function takes an INT or LONG column as input and returns a Base64 encoded sketch object which can be
+ *   deserialized and used to estimate the frequency of items in the dataset (how many times they appear).
+ * </p>
+ * <p><b>FREQUENT_STRINGS_SKETCH(col, maxMapSize=256)</b></p>
+ * <p>E.g.:</p>
+ * <ul>
+ *   <li><b>FREQUENT_LONGS_SKETCH(col)</b></li>
+ *   <li><b>FREQUENT_LONGS_SKETCH(col, 1024)</b></li>
+ * </ul>
+ *
+ * <p>
+ *   If the column type is BYTES, the aggregation function will assume it is a serialized FrequentItems data sketch
+ *   of type `LongsSketch`and will attempt to deserialize it for merging with other sketch objects.
+ * </p>
+ *
+ * <p>
+ *   There is a variation of the function (<b>FREQUENT_STRINGS_SKETCH</b>) which accepts STRING type input columns.
+ * </p>
+ */
+public class FrequentLongsSketchAggregationFunction
+    extends BaseSingleInputAggregationFunction<LongsSketch, Comparable<?>> {
+  protected static final int DEFAULT_MAX_MAP_SIZE = 256;
+
+  protected int _maxMapSize;
+
+  public FrequentLongsSketchAggregationFunction(List<ExpressionContext> arguments) {
+    super(arguments.get(0));
+    int numArguments = arguments.size();
+    Preconditions.checkArgument(numArguments == 1 || numArguments == 2,
+        "Expecting 1 or 2 arguments for FrequentLongsSketch function: FREQUENTITEMSSKETCH(column, maxMapSize");
+    _maxMapSize = numArguments == 2 ? arguments.get(1).getLiteral().getIntValue() : DEFAULT_MAX_MAP_SIZE;
+  }
+
+  @Override
+  public AggregationFunctionType getType() {
+    return AggregationFunctionType.FREQUENTLONGSSKETCH;
+  }
+
+  @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 valueSet = blockValSetMap.get(_expression);
+    FieldSpec.DataType valueType = valueSet.getValueType();
+
+    LongsSketch sketch = getOrCreateSketch(aggregationResultHolder);
+
+    if (valueType == FieldSpec.DataType.BYTES) {
+      // Assuming the column contains serialized data sketch
+      LongsSketch[] deserializedSketches =
+          deserializeSketches(blockValSetMap.get(_expression).getBytesValuesSV());
+      sketch = getOrCreateSketch(aggregationResultHolder);
+
+      for (int i = 0; i < length; i++) {

Review Comment:
   Done



##########
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/FrequentLongsSketchAggregationFunction.java:
##########
@@ -0,0 +1,251 @@
+/**
+ * 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 com.google.common.base.Preconditions;
+import java.util.List;
+import java.util.Map;
+import org.apache.datasketches.frequencies.LongsSketch;
+import org.apache.datasketches.memory.Memory;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.BlockValSet;
+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.segment.local.customobject.SerializedFrequentLongsSketch;
+import org.apache.pinot.segment.spi.AggregationFunctionType;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * <p>
+ *  {@code FrequentLongsSketchAggregationFunction} provides an approximate FrequentItems aggregation function based on
+ *  <a href="https://datasketches.apache.org/docs/Frequency/FrequentItemsOverview.html">Apache DataSketches library</a>.
+ *  It is memory efficient compared to exact counting.
+ * </p>
+ * <p>
+ *   The function takes an INT or LONG column as input and returns a Base64 encoded sketch object which can be
+ *   deserialized and used to estimate the frequency of items in the dataset (how many times they appear).
+ * </p>
+ * <p><b>FREQUENT_STRINGS_SKETCH(col, maxMapSize=256)</b></p>
+ * <p>E.g.:</p>
+ * <ul>
+ *   <li><b>FREQUENT_LONGS_SKETCH(col)</b></li>
+ *   <li><b>FREQUENT_LONGS_SKETCH(col, 1024)</b></li>
+ * </ul>
+ *
+ * <p>
+ *   If the column type is BYTES, the aggregation function will assume it is a serialized FrequentItems data sketch
+ *   of type `LongsSketch`and will attempt to deserialize it for merging with other sketch objects.
+ * </p>
+ *
+ * <p>
+ *   There is a variation of the function (<b>FREQUENT_STRINGS_SKETCH</b>) which accepts STRING type input columns.
+ * </p>
+ */
+public class FrequentLongsSketchAggregationFunction
+    extends BaseSingleInputAggregationFunction<LongsSketch, Comparable<?>> {
+  protected static final int DEFAULT_MAX_MAP_SIZE = 256;
+
+  protected int _maxMapSize;
+
+  public FrequentLongsSketchAggregationFunction(List<ExpressionContext> arguments) {
+    super(arguments.get(0));
+    int numArguments = arguments.size();
+    Preconditions.checkArgument(numArguments == 1 || numArguments == 2,
+        "Expecting 1 or 2 arguments for FrequentLongsSketch function: FREQUENTITEMSSKETCH(column, maxMapSize");
+    _maxMapSize = numArguments == 2 ? arguments.get(1).getLiteral().getIntValue() : DEFAULT_MAX_MAP_SIZE;
+  }
+
+  @Override
+  public AggregationFunctionType getType() {
+    return AggregationFunctionType.FREQUENTLONGSSKETCH;
+  }
+
+  @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 valueSet = blockValSetMap.get(_expression);
+    FieldSpec.DataType valueType = valueSet.getValueType();
+
+    LongsSketch sketch = getOrCreateSketch(aggregationResultHolder);
+
+    if (valueType == FieldSpec.DataType.BYTES) {

Review Comment:
   Updated



-- 
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: commits-unsubscribe@pinot.apache.org

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] [pinot] chenboat commented on a diff in pull request #11098: Introduce FrequentStringsSketch and FrequentLongsSketch aggregation functions

Posted by "chenboat (via GitHub)" <gi...@apache.org>.
chenboat commented on code in PR #11098:
URL: https://github.com/apache/pinot/pull/11098#discussion_r1281305652


##########
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/FrequentLongsSketchAggregationFunction.java:
##########
@@ -0,0 +1,251 @@
+/**
+ * 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 com.google.common.base.Preconditions;
+import java.util.List;
+import java.util.Map;
+import org.apache.datasketches.frequencies.LongsSketch;
+import org.apache.datasketches.memory.Memory;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.BlockValSet;
+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.segment.local.customobject.SerializedFrequentLongsSketch;
+import org.apache.pinot.segment.spi.AggregationFunctionType;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * <p>
+ *  {@code FrequentLongsSketchAggregationFunction} provides an approximate FrequentItems aggregation function based on
+ *  <a href="https://datasketches.apache.org/docs/Frequency/FrequentItemsOverview.html">Apache DataSketches library</a>.
+ *  It is memory efficient compared to exact counting.
+ * </p>
+ * <p>
+ *   The function takes an INT or LONG column as input and returns a Base64 encoded sketch object which can be
+ *   deserialized and used to estimate the frequency of items in the dataset (how many times they appear).
+ * </p>
+ * <p><b>FREQUENT_STRINGS_SKETCH(col, maxMapSize=256)</b></p>

Review Comment:
   Can you explain what maxMapSize means exactly in the Javadoc? Also for users what is the recommend value? Is bigger always better? Any tradeoff? 



-- 
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: commits-unsubscribe@pinot.apache.org

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] [pinot] cbalci commented on a diff in pull request #11098: Introduce FrequentStringsSketch and FrequentLongsSketch aggregation functions

Posted by "cbalci (via GitHub)" <gi...@apache.org>.
cbalci commented on code in PR #11098:
URL: https://github.com/apache/pinot/pull/11098#discussion_r1290848886


##########
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/FrequentLongsSketchAggregationFunction.java:
##########
@@ -0,0 +1,251 @@
+/**
+ * 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 com.google.common.base.Preconditions;
+import java.util.List;
+import java.util.Map;
+import org.apache.datasketches.frequencies.LongsSketch;
+import org.apache.datasketches.memory.Memory;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.BlockValSet;
+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.segment.local.customobject.SerializedFrequentLongsSketch;
+import org.apache.pinot.segment.spi.AggregationFunctionType;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * <p>
+ *  {@code FrequentLongsSketchAggregationFunction} provides an approximate FrequentItems aggregation function based on
+ *  <a href="https://datasketches.apache.org/docs/Frequency/FrequentItemsOverview.html">Apache DataSketches library</a>.
+ *  It is memory efficient compared to exact counting.
+ * </p>
+ * <p>
+ *   The function takes an INT or LONG column as input and returns a Base64 encoded sketch object which can be
+ *   deserialized and used to estimate the frequency of items in the dataset (how many times they appear).
+ * </p>
+ * <p><b>FREQUENT_STRINGS_SKETCH(col, maxMapSize=256)</b></p>

Review Comment:
   Good point. I added some basic explanation to the Javadoc:
   
   > Second argument, maxMapsSize, refers to the size of the physical length of the hashmap which stores counts. It influences the accuracy of the sketch and should be a power of 2.
   
    I'll also follow up with a more detailed description for the function and how to use it in Pinot Docs repo.



-- 
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: commits-unsubscribe@pinot.apache.org

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] [pinot] chenboat merged pull request #11098: Introduce FrequentStringsSketch and FrequentLongsSketch aggregation functions

Posted by "chenboat (via GitHub)" <gi...@apache.org>.
chenboat merged PR #11098:
URL: https://github.com/apache/pinot/pull/11098


-- 
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: commits-unsubscribe@pinot.apache.org

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] [pinot] cbalci commented on pull request #11098: Introduce FrequentStringsSketch and FrequentLongsSketch aggregation functions

Posted by "cbalci (via GitHub)" <gi...@apache.org>.
cbalci commented on PR #11098:
URL: https://github.com/apache/pinot/pull/11098#issuecomment-1634715091

   @chenboat @Jackie-Jiang please take a look. 
   cc @mayankshriv 


-- 
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: commits-unsubscribe@pinot.apache.org

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] [pinot] cbalci commented on a diff in pull request #11098: Introduce FrequentStringsSketch and FrequentLongsSketch aggregation functions

Posted by "cbalci (via GitHub)" <gi...@apache.org>.
cbalci commented on code in PR #11098:
URL: https://github.com/apache/pinot/pull/11098#discussion_r1290848886


##########
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/FrequentLongsSketchAggregationFunction.java:
##########
@@ -0,0 +1,251 @@
+/**
+ * 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 com.google.common.base.Preconditions;
+import java.util.List;
+import java.util.Map;
+import org.apache.datasketches.frequencies.LongsSketch;
+import org.apache.datasketches.memory.Memory;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.BlockValSet;
+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.segment.local.customobject.SerializedFrequentLongsSketch;
+import org.apache.pinot.segment.spi.AggregationFunctionType;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * <p>
+ *  {@code FrequentLongsSketchAggregationFunction} provides an approximate FrequentItems aggregation function based on
+ *  <a href="https://datasketches.apache.org/docs/Frequency/FrequentItemsOverview.html">Apache DataSketches library</a>.
+ *  It is memory efficient compared to exact counting.
+ * </p>
+ * <p>
+ *   The function takes an INT or LONG column as input and returns a Base64 encoded sketch object which can be
+ *   deserialized and used to estimate the frequency of items in the dataset (how many times they appear).
+ * </p>
+ * <p><b>FREQUENT_STRINGS_SKETCH(col, maxMapSize=256)</b></p>

Review Comment:
   Good point. Added some basic explanation to the Javadoc. I'll also follow up with a more detailed description for the function and how to use it in Pinot Docs repo.



-- 
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: commits-unsubscribe@pinot.apache.org

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] [pinot] cbalci commented on pull request #11098: Introduce FrequentStringsSketch and FrequentLongsSketch aggregation functions

Posted by "cbalci (via GitHub)" <gi...@apache.org>.
cbalci commented on PR #11098:
URL: https://github.com/apache/pinot/pull/11098#issuecomment-1674156926

   Thanks for the review @chenboat. Addressed the comments, please take another look when you get a chance.


-- 
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: commits-unsubscribe@pinot.apache.org

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] [pinot] chenboat commented on a diff in pull request #11098: Introduce FrequentStringsSketch and FrequentLongsSketch aggregation functions

Posted by "chenboat (via GitHub)" <gi...@apache.org>.
chenboat commented on code in PR #11098:
URL: https://github.com/apache/pinot/pull/11098#discussion_r1281306126


##########
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/FrequentLongsSketchAggregationFunction.java:
##########
@@ -0,0 +1,251 @@
+/**
+ * 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 com.google.common.base.Preconditions;
+import java.util.List;
+import java.util.Map;
+import org.apache.datasketches.frequencies.LongsSketch;
+import org.apache.datasketches.memory.Memory;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.BlockValSet;
+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.segment.local.customobject.SerializedFrequentLongsSketch;
+import org.apache.pinot.segment.spi.AggregationFunctionType;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * <p>
+ *  {@code FrequentLongsSketchAggregationFunction} provides an approximate FrequentItems aggregation function based on
+ *  <a href="https://datasketches.apache.org/docs/Frequency/FrequentItemsOverview.html">Apache DataSketches library</a>.
+ *  It is memory efficient compared to exact counting.
+ * </p>
+ * <p>
+ *   The function takes an INT or LONG column as input and returns a Base64 encoded sketch object which can be
+ *   deserialized and used to estimate the frequency of items in the dataset (how many times they appear).
+ * </p>
+ * <p><b>FREQUENT_STRINGS_SKETCH(col, maxMapSize=256)</b></p>
+ * <p>E.g.:</p>
+ * <ul>
+ *   <li><b>FREQUENT_LONGS_SKETCH(col)</b></li>
+ *   <li><b>FREQUENT_LONGS_SKETCH(col, 1024)</b></li>
+ * </ul>
+ *
+ * <p>
+ *   If the column type is BYTES, the aggregation function will assume it is a serialized FrequentItems data sketch
+ *   of type `LongsSketch`and will attempt to deserialize it for merging with other sketch objects.
+ * </p>
+ *
+ * <p>
+ *   There is a variation of the function (<b>FREQUENT_STRINGS_SKETCH</b>) which accepts STRING type input columns.
+ * </p>
+ */
+public class FrequentLongsSketchAggregationFunction
+    extends BaseSingleInputAggregationFunction<LongsSketch, Comparable<?>> {
+  protected static final int DEFAULT_MAX_MAP_SIZE = 256;
+
+  protected int _maxMapSize;
+
+  public FrequentLongsSketchAggregationFunction(List<ExpressionContext> arguments) {
+    super(arguments.get(0));
+    int numArguments = arguments.size();
+    Preconditions.checkArgument(numArguments == 1 || numArguments == 2,
+        "Expecting 1 or 2 arguments for FrequentLongsSketch function: FREQUENTITEMSSKETCH(column, maxMapSize");
+    _maxMapSize = numArguments == 2 ? arguments.get(1).getLiteral().getIntValue() : DEFAULT_MAX_MAP_SIZE;
+  }
+
+  @Override
+  public AggregationFunctionType getType() {
+    return AggregationFunctionType.FREQUENTLONGSSKETCH;
+  }
+
+  @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 valueSet = blockValSetMap.get(_expression);
+    FieldSpec.DataType valueType = valueSet.getValueType();
+
+    LongsSketch sketch = getOrCreateSketch(aggregationResultHolder);
+
+    if (valueType == FieldSpec.DataType.BYTES) {
+      // Assuming the column contains serialized data sketch
+      LongsSketch[] deserializedSketches =
+          deserializeSketches(blockValSetMap.get(_expression).getBytesValuesSV());
+      sketch = getOrCreateSketch(aggregationResultHolder);
+
+      for (int i = 0; i < length; i++) {

Review Comment:
   NIT: can we use foreach syntax to make it more readable?



-- 
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: commits-unsubscribe@pinot.apache.org

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