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 2022/12/05 21:15:02 UTC

[GitHub] [pinot] snleee opened a new pull request, #9910: [WIP] Add Variance and Standard Deviation Aggregation Functions

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

   This PR adds `VAR_POP()`, `VAR_SAMP()`, `STDDEV_POP()`, `STDDEV_SAMP()` aggregation functions.


-- 
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] snleee commented on a diff in pull request #9910: Add Variance and Standard Deviation Aggregation Functions

Posted by GitBox <gi...@apache.org>.
snleee commented on code in PR #9910:
URL: https://github.com/apache/pinot/pull/9910#discussion_r1041809590


##########
pinot-core/src/main/java/org/apache/pinot/core/common/ObjectSerDeUtils.java:
##########
@@ -123,7 +124,9 @@ public enum ObjectType {
     FloatLongPair(29),
     DoubleLongPair(30),
     StringLongPair(31),
-    CovarianceTuple(32);
+    CovarianceTuple(32),

Review Comment:
   If I reuse the covariance implementation, then it will become `naive algorithm` on https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Na%C3%AFve_algorithm
   
   According to the documentation, directly computing `Var = (SumSq − (Sum × Sum) / n) / (n − 1)` is not numerically stable. 
   
   The existing implementation is being used by Hive and Presto and they look to have separate implementation for covariance and variance.



-- 
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] snleee commented on a diff in pull request #9910: Add Variance and Standard Deviation Aggregation Functions

Posted by GitBox <gi...@apache.org>.
snleee commented on code in PR #9910:
URL: https://github.com/apache/pinot/pull/9910#discussion_r1041821415


##########
pinot-core/src/test/java/org/apache/pinot/queries/VarianceQueriesTest.java:
##########
@@ -0,0 +1,446 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.math3.stat.descriptive.moment.StandardDeviation;
+import org.apache.commons.math3.stat.descriptive.moment.Variance;
+import org.apache.commons.math3.util.Precision;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.core.operator.blocks.results.AggregationResultsBlock;
+import org.apache.pinot.core.operator.blocks.results.GroupByResultsBlock;
+import org.apache.pinot.core.operator.query.AggregationOperator;
+import org.apache.pinot.core.operator.query.GroupByOperator;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByResult;
+import org.apache.pinot.segment.local.customobject.VarianceTuple;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertTrue;
+
+
+public class VarianceQueriesTest extends BaseQueriesTest {
+
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "VarianceQueriesTest");
+
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+
+  private static final int NUM_RECORDS = 2000;
+  private static final int NUM_GROUPS = 10;
+  private static final int MAX_VALUE = 500;
+  private static final double RELATIVE_EPSILON = 0.0001;
+  private static final double DELTA = 0.0001;
+
+  private static final String INT_COLUMN = "intColumn";
+  private static final String LONG_COLUMN = "longColumn";
+  private static final String FLOAT_COLUMN = "floatColumn";
+  private static final String DOUBLE_COLUMN = "doubleColumn";
+  private static final String GROUP_BY_COLUMN = "groupByColumn";
+
+  private static final Schema SCHEMA =
+      new Schema.SchemaBuilder().addSingleValueDimension(INT_COLUMN, FieldSpec.DataType.INT)
+          .addSingleValueDimension(LONG_COLUMN, FieldSpec.DataType.LONG)
+          .addSingleValueDimension(FLOAT_COLUMN, FieldSpec.DataType.FLOAT)
+          .addSingleValueDimension(DOUBLE_COLUMN, FieldSpec.DataType.DOUBLE)
+          .addSingleValueDimension(GROUP_BY_COLUMN, FieldSpec.DataType.DOUBLE).build();
+
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  int[] _intValues = new int[NUM_RECORDS];
+  long[] _longValues = new long[NUM_RECORDS];
+  float[] _floatValues = new float[NUM_RECORDS];
+  double[] _doubleValues = new double[NUM_RECORDS];
+
+  @Override
+  protected String getFilter() {
+    // filter out half of the rows based on group id
+    return " WHERE groupByColumn < " + (NUM_GROUPS / 2);
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+    Random random = new Random();
+    List<GenericRow> records = new ArrayList<>(NUM_RECORDS);
+
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      GenericRow record = new GenericRow();
+      int intValue = -MAX_VALUE + random.nextInt() * 2 * MAX_VALUE;
+      long longValue = -MAX_VALUE + random.nextLong() * 2 * MAX_VALUE;
+      float floatValue = -MAX_VALUE + random.nextFloat() * 2 * MAX_VALUE;
+      double doubleValue = -MAX_VALUE + random.nextDouble() * 2 * MAX_VALUE;
+
+      _intValues[i] = intValue;
+      _longValues[i] = longValue;
+      _floatValues[i] = floatValue;
+      _doubleValues[i] = doubleValue;
+
+      record.putValue(INT_COLUMN, _intValues[i]);
+      record.putValue(LONG_COLUMN, _longValues[i]);
+      record.putValue(FLOAT_COLUMN, _floatValues[i]);
+      record.putValue(DOUBLE_COLUMN, _doubleValues[i]);
+      record.putValue(GROUP_BY_COLUMN, Math.floor(i / (NUM_RECORDS / NUM_GROUPS)));
+      records.add(record);
+    }
+
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(TABLE_CONFIG, SCHEMA);
+    segmentGeneratorConfig.setTableName(RAW_TABLE_NAME);
+    segmentGeneratorConfig.setSegmentName(SEGMENT_NAME);
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getPath());
+
+    SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig, new GenericRowRecordReader(records));
+    driver.build();
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, SEGMENT_NAME), ReadMode.mmap);
+    _indexSegment = immutableSegment;
+    _indexSegments = Arrays.asList(immutableSegment, immutableSegment);
+  }
+
+  @Test
+  public void testVarianceAggregationOnly() {
+    // Compute the expected values
+    Variance[] expectedVariances = new Variance[8];
+    for (int i = 0; i < 8; i++) {
+      if (i < 4) {
+        expectedVariances[i] = new Variance(false);
+      } else {
+        expectedVariances[i] = new Variance(true);
+      }
+    }
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      expectedVariances[0].increment(_intValues[i]);
+      expectedVariances[1].increment(_longValues[i]);
+      expectedVariances[2].increment(_floatValues[i]);
+      expectedVariances[3].increment(_doubleValues[i]);
+      expectedVariances[4].increment(_intValues[i]);
+      expectedVariances[5].increment(_longValues[i]);
+      expectedVariances[6].increment(_floatValues[i]);
+      expectedVariances[7].increment(_doubleValues[i]);
+    }
+    double expectedIntSum = Arrays.stream(_intValues).asDoubleStream().sum();
+    double expectedLongSum = Arrays.stream(_longValues).asDoubleStream().sum();
+    double expectedFloatSum = 0.0;
+    for (int i = 0; i < _floatValues.length; i++) {
+      expectedFloatSum += _floatValues[i];
+    }
+    double expectedDoubleSum = Arrays.stream(_doubleValues).sum();
+
+    // Compute the query
+    String query = "SELECT VAR_POP(intColumn), VAR_POP(longColumn), VAR_POP(floatColumn), VAR_POP(doubleColumn),"
+        + "VAR_SAMP(intColumn), VAR_SAMP(longColumn), VAR_SAMP(floatColumn), VAR_SAMP(doubleColumn) FROM testTable";
+    AggregationOperator aggregationOperator = getOperator(query);
+    AggregationResultsBlock resultsBlock = aggregationOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(aggregationOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 4, NUM_RECORDS);
+    List<Object> aggregationResult = resultsBlock.getResults();
+
+    // Validate the aggregation results
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(0), NUM_RECORDS, expectedIntSum,
+        expectedVariances[0].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(1), NUM_RECORDS, expectedLongSum,
+        expectedVariances[1].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(2), NUM_RECORDS, expectedFloatSum,
+        expectedVariances[2].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(3), NUM_RECORDS, expectedDoubleSum,
+        expectedVariances[3].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(4), NUM_RECORDS, expectedIntSum,
+        expectedVariances[4].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(5), NUM_RECORDS, expectedLongSum,
+        expectedVariances[5].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(6), NUM_RECORDS, expectedFloatSum,
+        expectedVariances[6].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(7), NUM_RECORDS, expectedDoubleSum,
+        expectedVariances[7].getResult(), true);
+
+    // Update the expected result by 3 more times (broker query will compute 4 identical segments)
+    for (int i = 0; i < NUM_RECORDS * 3; i++) {
+      int pos = i % NUM_RECORDS;
+      expectedVariances[0].increment(_intValues[pos]);
+      expectedVariances[1].increment(_longValues[pos]);
+      expectedVariances[2].increment(_floatValues[pos]);
+      expectedVariances[3].increment(_doubleValues[pos]);
+      expectedVariances[4].increment(_intValues[pos]);
+      expectedVariances[5].increment(_longValues[pos]);
+      expectedVariances[6].increment(_floatValues[pos]);
+      expectedVariances[7].increment(_doubleValues[pos]);
+    }
+
+    // Validate the response
+    BrokerResponseNative brokerResponse = getBrokerResponse(query);
+    brokerResponse.getResultTable();
+    Object[] results = brokerResponse.getResultTable().getRows().get(0);
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[0], expectedVariances[0].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[1], expectedVariances[1].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[2], expectedVariances[2].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[3], expectedVariances[3].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[4], expectedVariances[4].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[5], expectedVariances[5].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[6], expectedVariances[6].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[7], expectedVariances[7].getResult(), RELATIVE_EPSILON));
+
+    // Validate the response for a query with a filter
+    query = "SELECT VAR_POP(intColumn) from testTable" + getFilter();
+    brokerResponse = getBrokerResponse(query);
+    brokerResponse.getResultTable();
+    results = brokerResponse.getResultTable().getRows().get(0);
+    Variance filterExpectedVariance = new Variance(false);
+    for (int i = 0; i < NUM_RECORDS / 2; i++) {
+      filterExpectedVariance.increment(_intValues[i]);
+    }
+    assertTrue(Precision.equalsWithRelativeTolerance((double) results[0], filterExpectedVariance.getResult(),
+        RELATIVE_EPSILON));
+  }
+
+  @Test
+  public void testVarianceAggregationGroupBy() {
+    // Compute expected group results
+    Variance[] expectedGroupByResult = new Variance[NUM_GROUPS];
+    double[] expectedSum = new double[NUM_GROUPS];
+
+    for (int i = 0; i < NUM_GROUPS; i++) {
+      expectedGroupByResult[i] = new Variance(false);
+    }
+    for (int j = 0; j < NUM_RECORDS; j++) {
+      int pos = j / (NUM_RECORDS / NUM_GROUPS);
+      expectedGroupByResult[pos].increment(_intValues[j]);
+      expectedSum[pos] += _intValues[j];
+    }
+
+    String query = "SELECT VAR_POP(intColumn) FROM testTable GROUP BY groupByColumn ORDER BY groupByColumn";
+    GroupByOperator groupByOperator = getOperator(query);
+    GroupByResultsBlock resultsBlock = groupByOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(groupByOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 2, NUM_RECORDS);
+    AggregationGroupByResult aggregationGroupByResult = resultsBlock.getAggregationGroupByResult();
+    assertNotNull(aggregationGroupByResult);
+    for (int i = 0; i < NUM_GROUPS; i++) {
+
+      VarianceTuple actualVarianceTuple = (VarianceTuple) aggregationGroupByResult.getResultForGroupId(0, i);
+      checkWithPrecisionForVariance(actualVarianceTuple, NUM_RECORDS / NUM_GROUPS, expectedSum[i],
+          expectedGroupByResult[i].getResult(), false);
+    }
+  }
+
+  @Test
+  public void testStandardDeviationAggregationOnly() {
+    // Compute the expected values
+    StandardDeviation[] expectedStdDevs = new StandardDeviation[8];
+    for (int i = 0; i < 8; i++) {
+      if (i < 4) {
+        expectedStdDevs[i] = new StandardDeviation(false);
+      } else {
+        expectedStdDevs[i] = new StandardDeviation(true);
+      }
+    }
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      expectedStdDevs[0].increment(_intValues[i]);
+      expectedStdDevs[1].increment(_longValues[i]);
+      expectedStdDevs[2].increment(_floatValues[i]);
+      expectedStdDevs[3].increment(_doubleValues[i]);
+      expectedStdDevs[4].increment(_intValues[i]);
+      expectedStdDevs[5].increment(_longValues[i]);
+      expectedStdDevs[6].increment(_floatValues[i]);
+      expectedStdDevs[7].increment(_doubleValues[i]);
+    }
+
+    double expectedIntSum = Arrays.stream(_intValues).asDoubleStream().sum();
+    double expectedLongSum = Arrays.stream(_longValues).asDoubleStream().sum();
+    double expectedFloatSum = 0.0;
+    for (int i = 0; i < _floatValues.length; i++) {
+      expectedFloatSum += _floatValues[i];
+    }
+    double expectedDoubleSum = Arrays.stream(_doubleValues).sum();
+
+    // Compute the query
+    String query =
+        "SELECT STDDEV_POP(intColumn), STDDEV_POP(longColumn), STDDEV_POP(floatColumn), STDDEV_POP(doubleColumn),"
+            + "STDDEV_SAMP(intColumn), STDDEV_SAMP(longColumn), STDDEV_SAMP(floatColumn), STDDEV_SAMP(doubleColumn) "
+            + "FROM testTable";
+    AggregationOperator aggregationOperator = getOperator(query);
+    AggregationResultsBlock resultsBlock = aggregationOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(aggregationOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 4, NUM_RECORDS);
+    List<Object> aggregationResult = resultsBlock.getResults();
+
+    // Validate the aggregation results
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(0), NUM_RECORDS, expectedIntSum,
+        expectedStdDevs[0].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(1), NUM_RECORDS, expectedLongSum,
+        expectedStdDevs[1].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(2), NUM_RECORDS, expectedFloatSum,
+        expectedStdDevs[2].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(3), NUM_RECORDS, expectedDoubleSum,
+        expectedStdDevs[3].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(4), NUM_RECORDS, expectedIntSum,
+        expectedStdDevs[4].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(5), NUM_RECORDS, expectedLongSum,
+        expectedStdDevs[5].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(6), NUM_RECORDS, expectedFloatSum,
+        expectedStdDevs[6].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(7), NUM_RECORDS, expectedDoubleSum,
+        expectedStdDevs[7].getResult(), true);
+
+    // Update the expected result by 3 more times (broker query will compute 4 identical segments)

Review Comment:
   @jasperjiaguo Can you elaborate on your suggestion?
   
   I referred covariance test and it also uses `getBrokerResponse()` to get the result. According to the documentation, `getBrokerResponse()` seems to compute the result from 4 identical segments.  FYI,`getBrokerResponse()` internally calls `getDistinctInstances()`.
   



-- 
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] snleee commented on a diff in pull request #9910: Add Variance and Standard Deviation Aggregation Functions

Posted by GitBox <gi...@apache.org>.
snleee commented on code in PR #9910:
URL: https://github.com/apache/pinot/pull/9910#discussion_r1042826090


##########
pinot-core/src/test/java/org/apache/pinot/queries/VarianceQueriesTest.java:
##########
@@ -0,0 +1,446 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.math3.stat.descriptive.moment.StandardDeviation;
+import org.apache.commons.math3.stat.descriptive.moment.Variance;
+import org.apache.commons.math3.util.Precision;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.core.operator.blocks.results.AggregationResultsBlock;
+import org.apache.pinot.core.operator.blocks.results.GroupByResultsBlock;
+import org.apache.pinot.core.operator.query.AggregationOperator;
+import org.apache.pinot.core.operator.query.GroupByOperator;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByResult;
+import org.apache.pinot.segment.local.customobject.VarianceTuple;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertTrue;
+
+
+public class VarianceQueriesTest extends BaseQueriesTest {
+
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "VarianceQueriesTest");
+
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+
+  private static final int NUM_RECORDS = 2000;
+  private static final int NUM_GROUPS = 10;
+  private static final int MAX_VALUE = 500;
+  private static final double RELATIVE_EPSILON = 0.0001;
+  private static final double DELTA = 0.0001;
+
+  private static final String INT_COLUMN = "intColumn";
+  private static final String LONG_COLUMN = "longColumn";
+  private static final String FLOAT_COLUMN = "floatColumn";
+  private static final String DOUBLE_COLUMN = "doubleColumn";
+  private static final String GROUP_BY_COLUMN = "groupByColumn";
+
+  private static final Schema SCHEMA =
+      new Schema.SchemaBuilder().addSingleValueDimension(INT_COLUMN, FieldSpec.DataType.INT)
+          .addSingleValueDimension(LONG_COLUMN, FieldSpec.DataType.LONG)
+          .addSingleValueDimension(FLOAT_COLUMN, FieldSpec.DataType.FLOAT)
+          .addSingleValueDimension(DOUBLE_COLUMN, FieldSpec.DataType.DOUBLE)
+          .addSingleValueDimension(GROUP_BY_COLUMN, FieldSpec.DataType.DOUBLE).build();
+
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  int[] _intValues = new int[NUM_RECORDS];
+  long[] _longValues = new long[NUM_RECORDS];
+  float[] _floatValues = new float[NUM_RECORDS];
+  double[] _doubleValues = new double[NUM_RECORDS];
+
+  @Override
+  protected String getFilter() {
+    // filter out half of the rows based on group id
+    return " WHERE groupByColumn < " + (NUM_GROUPS / 2);
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+    Random random = new Random();
+    List<GenericRow> records = new ArrayList<>(NUM_RECORDS);
+
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      GenericRow record = new GenericRow();
+      int intValue = -MAX_VALUE + random.nextInt() * 2 * MAX_VALUE;
+      long longValue = -MAX_VALUE + random.nextLong() * 2 * MAX_VALUE;
+      float floatValue = -MAX_VALUE + random.nextFloat() * 2 * MAX_VALUE;
+      double doubleValue = -MAX_VALUE + random.nextDouble() * 2 * MAX_VALUE;
+
+      _intValues[i] = intValue;
+      _longValues[i] = longValue;
+      _floatValues[i] = floatValue;
+      _doubleValues[i] = doubleValue;
+
+      record.putValue(INT_COLUMN, _intValues[i]);
+      record.putValue(LONG_COLUMN, _longValues[i]);
+      record.putValue(FLOAT_COLUMN, _floatValues[i]);
+      record.putValue(DOUBLE_COLUMN, _doubleValues[i]);
+      record.putValue(GROUP_BY_COLUMN, Math.floor(i / (NUM_RECORDS / NUM_GROUPS)));
+      records.add(record);
+    }
+
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(TABLE_CONFIG, SCHEMA);
+    segmentGeneratorConfig.setTableName(RAW_TABLE_NAME);
+    segmentGeneratorConfig.setSegmentName(SEGMENT_NAME);
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getPath());
+
+    SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig, new GenericRowRecordReader(records));
+    driver.build();
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, SEGMENT_NAME), ReadMode.mmap);
+    _indexSegment = immutableSegment;
+    _indexSegments = Arrays.asList(immutableSegment, immutableSegment);
+  }
+
+  @Test
+  public void testVarianceAggregationOnly() {
+    // Compute the expected values
+    Variance[] expectedVariances = new Variance[8];
+    for (int i = 0; i < 8; i++) {
+      if (i < 4) {
+        expectedVariances[i] = new Variance(false);
+      } else {
+        expectedVariances[i] = new Variance(true);
+      }
+    }
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      expectedVariances[0].increment(_intValues[i]);
+      expectedVariances[1].increment(_longValues[i]);
+      expectedVariances[2].increment(_floatValues[i]);
+      expectedVariances[3].increment(_doubleValues[i]);
+      expectedVariances[4].increment(_intValues[i]);
+      expectedVariances[5].increment(_longValues[i]);
+      expectedVariances[6].increment(_floatValues[i]);
+      expectedVariances[7].increment(_doubleValues[i]);
+    }
+    double expectedIntSum = Arrays.stream(_intValues).asDoubleStream().sum();
+    double expectedLongSum = Arrays.stream(_longValues).asDoubleStream().sum();
+    double expectedFloatSum = 0.0;
+    for (int i = 0; i < _floatValues.length; i++) {
+      expectedFloatSum += _floatValues[i];
+    }
+    double expectedDoubleSum = Arrays.stream(_doubleValues).sum();
+
+    // Compute the query
+    String query = "SELECT VAR_POP(intColumn), VAR_POP(longColumn), VAR_POP(floatColumn), VAR_POP(doubleColumn),"
+        + "VAR_SAMP(intColumn), VAR_SAMP(longColumn), VAR_SAMP(floatColumn), VAR_SAMP(doubleColumn) FROM testTable";
+    AggregationOperator aggregationOperator = getOperator(query);
+    AggregationResultsBlock resultsBlock = aggregationOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(aggregationOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 4, NUM_RECORDS);
+    List<Object> aggregationResult = resultsBlock.getResults();
+
+    // Validate the aggregation results
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(0), NUM_RECORDS, expectedIntSum,
+        expectedVariances[0].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(1), NUM_RECORDS, expectedLongSum,
+        expectedVariances[1].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(2), NUM_RECORDS, expectedFloatSum,
+        expectedVariances[2].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(3), NUM_RECORDS, expectedDoubleSum,
+        expectedVariances[3].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(4), NUM_RECORDS, expectedIntSum,
+        expectedVariances[4].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(5), NUM_RECORDS, expectedLongSum,
+        expectedVariances[5].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(6), NUM_RECORDS, expectedFloatSum,
+        expectedVariances[6].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(7), NUM_RECORDS, expectedDoubleSum,
+        expectedVariances[7].getResult(), true);
+
+    // Update the expected result by 3 more times (broker query will compute 4 identical segments)
+    for (int i = 0; i < NUM_RECORDS * 3; i++) {
+      int pos = i % NUM_RECORDS;
+      expectedVariances[0].increment(_intValues[pos]);
+      expectedVariances[1].increment(_longValues[pos]);
+      expectedVariances[2].increment(_floatValues[pos]);
+      expectedVariances[3].increment(_doubleValues[pos]);
+      expectedVariances[4].increment(_intValues[pos]);
+      expectedVariances[5].increment(_longValues[pos]);
+      expectedVariances[6].increment(_floatValues[pos]);
+      expectedVariances[7].increment(_doubleValues[pos]);
+    }
+
+    // Validate the response
+    BrokerResponseNative brokerResponse = getBrokerResponse(query);
+    brokerResponse.getResultTable();
+    Object[] results = brokerResponse.getResultTable().getRows().get(0);
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[0], expectedVariances[0].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[1], expectedVariances[1].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[2], expectedVariances[2].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[3], expectedVariances[3].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[4], expectedVariances[4].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[5], expectedVariances[5].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[6], expectedVariances[6].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[7], expectedVariances[7].getResult(), RELATIVE_EPSILON));
+
+    // Validate the response for a query with a filter
+    query = "SELECT VAR_POP(intColumn) from testTable" + getFilter();
+    brokerResponse = getBrokerResponse(query);
+    brokerResponse.getResultTable();
+    results = brokerResponse.getResultTable().getRows().get(0);
+    Variance filterExpectedVariance = new Variance(false);
+    for (int i = 0; i < NUM_RECORDS / 2; i++) {
+      filterExpectedVariance.increment(_intValues[i]);
+    }
+    assertTrue(Precision.equalsWithRelativeTolerance((double) results[0], filterExpectedVariance.getResult(),
+        RELATIVE_EPSILON));
+  }
+
+  @Test
+  public void testVarianceAggregationGroupBy() {
+    // Compute expected group results
+    Variance[] expectedGroupByResult = new Variance[NUM_GROUPS];
+    double[] expectedSum = new double[NUM_GROUPS];
+
+    for (int i = 0; i < NUM_GROUPS; i++) {
+      expectedGroupByResult[i] = new Variance(false);
+    }
+    for (int j = 0; j < NUM_RECORDS; j++) {
+      int pos = j / (NUM_RECORDS / NUM_GROUPS);
+      expectedGroupByResult[pos].increment(_intValues[j]);
+      expectedSum[pos] += _intValues[j];
+    }
+
+    String query = "SELECT VAR_POP(intColumn) FROM testTable GROUP BY groupByColumn ORDER BY groupByColumn";
+    GroupByOperator groupByOperator = getOperator(query);
+    GroupByResultsBlock resultsBlock = groupByOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(groupByOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 2, NUM_RECORDS);
+    AggregationGroupByResult aggregationGroupByResult = resultsBlock.getAggregationGroupByResult();
+    assertNotNull(aggregationGroupByResult);
+    for (int i = 0; i < NUM_GROUPS; i++) {
+
+      VarianceTuple actualVarianceTuple = (VarianceTuple) aggregationGroupByResult.getResultForGroupId(0, i);
+      checkWithPrecisionForVariance(actualVarianceTuple, NUM_RECORDS / NUM_GROUPS, expectedSum[i],
+          expectedGroupByResult[i].getResult(), false);
+    }
+  }
+
+  @Test
+  public void testStandardDeviationAggregationOnly() {
+    // Compute the expected values
+    StandardDeviation[] expectedStdDevs = new StandardDeviation[8];
+    for (int i = 0; i < 8; i++) {
+      if (i < 4) {
+        expectedStdDevs[i] = new StandardDeviation(false);
+      } else {
+        expectedStdDevs[i] = new StandardDeviation(true);
+      }
+    }
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      expectedStdDevs[0].increment(_intValues[i]);
+      expectedStdDevs[1].increment(_longValues[i]);
+      expectedStdDevs[2].increment(_floatValues[i]);
+      expectedStdDevs[3].increment(_doubleValues[i]);
+      expectedStdDevs[4].increment(_intValues[i]);
+      expectedStdDevs[5].increment(_longValues[i]);
+      expectedStdDevs[6].increment(_floatValues[i]);
+      expectedStdDevs[7].increment(_doubleValues[i]);
+    }
+
+    double expectedIntSum = Arrays.stream(_intValues).asDoubleStream().sum();
+    double expectedLongSum = Arrays.stream(_longValues).asDoubleStream().sum();
+    double expectedFloatSum = 0.0;
+    for (int i = 0; i < _floatValues.length; i++) {
+      expectedFloatSum += _floatValues[i];
+    }
+    double expectedDoubleSum = Arrays.stream(_doubleValues).sum();
+
+    // Compute the query
+    String query =
+        "SELECT STDDEV_POP(intColumn), STDDEV_POP(longColumn), STDDEV_POP(floatColumn), STDDEV_POP(doubleColumn),"
+            + "STDDEV_SAMP(intColumn), STDDEV_SAMP(longColumn), STDDEV_SAMP(floatColumn), STDDEV_SAMP(doubleColumn) "
+            + "FROM testTable";
+    AggregationOperator aggregationOperator = getOperator(query);
+    AggregationResultsBlock resultsBlock = aggregationOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(aggregationOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 4, NUM_RECORDS);
+    List<Object> aggregationResult = resultsBlock.getResults();
+
+    // Validate the aggregation results
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(0), NUM_RECORDS, expectedIntSum,
+        expectedStdDevs[0].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(1), NUM_RECORDS, expectedLongSum,
+        expectedStdDevs[1].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(2), NUM_RECORDS, expectedFloatSum,
+        expectedStdDevs[2].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(3), NUM_RECORDS, expectedDoubleSum,
+        expectedStdDevs[3].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(4), NUM_RECORDS, expectedIntSum,
+        expectedStdDevs[4].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(5), NUM_RECORDS, expectedLongSum,
+        expectedStdDevs[5].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(6), NUM_RECORDS, expectedFloatSum,
+        expectedStdDevs[6].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(7), NUM_RECORDS, expectedDoubleSum,
+        expectedStdDevs[7].getResult(), true);
+
+    // Update the expected result by 3 more times (broker query will compute 4 identical segments)

