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 2021/11/17 04:34:02 UTC

[GitHub] [pinot] weixiangsun opened a new pull request #7781: Add Post-Aggregation Gapfilling functionality.

weixiangsun opened a new pull request #7781:
URL: https://github.com/apache/pinot/pull/7781


   ## Description
   DateConvertor function can create the time bucket. Group By time bucket will create the result set where some data is missing for some time bucket. This change is used to gap fill the data for the time bucket with missing data.
   ## Upgrade Notes
   Does this PR prevent a zero down-time upgrade? (Assume upgrade order: Controller, Broker, Server, Minion)
   * [ ] Yes (Please label as **<code>backward-incompat</code>**, and complete the section below on Release Notes)
   
   Does this PR fix a zero-downtime upgrade introduced earlier?
   * [ ] Yes (Please label this as **<code>backward-incompat</code>**, and complete the section below on Release Notes)
   
   Does this PR otherwise need attention when creating release notes? Things to consider:
   - New configuration options
   - Deprecation of configurations
   - Signature changes to public methods/interfaces
   - New plugins added or old plugins removed
   * [ ] Yes (Please label this PR as **<code>release-notes</code>** and complete the section on Release Notes)
   ## Release Notes
   <!-- If you have tagged this as either backward-incompat or release-notes,
   you MUST add text here that you would like to see appear in release notes of the
   next release. -->
   
   <!-- If you have a series of commits adding or enabling a feature, then
   add this section only in final commit that marks the feature completed.
   Refer to earlier release notes to see examples of text.
   -->
   ## Documentation
   <!-- If you have introduced a new feature or configuration, please add it to the documentation as well.
   See https://docs.pinot.apache.org/developers/developers-and-contributors/update-document
   -->
   


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

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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r755365577



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,490 @@
+/**
+ * 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.reduce;
+
+import com.google.common.base.Preconditions;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.request.context.OrderByExpressionContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GapfillUtils;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _groupByKeys;
+  private final Map<Key, Object[]> _previousByGroupKey;
+  private final int _numOfGroupByKeys;
+  private final List<Integer> _groupByKeyIndexes;
+  private final boolean [] _isGroupBySelections;
+  private int _timeBucketIndex = -1;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    Preconditions.checkArgument(
+        queryContext.getBrokerRequest().getPinotQuery() != null, "GapFill can only be applied to sql query");
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+
+    ExpressionContext gapFillSelection = null;
+    for (ExpressionContext expressionContext : _queryContext.getSelectExpressions()) {
+      if (GapfillUtils.isPostAggregateGapfill(expressionContext)) {
+        gapFillSelection = expressionContext;
+        break;
+      }
+    }
+
+    List<ExpressionContext> args = gapFillSelection.getFunction().getArguments();
+    Preconditions.checkArgument(
+        args.size() == 5, "PostAggregateGapFill does not correct number of arguments.");
+    Preconditions.checkArgument(
+        args.get(1).getLiteral() != null, "The second argument of PostAggregateGapFill should be TimeFormatter.");
+    Preconditions.checkArgument(
+        args.get(2).getLiteral() != null, "The third argument of PostAggregateGapFill should be start time.");
+    Preconditions.checkArgument(
+        args.get(3).getLiteral() != null, "The fourth argument of PostAggregateGapFill should be end time.");
+    Preconditions.checkArgument(
+        args.get(4).getLiteral() != null, "The fifth argument of PostAggregateGapFill should be time bucket size.");
+
+    boolean orderByTimeBucket = false;
+    if (_queryContext.getOrderByExpressions() != null) {
+      for (OrderByExpressionContext expressionContext : _queryContext.getOrderByExpressions()) {
+        if (expressionContext.getExpression().equals(gapFillSelection)) {
+          orderByTimeBucket = true;
+          break;
+        }
+      }
+    }
+
+    Preconditions.checkArgument(
+        orderByTimeBucket, "PostAggregateGapFill does not work if the time bucket is not ordered.");

Review comment:
       The result should be ordered by time bucket, but it can be ordered by more than one field along with time bucket.




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r753594902



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));

Review comment:
       _startMs and _endMs have already been truncated.




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r753586767



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // 1. groupByMode = sql, responseFormat = sql

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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r760743116



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/PostAggregationGapfillQueriesTest.java
##########
@@ -0,0 +1,592 @@
+/**
+ * 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.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+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.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+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.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+
+/**
+ * Queries test for PostAggregationGapfill queries.
+ */
+@SuppressWarnings("rawtypes")
+public class PostAggregationGapfillQueriesTest extends BaseQueriesTest {
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "PostAggregationGapfillQueriesTest");
+  private static final String RAW_TABLE_NAME = "parkingData";
+  private static final String SEGMENT_NAME = "testSegment";
+  private static final Random RANDOM = new Random();
+
+  private static final int NUM_LOTS = 4;
+
+  private static final String IS_OCCUPIED_COLUMN = "isOccupied";
+  private static final String LOT_ID_COLUMN = "lotId";
+  private static final String EVENT_TIME_COLUMN = "eventTime";
+  private static final Schema SCHEMA = new Schema.SchemaBuilder()
+      .addSingleValueDimension(IS_OCCUPIED_COLUMN, DataType.BOOLEAN)
+      .addSingleValueDimension(LOT_ID_COLUMN, DataType.STRING)
+      .addSingleValueDimension(EVENT_TIME_COLUMN, DataType.LONG)
+      .setPrimaryKeyColumns(Arrays.asList(LOT_ID_COLUMN, EVENT_TIME_COLUMN))
+      .build();
+  private static final TableConfig TABLE_CONFIG = new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME)
+      .build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  @Override
+  protected String getFilter() {
+    // NOTE: Use a match all filter to switch between DictionaryBasedAggregationOperator and AggregationOperator
+    return " WHERE eventTime >= 0";
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+
+    long current = 1636286400000L; //November 7, 2021 12:00:00 PM
+    int duplicates = 16;
+    int interval = 1000 * 900; // 15 minutes
+    long start = current - duplicates * 2 * interval; //November 7, 2021 4:00:00 AM
+
+    List<GenericRow> records = new ArrayList<>(NUM_LOTS * 2);
+    for (int i = 0; i < NUM_LOTS; i++) {
+      for (int j = 0; j < duplicates; j++) {
+        if (j == 4 || j == 5 || j == 6 || j == 7 || j == 10 || j == 11) {
+          continue;
+        }
+        long parkingTime = start + interval * 2 * j + RANDOM.nextInt(interval);
+        long departingTime = j == 3 ? start + interval * (2 * j + 6) + RANDOM.nextInt(interval) : start
+            + interval * (2 * j + 1) + RANDOM.nextInt(interval);
+
+        GenericRow parkingRow = new GenericRow();
+        parkingRow.putValue(EVENT_TIME_COLUMN, parkingTime);
+        parkingRow.putValue(LOT_ID_COLUMN, "LotId_" + String.valueOf(i));
+        parkingRow.putValue(IS_OCCUPIED_COLUMN, true);
+        records.add(parkingRow);
+
+        GenericRow departingRow = new GenericRow();
+        departingRow.putValue(EVENT_TIME_COLUMN, departingTime);
+        departingRow.putValue(LOT_ID_COLUMN, "LotId_" + String.valueOf(i));
+        departingRow.putValue(IS_OCCUPIED_COLUMN, false);
+        records.add(departingRow);
+      }
+    }
+
+    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);
+  }
+
+  @Test
+  public void datetimeconvertGapfillTest() {
+    String dataTimeConvertQuery = "SELECT "
+        + "DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN')"
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    BrokerResponseNative dateTimeConvertBrokerResponse = getBrokerResponseForSqlQuery(dataTimeConvertQuery);
+
+    ResultTable dateTimeConvertResultTable = dateTimeConvertBrokerResponse.getResultTable();
+    Assert.assertEquals(dateTimeConvertResultTable.getRows().size(), 24);
+
+    String gapfillQuery = "SELECT "
+        + "PostAggregateGapFill(DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS'), "
+        + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+        + "'2021-11-07 3:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_PREVIOUS_VALUE') as status1, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_DEFAULT_VALUE') as status2, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN') as status3 "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    DateTimeFormatSpec dateTimeFormatter
+        = new DateTimeFormatSpec("1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+
+    BrokerResponseNative gapfillBrokerResponse = getBrokerResponseForSqlQuery(gapfillQuery);
+
+    ResultTable gapFillResultTable = gapfillBrokerResponse.getResultTable();
+    Assert.assertEquals(gapFillResultTable.getRows().size(), 32);
+    List<Object[]> gapFillRows = gapFillResultTable.getRows();
+    long start = dateTimeFormatter.fromFormatToMillis("2021-11-07 03:00:00.000");
+    for (int i = 0; i < 32; i += 4) {
+      String firstTimeCol = (String) gapFillRows.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Set<String> lots = new HashSet<>();
+      lots.add((String) gapFillRows.get(i)[1]);
+      for (int j = 1; j < 4; j++) {
+        Assert.assertEquals(gapFillRows.get(i)[0], gapFillRows.get(i + j)[0]);
+        Assert.assertFalse(lots.contains(gapFillRows.get(i + j)[1]));
+        lots.add((String) gapFillRows.get(i + j)[1]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochHoursGapfillTest() {
+    String dataTimeConvertQuery = "SELECT "
+        + "ToEpochHours(eventTime) AS time_col, "
+        + "lotId, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN')"
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    BrokerResponseNative dateTimeConvertBrokerResponse = getBrokerResponseForSqlQuery(dataTimeConvertQuery);
+
+    ResultTable dateTimeConvertResultTable = dateTimeConvertBrokerResponse.getResultTable();
+    Assert.assertEquals(dateTimeConvertResultTable.getRows().size(), 24);
+
+    String gapfillQuery = "SELECT "
+        + "PostAggregateGapFill(ToEpochHours(eventTime), '1:HOURS:EPOCH', "
+        + "'454515',  '454524', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_PREVIOUS_VALUE') as status1, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_DEFAULT_VALUE') as status2, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN') as status3 "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+
+    BrokerResponseNative gapfillBrokerResponse = getBrokerResponseForSqlQuery(gapfillQuery);
+
+    ResultTable gapFillResultTable = gapfillBrokerResponse.getResultTable();
+    Assert.assertEquals(gapFillResultTable.getRows().size(), 32);
+    List<Object[]> gapFillRows = gapFillResultTable.getRows();
+    long start = dateTimeFormatter.fromFormatToMillis("454515");
+    for (int i = 0; i < 32; i += 4) {
+      Long firstTimeCol = (Long) gapFillRows.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol.toString());
+      Assert.assertEquals(timeStamp, start);
+      Set<String> lots = new HashSet<>();
+      lots.add((String) gapFillRows.get(i)[1]);
+      for (int j = 1; j < 4; j++) {
+        Assert.assertEquals(gapFillRows.get(i)[0], gapFillRows.get(i + j)[0]);
+        Assert.assertFalse(lots.contains(gapFillRows.get(i + j)[1]));
+        lots.add((String) gapFillRows.get(i + j)[1]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochMinutesRoundedHoursGapfillTest() {
+    String dataTimeConvertQuery = "SELECT "
+        + "ToEpochMinutesRounded(eventTime, 60) AS time_col, "
+        + "lotId, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN')"
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    BrokerResponseNative dateTimeConvertBrokerResponse = getBrokerResponseForSqlQuery(dataTimeConvertQuery);
+
+    ResultTable dateTimeConvertResultTable = dateTimeConvertBrokerResponse.getResultTable();
+    Assert.assertEquals(dateTimeConvertResultTable.getRows().size(), 24);
+
+    String gapfillQuery = "SELECT "
+        + "PostAggregateGapFill(ToEpochMinutesRounded(eventTime, 60), '1:HOURS:EPOCH', "
+        + "'454515',  '454524', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_PREVIOUS_VALUE') as status1, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_DEFAULT_VALUE') as status2, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN') as status3 "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+
+    BrokerResponseNative gapfillBrokerResponse = getBrokerResponseForSqlQuery(gapfillQuery);
+
+    ResultTable gapFillResultTable = gapfillBrokerResponse.getResultTable();
+    Assert.assertEquals(gapFillResultTable.getRows().size(), 32);
+    List<Object[]> gapFillRows = gapFillResultTable.getRows();
+    long start = dateTimeFormatter.fromFormatToMillis("454515");
+    for (int i = 0; i < 32; i += 4) {
+      Long firstTimeCol = (Long) gapFillRows.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol.toString());
+      Assert.assertEquals(timeStamp, start);
+      Set<String> lots = new HashSet<>();
+      lots.add((String) gapFillRows.get(i)[1]);
+      for (int j = 1; j < 4; j++) {
+        Assert.assertEquals(gapFillRows.get(i)[0], gapFillRows.get(i + j)[0]);
+        Assert.assertFalse(lots.contains(gapFillRows.get(i + j)[1]));
+        lots.add((String) gapFillRows.get(i + j)[1]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochMinutesBucketHoursGapfillTest() {
+    String dataTimeConvertQuery = "SELECT "
+        + "ToEpochMinutesBucket(eventTime, 60) AS time_col, "
+        + "lotId, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN')"
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    BrokerResponseNative dateTimeConvertBrokerResponse = getBrokerResponseForSqlQuery(dataTimeConvertQuery);
+
+    ResultTable dateTimeConvertResultTable = dateTimeConvertBrokerResponse.getResultTable();
+    Assert.assertEquals(dateTimeConvertResultTable.getRows().size(), 24);
+
+    String gapfillQuery = "SELECT "
+        + "PostAggregateGapFill(ToEpochMinutesBucket(eventTime, 60), '1:HOURS:EPOCH', "
+        + "'454515',  '454524', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_PREVIOUS_VALUE') as status1, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_DEFAULT_VALUE') as status2, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN') as status3 "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+
+    BrokerResponseNative gapfillBrokerResponse = getBrokerResponseForSqlQuery(gapfillQuery);
+
+    ResultTable gapFillResultTable = gapfillBrokerResponse.getResultTable();
+    Assert.assertEquals(gapFillResultTable.getRows().size(), 32);
+    List<Object[]> gapFillRows = gapFillResultTable.getRows();
+    long start = dateTimeFormatter.fromFormatToMillis("454515");
+    for (int i = 0; i < 32; i += 4) {
+      Long firstTimeCol = (Long) gapFillRows.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol.toString());
+      Assert.assertEquals(timeStamp, start);
+      Set<String> lots = new HashSet<>();
+      lots.add((String) gapFillRows.get(i)[1]);
+      for (int j = 1; j < 4; j++) {
+        Assert.assertEquals(gapFillRows.get(i)[0], gapFillRows.get(i + j)[0]);
+        Assert.assertFalse(lots.contains(gapFillRows.get(i + j)[1]));
+        lots.add((String) gapFillRows.get(i + j)[1]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void dateTruncHoursGapfillTest() {
+    String dataTimeConvertQuery = "SELECT "
+        + "DATETRUNC('hour', eventTime, 'milliseconds') AS time_col, "
+        + "lotId, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN')"
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    BrokerResponseNative dateTimeConvertBrokerResponse = getBrokerResponseForSqlQuery(dataTimeConvertQuery);
+
+    ResultTable dateTimeConvertResultTable = dateTimeConvertBrokerResponse.getResultTable();
+    Assert.assertEquals(dateTimeConvertResultTable.getRows().size(), 24);
+
+    String gapfillQuery = "SELECT "
+        + "PostAggregateGapFill(DATETRUNC('hour', eventTime, 'milliseconds'), '1:HOURS:EPOCH', "
+        + "'454515',  '454524', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_PREVIOUS_VALUE') as status1, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_DEFAULT_VALUE') as status2, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN') as status3 "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+
+    BrokerResponseNative gapfillBrokerResponse = getBrokerResponseForSqlQuery(gapfillQuery);
+
+    ResultTable gapFillResultTable = gapfillBrokerResponse.getResultTable();
+    Assert.assertEquals(gapFillResultTable.getRows().size(), 32);
+    List<Object[]> gapFillRows = gapFillResultTable.getRows();
+    long start = dateTimeFormatter.fromFormatToMillis("454515");
+    for (int i = 0; i < 32; i += 4) {
+      Long firstTimeCol = (Long) gapFillRows.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol.toString());
+      Assert.assertEquals(timeStamp, start);
+      Set<String> lots = new HashSet<>();
+      lots.add((String) gapFillRows.get(i)[1]);
+      for (int j = 1; j < 4; j++) {
+        Assert.assertEquals(gapFillRows.get(i)[0], gapFillRows.get(i + j)[0]);
+        Assert.assertFalse(lots.contains(gapFillRows.get(i + j)[1]));
+        lots.add((String) gapFillRows.get(i + j)[1]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void datetimeconvertGapfillTestWithoutTimeBucketOrdering() {
+    try {
+      String gapfillQuery = "SELECT "
+          + "PostAggregateGapFill(DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+          + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS'), "
+          + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+          + "'2021-11-07 3:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS') AS time_col, "
+          + "lotId, "
+          + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_PREVIOUS_VALUE') as status1, "
+          + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_DEFAULT_VALUE') as status2, "
+          + "lastWithTime(isOccupied, eventTime, 'BOOLEAN') as status3 "
+          + "FROM parkingData "
+          + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+          + "GROUP BY 1, 2 "
+          + "LIMIT 200";
+
+      getBrokerResponseForSqlQuery(gapfillQuery);
+      Assert.fail();
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(e.getMessage(), "PostAggregateGapFill does not work if the time bucket is not ordered.");
+    }
+  }
+
+  @Test
+  public void datetimeconvertGapfillTestWithHavingClause() {
+    String dataTimeConvertQuery = "SELECT "
+        + "DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN')"
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "HAVING lotId IN ('LotId_0', 'LotId_1', 'LotId_2') "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    BrokerResponseNative dateTimeConvertBrokerResponse = getBrokerResponseForSqlQuery(dataTimeConvertQuery);
+
+    ResultTable dateTimeConvertResultTable = dateTimeConvertBrokerResponse.getResultTable();
+    Assert.assertEquals(dateTimeConvertResultTable.getRows().size(), 18);
+
+    String gapfillQuery = "SELECT "
+        + "PostAggregateGapFill(DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS'), "
+        + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+        + "'2021-11-07 3:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_PREVIOUS_VALUE') as status1, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_DEFAULT_VALUE') as status2, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN') as status3 "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "HAVING lotId IN ('LotId_0', 'LotId_1', 'LotId_2') "

Review comment:
       Done




-- 
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] richardstartin commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751157541



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,725 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,
+    // a result is generated for each aggregation in the query,
+    // and the group by keys are not the same across the aggregations
+    // This PQL style of execution makes it impossible to support order by on group by.
+    //
+    // We could not simply change the group by execution behavior,
+    // as that would not be backward compatible for existing users of group by.
+    // As a result, we have 2 modes of group by execution - pql and sql - which can be controlled via query options
+    //
+    // Long term, we may completely move to sql, and keep only full sql mode alive
+    // Until then, we need to support responseFormat = sql for both the modes of execution.
+    // The 4 variants are as described below:
+
+    if (_groupByModeSql) {
+
+      if (_responseFormatSql) {
+        // 1. groupByMode = sql, responseFormat = sql
+        // This is the primary SQL compliant group by
+
+        try {
+          setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+              brokerMetrics);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // 2. groupByMode = sql, responseFormat = pql
+        // This mode will invoke SQL style group by execution, but present results in PQL way
+        // This mode is useful for users who want to avail of SQL compliant group by behavior,
+        // w/o having to forcefully move to a new result type
+
+        try {
+          setSQLGroupByInAggregationResults(brokerResponseNative, dataSchema, dataTables, reducerContext);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    } else {
+
+      // 3. groupByMode = pql, responseFormat = sql
+      // This mode is for users who want response presented in SQL style, but want PQL style group by behavior
+      // Multiple aggregations in PQL violates the tabular nature of results
+      // As a result, in this mode, only single aggregations are supported
+
+      // 4. groupByMode = pql, responseFormat = pql
+      // This is the primary PQL compliant group by
+
+      setGroupByResults(brokerResponseNative, dataTables);
+
+      if (_responseFormatSql) {
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // We emit the group by size when the result isn't empty. All the sizes among group-by results should be the
+        // same.
+        // Thus, we can just emit the one from the 1st result.
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    }
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    Object [] keyColumns = new Object[_numOfKeyColumns];
+    for (int i = 0; i < _numOfKeyColumns; i++) {
+      keyColumns[i] = row[i + 1];
+    }
+    return new Key(keyColumns);
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      int numResultColumns = resultColumnDataTypes.length;
+      int numResultRows = rows.size();
+      List<Object[]> resultRows = new ArrayList<>(numResultRows);
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < numResultColumns; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {
+      Set<Key> keys = new HashSet<>(_primaryKeys);
+      while (index < resultRows.size()) {
+        long timeCol = _dateTimeFormatter.fromFormatToMillis((String) resultRows.get(index)[0]);
+        if (timeCol < time) {
+          index++;
+        } else if (timeCol == time) {
+          gapfillResultRows.add(resultRows.get(index));
+          if (gapfillResultRows.size() == limit) {
+            return gapfillResultRows;
+          }
+          Key key = constructKey(resultRows.get(index));
+          keys.remove(key);
+          for (int colIndex = 2; colIndex < numResultColumns; colIndex++) {
+            _previous.put(key, resultRows.get(index));
+          }
+          index++;
+        } else {
+          break;
+        }
+      }
+      for (Key key : keys) {
+        Object[] gapfillRow = new Object[numResultColumns];
+        gapfillRow[0] = _dateTimeFormatter.fromMillisToFormat(time);
+        for (int i = 0; i < _numOfKeyColumns; i++) {
+          gapfillRow[i + 1] = key.getValues()[i];
+        }
+
+        for (int i = _numOfKeyColumns + 1; i < numResultColumns; i++) {
+          gapfillRow[i] = getFillValue(i, key, resultColumnDataTypes[i]);
+        }
+        gapfillResultRows.add(gapfillRow);
+        if (gapfillResultRows.size() == limit) {
+          return gapfillResultRows;
+        }
+      }
+    }
+    return gapfillResultRows;
+  }
+
+  Object getFillValue(int columIndex, Object key, ColumnDataType dataType) {
+    ExpressionContext expressionContext = _queryContext.getSelectExpressions().get(columIndex);
+    if (expressionContext.getFunction() != null
+        && expressionContext.getFunction().getFunctionName().equalsIgnoreCase("fill")) {
+      List<ExpressionContext> args = expressionContext.getFunction().getArguments();
+      if (args.get(1).getLiteral() == null) {
+        throw new UnsupportedOperationException("Wrong Sql.");
+      }
+      FillType fillType = FillType.valueOf(args.get(1).getLiteral());
+      if (fillType == FillType.FILL_DEFAULT_VALUE) {
+        // TODO: may fill the default value from sql in the future.
+        return SelectionOperatorUtils.getDefaultValue(dataType);
+      } else if (fillType == FillType.FILL_PREVIOUS_VALUE) {
+        if (_previous.containsKey(key)) {
+          return _previous.get(key)[columIndex];
+        } else {
+          return SelectionOperatorUtils.getDefaultValue(dataType);
+        }
+      } else {
+        throw new UnsupportedOperationException("unsupported fill type.");
+      }
+    } else {
+      return SelectionOperatorUtils.getDefaultValue(dataType);
+    }
+  }
+
+  enum FillType {
+    FILL_DEFAULT_VALUE,
+    FILL_PREVIOUS_VALUE,
+  }
+
+  /**
+   * Helper method to extract the final aggregation results for the given row (in-place).
+   */
+  private void extractFinalAggregationResults(Object[] row) {
+    for (int i = 0; i < _numAggregationFunctions; i++) {
+      int valueIndex = i + _numGroupByExpressions;
+      row[valueIndex] = _aggregationFunctions[i].extractFinalResult(row[valueIndex]);
+    }
+  }
+
+  /**
+   * Constructs the DataSchema for the rows before the post-aggregation (SQL mode).
+   */
+  private DataSchema getPrePostAggregationDataSchema(DataSchema dataSchema) {
+    String[] columnNames = dataSchema.getColumnNames();
+    ColumnDataType[] columnDataTypes = new ColumnDataType[_numColumns];
+    System.arraycopy(dataSchema.getColumnDataTypes(), 0, columnDataTypes, 0, _numGroupByExpressions);
+    for (int i = 0; i < _numAggregationFunctions; i++) {
+      columnDataTypes[i + _numGroupByExpressions] = _aggregationFunctions[i].getFinalResultColumnType();
+    }
+    return new DataSchema(columnNames, columnDataTypes);
+  }
+
+  private IndexedTable getIndexedTable(DataSchema dataSchema, Collection<DataTable> dataTablesToReduce,
+      DataTableReducerContext reducerContext)
+      throws TimeoutException {
+    long start = System.currentTimeMillis();
+    int numDataTables = dataTablesToReduce.size();
+
+    // Get the number of threads to use for reducing.
+    // In case of single reduce thread, fall back to SimpleIndexedTable to avoid redundant locking/unlocking calls.
+    int numReduceThreadsToUse = getNumReduceThreadsToUse(numDataTables, reducerContext.getMaxReduceThreadsPerQuery());
+    int limit = _queryContext.getLimit();
+    // TODO: Make minTrimSize configurable
+    int trimSize = GroupByUtils.getTableCapacity(limit);
+    // NOTE: For query with HAVING clause, use trimSize as resultSize to ensure the result accuracy.
+    // TODO: Resolve the HAVING clause within the IndexedTable before returning the result
+    int resultSize = _queryContext.getHavingFilter() != null ? trimSize : limit;
+    int trimThreshold = reducerContext.getGroupByTrimThreshold();
+    IndexedTable indexedTable;
+    if (numReduceThreadsToUse <= 1) {
+      indexedTable = new SimpleIndexedTable(dataSchema, _queryContext, resultSize, trimSize, trimThreshold);
+    } else {
+      if (trimThreshold >= GroupByOrderByCombineOperator.MAX_TRIM_THRESHOLD) {
+        // special case of trim threshold where it is set to max value.
+        // there won't be any trimming during upsert in this case.
+        // thus we can avoid the overhead of read-lock and write-lock
+        // in the upsert method.
+        indexedTable = new UnboundedConcurrentIndexedTable(dataSchema, _queryContext, resultSize);
+      } else {
+        indexedTable = new ConcurrentIndexedTable(dataSchema, _queryContext, resultSize, trimSize, trimThreshold);
+      }
+    }
+
+    Future[] futures = new Future[numDataTables];
+    CountDownLatch countDownLatch = new CountDownLatch(numDataTables);
+
+    // Create groups of data tables that each thread can process concurrently.
+    // Given that numReduceThreads is <= numDataTables, each group will have at least one data table.
+    ArrayList<DataTable> dataTables = new ArrayList<>(dataTablesToReduce);
+    List<List<DataTable>> reduceGroups = new ArrayList<>(numReduceThreadsToUse);
+
+    for (int i = 0; i < numReduceThreadsToUse; i++) {
+      reduceGroups.add(new ArrayList<>());
+    }
+    for (int i = 0; i < numDataTables; i++) {
+      reduceGroups.get(i % numReduceThreadsToUse).add(dataTables.get(i));
+    }
+
+    int cnt = 0;
+    ColumnDataType[] storedColumnDataTypes = dataSchema.getStoredColumnDataTypes();
+    for (List<DataTable> reduceGroup : reduceGroups) {
+      futures[cnt++] = reducerContext.getExecutorService().submit(new TraceRunnable() {
+        @Override
+        public void runJob() {
+          for (DataTable dataTable : reduceGroup) {
+            int numRows = dataTable.getNumberOfRows();
+
+            try {
+              for (int rowId = 0; rowId < numRows; rowId++) {
+                Object[] values = new Object[_numColumns];
+                for (int colId = 0; colId < _numColumns; colId++) {
+                  switch (storedColumnDataTypes[colId]) {
+                    case INT:
+                      values[colId] = dataTable.getInt(rowId, colId);
+                      break;
+                    case LONG:
+                      values[colId] = dataTable.getLong(rowId, colId);
+                      break;
+                    case FLOAT:
+                      values[colId] = dataTable.getFloat(rowId, colId);
+                      break;
+                    case DOUBLE:
+                      values[colId] = dataTable.getDouble(rowId, colId);
+                      break;
+                    case STRING:
+                      values[colId] = dataTable.getString(rowId, colId);
+                      break;
+                    case BYTES:
+                      values[colId] = dataTable.getBytes(rowId, colId);
+                      break;
+                    case OBJECT:
+                      values[colId] = dataTable.getObject(rowId, colId);
+                      break;
+                    // Add other aggregation intermediate result / group-by column type supports here
+                    default:
+                      throw new IllegalStateException();
+                  }
+                }
+                indexedTable.upsert(new Record(values));
+              }
+            } finally {
+              countDownLatch.countDown();
+            }
+          }
+        }
+      });
+    }
+
+    try {
+      long timeOutMs = reducerContext.getReduceTimeOutMs() - (System.currentTimeMillis() - start);
+      countDownLatch.await(timeOutMs, TimeUnit.MILLISECONDS);
+    } catch (InterruptedException e) {
+      for (Future future : futures) {
+        if (!future.isDone()) {
+          future.cancel(true);
+        }
+      }
+      throw new TimeoutException("Timed out in broker reduce phase.");
+    }
+
+    indexedTable.finish(true);
+    return indexedTable;
+  }
+
+  /**
+   * Computes the number of reduce threads to use per query.
+   * <ul>
+   *   <li> Use single thread if number of data tables to reduce is less than
+   *   {@value #MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE}.</li>
+   *   <li> Else, use min of max allowed reduce threads per query, and number of data tables.</li>
+   * </ul>
+   *
+   * @param numDataTables Number of data tables to reduce
+   * @param maxReduceThreadsPerQuery Max allowed reduce threads per query
+   * @return Number of reduce threads to use for the query
+   */
+  private int getNumReduceThreadsToUse(int numDataTables, int maxReduceThreadsPerQuery) {
+    // Use single thread if number of data tables < MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE.
+    if (numDataTables < MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE) {
+      return Math.min(1, numDataTables); // Number of data tables can be zero.
+    }
+
+    return Math.min(maxReduceThreadsPerQuery, numDataTables);
+  }
+
+  /**
+   * Extract the results of group by order by into a List of {@link AggregationResult}
+   * There will be 1 aggregation result per aggregation. The group by keys will be the same across all aggregations
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @throws TimeoutException If unable to complete within the timeout.
+   */
+  private void setSQLGroupByInAggregationResults(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext)
+      throws TimeoutException {
+
+    List<String> groupByColumns = new ArrayList<>(_numGroupByExpressions);
+    int idx = 0;
+    while (idx < _numGroupByExpressions) {
+      groupByColumns.add(dataSchema.getColumnName(idx));
+      idx++;
+    }
+
+    List<String> aggregationColumns = new ArrayList<>(_numAggregationFunctions);
+    List<List<GroupByResult>> groupByResults = new ArrayList<>(_numAggregationFunctions);
+    while (idx < _numColumns) {
+      aggregationColumns.add(dataSchema.getColumnName(idx));
+      groupByResults.add(new ArrayList<>());

Review comment:
       I think this will use more memory than is required, the default size is 16 array elements.




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751450736



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,725 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,
+    // a result is generated for each aggregation in the query,
+    // and the group by keys are not the same across the aggregations
+    // This PQL style of execution makes it impossible to support order by on group by.
+    //
+    // We could not simply change the group by execution behavior,
+    // as that would not be backward compatible for existing users of group by.
+    // As a result, we have 2 modes of group by execution - pql and sql - which can be controlled via query options
+    //
+    // Long term, we may completely move to sql, and keep only full sql mode alive
+    // Until then, we need to support responseFormat = sql for both the modes of execution.
+    // The 4 variants are as described below:
+
+    if (_groupByModeSql) {
+
+      if (_responseFormatSql) {
+        // 1. groupByMode = sql, responseFormat = sql
+        // This is the primary SQL compliant group by
+
+        try {
+          setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+              brokerMetrics);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // 2. groupByMode = sql, responseFormat = pql
+        // This mode will invoke SQL style group by execution, but present results in PQL way
+        // This mode is useful for users who want to avail of SQL compliant group by behavior,
+        // w/o having to forcefully move to a new result type
+
+        try {
+          setSQLGroupByInAggregationResults(brokerResponseNative, dataSchema, dataTables, reducerContext);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    } else {
+
+      // 3. groupByMode = pql, responseFormat = sql
+      // This mode is for users who want response presented in SQL style, but want PQL style group by behavior
+      // Multiple aggregations in PQL violates the tabular nature of results
+      // As a result, in this mode, only single aggregations are supported
+
+      // 4. groupByMode = pql, responseFormat = pql
+      // This is the primary PQL compliant group by
+
+      setGroupByResults(brokerResponseNative, dataTables);
+
+      if (_responseFormatSql) {
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // We emit the group by size when the result isn't empty. All the sizes among group-by results should be the
+        // same.
+        // Thus, we can just emit the one from the 1st result.
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    }
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    Object [] keyColumns = new Object[_numOfKeyColumns];
+    for (int i = 0; i < _numOfKeyColumns; i++) {
+      keyColumns[i] = row[i + 1];
+    }
+    return new Key(keyColumns);

Review comment:
       Done




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r752507535



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java
##########
@@ -223,8 +223,11 @@ private BrokerResponseNative getBrokerResponse(QueryContext queryContext, PlanMa
       byte[] serializedResponse = instanceResponse.toBytes();
       dataTableMap.put(new ServerRoutingInstance("localhost", 1234, TableType.OFFLINE),
           DataTableFactory.getDataTable(serializedResponse));
-      dataTableMap.put(new ServerRoutingInstance("localhost", 1234, TableType.REALTIME),
-          DataTableFactory.getDataTable(serializedResponse));
+      // skip creating the realtime table for gapfill test case.
+      if (!queryContext.isAggregateGapfill()) {

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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r755366473



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,490 @@
+/**
+ * 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.reduce;
+
+import com.google.common.base.Preconditions;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.request.context.OrderByExpressionContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GapfillUtils;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _groupByKeys;
+  private final Map<Key, Object[]> _previousByGroupKey;
+  private final int _numOfGroupByKeys;
+  private final List<Integer> _groupByKeyIndexes;
+  private final boolean [] _isGroupBySelections;
+  private int _timeBucketIndex = -1;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    Preconditions.checkArgument(
+        queryContext.getBrokerRequest().getPinotQuery() != null, "GapFill can only be applied to sql query");
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+
+    ExpressionContext gapFillSelection = null;
+    for (ExpressionContext expressionContext : _queryContext.getSelectExpressions()) {
+      if (GapfillUtils.isPostAggregateGapfill(expressionContext)) {
+        gapFillSelection = expressionContext;
+        break;
+      }
+    }
+
+    List<ExpressionContext> args = gapFillSelection.getFunction().getArguments();
+    Preconditions.checkArgument(
+        args.size() == 5, "PostAggregateGapFill does not correct number of arguments.");
+    Preconditions.checkArgument(

Review comment:
       I prefer not to do it since 1. there is no general way to tell which function is time-bucket except checking against the time bucket functions. 2. we can add new time function in future.




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751458239



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,725 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,
+    // a result is generated for each aggregation in the query,
+    // and the group by keys are not the same across the aggregations
+    // This PQL style of execution makes it impossible to support order by on group by.
+    //
+    // We could not simply change the group by execution behavior,
+    // as that would not be backward compatible for existing users of group by.
+    // As a result, we have 2 modes of group by execution - pql and sql - which can be controlled via query options
+    //
+    // Long term, we may completely move to sql, and keep only full sql mode alive
+    // Until then, we need to support responseFormat = sql for both the modes of execution.
+    // The 4 variants are as described below:
+
+    if (_groupByModeSql) {
+
+      if (_responseFormatSql) {
+        // 1. groupByMode = sql, responseFormat = sql
+        // This is the primary SQL compliant group by
+
+        try {
+          setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+              brokerMetrics);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // 2. groupByMode = sql, responseFormat = pql
+        // This mode will invoke SQL style group by execution, but present results in PQL way
+        // This mode is useful for users who want to avail of SQL compliant group by behavior,
+        // w/o having to forcefully move to a new result type
+
+        try {
+          setSQLGroupByInAggregationResults(brokerResponseNative, dataSchema, dataTables, reducerContext);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    } else {
+
+      // 3. groupByMode = pql, responseFormat = sql
+      // This mode is for users who want response presented in SQL style, but want PQL style group by behavior
+      // Multiple aggregations in PQL violates the tabular nature of results
+      // As a result, in this mode, only single aggregations are supported
+
+      // 4. groupByMode = pql, responseFormat = pql
+      // This is the primary PQL compliant group by
+
+      setGroupByResults(brokerResponseNative, dataTables);
+
+      if (_responseFormatSql) {
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // We emit the group by size when the result isn't empty. All the sizes among group-by results should be the
+        // same.
+        // Thus, we can just emit the one from the 1st result.
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    }
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    Object [] keyColumns = new Object[_numOfKeyColumns];
+    for (int i = 0; i < _numOfKeyColumns; i++) {
+      keyColumns[i] = row[i + 1];
+    }
+    return new Key(keyColumns);
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      int numResultColumns = resultColumnDataTypes.length;
+      int numResultRows = rows.size();
+      List<Object[]> resultRows = new ArrayList<>(numResultRows);
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < numResultColumns; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {
+      Set<Key> keys = new HashSet<>(_primaryKeys);
+      while (index < resultRows.size()) {
+        long timeCol = _dateTimeFormatter.fromFormatToMillis((String) resultRows.get(index)[0]);
+        if (timeCol < time) {
+          index++;
+        } else if (timeCol == time) {
+          gapfillResultRows.add(resultRows.get(index));
+          if (gapfillResultRows.size() == limit) {
+            return gapfillResultRows;
+          }
+          Key key = constructKey(resultRows.get(index));
+          keys.remove(key);
+          for (int colIndex = 2; colIndex < numResultColumns; colIndex++) {
+            _previous.put(key, resultRows.get(index));
+          }
+          index++;
+        } else {
+          break;
+        }
+      }
+      for (Key key : keys) {
+        Object[] gapfillRow = new Object[numResultColumns];
+        gapfillRow[0] = _dateTimeFormatter.fromMillisToFormat(time);
+        for (int i = 0; i < _numOfKeyColumns; i++) {
+          gapfillRow[i + 1] = key.getValues()[i];
+        }

Review comment:
       Done




-- 
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] lakshmanan-v commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
lakshmanan-v commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r755529666



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,490 @@
+/**
+ * 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.reduce;
+
+import com.google.common.base.Preconditions;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.request.context.OrderByExpressionContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GapfillUtils;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _groupByKeys;
+  private final Map<Key, Object[]> _previousByGroupKey;
+  private final int _numOfGroupByKeys;
+  private final List<Integer> _groupByKeyIndexes;
+  private final boolean [] _isGroupBySelections;
+  private int _timeBucketIndex = -1;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    Preconditions.checkArgument(
+        queryContext.getBrokerRequest().getPinotQuery() != null, "GapFill can only be applied to sql query");
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+
+    ExpressionContext gapFillSelection = null;
+    for (ExpressionContext expressionContext : _queryContext.getSelectExpressions()) {
+      if (GapfillUtils.isPostAggregateGapfill(expressionContext)) {
+        gapFillSelection = expressionContext;
+        break;
+      }
+    }
+
+    List<ExpressionContext> args = gapFillSelection.getFunction().getArguments();
+    Preconditions.checkArgument(
+        args.size() == 5, "PostAggregateGapFill does not correct number of arguments.");
+    Preconditions.checkArgument(
+        args.get(1).getLiteral() != null, "The second argument of PostAggregateGapFill should be TimeFormatter.");
+    Preconditions.checkArgument(
+        args.get(2).getLiteral() != null, "The third argument of PostAggregateGapFill should be start time.");
+    Preconditions.checkArgument(
+        args.get(3).getLiteral() != null, "The fourth argument of PostAggregateGapFill should be end time.");
+    Preconditions.checkArgument(
+        args.get(4).getLiteral() != null, "The fifth argument of PostAggregateGapFill should be time bucket size.");
+
+    boolean orderByTimeBucket = false;
+    if (_queryContext.getOrderByExpressions() != null) {
+      for (OrderByExpressionContext expressionContext : _queryContext.getOrderByExpressions()) {
+        if (expressionContext.getExpression().equals(gapFillSelection)) {
+          orderByTimeBucket = true;
+          break;
+        }
+      }
+    }
+
+    Preconditions.checkArgument(
+        orderByTimeBucket, "PostAggregateGapFill does not work if the time bucket is not ordered.");

Review comment:
       Could you add a test for this ?




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r755528039



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,490 @@
+/**
+ * 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.reduce;
+
+import com.google.common.base.Preconditions;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.request.context.OrderByExpressionContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GapfillUtils;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _groupByKeys;
+  private final Map<Key, Object[]> _previousByGroupKey;
+  private final int _numOfGroupByKeys;
+  private final List<Integer> _groupByKeyIndexes;
+  private final boolean [] _isGroupBySelections;
+  private int _timeBucketIndex = -1;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    Preconditions.checkArgument(
+        queryContext.getBrokerRequest().getPinotQuery() != null, "GapFill can only be applied to sql query");
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+
+    ExpressionContext gapFillSelection = null;
+    for (ExpressionContext expressionContext : _queryContext.getSelectExpressions()) {
+      if (GapfillUtils.isPostAggregateGapfill(expressionContext)) {
+        gapFillSelection = expressionContext;
+        break;
+      }
+    }
+
+    List<ExpressionContext> args = gapFillSelection.getFunction().getArguments();
+    Preconditions.checkArgument(
+        args.size() == 5, "PostAggregateGapFill does not correct number of arguments.");
+    Preconditions.checkArgument(

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] Jackie-Jiang merged pull request #7781: Add Post-Aggregation Gapfilling functionality.

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


   


-- 
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 edited a comment on pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#issuecomment-971204560


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?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 [#7781](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (442f220) into [master](https://codecov.io/gh/apache/pinot/commit/09bb91cecc8cff459a9d7814a7fe61aa67f2a393?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (09bb91c) will **decrease** coverage by `0.18%`.
   > The diff coverage is `41.87%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7781/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #7781      +/-   ##
   ============================================
   - Coverage     71.62%   71.43%   -0.19%     
     Complexity     4079     4079              
   ============================================
     Files          1578     1579       +1     
     Lines         80641    81044     +403     
     Branches      11987    12069      +82     
   ============================================
   + Hits          57758    57897     +139     
   - Misses        18994    19233     +239     
   - Partials       3889     3914      +25     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `29.00% <2.70%> (-0.19%)` | :arrow_down: |
   | integration2 | `27.77% <2.70%> (-0.08%)` | :arrow_down: |
   | unittests1 | `68.43% <41.87%> (-0.18%)` | :arrow_down: |
   | unittests2 | `14.51% <0.00%> (-0.10%)` | :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/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...t/core/query/selection/SelectionOperatorUtils.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9zZWxlY3Rpb24vU2VsZWN0aW9uT3BlcmF0b3JVdGlscy5qYXZh) | `82.55% <2.70%> (-9.63%)` | :arrow_down: |
   | [...e/query/reduce/GapFillGroupByDataTableReducer.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvR2FwRmlsbEdyb3VwQnlEYXRhVGFibGVSZWR1Y2VyLmphdmE=) | `43.46% <43.46%> (ø)` | |
   | [...pinot/core/query/request/context/QueryContext.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZXF1ZXN0L2NvbnRleHQvUXVlcnlDb250ZXh0LmphdmE=) | `97.42% <66.66%> (-0.49%)` | :arrow_down: |
   | [...not/core/operator/transform/TransformOperator.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vVHJhbnNmb3JtT3BlcmF0b3IuamF2YQ==) | `95.23% <100.00%> (+0.23%)` | :arrow_up: |
   | [...r/transform/function/TransformFunctionFactory.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vVHJhbnNmb3JtRnVuY3Rpb25GYWN0b3J5LmphdmE=) | `85.58% <100.00%> (+0.97%)` | :arrow_up: |
   | [...inot/core/query/reduce/PostAggregationHandler.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUG9zdEFnZ3JlZ2F0aW9uSGFuZGxlci5qYXZh) | `92.30% <100.00%> (+0.09%)` | :arrow_up: |
   | [.../pinot/core/query/reduce/ResultReducerFactory.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUmVzdWx0UmVkdWNlckZhY3RvcnkuamF2YQ==) | `100.00% <100.00%> (ø)` | |
   | [...pinot/common/utils/fetcher/HttpSegmentFetcher.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvZmV0Y2hlci9IdHRwU2VnbWVudEZldGNoZXIuamF2YQ==) | `61.53% <0.00%> (-10.26%)` | :arrow_down: |
   | [...r/helix/SegmentOnlineOfflineStateModelFactory.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3Qtc2VydmVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zZXJ2ZXIvc3RhcnRlci9oZWxpeC9TZWdtZW50T25saW5lT2ZmbGluZVN0YXRlTW9kZWxGYWN0b3J5LmphdmE=) | `58.49% <0.00%> (-8.50%)` | :arrow_down: |
   | [...nction/DistinctCountBitmapAggregationFunction.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9hZ2dyZWdhdGlvbi9mdW5jdGlvbi9EaXN0aW5jdENvdW50Qml0bWFwQWdncmVnYXRpb25GdW5jdGlvbi5qYXZh) | `47.15% <0.00%> (-7.26%)` | :arrow_down: |
   | ... and [20 more](https://codecov.io/gh/apache/pinot/pull/7781/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [09bb91c...442f220](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r755367238



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/PostAggregationGapfillQueriesTest.java
##########
@@ -0,0 +1,550 @@
+/**
+ * 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.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+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.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+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.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+
+/**
+ * Queries test for LASTWITHTIME queries.

Review comment:
       Done




-- 
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] richardstartin commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751151335



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,725 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,
+    // a result is generated for each aggregation in the query,
+    // and the group by keys are not the same across the aggregations
+    // This PQL style of execution makes it impossible to support order by on group by.
+    //
+    // We could not simply change the group by execution behavior,
+    // as that would not be backward compatible for existing users of group by.
+    // As a result, we have 2 modes of group by execution - pql and sql - which can be controlled via query options
+    //
+    // Long term, we may completely move to sql, and keep only full sql mode alive
+    // Until then, we need to support responseFormat = sql for both the modes of execution.
+    // The 4 variants are as described below:
+
+    if (_groupByModeSql) {
+
+      if (_responseFormatSql) {
+        // 1. groupByMode = sql, responseFormat = sql
+        // This is the primary SQL compliant group by
+
+        try {
+          setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+              brokerMetrics);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // 2. groupByMode = sql, responseFormat = pql
+        // This mode will invoke SQL style group by execution, but present results in PQL way
+        // This mode is useful for users who want to avail of SQL compliant group by behavior,
+        // w/o having to forcefully move to a new result type
+
+        try {
+          setSQLGroupByInAggregationResults(brokerResponseNative, dataSchema, dataTables, reducerContext);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    } else {
+
+      // 3. groupByMode = pql, responseFormat = sql
+      // This mode is for users who want response presented in SQL style, but want PQL style group by behavior
+      // Multiple aggregations in PQL violates the tabular nature of results
+      // As a result, in this mode, only single aggregations are supported
+
+      // 4. groupByMode = pql, responseFormat = pql
+      // This is the primary PQL compliant group by
+
+      setGroupByResults(brokerResponseNative, dataTables);
+
+      if (_responseFormatSql) {
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // We emit the group by size when the result isn't empty. All the sizes among group-by results should be the
+        // same.
+        // Thus, we can just emit the one from the 1st result.
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    }
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    Object [] keyColumns = new Object[_numOfKeyColumns];
+    for (int i = 0; i < _numOfKeyColumns; i++) {
+      keyColumns[i] = row[i + 1];
+    }
+    return new Key(keyColumns);
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      int numResultColumns = resultColumnDataTypes.length;
+      int numResultRows = rows.size();
+      List<Object[]> resultRows = new ArrayList<>(numResultRows);
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < numResultColumns; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {
+      Set<Key> keys = new HashSet<>(_primaryKeys);
+      while (index < resultRows.size()) {
+        long timeCol = _dateTimeFormatter.fromFormatToMillis((String) resultRows.get(index)[0]);
+        if (timeCol < time) {
+          index++;
+        } else if (timeCol == time) {
+          gapfillResultRows.add(resultRows.get(index));
+          if (gapfillResultRows.size() == limit) {
+            return gapfillResultRows;
+          }
+          Key key = constructKey(resultRows.get(index));
+          keys.remove(key);
+          for (int colIndex = 2; colIndex < numResultColumns; colIndex++) {
+            _previous.put(key, resultRows.get(index));
+          }
+          index++;
+        } else {
+          break;
+        }
+      }
+      for (Key key : keys) {
+        Object[] gapfillRow = new Object[numResultColumns];
+        gapfillRow[0] = _dateTimeFormatter.fromMillisToFormat(time);
+        for (int i = 0; i < _numOfKeyColumns; i++) {
+          gapfillRow[i + 1] = key.getValues()[i];
+        }
+
+        for (int i = _numOfKeyColumns + 1; i < numResultColumns; i++) {
+          gapfillRow[i] = getFillValue(i, key, resultColumnDataTypes[i]);
+        }
+        gapfillResultRows.add(gapfillRow);
+        if (gapfillResultRows.size() == limit) {
+          return gapfillResultRows;
+        }
+      }
+    }
+    return gapfillResultRows;
+  }
+
+  Object getFillValue(int columIndex, Object key, ColumnDataType dataType) {
+    ExpressionContext expressionContext = _queryContext.getSelectExpressions().get(columIndex);
+    if (expressionContext.getFunction() != null
+        && expressionContext.getFunction().getFunctionName().equalsIgnoreCase("fill")) {
+      List<ExpressionContext> args = expressionContext.getFunction().getArguments();
+      if (args.get(1).getLiteral() == null) {
+        throw new UnsupportedOperationException("Wrong Sql.");
+      }
+      FillType fillType = FillType.valueOf(args.get(1).getLiteral());
+      if (fillType == FillType.FILL_DEFAULT_VALUE) {
+        // TODO: may fill the default value from sql in the future.
+        return SelectionOperatorUtils.getDefaultValue(dataType);
+      } else if (fillType == FillType.FILL_PREVIOUS_VALUE) {
+        if (_previous.containsKey(key)) {
+          return _previous.get(key)[columIndex];
+        } else {
+          return SelectionOperatorUtils.getDefaultValue(dataType);
+        }
+      } else {
+        throw new UnsupportedOperationException("unsupported fill type.");
+      }
+    } else {
+      return SelectionOperatorUtils.getDefaultValue(dataType);
+    }
+  }
+
+  enum FillType {
+    FILL_DEFAULT_VALUE,
+    FILL_PREVIOUS_VALUE,
+  }
+
+  /**
+   * Helper method to extract the final aggregation results for the given row (in-place).
+   */
+  private void extractFinalAggregationResults(Object[] row) {
+    for (int i = 0; i < _numAggregationFunctions; i++) {
+      int valueIndex = i + _numGroupByExpressions;
+      row[valueIndex] = _aggregationFunctions[i].extractFinalResult(row[valueIndex]);
+    }
+  }
+
+  /**
+   * Constructs the DataSchema for the rows before the post-aggregation (SQL mode).
+   */
+  private DataSchema getPrePostAggregationDataSchema(DataSchema dataSchema) {
+    String[] columnNames = dataSchema.getColumnNames();
+    ColumnDataType[] columnDataTypes = new ColumnDataType[_numColumns];
+    System.arraycopy(dataSchema.getColumnDataTypes(), 0, columnDataTypes, 0, _numGroupByExpressions);
+    for (int i = 0; i < _numAggregationFunctions; i++) {
+      columnDataTypes[i + _numGroupByExpressions] = _aggregationFunctions[i].getFinalResultColumnType();
+    }
+    return new DataSchema(columnNames, columnDataTypes);
+  }
+
+  private IndexedTable getIndexedTable(DataSchema dataSchema, Collection<DataTable> dataTablesToReduce,
+      DataTableReducerContext reducerContext)
+      throws TimeoutException {
+    long start = System.currentTimeMillis();
+    int numDataTables = dataTablesToReduce.size();
+
+    // Get the number of threads to use for reducing.
+    // In case of single reduce thread, fall back to SimpleIndexedTable to avoid redundant locking/unlocking calls.
+    int numReduceThreadsToUse = getNumReduceThreadsToUse(numDataTables, reducerContext.getMaxReduceThreadsPerQuery());
+    int limit = _queryContext.getLimit();
+    // TODO: Make minTrimSize configurable
+    int trimSize = GroupByUtils.getTableCapacity(limit);
+    // NOTE: For query with HAVING clause, use trimSize as resultSize to ensure the result accuracy.
+    // TODO: Resolve the HAVING clause within the IndexedTable before returning the result
+    int resultSize = _queryContext.getHavingFilter() != null ? trimSize : limit;
+    int trimThreshold = reducerContext.getGroupByTrimThreshold();
+    IndexedTable indexedTable;
+    if (numReduceThreadsToUse <= 1) {
+      indexedTable = new SimpleIndexedTable(dataSchema, _queryContext, resultSize, trimSize, trimThreshold);
+    } else {
+      if (trimThreshold >= GroupByOrderByCombineOperator.MAX_TRIM_THRESHOLD) {
+        // special case of trim threshold where it is set to max value.
+        // there won't be any trimming during upsert in this case.
+        // thus we can avoid the overhead of read-lock and write-lock
+        // in the upsert method.
+        indexedTable = new UnboundedConcurrentIndexedTable(dataSchema, _queryContext, resultSize);
+      } else {
+        indexedTable = new ConcurrentIndexedTable(dataSchema, _queryContext, resultSize, trimSize, trimThreshold);
+      }
+    }
+
+    Future[] futures = new Future[numDataTables];
+    CountDownLatch countDownLatch = new CountDownLatch(numDataTables);
+
+    // Create groups of data tables that each thread can process concurrently.
+    // Given that numReduceThreads is <= numDataTables, each group will have at least one data table.
+    ArrayList<DataTable> dataTables = new ArrayList<>(dataTablesToReduce);
+    List<List<DataTable>> reduceGroups = new ArrayList<>(numReduceThreadsToUse);
+
+    for (int i = 0; i < numReduceThreadsToUse; i++) {
+      reduceGroups.add(new ArrayList<>());
+    }
+    for (int i = 0; i < numDataTables; i++) {
+      reduceGroups.get(i % numReduceThreadsToUse).add(dataTables.get(i));
+    }
+
+    int cnt = 0;
+    ColumnDataType[] storedColumnDataTypes = dataSchema.getStoredColumnDataTypes();
+    for (List<DataTable> reduceGroup : reduceGroups) {
+      futures[cnt++] = reducerContext.getExecutorService().submit(new TraceRunnable() {

Review comment:
       What about 
   
   ```java
   List<Future<?>> futures = reducerContext.getExecutorService().invokeAll(reduceGroups.stream().map(reduceGroup -> new TraceCallable<Void>() {
         @Override
         public Void callJob()
             throws Exception {
           // do stuff
           return null;
         }
       }).collect(Collectors.toList()), timeOutMs, TimeUnit.MILLISECONDS);
   
   
   ```




-- 
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] richardstartin commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751190529



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/selection/SelectionOperatorUtils.java
##########
@@ -339,6 +339,74 @@ public static DataTable getDataTableFromRows(Collection<Object[]> rows, DataSche
     return dataTableBuilder.build();
   }
 
+  /**
+   * The default value for each column type.
+   */
+  public static Serializable getDefaultValue(ColumnDataType dataType) {
+    switch (dataType) {
+      // Single-value column
+      case INT:
+        return THREAD_LOCAL_INT_FORMAT.get().format(((Number) 0).intValue());
+      case LONG:
+        return THREAD_LOCAL_LONG_FORMAT.get().format(((Number) 0).longValue());
+      case FLOAT:
+        return THREAD_LOCAL_FLOAT_FORMAT.get().format(((Number) 0).floatValue());
+      case DOUBLE:
+        return THREAD_LOCAL_DOUBLE_FORMAT.get().format(((Number) 0).doubleValue());
+      case BOOLEAN:
+        return "false";
+      case TIMESTAMP:
+        return new Timestamp(0L).toString();
+      // NOTE: Return String for BYTES columns for backward-compatibility
+      case BYTES:
+        return "";
+
+      // Multi-value column
+      case INT_ARRAY:
+        DecimalFormat intFormat = THREAD_LOCAL_INT_FORMAT.get();
+        int[] ints = new int[0];

Review comment:
       is this intended?




-- 
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] richardstartin commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751166491



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/selection/SelectionOperatorUtils.java
##########
@@ -339,6 +339,74 @@ public static DataTable getDataTableFromRows(Collection<Object[]> rows, DataSche
     return dataTableBuilder.build();
   }
 
+  /**
+   * The default value for each column type.
+   */
+  public static Serializable getDefaultValue(ColumnDataType dataType) {
+    switch (dataType) {
+      // Single-value column
+      case INT:
+        return THREAD_LOCAL_INT_FORMAT.get().format(((Number) 0).intValue());
+      case LONG:
+        return THREAD_LOCAL_LONG_FORMAT.get().format(((Number) 0).longValue());
+      case FLOAT:
+        return THREAD_LOCAL_FLOAT_FORMAT.get().format(((Number) 0).floatValue());
+      case DOUBLE:
+        return THREAD_LOCAL_DOUBLE_FORMAT.get().format(((Number) 0).doubleValue());

Review comment:
       ```java
         case INT:
            return THREAD_LOCAL_INT_FORMAT.get().format(0);
          case LONG:
            return THREAD_LOCAL_LONG_FORMAT.get().format(0L);
          case FLOAT:
            return THREAD_LOCAL_FLOAT_FORMAT.get().format(0F);
          case DOUBLE:
            return THREAD_LOCAL_DOUBLE_FORMAT.get().format(0D);
   ```




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r753590209



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/request/context/QueryContext.java
##########
@@ -201,6 +202,11 @@ public int getOffset() {
     return _offset;
   }
 
+  public boolean isAggregateGapfill() {
+    return !_selectExpressions.isEmpty()

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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r760743826



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/PostAggregationHandler.java
##########
@@ -67,7 +68,8 @@ public PostAggregationHandler(QueryContext queryContext, DataSchema dataSchema)
     String[] columnNames = new String[numSelectExpressions];
     ColumnDataType[] columnDataTypes = new ColumnDataType[numSelectExpressions];
     for (int i = 0; i < numSelectExpressions; i++) {
-      ValueExtractor valueExtractor = getValueExtractor(selectExpressions.get(i));
+      ValueExtractor valueExtractor
+          = getValueExtractor(selectExpressions.get(i));

Review comment:
       Done




-- 
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 edited a comment on pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#issuecomment-971204560


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?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 [#7781](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (2bac553) into [master](https://codecov.io/gh/apache/pinot/commit/09bb91cecc8cff459a9d7814a7fe61aa67f2a393?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (09bb91c) will **increase** coverage by `0.01%`.
   > The diff coverage is `73.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7781/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #7781      +/-   ##
   ============================================
   + Coverage     71.62%   71.63%   +0.01%     
   - Complexity     4079     4085       +6     
   ============================================
     Files          1578     1581       +3     
     Lines         80641    81107     +466     
     Branches      11987    12082      +95     
   ============================================
   + Hits          57758    58105     +347     
   - Misses        18994    19064      +70     
   - Partials       3889     3938      +49     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `29.13% <6.00%> (-0.05%)` | :arrow_down: |
   | integration2 | `27.82% <6.00%> (-0.04%)` | :arrow_down: |
   | unittests1 | `68.70% <73.00%> (+0.09%)` | :arrow_up: |
   | unittests2 | `14.51% <0.00%> (-0.09%)` | :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/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../java/org/apache/pinot/core/util/GapfillUtils.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS91dGlsL0dhcGZpbGxVdGlscy5qYXZh) | `47.82% <47.82%> (ø)` | |
   | [...e/query/reduce/GapFillGroupByDataTableReducer.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvR2FwRmlsbEdyb3VwQnlEYXRhVGFibGVSZWR1Y2VyLmphdmE=) | `76.34% <76.34%> (ø)` | |
   | [...not/core/operator/transform/TransformOperator.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vVHJhbnNmb3JtT3BlcmF0b3IuamF2YQ==) | `95.23% <100.00%> (+0.23%)` | :arrow_up: |
   | [...inot/core/query/reduce/PostAggregationHandler.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUG9zdEFnZ3JlZ2F0aW9uSGFuZGxlci5qYXZh) | `92.40% <100.00%> (+0.19%)` | :arrow_up: |
   | [.../pinot/core/query/reduce/ResultReducerFactory.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUmVzdWx0UmVkdWNlckZhY3RvcnkuamF2YQ==) | `100.00% <100.00%> (ø)` | |
   | [...pinot/core/query/request/context/QueryContext.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZXF1ZXN0L2NvbnRleHQvUXVlcnlDb250ZXh0LmphdmE=) | `97.95% <100.00%> (+0.05%)` | :arrow_up: |
   | [...operator/filter/RangeIndexBasedFilterOperator.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9maWx0ZXIvUmFuZ2VJbmRleEJhc2VkRmlsdGVyT3BlcmF0b3IuamF2YQ==) | `25.53% <0.00%> (-14.90%)` | :arrow_down: |
   | [...ava/org/apache/pinot/client/ConnectionFactory.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Nvbm5lY3Rpb25GYWN0b3J5LmphdmE=) | `70.58% <0.00%> (-12.75%)` | :arrow_down: |
   | [...pinot/common/utils/fetcher/HttpSegmentFetcher.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvZmV0Y2hlci9IdHRwU2VnbWVudEZldGNoZXIuamF2YQ==) | `61.53% <0.00%> (-10.26%)` | :arrow_down: |
   | [...org/apache/pinot/broker/api/RequestStatistics.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYXBpL1JlcXVlc3RTdGF0aXN0aWNzLmphdmE=) | `55.10% <0.00%> (-9.77%)` | :arrow_down: |
   | ... and [51 more](https://codecov.io/gh/apache/pinot/pull/7781/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [09bb91c...2bac553](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r754569618



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);

Review comment:
       Done




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751563767



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,725 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,
+    // a result is generated for each aggregation in the query,
+    // and the group by keys are not the same across the aggregations
+    // This PQL style of execution makes it impossible to support order by on group by.
+    //
+    // We could not simply change the group by execution behavior,
+    // as that would not be backward compatible for existing users of group by.
+    // As a result, we have 2 modes of group by execution - pql and sql - which can be controlled via query options
+    //
+    // Long term, we may completely move to sql, and keep only full sql mode alive
+    // Until then, we need to support responseFormat = sql for both the modes of execution.
+    // The 4 variants are as described below:
+
+    if (_groupByModeSql) {
+
+      if (_responseFormatSql) {
+        // 1. groupByMode = sql, responseFormat = sql
+        // This is the primary SQL compliant group by
+
+        try {
+          setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+              brokerMetrics);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // 2. groupByMode = sql, responseFormat = pql
+        // This mode will invoke SQL style group by execution, but present results in PQL way
+        // This mode is useful for users who want to avail of SQL compliant group by behavior,
+        // w/o having to forcefully move to a new result type
+
+        try {
+          setSQLGroupByInAggregationResults(brokerResponseNative, dataSchema, dataTables, reducerContext);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    } else {
+
+      // 3. groupByMode = pql, responseFormat = sql
+      // This mode is for users who want response presented in SQL style, but want PQL style group by behavior
+      // Multiple aggregations in PQL violates the tabular nature of results
+      // As a result, in this mode, only single aggregations are supported
+
+      // 4. groupByMode = pql, responseFormat = pql
+      // This is the primary PQL compliant group by
+
+      setGroupByResults(brokerResponseNative, dataTables);
+
+      if (_responseFormatSql) {
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // We emit the group by size when the result isn't empty. All the sizes among group-by results should be the
+        // same.
+        // Thus, we can just emit the one from the 1st result.
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    }
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    Object [] keyColumns = new Object[_numOfKeyColumns];
+    for (int i = 0; i < _numOfKeyColumns; i++) {
+      keyColumns[i] = row[i + 1];
+    }
+    return new Key(keyColumns);
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      int numResultColumns = resultColumnDataTypes.length;
+      int numResultRows = rows.size();
+      List<Object[]> resultRows = new ArrayList<>(numResultRows);
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < numResultColumns; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {
+      Set<Key> keys = new HashSet<>(_primaryKeys);
+      while (index < resultRows.size()) {
+        long timeCol = _dateTimeFormatter.fromFormatToMillis((String) resultRows.get(index)[0]);
+        if (timeCol < time) {
+          index++;
+        } else if (timeCol == time) {
+          gapfillResultRows.add(resultRows.get(index));
+          if (gapfillResultRows.size() == limit) {
+            return gapfillResultRows;
+          }
+          Key key = constructKey(resultRows.get(index));
+          keys.remove(key);
+          for (int colIndex = 2; colIndex < numResultColumns; colIndex++) {
+            _previous.put(key, resultRows.get(index));
+          }
+          index++;
+        } else {
+          break;
+        }
+      }
+      for (Key key : keys) {
+        Object[] gapfillRow = new Object[numResultColumns];
+        gapfillRow[0] = _dateTimeFormatter.fromMillisToFormat(time);
+        for (int i = 0; i < _numOfKeyColumns; i++) {
+          gapfillRow[i + 1] = key.getValues()[i];
+        }
+
+        for (int i = _numOfKeyColumns + 1; i < numResultColumns; i++) {
+          gapfillRow[i] = getFillValue(i, key, resultColumnDataTypes[i]);
+        }
+        gapfillResultRows.add(gapfillRow);
+        if (gapfillResultRows.size() == limit) {
+          return gapfillResultRows;
+        }
+      }
+    }
+    return gapfillResultRows;
+  }
+
+  Object getFillValue(int columIndex, Object key, ColumnDataType dataType) {
+    ExpressionContext expressionContext = _queryContext.getSelectExpressions().get(columIndex);
+    if (expressionContext.getFunction() != null
+        && expressionContext.getFunction().getFunctionName().equalsIgnoreCase("fill")) {
+      List<ExpressionContext> args = expressionContext.getFunction().getArguments();
+      if (args.get(1).getLiteral() == null) {
+        throw new UnsupportedOperationException("Wrong Sql.");
+      }
+      FillType fillType = FillType.valueOf(args.get(1).getLiteral());
+      if (fillType == FillType.FILL_DEFAULT_VALUE) {
+        // TODO: may fill the default value from sql in the future.
+        return SelectionOperatorUtils.getDefaultValue(dataType);
+      } else if (fillType == FillType.FILL_PREVIOUS_VALUE) {
+        if (_previous.containsKey(key)) {
+          return _previous.get(key)[columIndex];
+        } else {
+          return SelectionOperatorUtils.getDefaultValue(dataType);
+        }
+      } else {
+        throw new UnsupportedOperationException("unsupported fill type.");
+      }
+    } else {
+      return SelectionOperatorUtils.getDefaultValue(dataType);
+    }
+  }
+
+  enum FillType {
+    FILL_DEFAULT_VALUE,
+    FILL_PREVIOUS_VALUE,
+  }
+
+  /**
+   * Helper method to extract the final aggregation results for the given row (in-place).
+   */
+  private void extractFinalAggregationResults(Object[] row) {
+    for (int i = 0; i < _numAggregationFunctions; i++) {
+      int valueIndex = i + _numGroupByExpressions;
+      row[valueIndex] = _aggregationFunctions[i].extractFinalResult(row[valueIndex]);
+    }
+  }
+
+  /**
+   * Constructs the DataSchema for the rows before the post-aggregation (SQL mode).
+   */
+  private DataSchema getPrePostAggregationDataSchema(DataSchema dataSchema) {
+    String[] columnNames = dataSchema.getColumnNames();
+    ColumnDataType[] columnDataTypes = new ColumnDataType[_numColumns];
+    System.arraycopy(dataSchema.getColumnDataTypes(), 0, columnDataTypes, 0, _numGroupByExpressions);
+    for (int i = 0; i < _numAggregationFunctions; i++) {
+      columnDataTypes[i + _numGroupByExpressions] = _aggregationFunctions[i].getFinalResultColumnType();
+    }
+    return new DataSchema(columnNames, columnDataTypes);
+  }
+
+  private IndexedTable getIndexedTable(DataSchema dataSchema, Collection<DataTable> dataTablesToReduce,
+      DataTableReducerContext reducerContext)
+      throws TimeoutException {
+    long start = System.currentTimeMillis();
+    int numDataTables = dataTablesToReduce.size();
+
+    // Get the number of threads to use for reducing.
+    // In case of single reduce thread, fall back to SimpleIndexedTable to avoid redundant locking/unlocking calls.
+    int numReduceThreadsToUse = getNumReduceThreadsToUse(numDataTables, reducerContext.getMaxReduceThreadsPerQuery());
+    int limit = _queryContext.getLimit();
+    // TODO: Make minTrimSize configurable
+    int trimSize = GroupByUtils.getTableCapacity(limit);
+    // NOTE: For query with HAVING clause, use trimSize as resultSize to ensure the result accuracy.
+    // TODO: Resolve the HAVING clause within the IndexedTable before returning the result
+    int resultSize = _queryContext.getHavingFilter() != null ? trimSize : limit;
+    int trimThreshold = reducerContext.getGroupByTrimThreshold();
+    IndexedTable indexedTable;
+    if (numReduceThreadsToUse <= 1) {
+      indexedTable = new SimpleIndexedTable(dataSchema, _queryContext, resultSize, trimSize, trimThreshold);
+    } else {
+      if (trimThreshold >= GroupByOrderByCombineOperator.MAX_TRIM_THRESHOLD) {
+        // special case of trim threshold where it is set to max value.
+        // there won't be any trimming during upsert in this case.
+        // thus we can avoid the overhead of read-lock and write-lock
+        // in the upsert method.
+        indexedTable = new UnboundedConcurrentIndexedTable(dataSchema, _queryContext, resultSize);
+      } else {
+        indexedTable = new ConcurrentIndexedTable(dataSchema, _queryContext, resultSize, trimSize, trimThreshold);
+      }
+    }
+
+    Future[] futures = new Future[numDataTables];
+    CountDownLatch countDownLatch = new CountDownLatch(numDataTables);
+
+    // Create groups of data tables that each thread can process concurrently.
+    // Given that numReduceThreads is <= numDataTables, each group will have at least one data table.
+    ArrayList<DataTable> dataTables = new ArrayList<>(dataTablesToReduce);
+    List<List<DataTable>> reduceGroups = new ArrayList<>(numReduceThreadsToUse);
+
+    for (int i = 0; i < numReduceThreadsToUse; i++) {
+      reduceGroups.add(new ArrayList<>());
+    }
+    for (int i = 0; i < numDataTables; i++) {
+      reduceGroups.get(i % numReduceThreadsToUse).add(dataTables.get(i));
+    }
+
+    int cnt = 0;
+    ColumnDataType[] storedColumnDataTypes = dataSchema.getStoredColumnDataTypes();
+    for (List<DataTable> reduceGroup : reduceGroups) {
+      futures[cnt++] = reducerContext.getExecutorService().submit(new TraceRunnable() {
+        @Override
+        public void runJob() {
+          for (DataTable dataTable : reduceGroup) {
+            int numRows = dataTable.getNumberOfRows();
+
+            try {
+              for (int rowId = 0; rowId < numRows; rowId++) {
+                Object[] values = new Object[_numColumns];
+                for (int colId = 0; colId < _numColumns; colId++) {
+                  switch (storedColumnDataTypes[colId]) {
+                    case INT:
+                      values[colId] = dataTable.getInt(rowId, colId);
+                      break;
+                    case LONG:
+                      values[colId] = dataTable.getLong(rowId, colId);
+                      break;
+                    case FLOAT:
+                      values[colId] = dataTable.getFloat(rowId, colId);
+                      break;
+                    case DOUBLE:
+                      values[colId] = dataTable.getDouble(rowId, colId);
+                      break;
+                    case STRING:
+                      values[colId] = dataTable.getString(rowId, colId);
+                      break;
+                    case BYTES:
+                      values[colId] = dataTable.getBytes(rowId, colId);
+                      break;
+                    case OBJECT:
+                      values[colId] = dataTable.getObject(rowId, colId);
+                      break;
+                    // Add other aggregation intermediate result / group-by column type supports here
+                    default:
+                      throw new IllegalStateException();
+                  }
+                }
+                indexedTable.upsert(new Record(values));
+              }
+            } finally {
+              countDownLatch.countDown();

Review comment:
       Done




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r752443884



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java
##########
@@ -223,8 +223,11 @@ private BrokerResponseNative getBrokerResponse(QueryContext queryContext, PlanMa
       byte[] serializedResponse = instanceResponse.toBytes();
       dataTableMap.put(new ServerRoutingInstance("localhost", 1234, TableType.OFFLINE),
           DataTableFactory.getDataTable(serializedResponse));
-      dataTableMap.put(new ServerRoutingInstance("localhost", 1234, TableType.REALTIME),
-          DataTableFactory.getDataTable(serializedResponse));
+      // skip creating the realtime table for gapfill test case.
+      if (!queryContext.isAggregateGapfill()) {

Review comment:
       Do you mean sqlQuery will not hit the realtime table, right? How about pqlQuery? It seems that pqlQuery will query both tables. 




-- 
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] richardstartin commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751139522



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,725 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,
+    // a result is generated for each aggregation in the query,
+    // and the group by keys are not the same across the aggregations
+    // This PQL style of execution makes it impossible to support order by on group by.
+    //
+    // We could not simply change the group by execution behavior,
+    // as that would not be backward compatible for existing users of group by.
+    // As a result, we have 2 modes of group by execution - pql and sql - which can be controlled via query options
+    //
+    // Long term, we may completely move to sql, and keep only full sql mode alive
+    // Until then, we need to support responseFormat = sql for both the modes of execution.
+    // The 4 variants are as described below:
+
+    if (_groupByModeSql) {
+
+      if (_responseFormatSql) {
+        // 1. groupByMode = sql, responseFormat = sql
+        // This is the primary SQL compliant group by
+
+        try {
+          setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+              brokerMetrics);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // 2. groupByMode = sql, responseFormat = pql
+        // This mode will invoke SQL style group by execution, but present results in PQL way
+        // This mode is useful for users who want to avail of SQL compliant group by behavior,
+        // w/o having to forcefully move to a new result type
+
+        try {
+          setSQLGroupByInAggregationResults(brokerResponseNative, dataSchema, dataTables, reducerContext);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    } else {
+
+      // 3. groupByMode = pql, responseFormat = sql
+      // This mode is for users who want response presented in SQL style, but want PQL style group by behavior
+      // Multiple aggregations in PQL violates the tabular nature of results
+      // As a result, in this mode, only single aggregations are supported
+
+      // 4. groupByMode = pql, responseFormat = pql
+      // This is the primary PQL compliant group by
+
+      setGroupByResults(brokerResponseNative, dataTables);
+
+      if (_responseFormatSql) {
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // We emit the group by size when the result isn't empty. All the sizes among group-by results should be the
+        // same.
+        // Thus, we can just emit the one from the 1st result.
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    }
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    Object [] keyColumns = new Object[_numOfKeyColumns];
+    for (int i = 0; i < _numOfKeyColumns; i++) {
+      keyColumns[i] = row[i + 1];
+    }
+    return new Key(keyColumns);
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      int numResultColumns = resultColumnDataTypes.length;
+      int numResultRows = rows.size();

Review comment:
       inline these where they are used, these variable don't help readability and don't improve performance.




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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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



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


[GitHub] [pinot] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751622057



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/selection/SelectionOperatorUtils.java
##########
@@ -339,6 +339,74 @@ public static DataTable getDataTableFromRows(Collection<Object[]> rows, DataSche
     return dataTableBuilder.build();
   }
 
+  /**
+   * The default value for each column type.
+   */
+  public static Serializable getDefaultValue(ColumnDataType dataType) {
+    switch (dataType) {
+      // Single-value column
+      case INT:
+        return THREAD_LOCAL_INT_FORMAT.get().format(((Number) 0).intValue());
+      case LONG:
+        return THREAD_LOCAL_LONG_FORMAT.get().format(((Number) 0).longValue());
+      case FLOAT:
+        return THREAD_LOCAL_FLOAT_FORMAT.get().format(((Number) 0).floatValue());
+      case DOUBLE:
+        return THREAD_LOCAL_DOUBLE_FORMAT.get().format(((Number) 0).doubleValue());

Review comment:
       Done




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751576834



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,725 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,
+    // a result is generated for each aggregation in the query,
+    // and the group by keys are not the same across the aggregations
+    // This PQL style of execution makes it impossible to support order by on group by.
+    //
+    // We could not simply change the group by execution behavior,
+    // as that would not be backward compatible for existing users of group by.
+    // As a result, we have 2 modes of group by execution - pql and sql - which can be controlled via query options
+    //
+    // Long term, we may completely move to sql, and keep only full sql mode alive
+    // Until then, we need to support responseFormat = sql for both the modes of execution.
+    // The 4 variants are as described below:
+
+    if (_groupByModeSql) {
+
+      if (_responseFormatSql) {
+        // 1. groupByMode = sql, responseFormat = sql
+        // This is the primary SQL compliant group by
+
+        try {
+          setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+              brokerMetrics);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // 2. groupByMode = sql, responseFormat = pql
+        // This mode will invoke SQL style group by execution, but present results in PQL way
+        // This mode is useful for users who want to avail of SQL compliant group by behavior,
+        // w/o having to forcefully move to a new result type
+
+        try {
+          setSQLGroupByInAggregationResults(brokerResponseNative, dataSchema, dataTables, reducerContext);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    } else {
+
+      // 3. groupByMode = pql, responseFormat = sql
+      // This mode is for users who want response presented in SQL style, but want PQL style group by behavior
+      // Multiple aggregations in PQL violates the tabular nature of results
+      // As a result, in this mode, only single aggregations are supported
+
+      // 4. groupByMode = pql, responseFormat = pql
+      // This is the primary PQL compliant group by
+
+      setGroupByResults(brokerResponseNative, dataTables);
+
+      if (_responseFormatSql) {
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // We emit the group by size when the result isn't empty. All the sizes among group-by results should be the
+        // same.
+        // Thus, we can just emit the one from the 1st result.
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    }
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    Object [] keyColumns = new Object[_numOfKeyColumns];
+    for (int i = 0; i < _numOfKeyColumns; i++) {
+      keyColumns[i] = row[i + 1];
+    }
+    return new Key(keyColumns);
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      int numResultColumns = resultColumnDataTypes.length;
+      int numResultRows = rows.size();
+      List<Object[]> resultRows = new ArrayList<>(numResultRows);
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < numResultColumns; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {
+      Set<Key> keys = new HashSet<>(_primaryKeys);
+      while (index < resultRows.size()) {
+        long timeCol = _dateTimeFormatter.fromFormatToMillis((String) resultRows.get(index)[0]);
+        if (timeCol < time) {
+          index++;
+        } else if (timeCol == time) {
+          gapfillResultRows.add(resultRows.get(index));
+          if (gapfillResultRows.size() == limit) {
+            return gapfillResultRows;
+          }
+          Key key = constructKey(resultRows.get(index));
+          keys.remove(key);
+          for (int colIndex = 2; colIndex < numResultColumns; colIndex++) {
+            _previous.put(key, resultRows.get(index));
+          }
+          index++;
+        } else {
+          break;
+        }
+      }
+      for (Key key : keys) {
+        Object[] gapfillRow = new Object[numResultColumns];
+        gapfillRow[0] = _dateTimeFormatter.fromMillisToFormat(time);
+        for (int i = 0; i < _numOfKeyColumns; i++) {
+          gapfillRow[i + 1] = key.getValues()[i];
+        }
+
+        for (int i = _numOfKeyColumns + 1; i < numResultColumns; i++) {
+          gapfillRow[i] = getFillValue(i, key, resultColumnDataTypes[i]);
+        }
+        gapfillResultRows.add(gapfillRow);
+        if (gapfillResultRows.size() == limit) {
+          return gapfillResultRows;
+        }
+      }
+    }
+    return gapfillResultRows;
+  }
+
+  Object getFillValue(int columIndex, Object key, ColumnDataType dataType) {
+    ExpressionContext expressionContext = _queryContext.getSelectExpressions().get(columIndex);
+    if (expressionContext.getFunction() != null
+        && expressionContext.getFunction().getFunctionName().equalsIgnoreCase("fill")) {
+      List<ExpressionContext> args = expressionContext.getFunction().getArguments();
+      if (args.get(1).getLiteral() == null) {
+        throw new UnsupportedOperationException("Wrong Sql.");
+      }
+      FillType fillType = FillType.valueOf(args.get(1).getLiteral());
+      if (fillType == FillType.FILL_DEFAULT_VALUE) {
+        // TODO: may fill the default value from sql in the future.
+        return SelectionOperatorUtils.getDefaultValue(dataType);
+      } else if (fillType == FillType.FILL_PREVIOUS_VALUE) {
+        if (_previous.containsKey(key)) {
+          return _previous.get(key)[columIndex];
+        } else {
+          return SelectionOperatorUtils.getDefaultValue(dataType);
+        }
+      } else {
+        throw new UnsupportedOperationException("unsupported fill type.");
+      }
+    } else {
+      return SelectionOperatorUtils.getDefaultValue(dataType);
+    }
+  }
+
+  enum FillType {
+    FILL_DEFAULT_VALUE,
+    FILL_PREVIOUS_VALUE,
+  }
+
+  /**
+   * Helper method to extract the final aggregation results for the given row (in-place).
+   */
+  private void extractFinalAggregationResults(Object[] row) {
+    for (int i = 0; i < _numAggregationFunctions; i++) {
+      int valueIndex = i + _numGroupByExpressions;
+      row[valueIndex] = _aggregationFunctions[i].extractFinalResult(row[valueIndex]);
+    }
+  }
+
+  /**
+   * Constructs the DataSchema for the rows before the post-aggregation (SQL mode).
+   */
+  private DataSchema getPrePostAggregationDataSchema(DataSchema dataSchema) {
+    String[] columnNames = dataSchema.getColumnNames();
+    ColumnDataType[] columnDataTypes = new ColumnDataType[_numColumns];
+    System.arraycopy(dataSchema.getColumnDataTypes(), 0, columnDataTypes, 0, _numGroupByExpressions);
+    for (int i = 0; i < _numAggregationFunctions; i++) {
+      columnDataTypes[i + _numGroupByExpressions] = _aggregationFunctions[i].getFinalResultColumnType();
+    }
+    return new DataSchema(columnNames, columnDataTypes);
+  }
+
+  private IndexedTable getIndexedTable(DataSchema dataSchema, Collection<DataTable> dataTablesToReduce,
+      DataTableReducerContext reducerContext)
+      throws TimeoutException {
+    long start = System.currentTimeMillis();
+    int numDataTables = dataTablesToReduce.size();
+
+    // Get the number of threads to use for reducing.
+    // In case of single reduce thread, fall back to SimpleIndexedTable to avoid redundant locking/unlocking calls.
+    int numReduceThreadsToUse = getNumReduceThreadsToUse(numDataTables, reducerContext.getMaxReduceThreadsPerQuery());
+    int limit = _queryContext.getLimit();
+    // TODO: Make minTrimSize configurable
+    int trimSize = GroupByUtils.getTableCapacity(limit);
+    // NOTE: For query with HAVING clause, use trimSize as resultSize to ensure the result accuracy.
+    // TODO: Resolve the HAVING clause within the IndexedTable before returning the result
+    int resultSize = _queryContext.getHavingFilter() != null ? trimSize : limit;
+    int trimThreshold = reducerContext.getGroupByTrimThreshold();
+    IndexedTable indexedTable;
+    if (numReduceThreadsToUse <= 1) {
+      indexedTable = new SimpleIndexedTable(dataSchema, _queryContext, resultSize, trimSize, trimThreshold);
+    } else {
+      if (trimThreshold >= GroupByOrderByCombineOperator.MAX_TRIM_THRESHOLD) {
+        // special case of trim threshold where it is set to max value.
+        // there won't be any trimming during upsert in this case.
+        // thus we can avoid the overhead of read-lock and write-lock
+        // in the upsert method.
+        indexedTable = new UnboundedConcurrentIndexedTable(dataSchema, _queryContext, resultSize);
+      } else {
+        indexedTable = new ConcurrentIndexedTable(dataSchema, _queryContext, resultSize, trimSize, trimThreshold);
+      }
+    }
+
+    Future[] futures = new Future[numDataTables];
+    CountDownLatch countDownLatch = new CountDownLatch(numDataTables);
+
+    // Create groups of data tables that each thread can process concurrently.
+    // Given that numReduceThreads is <= numDataTables, each group will have at least one data table.
+    ArrayList<DataTable> dataTables = new ArrayList<>(dataTablesToReduce);
+    List<List<DataTable>> reduceGroups = new ArrayList<>(numReduceThreadsToUse);
+
+    for (int i = 0; i < numReduceThreadsToUse; i++) {
+      reduceGroups.add(new ArrayList<>());
+    }
+    for (int i = 0; i < numDataTables; i++) {
+      reduceGroups.get(i % numReduceThreadsToUse).add(dataTables.get(i));
+    }
+
+    int cnt = 0;
+    ColumnDataType[] storedColumnDataTypes = dataSchema.getStoredColumnDataTypes();
+    for (List<DataTable> reduceGroup : reduceGroups) {
+      futures[cnt++] = reducerContext.getExecutorService().submit(new TraceRunnable() {
+        @Override
+        public void runJob() {
+          for (DataTable dataTable : reduceGroup) {
+            int numRows = dataTable.getNumberOfRows();
+
+            try {
+              for (int rowId = 0; rowId < numRows; rowId++) {
+                Object[] values = new Object[_numColumns];
+                for (int colId = 0; colId < _numColumns; colId++) {
+                  switch (storedColumnDataTypes[colId]) {
+                    case INT:
+                      values[colId] = dataTable.getInt(rowId, colId);
+                      break;
+                    case LONG:
+                      values[colId] = dataTable.getLong(rowId, colId);
+                      break;
+                    case FLOAT:
+                      values[colId] = dataTable.getFloat(rowId, colId);
+                      break;
+                    case DOUBLE:
+                      values[colId] = dataTable.getDouble(rowId, colId);
+                      break;
+                    case STRING:
+                      values[colId] = dataTable.getString(rowId, colId);
+                      break;
+                    case BYTES:
+                      values[colId] = dataTable.getBytes(rowId, colId);
+                      break;
+                    case OBJECT:
+                      values[colId] = dataTable.getObject(rowId, colId);
+                      break;
+                    // Add other aggregation intermediate result / group-by column type supports here
+                    default:
+                      throw new IllegalStateException();
+                  }
+                }
+                indexedTable.upsert(new Record(values));
+              }
+            } finally {
+              countDownLatch.countDown();
+            }
+          }
+        }
+      });
+    }
+
+    try {
+      long timeOutMs = reducerContext.getReduceTimeOutMs() - (System.currentTimeMillis() - start);
+      countDownLatch.await(timeOutMs, TimeUnit.MILLISECONDS);
+    } catch (InterruptedException e) {
+      for (Future future : futures) {
+        if (!future.isDone()) {
+          future.cancel(true);
+        }
+      }
+      throw new TimeoutException("Timed out in broker reduce phase.");
+    }
+
+    indexedTable.finish(true);
+    return indexedTable;
+  }
+
+  /**
+   * Computes the number of reduce threads to use per query.
+   * <ul>
+   *   <li> Use single thread if number of data tables to reduce is less than
+   *   {@value #MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE}.</li>
+   *   <li> Else, use min of max allowed reduce threads per query, and number of data tables.</li>
+   * </ul>
+   *
+   * @param numDataTables Number of data tables to reduce
+   * @param maxReduceThreadsPerQuery Max allowed reduce threads per query
+   * @return Number of reduce threads to use for the query
+   */
+  private int getNumReduceThreadsToUse(int numDataTables, int maxReduceThreadsPerQuery) {
+    // Use single thread if number of data tables < MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE.
+    if (numDataTables < MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE) {
+      return Math.min(1, numDataTables); // Number of data tables can be zero.
+    }
+
+    return Math.min(maxReduceThreadsPerQuery, numDataTables);
+  }
+
+  /**
+   * Extract the results of group by order by into a List of {@link AggregationResult}
+   * There will be 1 aggregation result per aggregation. The group by keys will be the same across all aggregations
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @throws TimeoutException If unable to complete within the timeout.
+   */
+  private void setSQLGroupByInAggregationResults(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext)
+      throws TimeoutException {
+
+    List<String> groupByColumns = new ArrayList<>(_numGroupByExpressions);
+    int idx = 0;
+    while (idx < _numGroupByExpressions) {
+      groupByColumns.add(dataSchema.getColumnName(idx));
+      idx++;
+    }
+
+    List<String> aggregationColumns = new ArrayList<>(_numAggregationFunctions);
+    List<List<GroupByResult>> groupByResults = new ArrayList<>(_numAggregationFunctions);
+    while (idx < _numColumns) {
+      aggregationColumns.add(dataSchema.getColumnName(idx));
+      groupByResults.add(new ArrayList<>());

Review comment:
       Done




-- 
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 edited a comment on pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#issuecomment-971204560


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?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 [#7781](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (24a5867) into [master](https://codecov.io/gh/apache/pinot/commit/09bb91cecc8cff459a9d7814a7fe61aa67f2a393?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (09bb91c) will **decrease** coverage by `57.11%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7781/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #7781       +/-   ##
   =============================================
   - Coverage     71.62%   14.51%   -57.12%     
   + Complexity     4079       80     -3999     
   =============================================
     Files          1578     1534       -44     
     Lines         80641    79137     -1504     
     Branches      11987    11857      -130     
   =============================================
   - Hits          57758    11485    -46273     
   - Misses        18994    66809    +47815     
   + Partials       3889      843     -3046     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `?` | |
   | unittests2 | `14.51% <0.00%> (-0.10%)` | :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/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...not/core/operator/transform/TransformOperator.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vVHJhbnNmb3JtT3BlcmF0b3IuamF2YQ==) | `0.00% <0.00%> (-95.00%)` | :arrow_down: |
   | [...r/transform/function/TransformFunctionFactory.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vVHJhbnNmb3JtRnVuY3Rpb25GYWN0b3J5LmphdmE=) | `0.00% <0.00%> (-84.62%)` | :arrow_down: |
   | [...e/query/reduce/GapFillGroupByDataTableReducer.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvR2FwRmlsbEdyb3VwQnlEYXRhVGFibGVSZWR1Y2VyLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...inot/core/query/reduce/PostAggregationHandler.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUG9zdEFnZ3JlZ2F0aW9uSGFuZGxlci5qYXZh) | `0.00% <0.00%> (-92.21%)` | :arrow_down: |
   | [.../pinot/core/query/reduce/ResultReducerFactory.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUmVzdWx0UmVkdWNlckZhY3RvcnkuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pinot/core/query/request/context/QueryContext.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZXF1ZXN0L2NvbnRleHQvUXVlcnlDb250ZXh0LmphdmE=) | `0.00% <0.00%> (-97.91%)` | :arrow_down: |
   | [...t/core/query/selection/SelectionOperatorUtils.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9zZWxlY3Rpb24vU2VsZWN0aW9uT3BlcmF0b3JVdGlscy5qYXZh) | `0.00% <0.00%> (-92.19%)` | :arrow_down: |
   | [...ain/java/org/apache/pinot/core/data/table/Key.java](https://codecov.io/gh/apache/pinot/pull/7781/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: |
   | [.../java/org/apache/pinot/spi/utils/BooleanUtils.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQm9vbGVhblV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/data/table/Record.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL1JlY29yZC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [1257 more](https://codecov.io/gh/apache/pinot/pull/7781/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [09bb91c...24a5867](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r753582094



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/operator/transform/TransformOperator.java
##########
@@ -56,7 +57,10 @@ public TransformOperator(@Nullable QueryContext queryContext, ProjectionOperator
     _projectionOperator = projectionOperator;
     _dataSourceMap = projectionOperator.getDataSourceMap();
     for (ExpressionContext expression : expressions) {
-      TransformFunction transformFunction = TransformFunctionFactory.get(queryContext, expression, _dataSourceMap);
+      TransformFunction transformFunction = TransformFunctionFactory.get(

Review comment:
       Done




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r753596668



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;

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] lakshmanan-v commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
lakshmanan-v commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r754807839



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,490 @@
+/**
+ * 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.reduce;
+
+import com.google.common.base.Preconditions;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.request.context.OrderByExpressionContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GapfillUtils;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _groupByKeys;
+  private final Map<Key, Object[]> _previousByGroupKey;
+  private final int _numOfGroupByKeys;
+  private final List<Integer> _groupByKeyIndexes;
+  private final boolean [] _isGroupBySelections;
+  private int _timeBucketIndex = -1;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    Preconditions.checkArgument(
+        queryContext.getBrokerRequest().getPinotQuery() != null, "GapFill can only be applied to sql query");
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+
+    ExpressionContext gapFillSelection = null;
+    for (ExpressionContext expressionContext : _queryContext.getSelectExpressions()) {
+      if (GapfillUtils.isPostAggregateGapfill(expressionContext)) {
+        gapFillSelection = expressionContext;
+        break;
+      }
+    }
+
+    List<ExpressionContext> args = gapFillSelection.getFunction().getArguments();
+    Preconditions.checkArgument(
+        args.size() == 5, "PostAggregateGapFill does not correct number of arguments.");
+    Preconditions.checkArgument(
+        args.get(1).getLiteral() != null, "The second argument of PostAggregateGapFill should be TimeFormatter.");
+    Preconditions.checkArgument(
+        args.get(2).getLiteral() != null, "The third argument of PostAggregateGapFill should be start time.");
+    Preconditions.checkArgument(
+        args.get(3).getLiteral() != null, "The fourth argument of PostAggregateGapFill should be end time.");
+    Preconditions.checkArgument(
+        args.get(4).getLiteral() != null, "The fifth argument of PostAggregateGapFill should be time bucket size.");
+
+    boolean orderByTimeBucket = false;
+    if (_queryContext.getOrderByExpressions() != null) {
+      for (OrderByExpressionContext expressionContext : _queryContext.getOrderByExpressions()) {
+        if (expressionContext.getExpression().equals(gapFillSelection)) {
+          orderByTimeBucket = true;
+          break;
+        }
+      }
+    }
+
+    Preconditions.checkArgument(
+        orderByTimeBucket, "PostAggregateGapFill does not work if the time bucket is not ordered.");

Review comment:
       It might break if its ordered by more than one field along with timebucket?




-- 
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 edited a comment on pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#issuecomment-971204560


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?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 [#7781](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (2bac553) into [master](https://codecov.io/gh/apache/pinot/commit/09bb91cecc8cff459a9d7814a7fe61aa67f2a393?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (09bb91c) will **decrease** coverage by `0.95%`.
   > The diff coverage is `73.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7781/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #7781      +/-   ##
   ============================================
   - Coverage     71.62%   70.67%   -0.96%     
   - Complexity     4079     4085       +6     
   ============================================
     Files          1578     1581       +3     
     Lines         80641    81107     +466     
     Branches      11987    12082      +95     
   ============================================
   - Hits          57758    57321     -437     
   - Misses        18994    19851     +857     
   - Partials       3889     3935      +46     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `29.13% <6.00%> (-0.05%)` | :arrow_down: |
   | integration2 | `?` | |
   | unittests1 | `68.70% <73.00%> (+0.09%)` | :arrow_up: |
   | unittests2 | `14.51% <0.00%> (-0.09%)` | :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/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../java/org/apache/pinot/core/util/GapfillUtils.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS91dGlsL0dhcGZpbGxVdGlscy5qYXZh) | `47.82% <47.82%> (ø)` | |
   | [...e/query/reduce/GapFillGroupByDataTableReducer.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvR2FwRmlsbEdyb3VwQnlEYXRhVGFibGVSZWR1Y2VyLmphdmE=) | `76.34% <76.34%> (ø)` | |
   | [...not/core/operator/transform/TransformOperator.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vVHJhbnNmb3JtT3BlcmF0b3IuamF2YQ==) | `95.23% <100.00%> (+0.23%)` | :arrow_up: |
   | [...inot/core/query/reduce/PostAggregationHandler.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUG9zdEFnZ3JlZ2F0aW9uSGFuZGxlci5qYXZh) | `92.40% <100.00%> (+0.19%)` | :arrow_up: |
   | [.../pinot/core/query/reduce/ResultReducerFactory.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUmVzdWx0UmVkdWNlckZhY3RvcnkuamF2YQ==) | `100.00% <100.00%> (ø)` | |
   | [...pinot/core/query/request/context/QueryContext.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZXF1ZXN0L2NvbnRleHQvUXVlcnlDb250ZXh0LmphdmE=) | `97.95% <100.00%> (+0.05%)` | :arrow_up: |
   | [...ore/operator/streaming/StreamingResponseUtils.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9zdHJlYW1pbmcvU3RyZWFtaW5nUmVzcG9uc2VVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ager/realtime/PeerSchemeSplitSegmentCommitter.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvUGVlclNjaGVtZVNwbGl0U2VnbWVudENvbW1pdHRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pache/pinot/common/utils/grpc/GrpcQueryClient.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvZ3JwYy9HcnBjUXVlcnlDbGllbnQuamF2YQ==) | `0.00% <0.00%> (-94.74%)` | :arrow_down: |
   | [...ator/streaming/StreamingSelectionOnlyOperator.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9zdHJlYW1pbmcvU3RyZWFtaW5nU2VsZWN0aW9uT25seU9wZXJhdG9yLmphdmE=) | `0.00% <0.00%> (-92.31%)` | :arrow_down: |
   | ... and [117 more](https://codecov.io/gh/apache/pinot/pull/7781/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [09bb91c...2bac553](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov-commenter edited a comment on pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#issuecomment-971204560


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?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 [#7781](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (09b84f6) into [master](https://codecov.io/gh/apache/pinot/commit/09bb91cecc8cff459a9d7814a7fe61aa67f2a393?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (09bb91c) will **decrease** coverage by `1.18%`.
   > The diff coverage is `76.29%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7781/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #7781      +/-   ##
   ============================================
   - Coverage     71.62%   70.43%   -1.19%     
   - Complexity     4079     4087       +8     
   ============================================
     Files          1578     1582       +4     
     Lines         80641    81371     +730     
     Branches      11987    12129     +142     
   ============================================
   - Hits          57758    57313     -445     
   - Misses        18994    20120    +1126     
   - Partials       3889     3938      +49     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `27.81% <5.55%> (-0.05%)` | :arrow_down: |
   | unittests1 | `68.80% <76.29%> (+0.19%)` | :arrow_up: |
   | unittests2 | `14.51% <0.00%> (-0.09%)` | :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/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...not/core/operator/transform/TransformOperator.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vVHJhbnNmb3JtT3BlcmF0b3IuamF2YQ==) | `93.54% <ø> (-1.46%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/util/GapfillUtils.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS91dGlsL0dhcGZpbGxVdGlscy5qYXZh) | `63.63% <63.63%> (ø)` | |
   | [...e/query/reduce/GapFillGroupByDataTableReducer.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvR2FwRmlsbEdyb3VwQnlEYXRhVGFibGVSZWR1Y2VyLmphdmE=) | `77.77% <77.77%> (ø)` | |
   | [...inot/core/query/reduce/PostAggregationHandler.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUG9zdEFnZ3JlZ2F0aW9uSGFuZGxlci5qYXZh) | `92.30% <100.00%> (+0.09%)` | :arrow_up: |
   | [.../pinot/core/query/reduce/ResultReducerFactory.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUmVzdWx0UmVkdWNlckZhY3RvcnkuamF2YQ==) | `92.30% <100.00%> (-7.70%)` | :arrow_down: |
   | [...pinot/minion/exception/TaskCancelledException.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtbWluaW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9taW5pb24vZXhjZXB0aW9uL1Rhc2tDYW5jZWxsZWRFeGNlcHRpb24uamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...nverttorawindex/ConvertToRawIndexTaskExecutor.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtcGx1Z2lucy9waW5vdC1taW5pb24tdGFza3MvcGlub3QtbWluaW9uLWJ1aWx0aW4tdGFza3Mvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3BsdWdpbi9taW5pb24vdGFza3MvY29udmVydHRvcmF3aW5kZXgvQ29udmVydFRvUmF3SW5kZXhUYXNrRXhlY3V0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...e/pinot/common/minion/MergeRollupTaskMetadata.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWluaW9uL01lcmdlUm9sbHVwVGFza01ldGFkYXRhLmphdmE=) | `0.00% <0.00%> (-94.74%)` | :arrow_down: |
   | [...plugin/segmentuploader/SegmentUploaderDefault.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtcGx1Z2lucy9waW5vdC1zZWdtZW50LXVwbG9hZGVyL3Bpbm90LXNlZ21lbnQtdXBsb2FkZXItZGVmYXVsdC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcGx1Z2luL3NlZ21lbnR1cGxvYWRlci9TZWdtZW50VXBsb2FkZXJEZWZhdWx0LmphdmE=) | `0.00% <0.00%> (-87.10%)` | :arrow_down: |
   | [.../transform/function/MapValueTransformFunction.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vTWFwVmFsdWVUcmFuc2Zvcm1GdW5jdGlvbi5qYXZh) | `0.00% <0.00%> (-85.30%)` | :arrow_down: |
   | ... and [213 more](https://codecov.io/gh/apache/pinot/pull/7781/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [09bb91c...09b84f6](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov-commenter edited a comment on pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#issuecomment-971204560


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?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 [#7781](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (442f220) into [master](https://codecov.io/gh/apache/pinot/commit/09bb91cecc8cff459a9d7814a7fe61aa67f2a393?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (09bb91c) will **decrease** coverage by `1.32%`.
   > The diff coverage is `41.87%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7781/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #7781      +/-   ##
   ============================================
   - Coverage     71.62%   70.29%   -1.33%     
     Complexity     4079     4079              
   ============================================
     Files          1578     1579       +1     
     Lines         80641    81044     +403     
     Branches      11987    12069      +82     
   ============================================
   - Hits          57758    56973     -785     
   - Misses        18994    20163    +1169     
   - Partials       3889     3908      +19     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `27.77% <2.70%> (-0.08%)` | :arrow_down: |
   | unittests1 | `68.43% <41.87%> (-0.18%)` | :arrow_down: |
   | unittests2 | `14.51% <0.00%> (-0.10%)` | :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/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...t/core/query/selection/SelectionOperatorUtils.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9zZWxlY3Rpb24vU2VsZWN0aW9uT3BlcmF0b3JVdGlscy5qYXZh) | `82.55% <2.70%> (-9.63%)` | :arrow_down: |
   | [...e/query/reduce/GapFillGroupByDataTableReducer.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvR2FwRmlsbEdyb3VwQnlEYXRhVGFibGVSZWR1Y2VyLmphdmE=) | `43.46% <43.46%> (ø)` | |
   | [...pinot/core/query/request/context/QueryContext.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZXF1ZXN0L2NvbnRleHQvUXVlcnlDb250ZXh0LmphdmE=) | `97.42% <66.66%> (-0.49%)` | :arrow_down: |
   | [...not/core/operator/transform/TransformOperator.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vVHJhbnNmb3JtT3BlcmF0b3IuamF2YQ==) | `95.23% <100.00%> (+0.23%)` | :arrow_up: |
   | [...r/transform/function/TransformFunctionFactory.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vVHJhbnNmb3JtRnVuY3Rpb25GYWN0b3J5LmphdmE=) | `85.58% <100.00%> (+0.97%)` | :arrow_up: |
   | [...inot/core/query/reduce/PostAggregationHandler.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUG9zdEFnZ3JlZ2F0aW9uSGFuZGxlci5qYXZh) | `92.30% <100.00%> (+0.09%)` | :arrow_up: |
   | [.../pinot/core/query/reduce/ResultReducerFactory.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUmVzdWx0UmVkdWNlckZhY3RvcnkuamF2YQ==) | `100.00% <100.00%> (ø)` | |
   | [...pinot/minion/exception/TaskCancelledException.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtbWluaW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9taW5pb24vZXhjZXB0aW9uL1Rhc2tDYW5jZWxsZWRFeGNlcHRpb24uamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...nverttorawindex/ConvertToRawIndexTaskExecutor.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtcGx1Z2lucy9waW5vdC1taW5pb24tdGFza3MvcGlub3QtbWluaW9uLWJ1aWx0aW4tdGFza3Mvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3BsdWdpbi9taW5pb24vdGFza3MvY29udmVydHRvcmF3aW5kZXgvQ29udmVydFRvUmF3SW5kZXhUYXNrRXhlY3V0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...e/pinot/common/minion/MergeRollupTaskMetadata.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWluaW9uL01lcmdlUm9sbHVwVGFza01ldGFkYXRhLmphdmE=) | `0.00% <0.00%> (-94.74%)` | :arrow_down: |
   | ... and [103 more](https://codecov.io/gh/apache/pinot/pull/7781/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [09bb91c...442f220](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r752485910



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/TransformFunctionFactory.java
##########
@@ -198,6 +202,18 @@ public static TransformFunction get(ExpressionContext expression, Map<String, Da
     return get(null, expression, dataSourceMap);
   }
 
+  public static ExpressionContext stripGapfill(ExpressionContext expression) {

Review comment:
       Done




-- 
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] richardstartin commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751141305



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,725 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,
+    // a result is generated for each aggregation in the query,
+    // and the group by keys are not the same across the aggregations
+    // This PQL style of execution makes it impossible to support order by on group by.
+    //
+    // We could not simply change the group by execution behavior,
+    // as that would not be backward compatible for existing users of group by.
+    // As a result, we have 2 modes of group by execution - pql and sql - which can be controlled via query options
+    //
+    // Long term, we may completely move to sql, and keep only full sql mode alive
+    // Until then, we need to support responseFormat = sql for both the modes of execution.
+    // The 4 variants are as described below:
+
+    if (_groupByModeSql) {
+
+      if (_responseFormatSql) {
+        // 1. groupByMode = sql, responseFormat = sql
+        // This is the primary SQL compliant group by
+
+        try {
+          setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+              brokerMetrics);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // 2. groupByMode = sql, responseFormat = pql
+        // This mode will invoke SQL style group by execution, but present results in PQL way
+        // This mode is useful for users who want to avail of SQL compliant group by behavior,
+        // w/o having to forcefully move to a new result type
+
+        try {
+          setSQLGroupByInAggregationResults(brokerResponseNative, dataSchema, dataTables, reducerContext);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    } else {
+
+      // 3. groupByMode = pql, responseFormat = sql
+      // This mode is for users who want response presented in SQL style, but want PQL style group by behavior
+      // Multiple aggregations in PQL violates the tabular nature of results
+      // As a result, in this mode, only single aggregations are supported
+
+      // 4. groupByMode = pql, responseFormat = pql
+      // This is the primary PQL compliant group by
+
+      setGroupByResults(brokerResponseNative, dataTables);
+
+      if (_responseFormatSql) {
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // We emit the group by size when the result isn't empty. All the sizes among group-by results should be the
+        // same.
+        // Thus, we can just emit the one from the 1st result.
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    }
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    Object [] keyColumns = new Object[_numOfKeyColumns];
+    for (int i = 0; i < _numOfKeyColumns; i++) {
+      keyColumns[i] = row[i + 1];
+    }
+    return new Key(keyColumns);
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      int numResultColumns = resultColumnDataTypes.length;
+      int numResultRows = rows.size();
+      List<Object[]> resultRows = new ArrayList<>(numResultRows);
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < numResultColumns; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {
+      Set<Key> keys = new HashSet<>(_primaryKeys);
+      while (index < resultRows.size()) {
+        long timeCol = _dateTimeFormatter.fromFormatToMillis((String) resultRows.get(index)[0]);
+        if (timeCol < time) {
+          index++;
+        } else if (timeCol == time) {
+          gapfillResultRows.add(resultRows.get(index));
+          if (gapfillResultRows.size() == limit) {
+            return gapfillResultRows;
+          }
+          Key key = constructKey(resultRows.get(index));
+          keys.remove(key);
+          for (int colIndex = 2; colIndex < numResultColumns; colIndex++) {
+            _previous.put(key, resultRows.get(index));
+          }
+          index++;
+        } else {
+          break;
+        }
+      }
+      for (Key key : keys) {
+        Object[] gapfillRow = new Object[numResultColumns];
+        gapfillRow[0] = _dateTimeFormatter.fromMillisToFormat(time);
+        for (int i = 0; i < _numOfKeyColumns; i++) {
+          gapfillRow[i + 1] = key.getValues()[i];
+        }

Review comment:
       `Object[] gapfillRow = Arrays.copyOfRange(key.getValues(), 1, numResultColumns)`




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r753583360



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/GapfillUtil.java
##########
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.util;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FunctionContext;
+
+
+/**
+ * Util class to encapsulate all utilites required for gapfill.
+ */
+public class GapfillUtil {

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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r753585464



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/GapfillUtil.java
##########
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.util;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FunctionContext;
+
+
+/**
+ * Util class to encapsulate all utilites required for gapfill.
+ */
+public class GapfillUtil {
+  private static final String AGGREGATE_GAP_FILL = "aggregategapfill";

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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r753592068



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/PostAggregationHandler.java
##########
@@ -67,7 +68,8 @@ public PostAggregationHandler(QueryContext queryContext, DataSchema dataSchema)
     String[] columnNames = new String[numSelectExpressions];
     ColumnDataType[] columnDataTypes = new ColumnDataType[numSelectExpressions];
     for (int i = 0; i < numSelectExpressions; i++) {
-      ValueExtractor valueExtractor = getValueExtractor(selectExpressions.get(i));
+      ValueExtractor valueExtractor
+          = getValueExtractor(GapfillUtil.stripGapfill(selectExpressions.get(i)));

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] Jackie-Jiang commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

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



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/request/context/QueryContext.java
##########
@@ -201,6 +202,14 @@ public int getOffset() {
     return _offset;
   }
 
+  public boolean isAggregateGapfill() {

Review comment:
       Let's move this method to the `QueryContextUtils` as a util method

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/GapfillUtils.java
##########
@@ -0,0 +1,135 @@
+/**
+ * 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.util;
+
+import java.io.Serializable;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FunctionContext;
+import org.apache.pinot.common.utils.DataSchema;
+
+/**
+ * Util class to encapsulate all utilites required for gapfill.
+ */
+public class GapfillUtils {
+  private static final String POST_AGGREGATE_GAP_FILL = "postaggregategapfill";
+  private static final String FILL = "fill";
+
+  private GapfillUtils() {
+  }
+
+  public static ExpressionContext stripGapfill(ExpressionContext expression) {
+    if (expression.getType() != ExpressionContext.Type.FUNCTION) {
+      return expression;
+    }
+
+    FunctionContext function = expression.getFunction();
+    String functionName = canonicalizeFunctionName(function.getFunctionName());
+    if (functionName.equals(POST_AGGREGATE_GAP_FILL) || functionName.equals(FILL)) {
+      return function.getArguments().get(0);
+    }
+    return expression;
+  }
+
+  public static boolean isPostAggregateGapfill(ExpressionContext expressionContext) {
+    if (expressionContext.getType() != ExpressionContext.Type.FUNCTION) {
+      return false;
+    }
+
+    return POST_AGGREGATE_GAP_FILL.equals(canonicalizeFunctionName(expressionContext.getFunction().getFunctionName()));
+  }
+
+  public static boolean isFill(ExpressionContext expressionContext) {
+    if (expressionContext.getType() != ExpressionContext.Type.FUNCTION) {
+      return false;
+    }
+
+    return FILL.equals(canonicalizeFunctionName(expressionContext.getFunction().getFunctionName()));
+  }
+
+  static public enum FillType {
+    FILL_DEFAULT_VALUE,
+    FILL_PREVIOUS_VALUE,
+  }
+
+  /**
+   * The default gapfill value for each column type.
+   */
+  static public Serializable getDefaultValue(DataSchema.ColumnDataType dataType) {
+    switch (dataType) {
+      // Single-value column
+      case INT:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case BOOLEAN:
+      case TIMESTAMP:
+        return dataType.convertAndFormat(0);
+      case STRING:
+      case JSON:
+      case BYTES:
+        return "";
+      case INT_ARRAY:
+        return new int[0];
+      case LONG_ARRAY:
+        return new long[0];
+      case FLOAT_ARRAY:
+        return new float[0];
+      case DOUBLE_ARRAY:
+        return new double[0];
+      case STRING_ARRAY:
+      case TIMESTAMP_ARRAY:
+        return new String[0];
+      case BOOLEAN_ARRAY:
+        return new boolean[0];
+      case BYTES_ARRAY:
+        return new byte[0][0];
+      default:
+        throw new IllegalStateException(String.format("Cannot provide the default value for the type: %s", dataType));
+    }
+  }
+
+  public static boolean isTimeBucketTimeFunction(ExpressionContext expressionContext) {

Review comment:
       I feel this validation is not required and could reject some valid queries. The first argument doesn't have to be a time bucket function. It can be a time column (e.g. `hoursSinceEpoch`) or some other udf (e.g. `round`)

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/PostAggregationHandler.java
##########
@@ -67,7 +68,8 @@ public PostAggregationHandler(QueryContext queryContext, DataSchema dataSchema)
     String[] columnNames = new String[numSelectExpressions];
     ColumnDataType[] columnDataTypes = new ColumnDataType[numSelectExpressions];
     for (int i = 0; i < numSelectExpressions; i++) {
-      ValueExtractor valueExtractor = getValueExtractor(selectExpressions.get(i));
+      ValueExtractor valueExtractor
+          = getValueExtractor(selectExpressions.get(i));

Review comment:
       (minor) revert

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,494 @@
+/**
+ * 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.reduce;
+
+import com.google.common.base.Preconditions;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.request.context.OrderByExpressionContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GapfillUtils;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _groupByKeys;
+  private final Map<Key, Object[]> _previousByGroupKey;
+  private final int _numOfGroupByKeys;
+  private final List<Integer> _groupByKeyIndexes;
+  private final boolean [] _isGroupBySelections;
+  private int _timeBucketIndex = -1;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    Preconditions.checkArgument(
+        queryContext.getBrokerRequest().getPinotQuery() != null, "GapFill can only be applied to sql query");
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+
+    ExpressionContext gapFillSelection = null;
+    for (ExpressionContext expressionContext : _queryContext.getSelectExpressions()) {
+      if (GapfillUtils.isPostAggregateGapfill(expressionContext)) {
+        gapFillSelection = expressionContext;
+        break;
+      }
+    }
+
+    List<ExpressionContext> args = gapFillSelection.getFunction().getArguments();
+    Preconditions.checkArgument(
+        args.size() == 5, "PostAggregateGapFill does not have correct number of arguments.");
+    Preconditions.checkArgument(
+        args.get(1).getLiteral() != null, "The second argument of PostAggregateGapFill should be TimeFormatter.");
+    Preconditions.checkArgument(
+        args.get(2).getLiteral() != null, "The third argument of PostAggregateGapFill should be start time.");
+    Preconditions.checkArgument(
+        args.get(3).getLiteral() != null, "The fourth argument of PostAggregateGapFill should be end time.");
+    Preconditions.checkArgument(
+        args.get(4).getLiteral() != null, "The fifth argument of PostAggregateGapFill should be time bucket size.");
+
+    Preconditions.checkArgument(
+        GapfillUtils.isTimeBucketTimeFunction(args.get(0)),
+        "The first argument of PostAggregateGapFill should be timeBucket Function.");
+
+    boolean orderByTimeBucket = false;
+    if (_queryContext.getOrderByExpressions() != null) {
+      for (OrderByExpressionContext expressionContext : _queryContext.getOrderByExpressions()) {
+        if (expressionContext.getExpression().equals(gapFillSelection)) {

Review comment:
       Should we allow ordering on the first argument as well?
   E.g. `select postaggregategapfill(timeCol, ...) ... order by timeCol ...`
   
   We should also check if it is the first order by expression, and in ascending order
   

##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/PostAggregationGapfillQueriesTest.java
##########
@@ -0,0 +1,592 @@
+/**
+ * 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.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+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.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+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.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+
+/**
+ * Queries test for PostAggregationGapfill queries.
+ */
+@SuppressWarnings("rawtypes")
+public class PostAggregationGapfillQueriesTest extends BaseQueriesTest {
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "PostAggregationGapfillQueriesTest");
+  private static final String RAW_TABLE_NAME = "parkingData";
+  private static final String SEGMENT_NAME = "testSegment";
+  private static final Random RANDOM = new Random();
+
+  private static final int NUM_LOTS = 4;
+
+  private static final String IS_OCCUPIED_COLUMN = "isOccupied";
+  private static final String LOT_ID_COLUMN = "lotId";
+  private static final String EVENT_TIME_COLUMN = "eventTime";
+  private static final Schema SCHEMA = new Schema.SchemaBuilder()
+      .addSingleValueDimension(IS_OCCUPIED_COLUMN, DataType.BOOLEAN)
+      .addSingleValueDimension(LOT_ID_COLUMN, DataType.STRING)
+      .addSingleValueDimension(EVENT_TIME_COLUMN, DataType.LONG)
+      .setPrimaryKeyColumns(Arrays.asList(LOT_ID_COLUMN, EVENT_TIME_COLUMN))
+      .build();
+  private static final TableConfig TABLE_CONFIG = new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME)
+      .build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  @Override
+  protected String getFilter() {
+    // NOTE: Use a match all filter to switch between DictionaryBasedAggregationOperator and AggregationOperator
+    return " WHERE eventTime >= 0";
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+
+    long current = 1636286400000L; //November 7, 2021 12:00:00 PM
+    int duplicates = 16;
+    int interval = 1000 * 900; // 15 minutes
+    long start = current - duplicates * 2 * interval; //November 7, 2021 4:00:00 AM
+
+    List<GenericRow> records = new ArrayList<>(NUM_LOTS * 2);
+    for (int i = 0; i < NUM_LOTS; i++) {
+      for (int j = 0; j < duplicates; j++) {
+        if (j == 4 || j == 5 || j == 6 || j == 7 || j == 10 || j == 11) {
+          continue;
+        }
+        long parkingTime = start + interval * 2 * j + RANDOM.nextInt(interval);
+        long departingTime = j == 3 ? start + interval * (2 * j + 6) + RANDOM.nextInt(interval) : start
+            + interval * (2 * j + 1) + RANDOM.nextInt(interval);
+
+        GenericRow parkingRow = new GenericRow();
+        parkingRow.putValue(EVENT_TIME_COLUMN, parkingTime);
+        parkingRow.putValue(LOT_ID_COLUMN, "LotId_" + String.valueOf(i));
+        parkingRow.putValue(IS_OCCUPIED_COLUMN, true);
+        records.add(parkingRow);
+
+        GenericRow departingRow = new GenericRow();
+        departingRow.putValue(EVENT_TIME_COLUMN, departingTime);
+        departingRow.putValue(LOT_ID_COLUMN, "LotId_" + String.valueOf(i));
+        departingRow.putValue(IS_OCCUPIED_COLUMN, false);
+        records.add(departingRow);
+      }
+    }
+
+    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);
+  }
+
+  @Test
+  public void datetimeconvertGapfillTest() {
+    String dataTimeConvertQuery = "SELECT "
+        + "DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN')"
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    BrokerResponseNative dateTimeConvertBrokerResponse = getBrokerResponseForSqlQuery(dataTimeConvertQuery);
+
+    ResultTable dateTimeConvertResultTable = dateTimeConvertBrokerResponse.getResultTable();
+    Assert.assertEquals(dateTimeConvertResultTable.getRows().size(), 24);
+
+    String gapfillQuery = "SELECT "
+        + "PostAggregateGapFill(DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS'), "
+        + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+        + "'2021-11-07 3:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_PREVIOUS_VALUE') as status1, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_DEFAULT_VALUE') as status2, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN') as status3 "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    DateTimeFormatSpec dateTimeFormatter
+        = new DateTimeFormatSpec("1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+
+    BrokerResponseNative gapfillBrokerResponse = getBrokerResponseForSqlQuery(gapfillQuery);
+
+    ResultTable gapFillResultTable = gapfillBrokerResponse.getResultTable();
+    Assert.assertEquals(gapFillResultTable.getRows().size(), 32);
+    List<Object[]> gapFillRows = gapFillResultTable.getRows();
+    long start = dateTimeFormatter.fromFormatToMillis("2021-11-07 03:00:00.000");
+    for (int i = 0; i < 32; i += 4) {
+      String firstTimeCol = (String) gapFillRows.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Set<String> lots = new HashSet<>();
+      lots.add((String) gapFillRows.get(i)[1]);
+      for (int j = 1; j < 4; j++) {
+        Assert.assertEquals(gapFillRows.get(i)[0], gapFillRows.get(i + j)[0]);
+        Assert.assertFalse(lots.contains(gapFillRows.get(i + j)[1]));
+        lots.add((String) gapFillRows.get(i + j)[1]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochHoursGapfillTest() {
+    String dataTimeConvertQuery = "SELECT "
+        + "ToEpochHours(eventTime) AS time_col, "
+        + "lotId, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN')"
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    BrokerResponseNative dateTimeConvertBrokerResponse = getBrokerResponseForSqlQuery(dataTimeConvertQuery);
+
+    ResultTable dateTimeConvertResultTable = dateTimeConvertBrokerResponse.getResultTable();
+    Assert.assertEquals(dateTimeConvertResultTable.getRows().size(), 24);
+
+    String gapfillQuery = "SELECT "
+        + "PostAggregateGapFill(ToEpochHours(eventTime), '1:HOURS:EPOCH', "
+        + "'454515',  '454524', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_PREVIOUS_VALUE') as status1, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_DEFAULT_VALUE') as status2, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN') as status3 "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+
+    BrokerResponseNative gapfillBrokerResponse = getBrokerResponseForSqlQuery(gapfillQuery);
+
+    ResultTable gapFillResultTable = gapfillBrokerResponse.getResultTable();
+    Assert.assertEquals(gapFillResultTable.getRows().size(), 32);
+    List<Object[]> gapFillRows = gapFillResultTable.getRows();
+    long start = dateTimeFormatter.fromFormatToMillis("454515");
+    for (int i = 0; i < 32; i += 4) {
+      Long firstTimeCol = (Long) gapFillRows.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol.toString());
+      Assert.assertEquals(timeStamp, start);
+      Set<String> lots = new HashSet<>();
+      lots.add((String) gapFillRows.get(i)[1]);
+      for (int j = 1; j < 4; j++) {
+        Assert.assertEquals(gapFillRows.get(i)[0], gapFillRows.get(i + j)[0]);
+        Assert.assertFalse(lots.contains(gapFillRows.get(i + j)[1]));
+        lots.add((String) gapFillRows.get(i + j)[1]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochMinutesRoundedHoursGapfillTest() {
+    String dataTimeConvertQuery = "SELECT "
+        + "ToEpochMinutesRounded(eventTime, 60) AS time_col, "
+        + "lotId, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN')"
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    BrokerResponseNative dateTimeConvertBrokerResponse = getBrokerResponseForSqlQuery(dataTimeConvertQuery);
+
+    ResultTable dateTimeConvertResultTable = dateTimeConvertBrokerResponse.getResultTable();
+    Assert.assertEquals(dateTimeConvertResultTable.getRows().size(), 24);
+
+    String gapfillQuery = "SELECT "
+        + "PostAggregateGapFill(ToEpochMinutesRounded(eventTime, 60), '1:HOURS:EPOCH', "
+        + "'454515',  '454524', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_PREVIOUS_VALUE') as status1, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_DEFAULT_VALUE') as status2, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN') as status3 "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+
+    BrokerResponseNative gapfillBrokerResponse = getBrokerResponseForSqlQuery(gapfillQuery);
+
+    ResultTable gapFillResultTable = gapfillBrokerResponse.getResultTable();
+    Assert.assertEquals(gapFillResultTable.getRows().size(), 32);
+    List<Object[]> gapFillRows = gapFillResultTable.getRows();
+    long start = dateTimeFormatter.fromFormatToMillis("454515");
+    for (int i = 0; i < 32; i += 4) {
+      Long firstTimeCol = (Long) gapFillRows.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol.toString());
+      Assert.assertEquals(timeStamp, start);
+      Set<String> lots = new HashSet<>();
+      lots.add((String) gapFillRows.get(i)[1]);
+      for (int j = 1; j < 4; j++) {
+        Assert.assertEquals(gapFillRows.get(i)[0], gapFillRows.get(i + j)[0]);
+        Assert.assertFalse(lots.contains(gapFillRows.get(i + j)[1]));
+        lots.add((String) gapFillRows.get(i + j)[1]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochMinutesBucketHoursGapfillTest() {
+    String dataTimeConvertQuery = "SELECT "
+        + "ToEpochMinutesBucket(eventTime, 60) AS time_col, "
+        + "lotId, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN')"
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    BrokerResponseNative dateTimeConvertBrokerResponse = getBrokerResponseForSqlQuery(dataTimeConvertQuery);
+
+    ResultTable dateTimeConvertResultTable = dateTimeConvertBrokerResponse.getResultTable();
+    Assert.assertEquals(dateTimeConvertResultTable.getRows().size(), 24);
+
+    String gapfillQuery = "SELECT "
+        + "PostAggregateGapFill(ToEpochMinutesBucket(eventTime, 60), '1:HOURS:EPOCH', "
+        + "'454515',  '454524', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_PREVIOUS_VALUE') as status1, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_DEFAULT_VALUE') as status2, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN') as status3 "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+
+    BrokerResponseNative gapfillBrokerResponse = getBrokerResponseForSqlQuery(gapfillQuery);
+
+    ResultTable gapFillResultTable = gapfillBrokerResponse.getResultTable();
+    Assert.assertEquals(gapFillResultTable.getRows().size(), 32);
+    List<Object[]> gapFillRows = gapFillResultTable.getRows();
+    long start = dateTimeFormatter.fromFormatToMillis("454515");
+    for (int i = 0; i < 32; i += 4) {
+      Long firstTimeCol = (Long) gapFillRows.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol.toString());
+      Assert.assertEquals(timeStamp, start);
+      Set<String> lots = new HashSet<>();
+      lots.add((String) gapFillRows.get(i)[1]);
+      for (int j = 1; j < 4; j++) {
+        Assert.assertEquals(gapFillRows.get(i)[0], gapFillRows.get(i + j)[0]);
+        Assert.assertFalse(lots.contains(gapFillRows.get(i + j)[1]));
+        lots.add((String) gapFillRows.get(i + j)[1]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void dateTruncHoursGapfillTest() {
+    String dataTimeConvertQuery = "SELECT "
+        + "DATETRUNC('hour', eventTime, 'milliseconds') AS time_col, "
+        + "lotId, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN')"
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    BrokerResponseNative dateTimeConvertBrokerResponse = getBrokerResponseForSqlQuery(dataTimeConvertQuery);
+
+    ResultTable dateTimeConvertResultTable = dateTimeConvertBrokerResponse.getResultTable();
+    Assert.assertEquals(dateTimeConvertResultTable.getRows().size(), 24);
+
+    String gapfillQuery = "SELECT "
+        + "PostAggregateGapFill(DATETRUNC('hour', eventTime, 'milliseconds'), '1:HOURS:EPOCH', "
+        + "'454515',  '454524', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_PREVIOUS_VALUE') as status1, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_DEFAULT_VALUE') as status2, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN') as status3 "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+
+    BrokerResponseNative gapfillBrokerResponse = getBrokerResponseForSqlQuery(gapfillQuery);
+
+    ResultTable gapFillResultTable = gapfillBrokerResponse.getResultTable();
+    Assert.assertEquals(gapFillResultTable.getRows().size(), 32);
+    List<Object[]> gapFillRows = gapFillResultTable.getRows();
+    long start = dateTimeFormatter.fromFormatToMillis("454515");
+    for (int i = 0; i < 32; i += 4) {
+      Long firstTimeCol = (Long) gapFillRows.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol.toString());
+      Assert.assertEquals(timeStamp, start);
+      Set<String> lots = new HashSet<>();
+      lots.add((String) gapFillRows.get(i)[1]);
+      for (int j = 1; j < 4; j++) {
+        Assert.assertEquals(gapFillRows.get(i)[0], gapFillRows.get(i + j)[0]);
+        Assert.assertFalse(lots.contains(gapFillRows.get(i + j)[1]));
+        lots.add((String) gapFillRows.get(i + j)[1]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void datetimeconvertGapfillTestWithoutTimeBucketOrdering() {
+    try {
+      String gapfillQuery = "SELECT "
+          + "PostAggregateGapFill(DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+          + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS'), "
+          + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+          + "'2021-11-07 3:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS') AS time_col, "
+          + "lotId, "
+          + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_PREVIOUS_VALUE') as status1, "
+          + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_DEFAULT_VALUE') as status2, "
+          + "lastWithTime(isOccupied, eventTime, 'BOOLEAN') as status3 "
+          + "FROM parkingData "
+          + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+          + "GROUP BY 1, 2 "
+          + "LIMIT 200";
+
+      getBrokerResponseForSqlQuery(gapfillQuery);
+      Assert.fail();
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(e.getMessage(), "PostAggregateGapFill does not work if the time bucket is not ordered.");
+    }
+  }
+
+  @Test
+  public void datetimeconvertGapfillTestWithHavingClause() {
+    String dataTimeConvertQuery = "SELECT "
+        + "DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN')"
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "HAVING lotId IN ('LotId_0', 'LotId_1', 'LotId_2') "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    BrokerResponseNative dateTimeConvertBrokerResponse = getBrokerResponseForSqlQuery(dataTimeConvertQuery);
+
+    ResultTable dateTimeConvertResultTable = dateTimeConvertBrokerResponse.getResultTable();
+    Assert.assertEquals(dateTimeConvertResultTable.getRows().size(), 18);
+
+    String gapfillQuery = "SELECT "
+        + "PostAggregateGapFill(DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS'), "
+        + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+        + "'2021-11-07 3:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_PREVIOUS_VALUE') as status1, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_DEFAULT_VALUE') as status2, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN') as status3 "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "HAVING lotId IN ('LotId_0', 'LotId_1', 'LotId_2') "

Review comment:
       Having should be applied to post-aggregations (e.g. `status1`) instead of columns. This having can be rewritten as where clause.
   Currently having is applied before filling the values, so I believe it could cause unexpected results. Ideally we should first fill the values then applying the having filter.




-- 
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 edited a comment on pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#issuecomment-971204560


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?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 [#7781](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (442f220) into [master](https://codecov.io/gh/apache/pinot/commit/09bb91cecc8cff459a9d7814a7fe61aa67f2a393?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (09bb91c) will **decrease** coverage by `6.57%`.
   > The diff coverage is `41.87%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7781/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #7781      +/-   ##
   ============================================
   - Coverage     71.62%   65.04%   -6.58%     
     Complexity     4079     4079              
   ============================================
     Files          1578     1534      -44     
     Lines         80641    79174    -1467     
     Branches      11987    11866     -121     
   ============================================
   - Hits          57758    51500    -6258     
   - Misses        18994    23990    +4996     
   + Partials       3889     3684     -205     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `68.43% <41.87%> (-0.18%)` | :arrow_down: |
   | unittests2 | `14.51% <0.00%> (-0.10%)` | :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/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...t/core/query/selection/SelectionOperatorUtils.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9zZWxlY3Rpb24vU2VsZWN0aW9uT3BlcmF0b3JVdGlscy5qYXZh) | `77.61% <2.70%> (-14.57%)` | :arrow_down: |
   | [...e/query/reduce/GapFillGroupByDataTableReducer.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvR2FwRmlsbEdyb3VwQnlEYXRhVGFibGVSZWR1Y2VyLmphdmE=) | `43.46% <43.46%> (ø)` | |
   | [...pinot/core/query/request/context/QueryContext.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZXF1ZXN0L2NvbnRleHQvUXVlcnlDb250ZXh0LmphdmE=) | `97.42% <66.66%> (-0.49%)` | :arrow_down: |
   | [...not/core/operator/transform/TransformOperator.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vVHJhbnNmb3JtT3BlcmF0b3IuamF2YQ==) | `95.23% <100.00%> (+0.23%)` | :arrow_up: |
   | [...r/transform/function/TransformFunctionFactory.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vVHJhbnNmb3JtRnVuY3Rpb25GYWN0b3J5LmphdmE=) | `82.88% <100.00%> (-1.74%)` | :arrow_down: |
   | [...inot/core/query/reduce/PostAggregationHandler.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUG9zdEFnZ3JlZ2F0aW9uSGFuZGxlci5qYXZh) | `92.30% <100.00%> (+0.09%)` | :arrow_up: |
   | [.../pinot/core/query/reduce/ResultReducerFactory.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUmVzdWx0UmVkdWNlckZhY3RvcnkuamF2YQ==) | `100.00% <100.00%> (ø)` | |
   | [...a/org/apache/pinot/common/metrics/MinionMeter.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25NZXRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/pinot/common/metrics/ControllerMeter.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Db250cm9sbGVyTWV0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/BrokerQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Ccm9rZXJRdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [361 more](https://codecov.io/gh/apache/pinot/pull/7781/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [09bb91c...442f220](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751571429



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/selection/SelectionOperatorUtils.java
##########
@@ -339,6 +339,74 @@ public static DataTable getDataTableFromRows(Collection<Object[]> rows, DataSche
     return dataTableBuilder.build();
   }
 
+  /**
+   * The default value for each column type.
+   */
+  public static Serializable getDefaultValue(ColumnDataType dataType) {
+    switch (dataType) {
+      // Single-value column
+      case INT:
+        return THREAD_LOCAL_INT_FORMAT.get().format(((Number) 0).intValue());
+      case LONG:
+        return THREAD_LOCAL_LONG_FORMAT.get().format(((Number) 0).longValue());
+      case FLOAT:
+        return THREAD_LOCAL_FLOAT_FORMAT.get().format(((Number) 0).floatValue());
+      case DOUBLE:
+        return THREAD_LOCAL_DOUBLE_FORMAT.get().format(((Number) 0).doubleValue());
+      case BOOLEAN:
+        return "false";
+      case TIMESTAMP:
+        return new Timestamp(0L).toString();
+      // NOTE: Return String for BYTES columns for backward-compatibility
+      case BYTES:
+        return "";
+
+      // Multi-value column
+      case INT_ARRAY:
+        DecimalFormat intFormat = THREAD_LOCAL_INT_FORMAT.get();
+        int[] ints = new int[0];

Review comment:
       Yes, this is default value for gapfill.




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r752438692



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/TransformFunctionFactory.java
##########
@@ -59,6 +59,10 @@
  * Factory class for transformation functions.
  */
 public class TransformFunctionFactory {
+
+  private static final String AGGREGATE_GAP_FILL = "aggregategapfill";

Review comment:
       aggregategapfill is for post-aggregation gapfill, meaning aggregation, then gapfill
   We will use gapfillaggregate for pre-aggregate gapfill, meaning gapfill and then aggregate.




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r752495742



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,706 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,

Review comment:
       Done




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r753587707



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // 1. groupByMode = sql, responseFormat = sql
+    // This is the primary SQL compliant group by
+
+    try {
+      setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+          brokerMetrics);
+    } catch (TimeoutException e) {
+      brokerResponseNative.getProcessingExceptions()
+          .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+    }
+    int resultSize = brokerResponseNative.getResultTable().getRows().size();
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    return new Key(Arrays.copyOfRange(row, 1, _numOfKeyColumns + 1));
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      List<Object[]> resultRows = new ArrayList<>(rows.size());
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < resultColumnDataTypes.length; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {

Review comment:
       We are not looping over the rows multiple times. The "index" is for resultRows and also moving forward.




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r753582748



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/TransformFunctionFactory.java
##########
@@ -59,6 +59,7 @@
  * Factory class for transformation functions.
  */
 public class TransformFunctionFactory {
+

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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r753601491



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());

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] codecov-commenter edited a comment on pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#issuecomment-971204560


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?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 [#7781](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (4f30267) into [master](https://codecov.io/gh/apache/pinot/commit/09bb91cecc8cff459a9d7814a7fe61aa67f2a393?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (09bb91c) will **decrease** coverage by `57.09%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7781/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #7781       +/-   ##
   =============================================
   - Coverage     71.62%   14.52%   -57.10%     
   + Complexity     4079       80     -3999     
   =============================================
     Files          1578     1535       -43     
     Lines         80641    79180     -1461     
     Branches      11987    11860      -127     
   =============================================
   - Hits          57758    11502    -46256     
   - Misses        18994    66834    +47840     
   + Partials       3889      844     -3045     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `?` | |
   | unittests2 | `14.52% <0.00%> (-0.08%)` | :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/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...not/core/operator/transform/TransformOperator.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vVHJhbnNmb3JtT3BlcmF0b3IuamF2YQ==) | `0.00% <0.00%> (-95.00%)` | :arrow_down: |
   | [...e/query/reduce/GapFillGroupByDataTableReducer.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvR2FwRmlsbEdyb3VwQnlEYXRhVGFibGVSZWR1Y2VyLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...inot/core/query/reduce/PostAggregationHandler.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUG9zdEFnZ3JlZ2F0aW9uSGFuZGxlci5qYXZh) | `0.00% <0.00%> (-92.21%)` | :arrow_down: |
   | [.../pinot/core/query/reduce/ResultReducerFactory.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUmVzdWx0UmVkdWNlckZhY3RvcnkuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pinot/core/query/request/context/QueryContext.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZXF1ZXN0L2NvbnRleHQvUXVlcnlDb250ZXh0LmphdmE=) | `0.00% <0.00%> (-97.91%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/util/GapfillUtils.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS91dGlsL0dhcGZpbGxVdGlscy5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...ain/java/org/apache/pinot/core/data/table/Key.java](https://codecov.io/gh/apache/pinot/pull/7781/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: |
   | [.../java/org/apache/pinot/spi/utils/BooleanUtils.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQm9vbGVhblV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/data/table/Record.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL1JlY29yZC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/util/GroupByUtils.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS91dGlsL0dyb3VwQnlVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [1255 more](https://codecov.io/gh/apache/pinot/pull/7781/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [09bb91c...4f30267](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov-commenter edited a comment on pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#issuecomment-971204560


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?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 [#7781](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (4f30267) into [master](https://codecov.io/gh/apache/pinot/commit/09bb91cecc8cff459a9d7814a7fe61aa67f2a393?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (09bb91c) will **decrease** coverage by `6.36%`.
   > The diff coverage is `75.35%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7781/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #7781      +/-   ##
   ============================================
   - Coverage     71.62%   65.25%   -6.37%     
   - Complexity     4079     4085       +6     
   ============================================
     Files          1578     1535      -43     
     Lines         80641    79180    -1461     
     Branches      11987    11860     -127     
   ============================================
   - Hits          57758    51670    -6088     
   - Misses        18994    23816    +4822     
   + Partials       3889     3694     -195     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `68.73% <75.35%> (+0.12%)` | :arrow_up: |
   | unittests2 | `14.52% <0.00%> (-0.08%)` | :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/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../java/org/apache/pinot/core/util/GapfillUtils.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS91dGlsL0dhcGZpbGxVdGlscy5qYXZh) | `57.14% <57.14%> (ø)` | |
   | [...e/query/reduce/GapFillGroupByDataTableReducer.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvR2FwRmlsbEdyb3VwQnlEYXRhVGFibGVSZWR1Y2VyLmphdmE=) | `76.15% <76.15%> (ø)` | |
   | [...not/core/operator/transform/TransformOperator.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vVHJhbnNmb3JtT3BlcmF0b3IuamF2YQ==) | `95.23% <100.00%> (+0.23%)` | :arrow_up: |
   | [...inot/core/query/reduce/PostAggregationHandler.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUG9zdEFnZ3JlZ2F0aW9uSGFuZGxlci5qYXZh) | `92.40% <100.00%> (+0.19%)` | :arrow_up: |
   | [.../pinot/core/query/reduce/ResultReducerFactory.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUmVzdWx0UmVkdWNlckZhY3RvcnkuamF2YQ==) | `100.00% <100.00%> (ø)` | |
   | [...pinot/core/query/request/context/QueryContext.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZXF1ZXN0L2NvbnRleHQvUXVlcnlDb250ZXh0LmphdmE=) | `97.95% <100.00%> (+0.05%)` | :arrow_up: |
   | [...a/org/apache/pinot/common/metrics/MinionMeter.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25NZXRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/pinot/common/metrics/ControllerMeter.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Db250cm9sbGVyTWV0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/BrokerQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Ccm9rZXJRdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/MinionQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25RdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [372 more](https://codecov.io/gh/apache/pinot/pull/7781/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [09bb91c...4f30267](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r755364417



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,490 @@
+/**
+ * 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.reduce;
+
+import com.google.common.base.Preconditions;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.request.context.OrderByExpressionContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GapfillUtils;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _groupByKeys;
+  private final Map<Key, Object[]> _previousByGroupKey;
+  private final int _numOfGroupByKeys;
+  private final List<Integer> _groupByKeyIndexes;
+  private final boolean [] _isGroupBySelections;
+  private int _timeBucketIndex = -1;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    Preconditions.checkArgument(
+        queryContext.getBrokerRequest().getPinotQuery() != null, "GapFill can only be applied to sql query");
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+
+    ExpressionContext gapFillSelection = null;
+    for (ExpressionContext expressionContext : _queryContext.getSelectExpressions()) {
+      if (GapfillUtils.isPostAggregateGapfill(expressionContext)) {
+        gapFillSelection = expressionContext;
+        break;
+      }
+    }
+
+    List<ExpressionContext> args = gapFillSelection.getFunction().getArguments();
+    Preconditions.checkArgument(
+        args.size() == 5, "PostAggregateGapFill does not correct number of arguments.");

Review comment:
       Done




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751563523



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,725 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,
+    // a result is generated for each aggregation in the query,
+    // and the group by keys are not the same across the aggregations
+    // This PQL style of execution makes it impossible to support order by on group by.
+    //
+    // We could not simply change the group by execution behavior,
+    // as that would not be backward compatible for existing users of group by.
+    // As a result, we have 2 modes of group by execution - pql and sql - which can be controlled via query options
+    //
+    // Long term, we may completely move to sql, and keep only full sql mode alive
+    // Until then, we need to support responseFormat = sql for both the modes of execution.
+    // The 4 variants are as described below:
+
+    if (_groupByModeSql) {
+
+      if (_responseFormatSql) {
+        // 1. groupByMode = sql, responseFormat = sql
+        // This is the primary SQL compliant group by
+
+        try {
+          setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+              brokerMetrics);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // 2. groupByMode = sql, responseFormat = pql
+        // This mode will invoke SQL style group by execution, but present results in PQL way
+        // This mode is useful for users who want to avail of SQL compliant group by behavior,
+        // w/o having to forcefully move to a new result type
+
+        try {
+          setSQLGroupByInAggregationResults(brokerResponseNative, dataSchema, dataTables, reducerContext);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    } else {
+
+      // 3. groupByMode = pql, responseFormat = sql
+      // This mode is for users who want response presented in SQL style, but want PQL style group by behavior
+      // Multiple aggregations in PQL violates the tabular nature of results
+      // As a result, in this mode, only single aggregations are supported
+
+      // 4. groupByMode = pql, responseFormat = pql
+      // This is the primary PQL compliant group by
+
+      setGroupByResults(brokerResponseNative, dataTables);
+
+      if (_responseFormatSql) {
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // We emit the group by size when the result isn't empty. All the sizes among group-by results should be the
+        // same.
+        // Thus, we can just emit the one from the 1st result.
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    }
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    Object [] keyColumns = new Object[_numOfKeyColumns];
+    for (int i = 0; i < _numOfKeyColumns; i++) {
+      keyColumns[i] = row[i + 1];
+    }
+    return new Key(keyColumns);
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      int numResultColumns = resultColumnDataTypes.length;
+      int numResultRows = rows.size();
+      List<Object[]> resultRows = new ArrayList<>(numResultRows);
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < numResultColumns; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {
+      Set<Key> keys = new HashSet<>(_primaryKeys);
+      while (index < resultRows.size()) {
+        long timeCol = _dateTimeFormatter.fromFormatToMillis((String) resultRows.get(index)[0]);
+        if (timeCol < time) {
+          index++;
+        } else if (timeCol == time) {
+          gapfillResultRows.add(resultRows.get(index));
+          if (gapfillResultRows.size() == limit) {
+            return gapfillResultRows;
+          }
+          Key key = constructKey(resultRows.get(index));
+          keys.remove(key);
+          for (int colIndex = 2; colIndex < numResultColumns; colIndex++) {
+            _previous.put(key, resultRows.get(index));
+          }
+          index++;
+        } else {
+          break;
+        }
+      }
+      for (Key key : keys) {
+        Object[] gapfillRow = new Object[numResultColumns];
+        gapfillRow[0] = _dateTimeFormatter.fromMillisToFormat(time);
+        for (int i = 0; i < _numOfKeyColumns; i++) {
+          gapfillRow[i + 1] = key.getValues()[i];
+        }
+
+        for (int i = _numOfKeyColumns + 1; i < numResultColumns; i++) {
+          gapfillRow[i] = getFillValue(i, key, resultColumnDataTypes[i]);
+        }
+        gapfillResultRows.add(gapfillRow);
+        if (gapfillResultRows.size() == limit) {
+          return gapfillResultRows;
+        }
+      }
+    }
+    return gapfillResultRows;
+  }
+
+  Object getFillValue(int columIndex, Object key, ColumnDataType dataType) {
+    ExpressionContext expressionContext = _queryContext.getSelectExpressions().get(columIndex);
+    if (expressionContext.getFunction() != null
+        && expressionContext.getFunction().getFunctionName().equalsIgnoreCase("fill")) {
+      List<ExpressionContext> args = expressionContext.getFunction().getArguments();
+      if (args.get(1).getLiteral() == null) {
+        throw new UnsupportedOperationException("Wrong Sql.");
+      }
+      FillType fillType = FillType.valueOf(args.get(1).getLiteral());
+      if (fillType == FillType.FILL_DEFAULT_VALUE) {
+        // TODO: may fill the default value from sql in the future.
+        return SelectionOperatorUtils.getDefaultValue(dataType);
+      } else if (fillType == FillType.FILL_PREVIOUS_VALUE) {
+        if (_previous.containsKey(key)) {
+          return _previous.get(key)[columIndex];
+        } else {
+          return SelectionOperatorUtils.getDefaultValue(dataType);
+        }
+      } else {
+        throw new UnsupportedOperationException("unsupported fill type.");
+      }
+    } else {
+      return SelectionOperatorUtils.getDefaultValue(dataType);
+    }
+  }
+
+  enum FillType {
+    FILL_DEFAULT_VALUE,
+    FILL_PREVIOUS_VALUE,
+  }
+
+  /**
+   * Helper method to extract the final aggregation results for the given row (in-place).
+   */
+  private void extractFinalAggregationResults(Object[] row) {
+    for (int i = 0; i < _numAggregationFunctions; i++) {
+      int valueIndex = i + _numGroupByExpressions;
+      row[valueIndex] = _aggregationFunctions[i].extractFinalResult(row[valueIndex]);
+    }
+  }
+
+  /**
+   * Constructs the DataSchema for the rows before the post-aggregation (SQL mode).
+   */
+  private DataSchema getPrePostAggregationDataSchema(DataSchema dataSchema) {
+    String[] columnNames = dataSchema.getColumnNames();
+    ColumnDataType[] columnDataTypes = new ColumnDataType[_numColumns];
+    System.arraycopy(dataSchema.getColumnDataTypes(), 0, columnDataTypes, 0, _numGroupByExpressions);
+    for (int i = 0; i < _numAggregationFunctions; i++) {
+      columnDataTypes[i + _numGroupByExpressions] = _aggregationFunctions[i].getFinalResultColumnType();
+    }
+    return new DataSchema(columnNames, columnDataTypes);
+  }
+
+  private IndexedTable getIndexedTable(DataSchema dataSchema, Collection<DataTable> dataTablesToReduce,
+      DataTableReducerContext reducerContext)
+      throws TimeoutException {
+    long start = System.currentTimeMillis();
+    int numDataTables = dataTablesToReduce.size();
+
+    // Get the number of threads to use for reducing.
+    // In case of single reduce thread, fall back to SimpleIndexedTable to avoid redundant locking/unlocking calls.
+    int numReduceThreadsToUse = getNumReduceThreadsToUse(numDataTables, reducerContext.getMaxReduceThreadsPerQuery());
+    int limit = _queryContext.getLimit();
+    // TODO: Make minTrimSize configurable
+    int trimSize = GroupByUtils.getTableCapacity(limit);
+    // NOTE: For query with HAVING clause, use trimSize as resultSize to ensure the result accuracy.
+    // TODO: Resolve the HAVING clause within the IndexedTable before returning the result
+    int resultSize = _queryContext.getHavingFilter() != null ? trimSize : limit;
+    int trimThreshold = reducerContext.getGroupByTrimThreshold();
+    IndexedTable indexedTable;
+    if (numReduceThreadsToUse <= 1) {
+      indexedTable = new SimpleIndexedTable(dataSchema, _queryContext, resultSize, trimSize, trimThreshold);
+    } else {
+      if (trimThreshold >= GroupByOrderByCombineOperator.MAX_TRIM_THRESHOLD) {
+        // special case of trim threshold where it is set to max value.
+        // there won't be any trimming during upsert in this case.
+        // thus we can avoid the overhead of read-lock and write-lock
+        // in the upsert method.
+        indexedTable = new UnboundedConcurrentIndexedTable(dataSchema, _queryContext, resultSize);
+      } else {
+        indexedTable = new ConcurrentIndexedTable(dataSchema, _queryContext, resultSize, trimSize, trimThreshold);
+      }
+    }
+
+    Future[] futures = new Future[numDataTables];
+    CountDownLatch countDownLatch = new CountDownLatch(numDataTables);
+
+    // Create groups of data tables that each thread can process concurrently.
+    // Given that numReduceThreads is <= numDataTables, each group will have at least one data table.
+    ArrayList<DataTable> dataTables = new ArrayList<>(dataTablesToReduce);
+    List<List<DataTable>> reduceGroups = new ArrayList<>(numReduceThreadsToUse);
+
+    for (int i = 0; i < numReduceThreadsToUse; i++) {
+      reduceGroups.add(new ArrayList<>());
+    }
+    for (int i = 0; i < numDataTables; i++) {
+      reduceGroups.get(i % numReduceThreadsToUse).add(dataTables.get(i));
+    }
+
+    int cnt = 0;
+    ColumnDataType[] storedColumnDataTypes = dataSchema.getStoredColumnDataTypes();
+    for (List<DataTable> reduceGroup : reduceGroups) {
+      futures[cnt++] = reducerContext.getExecutorService().submit(new TraceRunnable() {

Review comment:
       Done




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r755366473



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,490 @@
+/**
+ * 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.reduce;
+
+import com.google.common.base.Preconditions;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.request.context.OrderByExpressionContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GapfillUtils;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _groupByKeys;
+  private final Map<Key, Object[]> _previousByGroupKey;
+  private final int _numOfGroupByKeys;
+  private final List<Integer> _groupByKeyIndexes;
+  private final boolean [] _isGroupBySelections;
+  private int _timeBucketIndex = -1;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    Preconditions.checkArgument(
+        queryContext.getBrokerRequest().getPinotQuery() != null, "GapFill can only be applied to sql query");
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+
+    ExpressionContext gapFillSelection = null;
+    for (ExpressionContext expressionContext : _queryContext.getSelectExpressions()) {
+      if (GapfillUtils.isPostAggregateGapfill(expressionContext)) {
+        gapFillSelection = expressionContext;
+        break;
+      }
+    }
+
+    List<ExpressionContext> args = gapFillSelection.getFunction().getArguments();
+    Preconditions.checkArgument(
+        args.size() == 5, "PostAggregateGapFill does not correct number of arguments.");
+    Preconditions.checkArgument(

Review comment:
       I prefer not to do it since we cannot add it more in future.




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r752464866



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/selection/SelectionOperatorUtils.java
##########
@@ -339,6 +339,43 @@ public static DataTable getDataTableFromRows(Collection<Object[]> rows, DataSche
     return dataTableBuilder.build();
   }
 
+  /**
+   * The default value for each column type.
+   */
+  public static Serializable getDefaultValue(ColumnDataType dataType) {

Review comment:
       Done




-- 
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 edited a comment on pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#issuecomment-971204560


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?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 [#7781](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (2fd212f) into [master](https://codecov.io/gh/apache/pinot/commit/09bb91cecc8cff459a9d7814a7fe61aa67f2a393?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (09bb91c) will **decrease** coverage by `1.12%`.
   > The diff coverage is `43.93%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7781/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #7781      +/-   ##
   ============================================
   - Coverage     71.62%   70.49%   -1.13%     
     Complexity     4079     4079              
   ============================================
     Files          1578     1579       +1     
     Lines         80641    81009     +368     
     Branches      11987    12061      +74     
   ============================================
   - Hits          57758    57109     -649     
   - Misses        18994    19989     +995     
   - Partials       3889     3911      +22     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `29.00% <2.96%> (-0.18%)` | :arrow_down: |
   | integration2 | `?` | |
   | unittests1 | `68.47% <43.93%> (-0.14%)` | :arrow_down: |
   | unittests2 | `14.52% <0.00%> (-0.09%)` | :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/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...t/core/query/selection/SelectionOperatorUtils.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9zZWxlY3Rpb24vU2VsZWN0aW9uT3BlcmF0b3JVdGlscy5qYXZh) | `88.75% <7.69%> (-3.44%)` | :arrow_down: |
   | [...e/query/reduce/GapFillGroupByDataTableReducer.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvR2FwRmlsbEdyb3VwQnlEYXRhVGFibGVSZWR1Y2VyLmphdmE=) | `42.81% <42.81%> (ø)` | |
   | [...pinot/core/query/request/context/QueryContext.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZXF1ZXN0L2NvbnRleHQvUXVlcnlDb250ZXh0LmphdmE=) | `97.42% <66.66%> (-0.49%)` | :arrow_down: |
   | [...not/core/operator/transform/TransformOperator.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vVHJhbnNmb3JtT3BlcmF0b3IuamF2YQ==) | `95.23% <100.00%> (+0.23%)` | :arrow_up: |
   | [...r/transform/function/TransformFunctionFactory.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vVHJhbnNmb3JtRnVuY3Rpb25GYWN0b3J5LmphdmE=) | `85.58% <100.00%> (+0.97%)` | :arrow_up: |
   | [...inot/core/query/reduce/PostAggregationHandler.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUG9zdEFnZ3JlZ2F0aW9uSGFuZGxlci5qYXZh) | `92.30% <100.00%> (+0.09%)` | :arrow_up: |
   | [.../pinot/core/query/reduce/ResultReducerFactory.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUmVzdWx0UmVkdWNlckZhY3RvcnkuamF2YQ==) | `100.00% <100.00%> (ø)` | |
   | [...ore/operator/streaming/StreamingResponseUtils.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9zdHJlYW1pbmcvU3RyZWFtaW5nUmVzcG9uc2VVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ager/realtime/PeerSchemeSplitSegmentCommitter.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvUGVlclNjaGVtZVNwbGl0U2VnbWVudENvbW1pdHRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pache/pinot/common/utils/grpc/GrpcQueryClient.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvZ3JwYy9HcnBjUXVlcnlDbGllbnQuamF2YQ==) | `0.00% <0.00%> (-94.74%)` | :arrow_down: |
   | ... and [89 more](https://codecov.io/gh/apache/pinot/pull/7781/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [09bb91c...2fd212f](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] richardstartin commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751151335



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,725 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,
+    // a result is generated for each aggregation in the query,
+    // and the group by keys are not the same across the aggregations
+    // This PQL style of execution makes it impossible to support order by on group by.
+    //
+    // We could not simply change the group by execution behavior,
+    // as that would not be backward compatible for existing users of group by.
+    // As a result, we have 2 modes of group by execution - pql and sql - which can be controlled via query options
+    //
+    // Long term, we may completely move to sql, and keep only full sql mode alive
+    // Until then, we need to support responseFormat = sql for both the modes of execution.
+    // The 4 variants are as described below:
+
+    if (_groupByModeSql) {
+
+      if (_responseFormatSql) {
+        // 1. groupByMode = sql, responseFormat = sql
+        // This is the primary SQL compliant group by
+
+        try {
+          setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+              brokerMetrics);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // 2. groupByMode = sql, responseFormat = pql
+        // This mode will invoke SQL style group by execution, but present results in PQL way
+        // This mode is useful for users who want to avail of SQL compliant group by behavior,
+        // w/o having to forcefully move to a new result type
+
+        try {
+          setSQLGroupByInAggregationResults(brokerResponseNative, dataSchema, dataTables, reducerContext);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    } else {
+
+      // 3. groupByMode = pql, responseFormat = sql
+      // This mode is for users who want response presented in SQL style, but want PQL style group by behavior
+      // Multiple aggregations in PQL violates the tabular nature of results
+      // As a result, in this mode, only single aggregations are supported
+
+      // 4. groupByMode = pql, responseFormat = pql
+      // This is the primary PQL compliant group by
+
+      setGroupByResults(brokerResponseNative, dataTables);
+
+      if (_responseFormatSql) {
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // We emit the group by size when the result isn't empty. All the sizes among group-by results should be the
+        // same.
+        // Thus, we can just emit the one from the 1st result.
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    }
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    Object [] keyColumns = new Object[_numOfKeyColumns];
+    for (int i = 0; i < _numOfKeyColumns; i++) {
+      keyColumns[i] = row[i + 1];
+    }
+    return new Key(keyColumns);
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      int numResultColumns = resultColumnDataTypes.length;
+      int numResultRows = rows.size();
+      List<Object[]> resultRows = new ArrayList<>(numResultRows);
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < numResultColumns; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {
+      Set<Key> keys = new HashSet<>(_primaryKeys);
+      while (index < resultRows.size()) {
+        long timeCol = _dateTimeFormatter.fromFormatToMillis((String) resultRows.get(index)[0]);
+        if (timeCol < time) {
+          index++;
+        } else if (timeCol == time) {
+          gapfillResultRows.add(resultRows.get(index));
+          if (gapfillResultRows.size() == limit) {
+            return gapfillResultRows;
+          }
+          Key key = constructKey(resultRows.get(index));
+          keys.remove(key);
+          for (int colIndex = 2; colIndex < numResultColumns; colIndex++) {
+            _previous.put(key, resultRows.get(index));
+          }
+          index++;
+        } else {
+          break;
+        }
+      }
+      for (Key key : keys) {
+        Object[] gapfillRow = new Object[numResultColumns];
+        gapfillRow[0] = _dateTimeFormatter.fromMillisToFormat(time);
+        for (int i = 0; i < _numOfKeyColumns; i++) {
+          gapfillRow[i + 1] = key.getValues()[i];
+        }
+
+        for (int i = _numOfKeyColumns + 1; i < numResultColumns; i++) {
+          gapfillRow[i] = getFillValue(i, key, resultColumnDataTypes[i]);
+        }
+        gapfillResultRows.add(gapfillRow);
+        if (gapfillResultRows.size() == limit) {
+          return gapfillResultRows;
+        }
+      }
+    }
+    return gapfillResultRows;
+  }
+
+  Object getFillValue(int columIndex, Object key, ColumnDataType dataType) {
+    ExpressionContext expressionContext = _queryContext.getSelectExpressions().get(columIndex);
+    if (expressionContext.getFunction() != null
+        && expressionContext.getFunction().getFunctionName().equalsIgnoreCase("fill")) {
+      List<ExpressionContext> args = expressionContext.getFunction().getArguments();
+      if (args.get(1).getLiteral() == null) {
+        throw new UnsupportedOperationException("Wrong Sql.");
+      }
+      FillType fillType = FillType.valueOf(args.get(1).getLiteral());
+      if (fillType == FillType.FILL_DEFAULT_VALUE) {
+        // TODO: may fill the default value from sql in the future.
+        return SelectionOperatorUtils.getDefaultValue(dataType);
+      } else if (fillType == FillType.FILL_PREVIOUS_VALUE) {
+        if (_previous.containsKey(key)) {
+          return _previous.get(key)[columIndex];
+        } else {
+          return SelectionOperatorUtils.getDefaultValue(dataType);
+        }
+      } else {
+        throw new UnsupportedOperationException("unsupported fill type.");
+      }
+    } else {
+      return SelectionOperatorUtils.getDefaultValue(dataType);
+    }
+  }
+
+  enum FillType {
+    FILL_DEFAULT_VALUE,
+    FILL_PREVIOUS_VALUE,
+  }
+
+  /**
+   * Helper method to extract the final aggregation results for the given row (in-place).
+   */
+  private void extractFinalAggregationResults(Object[] row) {
+    for (int i = 0; i < _numAggregationFunctions; i++) {
+      int valueIndex = i + _numGroupByExpressions;
+      row[valueIndex] = _aggregationFunctions[i].extractFinalResult(row[valueIndex]);
+    }
+  }
+
+  /**
+   * Constructs the DataSchema for the rows before the post-aggregation (SQL mode).
+   */
+  private DataSchema getPrePostAggregationDataSchema(DataSchema dataSchema) {
+    String[] columnNames = dataSchema.getColumnNames();
+    ColumnDataType[] columnDataTypes = new ColumnDataType[_numColumns];
+    System.arraycopy(dataSchema.getColumnDataTypes(), 0, columnDataTypes, 0, _numGroupByExpressions);
+    for (int i = 0; i < _numAggregationFunctions; i++) {
+      columnDataTypes[i + _numGroupByExpressions] = _aggregationFunctions[i].getFinalResultColumnType();
+    }
+    return new DataSchema(columnNames, columnDataTypes);
+  }
+
+  private IndexedTable getIndexedTable(DataSchema dataSchema, Collection<DataTable> dataTablesToReduce,
+      DataTableReducerContext reducerContext)
+      throws TimeoutException {
+    long start = System.currentTimeMillis();
+    int numDataTables = dataTablesToReduce.size();
+
+    // Get the number of threads to use for reducing.
+    // In case of single reduce thread, fall back to SimpleIndexedTable to avoid redundant locking/unlocking calls.
+    int numReduceThreadsToUse = getNumReduceThreadsToUse(numDataTables, reducerContext.getMaxReduceThreadsPerQuery());
+    int limit = _queryContext.getLimit();
+    // TODO: Make minTrimSize configurable
+    int trimSize = GroupByUtils.getTableCapacity(limit);
+    // NOTE: For query with HAVING clause, use trimSize as resultSize to ensure the result accuracy.
+    // TODO: Resolve the HAVING clause within the IndexedTable before returning the result
+    int resultSize = _queryContext.getHavingFilter() != null ? trimSize : limit;
+    int trimThreshold = reducerContext.getGroupByTrimThreshold();
+    IndexedTable indexedTable;
+    if (numReduceThreadsToUse <= 1) {
+      indexedTable = new SimpleIndexedTable(dataSchema, _queryContext, resultSize, trimSize, trimThreshold);
+    } else {
+      if (trimThreshold >= GroupByOrderByCombineOperator.MAX_TRIM_THRESHOLD) {
+        // special case of trim threshold where it is set to max value.
+        // there won't be any trimming during upsert in this case.
+        // thus we can avoid the overhead of read-lock and write-lock
+        // in the upsert method.
+        indexedTable = new UnboundedConcurrentIndexedTable(dataSchema, _queryContext, resultSize);
+      } else {
+        indexedTable = new ConcurrentIndexedTable(dataSchema, _queryContext, resultSize, trimSize, trimThreshold);
+      }
+    }
+
+    Future[] futures = new Future[numDataTables];
+    CountDownLatch countDownLatch = new CountDownLatch(numDataTables);
+
+    // Create groups of data tables that each thread can process concurrently.
+    // Given that numReduceThreads is <= numDataTables, each group will have at least one data table.
+    ArrayList<DataTable> dataTables = new ArrayList<>(dataTablesToReduce);
+    List<List<DataTable>> reduceGroups = new ArrayList<>(numReduceThreadsToUse);
+
+    for (int i = 0; i < numReduceThreadsToUse; i++) {
+      reduceGroups.add(new ArrayList<>());
+    }
+    for (int i = 0; i < numDataTables; i++) {
+      reduceGroups.get(i % numReduceThreadsToUse).add(dataTables.get(i));
+    }
+
+    int cnt = 0;
+    ColumnDataType[] storedColumnDataTypes = dataSchema.getStoredColumnDataTypes();
+    for (List<DataTable> reduceGroup : reduceGroups) {
+      futures[cnt++] = reducerContext.getExecutorService().submit(new TraceRunnable() {

Review comment:
       What about 
   
   ```java
   List<Future<?>> futures = reducerContext.getExecutorService().invokeAll(reduceGroups.stream().map(reduceGroup -> new TraceCallable<Void>() {
         @Override
         public Void callJob()
             throws Exception {
           // do stuff
           return null;
         }
       }).collect(Collectors.toList()));
   
   
   ```




-- 
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] richardstartin commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751573523



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/selection/SelectionOperatorUtils.java
##########
@@ -339,6 +339,74 @@ public static DataTable getDataTableFromRows(Collection<Object[]> rows, DataSche
     return dataTableBuilder.build();
   }
 
+  /**
+   * The default value for each column type.
+   */
+  public static Serializable getDefaultValue(ColumnDataType dataType) {
+    switch (dataType) {
+      // Single-value column
+      case INT:
+        return THREAD_LOCAL_INT_FORMAT.get().format(((Number) 0).intValue());
+      case LONG:
+        return THREAD_LOCAL_LONG_FORMAT.get().format(((Number) 0).longValue());
+      case FLOAT:
+        return THREAD_LOCAL_FLOAT_FORMAT.get().format(((Number) 0).floatValue());
+      case DOUBLE:
+        return THREAD_LOCAL_DOUBLE_FORMAT.get().format(((Number) 0).doubleValue());

Review comment:
       There’s no need to box 0 to get the right primitive type, I guess you’re doing this to make the type unambiguous, but you can do that with literal suffixes




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751452791



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,725 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,
+    // a result is generated for each aggregation in the query,
+    // and the group by keys are not the same across the aggregations
+    // This PQL style of execution makes it impossible to support order by on group by.
+    //
+    // We could not simply change the group by execution behavior,
+    // as that would not be backward compatible for existing users of group by.
+    // As a result, we have 2 modes of group by execution - pql and sql - which can be controlled via query options
+    //
+    // Long term, we may completely move to sql, and keep only full sql mode alive
+    // Until then, we need to support responseFormat = sql for both the modes of execution.
+    // The 4 variants are as described below:
+
+    if (_groupByModeSql) {
+
+      if (_responseFormatSql) {
+        // 1. groupByMode = sql, responseFormat = sql
+        // This is the primary SQL compliant group by
+
+        try {
+          setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+              brokerMetrics);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // 2. groupByMode = sql, responseFormat = pql
+        // This mode will invoke SQL style group by execution, but present results in PQL way
+        // This mode is useful for users who want to avail of SQL compliant group by behavior,
+        // w/o having to forcefully move to a new result type
+
+        try {
+          setSQLGroupByInAggregationResults(brokerResponseNative, dataSchema, dataTables, reducerContext);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    } else {
+
+      // 3. groupByMode = pql, responseFormat = sql
+      // This mode is for users who want response presented in SQL style, but want PQL style group by behavior
+      // Multiple aggregations in PQL violates the tabular nature of results
+      // As a result, in this mode, only single aggregations are supported
+
+      // 4. groupByMode = pql, responseFormat = pql
+      // This is the primary PQL compliant group by
+
+      setGroupByResults(brokerResponseNative, dataTables);
+
+      if (_responseFormatSql) {
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // We emit the group by size when the result isn't empty. All the sizes among group-by results should be the
+        // same.
+        // Thus, we can just emit the one from the 1st result.
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    }
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    Object [] keyColumns = new Object[_numOfKeyColumns];
+    for (int i = 0; i < _numOfKeyColumns; i++) {
+      keyColumns[i] = row[i + 1];
+    }
+    return new Key(keyColumns);
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      int numResultColumns = resultColumnDataTypes.length;
+      int numResultRows = rows.size();

Review comment:
       Done




-- 
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 edited a comment on pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#issuecomment-971204560


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?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 [#7781](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (2fd212f) into [master](https://codecov.io/gh/apache/pinot/commit/09bb91cecc8cff459a9d7814a7fe61aa67f2a393?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (09bb91c) will **decrease** coverage by `0.15%`.
   > The diff coverage is `43.93%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7781/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #7781      +/-   ##
   ============================================
   - Coverage     71.62%   71.47%   -0.16%     
     Complexity     4079     4079              
   ============================================
     Files          1578     1579       +1     
     Lines         80641    81009     +368     
     Branches      11987    12061      +74     
   ============================================
   + Hits          57758    57899     +141     
   - Misses        18994    19196     +202     
   - Partials       3889     3914      +25     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `29.00% <2.96%> (-0.18%)` | :arrow_down: |
   | integration2 | `27.75% <2.96%> (-0.10%)` | :arrow_down: |
   | unittests1 | `68.47% <43.93%> (-0.14%)` | :arrow_down: |
   | unittests2 | `14.52% <0.00%> (-0.09%)` | :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/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...t/core/query/selection/SelectionOperatorUtils.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9zZWxlY3Rpb24vU2VsZWN0aW9uT3BlcmF0b3JVdGlscy5qYXZh) | `88.75% <7.69%> (-3.44%)` | :arrow_down: |
   | [...e/query/reduce/GapFillGroupByDataTableReducer.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvR2FwRmlsbEdyb3VwQnlEYXRhVGFibGVSZWR1Y2VyLmphdmE=) | `42.81% <42.81%> (ø)` | |
   | [...pinot/core/query/request/context/QueryContext.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZXF1ZXN0L2NvbnRleHQvUXVlcnlDb250ZXh0LmphdmE=) | `97.42% <66.66%> (-0.49%)` | :arrow_down: |
   | [...not/core/operator/transform/TransformOperator.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vVHJhbnNmb3JtT3BlcmF0b3IuamF2YQ==) | `95.23% <100.00%> (+0.23%)` | :arrow_up: |
   | [...r/transform/function/TransformFunctionFactory.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vVHJhbnNmb3JtRnVuY3Rpb25GYWN0b3J5LmphdmE=) | `85.58% <100.00%> (+0.97%)` | :arrow_up: |
   | [...inot/core/query/reduce/PostAggregationHandler.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUG9zdEFnZ3JlZ2F0aW9uSGFuZGxlci5qYXZh) | `92.30% <100.00%> (+0.09%)` | :arrow_up: |
   | [.../pinot/core/query/reduce/ResultReducerFactory.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUmVzdWx0UmVkdWNlckZhY3RvcnkuamF2YQ==) | `100.00% <100.00%> (ø)` | |
   | [...nction/DistinctCountBitmapAggregationFunction.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9hZ2dyZWdhdGlvbi9mdW5jdGlvbi9EaXN0aW5jdENvdW50Qml0bWFwQWdncmVnYXRpb25GdW5jdGlvbi5qYXZh) | `41.45% <0.00%> (-12.96%)` | :arrow_down: |
   | [.../java/org/apache/pinot/spi/data/TimeFieldSpec.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9UaW1lRmllbGRTcGVjLmphdmE=) | `88.63% <0.00%> (-2.28%)` | :arrow_down: |
   | [...not/broker/broker/helix/ClusterChangeMediator.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL2hlbGl4L0NsdXN0ZXJDaGFuZ2VNZWRpYXRvci5qYXZh) | `78.02% <0.00%> (-2.20%)` | :arrow_down: |
   | ... and [11 more](https://codecov.io/gh/apache/pinot/pull/7781/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [09bb91c...2fd212f](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r759837924



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/PostAggregationGapfillQueriesTest.java
##########
@@ -0,0 +1,592 @@
+/**
+ * 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.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+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.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+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.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+
+/**
+ * Queries test for PostAggregationGapfill queries.
+ */
+@SuppressWarnings("rawtypes")
+public class PostAggregationGapfillQueriesTest extends BaseQueriesTest {
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "PostAggregationGapfillQueriesTest");
+  private static final String RAW_TABLE_NAME = "parkingData";
+  private static final String SEGMENT_NAME = "testSegment";
+  private static final Random RANDOM = new Random();
+
+  private static final int NUM_LOTS = 4;
+
+  private static final String IS_OCCUPIED_COLUMN = "isOccupied";
+  private static final String LOT_ID_COLUMN = "lotId";
+  private static final String EVENT_TIME_COLUMN = "eventTime";
+  private static final Schema SCHEMA = new Schema.SchemaBuilder()
+      .addSingleValueDimension(IS_OCCUPIED_COLUMN, DataType.BOOLEAN)
+      .addSingleValueDimension(LOT_ID_COLUMN, DataType.STRING)
+      .addSingleValueDimension(EVENT_TIME_COLUMN, DataType.LONG)
+      .setPrimaryKeyColumns(Arrays.asList(LOT_ID_COLUMN, EVENT_TIME_COLUMN))
+      .build();
+  private static final TableConfig TABLE_CONFIG = new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME)
+      .build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  @Override
+  protected String getFilter() {
+    // NOTE: Use a match all filter to switch between DictionaryBasedAggregationOperator and AggregationOperator
+    return " WHERE eventTime >= 0";
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+
+    long current = 1636286400000L; //November 7, 2021 12:00:00 PM
+    int duplicates = 16;
+    int interval = 1000 * 900; // 15 minutes
+    long start = current - duplicates * 2 * interval; //November 7, 2021 4:00:00 AM
+
+    List<GenericRow> records = new ArrayList<>(NUM_LOTS * 2);
+    for (int i = 0; i < NUM_LOTS; i++) {
+      for (int j = 0; j < duplicates; j++) {
+        if (j == 4 || j == 5 || j == 6 || j == 7 || j == 10 || j == 11) {
+          continue;
+        }
+        long parkingTime = start + interval * 2 * j + RANDOM.nextInt(interval);
+        long departingTime = j == 3 ? start + interval * (2 * j + 6) + RANDOM.nextInt(interval) : start
+            + interval * (2 * j + 1) + RANDOM.nextInt(interval);
+
+        GenericRow parkingRow = new GenericRow();
+        parkingRow.putValue(EVENT_TIME_COLUMN, parkingTime);
+        parkingRow.putValue(LOT_ID_COLUMN, "LotId_" + String.valueOf(i));
+        parkingRow.putValue(IS_OCCUPIED_COLUMN, true);
+        records.add(parkingRow);
+
+        GenericRow departingRow = new GenericRow();
+        departingRow.putValue(EVENT_TIME_COLUMN, departingTime);
+        departingRow.putValue(LOT_ID_COLUMN, "LotId_" + String.valueOf(i));
+        departingRow.putValue(IS_OCCUPIED_COLUMN, false);
+        records.add(departingRow);
+      }
+    }
+
+    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);
+  }
+
+  @Test
+  public void datetimeconvertGapfillTest() {
+    String dataTimeConvertQuery = "SELECT "
+        + "DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN')"
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    BrokerResponseNative dateTimeConvertBrokerResponse = getBrokerResponseForSqlQuery(dataTimeConvertQuery);
+
+    ResultTable dateTimeConvertResultTable = dateTimeConvertBrokerResponse.getResultTable();
+    Assert.assertEquals(dateTimeConvertResultTable.getRows().size(), 24);
+
+    String gapfillQuery = "SELECT "
+        + "PostAggregateGapFill(DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS'), "
+        + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+        + "'2021-11-07 3:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_PREVIOUS_VALUE') as status1, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_DEFAULT_VALUE') as status2, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN') as status3 "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    DateTimeFormatSpec dateTimeFormatter
+        = new DateTimeFormatSpec("1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+
+    BrokerResponseNative gapfillBrokerResponse = getBrokerResponseForSqlQuery(gapfillQuery);
+
+    ResultTable gapFillResultTable = gapfillBrokerResponse.getResultTable();
+    Assert.assertEquals(gapFillResultTable.getRows().size(), 32);
+    List<Object[]> gapFillRows = gapFillResultTable.getRows();
+    long start = dateTimeFormatter.fromFormatToMillis("2021-11-07 03:00:00.000");
+    for (int i = 0; i < 32; i += 4) {
+      String firstTimeCol = (String) gapFillRows.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Set<String> lots = new HashSet<>();
+      lots.add((String) gapFillRows.get(i)[1]);
+      for (int j = 1; j < 4; j++) {
+        Assert.assertEquals(gapFillRows.get(i)[0], gapFillRows.get(i + j)[0]);
+        Assert.assertFalse(lots.contains(gapFillRows.get(i + j)[1]));
+        lots.add((String) gapFillRows.get(i + j)[1]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochHoursGapfillTest() {
+    String dataTimeConvertQuery = "SELECT "
+        + "ToEpochHours(eventTime) AS time_col, "
+        + "lotId, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN')"
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    BrokerResponseNative dateTimeConvertBrokerResponse = getBrokerResponseForSqlQuery(dataTimeConvertQuery);
+
+    ResultTable dateTimeConvertResultTable = dateTimeConvertBrokerResponse.getResultTable();
+    Assert.assertEquals(dateTimeConvertResultTable.getRows().size(), 24);
+
+    String gapfillQuery = "SELECT "
+        + "PostAggregateGapFill(ToEpochHours(eventTime), '1:HOURS:EPOCH', "
+        + "'454515',  '454524', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_PREVIOUS_VALUE') as status1, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_DEFAULT_VALUE') as status2, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN') as status3 "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+
+    BrokerResponseNative gapfillBrokerResponse = getBrokerResponseForSqlQuery(gapfillQuery);
+
+    ResultTable gapFillResultTable = gapfillBrokerResponse.getResultTable();
+    Assert.assertEquals(gapFillResultTable.getRows().size(), 32);
+    List<Object[]> gapFillRows = gapFillResultTable.getRows();
+    long start = dateTimeFormatter.fromFormatToMillis("454515");
+    for (int i = 0; i < 32; i += 4) {
+      Long firstTimeCol = (Long) gapFillRows.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol.toString());
+      Assert.assertEquals(timeStamp, start);
+      Set<String> lots = new HashSet<>();
+      lots.add((String) gapFillRows.get(i)[1]);
+      for (int j = 1; j < 4; j++) {
+        Assert.assertEquals(gapFillRows.get(i)[0], gapFillRows.get(i + j)[0]);
+        Assert.assertFalse(lots.contains(gapFillRows.get(i + j)[1]));
+        lots.add((String) gapFillRows.get(i + j)[1]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochMinutesRoundedHoursGapfillTest() {
+    String dataTimeConvertQuery = "SELECT "
+        + "ToEpochMinutesRounded(eventTime, 60) AS time_col, "
+        + "lotId, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN')"
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    BrokerResponseNative dateTimeConvertBrokerResponse = getBrokerResponseForSqlQuery(dataTimeConvertQuery);
+
+    ResultTable dateTimeConvertResultTable = dateTimeConvertBrokerResponse.getResultTable();
+    Assert.assertEquals(dateTimeConvertResultTable.getRows().size(), 24);
+
+    String gapfillQuery = "SELECT "
+        + "PostAggregateGapFill(ToEpochMinutesRounded(eventTime, 60), '1:HOURS:EPOCH', "
+        + "'454515',  '454524', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_PREVIOUS_VALUE') as status1, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_DEFAULT_VALUE') as status2, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN') as status3 "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+
+    BrokerResponseNative gapfillBrokerResponse = getBrokerResponseForSqlQuery(gapfillQuery);
+
+    ResultTable gapFillResultTable = gapfillBrokerResponse.getResultTable();
+    Assert.assertEquals(gapFillResultTable.getRows().size(), 32);
+    List<Object[]> gapFillRows = gapFillResultTable.getRows();
+    long start = dateTimeFormatter.fromFormatToMillis("454515");
+    for (int i = 0; i < 32; i += 4) {
+      Long firstTimeCol = (Long) gapFillRows.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol.toString());
+      Assert.assertEquals(timeStamp, start);
+      Set<String> lots = new HashSet<>();
+      lots.add((String) gapFillRows.get(i)[1]);
+      for (int j = 1; j < 4; j++) {
+        Assert.assertEquals(gapFillRows.get(i)[0], gapFillRows.get(i + j)[0]);
+        Assert.assertFalse(lots.contains(gapFillRows.get(i + j)[1]));
+        lots.add((String) gapFillRows.get(i + j)[1]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochMinutesBucketHoursGapfillTest() {
+    String dataTimeConvertQuery = "SELECT "
+        + "ToEpochMinutesBucket(eventTime, 60) AS time_col, "
+        + "lotId, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN')"
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    BrokerResponseNative dateTimeConvertBrokerResponse = getBrokerResponseForSqlQuery(dataTimeConvertQuery);
+
+    ResultTable dateTimeConvertResultTable = dateTimeConvertBrokerResponse.getResultTable();
+    Assert.assertEquals(dateTimeConvertResultTable.getRows().size(), 24);
+
+    String gapfillQuery = "SELECT "
+        + "PostAggregateGapFill(ToEpochMinutesBucket(eventTime, 60), '1:HOURS:EPOCH', "
+        + "'454515',  '454524', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_PREVIOUS_VALUE') as status1, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_DEFAULT_VALUE') as status2, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN') as status3 "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+
+    BrokerResponseNative gapfillBrokerResponse = getBrokerResponseForSqlQuery(gapfillQuery);
+
+    ResultTable gapFillResultTable = gapfillBrokerResponse.getResultTable();
+    Assert.assertEquals(gapFillResultTable.getRows().size(), 32);
+    List<Object[]> gapFillRows = gapFillResultTable.getRows();
+    long start = dateTimeFormatter.fromFormatToMillis("454515");
+    for (int i = 0; i < 32; i += 4) {
+      Long firstTimeCol = (Long) gapFillRows.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol.toString());
+      Assert.assertEquals(timeStamp, start);
+      Set<String> lots = new HashSet<>();
+      lots.add((String) gapFillRows.get(i)[1]);
+      for (int j = 1; j < 4; j++) {
+        Assert.assertEquals(gapFillRows.get(i)[0], gapFillRows.get(i + j)[0]);
+        Assert.assertFalse(lots.contains(gapFillRows.get(i + j)[1]));
+        lots.add((String) gapFillRows.get(i + j)[1]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void dateTruncHoursGapfillTest() {
+    String dataTimeConvertQuery = "SELECT "
+        + "DATETRUNC('hour', eventTime, 'milliseconds') AS time_col, "
+        + "lotId, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN')"
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    BrokerResponseNative dateTimeConvertBrokerResponse = getBrokerResponseForSqlQuery(dataTimeConvertQuery);
+
+    ResultTable dateTimeConvertResultTable = dateTimeConvertBrokerResponse.getResultTable();
+    Assert.assertEquals(dateTimeConvertResultTable.getRows().size(), 24);
+
+    String gapfillQuery = "SELECT "
+        + "PostAggregateGapFill(DATETRUNC('hour', eventTime, 'milliseconds'), '1:HOURS:EPOCH', "
+        + "'454515',  '454524', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_PREVIOUS_VALUE') as status1, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_DEFAULT_VALUE') as status2, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN') as status3 "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+
+    BrokerResponseNative gapfillBrokerResponse = getBrokerResponseForSqlQuery(gapfillQuery);
+
+    ResultTable gapFillResultTable = gapfillBrokerResponse.getResultTable();
+    Assert.assertEquals(gapFillResultTable.getRows().size(), 32);
+    List<Object[]> gapFillRows = gapFillResultTable.getRows();
+    long start = dateTimeFormatter.fromFormatToMillis("454515");
+    for (int i = 0; i < 32; i += 4) {
+      Long firstTimeCol = (Long) gapFillRows.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol.toString());
+      Assert.assertEquals(timeStamp, start);
+      Set<String> lots = new HashSet<>();
+      lots.add((String) gapFillRows.get(i)[1]);
+      for (int j = 1; j < 4; j++) {
+        Assert.assertEquals(gapFillRows.get(i)[0], gapFillRows.get(i + j)[0]);
+        Assert.assertFalse(lots.contains(gapFillRows.get(i + j)[1]));
+        lots.add((String) gapFillRows.get(i + j)[1]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void datetimeconvertGapfillTestWithoutTimeBucketOrdering() {
+    try {
+      String gapfillQuery = "SELECT "
+          + "PostAggregateGapFill(DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+          + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS'), "
+          + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+          + "'2021-11-07 3:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS') AS time_col, "
+          + "lotId, "
+          + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_PREVIOUS_VALUE') as status1, "
+          + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_DEFAULT_VALUE') as status2, "
+          + "lastWithTime(isOccupied, eventTime, 'BOOLEAN') as status3 "
+          + "FROM parkingData "
+          + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+          + "GROUP BY 1, 2 "
+          + "LIMIT 200";
+
+      getBrokerResponseForSqlQuery(gapfillQuery);
+      Assert.fail();
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(e.getMessage(), "PostAggregateGapFill does not work if the time bucket is not ordered.");
+    }
+  }
+
+  @Test
+  public void datetimeconvertGapfillTestWithHavingClause() {
+    String dataTimeConvertQuery = "SELECT "
+        + "DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN')"
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "HAVING lotId IN ('LotId_0', 'LotId_1', 'LotId_2') "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    BrokerResponseNative dateTimeConvertBrokerResponse = getBrokerResponseForSqlQuery(dataTimeConvertQuery);
+
+    ResultTable dateTimeConvertResultTable = dateTimeConvertBrokerResponse.getResultTable();
+    Assert.assertEquals(dateTimeConvertResultTable.getRows().size(), 18);
+
+    String gapfillQuery = "SELECT "
+        + "PostAggregateGapFill(DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS'), "
+        + "'1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+        + "'2021-11-07 3:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS') AS time_col, "
+        + "lotId, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_PREVIOUS_VALUE') as status1, "
+        + "FILL(lastWithTime(isOccupied, eventTime, 'BOOLEAN'), 'FILL_DEFAULT_VALUE') as status2, "
+        + "lastWithTime(isOccupied, eventTime, 'BOOLEAN') as status3 "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1635940800000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1, 2 "
+        + "HAVING lotId IN ('LotId_0', 'LotId_1', 'LotId_2') "

Review comment:
       @Jackie-Jiang you are right. I will fix it. Having filter should happen after gapfilling. 




-- 
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] richardstartin commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751145324



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,725 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,
+    // a result is generated for each aggregation in the query,
+    // and the group by keys are not the same across the aggregations
+    // This PQL style of execution makes it impossible to support order by on group by.
+    //
+    // We could not simply change the group by execution behavior,
+    // as that would not be backward compatible for existing users of group by.
+    // As a result, we have 2 modes of group by execution - pql and sql - which can be controlled via query options
+    //
+    // Long term, we may completely move to sql, and keep only full sql mode alive
+    // Until then, we need to support responseFormat = sql for both the modes of execution.
+    // The 4 variants are as described below:
+
+    if (_groupByModeSql) {
+
+      if (_responseFormatSql) {
+        // 1. groupByMode = sql, responseFormat = sql
+        // This is the primary SQL compliant group by
+
+        try {
+          setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+              brokerMetrics);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // 2. groupByMode = sql, responseFormat = pql
+        // This mode will invoke SQL style group by execution, but present results in PQL way
+        // This mode is useful for users who want to avail of SQL compliant group by behavior,
+        // w/o having to forcefully move to a new result type
+
+        try {
+          setSQLGroupByInAggregationResults(brokerResponseNative, dataSchema, dataTables, reducerContext);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    } else {
+
+      // 3. groupByMode = pql, responseFormat = sql
+      // This mode is for users who want response presented in SQL style, but want PQL style group by behavior
+      // Multiple aggregations in PQL violates the tabular nature of results
+      // As a result, in this mode, only single aggregations are supported
+
+      // 4. groupByMode = pql, responseFormat = pql
+      // This is the primary PQL compliant group by
+
+      setGroupByResults(brokerResponseNative, dataTables);
+
+      if (_responseFormatSql) {
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // We emit the group by size when the result isn't empty. All the sizes among group-by results should be the
+        // same.
+        // Thus, we can just emit the one from the 1st result.
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    }
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    Object [] keyColumns = new Object[_numOfKeyColumns];
+    for (int i = 0; i < _numOfKeyColumns; i++) {
+      keyColumns[i] = row[i + 1];
+    }
+    return new Key(keyColumns);
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      int numResultColumns = resultColumnDataTypes.length;
+      int numResultRows = rows.size();
+      List<Object[]> resultRows = new ArrayList<>(numResultRows);
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < numResultColumns; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {
+      Set<Key> keys = new HashSet<>(_primaryKeys);
+      while (index < resultRows.size()) {
+        long timeCol = _dateTimeFormatter.fromFormatToMillis((String) resultRows.get(index)[0]);
+        if (timeCol < time) {
+          index++;
+        } else if (timeCol == time) {
+          gapfillResultRows.add(resultRows.get(index));
+          if (gapfillResultRows.size() == limit) {
+            return gapfillResultRows;
+          }
+          Key key = constructKey(resultRows.get(index));
+          keys.remove(key);
+          for (int colIndex = 2; colIndex < numResultColumns; colIndex++) {
+            _previous.put(key, resultRows.get(index));
+          }
+          index++;
+        } else {
+          break;
+        }
+      }
+      for (Key key : keys) {
+        Object[] gapfillRow = new Object[numResultColumns];
+        gapfillRow[0] = _dateTimeFormatter.fromMillisToFormat(time);
+        for (int i = 0; i < _numOfKeyColumns; i++) {
+          gapfillRow[i + 1] = key.getValues()[i];
+        }
+
+        for (int i = _numOfKeyColumns + 1; i < numResultColumns; i++) {
+          gapfillRow[i] = getFillValue(i, key, resultColumnDataTypes[i]);
+        }
+        gapfillResultRows.add(gapfillRow);
+        if (gapfillResultRows.size() == limit) {
+          return gapfillResultRows;
+        }
+      }
+    }
+    return gapfillResultRows;
+  }
+
+  Object getFillValue(int columIndex, Object key, ColumnDataType dataType) {
+    ExpressionContext expressionContext = _queryContext.getSelectExpressions().get(columIndex);
+    if (expressionContext.getFunction() != null
+        && expressionContext.getFunction().getFunctionName().equalsIgnoreCase("fill")) {
+      List<ExpressionContext> args = expressionContext.getFunction().getArguments();
+      if (args.get(1).getLiteral() == null) {
+        throw new UnsupportedOperationException("Wrong Sql.");
+      }
+      FillType fillType = FillType.valueOf(args.get(1).getLiteral());
+      if (fillType == FillType.FILL_DEFAULT_VALUE) {
+        // TODO: may fill the default value from sql in the future.
+        return SelectionOperatorUtils.getDefaultValue(dataType);
+      } else if (fillType == FillType.FILL_PREVIOUS_VALUE) {
+        if (_previous.containsKey(key)) {
+          return _previous.get(key)[columIndex];
+        } else {
+          return SelectionOperatorUtils.getDefaultValue(dataType);
+        }
+      } else {
+        throw new UnsupportedOperationException("unsupported fill type.");
+      }
+    } else {
+      return SelectionOperatorUtils.getDefaultValue(dataType);
+    }
+  }
+
+  enum FillType {
+    FILL_DEFAULT_VALUE,
+    FILL_PREVIOUS_VALUE,
+  }
+
+  /**
+   * Helper method to extract the final aggregation results for the given row (in-place).
+   */
+  private void extractFinalAggregationResults(Object[] row) {
+    for (int i = 0; i < _numAggregationFunctions; i++) {
+      int valueIndex = i + _numGroupByExpressions;
+      row[valueIndex] = _aggregationFunctions[i].extractFinalResult(row[valueIndex]);
+    }
+  }
+
+  /**
+   * Constructs the DataSchema for the rows before the post-aggregation (SQL mode).
+   */
+  private DataSchema getPrePostAggregationDataSchema(DataSchema dataSchema) {
+    String[] columnNames = dataSchema.getColumnNames();
+    ColumnDataType[] columnDataTypes = new ColumnDataType[_numColumns];
+    System.arraycopy(dataSchema.getColumnDataTypes(), 0, columnDataTypes, 0, _numGroupByExpressions);
+    for (int i = 0; i < _numAggregationFunctions; i++) {
+      columnDataTypes[i + _numGroupByExpressions] = _aggregationFunctions[i].getFinalResultColumnType();
+    }
+    return new DataSchema(columnNames, columnDataTypes);
+  }
+
+  private IndexedTable getIndexedTable(DataSchema dataSchema, Collection<DataTable> dataTablesToReduce,
+      DataTableReducerContext reducerContext)
+      throws TimeoutException {
+    long start = System.currentTimeMillis();
+    int numDataTables = dataTablesToReduce.size();
+
+    // Get the number of threads to use for reducing.
+    // In case of single reduce thread, fall back to SimpleIndexedTable to avoid redundant locking/unlocking calls.
+    int numReduceThreadsToUse = getNumReduceThreadsToUse(numDataTables, reducerContext.getMaxReduceThreadsPerQuery());
+    int limit = _queryContext.getLimit();
+    // TODO: Make minTrimSize configurable
+    int trimSize = GroupByUtils.getTableCapacity(limit);
+    // NOTE: For query with HAVING clause, use trimSize as resultSize to ensure the result accuracy.
+    // TODO: Resolve the HAVING clause within the IndexedTable before returning the result
+    int resultSize = _queryContext.getHavingFilter() != null ? trimSize : limit;
+    int trimThreshold = reducerContext.getGroupByTrimThreshold();
+    IndexedTable indexedTable;
+    if (numReduceThreadsToUse <= 1) {
+      indexedTable = new SimpleIndexedTable(dataSchema, _queryContext, resultSize, trimSize, trimThreshold);
+    } else {
+      if (trimThreshold >= GroupByOrderByCombineOperator.MAX_TRIM_THRESHOLD) {
+        // special case of trim threshold where it is set to max value.
+        // there won't be any trimming during upsert in this case.
+        // thus we can avoid the overhead of read-lock and write-lock
+        // in the upsert method.
+        indexedTable = new UnboundedConcurrentIndexedTable(dataSchema, _queryContext, resultSize);
+      } else {
+        indexedTable = new ConcurrentIndexedTable(dataSchema, _queryContext, resultSize, trimSize, trimThreshold);
+      }
+    }
+
+    Future[] futures = new Future[numDataTables];
+    CountDownLatch countDownLatch = new CountDownLatch(numDataTables);

Review comment:
       Could these be define closer to where they are used? I think this would improve readability.




-- 
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] richardstartin commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751144308



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,725 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,
+    // a result is generated for each aggregation in the query,
+    // and the group by keys are not the same across the aggregations
+    // This PQL style of execution makes it impossible to support order by on group by.
+    //
+    // We could not simply change the group by execution behavior,
+    // as that would not be backward compatible for existing users of group by.
+    // As a result, we have 2 modes of group by execution - pql and sql - which can be controlled via query options
+    //
+    // Long term, we may completely move to sql, and keep only full sql mode alive
+    // Until then, we need to support responseFormat = sql for both the modes of execution.
+    // The 4 variants are as described below:
+
+    if (_groupByModeSql) {
+
+      if (_responseFormatSql) {
+        // 1. groupByMode = sql, responseFormat = sql
+        // This is the primary SQL compliant group by
+
+        try {
+          setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+              brokerMetrics);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // 2. groupByMode = sql, responseFormat = pql
+        // This mode will invoke SQL style group by execution, but present results in PQL way
+        // This mode is useful for users who want to avail of SQL compliant group by behavior,
+        // w/o having to forcefully move to a new result type
+
+        try {
+          setSQLGroupByInAggregationResults(brokerResponseNative, dataSchema, dataTables, reducerContext);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    } else {
+
+      // 3. groupByMode = pql, responseFormat = sql
+      // This mode is for users who want response presented in SQL style, but want PQL style group by behavior
+      // Multiple aggregations in PQL violates the tabular nature of results
+      // As a result, in this mode, only single aggregations are supported
+
+      // 4. groupByMode = pql, responseFormat = pql
+      // This is the primary PQL compliant group by
+
+      setGroupByResults(brokerResponseNative, dataTables);
+
+      if (_responseFormatSql) {
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // We emit the group by size when the result isn't empty. All the sizes among group-by results should be the
+        // same.
+        // Thus, we can just emit the one from the 1st result.
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    }
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    Object [] keyColumns = new Object[_numOfKeyColumns];
+    for (int i = 0; i < _numOfKeyColumns; i++) {
+      keyColumns[i] = row[i + 1];
+    }
+    return new Key(keyColumns);
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      int numResultColumns = resultColumnDataTypes.length;
+      int numResultRows = rows.size();
+      List<Object[]> resultRows = new ArrayList<>(numResultRows);
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < numResultColumns; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {
+      Set<Key> keys = new HashSet<>(_primaryKeys);
+      while (index < resultRows.size()) {
+        long timeCol = _dateTimeFormatter.fromFormatToMillis((String) resultRows.get(index)[0]);
+        if (timeCol < time) {
+          index++;
+        } else if (timeCol == time) {
+          gapfillResultRows.add(resultRows.get(index));
+          if (gapfillResultRows.size() == limit) {
+            return gapfillResultRows;
+          }
+          Key key = constructKey(resultRows.get(index));
+          keys.remove(key);
+          for (int colIndex = 2; colIndex < numResultColumns; colIndex++) {
+            _previous.put(key, resultRows.get(index));
+          }
+          index++;
+        } else {
+          break;
+        }
+      }
+      for (Key key : keys) {
+        Object[] gapfillRow = new Object[numResultColumns];
+        gapfillRow[0] = _dateTimeFormatter.fromMillisToFormat(time);
+        for (int i = 0; i < _numOfKeyColumns; i++) {
+          gapfillRow[i + 1] = key.getValues()[i];
+        }
+
+        for (int i = _numOfKeyColumns + 1; i < numResultColumns; i++) {
+          gapfillRow[i] = getFillValue(i, key, resultColumnDataTypes[i]);
+        }
+        gapfillResultRows.add(gapfillRow);
+        if (gapfillResultRows.size() == limit) {
+          return gapfillResultRows;
+        }
+      }
+    }
+    return gapfillResultRows;
+  }
+
+  Object getFillValue(int columIndex, Object key, ColumnDataType dataType) {
+    ExpressionContext expressionContext = _queryContext.getSelectExpressions().get(columIndex);
+    if (expressionContext.getFunction() != null
+        && expressionContext.getFunction().getFunctionName().equalsIgnoreCase("fill")) {
+      List<ExpressionContext> args = expressionContext.getFunction().getArguments();
+      if (args.get(1).getLiteral() == null) {
+        throw new UnsupportedOperationException("Wrong Sql.");
+      }
+      FillType fillType = FillType.valueOf(args.get(1).getLiteral());
+      if (fillType == FillType.FILL_DEFAULT_VALUE) {
+        // TODO: may fill the default value from sql in the future.
+        return SelectionOperatorUtils.getDefaultValue(dataType);
+      } else if (fillType == FillType.FILL_PREVIOUS_VALUE) {
+        if (_previous.containsKey(key)) {
+          return _previous.get(key)[columIndex];

Review comment:
       For performance reasons it's much better to get and do a null check:
   
   ```java
   Object[] value = _previous.get(key);
   if (value != null) {
     return value[columnIndex];
   }
   ```
   
   This results in doing half the work to locate the value.




-- 
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] richardstartin commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751142485



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,725 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,
+    // a result is generated for each aggregation in the query,
+    // and the group by keys are not the same across the aggregations
+    // This PQL style of execution makes it impossible to support order by on group by.
+    //
+    // We could not simply change the group by execution behavior,
+    // as that would not be backward compatible for existing users of group by.
+    // As a result, we have 2 modes of group by execution - pql and sql - which can be controlled via query options
+    //
+    // Long term, we may completely move to sql, and keep only full sql mode alive
+    // Until then, we need to support responseFormat = sql for both the modes of execution.
+    // The 4 variants are as described below:
+
+    if (_groupByModeSql) {
+
+      if (_responseFormatSql) {
+        // 1. groupByMode = sql, responseFormat = sql
+        // This is the primary SQL compliant group by
+
+        try {
+          setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+              brokerMetrics);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // 2. groupByMode = sql, responseFormat = pql
+        // This mode will invoke SQL style group by execution, but present results in PQL way
+        // This mode is useful for users who want to avail of SQL compliant group by behavior,
+        // w/o having to forcefully move to a new result type
+
+        try {
+          setSQLGroupByInAggregationResults(brokerResponseNative, dataSchema, dataTables, reducerContext);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    } else {
+
+      // 3. groupByMode = pql, responseFormat = sql
+      // This mode is for users who want response presented in SQL style, but want PQL style group by behavior
+      // Multiple aggregations in PQL violates the tabular nature of results
+      // As a result, in this mode, only single aggregations are supported
+
+      // 4. groupByMode = pql, responseFormat = pql
+      // This is the primary PQL compliant group by
+
+      setGroupByResults(brokerResponseNative, dataTables);
+
+      if (_responseFormatSql) {
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // We emit the group by size when the result isn't empty. All the sizes among group-by results should be the
+        // same.
+        // Thus, we can just emit the one from the 1st result.
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    }
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    Object [] keyColumns = new Object[_numOfKeyColumns];
+    for (int i = 0; i < _numOfKeyColumns; i++) {
+      keyColumns[i] = row[i + 1];
+    }
+    return new Key(keyColumns);
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      int numResultColumns = resultColumnDataTypes.length;
+      int numResultRows = rows.size();
+      List<Object[]> resultRows = new ArrayList<>(numResultRows);
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < numResultColumns; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {
+      Set<Key> keys = new HashSet<>(_primaryKeys);
+      while (index < resultRows.size()) {
+        long timeCol = _dateTimeFormatter.fromFormatToMillis((String) resultRows.get(index)[0]);
+        if (timeCol < time) {
+          index++;
+        } else if (timeCol == time) {
+          gapfillResultRows.add(resultRows.get(index));
+          if (gapfillResultRows.size() == limit) {
+            return gapfillResultRows;
+          }
+          Key key = constructKey(resultRows.get(index));
+          keys.remove(key);
+          for (int colIndex = 2; colIndex < numResultColumns; colIndex++) {
+            _previous.put(key, resultRows.get(index));
+          }

Review comment:
       this loop performs the same operation in each iteration. Should this be `resultRows.get(colIndex)`? Are these lines covered by tests?




-- 
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] richardstartin commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751136856



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,725 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,
+    // a result is generated for each aggregation in the query,
+    // and the group by keys are not the same across the aggregations
+    // This PQL style of execution makes it impossible to support order by on group by.
+    //
+    // We could not simply change the group by execution behavior,
+    // as that would not be backward compatible for existing users of group by.
+    // As a result, we have 2 modes of group by execution - pql and sql - which can be controlled via query options
+    //
+    // Long term, we may completely move to sql, and keep only full sql mode alive
+    // Until then, we need to support responseFormat = sql for both the modes of execution.
+    // The 4 variants are as described below:
+
+    if (_groupByModeSql) {
+
+      if (_responseFormatSql) {
+        // 1. groupByMode = sql, responseFormat = sql
+        // This is the primary SQL compliant group by
+
+        try {
+          setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+              brokerMetrics);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // 2. groupByMode = sql, responseFormat = pql
+        // This mode will invoke SQL style group by execution, but present results in PQL way
+        // This mode is useful for users who want to avail of SQL compliant group by behavior,
+        // w/o having to forcefully move to a new result type
+
+        try {
+          setSQLGroupByInAggregationResults(brokerResponseNative, dataSchema, dataTables, reducerContext);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    } else {
+
+      // 3. groupByMode = pql, responseFormat = sql
+      // This mode is for users who want response presented in SQL style, but want PQL style group by behavior
+      // Multiple aggregations in PQL violates the tabular nature of results
+      // As a result, in this mode, only single aggregations are supported
+
+      // 4. groupByMode = pql, responseFormat = pql
+      // This is the primary PQL compliant group by
+
+      setGroupByResults(brokerResponseNative, dataTables);
+
+      if (_responseFormatSql) {
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // We emit the group by size when the result isn't empty. All the sizes among group-by results should be the
+        // same.
+        // Thus, we can just emit the one from the 1st result.
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    }
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    Object [] keyColumns = new Object[_numOfKeyColumns];
+    for (int i = 0; i < _numOfKeyColumns; i++) {
+      keyColumns[i] = row[i + 1];
+    }
+    return new Key(keyColumns);

Review comment:
       `return new Key(Arrays.copyOfRange(row, 1, _numOfKeyColumns));`




-- 
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] richardstartin commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751574018



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,725 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,
+    // a result is generated for each aggregation in the query,
+    // and the group by keys are not the same across the aggregations
+    // This PQL style of execution makes it impossible to support order by on group by.
+    //
+    // We could not simply change the group by execution behavior,
+    // as that would not be backward compatible for existing users of group by.
+    // As a result, we have 2 modes of group by execution - pql and sql - which can be controlled via query options
+    //
+    // Long term, we may completely move to sql, and keep only full sql mode alive
+    // Until then, we need to support responseFormat = sql for both the modes of execution.
+    // The 4 variants are as described below:
+
+    if (_groupByModeSql) {
+
+      if (_responseFormatSql) {
+        // 1. groupByMode = sql, responseFormat = sql
+        // This is the primary SQL compliant group by
+
+        try {
+          setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+              brokerMetrics);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // 2. groupByMode = sql, responseFormat = pql
+        // This mode will invoke SQL style group by execution, but present results in PQL way
+        // This mode is useful for users who want to avail of SQL compliant group by behavior,
+        // w/o having to forcefully move to a new result type
+
+        try {
+          setSQLGroupByInAggregationResults(brokerResponseNative, dataSchema, dataTables, reducerContext);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    } else {
+
+      // 3. groupByMode = pql, responseFormat = sql
+      // This mode is for users who want response presented in SQL style, but want PQL style group by behavior
+      // Multiple aggregations in PQL violates the tabular nature of results
+      // As a result, in this mode, only single aggregations are supported
+
+      // 4. groupByMode = pql, responseFormat = pql
+      // This is the primary PQL compliant group by
+
+      setGroupByResults(brokerResponseNative, dataTables);
+
+      if (_responseFormatSql) {
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // We emit the group by size when the result isn't empty. All the sizes among group-by results should be the
+        // same.
+        // Thus, we can just emit the one from the 1st result.
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    }
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    Object [] keyColumns = new Object[_numOfKeyColumns];
+    for (int i = 0; i < _numOfKeyColumns; i++) {
+      keyColumns[i] = row[i + 1];
+    }
+    return new Key(keyColumns);
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      int numResultColumns = resultColumnDataTypes.length;
+      int numResultRows = rows.size();
+      List<Object[]> resultRows = new ArrayList<>(numResultRows);
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < numResultColumns; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {
+      Set<Key> keys = new HashSet<>(_primaryKeys);
+      while (index < resultRows.size()) {
+        long timeCol = _dateTimeFormatter.fromFormatToMillis((String) resultRows.get(index)[0]);
+        if (timeCol < time) {
+          index++;
+        } else if (timeCol == time) {
+          gapfillResultRows.add(resultRows.get(index));
+          if (gapfillResultRows.size() == limit) {
+            return gapfillResultRows;
+          }
+          Key key = constructKey(resultRows.get(index));
+          keys.remove(key);
+          for (int colIndex = 2; colIndex < numResultColumns; colIndex++) {
+            _previous.put(key, resultRows.get(index));
+          }
+          index++;
+        } else {
+          break;
+        }
+      }
+      for (Key key : keys) {
+        Object[] gapfillRow = new Object[numResultColumns];
+        gapfillRow[0] = _dateTimeFormatter.fromMillisToFormat(time);
+        for (int i = 0; i < _numOfKeyColumns; i++) {
+          gapfillRow[i + 1] = key.getValues()[i];
+        }
+
+        for (int i = _numOfKeyColumns + 1; i < numResultColumns; i++) {
+          gapfillRow[i] = getFillValue(i, key, resultColumnDataTypes[i]);
+        }
+        gapfillResultRows.add(gapfillRow);
+        if (gapfillResultRows.size() == limit) {
+          return gapfillResultRows;
+        }
+      }
+    }
+    return gapfillResultRows;
+  }
+
+  Object getFillValue(int columIndex, Object key, ColumnDataType dataType) {
+    ExpressionContext expressionContext = _queryContext.getSelectExpressions().get(columIndex);
+    if (expressionContext.getFunction() != null
+        && expressionContext.getFunction().getFunctionName().equalsIgnoreCase("fill")) {
+      List<ExpressionContext> args = expressionContext.getFunction().getArguments();
+      if (args.get(1).getLiteral() == null) {
+        throw new UnsupportedOperationException("Wrong Sql.");
+      }
+      FillType fillType = FillType.valueOf(args.get(1).getLiteral());
+      if (fillType == FillType.FILL_DEFAULT_VALUE) {
+        // TODO: may fill the default value from sql in the future.
+        return SelectionOperatorUtils.getDefaultValue(dataType);
+      } else if (fillType == FillType.FILL_PREVIOUS_VALUE) {
+        if (_previous.containsKey(key)) {
+          return _previous.get(key)[columIndex];
+        } else {
+          return SelectionOperatorUtils.getDefaultValue(dataType);
+        }
+      } else {
+        throw new UnsupportedOperationException("unsupported fill type.");
+      }
+    } else {
+      return SelectionOperatorUtils.getDefaultValue(dataType);
+    }
+  }
+
+  enum FillType {
+    FILL_DEFAULT_VALUE,
+    FILL_PREVIOUS_VALUE,
+  }
+
+  /**
+   * Helper method to extract the final aggregation results for the given row (in-place).
+   */
+  private void extractFinalAggregationResults(Object[] row) {
+    for (int i = 0; i < _numAggregationFunctions; i++) {
+      int valueIndex = i + _numGroupByExpressions;
+      row[valueIndex] = _aggregationFunctions[i].extractFinalResult(row[valueIndex]);
+    }
+  }
+
+  /**
+   * Constructs the DataSchema for the rows before the post-aggregation (SQL mode).
+   */
+  private DataSchema getPrePostAggregationDataSchema(DataSchema dataSchema) {
+    String[] columnNames = dataSchema.getColumnNames();
+    ColumnDataType[] columnDataTypes = new ColumnDataType[_numColumns];
+    System.arraycopy(dataSchema.getColumnDataTypes(), 0, columnDataTypes, 0, _numGroupByExpressions);
+    for (int i = 0; i < _numAggregationFunctions; i++) {
+      columnDataTypes[i + _numGroupByExpressions] = _aggregationFunctions[i].getFinalResultColumnType();
+    }
+    return new DataSchema(columnNames, columnDataTypes);
+  }
+
+  private IndexedTable getIndexedTable(DataSchema dataSchema, Collection<DataTable> dataTablesToReduce,
+      DataTableReducerContext reducerContext)
+      throws TimeoutException {
+    long start = System.currentTimeMillis();
+    int numDataTables = dataTablesToReduce.size();
+
+    // Get the number of threads to use for reducing.
+    // In case of single reduce thread, fall back to SimpleIndexedTable to avoid redundant locking/unlocking calls.
+    int numReduceThreadsToUse = getNumReduceThreadsToUse(numDataTables, reducerContext.getMaxReduceThreadsPerQuery());
+    int limit = _queryContext.getLimit();
+    // TODO: Make minTrimSize configurable
+    int trimSize = GroupByUtils.getTableCapacity(limit);
+    // NOTE: For query with HAVING clause, use trimSize as resultSize to ensure the result accuracy.
+    // TODO: Resolve the HAVING clause within the IndexedTable before returning the result
+    int resultSize = _queryContext.getHavingFilter() != null ? trimSize : limit;
+    int trimThreshold = reducerContext.getGroupByTrimThreshold();
+    IndexedTable indexedTable;
+    if (numReduceThreadsToUse <= 1) {
+      indexedTable = new SimpleIndexedTable(dataSchema, _queryContext, resultSize, trimSize, trimThreshold);
+    } else {
+      if (trimThreshold >= GroupByOrderByCombineOperator.MAX_TRIM_THRESHOLD) {
+        // special case of trim threshold where it is set to max value.
+        // there won't be any trimming during upsert in this case.
+        // thus we can avoid the overhead of read-lock and write-lock
+        // in the upsert method.
+        indexedTable = new UnboundedConcurrentIndexedTable(dataSchema, _queryContext, resultSize);
+      } else {
+        indexedTable = new ConcurrentIndexedTable(dataSchema, _queryContext, resultSize, trimSize, trimThreshold);
+      }
+    }
+
+    Future[] futures = new Future[numDataTables];
+    CountDownLatch countDownLatch = new CountDownLatch(numDataTables);
+
+    // Create groups of data tables that each thread can process concurrently.
+    // Given that numReduceThreads is <= numDataTables, each group will have at least one data table.
+    ArrayList<DataTable> dataTables = new ArrayList<>(dataTablesToReduce);
+    List<List<DataTable>> reduceGroups = new ArrayList<>(numReduceThreadsToUse);
+
+    for (int i = 0; i < numReduceThreadsToUse; i++) {
+      reduceGroups.add(new ArrayList<>());
+    }
+    for (int i = 0; i < numDataTables; i++) {
+      reduceGroups.get(i % numReduceThreadsToUse).add(dataTables.get(i));
+    }
+
+    int cnt = 0;
+    ColumnDataType[] storedColumnDataTypes = dataSchema.getStoredColumnDataTypes();
+    for (List<DataTable> reduceGroup : reduceGroups) {
+      futures[cnt++] = reducerContext.getExecutorService().submit(new TraceRunnable() {
+        @Override
+        public void runJob() {
+          for (DataTable dataTable : reduceGroup) {
+            int numRows = dataTable.getNumberOfRows();
+
+            try {
+              for (int rowId = 0; rowId < numRows; rowId++) {
+                Object[] values = new Object[_numColumns];
+                for (int colId = 0; colId < _numColumns; colId++) {
+                  switch (storedColumnDataTypes[colId]) {
+                    case INT:
+                      values[colId] = dataTable.getInt(rowId, colId);
+                      break;
+                    case LONG:
+                      values[colId] = dataTable.getLong(rowId, colId);
+                      break;
+                    case FLOAT:
+                      values[colId] = dataTable.getFloat(rowId, colId);
+                      break;
+                    case DOUBLE:
+                      values[colId] = dataTable.getDouble(rowId, colId);
+                      break;
+                    case STRING:
+                      values[colId] = dataTable.getString(rowId, colId);
+                      break;
+                    case BYTES:
+                      values[colId] = dataTable.getBytes(rowId, colId);
+                      break;
+                    case OBJECT:
+                      values[colId] = dataTable.getObject(rowId, colId);
+                      break;
+                    // Add other aggregation intermediate result / group-by column type supports here
+                    default:
+                      throw new IllegalStateException();
+                  }
+                }
+                indexedTable.upsert(new Record(values));
+              }
+            } finally {
+              countDownLatch.countDown();
+            }
+          }
+        }
+      });
+    }
+
+    try {
+      long timeOutMs = reducerContext.getReduceTimeOutMs() - (System.currentTimeMillis() - start);
+      countDownLatch.await(timeOutMs, TimeUnit.MILLISECONDS);
+    } catch (InterruptedException e) {
+      for (Future future : futures) {
+        if (!future.isDone()) {
+          future.cancel(true);
+        }
+      }
+      throw new TimeoutException("Timed out in broker reduce phase.");
+    }
+
+    indexedTable.finish(true);
+    return indexedTable;
+  }
+
+  /**
+   * Computes the number of reduce threads to use per query.
+   * <ul>
+   *   <li> Use single thread if number of data tables to reduce is less than
+   *   {@value #MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE}.</li>
+   *   <li> Else, use min of max allowed reduce threads per query, and number of data tables.</li>
+   * </ul>
+   *
+   * @param numDataTables Number of data tables to reduce
+   * @param maxReduceThreadsPerQuery Max allowed reduce threads per query
+   * @return Number of reduce threads to use for the query
+   */
+  private int getNumReduceThreadsToUse(int numDataTables, int maxReduceThreadsPerQuery) {
+    // Use single thread if number of data tables < MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE.
+    if (numDataTables < MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE) {
+      return Math.min(1, numDataTables); // Number of data tables can be zero.
+    }
+
+    return Math.min(maxReduceThreadsPerQuery, numDataTables);
+  }
+
+  /**
+   * Extract the results of group by order by into a List of {@link AggregationResult}
+   * There will be 1 aggregation result per aggregation. The group by keys will be the same across all aggregations
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @throws TimeoutException If unable to complete within the timeout.
+   */
+  private void setSQLGroupByInAggregationResults(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext)
+      throws TimeoutException {
+
+    List<String> groupByColumns = new ArrayList<>(_numGroupByExpressions);
+    int idx = 0;
+    while (idx < _numGroupByExpressions) {
+      groupByColumns.add(dataSchema.getColumnName(idx));
+      idx++;
+    }
+
+    List<String> aggregationColumns = new ArrayList<>(_numAggregationFunctions);
+    List<List<GroupByResult>> groupByResults = new ArrayList<>(_numAggregationFunctions);
+    while (idx < _numColumns) {
+      aggregationColumns.add(dataSchema.getColumnName(idx));
+      groupByResults.add(new ArrayList<>());

Review comment:
       It’s not obvious how to calculate the size properly so feel free to ignore this




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751571702



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/selection/SelectionOperatorUtils.java
##########
@@ -339,6 +339,74 @@ public static DataTable getDataTableFromRows(Collection<Object[]> rows, DataSche
     return dataTableBuilder.build();
   }
 
+  /**
+   * The default value for each column type.
+   */
+  public static Serializable getDefaultValue(ColumnDataType dataType) {
+    switch (dataType) {
+      // Single-value column
+      case INT:
+        return THREAD_LOCAL_INT_FORMAT.get().format(((Number) 0).intValue());
+      case LONG:
+        return THREAD_LOCAL_LONG_FORMAT.get().format(((Number) 0).longValue());
+      case FLOAT:
+        return THREAD_LOCAL_FLOAT_FORMAT.get().format(((Number) 0).floatValue());
+      case DOUBLE:
+        return THREAD_LOCAL_DOUBLE_FORMAT.get().format(((Number) 0).doubleValue());

Review comment:
       Any comment?




-- 
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 #7781: Add Post-Aggregation Gapfilling functionality.

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


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?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 [#7781](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (442f220) into [master](https://codecov.io/gh/apache/pinot/commit/09bb91cecc8cff459a9d7814a7fe61aa67f2a393?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (09bb91c) will **decrease** coverage by `57.11%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7781/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #7781       +/-   ##
   =============================================
   - Coverage     71.62%   14.51%   -57.12%     
   + Complexity     4079       80     -3999     
   =============================================
     Files          1578     1534       -44     
     Lines         80641    79174     -1467     
     Branches      11987    11866      -121     
   =============================================
   - Hits          57758    11489    -46269     
   - Misses        18994    66844    +47850     
   + Partials       3889      841     -3048     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `?` | |
   | unittests2 | `14.51% <0.00%> (-0.10%)` | :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/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...not/core/operator/transform/TransformOperator.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vVHJhbnNmb3JtT3BlcmF0b3IuamF2YQ==) | `0.00% <0.00%> (-95.00%)` | :arrow_down: |
   | [...r/transform/function/TransformFunctionFactory.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vVHJhbnNmb3JtRnVuY3Rpb25GYWN0b3J5LmphdmE=) | `0.00% <0.00%> (-84.62%)` | :arrow_down: |
   | [...e/query/reduce/GapFillGroupByDataTableReducer.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvR2FwRmlsbEdyb3VwQnlEYXRhVGFibGVSZWR1Y2VyLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...inot/core/query/reduce/PostAggregationHandler.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUG9zdEFnZ3JlZ2F0aW9uSGFuZGxlci5qYXZh) | `0.00% <0.00%> (-92.21%)` | :arrow_down: |
   | [.../pinot/core/query/reduce/ResultReducerFactory.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUmVzdWx0UmVkdWNlckZhY3RvcnkuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pinot/core/query/request/context/QueryContext.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZXF1ZXN0L2NvbnRleHQvUXVlcnlDb250ZXh0LmphdmE=) | `0.00% <0.00%> (-97.91%)` | :arrow_down: |
   | [...t/core/query/selection/SelectionOperatorUtils.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9zZWxlY3Rpb24vU2VsZWN0aW9uT3BlcmF0b3JVdGlscy5qYXZh) | `0.00% <0.00%> (-92.19%)` | :arrow_down: |
   | [...ain/java/org/apache/pinot/core/data/table/Key.java](https://codecov.io/gh/apache/pinot/pull/7781/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: |
   | [.../java/org/apache/pinot/spi/utils/BooleanUtils.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQm9vbGVhblV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/data/table/Record.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL1JlY29yZC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [1256 more](https://codecov.io/gh/apache/pinot/pull/7781/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [09bb91c...442f220](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751464743



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,725 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,
+    // a result is generated for each aggregation in the query,
+    // and the group by keys are not the same across the aggregations
+    // This PQL style of execution makes it impossible to support order by on group by.
+    //
+    // We could not simply change the group by execution behavior,
+    // as that would not be backward compatible for existing users of group by.
+    // As a result, we have 2 modes of group by execution - pql and sql - which can be controlled via query options
+    //
+    // Long term, we may completely move to sql, and keep only full sql mode alive
+    // Until then, we need to support responseFormat = sql for both the modes of execution.
+    // The 4 variants are as described below:
+
+    if (_groupByModeSql) {
+
+      if (_responseFormatSql) {
+        // 1. groupByMode = sql, responseFormat = sql
+        // This is the primary SQL compliant group by
+
+        try {
+          setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+              brokerMetrics);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // 2. groupByMode = sql, responseFormat = pql
+        // This mode will invoke SQL style group by execution, but present results in PQL way
+        // This mode is useful for users who want to avail of SQL compliant group by behavior,
+        // w/o having to forcefully move to a new result type
+
+        try {
+          setSQLGroupByInAggregationResults(brokerResponseNative, dataSchema, dataTables, reducerContext);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    } else {
+
+      // 3. groupByMode = pql, responseFormat = sql
+      // This mode is for users who want response presented in SQL style, but want PQL style group by behavior
+      // Multiple aggregations in PQL violates the tabular nature of results
+      // As a result, in this mode, only single aggregations are supported
+
+      // 4. groupByMode = pql, responseFormat = pql
+      // This is the primary PQL compliant group by
+
+      setGroupByResults(brokerResponseNative, dataTables);
+
+      if (_responseFormatSql) {
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // We emit the group by size when the result isn't empty. All the sizes among group-by results should be the
+        // same.
+        // Thus, we can just emit the one from the 1st result.
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    }
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    Object [] keyColumns = new Object[_numOfKeyColumns];
+    for (int i = 0; i < _numOfKeyColumns; i++) {
+      keyColumns[i] = row[i + 1];
+    }
+    return new Key(keyColumns);
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      int numResultColumns = resultColumnDataTypes.length;
+      int numResultRows = rows.size();
+      List<Object[]> resultRows = new ArrayList<>(numResultRows);
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < numResultColumns; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {
+      Set<Key> keys = new HashSet<>(_primaryKeys);
+      while (index < resultRows.size()) {
+        long timeCol = _dateTimeFormatter.fromFormatToMillis((String) resultRows.get(index)[0]);
+        if (timeCol < time) {
+          index++;
+        } else if (timeCol == time) {
+          gapfillResultRows.add(resultRows.get(index));
+          if (gapfillResultRows.size() == limit) {
+            return gapfillResultRows;
+          }
+          Key key = constructKey(resultRows.get(index));
+          keys.remove(key);
+          for (int colIndex = 2; colIndex < numResultColumns; colIndex++) {
+            _previous.put(key, resultRows.get(index));
+          }
+          index++;
+        } else {
+          break;
+        }
+      }
+      for (Key key : keys) {
+        Object[] gapfillRow = new Object[numResultColumns];
+        gapfillRow[0] = _dateTimeFormatter.fromMillisToFormat(time);
+        for (int i = 0; i < _numOfKeyColumns; i++) {
+          gapfillRow[i + 1] = key.getValues()[i];
+        }
+
+        for (int i = _numOfKeyColumns + 1; i < numResultColumns; i++) {
+          gapfillRow[i] = getFillValue(i, key, resultColumnDataTypes[i]);
+        }
+        gapfillResultRows.add(gapfillRow);
+        if (gapfillResultRows.size() == limit) {
+          return gapfillResultRows;
+        }
+      }
+    }
+    return gapfillResultRows;
+  }
+
+  Object getFillValue(int columIndex, Object key, ColumnDataType dataType) {
+    ExpressionContext expressionContext = _queryContext.getSelectExpressions().get(columIndex);
+    if (expressionContext.getFunction() != null
+        && expressionContext.getFunction().getFunctionName().equalsIgnoreCase("fill")) {
+      List<ExpressionContext> args = expressionContext.getFunction().getArguments();
+      if (args.get(1).getLiteral() == null) {
+        throw new UnsupportedOperationException("Wrong Sql.");
+      }
+      FillType fillType = FillType.valueOf(args.get(1).getLiteral());
+      if (fillType == FillType.FILL_DEFAULT_VALUE) {
+        // TODO: may fill the default value from sql in the future.
+        return SelectionOperatorUtils.getDefaultValue(dataType);
+      } else if (fillType == FillType.FILL_PREVIOUS_VALUE) {
+        if (_previous.containsKey(key)) {
+          return _previous.get(key)[columIndex];
+        } else {
+          return SelectionOperatorUtils.getDefaultValue(dataType);
+        }
+      } else {
+        throw new UnsupportedOperationException("unsupported fill type.");
+      }
+    } else {
+      return SelectionOperatorUtils.getDefaultValue(dataType);
+    }
+  }
+
+  enum FillType {
+    FILL_DEFAULT_VALUE,
+    FILL_PREVIOUS_VALUE,
+  }
+
+  /**
+   * Helper method to extract the final aggregation results for the given row (in-place).
+   */
+  private void extractFinalAggregationResults(Object[] row) {
+    for (int i = 0; i < _numAggregationFunctions; i++) {
+      int valueIndex = i + _numGroupByExpressions;
+      row[valueIndex] = _aggregationFunctions[i].extractFinalResult(row[valueIndex]);
+    }
+  }
+
+  /**
+   * Constructs the DataSchema for the rows before the post-aggregation (SQL mode).
+   */
+  private DataSchema getPrePostAggregationDataSchema(DataSchema dataSchema) {
+    String[] columnNames = dataSchema.getColumnNames();
+    ColumnDataType[] columnDataTypes = new ColumnDataType[_numColumns];
+    System.arraycopy(dataSchema.getColumnDataTypes(), 0, columnDataTypes, 0, _numGroupByExpressions);
+    for (int i = 0; i < _numAggregationFunctions; i++) {
+      columnDataTypes[i + _numGroupByExpressions] = _aggregationFunctions[i].getFinalResultColumnType();
+    }
+    return new DataSchema(columnNames, columnDataTypes);
+  }
+
+  private IndexedTable getIndexedTable(DataSchema dataSchema, Collection<DataTable> dataTablesToReduce,
+      DataTableReducerContext reducerContext)
+      throws TimeoutException {
+    long start = System.currentTimeMillis();
+    int numDataTables = dataTablesToReduce.size();
+
+    // Get the number of threads to use for reducing.
+    // In case of single reduce thread, fall back to SimpleIndexedTable to avoid redundant locking/unlocking calls.
+    int numReduceThreadsToUse = getNumReduceThreadsToUse(numDataTables, reducerContext.getMaxReduceThreadsPerQuery());
+    int limit = _queryContext.getLimit();
+    // TODO: Make minTrimSize configurable
+    int trimSize = GroupByUtils.getTableCapacity(limit);
+    // NOTE: For query with HAVING clause, use trimSize as resultSize to ensure the result accuracy.
+    // TODO: Resolve the HAVING clause within the IndexedTable before returning the result
+    int resultSize = _queryContext.getHavingFilter() != null ? trimSize : limit;
+    int trimThreshold = reducerContext.getGroupByTrimThreshold();
+    IndexedTable indexedTable;
+    if (numReduceThreadsToUse <= 1) {
+      indexedTable = new SimpleIndexedTable(dataSchema, _queryContext, resultSize, trimSize, trimThreshold);
+    } else {
+      if (trimThreshold >= GroupByOrderByCombineOperator.MAX_TRIM_THRESHOLD) {
+        // special case of trim threshold where it is set to max value.
+        // there won't be any trimming during upsert in this case.
+        // thus we can avoid the overhead of read-lock and write-lock
+        // in the upsert method.
+        indexedTable = new UnboundedConcurrentIndexedTable(dataSchema, _queryContext, resultSize);
+      } else {
+        indexedTable = new ConcurrentIndexedTable(dataSchema, _queryContext, resultSize, trimSize, trimThreshold);
+      }
+    }
+
+    Future[] futures = new Future[numDataTables];
+    CountDownLatch countDownLatch = new CountDownLatch(numDataTables);

Review comment:
       Done




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751463198



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,725 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,
+    // a result is generated for each aggregation in the query,
+    // and the group by keys are not the same across the aggregations
+    // This PQL style of execution makes it impossible to support order by on group by.
+    //
+    // We could not simply change the group by execution behavior,
+    // as that would not be backward compatible for existing users of group by.
+    // As a result, we have 2 modes of group by execution - pql and sql - which can be controlled via query options
+    //
+    // Long term, we may completely move to sql, and keep only full sql mode alive
+    // Until then, we need to support responseFormat = sql for both the modes of execution.
+    // The 4 variants are as described below:
+
+    if (_groupByModeSql) {
+
+      if (_responseFormatSql) {
+        // 1. groupByMode = sql, responseFormat = sql
+        // This is the primary SQL compliant group by
+
+        try {
+          setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+              brokerMetrics);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // 2. groupByMode = sql, responseFormat = pql
+        // This mode will invoke SQL style group by execution, but present results in PQL way
+        // This mode is useful for users who want to avail of SQL compliant group by behavior,
+        // w/o having to forcefully move to a new result type
+
+        try {
+          setSQLGroupByInAggregationResults(brokerResponseNative, dataSchema, dataTables, reducerContext);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    } else {
+
+      // 3. groupByMode = pql, responseFormat = sql
+      // This mode is for users who want response presented in SQL style, but want PQL style group by behavior
+      // Multiple aggregations in PQL violates the tabular nature of results
+      // As a result, in this mode, only single aggregations are supported
+
+      // 4. groupByMode = pql, responseFormat = pql
+      // This is the primary PQL compliant group by
+
+      setGroupByResults(brokerResponseNative, dataTables);
+
+      if (_responseFormatSql) {
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // We emit the group by size when the result isn't empty. All the sizes among group-by results should be the
+        // same.
+        // Thus, we can just emit the one from the 1st result.
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    }
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    Object [] keyColumns = new Object[_numOfKeyColumns];
+    for (int i = 0; i < _numOfKeyColumns; i++) {
+      keyColumns[i] = row[i + 1];
+    }
+    return new Key(keyColumns);
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      int numResultColumns = resultColumnDataTypes.length;
+      int numResultRows = rows.size();
+      List<Object[]> resultRows = new ArrayList<>(numResultRows);
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < numResultColumns; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {
+      Set<Key> keys = new HashSet<>(_primaryKeys);
+      while (index < resultRows.size()) {
+        long timeCol = _dateTimeFormatter.fromFormatToMillis((String) resultRows.get(index)[0]);
+        if (timeCol < time) {
+          index++;
+        } else if (timeCol == time) {
+          gapfillResultRows.add(resultRows.get(index));
+          if (gapfillResultRows.size() == limit) {
+            return gapfillResultRows;
+          }
+          Key key = constructKey(resultRows.get(index));
+          keys.remove(key);
+          for (int colIndex = 2; colIndex < numResultColumns; colIndex++) {
+            _previous.put(key, resultRows.get(index));
+          }
+          index++;
+        } else {
+          break;
+        }
+      }
+      for (Key key : keys) {
+        Object[] gapfillRow = new Object[numResultColumns];
+        gapfillRow[0] = _dateTimeFormatter.fromMillisToFormat(time);
+        for (int i = 0; i < _numOfKeyColumns; i++) {
+          gapfillRow[i + 1] = key.getValues()[i];
+        }
+
+        for (int i = _numOfKeyColumns + 1; i < numResultColumns; i++) {
+          gapfillRow[i] = getFillValue(i, key, resultColumnDataTypes[i]);
+        }
+        gapfillResultRows.add(gapfillRow);
+        if (gapfillResultRows.size() == limit) {
+          return gapfillResultRows;
+        }
+      }
+    }
+    return gapfillResultRows;
+  }
+
+  Object getFillValue(int columIndex, Object key, ColumnDataType dataType) {
+    ExpressionContext expressionContext = _queryContext.getSelectExpressions().get(columIndex);
+    if (expressionContext.getFunction() != null
+        && expressionContext.getFunction().getFunctionName().equalsIgnoreCase("fill")) {
+      List<ExpressionContext> args = expressionContext.getFunction().getArguments();
+      if (args.get(1).getLiteral() == null) {
+        throw new UnsupportedOperationException("Wrong Sql.");
+      }
+      FillType fillType = FillType.valueOf(args.get(1).getLiteral());
+      if (fillType == FillType.FILL_DEFAULT_VALUE) {
+        // TODO: may fill the default value from sql in the future.
+        return SelectionOperatorUtils.getDefaultValue(dataType);
+      } else if (fillType == FillType.FILL_PREVIOUS_VALUE) {
+        if (_previous.containsKey(key)) {
+          return _previous.get(key)[columIndex];

Review comment:
       Great comment! Done




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r753585175



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/GapfillUtil.java
##########
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.util;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FunctionContext;
+
+
+/**
+ * Util class to encapsulate all utilites required for gapfill.
+ */
+public class GapfillUtil {
+  private static final String AGGREGATE_GAP_FILL = "aggregategapfill";
+  private static final String FILL = "fill";
+
+  private GapfillUtil() {
+  }
+
+  public static ExpressionContext stripGapfill(ExpressionContext expression) {
+    if (expression.getType() != ExpressionContext.Type.FUNCTION) {
+      return expression;
+    }
+    FunctionContext function = expression.getFunction();
+    String functionName = StringUtils.remove(function.getFunctionName(), '_').toLowerCase();
+    if (functionName.equalsIgnoreCase(AGGREGATE_GAP_FILL) || functionName.equalsIgnoreCase(FILL)) {
+      return function.getArguments().get(0);
+    }
+    return expression;
+  }
+
+  public static boolean isAggregateGapfill(String name) {
+    return AGGREGATE_GAP_FILL.equalsIgnoreCase(name);

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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r753585959



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/GapfillUtil.java
##########
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.util;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FunctionContext;
+
+
+/**
+ * Util class to encapsulate all utilites required for gapfill.
+ */
+public class GapfillUtil {
+  private static final String AGGREGATE_GAP_FILL = "aggregategapfill";
+  private static final String FILL = "fill";
+
+  private GapfillUtil() {
+  }
+
+  public static ExpressionContext stripGapfill(ExpressionContext expression) {
+    if (expression.getType() != ExpressionContext.Type.FUNCTION) {
+      return expression;
+    }
+    FunctionContext function = expression.getFunction();
+    String functionName = StringUtils.remove(function.getFunctionName(), '_').toLowerCase();
+    if (functionName.equalsIgnoreCase(AGGREGATE_GAP_FILL) || functionName.equalsIgnoreCase(FILL)) {

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] lakshmanan-v commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
lakshmanan-v commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r754807716



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,490 @@
+/**
+ * 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.reduce;
+
+import com.google.common.base.Preconditions;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.request.context.OrderByExpressionContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GapfillUtils;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _groupByKeys;
+  private final Map<Key, Object[]> _previousByGroupKey;
+  private final int _numOfGroupByKeys;
+  private final List<Integer> _groupByKeyIndexes;
+  private final boolean [] _isGroupBySelections;
+  private int _timeBucketIndex = -1;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    Preconditions.checkArgument(
+        queryContext.getBrokerRequest().getPinotQuery() != null, "GapFill can only be applied to sql query");
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+
+    ExpressionContext gapFillSelection = null;
+    for (ExpressionContext expressionContext : _queryContext.getSelectExpressions()) {
+      if (GapfillUtils.isPostAggregateGapfill(expressionContext)) {
+        gapFillSelection = expressionContext;
+        break;
+      }
+    }
+
+    List<ExpressionContext> args = gapFillSelection.getFunction().getArguments();
+    Preconditions.checkArgument(
+        args.size() == 5, "PostAggregateGapFill does not correct number of arguments.");
+    Preconditions.checkArgument(
+        args.get(1).getLiteral() != null, "The second argument of PostAggregateGapFill should be TimeFormatter.");
+    Preconditions.checkArgument(
+        args.get(2).getLiteral() != null, "The third argument of PostAggregateGapFill should be start time.");
+    Preconditions.checkArgument(
+        args.get(3).getLiteral() != null, "The fourth argument of PostAggregateGapFill should be end time.");
+    Preconditions.checkArgument(
+        args.get(4).getLiteral() != null, "The fifth argument of PostAggregateGapFill should be time bucket size.");
+
+    boolean orderByTimeBucket = false;
+    if (_queryContext.getOrderByExpressions() != null) {
+      for (OrderByExpressionContext expressionContext : _queryContext.getOrderByExpressions()) {
+        if (expressionContext.getExpression().equals(gapFillSelection)) {
+          orderByTimeBucket = true;
+          break;
+        }
+      }
+    }
+
+    Preconditions.checkArgument(
+        orderByTimeBucket, "PostAggregateGapFill does not work if the time bucket is not ordered.");

Review comment:
       Should we enforce , it should only be ordered by time bucket ?




-- 
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] lakshmanan-v commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
lakshmanan-v commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r754808428



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,490 @@
+/**
+ * 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.reduce;
+
+import com.google.common.base.Preconditions;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.request.context.OrderByExpressionContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GapfillUtils;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _groupByKeys;
+  private final Map<Key, Object[]> _previousByGroupKey;
+  private final int _numOfGroupByKeys;
+  private final List<Integer> _groupByKeyIndexes;
+  private final boolean [] _isGroupBySelections;
+  private int _timeBucketIndex = -1;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    Preconditions.checkArgument(
+        queryContext.getBrokerRequest().getPinotQuery() != null, "GapFill can only be applied to sql query");
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+
+    ExpressionContext gapFillSelection = null;
+    for (ExpressionContext expressionContext : _queryContext.getSelectExpressions()) {
+      if (GapfillUtils.isPostAggregateGapfill(expressionContext)) {
+        gapFillSelection = expressionContext;
+        break;
+      }
+    }
+
+    List<ExpressionContext> args = gapFillSelection.getFunction().getArguments();
+    Preconditions.checkArgument(
+        args.size() == 5, "PostAggregateGapFill does not correct number of arguments.");
+    Preconditions.checkArgument(

Review comment:
       Could you add check to make sure the first argument is a time bucket function ?




-- 
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] lakshmanan-v commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
lakshmanan-v commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r754807344



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,490 @@
+/**
+ * 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.reduce;
+
+import com.google.common.base.Preconditions;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.request.context.OrderByExpressionContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GapfillUtils;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _groupByKeys;
+  private final Map<Key, Object[]> _previousByGroupKey;
+  private final int _numOfGroupByKeys;
+  private final List<Integer> _groupByKeyIndexes;
+  private final boolean [] _isGroupBySelections;
+  private int _timeBucketIndex = -1;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    Preconditions.checkArgument(
+        queryContext.getBrokerRequest().getPinotQuery() != null, "GapFill can only be applied to sql query");
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+
+    ExpressionContext gapFillSelection = null;
+    for (ExpressionContext expressionContext : _queryContext.getSelectExpressions()) {
+      if (GapfillUtils.isPostAggregateGapfill(expressionContext)) {
+        gapFillSelection = expressionContext;
+        break;
+      }
+    }
+
+    List<ExpressionContext> args = gapFillSelection.getFunction().getArguments();
+    Preconditions.checkArgument(
+        args.size() == 5, "PostAggregateGapFill does not correct number of arguments.");

Review comment:
       nit: Error mesage looks incomplete. "PostAggregateGapFill does not **have** correct number of arguments." ?




-- 
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 change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

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



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/TransformFunctionFactory.java
##########
@@ -59,6 +59,7 @@
  * Factory class for transformation functions.
  */
 public class TransformFunctionFactory {
+

Review comment:
       (minor) revert the unnecessary change?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/GapfillUtil.java
##########
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.util;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FunctionContext;
+
+
+/**
+ * Util class to encapsulate all utilites required for gapfill.
+ */
+public class GapfillUtil {
+  private static final String AGGREGATE_GAP_FILL = "aggregategapfill";

Review comment:
       If it is meant for post-aggregate gap fill, I'd suggest making it clear `postaggregategapfill`. In the future we can add `preaggregategapfill`.

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/operator/transform/TransformOperator.java
##########
@@ -56,7 +57,10 @@ public TransformOperator(@Nullable QueryContext queryContext, ProjectionOperator
     _projectionOperator = projectionOperator;
     _dataSourceMap = projectionOperator.getDataSourceMap();
     for (ExpressionContext expression : expressions) {
-      TransformFunction transformFunction = TransformFunctionFactory.get(queryContext, expression, _dataSourceMap);
+      TransformFunction transformFunction = TransformFunctionFactory.get(

Review comment:
       (minor, code format) we usually put arguments in the same line

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;

Review comment:
       Let's make a `Preconditions` check here and directly fail the pql query to avoid unexpected results because gap fill does not work on pql
   ```suggestion
     Preconditions.checkArgument(queryContext.getBrokerRequest().getPinotQuery() != null, "GapFill cannot be applied to pql query);
   ```

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);

Review comment:
       IMO we should not force the time column to always be the first argument

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());

Review comment:
       Let's add some arguments check here (both count and type) so that users getting proper exception message when providing wrong arguments

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // 1. groupByMode = sql, responseFormat = sql

Review comment:
       (minor) Remove the comments as they are no longer relevant

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/GapfillUtil.java
##########
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.util;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FunctionContext;
+
+
+/**
+ * Util class to encapsulate all utilites required for gapfill.
+ */
+public class GapfillUtil {

Review comment:
       (minor)
   ```suggestion
   public class GapfillUtils {
   ```

##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/PostAggregationGapfillQueriesTest.java
##########
@@ -0,0 +1,424 @@
+/**
+ * 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.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+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.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+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.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+
+/**
+ * Queries test for LASTWITHTIME queries.
+ */
+@SuppressWarnings("rawtypes")
+public class PostAggregationGapfillQueriesTest extends BaseQueriesTest {

Review comment:
       Can we also test the HAVING clause? It should automatically work if all the expressions are properly stripped

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/request/context/QueryContext.java
##########
@@ -201,6 +202,11 @@ public int getOffset() {
     return _offset;
   }
 
+  public boolean isAggregateGapfill() {
+    return !_selectExpressions.isEmpty()

Review comment:
       Should we check all the select expressions? I don't think we force gapfill to be the first expression
   
   (minor) select expressions is never empty

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/GapfillUtil.java
##########
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.util;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FunctionContext;
+
+
+/**
+ * Util class to encapsulate all utilites required for gapfill.
+ */
+public class GapfillUtil {
+  private static final String AGGREGATE_GAP_FILL = "aggregategapfill";
+  private static final String FILL = "fill";
+
+  private GapfillUtil() {
+  }
+
+  public static ExpressionContext stripGapfill(ExpressionContext expression) {
+    if (expression.getType() != ExpressionContext.Type.FUNCTION) {
+      return expression;
+    }
+    FunctionContext function = expression.getFunction();
+    String functionName = StringUtils.remove(function.getFunctionName(), '_').toLowerCase();
+    if (functionName.equalsIgnoreCase(AGGREGATE_GAP_FILL) || functionName.equalsIgnoreCase(FILL)) {

Review comment:
       Since the name is already canonicalized, you may use `equals()`

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));

Review comment:
       We should also truncate the `_startMs` and `_endMs`

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/GapfillUtil.java
##########
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.util;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FunctionContext;
+
+
+/**
+ * Util class to encapsulate all utilites required for gapfill.
+ */
+public class GapfillUtil {
+  private static final String AGGREGATE_GAP_FILL = "aggregategapfill";
+  private static final String FILL = "fill";
+
+  private GapfillUtil() {
+  }
+
+  public static ExpressionContext stripGapfill(ExpressionContext expression) {
+    if (expression.getType() != ExpressionContext.Type.FUNCTION) {
+      return expression;
+    }
+    FunctionContext function = expression.getFunction();
+    String functionName = StringUtils.remove(function.getFunctionName(), '_').toLowerCase();
+    if (functionName.equalsIgnoreCase(AGGREGATE_GAP_FILL) || functionName.equalsIgnoreCase(FILL)) {
+      return function.getArguments().get(0);
+    }
+    return expression;
+  }
+
+  public static boolean isAggregateGapfill(String name) {
+    return AGGREGATE_GAP_FILL.equalsIgnoreCase(name);

Review comment:
       You should also canonicalize the function name here

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/PostAggregationHandler.java
##########
@@ -67,7 +68,8 @@ public PostAggregationHandler(QueryContext queryContext, DataSchema dataSchema)
     String[] columnNames = new String[numSelectExpressions];
     ColumnDataType[] columnDataTypes = new ColumnDataType[numSelectExpressions];
     for (int i = 0; i < numSelectExpressions; i++) {
-      ValueExtractor valueExtractor = getValueExtractor(selectExpressions.get(i));
+      ValueExtractor valueExtractor
+          = getValueExtractor(GapfillUtil.stripGapfill(selectExpressions.get(i)));

Review comment:
       I think `GapfillUtil.stripGapfill()` should be added to the `getValueExtracter()` method instead of here. `getValueExtracter()` is used in multiple places

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // 1. groupByMode = sql, responseFormat = sql
+    // This is the primary SQL compliant group by
+
+    try {
+      setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+          brokerMetrics);
+    } catch (TimeoutException e) {
+      brokerResponseNative.getProcessingExceptions()
+          .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+    }
+    int resultSize = brokerResponseNative.getResultTable().getRows().size();
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    return new Key(Arrays.copyOfRange(row, 1, _numOfKeyColumns + 1));
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      List<Object[]> resultRows = new ArrayList<>(rows.size());
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < resultColumnDataTypes.length; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {
+      Set<Key> keys = new HashSet<>(_primaryKeys);
+      while (index < resultRows.size()) {
+        long timeCol = _dateTimeFormatter.fromFormatToMillis(String.valueOf(resultRows.get(index)[0]));
+        if (timeCol < time) {
+          index++;
+        } else if (timeCol == time) {
+          gapfillResultRows.add(resultRows.get(index));
+          if (gapfillResultRows.size() == limit) {
+            return gapfillResultRows;
+          }
+          Key key = constructKey(resultRows.get(index));
+          keys.remove(key);
+          _previous.put(key, resultRows.get(index));
+          index++;
+        } else {
+          break;
+        }
+      }
+      for (Key key : keys) {
+        Object[] gapfillRow = new Object[numResultColumns];
+        if (resultColumnDataTypes[0] == ColumnDataType.LONG) {
+          gapfillRow[0] = Long.valueOf(_dateTimeFormatter.fromMillisToFormat(time));
+        } else {
+          gapfillRow[0] = _dateTimeFormatter.fromMillisToFormat(time);
+        }
+        System.arraycopy(key.getValues(), 0, gapfillRow, 1, _numOfKeyColumns);
+
+        for (int i = _numOfKeyColumns + 1; i < numResultColumns; i++) {
+          gapfillRow[i] = getFillValue(i, key, resultColumnDataTypes[i]);
+        }
+        gapfillResultRows.add(gapfillRow);
+        if (gapfillResultRows.size() == limit) {
+          return gapfillResultRows;
+        }
+      }
+    }
+    return gapfillResultRows;
+  }
+
+  Object getFillValue(int columIndex, Object key, ColumnDataType dataType) {
+    ExpressionContext expressionContext = _queryContext.getSelectExpressions().get(columIndex);
+    if (expressionContext.getFunction() != null
+        && expressionContext.getFunction().getFunctionName().equalsIgnoreCase("fill")) {
+      List<ExpressionContext> args = expressionContext.getFunction().getArguments();
+      if (args.get(1).getLiteral() == null) {
+        throw new UnsupportedOperationException("Wrong Sql.");
+      }
+      FillType fillType = FillType.valueOf(args.get(1).getLiteral());
+      if (fillType == FillType.FILL_DEFAULT_VALUE) {
+        // TODO: may fill the default value from sql in the future.
+        return getDefaultValue(dataType);
+      } else if (fillType == FillType.FILL_PREVIOUS_VALUE) {
+        Object[] row = _previous.get(key);
+        if (row != null) {
+          return row[columIndex];
+        } else {
+          return getDefaultValue(dataType);
+        }
+      } else {
+        throw new UnsupportedOperationException("unsupported fill type.");
+      }
+    } else {
+      return getDefaultValue(dataType);
+    }
+  }
+
+  enum FillType {
+    FILL_DEFAULT_VALUE,
+    FILL_PREVIOUS_VALUE,
+  }
+
+  /**
+   * The default value for each column type.
+   */
+  private Serializable getDefaultValue(ColumnDataType dataType) {
+    switch (dataType) {
+      // Single-value column
+      case INT:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case BOOLEAN:
+      case TIMESTAMP:
+        return dataType.convertAndFormat(0);
+      case STRING:
+      case JSON:
+      case BYTES:
+        return "";
+      case INT_ARRAY:
+        return new int[0];
+      case LONG_ARRAY:
+        return new long[0];
+      case FLOAT_ARRAY:
+        return new float[0];
+      case DOUBLE_ARRAY:
+        return new double[0];
+      case STRING_ARRAY:
+      case TIMESTAMP_ARRAY:
+        return new String[0];
+      case BOOLEAN_ARRAY:
+        return new boolean[0];
+      case BYTES_ARRAY:
+        return new byte[0][0];
+      default:
+        throw new IllegalStateException(String.format("Cannot provide the default value for the type: %s", dataType));
+    }
+  }

Review comment:
       Suggest moving these 2 methods into the utils class since they can be shared for pre-aggregate gapfill

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // 1. groupByMode = sql, responseFormat = sql
+    // This is the primary SQL compliant group by
+
+    try {
+      setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+          brokerMetrics);
+    } catch (TimeoutException e) {
+      brokerResponseNative.getProcessingExceptions()
+          .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+    }
+    int resultSize = brokerResponseNative.getResultTable().getRows().size();
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    return new Key(Arrays.copyOfRange(row, 1, _numOfKeyColumns + 1));
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      List<Object[]> resultRows = new ArrayList<>(rows.size());
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < resultColumnDataTypes.length; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {

Review comment:
       We are looping over the rows multiple times (once per time bucket). We can optimize it to only loop over the rows once. One option is to track the index of the previous iteration and start from the end index of the previous run

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // 1. groupByMode = sql, responseFormat = sql
+    // This is the primary SQL compliant group by
+
+    try {
+      setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+          brokerMetrics);
+    } catch (TimeoutException e) {
+      brokerResponseNative.getProcessingExceptions()
+          .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+    }
+    int resultSize = brokerResponseNative.getResultTable().getRows().size();
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    return new Key(Arrays.copyOfRange(row, 1, _numOfKeyColumns + 1));
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      List<Object[]> resultRows = new ArrayList<>(rows.size());
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < resultColumnDataTypes.length; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {
+      Set<Key> keys = new HashSet<>(_primaryKeys);
+      while (index < resultRows.size()) {
+        long timeCol = _dateTimeFormatter.fromFormatToMillis(String.valueOf(resultRows.get(index)[0]));

Review comment:
       Here we assume the query is ordering by time in ascending order. We should add a check somewhere to reject queries not ordering by time. 




-- 
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] richardstartin commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751141305



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,725 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,
+    // a result is generated for each aggregation in the query,
+    // and the group by keys are not the same across the aggregations
+    // This PQL style of execution makes it impossible to support order by on group by.
+    //
+    // We could not simply change the group by execution behavior,
+    // as that would not be backward compatible for existing users of group by.
+    // As a result, we have 2 modes of group by execution - pql and sql - which can be controlled via query options
+    //
+    // Long term, we may completely move to sql, and keep only full sql mode alive
+    // Until then, we need to support responseFormat = sql for both the modes of execution.
+    // The 4 variants are as described below:
+
+    if (_groupByModeSql) {
+
+      if (_responseFormatSql) {
+        // 1. groupByMode = sql, responseFormat = sql
+        // This is the primary SQL compliant group by
+
+        try {
+          setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+              brokerMetrics);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // 2. groupByMode = sql, responseFormat = pql
+        // This mode will invoke SQL style group by execution, but present results in PQL way
+        // This mode is useful for users who want to avail of SQL compliant group by behavior,
+        // w/o having to forcefully move to a new result type
+
+        try {
+          setSQLGroupByInAggregationResults(brokerResponseNative, dataSchema, dataTables, reducerContext);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    } else {
+
+      // 3. groupByMode = pql, responseFormat = sql
+      // This mode is for users who want response presented in SQL style, but want PQL style group by behavior
+      // Multiple aggregations in PQL violates the tabular nature of results
+      // As a result, in this mode, only single aggregations are supported
+
+      // 4. groupByMode = pql, responseFormat = pql
+      // This is the primary PQL compliant group by
+
+      setGroupByResults(brokerResponseNative, dataTables);
+
+      if (_responseFormatSql) {
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // We emit the group by size when the result isn't empty. All the sizes among group-by results should be the
+        // same.
+        // Thus, we can just emit the one from the 1st result.
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    }
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    Object [] keyColumns = new Object[_numOfKeyColumns];
+    for (int i = 0; i < _numOfKeyColumns; i++) {
+      keyColumns[i] = row[i + 1];
+    }
+    return new Key(keyColumns);
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      int numResultColumns = resultColumnDataTypes.length;
+      int numResultRows = rows.size();
+      List<Object[]> resultRows = new ArrayList<>(numResultRows);
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < numResultColumns; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {
+      Set<Key> keys = new HashSet<>(_primaryKeys);
+      while (index < resultRows.size()) {
+        long timeCol = _dateTimeFormatter.fromFormatToMillis((String) resultRows.get(index)[0]);
+        if (timeCol < time) {
+          index++;
+        } else if (timeCol == time) {
+          gapfillResultRows.add(resultRows.get(index));
+          if (gapfillResultRows.size() == limit) {
+            return gapfillResultRows;
+          }
+          Key key = constructKey(resultRows.get(index));
+          keys.remove(key);
+          for (int colIndex = 2; colIndex < numResultColumns; colIndex++) {
+            _previous.put(key, resultRows.get(index));
+          }
+          index++;
+        } else {
+          break;
+        }
+      }
+      for (Key key : keys) {
+        Object[] gapfillRow = new Object[numResultColumns];
+        gapfillRow[0] = _dateTimeFormatter.fromMillisToFormat(time);
+        for (int i = 0; i < _numOfKeyColumns; i++) {
+          gapfillRow[i + 1] = key.getValues()[i];
+        }

Review comment:
       ```java
   Object[] gapfillRow = new Object[numResultColumns];
   gapfillRow[0] = _dateTimeFormatter.fromMillisToFormat(time);
   System.arraycopy(key.getValues(), 0, gapfillRow, 1, _numOfKeyColumns);
   ```




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751460482



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,725 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,
+    // a result is generated for each aggregation in the query,
+    // and the group by keys are not the same across the aggregations
+    // This PQL style of execution makes it impossible to support order by on group by.
+    //
+    // We could not simply change the group by execution behavior,
+    // as that would not be backward compatible for existing users of group by.
+    // As a result, we have 2 modes of group by execution - pql and sql - which can be controlled via query options
+    //
+    // Long term, we may completely move to sql, and keep only full sql mode alive
+    // Until then, we need to support responseFormat = sql for both the modes of execution.
+    // The 4 variants are as described below:
+
+    if (_groupByModeSql) {
+
+      if (_responseFormatSql) {
+        // 1. groupByMode = sql, responseFormat = sql
+        // This is the primary SQL compliant group by
+
+        try {
+          setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+              brokerMetrics);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // 2. groupByMode = sql, responseFormat = pql
+        // This mode will invoke SQL style group by execution, but present results in PQL way
+        // This mode is useful for users who want to avail of SQL compliant group by behavior,
+        // w/o having to forcefully move to a new result type
+
+        try {
+          setSQLGroupByInAggregationResults(brokerResponseNative, dataSchema, dataTables, reducerContext);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    } else {
+
+      // 3. groupByMode = pql, responseFormat = sql
+      // This mode is for users who want response presented in SQL style, but want PQL style group by behavior
+      // Multiple aggregations in PQL violates the tabular nature of results
+      // As a result, in this mode, only single aggregations are supported
+
+      // 4. groupByMode = pql, responseFormat = pql
+      // This is the primary PQL compliant group by
+
+      setGroupByResults(brokerResponseNative, dataTables);
+
+      if (_responseFormatSql) {
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // We emit the group by size when the result isn't empty. All the sizes among group-by results should be the
+        // same.
+        // Thus, we can just emit the one from the 1st result.
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    }
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    Object [] keyColumns = new Object[_numOfKeyColumns];
+    for (int i = 0; i < _numOfKeyColumns; i++) {
+      keyColumns[i] = row[i + 1];
+    }
+    return new Key(keyColumns);
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      int numResultColumns = resultColumnDataTypes.length;
+      int numResultRows = rows.size();
+      List<Object[]> resultRows = new ArrayList<>(numResultRows);
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < numResultColumns; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {
+      Set<Key> keys = new HashSet<>(_primaryKeys);
+      while (index < resultRows.size()) {
+        long timeCol = _dateTimeFormatter.fromFormatToMillis((String) resultRows.get(index)[0]);
+        if (timeCol < time) {
+          index++;
+        } else if (timeCol == time) {
+          gapfillResultRows.add(resultRows.get(index));
+          if (gapfillResultRows.size() == limit) {
+            return gapfillResultRows;
+          }
+          Key key = constructKey(resultRows.get(index));
+          keys.remove(key);
+          for (int colIndex = 2; colIndex < numResultColumns; colIndex++) {
+            _previous.put(key, resultRows.get(index));
+          }

Review comment:
       This is the bug. We just need put resultRows.get(colIndex) inside the _previous once.




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r753604745



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // 1. groupByMode = sql, responseFormat = sql
+    // This is the primary SQL compliant group by
+
+    try {
+      setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+          brokerMetrics);
+    } catch (TimeoutException e) {
+      brokerResponseNative.getProcessingExceptions()
+          .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+    }
+    int resultSize = brokerResponseNative.getResultTable().getRows().size();
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    return new Key(Arrays.copyOfRange(row, 1, _numOfKeyColumns + 1));
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      List<Object[]> resultRows = new ArrayList<>(rows.size());
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < resultColumnDataTypes.length; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {
+      Set<Key> keys = new HashSet<>(_primaryKeys);
+      while (index < resultRows.size()) {
+        long timeCol = _dateTimeFormatter.fromFormatToMillis(String.valueOf(resultRows.get(index)[0]));

Review comment:
       Fixed. Also Test case is added to verify it.




-- 
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 change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

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



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/GapfillUtil.java
##########
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.util;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FunctionContext;
+
+
+/**
+ * Util class to encapsulate all utilites required for gapfill.
+ */
+public class GapfillUtil {
+  private static final String AGGREGATE_GAP_FILL = "aggregategapfill";
+  private static final String FILL = "fill";
+
+  private GapfillUtil() {
+  }
+
+  public static ExpressionContext stripGapfill(ExpressionContext expression) {
+    if (expression.getType() != ExpressionContext.Type.FUNCTION) {
+      return expression;
+    }
+    FunctionContext function = expression.getFunction();
+    String functionName = StringUtils.remove(function.getFunctionName(), '_').toLowerCase();
+    if (functionName.equalsIgnoreCase(AGGREGATE_GAP_FILL) || functionName.equalsIgnoreCase(FILL)) {

Review comment:
       (minor) Since the name is already canonicalized, you may use `equals()`




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r753609230



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/PostAggregationGapfillQueriesTest.java
##########
@@ -0,0 +1,424 @@
+/**
+ * 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.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+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.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+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.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+
+/**
+ * Queries test for LASTWITHTIME queries.
+ */
+@SuppressWarnings("rawtypes")
+public class PostAggregationGapfillQueriesTest extends BaseQueriesTest {

Review comment:
       Done




-- 
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 edited a comment on pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#issuecomment-971204560


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?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 [#7781](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (09b84f6) into [master](https://codecov.io/gh/apache/pinot/commit/09bb91cecc8cff459a9d7814a7fe61aa67f2a393?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (09bb91c) will **decrease** coverage by `57.10%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/7781/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #7781       +/-   ##
   =============================================
   - Coverage     71.62%   14.51%   -57.11%     
   + Complexity     4079       80     -3999     
   =============================================
     Files          1578     1537       -41     
     Lines         80641    79501     -1140     
     Branches      11987    11926       -61     
   =============================================
   - Hits          57758    11542    -46216     
   - Misses        18994    67105    +48111     
   + Partials       3889      854     -3035     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `?` | |
   | unittests2 | `14.51% <0.00%> (-0.09%)` | :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/7781?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...not/core/operator/transform/TransformOperator.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vVHJhbnNmb3JtT3BlcmF0b3IuamF2YQ==) | `0.00% <ø> (-95.00%)` | :arrow_down: |
   | [...e/query/reduce/GapFillGroupByDataTableReducer.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvR2FwRmlsbEdyb3VwQnlEYXRhVGFibGVSZWR1Y2VyLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...inot/core/query/reduce/PostAggregationHandler.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUG9zdEFnZ3JlZ2F0aW9uSGFuZGxlci5qYXZh) | `0.00% <0.00%> (-92.21%)` | :arrow_down: |
   | [.../pinot/core/query/reduce/ResultReducerFactory.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvUmVzdWx0UmVkdWNlckZhY3RvcnkuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/util/GapfillUtils.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS91dGlsL0dhcGZpbGxVdGlscy5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...ain/java/org/apache/pinot/core/data/table/Key.java](https://codecov.io/gh/apache/pinot/pull/7781/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: |
   | [.../java/org/apache/pinot/spi/utils/BooleanUtils.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQm9vbGVhblV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/data/table/Record.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL1JlY29yZC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/util/GroupByUtils.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS91dGlsL0dyb3VwQnlVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/spi/config/table/FSTType.java](https://codecov.io/gh/apache/pinot/pull/7781/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-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvY29uZmlnL3RhYmxlL0ZTVFR5cGUuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [1269 more](https://codecov.io/gh/apache/pinot/pull/7781/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [09bb91c...09b84f6](https://codecov.io/gh/apache/pinot/pull/7781?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] lakshmanan-v commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
lakshmanan-v commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r754808589



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/PostAggregationGapfillQueriesTest.java
##########
@@ -0,0 +1,550 @@
+/**
+ * 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.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+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.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+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.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+
+/**
+ * Queries test for LASTWITHTIME queries.

Review comment:
       nit: Fix comment.




-- 
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] richardstartin commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r751155991



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,725 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,
+    // a result is generated for each aggregation in the query,
+    // and the group by keys are not the same across the aggregations
+    // This PQL style of execution makes it impossible to support order by on group by.
+    //
+    // We could not simply change the group by execution behavior,
+    // as that would not be backward compatible for existing users of group by.
+    // As a result, we have 2 modes of group by execution - pql and sql - which can be controlled via query options
+    //
+    // Long term, we may completely move to sql, and keep only full sql mode alive
+    // Until then, we need to support responseFormat = sql for both the modes of execution.
+    // The 4 variants are as described below:
+
+    if (_groupByModeSql) {
+
+      if (_responseFormatSql) {
+        // 1. groupByMode = sql, responseFormat = sql
+        // This is the primary SQL compliant group by
+
+        try {
+          setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+              brokerMetrics);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // 2. groupByMode = sql, responseFormat = pql
+        // This mode will invoke SQL style group by execution, but present results in PQL way
+        // This mode is useful for users who want to avail of SQL compliant group by behavior,
+        // w/o having to forcefully move to a new result type
+
+        try {
+          setSQLGroupByInAggregationResults(brokerResponseNative, dataSchema, dataTables, reducerContext);
+        } catch (TimeoutException e) {
+          brokerResponseNative.getProcessingExceptions()
+              .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+        }
+
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    } else {
+
+      // 3. groupByMode = pql, responseFormat = sql
+      // This mode is for users who want response presented in SQL style, but want PQL style group by behavior
+      // Multiple aggregations in PQL violates the tabular nature of results
+      // As a result, in this mode, only single aggregations are supported
+
+      // 4. groupByMode = pql, responseFormat = pql
+      // This is the primary PQL compliant group by
+
+      setGroupByResults(brokerResponseNative, dataTables);
+
+      if (_responseFormatSql) {
+        resultSize = brokerResponseNative.getResultTable().getRows().size();
+      } else {
+        // We emit the group by size when the result isn't empty. All the sizes among group-by results should be the
+        // same.
+        // Thus, we can just emit the one from the 1st result.
+        if (!brokerResponseNative.getAggregationResults().isEmpty()) {
+          resultSize = brokerResponseNative.getAggregationResults().get(0).getGroupByResult().size();
+        }
+      }
+    }
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    Object [] keyColumns = new Object[_numOfKeyColumns];
+    for (int i = 0; i < _numOfKeyColumns; i++) {
+      keyColumns[i] = row[i + 1];
+    }
+    return new Key(keyColumns);
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      int numResultColumns = resultColumnDataTypes.length;
+      int numResultRows = rows.size();
+      List<Object[]> resultRows = new ArrayList<>(numResultRows);
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < numResultColumns; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {
+      Set<Key> keys = new HashSet<>(_primaryKeys);
+      while (index < resultRows.size()) {
+        long timeCol = _dateTimeFormatter.fromFormatToMillis((String) resultRows.get(index)[0]);
+        if (timeCol < time) {
+          index++;
+        } else if (timeCol == time) {
+          gapfillResultRows.add(resultRows.get(index));
+          if (gapfillResultRows.size() == limit) {
+            return gapfillResultRows;
+          }
+          Key key = constructKey(resultRows.get(index));
+          keys.remove(key);
+          for (int colIndex = 2; colIndex < numResultColumns; colIndex++) {
+            _previous.put(key, resultRows.get(index));
+          }
+          index++;
+        } else {
+          break;
+        }
+      }
+      for (Key key : keys) {
+        Object[] gapfillRow = new Object[numResultColumns];
+        gapfillRow[0] = _dateTimeFormatter.fromMillisToFormat(time);
+        for (int i = 0; i < _numOfKeyColumns; i++) {
+          gapfillRow[i + 1] = key.getValues()[i];
+        }
+
+        for (int i = _numOfKeyColumns + 1; i < numResultColumns; i++) {
+          gapfillRow[i] = getFillValue(i, key, resultColumnDataTypes[i]);
+        }
+        gapfillResultRows.add(gapfillRow);
+        if (gapfillResultRows.size() == limit) {
+          return gapfillResultRows;
+        }
+      }
+    }
+    return gapfillResultRows;
+  }
+
+  Object getFillValue(int columIndex, Object key, ColumnDataType dataType) {
+    ExpressionContext expressionContext = _queryContext.getSelectExpressions().get(columIndex);
+    if (expressionContext.getFunction() != null
+        && expressionContext.getFunction().getFunctionName().equalsIgnoreCase("fill")) {
+      List<ExpressionContext> args = expressionContext.getFunction().getArguments();
+      if (args.get(1).getLiteral() == null) {
+        throw new UnsupportedOperationException("Wrong Sql.");
+      }
+      FillType fillType = FillType.valueOf(args.get(1).getLiteral());
+      if (fillType == FillType.FILL_DEFAULT_VALUE) {
+        // TODO: may fill the default value from sql in the future.
+        return SelectionOperatorUtils.getDefaultValue(dataType);
+      } else if (fillType == FillType.FILL_PREVIOUS_VALUE) {
+        if (_previous.containsKey(key)) {
+          return _previous.get(key)[columIndex];
+        } else {
+          return SelectionOperatorUtils.getDefaultValue(dataType);
+        }
+      } else {
+        throw new UnsupportedOperationException("unsupported fill type.");
+      }
+    } else {
+      return SelectionOperatorUtils.getDefaultValue(dataType);
+    }
+  }
+
+  enum FillType {
+    FILL_DEFAULT_VALUE,
+    FILL_PREVIOUS_VALUE,
+  }
+
+  /**
+   * Helper method to extract the final aggregation results for the given row (in-place).
+   */
+  private void extractFinalAggregationResults(Object[] row) {
+    for (int i = 0; i < _numAggregationFunctions; i++) {
+      int valueIndex = i + _numGroupByExpressions;
+      row[valueIndex] = _aggregationFunctions[i].extractFinalResult(row[valueIndex]);
+    }
+  }
+
+  /**
+   * Constructs the DataSchema for the rows before the post-aggregation (SQL mode).
+   */
+  private DataSchema getPrePostAggregationDataSchema(DataSchema dataSchema) {
+    String[] columnNames = dataSchema.getColumnNames();
+    ColumnDataType[] columnDataTypes = new ColumnDataType[_numColumns];
+    System.arraycopy(dataSchema.getColumnDataTypes(), 0, columnDataTypes, 0, _numGroupByExpressions);
+    for (int i = 0; i < _numAggregationFunctions; i++) {
+      columnDataTypes[i + _numGroupByExpressions] = _aggregationFunctions[i].getFinalResultColumnType();
+    }
+    return new DataSchema(columnNames, columnDataTypes);
+  }
+
+  private IndexedTable getIndexedTable(DataSchema dataSchema, Collection<DataTable> dataTablesToReduce,
+      DataTableReducerContext reducerContext)
+      throws TimeoutException {
+    long start = System.currentTimeMillis();
+    int numDataTables = dataTablesToReduce.size();
+
+    // Get the number of threads to use for reducing.
+    // In case of single reduce thread, fall back to SimpleIndexedTable to avoid redundant locking/unlocking calls.
+    int numReduceThreadsToUse = getNumReduceThreadsToUse(numDataTables, reducerContext.getMaxReduceThreadsPerQuery());
+    int limit = _queryContext.getLimit();
+    // TODO: Make minTrimSize configurable
+    int trimSize = GroupByUtils.getTableCapacity(limit);
+    // NOTE: For query with HAVING clause, use trimSize as resultSize to ensure the result accuracy.
+    // TODO: Resolve the HAVING clause within the IndexedTable before returning the result
+    int resultSize = _queryContext.getHavingFilter() != null ? trimSize : limit;
+    int trimThreshold = reducerContext.getGroupByTrimThreshold();
+    IndexedTable indexedTable;
+    if (numReduceThreadsToUse <= 1) {
+      indexedTable = new SimpleIndexedTable(dataSchema, _queryContext, resultSize, trimSize, trimThreshold);
+    } else {
+      if (trimThreshold >= GroupByOrderByCombineOperator.MAX_TRIM_THRESHOLD) {
+        // special case of trim threshold where it is set to max value.
+        // there won't be any trimming during upsert in this case.
+        // thus we can avoid the overhead of read-lock and write-lock
+        // in the upsert method.
+        indexedTable = new UnboundedConcurrentIndexedTable(dataSchema, _queryContext, resultSize);
+      } else {
+        indexedTable = new ConcurrentIndexedTable(dataSchema, _queryContext, resultSize, trimSize, trimThreshold);
+      }
+    }
+
+    Future[] futures = new Future[numDataTables];
+    CountDownLatch countDownLatch = new CountDownLatch(numDataTables);
+
+    // Create groups of data tables that each thread can process concurrently.
+    // Given that numReduceThreads is <= numDataTables, each group will have at least one data table.
+    ArrayList<DataTable> dataTables = new ArrayList<>(dataTablesToReduce);
+    List<List<DataTable>> reduceGroups = new ArrayList<>(numReduceThreadsToUse);
+
+    for (int i = 0; i < numReduceThreadsToUse; i++) {
+      reduceGroups.add(new ArrayList<>());
+    }
+    for (int i = 0; i < numDataTables; i++) {
+      reduceGroups.get(i % numReduceThreadsToUse).add(dataTables.get(i));
+    }
+
+    int cnt = 0;
+    ColumnDataType[] storedColumnDataTypes = dataSchema.getStoredColumnDataTypes();
+    for (List<DataTable> reduceGroup : reduceGroups) {
+      futures[cnt++] = reducerContext.getExecutorService().submit(new TraceRunnable() {
+        @Override
+        public void runJob() {
+          for (DataTable dataTable : reduceGroup) {
+            int numRows = dataTable.getNumberOfRows();
+
+            try {
+              for (int rowId = 0; rowId < numRows; rowId++) {
+                Object[] values = new Object[_numColumns];
+                for (int colId = 0; colId < _numColumns; colId++) {
+                  switch (storedColumnDataTypes[colId]) {
+                    case INT:
+                      values[colId] = dataTable.getInt(rowId, colId);
+                      break;
+                    case LONG:
+                      values[colId] = dataTable.getLong(rowId, colId);
+                      break;
+                    case FLOAT:
+                      values[colId] = dataTable.getFloat(rowId, colId);
+                      break;
+                    case DOUBLE:
+                      values[colId] = dataTable.getDouble(rowId, colId);
+                      break;
+                    case STRING:
+                      values[colId] = dataTable.getString(rowId, colId);
+                      break;
+                    case BYTES:
+                      values[colId] = dataTable.getBytes(rowId, colId);
+                      break;
+                    case OBJECT:
+                      values[colId] = dataTable.getObject(rowId, colId);
+                      break;
+                    // Add other aggregation intermediate result / group-by column type supports here
+                    default:
+                      throw new IllegalStateException();
+                  }
+                }
+                indexedTable.upsert(new Record(values));
+              }
+            } finally {
+              countDownLatch.countDown();

Review comment:
       You don't need this if you use `ExecutorService.invokeAll()` (the futures are all _done_ when that method returns).




-- 
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 change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

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



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/TransformFunctionFactory.java
##########
@@ -198,6 +202,18 @@ public static TransformFunction get(ExpressionContext expression, Map<String, Da
     return get(null, expression, dataSourceMap);
   }
 
+  public static ExpressionContext stripGapfill(ExpressionContext expression) {

Review comment:
       I feel this util method does not belong to this class. Suggest adding a util class for gapfill, and put all the shared constants and methods there

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/TransformFunctionFactory.java
##########
@@ -59,6 +59,10 @@
  * Factory class for transformation functions.
  */
 public class TransformFunctionFactory {
+
+  private static final String AGGREGATE_GAP_FILL = "aggregategapfill";

Review comment:
       Any specific reason naming it `aggregategapfill` instead of `gapfill`?

##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java
##########
@@ -223,8 +223,11 @@ private BrokerResponseNative getBrokerResponse(QueryContext queryContext, PlanMa
       byte[] serializedResponse = instanceResponse.toBytes();
       dataTableMap.put(new ServerRoutingInstance("localhost", 1234, TableType.OFFLINE),
           DataTableFactory.getDataTable(serializedResponse));
-      dataTableMap.put(new ServerRoutingInstance("localhost", 1234, TableType.REALTIME),
-          DataTableFactory.getDataTable(serializedResponse));
+      // skip creating the realtime table for gapfill test case.
+      if (!queryContext.isAggregateGapfill()) {

Review comment:
       This should not be required. This test won't query the realtime table, but just mimic the behavior of hitting 2 servers

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/selection/SelectionOperatorUtils.java
##########
@@ -339,6 +339,43 @@ public static DataTable getDataTableFromRows(Collection<Object[]> rows, DataSche
     return dataTableBuilder.build();
   }
 
+  /**
+   * The default value for each column type.
+   */
+  public static Serializable getDefaultValue(ColumnDataType dataType) {

Review comment:
       These formatters are already deprecated (used for the deprecated pql query response). I think you can put this as a private method in `GapFillGroupByDataTableReducer` because the default value might not be the same for different queries. You may refer to `DateSchema.ColumnDataType.convertAndFormat()` for the expected return type for each column data type

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,706 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.AggregationResult;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.GroupByResult;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils;
+import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByTrimmingService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.QueryOptionsUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _preserveType;
+  private final boolean _groupByModeSql;
+  private final boolean _responseFormatSql;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    Map<String, String> queryOptions = queryContext.getQueryOptions();
+    _preserveType = QueryOptionsUtils.isPreserveType(queryOptions);
+    _groupByModeSql = QueryOptionsUtils.isGroupByModeSQL(queryOptions);
+    _responseFormatSql = QueryOptionsUtils.isResponseFormatSQL(queryOptions);
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    int resultSize = 0;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // For group by, PQL behavior is different than the SQL behavior. In the PQL way,

Review comment:
       Let's not worry about PQL behavior and simplify this class. PQL is already deprecated and will be cleaned up soon. Here we can assume `_groupByModeSql` and `_responseFormatSql` are always `true`




-- 
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] weixiangsun commented on a change in pull request #7781: Add Post-Aggregation Gapfilling functionality.

Posted by GitBox <gi...@apache.org>.
weixiangsun commented on a change in pull request #7781:
URL: https://github.com/apache/pinot/pull/7781#discussion_r753594637



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapFillGroupByDataTableReducer.java
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.reduce;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.QueryProcessingException;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.data.table.ConcurrentIndexedTable;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.data.table.SimpleIndexedTable;
+import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
+import org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GroupByUtils;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce data tables and set group by results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapFillGroupByDataTableReducer implements DataTableReducer {
+  private static final int MIN_DATA_TABLES_FOR_CONCURRENT_REDUCE = 2; // TBD, find a better value.
+
+  private final QueryContext _queryContext;
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  private final List<ExpressionContext> _groupByExpressions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final boolean _sqlQuery;
+  private final DateTimeGranularitySpec _dateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final Set<Key> _primaryKeys;
+  private final Map<Key, Object[]> _previous;
+  private final int _numOfKeyColumns;
+
+  GapFillGroupByDataTableReducer(QueryContext queryContext) {
+    _queryContext = queryContext;
+    _aggregationFunctions = queryContext.getAggregationFunctions();
+    assert _aggregationFunctions != null;
+    _numAggregationFunctions = _aggregationFunctions.length;
+    _groupByExpressions = queryContext.getGroupByExpressions();
+    assert _groupByExpressions != null;
+    _numGroupByExpressions = _groupByExpressions.size();
+    _numColumns = _numAggregationFunctions + _numGroupByExpressions;
+    _sqlQuery = queryContext.getBrokerRequest().getPinotQuery() != null;
+
+    ExpressionContext firstExpressionContext = _queryContext.getSelectExpressions().get(0);
+    List<ExpressionContext> args = firstExpressionContext.getFunction().getArguments();
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _dateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    String start = args.get(2).getLiteral();
+    String end = args.get(3).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _primaryKeys = new HashSet<>();
+    _previous = new HashMap<>();
+    _numOfKeyColumns = _queryContext.getGroupByExpressions().size() - 1;
+  }
+
+  private long truncate(long epoch) {
+    int sz = _dateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  /**
+   * Reduces and sets group by results into ResultTable, if responseFormat = sql
+   * By default, sets group by results into GroupByResults
+   */
+  @Override
+  public void reduceAndSetResults(String tableName, DataSchema dataSchema,
+      Map<ServerRoutingInstance, DataTable> dataTableMap, BrokerResponseNative brokerResponseNative,
+      DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) {
+    assert dataSchema != null;
+    Collection<DataTable> dataTables = dataTableMap.values();
+
+    // 1. groupByMode = sql, responseFormat = sql
+    // This is the primary SQL compliant group by
+
+    try {
+      setSQLGroupByInResultTable(brokerResponseNative, dataSchema, dataTables, reducerContext, tableName,
+          brokerMetrics);
+    } catch (TimeoutException e) {
+      brokerResponseNative.getProcessingExceptions()
+          .add(new QueryProcessingException(QueryException.BROKER_TIMEOUT_ERROR_CODE, e.getMessage()));
+    }
+    int resultSize = brokerResponseNative.getResultTable().getRows().size();
+
+    if (brokerMetrics != null && resultSize > 0) {
+      brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.GROUP_BY_SIZE, resultSize);
+    }
+  }
+
+  private Key constructKey(Object[] row) {
+    return new Key(Arrays.copyOfRange(row, 1, _numOfKeyColumns + 1));
+  }
+
+  /**
+   * Extract group by order by results and set into {@link ResultTable}
+   * @param brokerResponseNative broker response
+   * @param dataSchema data schema
+   * @param dataTables Collection of data tables
+   * @param reducerContext DataTableReducer context
+   * @param rawTableName table name
+   * @param brokerMetrics broker metrics (meters)
+   * @throws TimeoutException If unable complete within timeout.
+   */
+  private void setSQLGroupByInResultTable(BrokerResponseNative brokerResponseNative, DataSchema dataSchema,
+      Collection<DataTable> dataTables, DataTableReducerContext reducerContext, String rawTableName,
+      BrokerMetrics brokerMetrics)
+      throws TimeoutException {
+    IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);
+    if (brokerMetrics != null) {
+      brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.NUM_RESIZES, indexedTable.getNumResizes());
+      brokerMetrics.addValueToTableGauge(rawTableName, BrokerGauge.RESIZE_TIME_MS, indexedTable.getResizeTimeMs());
+    }
+    Iterator<Record> sortedIterator = indexedTable.iterator();
+    DataSchema prePostAggregationDataSchema = getPrePostAggregationDataSchema(dataSchema);
+    ColumnDataType[] columnDataTypes = prePostAggregationDataSchema.getColumnDataTypes();
+    int numColumns = columnDataTypes.length;
+    int limit = _queryContext.getLimit();
+    List<Object[]> rows = new ArrayList<>(limit);
+
+    if (_sqlQuery) {
+      // SQL query with SQL group-by mode and response format
+
+      PostAggregationHandler postAggregationHandler =
+          new PostAggregationHandler(_queryContext, prePostAggregationDataSchema);
+      FilterContext havingFilter = _queryContext.getHavingFilter();
+      if (havingFilter != null) {
+        HavingFilterHandler havingFilterHandler = new HavingFilterHandler(havingFilter, postAggregationHandler);
+        while (rows.size() < limit && sortedIterator.hasNext()) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int i = 0; i < numColumns; i++) {
+            row[i] = columnDataTypes[i].convert(row[i]);
+          }
+          if (havingFilterHandler.isMatch(row)) {
+            rows.add(row);
+          }
+        }
+      } else {
+        for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+          Object[] row = sortedIterator.next().getValues();
+          extractFinalAggregationResults(row);
+          for (int j = 0; j < numColumns; j++) {
+            row[j] = columnDataTypes[j].convert(row[j]);
+          }
+          rows.add(row);
+        }
+      }
+      DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema();
+      ColumnDataType[] resultColumnDataTypes = resultDataSchema.getColumnDataTypes();
+      List<Object[]> resultRows = new ArrayList<>(rows.size());
+      for (Object[] row : rows) {
+        Object[] resultRow = postAggregationHandler.getResult(row);
+        for (int i = 0; i < resultColumnDataTypes.length; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+        resultRows.add(resultRow);
+        _primaryKeys.add(constructKey(resultRow));
+      }
+      List<Object[]> gapfillResultRows = gapFill(resultRows, resultColumnDataTypes);
+      brokerResponseNative.setResultTable(new ResultTable(resultDataSchema, gapfillResultRows));
+    } else {
+      // PQL query with SQL group-by mode and response format
+      // NOTE: For PQL query, keep the order of columns as is (group-by expressions followed by aggregations), no need
+      //       to perform post-aggregation or filtering.
+
+      for (int i = 0; i < limit && sortedIterator.hasNext(); i++) {
+        Object[] row = sortedIterator.next().getValues();
+        extractFinalAggregationResults(row);
+        for (int j = 0; j < numColumns; j++) {
+          row[j] = columnDataTypes[j].convertAndFormat(row[j]);
+        }
+        rows.add(row);
+      }
+      brokerResponseNative.setResultTable(new ResultTable(prePostAggregationDataSchema, rows));
+    }
+  }
+
+  List<Object[]> gapFill(List<Object[]> resultRows, ColumnDataType[] resultColumnDataTypes) {
+    int limit = _queryContext.getLimit();
+    int numResultColumns = resultColumnDataTypes.length;
+    List<Object[]> gapfillResultRows = new ArrayList<>(limit);
+    long step = _dateTimeGranularity.granularityToMillis();
+    int index = 0;
+    for (long time = _startMs; time + 2 * step <= _endMs; time += step) {
+      Set<Key> keys = new HashSet<>(_primaryKeys);
+      while (index < resultRows.size()) {
+        long timeCol = _dateTimeFormatter.fromFormatToMillis(String.valueOf(resultRows.get(index)[0]));
+        if (timeCol < time) {
+          index++;
+        } else if (timeCol == time) {
+          gapfillResultRows.add(resultRows.get(index));
+          if (gapfillResultRows.size() == limit) {
+            return gapfillResultRows;
+          }
+          Key key = constructKey(resultRows.get(index));
+          keys.remove(key);
+          _previous.put(key, resultRows.get(index));
+          index++;
+        } else {
+          break;
+        }
+      }
+      for (Key key : keys) {
+        Object[] gapfillRow = new Object[numResultColumns];
+        if (resultColumnDataTypes[0] == ColumnDataType.LONG) {
+          gapfillRow[0] = Long.valueOf(_dateTimeFormatter.fromMillisToFormat(time));
+        } else {
+          gapfillRow[0] = _dateTimeFormatter.fromMillisToFormat(time);
+        }
+        System.arraycopy(key.getValues(), 0, gapfillRow, 1, _numOfKeyColumns);
+
+        for (int i = _numOfKeyColumns + 1; i < numResultColumns; i++) {
+          gapfillRow[i] = getFillValue(i, key, resultColumnDataTypes[i]);
+        }
+        gapfillResultRows.add(gapfillRow);
+        if (gapfillResultRows.size() == limit) {
+          return gapfillResultRows;
+        }
+      }
+    }
+    return gapfillResultRows;
+  }
+
+  Object getFillValue(int columIndex, Object key, ColumnDataType dataType) {
+    ExpressionContext expressionContext = _queryContext.getSelectExpressions().get(columIndex);
+    if (expressionContext.getFunction() != null
+        && expressionContext.getFunction().getFunctionName().equalsIgnoreCase("fill")) {
+      List<ExpressionContext> args = expressionContext.getFunction().getArguments();
+      if (args.get(1).getLiteral() == null) {
+        throw new UnsupportedOperationException("Wrong Sql.");
+      }
+      FillType fillType = FillType.valueOf(args.get(1).getLiteral());
+      if (fillType == FillType.FILL_DEFAULT_VALUE) {
+        // TODO: may fill the default value from sql in the future.
+        return getDefaultValue(dataType);
+      } else if (fillType == FillType.FILL_PREVIOUS_VALUE) {
+        Object[] row = _previous.get(key);
+        if (row != null) {
+          return row[columIndex];
+        } else {
+          return getDefaultValue(dataType);
+        }
+      } else {
+        throw new UnsupportedOperationException("unsupported fill type.");
+      }
+    } else {
+      return getDefaultValue(dataType);
+    }
+  }
+
+  enum FillType {
+    FILL_DEFAULT_VALUE,
+    FILL_PREVIOUS_VALUE,
+  }
+
+  /**
+   * The default value for each column type.
+   */
+  private Serializable getDefaultValue(ColumnDataType dataType) {
+    switch (dataType) {
+      // Single-value column
+      case INT:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case BOOLEAN:
+      case TIMESTAMP:
+        return dataType.convertAndFormat(0);
+      case STRING:
+      case JSON:
+      case BYTES:
+        return "";
+      case INT_ARRAY:
+        return new int[0];
+      case LONG_ARRAY:
+        return new long[0];
+      case FLOAT_ARRAY:
+        return new float[0];
+      case DOUBLE_ARRAY:
+        return new double[0];
+      case STRING_ARRAY:
+      case TIMESTAMP_ARRAY:
+        return new String[0];
+      case BOOLEAN_ARRAY:
+        return new boolean[0];
+      case BYTES_ARRAY:
+        return new byte[0][0];
+      default:
+        throw new IllegalStateException(String.format("Cannot provide the default value for the type: %s", dataType));
+    }
+  }

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