You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ja...@apache.org on 2023/05/23 18:33:04 UTC

[pinot] branch master updated: Clean up KLL functions with deprecated convention (#10795)

This is an automated email from the ASF dual-hosted git repository.

jackie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/pinot.git


The following commit(s) were added to refs/heads/master by this push:
     new 101b27e2d0 Clean up KLL functions with deprecated convention (#10795)
101b27e2d0 is described below

commit 101b27e2d014e016a5c2ac787c42a04b206ea4b2
Author: Xiaotian (Jackie) Jiang <17...@users.noreply.github.com>
AuthorDate: Tue May 23 11:32:58 2023 -0700

    Clean up KLL functions with deprecated convention (#10795)
---
 .../function/AggregationFunctionFactory.java       | 75 ++++++----------------
 .../function/PercentileKLLAggregationFunction.java | 33 ++++------
 .../PercentileKLLMVAggregationFunction.java        |  7 +-
 .../PercentileRawKLLAggregationFunction.java       |  7 +-
 .../PercentileRawKLLMVAggregationFunction.java     |  7 +-
 ...SegmentAggregationMultiValueRawQueriesTest.java | 24 ++++---
 .../pinot/queries/PercentileKLLMVQueriesTest.java  |  6 +-
 .../pinot/queries/PercentileKLLQueriesTest.java    | 28 +++-----
 8 files changed, 61 insertions(+), 126 deletions(-)

diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/AggregationFunctionFactory.java b/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/AggregationFunctionFactory.java
index 418b864400..f61375dc06 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/AggregationFunctionFactory.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/AggregationFunctionFactory.java
@@ -25,7 +25,7 @@ import org.apache.pinot.common.request.context.ExpressionContext;
 import org.apache.pinot.common.request.context.FunctionContext;
 import org.apache.pinot.core.query.request.context.QueryContext;
 import org.apache.pinot.segment.spi.AggregationFunctionType;
-import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
 import org.apache.pinot.spi.exception.BadQueryRequestException;
 
 
@@ -55,20 +55,22 @@ public class AggregationFunctionFactory {
         if (remainingFunctionName.equals("SMARTTDIGEST")) {
           return new PercentileSmartTDigestAggregationFunction(arguments);
         }
-        if (remainingFunctionName.contains("KLL")) {
-          if (remainingFunctionName.equals("KLL")) {
-            return new PercentileKLLAggregationFunction(arguments);
-          } else if (remainingFunctionName.equals("KLLMV")) {
-            return new PercentileKLLMVAggregationFunction(arguments);
-          } else if (remainingFunctionName.equals("RAWKLL")) {
-            return new PercentileRawKLLAggregationFunction(arguments);
-          } else if (remainingFunctionName.equals("RAWKLLMV")) {
-            return new PercentileRawKLLMVAggregationFunction(arguments);
-          }
+        if (remainingFunctionName.equals("KLL")) {
+          return new PercentileKLLAggregationFunction(arguments);
+        }
+        if (remainingFunctionName.equals("KLLMV")) {
+          return new PercentileKLLMVAggregationFunction(arguments);
+        }
+        if (remainingFunctionName.equals("RAWKLL")) {
+          return new PercentileRawKLLAggregationFunction(arguments);
+        }
+        if (remainingFunctionName.equals("RAWKLLMV")) {
+          return new PercentileRawKLLMVAggregationFunction(arguments);
         }
         int numArguments = arguments.size();
         if (numArguments == 1) {
           // Single argument percentile (e.g. Percentile99(foo), PercentileTDigest95(bar), etc.)
+          // NOTE: This convention is deprecated. DO NOT add new functions here
           if (remainingFunctionName.matches("\\d+")) {
             // Percentile
             return new PercentileAggregationFunction(firstArgument, parsePercentileToInt(remainingFunctionName));
@@ -88,14 +90,6 @@ public class AggregationFunctionFactory {
             // PercentileRawTDigest
             String percentileString = remainingFunctionName.substring(10);
             return new PercentileRawTDigestAggregationFunction(firstArgument, parsePercentileToInt(percentileString));
-          } else if (remainingFunctionName.matches("KLL\\d+")) {
-            // PercentileKLL
-            String percentileString = remainingFunctionName.substring(3);
-            return new PercentileKLLAggregationFunction(firstArgument, parsePercentileToInt(percentileString));
-          } else if (remainingFunctionName.matches("RAWKLL\\d+")) {
-            // PercentileRawKLL
-            String percentileString = remainingFunctionName.substring(6);
-            return new PercentileRawKLLAggregationFunction(firstArgument, parsePercentileToInt(percentileString));
           } else if (remainingFunctionName.matches("\\d+MV")) {
             // PercentileMV
             String percentileString = remainingFunctionName.substring(0, remainingFunctionName.length() - 2);
@@ -116,14 +110,6 @@ public class AggregationFunctionFactory {
             // PercentileRawTDigestMV
             String percentileString = remainingFunctionName.substring(10, remainingFunctionName.length() - 2);
             return new PercentileRawTDigestMVAggregationFunction(firstArgument, parsePercentileToInt(percentileString));
-          } else if (remainingFunctionName.matches("KLL\\d+MV")) {
-            // PercentileKLLMV
-            String percentileString = remainingFunctionName.substring(3, remainingFunctionName.length() - 2);
-            return new PercentileKLLMVAggregationFunction(firstArgument, parsePercentileToInt(percentileString));
-          } else if (remainingFunctionName.matches("RAWKLL\\d+MV")) {
-            // PercentileRawKLLMV
-            String percentileString = remainingFunctionName.substring(6, remainingFunctionName.length() - 2);
-            return new PercentileRawKLLMVAggregationFunction(firstArgument, parsePercentileToInt(percentileString));
           }
         } else if (numArguments == 2) {
           // Double arguments percentile (e.g. percentile(foo, 99), percentileTDigest(bar, 95), etc.) where the
@@ -150,14 +136,6 @@ public class AggregationFunctionFactory {
             // PercentileRawTDigest
             return new PercentileRawTDigestAggregationFunction(firstArgument, percentile);
           }
-          if (remainingFunctionName.equals("KLL")) {
-            // PercentileKLL
-            return new PercentileKLLAggregationFunction(firstArgument, percentile);
-          }
-          if (remainingFunctionName.equals("RAWKLL")) {
-            // PercentileRawKLL
-            return new PercentileRawKLLAggregationFunction(firstArgument, percentile);
-          }
           if (remainingFunctionName.equals("MV")) {
             // PercentileMV
             return new PercentileMVAggregationFunction(firstArgument, percentile);
@@ -178,14 +156,6 @@ public class AggregationFunctionFactory {
             // PercentileRawTDigestMV
             return new PercentileRawTDigestMVAggregationFunction(firstArgument, percentile);
           }
-          if (remainingFunctionName.equals("KLLMV")) {
-            // PercentileKLLMV
-            return new PercentileKLLMVAggregationFunction(firstArgument, percentile);
-          }
-          if (remainingFunctionName.equals("RAWKLLMV")) {
-            // PercentileRawKLLMV
-            return new PercentileRawKLLMVAggregationFunction(firstArgument, percentile);
-          }
         } else if (numArguments == 3) {
           // Triple arguments percentile (e.g. percentileTDigest(bar, 95, 1000), etc.) where the
           // second argument is a decimal number from 0.0 to 100.0 and third argument is a decimal number indicating
@@ -237,13 +207,12 @@ public class AggregationFunctionFactory {
                 throw new IllegalArgumentException("Third argument of firstWithTime Function should be literal."
                     + " The function can be used as firstWithTime(dataColumn, timeColumn, 'dataType')");
               }
-              FieldSpec.DataType fieldDataType
-                  = FieldSpec.DataType.valueOf(dataType.getLiteral().getStringValue().toUpperCase());
+              DataType fieldDataType = DataType.valueOf(dataType.getLiteral().getStringValue().toUpperCase());
               switch (fieldDataType) {
                 case BOOLEAN:
                 case INT:
-                  return new FirstIntValueWithTimeAggregationFunction(
-                      firstArgument, timeCol, fieldDataType == FieldSpec.DataType.BOOLEAN);
+                  return new FirstIntValueWithTimeAggregationFunction(firstArgument, timeCol,
+                      fieldDataType == DataType.BOOLEAN);
                 case LONG:
                   return new FirstLongValueWithTimeAggregationFunction(firstArgument, timeCol);
                 case FLOAT:
@@ -267,13 +236,12 @@ public class AggregationFunctionFactory {
                 throw new IllegalArgumentException("Third argument of lastWithTime Function should be literal."
                     + " The function can be used as lastWithTime(dataColumn, timeColumn, 'dataType')");
               }
-              FieldSpec.DataType fieldDataType =
-                  FieldSpec.DataType.valueOf(dataType.getLiteral().getStringValue().toUpperCase());
+              DataType fieldDataType = DataType.valueOf(dataType.getLiteral().getStringValue().toUpperCase());
               switch (fieldDataType) {
                 case BOOLEAN:
                 case INT:
                   return new LastIntValueWithTimeAggregationFunction(firstArgument, timeCol,
-                      fieldDataType == FieldSpec.DataType.BOOLEAN);
+                      fieldDataType == DataType.BOOLEAN);
                 case LONG:
                   return new LastLongValueWithTimeAggregationFunction(firstArgument, timeCol);
                 case FLOAT:
@@ -378,15 +346,14 @@ public class AggregationFunctionFactory {
             return new ChildArgMinMaxAggregationFunction(arguments, false);
           case ARGMAX:
           case ARGMIN:
-            throw new IllegalArgumentException("Aggregation function: " + function
-                + " is only supported in selection without alias.");
+            throw new IllegalArgumentException(
+                "Aggregation function: " + function + " is only supported in selection without alias.");
           default:
             throw new IllegalArgumentException();
         }
       }
     } catch (Exception e) {
-      throw new BadQueryRequestException(
-          "Invalid aggregation function: " + function + "; Reason: " + e.getMessage());
+      throw new BadQueryRequestException("Invalid aggregation function: " + function + "; Reason: " + e.getMessage());
     }
   }
 
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/PercentileKLLAggregationFunction.java b/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/PercentileKLLAggregationFunction.java
index 3584aafa41..6d2b3b8697 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/PercentileKLLAggregationFunction.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/PercentileKLLAggregationFunction.java
@@ -33,6 +33,7 @@ import org.apache.pinot.core.query.aggregation.groupby.ObjectGroupByResultHolder
 import org.apache.pinot.segment.spi.AggregationFunctionType;
 import org.apache.pinot.spi.data.FieldSpec.DataType;
 
+
 /**
  * <p>
  *  {@code PercentileKLLAggregationFunction} provides an approximate percentile calculator using the KLL algorithm
@@ -60,10 +61,11 @@ import org.apache.pinot.spi.data.FieldSpec.DataType;
  * </p>
  */
 public class PercentileKLLAggregationFunction
-    extends BaseSingleInputAggregationFunction<KllDoublesSketch, Comparable> {
+    extends BaseSingleInputAggregationFunction<KllDoublesSketch, Comparable<?>> {
+  protected static final int DEFAULT_K_VALUE = 200;
 
   protected final double _percentile;
-  protected int _kValue = 200; // size of the sketch. This is the default size used by DataSketches lib as well
+  protected int _kValue;
 
   public PercentileKLLAggregationFunction(List<ExpressionContext> arguments) {
     super(arguments.get(0));
@@ -71,20 +73,13 @@ public class PercentileKLLAggregationFunction
     // Check that there are correct number of arguments
     int numArguments = arguments.size();
     Preconditions.checkArgument(numArguments == 2 || numArguments == 3,
-        "Expecting 2 or 3 arguments for PercentileKLL function: "
-            + "PERCENTILE_KLL(column, percentile, k=200");
+        "Expecting 2 or 3 arguments for PercentileKLL function: PERCENTILE_KLL(column, percentile, k=200");
 
     _percentile = arguments.get(1).getLiteral().getDoubleValue();
     Preconditions.checkArgument(_percentile >= 0 && _percentile <= 100,
-            "Percentile value needs to be in range 0-100, inclusive");
-    if (numArguments == 3) {
-      _kValue = arguments.get(2).getLiteral().getIntValue();
-    }
-  }
+        "Percentile value needs to be in range 0-100, inclusive");
 
-  public PercentileKLLAggregationFunction(ExpressionContext expression, double percentile) {
-    super(expression);
-    _percentile = percentile;
+    _kValue = numArguments == 3 ? arguments.get(2).getLiteral().getIntValue() : DEFAULT_K_VALUE;
   }
 
   @Override
@@ -111,8 +106,7 @@ public class PercentileKLLAggregationFunction
 
     if (valueType == DataType.BYTES) {
       // Assuming the column contains serialized data sketch
-      KllDoublesSketch[] deserializedSketches =
-          deserializeSketches(blockValSetMap.get(_expression).getBytesValuesSV());
+      KllDoublesSketch[] deserializedSketches = deserializeSketches(blockValSetMap.get(_expression).getBytesValuesSV());
       for (int i = 0; i < length; i++) {
         sketch.merge(deserializedSketches[i]);
       }
@@ -132,8 +126,7 @@ public class PercentileKLLAggregationFunction
 
     if (valueType == DataType.BYTES) {
       // serialized sketch
-      KllDoublesSketch[] deserializedSketches =
-          deserializeSketches(blockValSetMap.get(_expression).getBytesValuesSV());
+      KllDoublesSketch[] deserializedSketches = deserializeSketches(blockValSetMap.get(_expression).getBytesValuesSV());
       for (int i = 0; i < length; i++) {
         KllDoublesSketch sketch = getOrCreateSketch(groupByResultHolder, groupKeyArray[i]);
         sketch.merge(deserializedSketches[i]);
@@ -155,8 +148,7 @@ public class PercentileKLLAggregationFunction
 
     if (valueType == DataType.BYTES) {
       // serialized sketch
-      KllDoublesSketch[] deserializedSketches =
-          deserializeSketches(blockValSetMap.get(_expression).getBytesValuesSV());
+      KllDoublesSketch[] deserializedSketches = deserializeSketches(blockValSetMap.get(_expression).getBytesValuesSV());
       for (int i = 0; i < length; i++) {
         for (int groupKey : groupKeysArray[i]) {
           KllDoublesSketch sketch = getOrCreateSketch(groupByResultHolder, groupKey);
@@ -244,12 +236,11 @@ public class PercentileKLLAggregationFunction
 
   @Override
   public String getResultColumnName() {
-    return AggregationFunctionType.PERCENTILEKLL.getName().toLowerCase()
-        + "(" + _expression + ", " + _percentile + ")";
+    return AggregationFunctionType.PERCENTILEKLL.getName().toLowerCase() + "(" + _expression + ", " + _percentile + ")";
   }
 
   @Override
-  public Comparable extractFinalResult(KllDoublesSketch sketch) {
+  public Comparable<?> extractFinalResult(KllDoublesSketch sketch) {
     return sketch.getQuantile(_percentile / 100);
   }
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/PercentileKLLMVAggregationFunction.java b/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/PercentileKLLMVAggregationFunction.java
index 61d1bd0bce..4653e9051d 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/PercentileKLLMVAggregationFunction.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/PercentileKLLMVAggregationFunction.java
@@ -30,9 +30,6 @@ import org.apache.pinot.spi.data.FieldSpec.DataType;
 
 
 public class PercentileKLLMVAggregationFunction extends PercentileKLLAggregationFunction {
-  public PercentileKLLMVAggregationFunction(ExpressionContext expression, double percentile) {
-    super(expression, percentile);
-  }
 
   public PercentileKLLMVAggregationFunction(List<ExpressionContext> arguments) {
     super(arguments);
@@ -120,7 +117,7 @@ public class PercentileKLLMVAggregationFunction extends PercentileKLLAggregation
 
   @Override
   public String getResultColumnName() {
-    return AggregationFunctionType.PERCENTILEKLLMV.getName().toLowerCase()
-        + "(" + _expression + ", " + _percentile + ")";
+    return AggregationFunctionType.PERCENTILEKLLMV.getName().toLowerCase() + "(" + _expression + ", " + _percentile
+        + ")";
   }
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/PercentileRawKLLAggregationFunction.java b/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/PercentileRawKLLAggregationFunction.java
index 48bd421ee3..39c2022ff0 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/PercentileRawKLLAggregationFunction.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/PercentileRawKLLAggregationFunction.java
@@ -27,9 +27,6 @@ import org.apache.pinot.segment.spi.AggregationFunctionType;
 
 
 public class PercentileRawKLLAggregationFunction extends PercentileKLLAggregationFunction {
-  public PercentileRawKLLAggregationFunction(ExpressionContext expression, double percentile) {
-    super(expression, percentile);
-  }
 
   public PercentileRawKLLAggregationFunction(List<ExpressionContext> arguments) {
     super(arguments);
@@ -47,8 +44,8 @@ public class PercentileRawKLLAggregationFunction extends PercentileKLLAggregatio
 
   @Override
   public String getResultColumnName() {
-    return AggregationFunctionType.PERCENTILERAWKLL.getName().toLowerCase()
-        + "(" + _expression + ", " + _percentile + ")";
+    return AggregationFunctionType.PERCENTILERAWKLL.getName().toLowerCase() + "(" + _expression + ", " + _percentile
+        + ")";
   }
 
   @Override
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/PercentileRawKLLMVAggregationFunction.java b/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/PercentileRawKLLMVAggregationFunction.java
index 3c0885e970..80a6a3ac6b 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/PercentileRawKLLMVAggregationFunction.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/PercentileRawKLLMVAggregationFunction.java
@@ -27,9 +27,6 @@ import org.apache.pinot.segment.spi.AggregationFunctionType;
 
 
 public class PercentileRawKLLMVAggregationFunction extends PercentileKLLMVAggregationFunction {
-  public PercentileRawKLLMVAggregationFunction(ExpressionContext expression, double percentile) {
-    super(expression, percentile);
-  }
 
   public PercentileRawKLLMVAggregationFunction(List<ExpressionContext> arguments) {
     super(arguments);
@@ -47,8 +44,8 @@ public class PercentileRawKLLMVAggregationFunction extends PercentileKLLMVAggreg
 
   @Override
   public String getResultColumnName() {
-    return AggregationFunctionType.PERCENTILERAWKLLMV.getName().toLowerCase()
-        + "(" + _expression + ", " + _percentile + ")";
+    return AggregationFunctionType.PERCENTILERAWKLLMV.getName().toLowerCase() + "(" + _expression + ", " + _percentile
+        + ")";
   }
 
   @Override
diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/InterSegmentAggregationMultiValueRawQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/InterSegmentAggregationMultiValueRawQueriesTest.java
index 65480aa95f..c9f0c444af 100644
--- a/pinot-core/src/test/java/org/apache/pinot/queries/InterSegmentAggregationMultiValueRawQueriesTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/queries/InterSegmentAggregationMultiValueRawQueriesTest.java
@@ -554,19 +554,17 @@ public class InterSegmentAggregationMultiValueRawQueriesTest extends BaseMultiVa
   }
 
   private void testPercentileRawKLLMV(int percentile) {
-    Function<Object, Object> quantileExtractor =
-        value -> {
-          try {
-            KllDoublesSketch sketch =
-                (KllDoublesSketch) ObjectSerDeUtils.KLL_SKETCH_SER_DE.deserialize(Base64.decode((String) value));
-            return sketch.getQuantile(percentile / 100.0);
-          } catch (IOException e) {
-            return null;
-          }
-        };
-
-    String rawKllQuery = String.format("SELECT PERCENTILERAWKLL%dMV(column6) AS value FROM testTable", percentile);
-    String regularQuery = String.format("SELECT PERCENTILE%dMV(column6) AS value FROM testTable", percentile);
+    Function<Object, Object> quantileExtractor = value -> {
+      try {
+        KllDoublesSketch sketch = ObjectSerDeUtils.KLL_SKETCH_SER_DE.deserialize(Base64.decode((String) value));
+        return sketch.getQuantile(percentile / 100.0);
+      } catch (IOException e) {
+        return null;
+      }
+    };
+
+    String rawKllQuery = String.format("SELECT PERCENTILERAWKLLMV(column6, %d) AS value FROM testTable", percentile);
+    String regularQuery = String.format("SELECT PERCENTILEMV(column6, %d) AS value FROM testTable", percentile);
     QueriesTestUtils.testInterSegmentsResult(getBrokerResponse(rawKllQuery), getBrokerResponse(regularQuery),
         quantileExtractor, PERCENTILE_KLL_DELTA);
     QueriesTestUtils.testInterSegmentsResult(getBrokerResponse(rawKllQuery + FILTER),
diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/PercentileKLLMVQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/PercentileKLLMVQueriesTest.java
index 40a8b9046d..9a0473ea5a 100644
--- a/pinot-core/src/test/java/org/apache/pinot/queries/PercentileKLLMVQueriesTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/queries/PercentileKLLMVQueriesTest.java
@@ -87,8 +87,8 @@ public class PercentileKLLMVQueriesTest extends PercentileKLLQueriesTest {
 
   @Override
   protected String getAggregationQuery(int percentile) {
-    return String.format("SELECT PERCENTILE%1$dMV(%2$s), PERCENTILEKLL%1$dMV(%2$s), PERCENTILEKLL%1$d(%3$s), "
-            + "PERCENTILEMV(%2$s, %1$d), PERCENTILEKLLMV(%2$s, %1$d), PERCENTILEKLL(%3$s, %1$d) FROM %4$s",
-        percentile, DOUBLE_COLUMN, KLL_COLUMN, TABLE_NAME);
+    return String.format(
+        "SELECT PERCENTILEMV(%2$s, %1$d), PERCENTILEKLLMV(%2$s, %1$d), PERCENTILEKLL(%3$s, %1$d) FROM %4$s", percentile,
+        DOUBLE_COLUMN, KLL_COLUMN, TABLE_NAME);
   }
 }
diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/PercentileKLLQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/PercentileKLLQueriesTest.java
index 05e626e4b4..2f5b1e67dc 100644
--- a/pinot-core/src/test/java/org/apache/pinot/queries/PercentileKLLQueriesTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/queries/PercentileKLLQueriesTest.java
@@ -161,17 +161,11 @@ public class PercentileKLLQueriesTest extends BaseQueriesTest {
     AggregationResultsBlock resultsBlock = aggregationOperator.nextBlock();
     List<Object> aggregationResult = resultsBlock.getResults();
     assertNotNull(aggregationResult);
-    assertEquals(aggregationResult.size(), 6);
+    assertEquals(aggregationResult.size(), 3);
     DoubleList doubleList0 = (DoubleList) aggregationResult.get(0);
     Collections.sort(doubleList0);
     assertSketch((KllDoublesSketch) aggregationResult.get(1), doubleList0);
     assertSketch((KllDoublesSketch) aggregationResult.get(2), doubleList0);
-
-    DoubleList doubleList3 = (DoubleList) aggregationResult.get(3);
-    Collections.sort(doubleList3);
-    assertEquals(doubleList3, doubleList0);
-    assertSketch((KllDoublesSketch) aggregationResult.get(4), doubleList0);
-    assertSketch((KllDoublesSketch) aggregationResult.get(5), doubleList0);
   }
 
   @Test
@@ -179,9 +173,9 @@ public class PercentileKLLQueriesTest extends BaseQueriesTest {
     for (int percentile = 0; percentile <= 100; percentile++) {
       BrokerResponseNative brokerResponse = getBrokerResponse(getAggregationQuery(percentile));
       Object[] results = brokerResponse.getResultTable().getRows().get(0);
-      assertEquals(results.length, 6);
+      assertEquals(results.length, 3);
       double expectedResult = (Double) results[0];
-      for (int i = 1; i < 6; i++) {
+      for (int i = 1; i < 3; i++) {
         assertEquals((Double) results[i], expectedResult, DELTA, ERROR_MESSAGE);
       }
     }
@@ -201,12 +195,6 @@ public class PercentileKLLQueriesTest extends BaseQueriesTest {
       Collections.sort(doubleList0);
       assertSketch((KllDoublesSketch) groupByResult.getResultForGroupId(1, groupId), doubleList0);
       assertSketch((KllDoublesSketch) groupByResult.getResultForGroupId(2, groupId), doubleList0);
-
-      DoubleList doubleList3 = (DoubleList) groupByResult.getResultForGroupId(3, groupId);
-      Collections.sort(doubleList3);
-      assertEquals(doubleList3, doubleList0);
-      assertSketch((KllDoublesSketch) groupByResult.getResultForGroupId(4, groupId), doubleList0);
-      assertSketch((KllDoublesSketch) groupByResult.getResultForGroupId(5, groupId), doubleList0);
     }
   }
 
@@ -217,9 +205,9 @@ public class PercentileKLLQueriesTest extends BaseQueriesTest {
       List<Object[]> rows = brokerResponse.getResultTable().getRows();
       assertEquals(rows.size(), 3);
       for (Object[] row : rows) {
-        assertEquals(row.length, 6);
+        assertEquals(row.length, 3);
         double expectedResult = (Double) row[0];
-        for (int i = 1; i < 6; i++) {
+        for (int i = 1; i < 3; i++) {
           assertEquals((Double) row[i], expectedResult, DELTA, ERROR_MESSAGE);
         }
       }
@@ -227,9 +215,9 @@ public class PercentileKLLQueriesTest extends BaseQueriesTest {
   }
 
   protected String getAggregationQuery(int percentile) {
-    return String.format("SELECT PERCENTILE%1$d(%2$s), PERCENTILEKLL%1$d(%2$s), PERCENTILEKLL%1$d(%3$s), "
-            + "PERCENTILE(%2$s, %1$d), PERCENTILEKLL(%2$s, %1$d), PERCENTILEKLL(%3$s, %1$d) FROM %4$s",
-        percentile, DOUBLE_COLUMN, KLL_COLUMN, TABLE_NAME);
+    return String.format(
+        "SELECT PERCENTILE(%2$s, %1$d), PERCENTILEKLL(%2$s, %1$d), PERCENTILEKLL(%3$s, %1$d) FROM %4$s", percentile,
+        DOUBLE_COLUMN, KLL_COLUMN, TABLE_NAME);
   }
 
   private String getGroupByQuery(int percentile) {


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