Review Comment:
   @jasperjiaguo Thanks for pointing out. I actually merged `Covariance & Variance tests`. This actually helped me to identify a bug when we merging empty `VarianceTuple` with non empty `VarianceTuple`! 
   
   Can you go over the pr once more?



-- 
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] jasperjiaguo commented on a diff in pull request #9910: Add Variance and Standard Deviation Aggregation Functions

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on code in PR #9910:
URL: https://github.com/apache/pinot/pull/9910#discussion_r1042533813


##########
pinot-core/src/main/java/org/apache/pinot/core/common/ObjectSerDeUtils.java:
##########
@@ -123,7 +124,9 @@ public enum ObjectType {
     FloatLongPair(29),
     DoubleLongPair(30),
     StringLongPair(31),
-    CovarianceTuple(32);
+    CovarianceTuple(32),

Review Comment:
   we can indeed change the covar algo to numerical stable version: similar to 
    https://github.com/trinodb/trino/blob/1866a23e3b0377144c1820de892c0de2762351a8/core/trino-main/src/main/java/io/trino/operator/aggregation/state/CovarianceState.java



##########
pinot-core/src/main/java/org/apache/pinot/core/common/ObjectSerDeUtils.java:
##########
@@ -123,7 +124,9 @@ public enum ObjectType {
     FloatLongPair(29),
     DoubleLongPair(30),
     StringLongPair(31),
-    CovarianceTuple(32);
+    CovarianceTuple(32),

Review Comment:
   we can indeed change the covar algo to numerical stable version, similar to 
    https://github.com/trinodb/trino/blob/1866a23e3b0377144c1820de892c0de2762351a8/core/trino-main/src/main/java/io/trino/operator/aggregation/state/CovarianceState.java



-- 
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] snleee commented on a diff in pull request #9910: Add Variance and Standard Deviation Aggregation Functions

Posted by GitBox <gi...@apache.org>.
snleee commented on code in PR #9910:
URL: https://github.com/apache/pinot/pull/9910#discussion_r1041827118


##########
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/VarianceAggregationFunction.java:
##########
@@ -0,0 +1,204 @@
+/**
+ * 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.Map;
+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.VarianceTuple;
+import org.apache.pinot.segment.spi.AggregationFunctionType;
+
+
+/**
+ * Aggregation function which computes Variance and Standard Deviation
+ *
+ * The algorithm to compute variance is based on "Updating Formulae and a Pairwise Algorithm for Computing
+ * Sample Variances" by Chan et al. Please refer to the "Parallel Algorithm" section from the following wiki:
+ * - https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Parallel_algorithm
+ */
+public class VarianceAggregationFunction extends BaseSingleInputAggregationFunction<VarianceTuple, Double> {

Review Comment:
   I took out a common function that can be shared and created a util class. Please take a look



-- 
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] walterddr commented on a diff in pull request #9910: Add Variance and Standard Deviation Aggregation Functions

Posted by GitBox <gi...@apache.org>.
walterddr commented on code in PR #9910:
URL: https://github.com/apache/pinot/pull/9910#discussion_r1043941051


##########
pinot-core/src/test/java/org/apache/pinot/queries/StatisticalQueriesTest.java:
##########
@@ -0,0 +1,749 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.math3.stat.correlation.Covariance;
+import org.apache.commons.math3.stat.descriptive.moment.StandardDeviation;
+import org.apache.commons.math3.stat.descriptive.moment.Variance;
+import org.apache.commons.math3.util.Precision;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.core.operator.blocks.results.AggregationResultsBlock;
+import org.apache.pinot.core.operator.blocks.results.GroupByResultsBlock;
+import org.apache.pinot.core.operator.query.AggregationOperator;
+import org.apache.pinot.core.operator.query.GroupByOperator;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByResult;
+import org.apache.pinot.segment.local.customobject.CovarianceTuple;
+import org.apache.pinot.segment.local.customobject.VarianceTuple;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * Queries test for statistical queries (i.e Variance, Covariance, Standard Deviation etc)
+ */
+public class StatisticalQueriesTest extends BaseQueriesTest {
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "CovarianceQueriesTest");
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+
+  // test segments 1-4 evenly divide testSegment into 4 distinct segments
+  private static final String SEGMENT_NAME_1 = "testSegment1";
+  private static final String SEGMENT_NAME_2 = "testSegment2";
+  private static final String SEGMENT_NAME_3 = "testSegment3";
+  private static final String SEGMENT_NAME_4 = "testSegment4";
+
+  private static final int NUM_RECORDS = 2000;
+  private static final int NUM_GROUPS = 10;
+  private static final int MAX_VALUE = 500;
+  private static final double RELATIVE_EPSILON = 0.0001;
+  private static final double DELTA = 0.0001;
+
+  private static final String INT_COLUMN_X = "intColumnX";
+  private static final String INT_COLUMN_Y = "intColumnY";
+  private static final String DOUBLE_COLUMN_X = "doubleColumnX";
+  private static final String DOUBLE_COLUMN_Y = "doubleColumnY";
+  private static final String LONG_COLUMN = "longColumn";
+  private static final String FLOAT_COLUMN = "floatColumn";
+  private static final String GROUP_BY_COLUMN = "groupByColumn";
+
+  private static final Schema SCHEMA =
+      new Schema.SchemaBuilder().addSingleValueDimension(INT_COLUMN_X, FieldSpec.DataType.INT)
+          .addSingleValueDimension(INT_COLUMN_Y, FieldSpec.DataType.INT)
+          .addSingleValueDimension(DOUBLE_COLUMN_X, FieldSpec.DataType.DOUBLE)
+          .addSingleValueDimension(DOUBLE_COLUMN_Y, FieldSpec.DataType.DOUBLE)
+          .addSingleValueDimension(LONG_COLUMN, FieldSpec.DataType.LONG)
+          .addSingleValueDimension(FLOAT_COLUMN, FieldSpec.DataType.FLOAT)
+          .addSingleValueDimension(GROUP_BY_COLUMN, FieldSpec.DataType.DOUBLE).build();
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+  private List<List<IndexSegment>> _distinctInstances;
+  private int _sumIntX = 0;
+  private int _sumIntY = 0;
+  private int _sumIntXY = 0;
+
+  private double _sumDoubleX = 0;
+  private double _sumDoubleY = 0;
+  private double _sumDoubleXY = 0;
+
+  private long _sumLong = 0L;
+  private double _sumFloat = 0;
+
+  private double _sumIntDouble = 0;
+  private long _sumIntLong = 0L;
+  private double _sumIntFloat = 0;
+  private double _sumDoubleLong = 0;
+  private double _sumDoubleFloat = 0;
+  private double _sumLongFloat = 0;
+
+  private double _expectedCovIntXY;
+  private double _expectedCovDoubleXY;
+  private double _expectedCovIntDouble;
+  private double _expectedCovIntLong;
+  private double _expectedCovIntFloat;
+  private double _expectedCovDoubleLong;
+  private double _expectedCovDoubleFloat;
+  private double _expectedCovLongFloat;
+
+  private double _expectedCovWithFilter;
+
+  private final CovarianceTuple[] _expectedGroupByResultVer1 = new CovarianceTuple[NUM_GROUPS];
+  private final CovarianceTuple[] _expectedGroupByResultVer2 = new CovarianceTuple[NUM_GROUPS];
+  private final double[] _expectedFinalResultVer1 = new double[NUM_GROUPS];
+  private final double[] _expectedFinalResultVer2 = new double[NUM_GROUPS];
+
+  private boolean _useIdenticalSegment = false;
+
+  int[] _intColX = new int[NUM_RECORDS];
+  int[] _intColY = new int[NUM_RECORDS];
+  long[] _longCol = new long[NUM_RECORDS];
+  double[] _floatCol = new double[NUM_RECORDS];
+  double[] _doubleColX = new double[NUM_RECORDS];
+  double[] _doubleColY = new double[NUM_RECORDS];
+  double[] _groupByCol = new double[NUM_RECORDS];
+
+  @Override
+  protected String getFilter() {
+    // filter out half of the rows based on group id
+    return " WHERE groupByColumn < " + (NUM_GROUPS / 2);
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @Override
+  protected List<List<IndexSegment>> getDistinctInstances() {
+    if (_useIdenticalSegment) {
+      return Collections.singletonList(_indexSegments);
+    }
+    return _distinctInstances;
+  }
+
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+
+    List<GenericRow> records = new ArrayList<>(NUM_RECORDS);
+
+    Random rand = new Random();
+    _intColX = rand.ints(NUM_RECORDS, -MAX_VALUE, MAX_VALUE).toArray();
+    _intColY = rand.ints(NUM_RECORDS, -MAX_VALUE, MAX_VALUE).toArray();
+    _doubleColX = rand.doubles(NUM_RECORDS, -MAX_VALUE, MAX_VALUE).toArray();
+    _doubleColY = rand.doubles(NUM_RECORDS, -MAX_VALUE, MAX_VALUE).toArray();
+    _longCol = rand.longs(NUM_RECORDS, -MAX_VALUE, MAX_VALUE).toArray();
+
+    int groupSize = NUM_RECORDS / NUM_GROUPS;
+    double sumX = 0;
+    double sumY = 0;
+    double sumGroupBy = 0;
+    double sumXY = 0;
+    double sumXGroupBy = 0;
+    int groupByVal = 0;
+
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      GenericRow record = new GenericRow();
+      int intX = _intColX[i];
+      int intY = _intColY[i];
+      double doubleX = _doubleColX[i];
+      double doubleY = _doubleColY[i];
+      long longVal = _longCol[i];
+      float floatVal = -MAX_VALUE + rand.nextFloat() * 2 * MAX_VALUE;
+
+      // set up inner segment group by results
+      groupByVal = (int) Math.floor(i / groupSize);
+      if (i % groupSize == 0 && groupByVal > 0) {
+        _expectedGroupByResultVer1[groupByVal - 1] = new CovarianceTuple(sumX, sumGroupBy, sumXGroupBy, groupSize);
+        _expectedGroupByResultVer2[groupByVal - 1] = new CovarianceTuple(sumX, sumY, sumXY, groupSize);
+        sumX = 0;
+        sumY = 0;
+        sumGroupBy = 0;
+        sumXY = 0;
+        sumXGroupBy = 0;
+      }
+
+      sumX += doubleX;
+      sumY += doubleY;
+      sumGroupBy += groupByVal;
+      sumXY += doubleX * doubleY;
+      sumXGroupBy += doubleX * groupByVal;
+
+      _floatCol[i] = floatVal;
+      _groupByCol[i] = groupByVal;
+
+      // calculate inner segment results
+      _sumIntX += intX;
+      _sumIntY += intY;
+      _sumDoubleX += doubleX;
+      _sumDoubleY += doubleY;
+      _sumLong += longVal;
+      _sumFloat += floatVal;
+      _sumIntXY += intX * intY;
+      _sumDoubleXY += doubleX * doubleY;
+      _sumIntDouble += intX * doubleX;
+      _sumIntLong += intX * longVal;
+      _sumIntFloat += intX * _floatCol[i];
+      _sumDoubleLong += doubleX * longVal;
+      _sumDoubleFloat += doubleX * _floatCol[i];
+      _sumLongFloat += longVal * _floatCol[i];
+
+      record.putValue(INT_COLUMN_X, intX);
+      record.putValue(INT_COLUMN_Y, intY);
+      record.putValue(DOUBLE_COLUMN_X, doubleX);
+      record.putValue(DOUBLE_COLUMN_Y, doubleY);
+      record.putValue(LONG_COLUMN, longVal);
+      record.putValue(FLOAT_COLUMN, floatVal);
+      record.putValue(GROUP_BY_COLUMN, groupByVal);
+      records.add(record);
+    }
+    _expectedGroupByResultVer1[groupByVal] = new CovarianceTuple(sumX, sumGroupBy, sumXGroupBy, groupSize);
+    _expectedGroupByResultVer2[groupByVal] = new CovarianceTuple(sumX, sumY, sumXY, groupSize);
+
+    // calculate inter segment result
+    Covariance cov = new Covariance();
+    double[] newIntColX = Arrays.stream(_intColX).asDoubleStream().toArray();
+    double[] newIntColY = Arrays.stream(_intColY).asDoubleStream().toArray();
+    double[] newLongCol = Arrays.stream(_longCol).asDoubleStream().toArray();
+    _expectedCovIntXY = cov.covariance(newIntColX, newIntColY, false);
+    _expectedCovDoubleXY = cov.covariance(_doubleColX, _doubleColY, false);
+    _expectedCovIntDouble = cov.covariance(newIntColX, _doubleColX, false);
+    _expectedCovIntLong = cov.covariance(newIntColX, newLongCol, false);
+    _expectedCovIntFloat = cov.covariance(newIntColX, _floatCol, false);
+    _expectedCovDoubleLong = cov.covariance(_doubleColX, newLongCol, false);
+    _expectedCovDoubleFloat = cov.covariance(_doubleColX, _floatCol, false);
+    _expectedCovLongFloat = cov.covariance(newLongCol, _floatCol, false);
+
+    double[] filteredX = Arrays.copyOfRange(_doubleColX, 0, NUM_RECORDS / 2);
+    double[] filteredY = Arrays.copyOfRange(_doubleColY, 0, NUM_RECORDS / 2);
+    _expectedCovWithFilter = cov.covariance(filteredX, filteredY, false);
+
+    // calculate inter segment group by results
+    for (int i = 0; i < NUM_GROUPS; i++) {
+      double[] colX = Arrays.copyOfRange(_doubleColX, i * groupSize, (i + 1) * groupSize);
+      double[] colGroupBy = Arrays.copyOfRange(_groupByCol, i * groupSize, (i + 1) * groupSize);
+      double[] colY = Arrays.copyOfRange(_doubleColY, i * groupSize, (i + 1) * groupSize);
+      _expectedFinalResultVer1[i] = cov.covariance(colX, colGroupBy, false);
+      _expectedFinalResultVer2[i] = cov.covariance(colX, colY, false);
+    }
+
+    // generate testSegment
+    ImmutableSegment immutableSegment = setUpSingleSegment(records, SEGMENT_NAME);
+    _indexSegment = immutableSegment;
+    _indexSegments = Arrays.asList(immutableSegment, immutableSegment);
+
+    // divide testSegment into 4 distinct segments for distinct inter segment tests
+    // by doing so, we can avoid calculating global covariance again
+    _distinctInstances = new ArrayList<>();
+    int segmentSize = NUM_RECORDS / 4;
+    ImmutableSegment immutableSegment1 = setUpSingleSegment(records.subList(0, segmentSize), SEGMENT_NAME_1);
+    ImmutableSegment immutableSegment2 =
+        setUpSingleSegment(records.subList(segmentSize, segmentSize * 2), SEGMENT_NAME_2);
+    ImmutableSegment immutableSegment3 =
+        setUpSingleSegment(records.subList(segmentSize * 2, segmentSize * 3), SEGMENT_NAME_3);
+    ImmutableSegment immutableSegment4 =
+        setUpSingleSegment(records.subList(segmentSize * 3, NUM_RECORDS), SEGMENT_NAME_4);
+    // generate 2 instances each with 2 distinct segments
+    _distinctInstances.add(Arrays.asList(immutableSegment1, immutableSegment2));
+    _distinctInstances.add(Arrays.asList(immutableSegment3, immutableSegment4));
+  }
+
+  private ImmutableSegment setUpSingleSegment(List<GenericRow> recordSet, String segmentName)
+      throws Exception {
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(TABLE_CONFIG, SCHEMA);
+    segmentGeneratorConfig.setTableName(RAW_TABLE_NAME);
+    segmentGeneratorConfig.setSegmentName(segmentName);
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getPath());
+
+    SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig, new GenericRowRecordReader(recordSet));
+    driver.build();
+
+    return ImmutableSegmentLoader.load(new File(INDEX_DIR, segmentName), ReadMode.mmap);
+  }
+
+  @Test
+  public void testCovarianceAggregationOnly() {
+    // Inner Segment
+    String query =
+        "SELECT COVAR_POP(intColumnX, intColumnY), COVAR_POP(doubleColumnX, doubleColumnY), COVAR_POP(intColumnX, "
+            + "doubleColumnX), " + "COVAR_POP(intColumnX, longColumn), COVAR_POP(intColumnX, floatColumn), "
+            + "COVAR_POP(doubleColumnX, longColumn), COVAR_POP(doubleColumnX, floatColumn), COVAR_POP(longColumn, "
+            + "floatColumn)  FROM testTable";
+    AggregationOperator aggregationOperator = getOperator(query);
+    AggregationResultsBlock resultsBlock = aggregationOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(aggregationOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 6, NUM_RECORDS);
+    List<Object> aggregationResult = resultsBlock.getResults();
+    assertNotNull(aggregationResult);
+    checkWithPrecisionForCovariance((CovarianceTuple) aggregationResult.get(0), _sumIntX, _sumIntY, _sumIntXY,
+        NUM_RECORDS);
+    checkWithPrecisionForCovariance((CovarianceTuple) aggregationResult.get(1), _sumDoubleX, _sumDoubleY, _sumDoubleXY,
+        NUM_RECORDS);
+    checkWithPrecisionForCovariance((CovarianceTuple) aggregationResult.get(2), _sumIntX, _sumDoubleX, _sumIntDouble,
+        NUM_RECORDS);
+    checkWithPrecisionForCovariance((CovarianceTuple) aggregationResult.get(3), _sumIntX, _sumLong, _sumIntLong,
+        NUM_RECORDS);
+    checkWithPrecisionForCovariance((CovarianceTuple) aggregationResult.get(4), _sumIntX, _sumFloat, _sumIntFloat,
+        NUM_RECORDS);
+    checkWithPrecisionForCovariance((CovarianceTuple) aggregationResult.get(5), _sumDoubleX, _sumLong, _sumDoubleLong,
+        NUM_RECORDS);
+    checkWithPrecisionForCovariance((CovarianceTuple) aggregationResult.get(6), _sumDoubleX, _sumFloat, _sumDoubleFloat,
+        NUM_RECORDS);
+    checkWithPrecisionForCovariance((CovarianceTuple) aggregationResult.get(7), _sumLong, _sumFloat, _sumLongFloat,
+        NUM_RECORDS);
+
+    // Inter segments with 4 identical segments (2 instances each having 2 identical segments)
+    _useIdenticalSegment = true;
+    BrokerResponseNative brokerResponse = getBrokerResponse(query);
+    _useIdenticalSegment = false;
+    assertEquals(brokerResponse.getNumDocsScanned(), 4 * NUM_RECORDS);
+    assertEquals(brokerResponse.getNumEntriesScannedInFilter(), 0);
+    assertEquals(brokerResponse.getNumEntriesScannedPostFilter(), 4 * 6 * NUM_RECORDS);
+    assertEquals(brokerResponse.getTotalDocs(), 4 * NUM_RECORDS);
+    checkResultTableWithPrecisionForCovariance(brokerResponse);
+
+    // Inter segments with 4 distinct segments (2 instances each having 2 distinct segments)
+    brokerResponse = getBrokerResponse(query);
+    assertEquals(brokerResponse.getNumDocsScanned(), NUM_RECORDS);
+    assertEquals(brokerResponse.getNumEntriesScannedInFilter(), 0);
+    assertEquals(brokerResponse.getNumEntriesScannedPostFilter(), 6 * NUM_RECORDS);
+    assertEquals(brokerResponse.getTotalDocs(), NUM_RECORDS);
+    checkResultTableWithPrecisionForCovariance(brokerResponse);
+
+    // Inter segments with 4 identical segments with filter
+    _useIdenticalSegment = true;
+    query = "SELECT COVAR_POP(doubleColumnX, doubleColumnY) FROM testTable" + getFilter();
+    brokerResponse = getBrokerResponse(query);
+    _useIdenticalSegment = false;
+    assertEquals(brokerResponse.getNumDocsScanned(), 2 * NUM_RECORDS);
+    assertEquals(brokerResponse.getNumEntriesScannedInFilter(), 0);
+    assertEquals(brokerResponse.getNumEntriesScannedPostFilter(), 4 * NUM_RECORDS);
+    assertEquals(brokerResponse.getTotalDocs(), 4 * NUM_RECORDS);
+    Object[] results = brokerResponse.getResultTable().getRows().get(0);
+    assertTrue(Precision.equalsWithRelativeTolerance((double) results[0], _expectedCovWithFilter, RELATIVE_EPSILON));
+  }
+
+  @Test
+  public void testCovarianceAggregationGroupBy() {
+    // Inner Segment
+    // case 1: (col1, groupByCol) group by groupByCol => all covariances are 0's
+    String query =
+        "SELECT COVAR_POP(doubleColumnX, groupByColumn) FROM testTable GROUP BY groupByColumn ORDER BY groupByColumn";
+    GroupByOperator groupByOperator = getOperator(query);
+    GroupByResultsBlock resultsBlock = groupByOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(groupByOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 2, NUM_RECORDS);
+    AggregationGroupByResult aggregationGroupByResult = resultsBlock.getAggregationGroupByResult();
+    assertNotNull(aggregationGroupByResult);
+    for (int i = 0; i < NUM_GROUPS; i++) {
+      CovarianceTuple actualCovTuple = (CovarianceTuple) aggregationGroupByResult.getResultForGroupId(0, i);
+      CovarianceTuple expectedCovTuple = _expectedGroupByResultVer1[i];
+      checkWithPrecisionForCovariance(actualCovTuple, expectedCovTuple);
+    }
+
+    // Inter Segment with 4 identical segments
+    _useIdenticalSegment = true;
+    BrokerResponseNative brokerResponse = getBrokerResponse(query);
+    checkGroupByResultsForCovariance(brokerResponse, _expectedFinalResultVer1);
+    _useIdenticalSegment = false;
+    // Inter Segment with 4 distinct segments
+    brokerResponse = getBrokerResponse(query);
+    checkGroupByResultsForCovariance(brokerResponse, _expectedFinalResultVer1);
+
+    // Inner Segment
+    // case 2: COVAR_POP(col1, col2) group by groupByCol => nondeterministic cov
+    query =
+        "SELECT COVAR_POP(doubleColumnX, doubleColumnY) FROM testTable GROUP BY groupByColumn ORDER BY groupByColumn";
+    groupByOperator = getOperator(query);
+    resultsBlock = groupByOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(groupByOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 3, NUM_RECORDS);
+    aggregationGroupByResult = resultsBlock.getAggregationGroupByResult();
+    assertNotNull(aggregationGroupByResult);
+
+    for (int i = 0; i < NUM_GROUPS; i++) {
+      CovarianceTuple actualCovTuple = (CovarianceTuple) aggregationGroupByResult.getResultForGroupId(0, i);
+      CovarianceTuple expectedCovTuple = _expectedGroupByResultVer2[i];
+      checkWithPrecisionForCovariance(actualCovTuple, expectedCovTuple);
+    }
+
+    // Inter Segment with 4 identical segments
+    _useIdenticalSegment = true;
+    brokerResponse = getBrokerResponse(query);
+    checkGroupByResultsForCovariance(brokerResponse, _expectedFinalResultVer2);
+    _useIdenticalSegment = false;
+    // Inter Segment with 4 distinct segments
+    brokerResponse = getBrokerResponse(query);
+    checkGroupByResultsForCovariance(brokerResponse, _expectedFinalResultVer2);
+  }
+
+  @Test
+  public void testVarianceAggregationOnly() {
+    // Compute the expected values
+    Variance[] expectedVariances = new Variance[8];
+    for (int i = 0; i < 8; i++) {
+      if (i < 4) {
+        expectedVariances[i] = new Variance(false);
+      } else {
+        expectedVariances[i] = new Variance(true);
+      }
+    }
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      expectedVariances[0].increment(_intColX[i]);
+      expectedVariances[1].increment(_longCol[i]);
+      expectedVariances[2].increment(_floatCol[i]);
+      expectedVariances[3].increment(_doubleColX[i]);
+      expectedVariances[4].increment(_intColX[i]);
+      expectedVariances[5].increment(_longCol[i]);
+      expectedVariances[6].increment(_floatCol[i]);
+      expectedVariances[7].increment(_doubleColX[i]);
+    }
+    double expectedIntSum = Arrays.stream(_intColX).asDoubleStream().sum();
+    double expectedLongSum = Arrays.stream(_longCol).asDoubleStream().sum();
+    double expectedFloatSum = 0.0;
+    for (int i = 0; i < _floatCol.length; i++) {
+      expectedFloatSum += _floatCol[i];
+    }
+    double expectedDoubleSum = Arrays.stream(_doubleColX).sum();
+
+    // Compute the query
+    String query = "SELECT VAR_POP(intColumnX), VAR_POP(longColumn), VAR_POP(floatColumn), VAR_POP(doubleColumnX),"
+        + "VAR_SAMP(intColumnX), VAR_SAMP(longColumn), VAR_SAMP(floatColumn), VAR_SAMP(doubleColumnX) FROM testTable";
+    AggregationOperator aggregationOperator = getOperator(query);
+    AggregationResultsBlock resultsBlock = aggregationOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(aggregationOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 4, NUM_RECORDS);
+    List<Object> aggregationResult = resultsBlock.getResults();
+
+    // Validate the aggregation results
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(0), NUM_RECORDS, expectedIntSum,
+        expectedVariances[0].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(1), NUM_RECORDS, expectedLongSum,
+        expectedVariances[1].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(2), NUM_RECORDS, expectedFloatSum,
+        expectedVariances[2].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(3), NUM_RECORDS, expectedDoubleSum,
+        expectedVariances[3].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(4), NUM_RECORDS, expectedIntSum,
+        expectedVariances[4].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(5), NUM_RECORDS, expectedLongSum,
+        expectedVariances[5].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(6), NUM_RECORDS, expectedFloatSum,
+        expectedVariances[6].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(7), NUM_RECORDS, expectedDoubleSum,
+        expectedVariances[7].getResult(), true);
+
+    // Validate the response
+    BrokerResponseNative brokerResponse = getBrokerResponse(query);
+    Object[] results = brokerResponse.getResultTable().getRows().get(0);
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[0], expectedVariances[0].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[1], expectedVariances[1].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[2], expectedVariances[2].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[3], expectedVariances[3].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[4], expectedVariances[4].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[5], expectedVariances[5].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[6], expectedVariances[6].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[7], expectedVariances[7].getResult(), RELATIVE_EPSILON));
+
+    VarianceTuple test = ((VarianceTuple) aggregationResult.get(0));
+    test.apply((new VarianceTuple(0, 0, 0.0d)));
+    System.out.println(test.getM2());
+    // Validate the response for a query with a filter
+    query = "SELECT VAR_POP(intColumnX) from testTable" + getFilter();
+    brokerResponse = getBrokerResponse(query);
+    brokerResponse.getResultTable();
+    results = brokerResponse.getResultTable().getRows().get(0);
+    Variance filterExpectedVariance = new Variance(false);
+    for (int i = 0; i < NUM_RECORDS / 2; i++) {
+      filterExpectedVariance.increment(_intColX[i]);
+    }
+    assertTrue(Precision.equalsWithRelativeTolerance((double) results[0], filterExpectedVariance.getResult(),
+        RELATIVE_EPSILON));
+  }
+
+  @Test
+  public void testVarianceAggregationGroupBy() {
+    // Compute expected group results
+    Variance[] expectedGroupByResult = new Variance[NUM_GROUPS];
+    double[] expectedSum = new double[NUM_GROUPS];
+
+    for (int i = 0; i < NUM_GROUPS; i++) {
+      expectedGroupByResult[i] = new Variance(false);
+    }
+    for (int j = 0; j < NUM_RECORDS; j++) {
+      int pos = j / (NUM_RECORDS / NUM_GROUPS);
+      expectedGroupByResult[pos].increment(_intColX[j]);
+      expectedSum[pos] += _intColX[j];
+    }
+
+    String query = "SELECT VAR_POP(intColumnX) FROM testTable GROUP BY groupByColumn ORDER BY groupByColumn";
+    GroupByOperator groupByOperator = getOperator(query);
+    GroupByResultsBlock resultsBlock = groupByOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(groupByOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 2, NUM_RECORDS);
+    AggregationGroupByResult aggregationGroupByResult = resultsBlock.getAggregationGroupByResult();
+    assertNotNull(aggregationGroupByResult);
+    for (int i = 0; i < NUM_GROUPS; i++) {
+
+      VarianceTuple actualVarianceTuple = (VarianceTuple) aggregationGroupByResult.getResultForGroupId(0, i);
+      checkWithPrecisionForVariance(actualVarianceTuple, NUM_RECORDS / NUM_GROUPS, expectedSum[i],
+          expectedGroupByResult[i].getResult(), false);
+    }
+  }
+
+  @Test
+  public void testStandardDeviationAggregationOnly() {

Review Comment:
   saw some instability on https://github.com/apache/pinot/actions/runs/3652424011/jobs/6170799903. 



-- 
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 #9910: Add Variance and Standard Deviation Aggregation Functions

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on PR #9910:
URL: https://github.com/apache/pinot/pull/9910#issuecomment-1338981611

   # [Codecov](https://codecov.io/gh/apache/pinot/pull/9910?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#9910](https://codecov.io/gh/apache/pinot/pull/9910?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (de8dcc4) into [master](https://codecov.io/gh/apache/pinot/commit/ecf41be2ecd007853c2db19e1c6a038cf356cb9e?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (ecf41be) will **decrease** coverage by `48.50%`.
   > The diff coverage is `0.00%`.
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #9910       +/-   ##
   =============================================
   - Coverage     64.30%   15.79%   -48.51%     
   + Complexity     5034      175     -4859     
   =============================================
     Files          1928     1930        +2     
     Lines        103874   104001      +127     
     Branches      15823    15846       +23     
   =============================================
   - Hits          66793    16426    -50367     
   - Misses        32254    86373    +54119     
   + Partials       4827     1202     -3625     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | unittests1 | `?` | |
   | unittests2 | `15.79% <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=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/9910?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../pinot/common/function/scalar/StringFunctions.java](https://codecov.io/gh/apache/pinot/pull/9910/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vc2NhbGFyL1N0cmluZ0Z1bmN0aW9ucy5qYXZh) | `0.00% <0.00%> (-56.25%)` | :arrow_down: |
   | [...org/apache/pinot/core/common/ObjectSerDeUtils.java](https://codecov.io/gh/apache/pinot/pull/9910/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9jb21tb24vT2JqZWN0U2VyRGVVdGlscy5qYXZh) | `0.00% <0.00%> (-90.73%)` | :arrow_down: |
   | [...gregation/function/AggregationFunctionFactory.java](https://codecov.io/gh/apache/pinot/pull/9910/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9hZ2dyZWdhdGlvbi9mdW5jdGlvbi9BZ2dyZWdhdGlvbkZ1bmN0aW9uRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-81.03%)` | :arrow_down: |
   | [...regation/function/VarianceAggregationFunction.java](https://codecov.io/gh/apache/pinot/pull/9910/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9hZ2dyZWdhdGlvbi9mdW5jdGlvbi9WYXJpYW5jZUFnZ3JlZ2F0aW9uRnVuY3Rpb24uamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...inot/segment/local/customobject/VarianceTuple.java](https://codecov.io/gh/apache/pinot/pull/9910/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9jdXN0b21vYmplY3QvVmFyaWFuY2VUdXBsZS5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...che/pinot/segment/spi/AggregationFunctionType.java](https://codecov.io/gh/apache/pinot/pull/9910/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1zcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3NlZ21lbnQvc3BpL0FnZ3JlZ2F0aW9uRnVuY3Rpb25UeXBlLmphdmE=) | `0.00% <0.00%> (-90.00%)` | :arrow_down: |
   | [...src/main/java/org/apache/pinot/sql/FilterKind.java](https://codecov.io/gh/apache/pinot/pull/9910/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcWwvRmlsdGVyS2luZC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ain/java/org/apache/pinot/core/data/table/Key.java](https://codecov.io/gh/apache/pinot/pull/9910/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL0tleS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...in/java/org/apache/pinot/spi/utils/BytesUtils.java](https://codecov.io/gh/apache/pinot/pull/9910/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQnl0ZXNVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...n/java/org/apache/pinot/core/data/table/Table.java](https://codecov.io/gh/apache/pinot/pull/9910/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL1RhYmxlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [1227 more](https://codecov.io/gh/apache/pinot/pull/9910/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   :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=The+Apache+Software+Foundation)
   


-- 
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] jasperjiaguo commented on a diff in pull request #9910: Add Variance and Standard Deviation Aggregation Functions

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on code in PR #9910:
URL: https://github.com/apache/pinot/pull/9910#discussion_r1042545235


##########
pinot-core/src/test/java/org/apache/pinot/queries/VarianceQueriesTest.java:
##########
@@ -0,0 +1,446 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.math3.stat.descriptive.moment.StandardDeviation;
+import org.apache.commons.math3.stat.descriptive.moment.Variance;
+import org.apache.commons.math3.util.Precision;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.core.operator.blocks.results.AggregationResultsBlock;
+import org.apache.pinot.core.operator.blocks.results.GroupByResultsBlock;
+import org.apache.pinot.core.operator.query.AggregationOperator;
+import org.apache.pinot.core.operator.query.GroupByOperator;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByResult;
+import org.apache.pinot.segment.local.customobject.VarianceTuple;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertTrue;
+
+
+public class VarianceQueriesTest extends BaseQueriesTest {
+
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "VarianceQueriesTest");
+
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+
+  private static final int NUM_RECORDS = 2000;
+  private static final int NUM_GROUPS = 10;
+  private static final int MAX_VALUE = 500;
+  private static final double RELATIVE_EPSILON = 0.0001;
+  private static final double DELTA = 0.0001;
+
+  private static final String INT_COLUMN = "intColumn";
+  private static final String LONG_COLUMN = "longColumn";
+  private static final String FLOAT_COLUMN = "floatColumn";
+  private static final String DOUBLE_COLUMN = "doubleColumn";
+  private static final String GROUP_BY_COLUMN = "groupByColumn";
+
+  private static final Schema SCHEMA =
+      new Schema.SchemaBuilder().addSingleValueDimension(INT_COLUMN, FieldSpec.DataType.INT)
+          .addSingleValueDimension(LONG_COLUMN, FieldSpec.DataType.LONG)
+          .addSingleValueDimension(FLOAT_COLUMN, FieldSpec.DataType.FLOAT)
+          .addSingleValueDimension(DOUBLE_COLUMN, FieldSpec.DataType.DOUBLE)
+          .addSingleValueDimension(GROUP_BY_COLUMN, FieldSpec.DataType.DOUBLE).build();
+
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  int[] _intValues = new int[NUM_RECORDS];
+  long[] _longValues = new long[NUM_RECORDS];
+  float[] _floatValues = new float[NUM_RECORDS];
+  double[] _doubleValues = new double[NUM_RECORDS];
+
+  @Override
+  protected String getFilter() {
+    // filter out half of the rows based on group id
+    return " WHERE groupByColumn < " + (NUM_GROUPS / 2);
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+    Random random = new Random();
+    List<GenericRow> records = new ArrayList<>(NUM_RECORDS);
+
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      GenericRow record = new GenericRow();
+      int intValue = -MAX_VALUE + random.nextInt() * 2 * MAX_VALUE;
+      long longValue = -MAX_VALUE + random.nextLong() * 2 * MAX_VALUE;
+      float floatValue = -MAX_VALUE + random.nextFloat() * 2 * MAX_VALUE;
+      double doubleValue = -MAX_VALUE + random.nextDouble() * 2 * MAX_VALUE;
+
+      _intValues[i] = intValue;
+      _longValues[i] = longValue;
+      _floatValues[i] = floatValue;
+      _doubleValues[i] = doubleValue;
+
+      record.putValue(INT_COLUMN, _intValues[i]);
+      record.putValue(LONG_COLUMN, _longValues[i]);
+      record.putValue(FLOAT_COLUMN, _floatValues[i]);
+      record.putValue(DOUBLE_COLUMN, _doubleValues[i]);
+      record.putValue(GROUP_BY_COLUMN, Math.floor(i / (NUM_RECORDS / NUM_GROUPS)));
+      records.add(record);
+    }
+
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(TABLE_CONFIG, SCHEMA);
+    segmentGeneratorConfig.setTableName(RAW_TABLE_NAME);
+    segmentGeneratorConfig.setSegmentName(SEGMENT_NAME);
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getPath());
+
+    SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig, new GenericRowRecordReader(records));
+    driver.build();
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, SEGMENT_NAME), ReadMode.mmap);
+    _indexSegment = immutableSegment;
+    _indexSegments = Arrays.asList(immutableSegment, immutableSegment);
+  }
+
+  @Test
+  public void testVarianceAggregationOnly() {
+    // Compute the expected values
+    Variance[] expectedVariances = new Variance[8];
+    for (int i = 0; i < 8; i++) {
+      if (i < 4) {
+        expectedVariances[i] = new Variance(false);
+      } else {
+        expectedVariances[i] = new Variance(true);
+      }
+    }
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      expectedVariances[0].increment(_intValues[i]);
+      expectedVariances[1].increment(_longValues[i]);
+      expectedVariances[2].increment(_floatValues[i]);
+      expectedVariances[3].increment(_doubleValues[i]);
+      expectedVariances[4].increment(_intValues[i]);
+      expectedVariances[5].increment(_longValues[i]);
+      expectedVariances[6].increment(_floatValues[i]);
+      expectedVariances[7].increment(_doubleValues[i]);
+    }
+    double expectedIntSum = Arrays.stream(_intValues).asDoubleStream().sum();
+    double expectedLongSum = Arrays.stream(_longValues).asDoubleStream().sum();
+    double expectedFloatSum = 0.0;
+    for (int i = 0; i < _floatValues.length; i++) {
+      expectedFloatSum += _floatValues[i];
+    }
+    double expectedDoubleSum = Arrays.stream(_doubleValues).sum();
+
+    // Compute the query
+    String query = "SELECT VAR_POP(intColumn), VAR_POP(longColumn), VAR_POP(floatColumn), VAR_POP(doubleColumn),"
+        + "VAR_SAMP(intColumn), VAR_SAMP(longColumn), VAR_SAMP(floatColumn), VAR_SAMP(doubleColumn) FROM testTable";
+    AggregationOperator aggregationOperator = getOperator(query);
+    AggregationResultsBlock resultsBlock = aggregationOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(aggregationOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 4, NUM_RECORDS);
+    List<Object> aggregationResult = resultsBlock.getResults();
+
+    // Validate the aggregation results
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(0), NUM_RECORDS, expectedIntSum,
+        expectedVariances[0].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(1), NUM_RECORDS, expectedLongSum,
+        expectedVariances[1].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(2), NUM_RECORDS, expectedFloatSum,
+        expectedVariances[2].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(3), NUM_RECORDS, expectedDoubleSum,
+        expectedVariances[3].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(4), NUM_RECORDS, expectedIntSum,
+        expectedVariances[4].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(5), NUM_RECORDS, expectedLongSum,
+        expectedVariances[5].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(6), NUM_RECORDS, expectedFloatSum,
+        expectedVariances[6].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(7), NUM_RECORDS, expectedDoubleSum,
+        expectedVariances[7].getResult(), true);
+
+    // Update the expected result by 3 more times (broker query will compute 4 identical segments)
+    for (int i = 0; i < NUM_RECORDS * 3; i++) {
+      int pos = i % NUM_RECORDS;
+      expectedVariances[0].increment(_intValues[pos]);
+      expectedVariances[1].increment(_longValues[pos]);
+      expectedVariances[2].increment(_floatValues[pos]);
+      expectedVariances[3].increment(_doubleValues[pos]);
+      expectedVariances[4].increment(_intValues[pos]);
+      expectedVariances[5].increment(_longValues[pos]);
+      expectedVariances[6].increment(_floatValues[pos]);
+      expectedVariances[7].increment(_doubleValues[pos]);
+    }
+
+    // Validate the response
+    BrokerResponseNative brokerResponse = getBrokerResponse(query);
+    brokerResponse.getResultTable();
+    Object[] results = brokerResponse.getResultTable().getRows().get(0);
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[0], expectedVariances[0].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[1], expectedVariances[1].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[2], expectedVariances[2].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[3], expectedVariances[3].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[4], expectedVariances[4].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[5], expectedVariances[5].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[6], expectedVariances[6].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[7], expectedVariances[7].getResult(), RELATIVE_EPSILON));
+
+    // Validate the response for a query with a filter
+    query = "SELECT VAR_POP(intColumn) from testTable" + getFilter();
+    brokerResponse = getBrokerResponse(query);
+    brokerResponse.getResultTable();
+    results = brokerResponse.getResultTable().getRows().get(0);
+    Variance filterExpectedVariance = new Variance(false);
+    for (int i = 0; i < NUM_RECORDS / 2; i++) {
+      filterExpectedVariance.increment(_intValues[i]);
+    }
+    assertTrue(Precision.equalsWithRelativeTolerance((double) results[0], filterExpectedVariance.getResult(),
+        RELATIVE_EPSILON));
+  }
+
+  @Test
+  public void testVarianceAggregationGroupBy() {
+    // Compute expected group results
+    Variance[] expectedGroupByResult = new Variance[NUM_GROUPS];
+    double[] expectedSum = new double[NUM_GROUPS];
+
+    for (int i = 0; i < NUM_GROUPS; i++) {
+      expectedGroupByResult[i] = new Variance(false);
+    }
+    for (int j = 0; j < NUM_RECORDS; j++) {
+      int pos = j / (NUM_RECORDS / NUM_GROUPS);
+      expectedGroupByResult[pos].increment(_intValues[j]);
+      expectedSum[pos] += _intValues[j];
+    }
+
+    String query = "SELECT VAR_POP(intColumn) FROM testTable GROUP BY groupByColumn ORDER BY groupByColumn";
+    GroupByOperator groupByOperator = getOperator(query);
+    GroupByResultsBlock resultsBlock = groupByOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(groupByOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 2, NUM_RECORDS);
+    AggregationGroupByResult aggregationGroupByResult = resultsBlock.getAggregationGroupByResult();
+    assertNotNull(aggregationGroupByResult);
+    for (int i = 0; i < NUM_GROUPS; i++) {
+
+      VarianceTuple actualVarianceTuple = (VarianceTuple) aggregationGroupByResult.getResultForGroupId(0, i);
+      checkWithPrecisionForVariance(actualVarianceTuple, NUM_RECORDS / NUM_GROUPS, expectedSum[i],
+          expectedGroupByResult[i].getResult(), false);
+    }
+  }
+
+  @Test
+  public void testStandardDeviationAggregationOnly() {
+    // Compute the expected values
+    StandardDeviation[] expectedStdDevs = new StandardDeviation[8];
+    for (int i = 0; i < 8; i++) {
+      if (i < 4) {
+        expectedStdDevs[i] = new StandardDeviation(false);
+      } else {
+        expectedStdDevs[i] = new StandardDeviation(true);
+      }
+    }
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      expectedStdDevs[0].increment(_intValues[i]);
+      expectedStdDevs[1].increment(_longValues[i]);
+      expectedStdDevs[2].increment(_floatValues[i]);
+      expectedStdDevs[3].increment(_doubleValues[i]);
+      expectedStdDevs[4].increment(_intValues[i]);
+      expectedStdDevs[5].increment(_longValues[i]);
+      expectedStdDevs[6].increment(_floatValues[i]);
+      expectedStdDevs[7].increment(_doubleValues[i]);
+    }
+
+    double expectedIntSum = Arrays.stream(_intValues).asDoubleStream().sum();
+    double expectedLongSum = Arrays.stream(_longValues).asDoubleStream().sum();
+    double expectedFloatSum = 0.0;
+    for (int i = 0; i < _floatValues.length; i++) {
+      expectedFloatSum += _floatValues[i];
+    }
+    double expectedDoubleSum = Arrays.stream(_doubleValues).sum();
+
+    // Compute the query
+    String query =
+        "SELECT STDDEV_POP(intColumn), STDDEV_POP(longColumn), STDDEV_POP(floatColumn), STDDEV_POP(doubleColumn),"
+            + "STDDEV_SAMP(intColumn), STDDEV_SAMP(longColumn), STDDEV_SAMP(floatColumn), STDDEV_SAMP(doubleColumn) "
+            + "FROM testTable";
+    AggregationOperator aggregationOperator = getOperator(query);
+    AggregationResultsBlock resultsBlock = aggregationOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(aggregationOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 4, NUM_RECORDS);
+    List<Object> aggregationResult = resultsBlock.getResults();
+
+    // Validate the aggregation results
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(0), NUM_RECORDS, expectedIntSum,
+        expectedStdDevs[0].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(1), NUM_RECORDS, expectedLongSum,
+        expectedStdDevs[1].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(2), NUM_RECORDS, expectedFloatSum,
+        expectedStdDevs[2].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(3), NUM_RECORDS, expectedDoubleSum,
+        expectedStdDevs[3].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(4), NUM_RECORDS, expectedIntSum,
+        expectedStdDevs[4].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(5), NUM_RECORDS, expectedLongSum,
+        expectedStdDevs[5].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(6), NUM_RECORDS, expectedFloatSum,
+        expectedStdDevs[6].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(7), NUM_RECORDS, expectedDoubleSum,
+        expectedStdDevs[7].getResult(), true);
+
+    // Update the expected result by 3 more times (broker query will compute 4 identical segments)

Review Comment:
   Yes, in `CovarianceQueriesTest`, @SabrinaZhaozyf has added the [getDistinctInstances()](https://github.com/apache/pinot/blob/270315a13f457376df04cd47bc62f84d1c1e558e/pinot-core/src/test/java/org/apache/pinot/queries/CovarianceQueriesTest.java#L157) to use 4 distinct segments for these statistical functions because Covar on 4 identical segments is the same as Covar on one of each segment, as kind of defeats the purpose of testing merge logic over multiple segments. The `getDistinctInstances` in base class is still returning 4 identical segments for backward compat. We can probably inherit `CovarianceQueriesTest` in `VarianceQueriesTest`.



-- 
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] snleee commented on a diff in pull request #9910: Add Variance and Standard Deviation Aggregation Functions

Posted by GitBox <gi...@apache.org>.
snleee commented on code in PR #9910:
URL: https://github.com/apache/pinot/pull/9910#discussion_r1041815451


##########
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/VarianceAggregationFunction.java:
##########
@@ -0,0 +1,204 @@
+/**
+ * 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.Map;
+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.VarianceTuple;
+import org.apache.pinot.segment.spi.AggregationFunctionType;
+
+
+/**
+ * Aggregation function which computes Variance and Standard Deviation
+ *
+ * The algorithm to compute variance is based on "Updating Formulae and a Pairwise Algorithm for Computing
+ * Sample Variances" by Chan et al. Please refer to the "Parallel Algorithm" section from the following wiki:
+ * - https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Parallel_algorithm
+ */
+public class VarianceAggregationFunction extends BaseSingleInputAggregationFunction<VarianceTuple, Double> {

Review Comment:
   Interestingly, they look very similar but the implementations of all functions are slightly different due to the algorithm difference. Also, we use a different intermediate class `VarianceTuple` instead of `CovarianceTuple`.



-- 
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] snleee commented on a diff in pull request #9910: Add Variance and Standard Deviation Aggregation Functions

Posted by GitBox <gi...@apache.org>.
snleee commented on code in PR #9910:
URL: https://github.com/apache/pinot/pull/9910#discussion_r1041816310


##########
pinot-core/src/main/java/org/apache/pinot/core/common/ObjectSerDeUtils.java:
##########
@@ -123,7 +124,9 @@ public enum ObjectType {
     FloatLongPair(29),
     DoubleLongPair(30),
     StringLongPair(31),
-    CovarianceTuple(32);
+    CovarianceTuple(32),
+

Review Comment:
   fixed



-- 
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] walterddr commented on a diff in pull request #9910: Add Variance and Standard Deviation Aggregation Functions

Posted by GitBox <gi...@apache.org>.
walterddr commented on code in PR #9910:
URL: https://github.com/apache/pinot/pull/9910#discussion_r1042370739


##########
pinot-core/src/main/java/org/apache/pinot/core/common/ObjectSerDeUtils.java:
##########
@@ -123,7 +124,9 @@ public enum ObjectType {
     FloatLongPair(29),
     DoubleLongPair(30),
     StringLongPair(31),
-    CovarianceTuple(32);
+    CovarianceTuple(32),

Review Comment:
   sweet! thanks for the clarification. this also means there's numeric stability for covariance. probably we should also document that. (or change the cover algo). 



-- 
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] jasperjiaguo commented on a diff in pull request #9910: Add Variance and Standard Deviation Aggregation Functions

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on code in PR #9910:
URL: https://github.com/apache/pinot/pull/9910#discussion_r1042533813


##########
pinot-core/src/main/java/org/apache/pinot/core/common/ObjectSerDeUtils.java:
##########
@@ -123,7 +124,9 @@ public enum ObjectType {
     FloatLongPair(29),
     DoubleLongPair(30),
     StringLongPair(31),
-    CovarianceTuple(32);
+    CovarianceTuple(32),

Review Comment:
   we can indeed change the covar algo to numerical stable version, similar to 
    https://github.com/trinodb/trino/blob/1866a23e3b0377144c1820de892c0de2762351a8/core/trino-main/src/main/java/io/trino/operator/aggregation/state/CovarianceState.java
   
   created issue: https://github.com/apache/pinot/issues/9936



-- 
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] jasperjiaguo commented on a diff in pull request #9910: Add Variance and Standard Deviation Aggregation Functions

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on code in PR #9910:
URL: https://github.com/apache/pinot/pull/9910#discussion_r1042545235


##########
pinot-core/src/test/java/org/apache/pinot/queries/VarianceQueriesTest.java:
##########
@@ -0,0 +1,446 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.math3.stat.descriptive.moment.StandardDeviation;
+import org.apache.commons.math3.stat.descriptive.moment.Variance;
+import org.apache.commons.math3.util.Precision;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.core.operator.blocks.results.AggregationResultsBlock;
+import org.apache.pinot.core.operator.blocks.results.GroupByResultsBlock;
+import org.apache.pinot.core.operator.query.AggregationOperator;
+import org.apache.pinot.core.operator.query.GroupByOperator;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByResult;
+import org.apache.pinot.segment.local.customobject.VarianceTuple;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertTrue;
+
+
+public class VarianceQueriesTest extends BaseQueriesTest {
+
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "VarianceQueriesTest");
+
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+
+  private static final int NUM_RECORDS = 2000;
+  private static final int NUM_GROUPS = 10;
+  private static final int MAX_VALUE = 500;
+  private static final double RELATIVE_EPSILON = 0.0001;
+  private static final double DELTA = 0.0001;
+
+  private static final String INT_COLUMN = "intColumn";
+  private static final String LONG_COLUMN = "longColumn";
+  private static final String FLOAT_COLUMN = "floatColumn";
+  private static final String DOUBLE_COLUMN = "doubleColumn";
+  private static final String GROUP_BY_COLUMN = "groupByColumn";
+
+  private static final Schema SCHEMA =
+      new Schema.SchemaBuilder().addSingleValueDimension(INT_COLUMN, FieldSpec.DataType.INT)
+          .addSingleValueDimension(LONG_COLUMN, FieldSpec.DataType.LONG)
+          .addSingleValueDimension(FLOAT_COLUMN, FieldSpec.DataType.FLOAT)
+          .addSingleValueDimension(DOUBLE_COLUMN, FieldSpec.DataType.DOUBLE)
+          .addSingleValueDimension(GROUP_BY_COLUMN, FieldSpec.DataType.DOUBLE).build();
+
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  int[] _intValues = new int[NUM_RECORDS];
+  long[] _longValues = new long[NUM_RECORDS];
+  float[] _floatValues = new float[NUM_RECORDS];
+  double[] _doubleValues = new double[NUM_RECORDS];
+
+  @Override
+  protected String getFilter() {
+    // filter out half of the rows based on group id
+    return " WHERE groupByColumn < " + (NUM_GROUPS / 2);
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+    Random random = new Random();
+    List<GenericRow> records = new ArrayList<>(NUM_RECORDS);
+
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      GenericRow record = new GenericRow();
+      int intValue = -MAX_VALUE + random.nextInt() * 2 * MAX_VALUE;
+      long longValue = -MAX_VALUE + random.nextLong() * 2 * MAX_VALUE;
+      float floatValue = -MAX_VALUE + random.nextFloat() * 2 * MAX_VALUE;
+      double doubleValue = -MAX_VALUE + random.nextDouble() * 2 * MAX_VALUE;
+
+      _intValues[i] = intValue;
+      _longValues[i] = longValue;
+      _floatValues[i] = floatValue;
+      _doubleValues[i] = doubleValue;
+
+      record.putValue(INT_COLUMN, _intValues[i]);
+      record.putValue(LONG_COLUMN, _longValues[i]);
+      record.putValue(FLOAT_COLUMN, _floatValues[i]);
+      record.putValue(DOUBLE_COLUMN, _doubleValues[i]);
+      record.putValue(GROUP_BY_COLUMN, Math.floor(i / (NUM_RECORDS / NUM_GROUPS)));
+      records.add(record);
+    }
+
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(TABLE_CONFIG, SCHEMA);
+    segmentGeneratorConfig.setTableName(RAW_TABLE_NAME);
+    segmentGeneratorConfig.setSegmentName(SEGMENT_NAME);
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getPath());
+
+    SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig, new GenericRowRecordReader(records));
+    driver.build();
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, SEGMENT_NAME), ReadMode.mmap);
+    _indexSegment = immutableSegment;
+    _indexSegments = Arrays.asList(immutableSegment, immutableSegment);
+  }
+
+  @Test
+  public void testVarianceAggregationOnly() {
+    // Compute the expected values
+    Variance[] expectedVariances = new Variance[8];
+    for (int i = 0; i < 8; i++) {
+      if (i < 4) {
+        expectedVariances[i] = new Variance(false);
+      } else {
+        expectedVariances[i] = new Variance(true);
+      }
+    }
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      expectedVariances[0].increment(_intValues[i]);
+      expectedVariances[1].increment(_longValues[i]);
+      expectedVariances[2].increment(_floatValues[i]);
+      expectedVariances[3].increment(_doubleValues[i]);
+      expectedVariances[4].increment(_intValues[i]);
+      expectedVariances[5].increment(_longValues[i]);
+      expectedVariances[6].increment(_floatValues[i]);
+      expectedVariances[7].increment(_doubleValues[i]);
+    }
+    double expectedIntSum = Arrays.stream(_intValues).asDoubleStream().sum();
+    double expectedLongSum = Arrays.stream(_longValues).asDoubleStream().sum();
+    double expectedFloatSum = 0.0;
+    for (int i = 0; i < _floatValues.length; i++) {
+      expectedFloatSum += _floatValues[i];
+    }
+    double expectedDoubleSum = Arrays.stream(_doubleValues).sum();
+
+    // Compute the query
+    String query = "SELECT VAR_POP(intColumn), VAR_POP(longColumn), VAR_POP(floatColumn), VAR_POP(doubleColumn),"
+        + "VAR_SAMP(intColumn), VAR_SAMP(longColumn), VAR_SAMP(floatColumn), VAR_SAMP(doubleColumn) FROM testTable";
+    AggregationOperator aggregationOperator = getOperator(query);
+    AggregationResultsBlock resultsBlock = aggregationOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(aggregationOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 4, NUM_RECORDS);
+    List<Object> aggregationResult = resultsBlock.getResults();
+
+    // Validate the aggregation results
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(0), NUM_RECORDS, expectedIntSum,
+        expectedVariances[0].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(1), NUM_RECORDS, expectedLongSum,
+        expectedVariances[1].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(2), NUM_RECORDS, expectedFloatSum,
+        expectedVariances[2].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(3), NUM_RECORDS, expectedDoubleSum,
+        expectedVariances[3].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(4), NUM_RECORDS, expectedIntSum,
+        expectedVariances[4].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(5), NUM_RECORDS, expectedLongSum,
+        expectedVariances[5].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(6), NUM_RECORDS, expectedFloatSum,
+        expectedVariances[6].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(7), NUM_RECORDS, expectedDoubleSum,
+        expectedVariances[7].getResult(), true);
+
+    // Update the expected result by 3 more times (broker query will compute 4 identical segments)
+    for (int i = 0; i < NUM_RECORDS * 3; i++) {
+      int pos = i % NUM_RECORDS;
+      expectedVariances[0].increment(_intValues[pos]);
+      expectedVariances[1].increment(_longValues[pos]);
+      expectedVariances[2].increment(_floatValues[pos]);
+      expectedVariances[3].increment(_doubleValues[pos]);
+      expectedVariances[4].increment(_intValues[pos]);
+      expectedVariances[5].increment(_longValues[pos]);
+      expectedVariances[6].increment(_floatValues[pos]);
+      expectedVariances[7].increment(_doubleValues[pos]);
+    }
+
+    // Validate the response
+    BrokerResponseNative brokerResponse = getBrokerResponse(query);
+    brokerResponse.getResultTable();
+    Object[] results = brokerResponse.getResultTable().getRows().get(0);
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[0], expectedVariances[0].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[1], expectedVariances[1].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[2], expectedVariances[2].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[3], expectedVariances[3].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[4], expectedVariances[4].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[5], expectedVariances[5].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[6], expectedVariances[6].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[7], expectedVariances[7].getResult(), RELATIVE_EPSILON));
+
+    // Validate the response for a query with a filter
+    query = "SELECT VAR_POP(intColumn) from testTable" + getFilter();
+    brokerResponse = getBrokerResponse(query);
+    brokerResponse.getResultTable();
+    results = brokerResponse.getResultTable().getRows().get(0);
+    Variance filterExpectedVariance = new Variance(false);
+    for (int i = 0; i < NUM_RECORDS / 2; i++) {
+      filterExpectedVariance.increment(_intValues[i]);
+    }
+    assertTrue(Precision.equalsWithRelativeTolerance((double) results[0], filterExpectedVariance.getResult(),
+        RELATIVE_EPSILON));
+  }
+
+  @Test
+  public void testVarianceAggregationGroupBy() {
+    // Compute expected group results
+    Variance[] expectedGroupByResult = new Variance[NUM_GROUPS];
+    double[] expectedSum = new double[NUM_GROUPS];
+
+    for (int i = 0; i < NUM_GROUPS; i++) {
+      expectedGroupByResult[i] = new Variance(false);
+    }
+    for (int j = 0; j < NUM_RECORDS; j++) {
+      int pos = j / (NUM_RECORDS / NUM_GROUPS);
+      expectedGroupByResult[pos].increment(_intValues[j]);
+      expectedSum[pos] += _intValues[j];
+    }
+
+    String query = "SELECT VAR_POP(intColumn) FROM testTable GROUP BY groupByColumn ORDER BY groupByColumn";
+    GroupByOperator groupByOperator = getOperator(query);
+    GroupByResultsBlock resultsBlock = groupByOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(groupByOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 2, NUM_RECORDS);
+    AggregationGroupByResult aggregationGroupByResult = resultsBlock.getAggregationGroupByResult();
+    assertNotNull(aggregationGroupByResult);
+    for (int i = 0; i < NUM_GROUPS; i++) {
+
+      VarianceTuple actualVarianceTuple = (VarianceTuple) aggregationGroupByResult.getResultForGroupId(0, i);
+      checkWithPrecisionForVariance(actualVarianceTuple, NUM_RECORDS / NUM_GROUPS, expectedSum[i],
+          expectedGroupByResult[i].getResult(), false);
+    }
+  }
+
+  @Test
+  public void testStandardDeviationAggregationOnly() {
+    // Compute the expected values
+    StandardDeviation[] expectedStdDevs = new StandardDeviation[8];
+    for (int i = 0; i < 8; i++) {
+      if (i < 4) {
+        expectedStdDevs[i] = new StandardDeviation(false);
+      } else {
+        expectedStdDevs[i] = new StandardDeviation(true);
+      }
+    }
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      expectedStdDevs[0].increment(_intValues[i]);
+      expectedStdDevs[1].increment(_longValues[i]);
+      expectedStdDevs[2].increment(_floatValues[i]);
+      expectedStdDevs[3].increment(_doubleValues[i]);
+      expectedStdDevs[4].increment(_intValues[i]);
+      expectedStdDevs[5].increment(_longValues[i]);
+      expectedStdDevs[6].increment(_floatValues[i]);
+      expectedStdDevs[7].increment(_doubleValues[i]);
+    }
+
+    double expectedIntSum = Arrays.stream(_intValues).asDoubleStream().sum();
+    double expectedLongSum = Arrays.stream(_longValues).asDoubleStream().sum();
+    double expectedFloatSum = 0.0;
+    for (int i = 0; i < _floatValues.length; i++) {
+      expectedFloatSum += _floatValues[i];
+    }
+    double expectedDoubleSum = Arrays.stream(_doubleValues).sum();
+
+    // Compute the query
+    String query =
+        "SELECT STDDEV_POP(intColumn), STDDEV_POP(longColumn), STDDEV_POP(floatColumn), STDDEV_POP(doubleColumn),"
+            + "STDDEV_SAMP(intColumn), STDDEV_SAMP(longColumn), STDDEV_SAMP(floatColumn), STDDEV_SAMP(doubleColumn) "
+            + "FROM testTable";
+    AggregationOperator aggregationOperator = getOperator(query);
+    AggregationResultsBlock resultsBlock = aggregationOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(aggregationOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 4, NUM_RECORDS);
+    List<Object> aggregationResult = resultsBlock.getResults();
+
+    // Validate the aggregation results
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(0), NUM_RECORDS, expectedIntSum,
+        expectedStdDevs[0].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(1), NUM_RECORDS, expectedLongSum,
+        expectedStdDevs[1].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(2), NUM_RECORDS, expectedFloatSum,
+        expectedStdDevs[2].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(3), NUM_RECORDS, expectedDoubleSum,
+        expectedStdDevs[3].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(4), NUM_RECORDS, expectedIntSum,
+        expectedStdDevs[4].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(5), NUM_RECORDS, expectedLongSum,
+        expectedStdDevs[5].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(6), NUM_RECORDS, expectedFloatSum,
+        expectedStdDevs[6].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(7), NUM_RECORDS, expectedDoubleSum,
+        expectedStdDevs[7].getResult(), true);
+
+    // Update the expected result by 3 more times (broker query will compute 4 identical segments)

Review Comment:
   Yes, in `CovarianceQueriesTest`, @SabrinaZhaozyf has added the [getDistinctInstances()](https://github.com/apache/pinot/blob/270315a13f457376df04cd47bc62f84d1c1e558e/pinot-core/src/test/java/org/apache/pinot/queries/CovarianceQueriesTest.java#L157) to use 4 distinct segments for these statistical functions, because Covar on 4 identical segments is the same as Covar on one of each segment, as kind of defeats the purpose of testing merge logic over multiple segments. The `getDistinctInstances` in base class is still returning 4 identical segments though for backward compat. We can probably inherit `CovarianceQueriesTest` in `VarianceQueriesTest`.



-- 
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] jasperjiaguo commented on a diff in pull request #9910: Add Variance and Standard Deviation Aggregation Functions

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on code in PR #9910:
URL: https://github.com/apache/pinot/pull/9910#discussion_r1042545235


##########
pinot-core/src/test/java/org/apache/pinot/queries/VarianceQueriesTest.java:
##########
@@ -0,0 +1,446 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.math3.stat.descriptive.moment.StandardDeviation;
+import org.apache.commons.math3.stat.descriptive.moment.Variance;
+import org.apache.commons.math3.util.Precision;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.core.operator.blocks.results.AggregationResultsBlock;
+import org.apache.pinot.core.operator.blocks.results.GroupByResultsBlock;
+import org.apache.pinot.core.operator.query.AggregationOperator;
+import org.apache.pinot.core.operator.query.GroupByOperator;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByResult;
+import org.apache.pinot.segment.local.customobject.VarianceTuple;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertTrue;
+
+
+public class VarianceQueriesTest extends BaseQueriesTest {
+
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "VarianceQueriesTest");
+
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+
+  private static final int NUM_RECORDS = 2000;
+  private static final int NUM_GROUPS = 10;
+  private static final int MAX_VALUE = 500;
+  private static final double RELATIVE_EPSILON = 0.0001;
+  private static final double DELTA = 0.0001;
+
+  private static final String INT_COLUMN = "intColumn";
+  private static final String LONG_COLUMN = "longColumn";
+  private static final String FLOAT_COLUMN = "floatColumn";
+  private static final String DOUBLE_COLUMN = "doubleColumn";
+  private static final String GROUP_BY_COLUMN = "groupByColumn";
+
+  private static final Schema SCHEMA =
+      new Schema.SchemaBuilder().addSingleValueDimension(INT_COLUMN, FieldSpec.DataType.INT)
+          .addSingleValueDimension(LONG_COLUMN, FieldSpec.DataType.LONG)
+          .addSingleValueDimension(FLOAT_COLUMN, FieldSpec.DataType.FLOAT)
+          .addSingleValueDimension(DOUBLE_COLUMN, FieldSpec.DataType.DOUBLE)
+          .addSingleValueDimension(GROUP_BY_COLUMN, FieldSpec.DataType.DOUBLE).build();
+
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  int[] _intValues = new int[NUM_RECORDS];
+  long[] _longValues = new long[NUM_RECORDS];
+  float[] _floatValues = new float[NUM_RECORDS];
+  double[] _doubleValues = new double[NUM_RECORDS];
+
+  @Override
+  protected String getFilter() {
+    // filter out half of the rows based on group id
+    return " WHERE groupByColumn < " + (NUM_GROUPS / 2);
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+    Random random = new Random();
+    List<GenericRow> records = new ArrayList<>(NUM_RECORDS);
+
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      GenericRow record = new GenericRow();
+      int intValue = -MAX_VALUE + random.nextInt() * 2 * MAX_VALUE;
+      long longValue = -MAX_VALUE + random.nextLong() * 2 * MAX_VALUE;
+      float floatValue = -MAX_VALUE + random.nextFloat() * 2 * MAX_VALUE;
+      double doubleValue = -MAX_VALUE + random.nextDouble() * 2 * MAX_VALUE;
+
+      _intValues[i] = intValue;
+      _longValues[i] = longValue;
+      _floatValues[i] = floatValue;
+      _doubleValues[i] = doubleValue;
+
+      record.putValue(INT_COLUMN, _intValues[i]);
+      record.putValue(LONG_COLUMN, _longValues[i]);
+      record.putValue(FLOAT_COLUMN, _floatValues[i]);
+      record.putValue(DOUBLE_COLUMN, _doubleValues[i]);
+      record.putValue(GROUP_BY_COLUMN, Math.floor(i / (NUM_RECORDS / NUM_GROUPS)));
+      records.add(record);
+    }
+
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(TABLE_CONFIG, SCHEMA);
+    segmentGeneratorConfig.setTableName(RAW_TABLE_NAME);
+    segmentGeneratorConfig.setSegmentName(SEGMENT_NAME);
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getPath());
+
+    SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig, new GenericRowRecordReader(records));
+    driver.build();
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, SEGMENT_NAME), ReadMode.mmap);
+    _indexSegment = immutableSegment;
+    _indexSegments = Arrays.asList(immutableSegment, immutableSegment);
+  }
+
+  @Test
+  public void testVarianceAggregationOnly() {
+    // Compute the expected values
+    Variance[] expectedVariances = new Variance[8];
+    for (int i = 0; i < 8; i++) {
+      if (i < 4) {
+        expectedVariances[i] = new Variance(false);
+      } else {
+        expectedVariances[i] = new Variance(true);
+      }
+    }
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      expectedVariances[0].increment(_intValues[i]);
+      expectedVariances[1].increment(_longValues[i]);
+      expectedVariances[2].increment(_floatValues[i]);
+      expectedVariances[3].increment(_doubleValues[i]);
+      expectedVariances[4].increment(_intValues[i]);
+      expectedVariances[5].increment(_longValues[i]);
+      expectedVariances[6].increment(_floatValues[i]);
+      expectedVariances[7].increment(_doubleValues[i]);
+    }
+    double expectedIntSum = Arrays.stream(_intValues).asDoubleStream().sum();
+    double expectedLongSum = Arrays.stream(_longValues).asDoubleStream().sum();
+    double expectedFloatSum = 0.0;
+    for (int i = 0; i < _floatValues.length; i++) {
+      expectedFloatSum += _floatValues[i];
+    }
+    double expectedDoubleSum = Arrays.stream(_doubleValues).sum();
+
+    // Compute the query
+    String query = "SELECT VAR_POP(intColumn), VAR_POP(longColumn), VAR_POP(floatColumn), VAR_POP(doubleColumn),"
+        + "VAR_SAMP(intColumn), VAR_SAMP(longColumn), VAR_SAMP(floatColumn), VAR_SAMP(doubleColumn) FROM testTable";
+    AggregationOperator aggregationOperator = getOperator(query);
+    AggregationResultsBlock resultsBlock = aggregationOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(aggregationOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 4, NUM_RECORDS);
+    List<Object> aggregationResult = resultsBlock.getResults();
+
+    // Validate the aggregation results
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(0), NUM_RECORDS, expectedIntSum,
+        expectedVariances[0].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(1), NUM_RECORDS, expectedLongSum,
+        expectedVariances[1].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(2), NUM_RECORDS, expectedFloatSum,
+        expectedVariances[2].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(3), NUM_RECORDS, expectedDoubleSum,
+        expectedVariances[3].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(4), NUM_RECORDS, expectedIntSum,
+        expectedVariances[4].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(5), NUM_RECORDS, expectedLongSum,
+        expectedVariances[5].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(6), NUM_RECORDS, expectedFloatSum,
+        expectedVariances[6].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(7), NUM_RECORDS, expectedDoubleSum,
+        expectedVariances[7].getResult(), true);
+
+    // Update the expected result by 3 more times (broker query will compute 4 identical segments)
+    for (int i = 0; i < NUM_RECORDS * 3; i++) {
+      int pos = i % NUM_RECORDS;
+      expectedVariances[0].increment(_intValues[pos]);
+      expectedVariances[1].increment(_longValues[pos]);
+      expectedVariances[2].increment(_floatValues[pos]);
+      expectedVariances[3].increment(_doubleValues[pos]);
+      expectedVariances[4].increment(_intValues[pos]);
+      expectedVariances[5].increment(_longValues[pos]);
+      expectedVariances[6].increment(_floatValues[pos]);
+      expectedVariances[7].increment(_doubleValues[pos]);
+    }
+
+    // Validate the response
+    BrokerResponseNative brokerResponse = getBrokerResponse(query);
+    brokerResponse.getResultTable();
+    Object[] results = brokerResponse.getResultTable().getRows().get(0);
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[0], expectedVariances[0].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[1], expectedVariances[1].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[2], expectedVariances[2].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[3], expectedVariances[3].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[4], expectedVariances[4].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[5], expectedVariances[5].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[6], expectedVariances[6].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[7], expectedVariances[7].getResult(), RELATIVE_EPSILON));
+
+    // Validate the response for a query with a filter
+    query = "SELECT VAR_POP(intColumn) from testTable" + getFilter();
+    brokerResponse = getBrokerResponse(query);
+    brokerResponse.getResultTable();
+    results = brokerResponse.getResultTable().getRows().get(0);
+    Variance filterExpectedVariance = new Variance(false);
+    for (int i = 0; i < NUM_RECORDS / 2; i++) {
+      filterExpectedVariance.increment(_intValues[i]);
+    }
+    assertTrue(Precision.equalsWithRelativeTolerance((double) results[0], filterExpectedVariance.getResult(),
+        RELATIVE_EPSILON));
+  }
+
+  @Test
+  public void testVarianceAggregationGroupBy() {
+    // Compute expected group results
+    Variance[] expectedGroupByResult = new Variance[NUM_GROUPS];
+    double[] expectedSum = new double[NUM_GROUPS];
+
+    for (int i = 0; i < NUM_GROUPS; i++) {
+      expectedGroupByResult[i] = new Variance(false);
+    }
+    for (int j = 0; j < NUM_RECORDS; j++) {
+      int pos = j / (NUM_RECORDS / NUM_GROUPS);
+      expectedGroupByResult[pos].increment(_intValues[j]);
+      expectedSum[pos] += _intValues[j];
+    }
+
+    String query = "SELECT VAR_POP(intColumn) FROM testTable GROUP BY groupByColumn ORDER BY groupByColumn";
+    GroupByOperator groupByOperator = getOperator(query);
+    GroupByResultsBlock resultsBlock = groupByOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(groupByOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 2, NUM_RECORDS);
+    AggregationGroupByResult aggregationGroupByResult = resultsBlock.getAggregationGroupByResult();
+    assertNotNull(aggregationGroupByResult);
+    for (int i = 0; i < NUM_GROUPS; i++) {
+
+      VarianceTuple actualVarianceTuple = (VarianceTuple) aggregationGroupByResult.getResultForGroupId(0, i);
+      checkWithPrecisionForVariance(actualVarianceTuple, NUM_RECORDS / NUM_GROUPS, expectedSum[i],
+          expectedGroupByResult[i].getResult(), false);
+    }
+  }
+
+  @Test
+  public void testStandardDeviationAggregationOnly() {
+    // Compute the expected values
+    StandardDeviation[] expectedStdDevs = new StandardDeviation[8];
+    for (int i = 0; i < 8; i++) {
+      if (i < 4) {
+        expectedStdDevs[i] = new StandardDeviation(false);
+      } else {
+        expectedStdDevs[i] = new StandardDeviation(true);
+      }
+    }
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      expectedStdDevs[0].increment(_intValues[i]);
+      expectedStdDevs[1].increment(_longValues[i]);
+      expectedStdDevs[2].increment(_floatValues[i]);
+      expectedStdDevs[3].increment(_doubleValues[i]);
+      expectedStdDevs[4].increment(_intValues[i]);
+      expectedStdDevs[5].increment(_longValues[i]);
+      expectedStdDevs[6].increment(_floatValues[i]);
+      expectedStdDevs[7].increment(_doubleValues[i]);
+    }
+
+    double expectedIntSum = Arrays.stream(_intValues).asDoubleStream().sum();
+    double expectedLongSum = Arrays.stream(_longValues).asDoubleStream().sum();
+    double expectedFloatSum = 0.0;
+    for (int i = 0; i < _floatValues.length; i++) {
+      expectedFloatSum += _floatValues[i];
+    }
+    double expectedDoubleSum = Arrays.stream(_doubleValues).sum();
+
+    // Compute the query
+    String query =
+        "SELECT STDDEV_POP(intColumn), STDDEV_POP(longColumn), STDDEV_POP(floatColumn), STDDEV_POP(doubleColumn),"
+            + "STDDEV_SAMP(intColumn), STDDEV_SAMP(longColumn), STDDEV_SAMP(floatColumn), STDDEV_SAMP(doubleColumn) "
+            + "FROM testTable";
+    AggregationOperator aggregationOperator = getOperator(query);
+    AggregationResultsBlock resultsBlock = aggregationOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(aggregationOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 4, NUM_RECORDS);
+    List<Object> aggregationResult = resultsBlock.getResults();
+
+    // Validate the aggregation results
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(0), NUM_RECORDS, expectedIntSum,
+        expectedStdDevs[0].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(1), NUM_RECORDS, expectedLongSum,
+        expectedStdDevs[1].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(2), NUM_RECORDS, expectedFloatSum,
+        expectedStdDevs[2].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(3), NUM_RECORDS, expectedDoubleSum,
+        expectedStdDevs[3].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(4), NUM_RECORDS, expectedIntSum,
+        expectedStdDevs[4].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(5), NUM_RECORDS, expectedLongSum,
+        expectedStdDevs[5].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(6), NUM_RECORDS, expectedFloatSum,
+        expectedStdDevs[6].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(7), NUM_RECORDS, expectedDoubleSum,
+        expectedStdDevs[7].getResult(), true);
+
+    // Update the expected result by 3 more times (broker query will compute 4 identical segments)

Review Comment:
   Yes, in `CovarianceQueriesTest`, @SabrinaZhaozyf has added the [getDistinctInstances()](https://github.com/apache/pinot/blob/270315a13f457376df04cd47bc62f84d1c1e558e/pinot-core/src/test/java/org/apache/pinot/queries/CovarianceQueriesTest.java#L157) to use 4 distinct segments for these statistical functions, because Covar on 4 identical segments is the same as Covar on one of each segment, as kind of defeats the purpose of testing merge logic over multiple segments. The `getDistinctInstances` in base class is still returning 4 identical segments for backward compat. We can probably inherit `CovarianceQueriesTest` in `VarianceQueriesTest`.



-- 
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] Jackie-Jiang commented on a diff in pull request #9910: Add Variance and Standard Deviation Aggregation Functions

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on code in PR #9910:
URL: https://github.com/apache/pinot/pull/9910#discussion_r1043947115


##########
pinot-core/src/test/java/org/apache/pinot/queries/StatisticalQueriesTest.java:
##########
@@ -0,0 +1,749 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.math3.stat.correlation.Covariance;
+import org.apache.commons.math3.stat.descriptive.moment.StandardDeviation;
+import org.apache.commons.math3.stat.descriptive.moment.Variance;
+import org.apache.commons.math3.util.Precision;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.core.operator.blocks.results.AggregationResultsBlock;
+import org.apache.pinot.core.operator.blocks.results.GroupByResultsBlock;
+import org.apache.pinot.core.operator.query.AggregationOperator;
+import org.apache.pinot.core.operator.query.GroupByOperator;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByResult;
+import org.apache.pinot.segment.local.customobject.CovarianceTuple;
+import org.apache.pinot.segment.local.customobject.VarianceTuple;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * Queries test for statistical queries (i.e Variance, Covariance, Standard Deviation etc)
+ */
+public class StatisticalQueriesTest extends BaseQueriesTest {
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "CovarianceQueriesTest");
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+
+  // test segments 1-4 evenly divide testSegment into 4 distinct segments
+  private static final String SEGMENT_NAME_1 = "testSegment1";
+  private static final String SEGMENT_NAME_2 = "testSegment2";
+  private static final String SEGMENT_NAME_3 = "testSegment3";
+  private static final String SEGMENT_NAME_4 = "testSegment4";
+
+  private static final int NUM_RECORDS = 2000;
+  private static final int NUM_GROUPS = 10;
+  private static final int MAX_VALUE = 500;
+  private static final double RELATIVE_EPSILON = 0.0001;
+  private static final double DELTA = 0.0001;
+
+  private static final String INT_COLUMN_X = "intColumnX";
+  private static final String INT_COLUMN_Y = "intColumnY";
+  private static final String DOUBLE_COLUMN_X = "doubleColumnX";
+  private static final String DOUBLE_COLUMN_Y = "doubleColumnY";
+  private static final String LONG_COLUMN = "longColumn";
+  private static final String FLOAT_COLUMN = "floatColumn";
+  private static final String GROUP_BY_COLUMN = "groupByColumn";
+
+  private static final Schema SCHEMA =
+      new Schema.SchemaBuilder().addSingleValueDimension(INT_COLUMN_X, FieldSpec.DataType.INT)
+          .addSingleValueDimension(INT_COLUMN_Y, FieldSpec.DataType.INT)
+          .addSingleValueDimension(DOUBLE_COLUMN_X, FieldSpec.DataType.DOUBLE)
+          .addSingleValueDimension(DOUBLE_COLUMN_Y, FieldSpec.DataType.DOUBLE)
+          .addSingleValueDimension(LONG_COLUMN, FieldSpec.DataType.LONG)
+          .addSingleValueDimension(FLOAT_COLUMN, FieldSpec.DataType.FLOAT)
+          .addSingleValueDimension(GROUP_BY_COLUMN, FieldSpec.DataType.DOUBLE).build();
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+  private List<List<IndexSegment>> _distinctInstances;
+  private int _sumIntX = 0;
+  private int _sumIntY = 0;
+  private int _sumIntXY = 0;
+
+  private double _sumDoubleX = 0;
+  private double _sumDoubleY = 0;
+  private double _sumDoubleXY = 0;
+
+  private long _sumLong = 0L;
+  private double _sumFloat = 0;
+
+  private double _sumIntDouble = 0;
+  private long _sumIntLong = 0L;
+  private double _sumIntFloat = 0;
+  private double _sumDoubleLong = 0;
+  private double _sumDoubleFloat = 0;
+  private double _sumLongFloat = 0;
+
+  private double _expectedCovIntXY;
+  private double _expectedCovDoubleXY;
+  private double _expectedCovIntDouble;
+  private double _expectedCovIntLong;
+  private double _expectedCovIntFloat;
+  private double _expectedCovDoubleLong;
+  private double _expectedCovDoubleFloat;
+  private double _expectedCovLongFloat;
+
+  private double _expectedCovWithFilter;
+
+  private final CovarianceTuple[] _expectedGroupByResultVer1 = new CovarianceTuple[NUM_GROUPS];
+  private final CovarianceTuple[] _expectedGroupByResultVer2 = new CovarianceTuple[NUM_GROUPS];
+  private final double[] _expectedFinalResultVer1 = new double[NUM_GROUPS];
+  private final double[] _expectedFinalResultVer2 = new double[NUM_GROUPS];
+
+  private boolean _useIdenticalSegment = false;
+
+  int[] _intColX = new int[NUM_RECORDS];
+  int[] _intColY = new int[NUM_RECORDS];
+  long[] _longCol = new long[NUM_RECORDS];
+  double[] _floatCol = new double[NUM_RECORDS];
+  double[] _doubleColX = new double[NUM_RECORDS];
+  double[] _doubleColY = new double[NUM_RECORDS];
+  double[] _groupByCol = new double[NUM_RECORDS];
+
+  @Override
+  protected String getFilter() {
+    // filter out half of the rows based on group id
+    return " WHERE groupByColumn < " + (NUM_GROUPS / 2);
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @Override
+  protected List<List<IndexSegment>> getDistinctInstances() {
+    if (_useIdenticalSegment) {
+      return Collections.singletonList(_indexSegments);
+    }
+    return _distinctInstances;
+  }
+
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+
+    List<GenericRow> records = new ArrayList<>(NUM_RECORDS);
+
+    Random rand = new Random();
+    _intColX = rand.ints(NUM_RECORDS, -MAX_VALUE, MAX_VALUE).toArray();
+    _intColY = rand.ints(NUM_RECORDS, -MAX_VALUE, MAX_VALUE).toArray();
+    _doubleColX = rand.doubles(NUM_RECORDS, -MAX_VALUE, MAX_VALUE).toArray();
+    _doubleColY = rand.doubles(NUM_RECORDS, -MAX_VALUE, MAX_VALUE).toArray();
+    _longCol = rand.longs(NUM_RECORDS, -MAX_VALUE, MAX_VALUE).toArray();
+
+    int groupSize = NUM_RECORDS / NUM_GROUPS;
+    double sumX = 0;
+    double sumY = 0;
+    double sumGroupBy = 0;
+    double sumXY = 0;
+    double sumXGroupBy = 0;
+    int groupByVal = 0;
+
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      GenericRow record = new GenericRow();
+      int intX = _intColX[i];
+      int intY = _intColY[i];
+      double doubleX = _doubleColX[i];
+      double doubleY = _doubleColY[i];
+      long longVal = _longCol[i];
+      float floatVal = -MAX_VALUE + rand.nextFloat() * 2 * MAX_VALUE;
+
+      // set up inner segment group by results
+      groupByVal = (int) Math.floor(i / groupSize);
+      if (i % groupSize == 0 && groupByVal > 0) {
+        _expectedGroupByResultVer1[groupByVal - 1] = new CovarianceTuple(sumX, sumGroupBy, sumXGroupBy, groupSize);
+        _expectedGroupByResultVer2[groupByVal - 1] = new CovarianceTuple(sumX, sumY, sumXY, groupSize);
+        sumX = 0;
+        sumY = 0;
+        sumGroupBy = 0;
+        sumXY = 0;
+        sumXGroupBy = 0;
+      }
+
+      sumX += doubleX;
+      sumY += doubleY;
+      sumGroupBy += groupByVal;
+      sumXY += doubleX * doubleY;
+      sumXGroupBy += doubleX * groupByVal;
+
+      _floatCol[i] = floatVal;
+      _groupByCol[i] = groupByVal;
+
+      // calculate inner segment results
+      _sumIntX += intX;
+      _sumIntY += intY;
+      _sumDoubleX += doubleX;
+      _sumDoubleY += doubleY;
+      _sumLong += longVal;
+      _sumFloat += floatVal;
+      _sumIntXY += intX * intY;
+      _sumDoubleXY += doubleX * doubleY;
+      _sumIntDouble += intX * doubleX;
+      _sumIntLong += intX * longVal;
+      _sumIntFloat += intX * _floatCol[i];
+      _sumDoubleLong += doubleX * longVal;
+      _sumDoubleFloat += doubleX * _floatCol[i];
+      _sumLongFloat += longVal * _floatCol[i];
+
+      record.putValue(INT_COLUMN_X, intX);
+      record.putValue(INT_COLUMN_Y, intY);
+      record.putValue(DOUBLE_COLUMN_X, doubleX);
+      record.putValue(DOUBLE_COLUMN_Y, doubleY);
+      record.putValue(LONG_COLUMN, longVal);
+      record.putValue(FLOAT_COLUMN, floatVal);
+      record.putValue(GROUP_BY_COLUMN, groupByVal);
+      records.add(record);
+    }
+    _expectedGroupByResultVer1[groupByVal] = new CovarianceTuple(sumX, sumGroupBy, sumXGroupBy, groupSize);
+    _expectedGroupByResultVer2[groupByVal] = new CovarianceTuple(sumX, sumY, sumXY, groupSize);
+
+    // calculate inter segment result
+    Covariance cov = new Covariance();
+    double[] newIntColX = Arrays.stream(_intColX).asDoubleStream().toArray();
+    double[] newIntColY = Arrays.stream(_intColY).asDoubleStream().toArray();
+    double[] newLongCol = Arrays.stream(_longCol).asDoubleStream().toArray();
+    _expectedCovIntXY = cov.covariance(newIntColX, newIntColY, false);
+    _expectedCovDoubleXY = cov.covariance(_doubleColX, _doubleColY, false);
+    _expectedCovIntDouble = cov.covariance(newIntColX, _doubleColX, false);
+    _expectedCovIntLong = cov.covariance(newIntColX, newLongCol, false);
+    _expectedCovIntFloat = cov.covariance(newIntColX, _floatCol, false);
+    _expectedCovDoubleLong = cov.covariance(_doubleColX, newLongCol, false);
+    _expectedCovDoubleFloat = cov.covariance(_doubleColX, _floatCol, false);
+    _expectedCovLongFloat = cov.covariance(newLongCol, _floatCol, false);
+
+    double[] filteredX = Arrays.copyOfRange(_doubleColX, 0, NUM_RECORDS / 2);
+    double[] filteredY = Arrays.copyOfRange(_doubleColY, 0, NUM_RECORDS / 2);
+    _expectedCovWithFilter = cov.covariance(filteredX, filteredY, false);
+
+    // calculate inter segment group by results
+    for (int i = 0; i < NUM_GROUPS; i++) {
+      double[] colX = Arrays.copyOfRange(_doubleColX, i * groupSize, (i + 1) * groupSize);
+      double[] colGroupBy = Arrays.copyOfRange(_groupByCol, i * groupSize, (i + 1) * groupSize);
+      double[] colY = Arrays.copyOfRange(_doubleColY, i * groupSize, (i + 1) * groupSize);
+      _expectedFinalResultVer1[i] = cov.covariance(colX, colGroupBy, false);
+      _expectedFinalResultVer2[i] = cov.covariance(colX, colY, false);
+    }
+
+    // generate testSegment
+    ImmutableSegment immutableSegment = setUpSingleSegment(records, SEGMENT_NAME);
+    _indexSegment = immutableSegment;
+    _indexSegments = Arrays.asList(immutableSegment, immutableSegment);
+
+    // divide testSegment into 4 distinct segments for distinct inter segment tests
+    // by doing so, we can avoid calculating global covariance again
+    _distinctInstances = new ArrayList<>();
+    int segmentSize = NUM_RECORDS / 4;
+    ImmutableSegment immutableSegment1 = setUpSingleSegment(records.subList(0, segmentSize), SEGMENT_NAME_1);
+    ImmutableSegment immutableSegment2 =
+        setUpSingleSegment(records.subList(segmentSize, segmentSize * 2), SEGMENT_NAME_2);
+    ImmutableSegment immutableSegment3 =
+        setUpSingleSegment(records.subList(segmentSize * 2, segmentSize * 3), SEGMENT_NAME_3);
+    ImmutableSegment immutableSegment4 =
+        setUpSingleSegment(records.subList(segmentSize * 3, NUM_RECORDS), SEGMENT_NAME_4);
+    // generate 2 instances each with 2 distinct segments
+    _distinctInstances.add(Arrays.asList(immutableSegment1, immutableSegment2));
+    _distinctInstances.add(Arrays.asList(immutableSegment3, immutableSegment4));
+  }
+
+  private ImmutableSegment setUpSingleSegment(List<GenericRow> recordSet, String segmentName)
+      throws Exception {
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(TABLE_CONFIG, SCHEMA);
+    segmentGeneratorConfig.setTableName(RAW_TABLE_NAME);
+    segmentGeneratorConfig.setSegmentName(segmentName);
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getPath());
+
+    SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig, new GenericRowRecordReader(recordSet));
+    driver.build();
+
+    return ImmutableSegmentLoader.load(new File(INDEX_DIR, segmentName), ReadMode.mmap);
+  }
+
+  @Test
+  public void testCovarianceAggregationOnly() {
+    // Inner Segment
+    String query =
+        "SELECT COVAR_POP(intColumnX, intColumnY), COVAR_POP(doubleColumnX, doubleColumnY), COVAR_POP(intColumnX, "
+            + "doubleColumnX), " + "COVAR_POP(intColumnX, longColumn), COVAR_POP(intColumnX, floatColumn), "
+            + "COVAR_POP(doubleColumnX, longColumn), COVAR_POP(doubleColumnX, floatColumn), COVAR_POP(longColumn, "
+            + "floatColumn)  FROM testTable";
+    AggregationOperator aggregationOperator = getOperator(query);
+    AggregationResultsBlock resultsBlock = aggregationOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(aggregationOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 6, NUM_RECORDS);
+    List<Object> aggregationResult = resultsBlock.getResults();
+    assertNotNull(aggregationResult);
+    checkWithPrecisionForCovariance((CovarianceTuple) aggregationResult.get(0), _sumIntX, _sumIntY, _sumIntXY,
+        NUM_RECORDS);
+    checkWithPrecisionForCovariance((CovarianceTuple) aggregationResult.get(1), _sumDoubleX, _sumDoubleY, _sumDoubleXY,
+        NUM_RECORDS);
+    checkWithPrecisionForCovariance((CovarianceTuple) aggregationResult.get(2), _sumIntX, _sumDoubleX, _sumIntDouble,
+        NUM_RECORDS);
+    checkWithPrecisionForCovariance((CovarianceTuple) aggregationResult.get(3), _sumIntX, _sumLong, _sumIntLong,
+        NUM_RECORDS);
+    checkWithPrecisionForCovariance((CovarianceTuple) aggregationResult.get(4), _sumIntX, _sumFloat, _sumIntFloat,
+        NUM_RECORDS);
+    checkWithPrecisionForCovariance((CovarianceTuple) aggregationResult.get(5), _sumDoubleX, _sumLong, _sumDoubleLong,
+        NUM_RECORDS);
+    checkWithPrecisionForCovariance((CovarianceTuple) aggregationResult.get(6), _sumDoubleX, _sumFloat, _sumDoubleFloat,
+        NUM_RECORDS);
+    checkWithPrecisionForCovariance((CovarianceTuple) aggregationResult.get(7), _sumLong, _sumFloat, _sumLongFloat,
+        NUM_RECORDS);
+
+    // Inter segments with 4 identical segments (2 instances each having 2 identical segments)
+    _useIdenticalSegment = true;
+    BrokerResponseNative brokerResponse = getBrokerResponse(query);
+    _useIdenticalSegment = false;
+    assertEquals(brokerResponse.getNumDocsScanned(), 4 * NUM_RECORDS);
+    assertEquals(brokerResponse.getNumEntriesScannedInFilter(), 0);
+    assertEquals(brokerResponse.getNumEntriesScannedPostFilter(), 4 * 6 * NUM_RECORDS);
+    assertEquals(brokerResponse.getTotalDocs(), 4 * NUM_RECORDS);
+    checkResultTableWithPrecisionForCovariance(brokerResponse);
+
+    // Inter segments with 4 distinct segments (2 instances each having 2 distinct segments)
+    brokerResponse = getBrokerResponse(query);
+    assertEquals(brokerResponse.getNumDocsScanned(), NUM_RECORDS);
+    assertEquals(brokerResponse.getNumEntriesScannedInFilter(), 0);
+    assertEquals(brokerResponse.getNumEntriesScannedPostFilter(), 6 * NUM_RECORDS);
+    assertEquals(brokerResponse.getTotalDocs(), NUM_RECORDS);
+    checkResultTableWithPrecisionForCovariance(brokerResponse);
+
+    // Inter segments with 4 identical segments with filter
+    _useIdenticalSegment = true;
+    query = "SELECT COVAR_POP(doubleColumnX, doubleColumnY) FROM testTable" + getFilter();
+    brokerResponse = getBrokerResponse(query);
+    _useIdenticalSegment = false;
+    assertEquals(brokerResponse.getNumDocsScanned(), 2 * NUM_RECORDS);
+    assertEquals(brokerResponse.getNumEntriesScannedInFilter(), 0);
+    assertEquals(brokerResponse.getNumEntriesScannedPostFilter(), 4 * NUM_RECORDS);
+    assertEquals(brokerResponse.getTotalDocs(), 4 * NUM_RECORDS);
+    Object[] results = brokerResponse.getResultTable().getRows().get(0);
+    assertTrue(Precision.equalsWithRelativeTolerance((double) results[0], _expectedCovWithFilter, RELATIVE_EPSILON));
+  }
+
+  @Test
+  public void testCovarianceAggregationGroupBy() {
+    // Inner Segment
+    // case 1: (col1, groupByCol) group by groupByCol => all covariances are 0's
+    String query =
+        "SELECT COVAR_POP(doubleColumnX, groupByColumn) FROM testTable GROUP BY groupByColumn ORDER BY groupByColumn";
+    GroupByOperator groupByOperator = getOperator(query);
+    GroupByResultsBlock resultsBlock = groupByOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(groupByOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 2, NUM_RECORDS);
+    AggregationGroupByResult aggregationGroupByResult = resultsBlock.getAggregationGroupByResult();
+    assertNotNull(aggregationGroupByResult);
+    for (int i = 0; i < NUM_GROUPS; i++) {
+      CovarianceTuple actualCovTuple = (CovarianceTuple) aggregationGroupByResult.getResultForGroupId(0, i);
+      CovarianceTuple expectedCovTuple = _expectedGroupByResultVer1[i];
+      checkWithPrecisionForCovariance(actualCovTuple, expectedCovTuple);
+    }
+
+    // Inter Segment with 4 identical segments
+    _useIdenticalSegment = true;
+    BrokerResponseNative brokerResponse = getBrokerResponse(query);
+    checkGroupByResultsForCovariance(brokerResponse, _expectedFinalResultVer1);
+    _useIdenticalSegment = false;
+    // Inter Segment with 4 distinct segments
+    brokerResponse = getBrokerResponse(query);
+    checkGroupByResultsForCovariance(brokerResponse, _expectedFinalResultVer1);
+
+    // Inner Segment
+    // case 2: COVAR_POP(col1, col2) group by groupByCol => nondeterministic cov
+    query =
+        "SELECT COVAR_POP(doubleColumnX, doubleColumnY) FROM testTable GROUP BY groupByColumn ORDER BY groupByColumn";
+    groupByOperator = getOperator(query);
+    resultsBlock = groupByOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(groupByOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 3, NUM_RECORDS);
+    aggregationGroupByResult = resultsBlock.getAggregationGroupByResult();
+    assertNotNull(aggregationGroupByResult);
+
+    for (int i = 0; i < NUM_GROUPS; i++) {
+      CovarianceTuple actualCovTuple = (CovarianceTuple) aggregationGroupByResult.getResultForGroupId(0, i);
+      CovarianceTuple expectedCovTuple = _expectedGroupByResultVer2[i];
+      checkWithPrecisionForCovariance(actualCovTuple, expectedCovTuple);
+    }
+
+    // Inter Segment with 4 identical segments
+    _useIdenticalSegment = true;
+    brokerResponse = getBrokerResponse(query);
+    checkGroupByResultsForCovariance(brokerResponse, _expectedFinalResultVer2);
+    _useIdenticalSegment = false;
+    // Inter Segment with 4 distinct segments
+    brokerResponse = getBrokerResponse(query);
+    checkGroupByResultsForCovariance(brokerResponse, _expectedFinalResultVer2);
+  }
+
+  @Test
+  public void testVarianceAggregationOnly() {
+    // Compute the expected values
+    Variance[] expectedVariances = new Variance[8];
+    for (int i = 0; i < 8; i++) {
+      if (i < 4) {
+        expectedVariances[i] = new Variance(false);
+      } else {
+        expectedVariances[i] = new Variance(true);
+      }
+    }
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      expectedVariances[0].increment(_intColX[i]);
+      expectedVariances[1].increment(_longCol[i]);
+      expectedVariances[2].increment(_floatCol[i]);
+      expectedVariances[3].increment(_doubleColX[i]);
+      expectedVariances[4].increment(_intColX[i]);
+      expectedVariances[5].increment(_longCol[i]);
+      expectedVariances[6].increment(_floatCol[i]);
+      expectedVariances[7].increment(_doubleColX[i]);
+    }
+    double expectedIntSum = Arrays.stream(_intColX).asDoubleStream().sum();
+    double expectedLongSum = Arrays.stream(_longCol).asDoubleStream().sum();
+    double expectedFloatSum = 0.0;
+    for (int i = 0; i < _floatCol.length; i++) {
+      expectedFloatSum += _floatCol[i];
+    }
+    double expectedDoubleSum = Arrays.stream(_doubleColX).sum();
+
+    // Compute the query
+    String query = "SELECT VAR_POP(intColumnX), VAR_POP(longColumn), VAR_POP(floatColumn), VAR_POP(doubleColumnX),"
+        + "VAR_SAMP(intColumnX), VAR_SAMP(longColumn), VAR_SAMP(floatColumn), VAR_SAMP(doubleColumnX) FROM testTable";
+    AggregationOperator aggregationOperator = getOperator(query);
+    AggregationResultsBlock resultsBlock = aggregationOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(aggregationOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 4, NUM_RECORDS);
+    List<Object> aggregationResult = resultsBlock.getResults();
+
+    // Validate the aggregation results
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(0), NUM_RECORDS, expectedIntSum,
+        expectedVariances[0].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(1), NUM_RECORDS, expectedLongSum,
+        expectedVariances[1].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(2), NUM_RECORDS, expectedFloatSum,
+        expectedVariances[2].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(3), NUM_RECORDS, expectedDoubleSum,
+        expectedVariances[3].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(4), NUM_RECORDS, expectedIntSum,
+        expectedVariances[4].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(5), NUM_RECORDS, expectedLongSum,
+        expectedVariances[5].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(6), NUM_RECORDS, expectedFloatSum,
+        expectedVariances[6].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(7), NUM_RECORDS, expectedDoubleSum,
+        expectedVariances[7].getResult(), true);
+
+    // Validate the response
+    BrokerResponseNative brokerResponse = getBrokerResponse(query);
+    Object[] results = brokerResponse.getResultTable().getRows().get(0);
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[0], expectedVariances[0].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[1], expectedVariances[1].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[2], expectedVariances[2].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[3], expectedVariances[3].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[4], expectedVariances[4].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[5], expectedVariances[5].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[6], expectedVariances[6].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[7], expectedVariances[7].getResult(), RELATIVE_EPSILON));
+
+    VarianceTuple test = ((VarianceTuple) aggregationResult.get(0));
+    test.apply((new VarianceTuple(0, 0, 0.0d)));
+    System.out.println(test.getM2());
+    // Validate the response for a query with a filter
+    query = "SELECT VAR_POP(intColumnX) from testTable" + getFilter();
+    brokerResponse = getBrokerResponse(query);
+    brokerResponse.getResultTable();
+    results = brokerResponse.getResultTable().getRows().get(0);
+    Variance filterExpectedVariance = new Variance(false);
+    for (int i = 0; i < NUM_RECORDS / 2; i++) {
+      filterExpectedVariance.increment(_intColX[i]);
+    }
+    assertTrue(Precision.equalsWithRelativeTolerance((double) results[0], filterExpectedVariance.getResult(),
+        RELATIVE_EPSILON));
+  }
+
+  @Test
+  public void testVarianceAggregationGroupBy() {
+    // Compute expected group results
+    Variance[] expectedGroupByResult = new Variance[NUM_GROUPS];
+    double[] expectedSum = new double[NUM_GROUPS];
+
+    for (int i = 0; i < NUM_GROUPS; i++) {
+      expectedGroupByResult[i] = new Variance(false);
+    }
+    for (int j = 0; j < NUM_RECORDS; j++) {
+      int pos = j / (NUM_RECORDS / NUM_GROUPS);
+      expectedGroupByResult[pos].increment(_intColX[j]);
+      expectedSum[pos] += _intColX[j];
+    }
+
+    String query = "SELECT VAR_POP(intColumnX) FROM testTable GROUP BY groupByColumn ORDER BY groupByColumn";
+    GroupByOperator groupByOperator = getOperator(query);
+    GroupByResultsBlock resultsBlock = groupByOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(groupByOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 2, NUM_RECORDS);
+    AggregationGroupByResult aggregationGroupByResult = resultsBlock.getAggregationGroupByResult();
+    assertNotNull(aggregationGroupByResult);
+    for (int i = 0; i < NUM_GROUPS; i++) {
+
+      VarianceTuple actualVarianceTuple = (VarianceTuple) aggregationGroupByResult.getResultForGroupId(0, i);
+      checkWithPrecisionForVariance(actualVarianceTuple, NUM_RECORDS / NUM_GROUPS, expectedSum[i],
+          expectedGroupByResult[i].getResult(), false);
+    }
+  }
+
+  @Test
+  public void testStandardDeviationAggregationOnly() {

Review Comment:
   Should be fixed with #9948 



-- 
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] jasperjiaguo commented on a diff in pull request #9910: Add Variance and Standard Deviation Aggregation Functions

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on code in PR #9910:
URL: https://github.com/apache/pinot/pull/9910#discussion_r1041611525


##########
pinot-core/src/test/java/org/apache/pinot/queries/VarianceQueriesTest.java:
##########
@@ -0,0 +1,446 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.math3.stat.descriptive.moment.StandardDeviation;
+import org.apache.commons.math3.stat.descriptive.moment.Variance;
+import org.apache.commons.math3.util.Precision;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.core.operator.blocks.results.AggregationResultsBlock;
+import org.apache.pinot.core.operator.blocks.results.GroupByResultsBlock;
+import org.apache.pinot.core.operator.query.AggregationOperator;
+import org.apache.pinot.core.operator.query.GroupByOperator;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByResult;
+import org.apache.pinot.segment.local.customobject.VarianceTuple;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertTrue;
+
+
+public class VarianceQueriesTest extends BaseQueriesTest {
+
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "VarianceQueriesTest");
+
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+
+  private static final int NUM_RECORDS = 2000;
+  private static final int NUM_GROUPS = 10;
+  private static final int MAX_VALUE = 500;
+  private static final double RELATIVE_EPSILON = 0.0001;
+  private static final double DELTA = 0.0001;
+
+  private static final String INT_COLUMN = "intColumn";
+  private static final String LONG_COLUMN = "longColumn";
+  private static final String FLOAT_COLUMN = "floatColumn";
+  private static final String DOUBLE_COLUMN = "doubleColumn";
+  private static final String GROUP_BY_COLUMN = "groupByColumn";
+
+  private static final Schema SCHEMA =
+      new Schema.SchemaBuilder().addSingleValueDimension(INT_COLUMN, FieldSpec.DataType.INT)
+          .addSingleValueDimension(LONG_COLUMN, FieldSpec.DataType.LONG)
+          .addSingleValueDimension(FLOAT_COLUMN, FieldSpec.DataType.FLOAT)
+          .addSingleValueDimension(DOUBLE_COLUMN, FieldSpec.DataType.DOUBLE)
+          .addSingleValueDimension(GROUP_BY_COLUMN, FieldSpec.DataType.DOUBLE).build();
+
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  int[] _intValues = new int[NUM_RECORDS];
+  long[] _longValues = new long[NUM_RECORDS];
+  float[] _floatValues = new float[NUM_RECORDS];
+  double[] _doubleValues = new double[NUM_RECORDS];
+
+  @Override
+  protected String getFilter() {
+    // filter out half of the rows based on group id
+    return " WHERE groupByColumn < " + (NUM_GROUPS / 2);
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+    Random random = new Random();
+    List<GenericRow> records = new ArrayList<>(NUM_RECORDS);
+
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      GenericRow record = new GenericRow();
+      int intValue = -MAX_VALUE + random.nextInt() * 2 * MAX_VALUE;
+      long longValue = -MAX_VALUE + random.nextLong() * 2 * MAX_VALUE;
+      float floatValue = -MAX_VALUE + random.nextFloat() * 2 * MAX_VALUE;
+      double doubleValue = -MAX_VALUE + random.nextDouble() * 2 * MAX_VALUE;
+
+      _intValues[i] = intValue;
+      _longValues[i] = longValue;
+      _floatValues[i] = floatValue;
+      _doubleValues[i] = doubleValue;
+
+      record.putValue(INT_COLUMN, _intValues[i]);
+      record.putValue(LONG_COLUMN, _longValues[i]);
+      record.putValue(FLOAT_COLUMN, _floatValues[i]);
+      record.putValue(DOUBLE_COLUMN, _doubleValues[i]);
+      record.putValue(GROUP_BY_COLUMN, Math.floor(i / (NUM_RECORDS / NUM_GROUPS)));
+      records.add(record);
+    }
+
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(TABLE_CONFIG, SCHEMA);
+    segmentGeneratorConfig.setTableName(RAW_TABLE_NAME);
+    segmentGeneratorConfig.setSegmentName(SEGMENT_NAME);
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getPath());
+
+    SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig, new GenericRowRecordReader(records));
+    driver.build();
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, SEGMENT_NAME), ReadMode.mmap);
+    _indexSegment = immutableSegment;
+    _indexSegments = Arrays.asList(immutableSegment, immutableSegment);
+  }
+
+  @Test
+  public void testVarianceAggregationOnly() {
+    // Compute the expected values
+    Variance[] expectedVariances = new Variance[8];
+    for (int i = 0; i < 8; i++) {
+      if (i < 4) {
+        expectedVariances[i] = new Variance(false);
+      } else {
+        expectedVariances[i] = new Variance(true);
+      }
+    }
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      expectedVariances[0].increment(_intValues[i]);
+      expectedVariances[1].increment(_longValues[i]);
+      expectedVariances[2].increment(_floatValues[i]);
+      expectedVariances[3].increment(_doubleValues[i]);
+      expectedVariances[4].increment(_intValues[i]);
+      expectedVariances[5].increment(_longValues[i]);
+      expectedVariances[6].increment(_floatValues[i]);
+      expectedVariances[7].increment(_doubleValues[i]);
+    }
+    double expectedIntSum = Arrays.stream(_intValues).asDoubleStream().sum();
+    double expectedLongSum = Arrays.stream(_longValues).asDoubleStream().sum();
+    double expectedFloatSum = 0.0;
+    for (int i = 0; i < _floatValues.length; i++) {
+      expectedFloatSum += _floatValues[i];
+    }
+    double expectedDoubleSum = Arrays.stream(_doubleValues).sum();
+
+    // Compute the query
+    String query = "SELECT VAR_POP(intColumn), VAR_POP(longColumn), VAR_POP(floatColumn), VAR_POP(doubleColumn),"
+        + "VAR_SAMP(intColumn), VAR_SAMP(longColumn), VAR_SAMP(floatColumn), VAR_SAMP(doubleColumn) FROM testTable";
+    AggregationOperator aggregationOperator = getOperator(query);
+    AggregationResultsBlock resultsBlock = aggregationOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(aggregationOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 4, NUM_RECORDS);
+    List<Object> aggregationResult = resultsBlock.getResults();
+
+    // Validate the aggregation results
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(0), NUM_RECORDS, expectedIntSum,
+        expectedVariances[0].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(1), NUM_RECORDS, expectedLongSum,
+        expectedVariances[1].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(2), NUM_RECORDS, expectedFloatSum,
+        expectedVariances[2].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(3), NUM_RECORDS, expectedDoubleSum,
+        expectedVariances[3].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(4), NUM_RECORDS, expectedIntSum,
+        expectedVariances[4].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(5), NUM_RECORDS, expectedLongSum,
+        expectedVariances[5].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(6), NUM_RECORDS, expectedFloatSum,
+        expectedVariances[6].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(7), NUM_RECORDS, expectedDoubleSum,
+        expectedVariances[7].getResult(), true);
+
+    // Update the expected result by 3 more times (broker query will compute 4 identical segments)
+    for (int i = 0; i < NUM_RECORDS * 3; i++) {
+      int pos = i % NUM_RECORDS;
+      expectedVariances[0].increment(_intValues[pos]);
+      expectedVariances[1].increment(_longValues[pos]);
+      expectedVariances[2].increment(_floatValues[pos]);
+      expectedVariances[3].increment(_doubleValues[pos]);
+      expectedVariances[4].increment(_intValues[pos]);
+      expectedVariances[5].increment(_longValues[pos]);
+      expectedVariances[6].increment(_floatValues[pos]);
+      expectedVariances[7].increment(_doubleValues[pos]);
+    }
+
+    // Validate the response
+    BrokerResponseNative brokerResponse = getBrokerResponse(query);
+    brokerResponse.getResultTable();
+    Object[] results = brokerResponse.getResultTable().getRows().get(0);
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[0], expectedVariances[0].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[1], expectedVariances[1].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[2], expectedVariances[2].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[3], expectedVariances[3].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[4], expectedVariances[4].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[5], expectedVariances[5].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[6], expectedVariances[6].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[7], expectedVariances[7].getResult(), RELATIVE_EPSILON));
+
+    // Validate the response for a query with a filter
+    query = "SELECT VAR_POP(intColumn) from testTable" + getFilter();
+    brokerResponse = getBrokerResponse(query);
+    brokerResponse.getResultTable();
+    results = brokerResponse.getResultTable().getRows().get(0);
+    Variance filterExpectedVariance = new Variance(false);
+    for (int i = 0; i < NUM_RECORDS / 2; i++) {
+      filterExpectedVariance.increment(_intValues[i]);
+    }
+    assertTrue(Precision.equalsWithRelativeTolerance((double) results[0], filterExpectedVariance.getResult(),
+        RELATIVE_EPSILON));
+  }
+
+  @Test
+  public void testVarianceAggregationGroupBy() {
+    // Compute expected group results
+    Variance[] expectedGroupByResult = new Variance[NUM_GROUPS];
+    double[] expectedSum = new double[NUM_GROUPS];
+
+    for (int i = 0; i < NUM_GROUPS; i++) {
+      expectedGroupByResult[i] = new Variance(false);
+    }
+    for (int j = 0; j < NUM_RECORDS; j++) {
+      int pos = j / (NUM_RECORDS / NUM_GROUPS);
+      expectedGroupByResult[pos].increment(_intValues[j]);
+      expectedSum[pos] += _intValues[j];
+    }
+
+    String query = "SELECT VAR_POP(intColumn) FROM testTable GROUP BY groupByColumn ORDER BY groupByColumn";
+    GroupByOperator groupByOperator = getOperator(query);
+    GroupByResultsBlock resultsBlock = groupByOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(groupByOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 2, NUM_RECORDS);
+    AggregationGroupByResult aggregationGroupByResult = resultsBlock.getAggregationGroupByResult();
+    assertNotNull(aggregationGroupByResult);
+    for (int i = 0; i < NUM_GROUPS; i++) {
+
+      VarianceTuple actualVarianceTuple = (VarianceTuple) aggregationGroupByResult.getResultForGroupId(0, i);
+      checkWithPrecisionForVariance(actualVarianceTuple, NUM_RECORDS / NUM_GROUPS, expectedSum[i],
+          expectedGroupByResult[i].getResult(), false);
+    }
+  }
+
+  @Test
+  public void testStandardDeviationAggregationOnly() {
+    // Compute the expected values
+    StandardDeviation[] expectedStdDevs = new StandardDeviation[8];
+    for (int i = 0; i < 8; i++) {
+      if (i < 4) {
+        expectedStdDevs[i] = new StandardDeviation(false);
+      } else {
+        expectedStdDevs[i] = new StandardDeviation(true);
+      }
+    }
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      expectedStdDevs[0].increment(_intValues[i]);
+      expectedStdDevs[1].increment(_longValues[i]);
+      expectedStdDevs[2].increment(_floatValues[i]);
+      expectedStdDevs[3].increment(_doubleValues[i]);
+      expectedStdDevs[4].increment(_intValues[i]);
+      expectedStdDevs[5].increment(_longValues[i]);
+      expectedStdDevs[6].increment(_floatValues[i]);
+      expectedStdDevs[7].increment(_doubleValues[i]);
+    }
+
+    double expectedIntSum = Arrays.stream(_intValues).asDoubleStream().sum();
+    double expectedLongSum = Arrays.stream(_longValues).asDoubleStream().sum();
+    double expectedFloatSum = 0.0;
+    for (int i = 0; i < _floatValues.length; i++) {
+      expectedFloatSum += _floatValues[i];
+    }
+    double expectedDoubleSum = Arrays.stream(_doubleValues).sum();
+
+    // Compute the query
+    String query =
+        "SELECT STDDEV_POP(intColumn), STDDEV_POP(longColumn), STDDEV_POP(floatColumn), STDDEV_POP(doubleColumn),"
+            + "STDDEV_SAMP(intColumn), STDDEV_SAMP(longColumn), STDDEV_SAMP(floatColumn), STDDEV_SAMP(doubleColumn) "
+            + "FROM testTable";
+    AggregationOperator aggregationOperator = getOperator(query);
+    AggregationResultsBlock resultsBlock = aggregationOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(aggregationOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 4, NUM_RECORDS);
+    List<Object> aggregationResult = resultsBlock.getResults();
+
+    // Validate the aggregation results
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(0), NUM_RECORDS, expectedIntSum,
+        expectedStdDevs[0].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(1), NUM_RECORDS, expectedLongSum,
+        expectedStdDevs[1].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(2), NUM_RECORDS, expectedFloatSum,
+        expectedStdDevs[2].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(3), NUM_RECORDS, expectedDoubleSum,
+        expectedStdDevs[3].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(4), NUM_RECORDS, expectedIntSum,
+        expectedStdDevs[4].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(5), NUM_RECORDS, expectedLongSum,
+        expectedStdDevs[5].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(6), NUM_RECORDS, expectedFloatSum,
+        expectedStdDevs[6].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(7), NUM_RECORDS, expectedDoubleSum,
+        expectedStdDevs[7].getResult(), true);
+
+    // Update the expected result by 3 more times (broker query will compute 4 identical segments)

Review Comment:
   The variance on 4 identical segments is the same as variance on one of each segment. Should probably to consider using `getDistinctInstances` in `BaseQueriesTest`



-- 
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] walterddr commented on a diff in pull request #9910: Add Variance and Standard Deviation Aggregation Functions

Posted by GitBox <gi...@apache.org>.
walterddr commented on code in PR #9910:
URL: https://github.com/apache/pinot/pull/9910#discussion_r1041640617


##########
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/VarianceAggregationFunction.java:
##########
@@ -0,0 +1,204 @@
+/**
+ * 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.Map;
+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.VarianceTuple;
+import org.apache.pinot.segment.spi.AggregationFunctionType;
+
+
+/**
+ * Aggregation function which computes Variance and Standard Deviation
+ *
+ * The algorithm to compute variance is based on "Updating Formulae and a Pairwise Algorithm for Computing
+ * Sample Variances" by Chan et al. Please refer to the "Parallel Algorithm" section from the following wiki:
+ * - https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Parallel_algorithm
+ */
+public class VarianceAggregationFunction extends BaseSingleInputAggregationFunction<VarianceTuple, Double> {

Review Comment:
   majority of this code is shared with CoVariance agg. can we abstract out some comment utilities?



-- 
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] snleee merged pull request #9910: Add Variance and Standard Deviation Aggregation Functions

Posted by GitBox <gi...@apache.org>.
snleee merged PR #9910:
URL: https://github.com/apache/pinot/pull/9910


-- 
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] jasperjiaguo commented on a diff in pull request #9910: Add Variance and Standard Deviation Aggregation Functions

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on code in PR #9910:
URL: https://github.com/apache/pinot/pull/9910#discussion_r1043888203


##########
pinot-core/src/test/java/org/apache/pinot/queries/VarianceQueriesTest.java:
##########
@@ -0,0 +1,446 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.math3.stat.descriptive.moment.StandardDeviation;
+import org.apache.commons.math3.stat.descriptive.moment.Variance;
+import org.apache.commons.math3.util.Precision;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.core.operator.blocks.results.AggregationResultsBlock;
+import org.apache.pinot.core.operator.blocks.results.GroupByResultsBlock;
+import org.apache.pinot.core.operator.query.AggregationOperator;
+import org.apache.pinot.core.operator.query.GroupByOperator;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByResult;
+import org.apache.pinot.segment.local.customobject.VarianceTuple;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertTrue;
+
+
+public class VarianceQueriesTest extends BaseQueriesTest {
+
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "VarianceQueriesTest");
+
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+
+  private static final int NUM_RECORDS = 2000;
+  private static final int NUM_GROUPS = 10;
+  private static final int MAX_VALUE = 500;
+  private static final double RELATIVE_EPSILON = 0.0001;
+  private static final double DELTA = 0.0001;
+
+  private static final String INT_COLUMN = "intColumn";
+  private static final String LONG_COLUMN = "longColumn";
+  private static final String FLOAT_COLUMN = "floatColumn";
+  private static final String DOUBLE_COLUMN = "doubleColumn";
+  private static final String GROUP_BY_COLUMN = "groupByColumn";
+
+  private static final Schema SCHEMA =
+      new Schema.SchemaBuilder().addSingleValueDimension(INT_COLUMN, FieldSpec.DataType.INT)
+          .addSingleValueDimension(LONG_COLUMN, FieldSpec.DataType.LONG)
+          .addSingleValueDimension(FLOAT_COLUMN, FieldSpec.DataType.FLOAT)
+          .addSingleValueDimension(DOUBLE_COLUMN, FieldSpec.DataType.DOUBLE)
+          .addSingleValueDimension(GROUP_BY_COLUMN, FieldSpec.DataType.DOUBLE).build();
+
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  int[] _intValues = new int[NUM_RECORDS];
+  long[] _longValues = new long[NUM_RECORDS];
+  float[] _floatValues = new float[NUM_RECORDS];
+  double[] _doubleValues = new double[NUM_RECORDS];
+
+  @Override
+  protected String getFilter() {
+    // filter out half of the rows based on group id
+    return " WHERE groupByColumn < " + (NUM_GROUPS / 2);
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+    Random random = new Random();
+    List<GenericRow> records = new ArrayList<>(NUM_RECORDS);
+
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      GenericRow record = new GenericRow();
+      int intValue = -MAX_VALUE + random.nextInt() * 2 * MAX_VALUE;
+      long longValue = -MAX_VALUE + random.nextLong() * 2 * MAX_VALUE;
+      float floatValue = -MAX_VALUE + random.nextFloat() * 2 * MAX_VALUE;
+      double doubleValue = -MAX_VALUE + random.nextDouble() * 2 * MAX_VALUE;
+
+      _intValues[i] = intValue;
+      _longValues[i] = longValue;
+      _floatValues[i] = floatValue;
+      _doubleValues[i] = doubleValue;
+
+      record.putValue(INT_COLUMN, _intValues[i]);
+      record.putValue(LONG_COLUMN, _longValues[i]);
+      record.putValue(FLOAT_COLUMN, _floatValues[i]);
+      record.putValue(DOUBLE_COLUMN, _doubleValues[i]);
+      record.putValue(GROUP_BY_COLUMN, Math.floor(i / (NUM_RECORDS / NUM_GROUPS)));
+      records.add(record);
+    }
+
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(TABLE_CONFIG, SCHEMA);
+    segmentGeneratorConfig.setTableName(RAW_TABLE_NAME);
+    segmentGeneratorConfig.setSegmentName(SEGMENT_NAME);
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getPath());
+
+    SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig, new GenericRowRecordReader(records));
+    driver.build();
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, SEGMENT_NAME), ReadMode.mmap);
+    _indexSegment = immutableSegment;
+    _indexSegments = Arrays.asList(immutableSegment, immutableSegment);
+  }
+
+  @Test
+  public void testVarianceAggregationOnly() {
+    // Compute the expected values
+    Variance[] expectedVariances = new Variance[8];
+    for (int i = 0; i < 8; i++) {
+      if (i < 4) {
+        expectedVariances[i] = new Variance(false);
+      } else {
+        expectedVariances[i] = new Variance(true);
+      }
+    }
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      expectedVariances[0].increment(_intValues[i]);
+      expectedVariances[1].increment(_longValues[i]);
+      expectedVariances[2].increment(_floatValues[i]);
+      expectedVariances[3].increment(_doubleValues[i]);
+      expectedVariances[4].increment(_intValues[i]);
+      expectedVariances[5].increment(_longValues[i]);
+      expectedVariances[6].increment(_floatValues[i]);
+      expectedVariances[7].increment(_doubleValues[i]);
+    }
+    double expectedIntSum = Arrays.stream(_intValues).asDoubleStream().sum();
+    double expectedLongSum = Arrays.stream(_longValues).asDoubleStream().sum();
+    double expectedFloatSum = 0.0;
+    for (int i = 0; i < _floatValues.length; i++) {
+      expectedFloatSum += _floatValues[i];
+    }
+    double expectedDoubleSum = Arrays.stream(_doubleValues).sum();
+
+    // Compute the query
+    String query = "SELECT VAR_POP(intColumn), VAR_POP(longColumn), VAR_POP(floatColumn), VAR_POP(doubleColumn),"
+        + "VAR_SAMP(intColumn), VAR_SAMP(longColumn), VAR_SAMP(floatColumn), VAR_SAMP(doubleColumn) FROM testTable";
+    AggregationOperator aggregationOperator = getOperator(query);
+    AggregationResultsBlock resultsBlock = aggregationOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(aggregationOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 4, NUM_RECORDS);
+    List<Object> aggregationResult = resultsBlock.getResults();
+
+    // Validate the aggregation results
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(0), NUM_RECORDS, expectedIntSum,
+        expectedVariances[0].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(1), NUM_RECORDS, expectedLongSum,
+        expectedVariances[1].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(2), NUM_RECORDS, expectedFloatSum,
+        expectedVariances[2].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(3), NUM_RECORDS, expectedDoubleSum,
+        expectedVariances[3].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(4), NUM_RECORDS, expectedIntSum,
+        expectedVariances[4].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(5), NUM_RECORDS, expectedLongSum,
+        expectedVariances[5].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(6), NUM_RECORDS, expectedFloatSum,
+        expectedVariances[6].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(7), NUM_RECORDS, expectedDoubleSum,
+        expectedVariances[7].getResult(), true);
+
+    // Update the expected result by 3 more times (broker query will compute 4 identical segments)
+    for (int i = 0; i < NUM_RECORDS * 3; i++) {
+      int pos = i % NUM_RECORDS;
+      expectedVariances[0].increment(_intValues[pos]);
+      expectedVariances[1].increment(_longValues[pos]);
+      expectedVariances[2].increment(_floatValues[pos]);
+      expectedVariances[3].increment(_doubleValues[pos]);
+      expectedVariances[4].increment(_intValues[pos]);
+      expectedVariances[5].increment(_longValues[pos]);
+      expectedVariances[6].increment(_floatValues[pos]);
+      expectedVariances[7].increment(_doubleValues[pos]);
+    }
+
+    // Validate the response
+    BrokerResponseNative brokerResponse = getBrokerResponse(query);
+    brokerResponse.getResultTable();
+    Object[] results = brokerResponse.getResultTable().getRows().get(0);
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[0], expectedVariances[0].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[1], expectedVariances[1].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[2], expectedVariances[2].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[3], expectedVariances[3].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[4], expectedVariances[4].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[5], expectedVariances[5].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[6], expectedVariances[6].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[7], expectedVariances[7].getResult(), RELATIVE_EPSILON));
+
+    // Validate the response for a query with a filter
+    query = "SELECT VAR_POP(intColumn) from testTable" + getFilter();
+    brokerResponse = getBrokerResponse(query);
+    brokerResponse.getResultTable();
+    results = brokerResponse.getResultTable().getRows().get(0);
+    Variance filterExpectedVariance = new Variance(false);
+    for (int i = 0; i < NUM_RECORDS / 2; i++) {
+      filterExpectedVariance.increment(_intValues[i]);
+    }
+    assertTrue(Precision.equalsWithRelativeTolerance((double) results[0], filterExpectedVariance.getResult(),
+        RELATIVE_EPSILON));
+  }
+
+  @Test
+  public void testVarianceAggregationGroupBy() {
+    // Compute expected group results
+    Variance[] expectedGroupByResult = new Variance[NUM_GROUPS];
+    double[] expectedSum = new double[NUM_GROUPS];
+
+    for (int i = 0; i < NUM_GROUPS; i++) {
+      expectedGroupByResult[i] = new Variance(false);
+    }
+    for (int j = 0; j < NUM_RECORDS; j++) {
+      int pos = j / (NUM_RECORDS / NUM_GROUPS);
+      expectedGroupByResult[pos].increment(_intValues[j]);
+      expectedSum[pos] += _intValues[j];
+    }
+
+    String query = "SELECT VAR_POP(intColumn) FROM testTable GROUP BY groupByColumn ORDER BY groupByColumn";
+    GroupByOperator groupByOperator = getOperator(query);
+    GroupByResultsBlock resultsBlock = groupByOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(groupByOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 2, NUM_RECORDS);
+    AggregationGroupByResult aggregationGroupByResult = resultsBlock.getAggregationGroupByResult();
+    assertNotNull(aggregationGroupByResult);
+    for (int i = 0; i < NUM_GROUPS; i++) {
+
+      VarianceTuple actualVarianceTuple = (VarianceTuple) aggregationGroupByResult.getResultForGroupId(0, i);
+      checkWithPrecisionForVariance(actualVarianceTuple, NUM_RECORDS / NUM_GROUPS, expectedSum[i],
+          expectedGroupByResult[i].getResult(), false);
+    }
+  }
+
+  @Test
+  public void testStandardDeviationAggregationOnly() {
+    // Compute the expected values
+    StandardDeviation[] expectedStdDevs = new StandardDeviation[8];
+    for (int i = 0; i < 8; i++) {
+      if (i < 4) {
+        expectedStdDevs[i] = new StandardDeviation(false);
+      } else {
+        expectedStdDevs[i] = new StandardDeviation(true);
+      }
+    }
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      expectedStdDevs[0].increment(_intValues[i]);
+      expectedStdDevs[1].increment(_longValues[i]);
+      expectedStdDevs[2].increment(_floatValues[i]);
+      expectedStdDevs[3].increment(_doubleValues[i]);
+      expectedStdDevs[4].increment(_intValues[i]);
+      expectedStdDevs[5].increment(_longValues[i]);
+      expectedStdDevs[6].increment(_floatValues[i]);
+      expectedStdDevs[7].increment(_doubleValues[i]);
+    }
+
+    double expectedIntSum = Arrays.stream(_intValues).asDoubleStream().sum();
+    double expectedLongSum = Arrays.stream(_longValues).asDoubleStream().sum();
+    double expectedFloatSum = 0.0;
+    for (int i = 0; i < _floatValues.length; i++) {
+      expectedFloatSum += _floatValues[i];
+    }
+    double expectedDoubleSum = Arrays.stream(_doubleValues).sum();
+
+    // Compute the query
+    String query =
+        "SELECT STDDEV_POP(intColumn), STDDEV_POP(longColumn), STDDEV_POP(floatColumn), STDDEV_POP(doubleColumn),"
+            + "STDDEV_SAMP(intColumn), STDDEV_SAMP(longColumn), STDDEV_SAMP(floatColumn), STDDEV_SAMP(doubleColumn) "
+            + "FROM testTable";
+    AggregationOperator aggregationOperator = getOperator(query);
+    AggregationResultsBlock resultsBlock = aggregationOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(aggregationOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 4, NUM_RECORDS);
+    List<Object> aggregationResult = resultsBlock.getResults();
+
+    // Validate the aggregation results
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(0), NUM_RECORDS, expectedIntSum,
+        expectedStdDevs[0].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(1), NUM_RECORDS, expectedLongSum,
+        expectedStdDevs[1].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(2), NUM_RECORDS, expectedFloatSum,
+        expectedStdDevs[2].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(3), NUM_RECORDS, expectedDoubleSum,
+        expectedStdDevs[3].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(4), NUM_RECORDS, expectedIntSum,
+        expectedStdDevs[4].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(5), NUM_RECORDS, expectedLongSum,
+        expectedStdDevs[5].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(6), NUM_RECORDS, expectedFloatSum,
+        expectedStdDevs[6].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(7), NUM_RECORDS, expectedDoubleSum,
+        expectedStdDevs[7].getResult(), true);
+
+    // Update the expected result by 3 more times (broker query will compute 4 identical segments)

Review Comment:
   LGTM, thanks!



-- 
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] jasperjiaguo commented on a diff in pull request #9910: Add Variance and Standard Deviation Aggregation Functions

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on code in PR #9910:
URL: https://github.com/apache/pinot/pull/9910#discussion_r1041477398


##########
pinot-core/src/main/java/org/apache/pinot/core/common/ObjectSerDeUtils.java:
##########
@@ -123,7 +124,9 @@ public enum ObjectType {
     FloatLongPair(29),
     DoubleLongPair(30),
     StringLongPair(31),
-    CovarianceTuple(32);
+    CovarianceTuple(32),
+

Review Comment:
   (nit) remove blank line?



##########
pinot-core/src/test/java/org/apache/pinot/queries/VarianceQueriesTest.java:
##########
@@ -0,0 +1,446 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.math3.stat.descriptive.moment.StandardDeviation;
+import org.apache.commons.math3.stat.descriptive.moment.Variance;
+import org.apache.commons.math3.util.Precision;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.core.operator.blocks.results.AggregationResultsBlock;
+import org.apache.pinot.core.operator.blocks.results.GroupByResultsBlock;
+import org.apache.pinot.core.operator.query.AggregationOperator;
+import org.apache.pinot.core.operator.query.GroupByOperator;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByResult;
+import org.apache.pinot.segment.local.customobject.VarianceTuple;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertTrue;
+
+
+public class VarianceQueriesTest extends BaseQueriesTest {
+
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "VarianceQueriesTest");
+
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+
+  private static final int NUM_RECORDS = 2000;
+  private static final int NUM_GROUPS = 10;
+  private static final int MAX_VALUE = 500;
+  private static final double RELATIVE_EPSILON = 0.0001;
+  private static final double DELTA = 0.0001;
+
+  private static final String INT_COLUMN = "intColumn";
+  private static final String LONG_COLUMN = "longColumn";
+  private static final String FLOAT_COLUMN = "floatColumn";
+  private static final String DOUBLE_COLUMN = "doubleColumn";
+  private static final String GROUP_BY_COLUMN = "groupByColumn";
+
+  private static final Schema SCHEMA =
+      new Schema.SchemaBuilder().addSingleValueDimension(INT_COLUMN, FieldSpec.DataType.INT)
+          .addSingleValueDimension(LONG_COLUMN, FieldSpec.DataType.LONG)
+          .addSingleValueDimension(FLOAT_COLUMN, FieldSpec.DataType.FLOAT)
+          .addSingleValueDimension(DOUBLE_COLUMN, FieldSpec.DataType.DOUBLE)
+          .addSingleValueDimension(GROUP_BY_COLUMN, FieldSpec.DataType.DOUBLE).build();
+
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  int[] _intValues = new int[NUM_RECORDS];
+  long[] _longValues = new long[NUM_RECORDS];
+  float[] _floatValues = new float[NUM_RECORDS];
+  double[] _doubleValues = new double[NUM_RECORDS];
+
+  @Override
+  protected String getFilter() {
+    // filter out half of the rows based on group id
+    return " WHERE groupByColumn < " + (NUM_GROUPS / 2);
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+    Random random = new Random();
+    List<GenericRow> records = new ArrayList<>(NUM_RECORDS);
+
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      GenericRow record = new GenericRow();
+      int intValue = -MAX_VALUE + random.nextInt() * 2 * MAX_VALUE;
+      long longValue = -MAX_VALUE + random.nextLong() * 2 * MAX_VALUE;
+      float floatValue = -MAX_VALUE + random.nextFloat() * 2 * MAX_VALUE;
+      double doubleValue = -MAX_VALUE + random.nextDouble() * 2 * MAX_VALUE;
+
+      _intValues[i] = intValue;
+      _longValues[i] = longValue;
+      _floatValues[i] = floatValue;
+      _doubleValues[i] = doubleValue;
+
+      record.putValue(INT_COLUMN, _intValues[i]);
+      record.putValue(LONG_COLUMN, _longValues[i]);
+      record.putValue(FLOAT_COLUMN, _floatValues[i]);
+      record.putValue(DOUBLE_COLUMN, _doubleValues[i]);
+      record.putValue(GROUP_BY_COLUMN, Math.floor(i / (NUM_RECORDS / NUM_GROUPS)));
+      records.add(record);
+    }
+
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(TABLE_CONFIG, SCHEMA);
+    segmentGeneratorConfig.setTableName(RAW_TABLE_NAME);
+    segmentGeneratorConfig.setSegmentName(SEGMENT_NAME);
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getPath());
+
+    SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig, new GenericRowRecordReader(records));
+    driver.build();
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, SEGMENT_NAME), ReadMode.mmap);
+    _indexSegment = immutableSegment;
+    _indexSegments = Arrays.asList(immutableSegment, immutableSegment);
+  }
+
+  @Test
+  public void testVarianceAggregationOnly() {
+    // Compute the expected values
+    Variance[] expectedVariances = new Variance[8];
+    for (int i = 0; i < 8; i++) {
+      if (i < 4) {
+        expectedVariances[i] = new Variance(false);
+      } else {
+        expectedVariances[i] = new Variance(true);
+      }
+    }
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      expectedVariances[0].increment(_intValues[i]);
+      expectedVariances[1].increment(_longValues[i]);
+      expectedVariances[2].increment(_floatValues[i]);
+      expectedVariances[3].increment(_doubleValues[i]);
+      expectedVariances[4].increment(_intValues[i]);
+      expectedVariances[5].increment(_longValues[i]);
+      expectedVariances[6].increment(_floatValues[i]);
+      expectedVariances[7].increment(_doubleValues[i]);
+    }
+    double expectedIntSum = Arrays.stream(_intValues).asDoubleStream().sum();
+    double expectedLongSum = Arrays.stream(_longValues).asDoubleStream().sum();
+    double expectedFloatSum = 0.0;
+    for (int i = 0; i < _floatValues.length; i++) {
+      expectedFloatSum += _floatValues[i];
+    }
+    double expectedDoubleSum = Arrays.stream(_doubleValues).sum();
+
+    // Compute the query
+    String query = "SELECT VAR_POP(intColumn), VAR_POP(longColumn), VAR_POP(floatColumn), VAR_POP(doubleColumn),"
+        + "VAR_SAMP(intColumn), VAR_SAMP(longColumn), VAR_SAMP(floatColumn), VAR_SAMP(doubleColumn) FROM testTable";
+    AggregationOperator aggregationOperator = getOperator(query);
+    AggregationResultsBlock resultsBlock = aggregationOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(aggregationOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 4, NUM_RECORDS);
+    List<Object> aggregationResult = resultsBlock.getResults();
+
+    // Validate the aggregation results
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(0), NUM_RECORDS, expectedIntSum,
+        expectedVariances[0].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(1), NUM_RECORDS, expectedLongSum,
+        expectedVariances[1].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(2), NUM_RECORDS, expectedFloatSum,
+        expectedVariances[2].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(3), NUM_RECORDS, expectedDoubleSum,
+        expectedVariances[3].getResult(), false);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(4), NUM_RECORDS, expectedIntSum,
+        expectedVariances[4].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(5), NUM_RECORDS, expectedLongSum,
+        expectedVariances[5].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(6), NUM_RECORDS, expectedFloatSum,
+        expectedVariances[6].getResult(), true);
+    checkWithPrecisionForVariance((VarianceTuple) aggregationResult.get(7), NUM_RECORDS, expectedDoubleSum,
+        expectedVariances[7].getResult(), true);
+
+    // Update the expected result by 3 more times (broker query will compute 4 identical segments)
+    for (int i = 0; i < NUM_RECORDS * 3; i++) {
+      int pos = i % NUM_RECORDS;
+      expectedVariances[0].increment(_intValues[pos]);
+      expectedVariances[1].increment(_longValues[pos]);
+      expectedVariances[2].increment(_floatValues[pos]);
+      expectedVariances[3].increment(_doubleValues[pos]);
+      expectedVariances[4].increment(_intValues[pos]);
+      expectedVariances[5].increment(_longValues[pos]);
+      expectedVariances[6].increment(_floatValues[pos]);
+      expectedVariances[7].increment(_doubleValues[pos]);
+    }
+
+    // Validate the response
+    BrokerResponseNative brokerResponse = getBrokerResponse(query);
+    brokerResponse.getResultTable();
+    Object[] results = brokerResponse.getResultTable().getRows().get(0);
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[0], expectedVariances[0].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[1], expectedVariances[1].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[2], expectedVariances[2].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[3], expectedVariances[3].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[4], expectedVariances[4].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[5], expectedVariances[5].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[6], expectedVariances[6].getResult(), RELATIVE_EPSILON));
+    assertTrue(
+        Precision.equalsWithRelativeTolerance((double) results[7], expectedVariances[7].getResult(), RELATIVE_EPSILON));
+
+    // Validate the response for a query with a filter
+    query = "SELECT VAR_POP(intColumn) from testTable" + getFilter();
+    brokerResponse = getBrokerResponse(query);
+    brokerResponse.getResultTable();
+    results = brokerResponse.getResultTable().getRows().get(0);
+    Variance filterExpectedVariance = new Variance(false);
+    for (int i = 0; i < NUM_RECORDS / 2; i++) {
+      filterExpectedVariance.increment(_intValues[i]);
+    }
+    assertTrue(Precision.equalsWithRelativeTolerance((double) results[0], filterExpectedVariance.getResult(),
+        RELATIVE_EPSILON));
+  }
+
+  @Test
+  public void testVarianceAggregationGroupBy() {
+    // Compute expected group results
+    Variance[] expectedGroupByResult = new Variance[NUM_GROUPS];
+    double[] expectedSum = new double[NUM_GROUPS];
+
+    for (int i = 0; i < NUM_GROUPS; i++) {
+      expectedGroupByResult[i] = new Variance(false);
+    }
+    for (int j = 0; j < NUM_RECORDS; j++) {
+      int pos = j / (NUM_RECORDS / NUM_GROUPS);
+      expectedGroupByResult[pos].increment(_intValues[j]);
+      expectedSum[pos] += _intValues[j];
+    }
+
+    String query = "SELECT VAR_POP(intColumn) FROM testTable GROUP BY groupByColumn ORDER BY groupByColumn";
+    GroupByOperator groupByOperator = getOperator(query);
+    GroupByResultsBlock resultsBlock = groupByOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(groupByOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 2, NUM_RECORDS);
+    AggregationGroupByResult aggregationGroupByResult = resultsBlock.getAggregationGroupByResult();
+    assertNotNull(aggregationGroupByResult);
+    for (int i = 0; i < NUM_GROUPS; i++) {
+
+      VarianceTuple actualVarianceTuple = (VarianceTuple) aggregationGroupByResult.getResultForGroupId(0, i);
+      checkWithPrecisionForVariance(actualVarianceTuple, NUM_RECORDS / NUM_GROUPS, expectedSum[i],
+          expectedGroupByResult[i].getResult(), false);
+    }
+  }
+
+  @Test
+  public void testStandardDeviationAggregationOnly() {
+    // Compute the expected values
+    StandardDeviation[] expectedStdDevs = new StandardDeviation[8];
+    for (int i = 0; i < 8; i++) {
+      if (i < 4) {
+        expectedStdDevs[i] = new StandardDeviation(false);
+      } else {
+        expectedStdDevs[i] = new StandardDeviation(true);
+      }
+    }
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      expectedStdDevs[0].increment(_intValues[i]);
+      expectedStdDevs[1].increment(_longValues[i]);
+      expectedStdDevs[2].increment(_floatValues[i]);
+      expectedStdDevs[3].increment(_doubleValues[i]);
+      expectedStdDevs[4].increment(_intValues[i]);
+      expectedStdDevs[5].increment(_longValues[i]);
+      expectedStdDevs[6].increment(_floatValues[i]);
+      expectedStdDevs[7].increment(_doubleValues[i]);
+    }
+
+    double expectedIntSum = Arrays.stream(_intValues).asDoubleStream().sum();
+    double expectedLongSum = Arrays.stream(_longValues).asDoubleStream().sum();
+    double expectedFloatSum = 0.0;
+    for (int i = 0; i < _floatValues.length; i++) {
+      expectedFloatSum += _floatValues[i];
+    }
+    double expectedDoubleSum = Arrays.stream(_doubleValues).sum();
+
+    // Compute the query
+    String query =
+        "SELECT STDDEV_POP(intColumn), STDDEV_POP(longColumn), STDDEV_POP(floatColumn), STDDEV_POP(doubleColumn),"
+            + "STDDEV_SAMP(intColumn), STDDEV_SAMP(longColumn), STDDEV_SAMP(floatColumn), STDDEV_SAMP(doubleColumn) "
+            + "FROM testTable";
+    AggregationOperator aggregationOperator = getOperator(query);
+    AggregationResultsBlock resultsBlock = aggregationOperator.nextBlock();
+    QueriesTestUtils.testInnerSegmentExecutionStatistics(aggregationOperator.getExecutionStatistics(), NUM_RECORDS, 0,
+        NUM_RECORDS * 4, NUM_RECORDS);
+    List<Object> aggregationResult = resultsBlock.getResults();
+
+    // Validate the aggregation results
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(0), NUM_RECORDS, expectedIntSum,
+        expectedStdDevs[0].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(1), NUM_RECORDS, expectedLongSum,
+        expectedStdDevs[1].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(2), NUM_RECORDS, expectedFloatSum,
+        expectedStdDevs[2].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(3), NUM_RECORDS, expectedDoubleSum,
+        expectedStdDevs[3].getResult(), false);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(4), NUM_RECORDS, expectedIntSum,
+        expectedStdDevs[4].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(5), NUM_RECORDS, expectedLongSum,
+        expectedStdDevs[5].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(6), NUM_RECORDS, expectedFloatSum,
+        expectedStdDevs[6].getResult(), true);
+    checkWithPrecisionForStandardDeviation((VarianceTuple) aggregationResult.get(7), NUM_RECORDS, expectedDoubleSum,
+        expectedStdDevs[7].getResult(), true);
+
+    // Update the expected result by 3 more times (broker query will compute 4 identical segments)

Review Comment:
   The variance on 4 identical segments is the same as variance on one of these segments. Should probably to consider using `getDistinctInstances` in `BaseQueriesTest`



-- 
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] walterddr commented on a diff in pull request #9910: Add Variance and Standard Deviation Aggregation Functions

Posted by GitBox <gi...@apache.org>.
walterddr commented on code in PR #9910:
URL: https://github.com/apache/pinot/pull/9910#discussion_r1041639288


##########
pinot-core/src/main/java/org/apache/pinot/core/common/ObjectSerDeUtils.java:
##########
@@ -123,7 +124,9 @@ public enum ObjectType {
     FloatLongPair(29),
     DoubleLongPair(30),
     StringLongPair(31),
-    CovarianceTuple(32);
+    CovarianceTuple(32),

Review Comment:
   i was wondering if we can simply reuse CovarianceTuple and logic here. 
   basically Variance(x) = Covarance(x, x)
   
   



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