You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ja...@apache.org on 2022/03/21 21:51:44 UTC

[pinot] branch master updated: Add Time-Series Gapfilling functionality. (#8029)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 078c711  Add Time-Series Gapfilling functionality. (#8029)
078c711 is described below

commit 078c711d35769be2dc4e4b7e235e06744cf0bba7
Author: weixiangsun <91...@users.noreply.github.com>
AuthorDate: Mon Mar 21 14:51:18 2022 -0700

    Add Time-Series Gapfilling functionality. (#8029)
---
 .../requesthandler/BaseBrokerRequestHandler.java   |   40 +-
 .../requesthandler/GrpcBrokerRequestHandler.java   |    6 +-
 .../SingleConnectionBrokerRequestHandler.java      |   10 +-
 .../apache/pinot/common/request/DataSource.java    |  128 +-
 .../pinot/common/utils/request/RequestUtils.java   |    8 +
 .../apache/pinot/sql/parsers/CalciteSqlParser.java |   29 +-
 pinot-common/src/thrift/query.thrift               |    1 +
 .../api/resources/PinotQueryResource.java          |    4 +-
 .../core/query/reduce/BrokerReduceService.java     |   25 +-
 .../core/query/reduce/GapfillFilterHandler.java    |   80 +
 .../pinot/core/query/reduce/GapfillProcessor.java  |  477 +++
 .../core/query/reduce/HavingFilterHandler.java     |  129 +-
 .../core/query/reduce/PostAggregationHandler.java  |   90 +-
 .../core/query/reduce/RowBasedBlockValSet.java     |  192 +
 .../core/query/reduce/filter/AndRowMatcher.java    |   48 +
 .../query/reduce/filter/ColumnValueExtractor.java  |   51 +
 .../query/reduce/filter/LiteralValueExtractor.java |   48 +
 .../core/query/reduce/filter/OrRowMatcher.java     |   48 +
 .../query/reduce/filter/PredicateRowMatcher.java   |   66 +
 .../pinot/core/query/reduce/filter/RowMatcher.java |   29 +
 .../query/reduce/filter/RowMatcherFactory.java     |   47 +
 .../core/query/reduce/filter/ValueExtractor.java   |   42 +
 .../query/reduce/filter/ValueExtractorFactory.java |   35 +
 .../core/query/request/context/QueryContext.java   |   16 +-
 .../BrokerRequestToQueryContextConverter.java      |   17 +-
 .../org/apache/pinot/core/util/GapfillUtils.java   |  296 +-
 .../query/reduce/GapfillFilterHandlerTest.java     |   44 +
 .../org/apache/pinot/queries/BaseQueriesTest.java  |   29 +-
 .../apache/pinot/queries/DistinctQueriesTest.java  |    4 +-
 .../apache/pinot/queries/GapfillQueriesTest.java   | 3699 ++++++++++++++++++++
 30 files changed, 5453 insertions(+), 285 deletions(-)

diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java
index 19e74b2..69982cf 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java
@@ -78,6 +78,7 @@ import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils
 import org.apache.pinot.core.query.optimizer.QueryOptimizer;
 import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
 import org.apache.pinot.core.transport.ServerInstance;
+import org.apache.pinot.core.util.GapfillUtils;
 import org.apache.pinot.core.util.QueryOptionsUtils;
 import org.apache.pinot.pql.parsers.pql2.ast.FilterKind;
 import org.apache.pinot.segment.spi.AggregationFunctionType;
@@ -217,7 +218,10 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
       requestStatistics.setErrorCode(QueryException.PQL_PARSING_ERROR_CODE);
       return new BrokerResponseNative(QueryException.getException(QueryException.PQL_PARSING_ERROR, e));
     }
-    PinotQuery pinotQuery = brokerRequest.getPinotQuery();
+
+    setOptions(brokerRequest.getPinotQuery(), requestId, query, request);
+    BrokerRequest serverBrokerRequest = GapfillUtils.stripGapfill(brokerRequest);
+    PinotQuery pinotQuery = serverBrokerRequest.getPinotQuery();
     setOptions(pinotQuery, requestId, query, request);
 
     if (isLiteralOnlyQuery(pinotQuery)) {
@@ -248,7 +252,7 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
     }
 
     String tableName = getActualTableName(pinotQuery.getDataSource().getTableName());
-    setTableName(brokerRequest, tableName);
+    setTableName(serverBrokerRequest, tableName);
     String rawTableName = TableNameBuilder.extractRawTableName(tableName);
     requestStatistics.setTableName(rawTableName);
 
@@ -286,7 +290,7 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
         compilationEndTimeNs - compilationStartTimeNs);
 
     // Second-stage table-level access control
-    boolean hasTableAccess = _accessControlFactory.create().hasAccess(requesterIdentity, brokerRequest);
+    boolean hasTableAccess = _accessControlFactory.create().hasAccess(requesterIdentity, serverBrokerRequest);
     if (!hasTableAccess) {
       _brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.REQUEST_DROPPED_DUE_TO_ACCESS_ERROR, 1);
       LOGGER.info("Access denied for request {}: {}, table: {}", requestId, query, tableName);
@@ -374,11 +378,11 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
     Schema schema = _tableCache.getSchema(rawTableName);
     if (offlineTableName != null && realtimeTableName != null) {
       // Hybrid
-      offlineBrokerRequest = getOfflineBrokerRequest(brokerRequest);
+      offlineBrokerRequest = getOfflineBrokerRequest(serverBrokerRequest);
       PinotQuery offlinePinotQuery = offlineBrokerRequest.getPinotQuery();
       handleExpressionOverride(offlinePinotQuery, _tableCache.getExpressionOverrideMap(offlineTableName));
       _queryOptimizer.optimize(offlinePinotQuery, offlineTableConfig, schema);
-      realtimeBrokerRequest = getRealtimeBrokerRequest(brokerRequest);
+      realtimeBrokerRequest = getRealtimeBrokerRequest(serverBrokerRequest);
       PinotQuery realtimePinotQuery = realtimeBrokerRequest.getPinotQuery();
       handleExpressionOverride(realtimePinotQuery, _tableCache.getExpressionOverrideMap(realtimeTableName));
       _queryOptimizer.optimize(realtimePinotQuery, realtimeTableConfig, schema);
@@ -387,18 +391,18 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
       requestStatistics.setRealtimeServerTenant(getServerTenant(realtimeTableName));
     } else if (offlineTableName != null) {
       // OFFLINE only
-      setTableName(brokerRequest, offlineTableName);
+      setTableName(serverBrokerRequest, offlineTableName);
       handleExpressionOverride(pinotQuery, _tableCache.getExpressionOverrideMap(offlineTableName));
       _queryOptimizer.optimize(pinotQuery, offlineTableConfig, schema);
-      offlineBrokerRequest = brokerRequest;
+      offlineBrokerRequest = serverBrokerRequest;
       requestStatistics.setFanoutType(RequestStatistics.FanoutType.OFFLINE);
       requestStatistics.setOfflineServerTenant(getServerTenant(offlineTableName));
     } else {
       // REALTIME only
-      setTableName(brokerRequest, realtimeTableName);
+      setTableName(serverBrokerRequest, realtimeTableName);
       handleExpressionOverride(pinotQuery, _tableCache.getExpressionOverrideMap(realtimeTableName));
       _queryOptimizer.optimize(pinotQuery, offlineTableConfig, schema);
-      realtimeBrokerRequest = brokerRequest;
+      realtimeBrokerRequest = serverBrokerRequest;
       requestStatistics.setFanoutType(RequestStatistics.FanoutType.REALTIME);
       requestStatistics.setRealtimeServerTenant(getServerTenant(realtimeTableName));
     }
@@ -536,8 +540,8 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
       }
     }
     BrokerResponseNative brokerResponse =
-        processBrokerRequest(requestId, brokerRequest, offlineBrokerRequest, offlineRoutingTable, realtimeBrokerRequest,
-            realtimeRoutingTable, remainingTimeMs, serverStats, requestStatistics);
+        processBrokerRequest(requestId, brokerRequest, serverBrokerRequest, offlineBrokerRequest, offlineRoutingTable,
+            realtimeBrokerRequest, realtimeRoutingTable, remainingTimeMs, serverStats, requestStatistics);
     brokerResponse.setExceptions(exceptions);
     long executionEndTimeNs = System.nanoTime();
     _brokerMetrics.addPhaseTiming(rawTableName, BrokerQueryPhase.QUERY_EXECUTION,
@@ -859,8 +863,8 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
     // Execute the query
     ServerStats serverStats = new ServerStats();
     BrokerResponseNative brokerResponse =
-        processBrokerRequest(requestId, brokerRequest, offlineBrokerRequest, offlineRoutingTable, realtimeBrokerRequest,
-            realtimeRoutingTable, remainingTimeMs, serverStats, requestStatistics);
+        processBrokerRequest(requestId, brokerRequest, brokerRequest, offlineBrokerRequest, offlineRoutingTable,
+            realtimeBrokerRequest, realtimeRoutingTable, remainingTimeMs, serverStats, requestStatistics);
     long executionEndTimeNs = System.nanoTime();
     _brokerMetrics.addPhaseTiming(rawTableName, BrokerQueryPhase.QUERY_EXECUTION,
         executionEndTimeNs - routingEndTimeNs);
@@ -2080,6 +2084,10 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
         || !QueryOptionsUtils.isResponseFormatSQL(queryOptions)) {
       throw new IllegalStateException("SQL query should always have response format and group-by mode set to SQL");
     }
+
+    if (pinotQuery.getDataSource().getSubquery() != null) {
+      validateRequest(pinotQuery.getDataSource().getSubquery(), queryResponseLimit);
+    }
   }
 
   /**
@@ -2183,9 +2191,9 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
    * Processes the optimized broker requests for both OFFLINE and REALTIME table.
    */
   protected abstract BrokerResponseNative processBrokerRequest(long requestId, BrokerRequest originalBrokerRequest,
-      @Nullable BrokerRequest offlineBrokerRequest, @Nullable Map<ServerInstance, List<String>> offlineRoutingTable,
-      @Nullable BrokerRequest realtimeBrokerRequest, @Nullable Map<ServerInstance, List<String>> realtimeRoutingTable,
-      long timeoutMs, ServerStats serverStats, RequestStatistics requestStatistics)
+      BrokerRequest serverBrokerRequest, @Nullable BrokerRequest offlineBrokerRequest, @Nullable Map<ServerInstance,
+      List<String>> offlineRoutingTable, @Nullable BrokerRequest realtimeBrokerRequest, @Nullable Map<ServerInstance,
+      List<String>> realtimeRoutingTable, long timeoutMs, ServerStats serverStats, RequestStatistics requestStatistics)
       throws Exception;
 
   /**
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/GrpcBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/GrpcBrokerRequestHandler.java
index e2d5bd4..89f41c4 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/GrpcBrokerRequestHandler.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/GrpcBrokerRequestHandler.java
@@ -79,9 +79,9 @@ public class GrpcBrokerRequestHandler extends BaseBrokerRequestHandler {
 
   @Override
   protected BrokerResponseNative processBrokerRequest(long requestId, BrokerRequest originalBrokerRequest,
-      @Nullable BrokerRequest offlineBrokerRequest, @Nullable Map<ServerInstance, List<String>> offlineRoutingTable,
-      @Nullable BrokerRequest realtimeBrokerRequest, @Nullable Map<ServerInstance, List<String>> realtimeRoutingTable,
-      long timeoutMs, ServerStats serverStats, RequestStatistics requestStatistics)
+      BrokerRequest serverBrokerRequest, @Nullable BrokerRequest offlineBrokerRequest, @Nullable Map<ServerInstance,
+      List<String>> offlineRoutingTable, @Nullable BrokerRequest realtimeBrokerRequest, @Nullable Map<ServerInstance,
+      List<String>> realtimeRoutingTable, long timeoutMs, ServerStats serverStats, RequestStatistics requestStatistics)
       throws Exception {
     assert offlineBrokerRequest != null || realtimeBrokerRequest != null;
     Map<ServerRoutingInstance, Iterator<Server.ServerResponse>> responseMap = new HashMap<>();
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java
index e6923b1..3267e80 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java
@@ -81,9 +81,9 @@ public class SingleConnectionBrokerRequestHandler extends BaseBrokerRequestHandl
 
   @Override
   protected BrokerResponseNative processBrokerRequest(long requestId, BrokerRequest originalBrokerRequest,
-      @Nullable BrokerRequest offlineBrokerRequest, @Nullable Map<ServerInstance, List<String>> offlineRoutingTable,
-      @Nullable BrokerRequest realtimeBrokerRequest, @Nullable Map<ServerInstance, List<String>> realtimeRoutingTable,
-      long timeoutMs, ServerStats serverStats, RequestStatistics requestStatistics)
+      BrokerRequest serverBrokerRequest, @Nullable BrokerRequest offlineBrokerRequest, @Nullable Map<ServerInstance,
+      List<String>> offlineRoutingTable, @Nullable BrokerRequest realtimeBrokerRequest, @Nullable Map<ServerInstance,
+      List<String>> realtimeRoutingTable, long timeoutMs, ServerStats serverStats, RequestStatistics requestStatistics)
       throws Exception {
     assert offlineBrokerRequest != null || realtimeBrokerRequest != null;
 
@@ -116,8 +116,8 @@ public class SingleConnectionBrokerRequestHandler extends BaseBrokerRequestHandl
 
     long reduceStartTimeNs = System.nanoTime();
     long reduceTimeOutMs = timeoutMs - TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - scatterGatherStartTimeNs);
-    BrokerResponseNative brokerResponse =
-        _brokerReduceService.reduceOnDataTable(originalBrokerRequest, dataTableMap, reduceTimeOutMs, _brokerMetrics);
+    BrokerResponseNative brokerResponse = _brokerReduceService.reduceOnDataTable(
+        originalBrokerRequest, serverBrokerRequest, dataTableMap, reduceTimeOutMs, _brokerMetrics);
     final long reduceTimeNanos = System.nanoTime() - reduceStartTimeNs;
     requestStatistics.setReduceTimeNanos(reduceTimeNanos);
     _brokerMetrics.addPhaseTiming(rawTableName, BrokerQueryPhase.REDUCE, reduceTimeNanos);
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/request/DataSource.java b/pinot-common/src/main/java/org/apache/pinot/common/request/DataSource.java
index 7afdfb6..ce5e425 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/request/DataSource.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/request/DataSource.java
@@ -25,20 +25,23 @@
 package org.apache.pinot.common.request;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-09-28")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2022-01-18")
 public class DataSource implements org.apache.thrift.TBase<DataSource, DataSource._Fields>, java.io.Serializable, Cloneable, Comparable<DataSource> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DataSource");
 
   private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField SUBQUERY_FIELD_DESC = new org.apache.thrift.protocol.TField("subquery", org.apache.thrift.protocol.TType.STRUCT, (short)2);
 
   private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new DataSourceStandardSchemeFactory();
   private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new DataSourceTupleSchemeFactory();
 
-  private @org.apache.thrift.annotation.Nullable java.lang.String tableName; // optional
+  public @org.apache.thrift.annotation.Nullable java.lang.String tableName; // optional
+  public @org.apache.thrift.annotation.Nullable PinotQuery subquery; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    TABLE_NAME((short)1, "tableName");
+    TABLE_NAME((short)1, "tableName"),
+    SUBQUERY((short)2, "subquery");
 
     private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -56,6 +59,8 @@ public class DataSource implements org.apache.thrift.TBase<DataSource, DataSourc
       switch(fieldId) {
         case 1: // TABLE_NAME
           return TABLE_NAME;
+        case 2: // SUBQUERY
+          return SUBQUERY;
         default:
           return null;
       }
@@ -97,12 +102,14 @@ public class DataSource implements org.apache.thrift.TBase<DataSource, DataSourc
   }
 
   // isset id assignments
-  private static final _Fields optionals[] = {_Fields.TABLE_NAME};
+  private static final _Fields optionals[] = {_Fields.TABLE_NAME,_Fields.SUBQUERY};
   public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
     tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.OPTIONAL,
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.SUBQUERY, new org.apache.thrift.meta_data.FieldMetaData("subquery", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT, "PinotQuery")));
     metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(DataSource.class, metaDataMap);
   }
@@ -117,6 +124,9 @@ public class DataSource implements org.apache.thrift.TBase<DataSource, DataSourc
     if (other.isSetTableName()) {
       this.tableName = other.tableName;
     }
+    if (other.isSetSubquery()) {
+      this.subquery = new PinotQuery(other.subquery);
+    }
   }
 
   public DataSource deepCopy() {
@@ -126,6 +136,7 @@ public class DataSource implements org.apache.thrift.TBase<DataSource, DataSourc
   @Override
   public void clear() {
     this.tableName = null;
+    this.subquery = null;
   }
 
   @org.apache.thrift.annotation.Nullable
@@ -133,8 +144,9 @@ public class DataSource implements org.apache.thrift.TBase<DataSource, DataSourc
     return this.tableName;
   }
 
-  public void setTableName(@org.apache.thrift.annotation.Nullable java.lang.String tableName) {
+  public DataSource setTableName(@org.apache.thrift.annotation.Nullable java.lang.String tableName) {
     this.tableName = tableName;
+    return this;
   }
 
   public void unsetTableName() {
@@ -152,6 +164,31 @@ public class DataSource implements org.apache.thrift.TBase<DataSource, DataSourc
     }
   }
 
+  @org.apache.thrift.annotation.Nullable
+  public PinotQuery getSubquery() {
+    return this.subquery;
+  }
+
+  public DataSource setSubquery(@org.apache.thrift.annotation.Nullable PinotQuery subquery) {
+    this.subquery = subquery;
+    return this;
+  }
+
+  public void unsetSubquery() {
+    this.subquery = null;
+  }
+
+  /** Returns true if field subquery is set (has been assigned a value) and false otherwise */
+  public boolean isSetSubquery() {
+    return this.subquery != null;
+  }
+
+  public void setSubqueryIsSet(boolean value) {
+    if (!value) {
+      this.subquery = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
     switch (field) {
     case TABLE_NAME:
@@ -161,6 +198,15 @@ public class DataSource implements org.apache.thrift.TBase<DataSource, DataSourc
         setTableName((java.lang.String)value);
       }
       break;
+
+    case SUBQUERY:
+      if (value == null) {
+        unsetSubquery();
+      } else {
+        setSubquery((PinotQuery)value);
+      }
+      break;
+
     }
   }
 
@@ -169,6 +215,10 @@ public class DataSource implements org.apache.thrift.TBase<DataSource, DataSourc
     switch (field) {
     case TABLE_NAME:
       return getTableName();
+
+    case SUBQUERY:
+      return getSubquery();
+
     }
     throw new java.lang.IllegalStateException();
   }
@@ -182,6 +232,8 @@ public class DataSource implements org.apache.thrift.TBase<DataSource, DataSourc
     switch (field) {
     case TABLE_NAME:
       return isSetTableName();
+    case SUBQUERY:
+      return isSetSubquery();
     }
     throw new java.lang.IllegalStateException();
   }
@@ -210,6 +262,15 @@ public class DataSource implements org.apache.thrift.TBase<DataSource, DataSourc
         return false;
     }
 
+    boolean this_present_subquery = true && this.isSetSubquery();
+    boolean that_present_subquery = true && that.isSetSubquery();
+    if (this_present_subquery || that_present_subquery) {
+      if (!(this_present_subquery && that_present_subquery))
+        return false;
+      if (!this.subquery.equals(that.subquery))
+        return false;
+    }
+
     return true;
   }
 
@@ -221,6 +282,10 @@ public class DataSource implements org.apache.thrift.TBase<DataSource, DataSourc
     if (isSetTableName())
       hashCode = hashCode * 8191 + tableName.hashCode();
 
+    hashCode = hashCode * 8191 + ((isSetSubquery()) ? 131071 : 524287);
+    if (isSetSubquery())
+      hashCode = hashCode * 8191 + subquery.hashCode();
+
     return hashCode;
   }
 
@@ -242,6 +307,16 @@ public class DataSource implements org.apache.thrift.TBase<DataSource, DataSourc
         return lastComparison;
       }
     }
+    lastComparison = java.lang.Boolean.valueOf(isSetSubquery()).compareTo(other.isSetSubquery());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSubquery()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.subquery, other.subquery);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -272,6 +347,16 @@ public class DataSource implements org.apache.thrift.TBase<DataSource, DataSourc
       }
       first = false;
     }
+    if (isSetSubquery()) {
+      if (!first) sb.append(", ");
+      sb.append("subquery:");
+      if (this.subquery == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.subquery);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -323,12 +408,23 @@ public class DataSource implements org.apache.thrift.TBase<DataSource, DataSourc
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 2: // SUBQUERY
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.subquery = new PinotQuery();
+              struct.subquery.read(iprot);
+              struct.setSubqueryIsSet(true);
+            } else {
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
         iprot.readFieldEnd();
       }
       iprot.readStructEnd();
+
+      // check for required fields of primitive type, which can't be checked in the validate method
       struct.validate();
     }
 
@@ -343,6 +439,13 @@ public class DataSource implements org.apache.thrift.TBase<DataSource, DataSourc
           oprot.writeFieldEnd();
         }
       }
+      if (struct.subquery != null) {
+        if (struct.isSetSubquery()) {
+          oprot.writeFieldBegin(SUBQUERY_FIELD_DESC);
+          struct.subquery.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -363,20 +466,31 @@ public class DataSource implements org.apache.thrift.TBase<DataSource, DataSourc
       if (struct.isSetTableName()) {
         optionals.set(0);
       }
-      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetSubquery()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
       if (struct.isSetTableName()) {
         oprot.writeString(struct.tableName);
       }
+      if (struct.isSetSubquery()) {
+        struct.subquery.write(oprot);
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, DataSource struct) throws org.apache.thrift.TException {
       org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-      java.util.BitSet incoming = iprot.readBitSet(1);
+      java.util.BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         struct.tableName = iprot.readString();
         struct.setTableNameIsSet(true);
       }
+      if (incoming.get(1)) {
+        struct.subquery = new PinotQuery();
+        struct.subquery.read(iprot);
+        struct.setSubqueryIsSet(true);
+      }
     }
   }
 
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/request/RequestUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/request/RequestUtils.java
index 296c7e5..a3221a5 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/utils/request/RequestUtils.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/request/RequestUtils.java
@@ -34,6 +34,7 @@ import org.apache.pinot.common.request.FilterQueryMap;
 import org.apache.pinot.common.request.Function;
 import org.apache.pinot.common.request.Identifier;
 import org.apache.pinot.common.request.Literal;
+import org.apache.pinot.common.request.PinotQuery;
 import org.apache.pinot.pql.parsers.pql2.ast.AstNode;
 import org.apache.pinot.pql.parsers.pql2.ast.FilterKind;
 import org.apache.pinot.pql.parsers.pql2.ast.FloatingPointLiteralAstNode;
@@ -331,4 +332,11 @@ public class RequestUtils {
     }
     return null;
   }
+
+  public static String getTableName(PinotQuery pinotQuery) {
+    while (pinotQuery.getDataSource().getSubquery() != null) {
+      pinotQuery = pinotQuery.getDataSource().getSubquery();
+    }
+    return pinotQuery.getDataSource().getTableName();
+  }
 }
diff --git a/pinot-common/src/main/java/org/apache/pinot/sql/parsers/CalciteSqlParser.java b/pinot-common/src/main/java/org/apache/pinot/sql/parsers/CalciteSqlParser.java
index 49a75ae..10e80ed 100644
--- a/pinot-common/src/main/java/org/apache/pinot/sql/parsers/CalciteSqlParser.java
+++ b/pinot-common/src/main/java/org/apache/pinot/sql/parsers/CalciteSqlParser.java
@@ -70,6 +70,7 @@ public class CalciteSqlParser {
   private CalciteSqlParser() {
   }
 
+  public static final List<QueryRewriter> QUERY_REWRITERS = new ArrayList<>(QueryRewriterFactory.getQueryRewriters());
   private static final Logger LOGGER = LoggerFactory.getLogger(CalciteSqlParser.class);
 
   /** Lexical policy similar to MySQL with ANSI_QUOTES option enabled. (To be
@@ -84,9 +85,6 @@ public class CalciteSqlParser {
   private static final SqlParser.Config PARSER_CONFIG =
       SqlParser.configBuilder().setLex(PINOT_LEX).setConformance(SqlConformanceEnum.BABEL)
           .setParserFactory(SqlBabelParserImpl.FACTORY).build();
-
-  public static final List<QueryRewriter> QUERY_REWRITERS = new ArrayList<>(QueryRewriterFactory.getQueryRewriters());
-
   // To Keep the backward compatibility with 'OPTION' Functionality in PQL, which is used to
   // provide more hints for query processing.
   //
@@ -132,8 +130,17 @@ public class CalciteSqlParser {
     if (!options.isEmpty()) {
       sql = removeOptionsFromSql(sql);
     }
+
+    SqlParser sqlParser = SqlParser.create(sql, PARSER_CONFIG);
+    SqlNode sqlNode;
+    try {
+      sqlNode = sqlParser.parseQuery();
+    } catch (SqlParseException e) {
+      throw new SqlCompilationException("Caught exception while parsing query: " + sql, e);
+    }
+
     // Compile Sql without OPTION statements.
-    PinotQuery pinotQuery = compileCalciteSqlToPinotQuery(sql);
+    PinotQuery pinotQuery = compileSqlNodeToPinotQuery(sqlNode);
 
     // Set Option statements to PinotQuery.
     setOptions(pinotQuery, options);
@@ -330,21 +337,14 @@ public class CalciteSqlParser {
     pinotQuery.setQueryOptions(options);
   }
 
-  private static PinotQuery compileCalciteSqlToPinotQuery(String sql) {
-    SqlParser sqlParser = SqlParser.create(sql, PARSER_CONFIG);
-    SqlNode sqlNode;
-    try {
-      sqlNode = sqlParser.parseQuery();
-    } catch (SqlParseException e) {
-      throw new SqlCompilationException("Caught exception while parsing query: " + sql, e);
-    }
-
+  private static PinotQuery compileSqlNodeToPinotQuery(SqlNode sqlNode) {
     PinotQuery pinotQuery = new PinotQuery();
     if (sqlNode instanceof SqlExplain) {
       // Extract sql node for the query
       sqlNode = ((SqlExplain) sqlNode).getExplicandum();
       pinotQuery.setExplain(true);
     }
+
     SqlSelect selectNode;
     if (sqlNode instanceof SqlOrderBy) {
       // Store order-by info into the select sql node
@@ -374,6 +374,9 @@ public class CalciteSqlParser {
       DataSource dataSource = new DataSource();
       dataSource.setTableName(fromNode.toString());
       pinotQuery.setDataSource(dataSource);
+      if (fromNode instanceof SqlSelect || fromNode instanceof SqlOrderBy) {
+        dataSource.setSubquery(compileSqlNodeToPinotQuery(fromNode));
+      }
     }
     // WHERE
     SqlNode whereNode = selectNode.getWhere();
diff --git a/pinot-common/src/thrift/query.thrift b/pinot-common/src/thrift/query.thrift
index 0f1396a..15227f7 100644
--- a/pinot-common/src/thrift/query.thrift
+++ b/pinot-common/src/thrift/query.thrift
@@ -20,6 +20,7 @@ namespace java org.apache.pinot.common.request
 
 struct DataSource {
   1: optional string tableName;
+  2: optional PinotQuery subquery;
 }
 
 struct PinotQuery {
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java
index 77a72b4..1c7895c 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java
@@ -46,6 +46,7 @@ import org.apache.commons.lang3.tuple.Pair;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.pinot.common.Utils;
 import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.utils.request.RequestUtils;
 import org.apache.pinot.controller.ControllerConf;
 import org.apache.pinot.controller.api.access.AccessControl;
 import org.apache.pinot.controller.api.access.AccessControlFactory;
@@ -162,8 +163,7 @@ public class PinotQueryResource {
       String inputTableName;
       switch (querySyntax) {
         case CommonConstants.Broker.Request.SQL:
-          inputTableName =
-              SQL_QUERY_COMPILER.compileToBrokerRequest(query).getPinotQuery().getDataSource().getTableName();
+          inputTableName = RequestUtils.getTableName(SQL_QUERY_COMPILER.compileToBrokerRequest(query).getPinotQuery());
           break;
         case CommonConstants.Broker.Request.PQL:
           inputTableName = PQL_QUERY_COMPILER.compileToBrokerRequest(query).getQuerySource().getTableName();
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java
index f2796fc..bfd686b 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java
@@ -31,6 +31,7 @@ import org.apache.pinot.common.utils.DataTable;
 import org.apache.pinot.core.query.request.context.QueryContext;
 import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
 import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GapfillUtils;
 import org.apache.pinot.spi.env.PinotConfiguration;
 import org.apache.pinot.spi.utils.CommonConstants;
 import org.apache.pinot.spi.utils.builder.TableNameBuilder;
@@ -46,16 +47,16 @@ public class BrokerReduceService extends BaseReduceService {
     super(config);
   }
 
-  public BrokerResponseNative reduceOnDataTable(BrokerRequest brokerRequest,
+  public BrokerResponseNative reduceOnDataTable(BrokerRequest brokerRequest, BrokerRequest serverBrokerRequest,
       Map<ServerRoutingInstance, DataTable> dataTableMap, long reduceTimeOutMs, @Nullable BrokerMetrics brokerMetrics) {
     if (dataTableMap.isEmpty()) {
       // Empty response.
       return BrokerResponseNative.empty();
     }
 
-    PinotQuery pinotQuery = brokerRequest.getPinotQuery();
+    PinotQuery pinotQuery = serverBrokerRequest.getPinotQuery();
     Map<String, String> queryOptions =
-        pinotQuery != null ? pinotQuery.getQueryOptions() : brokerRequest.getQueryOptions();
+        pinotQuery != null ? pinotQuery.getQueryOptions() : serverBrokerRequest.getQueryOptions();
     boolean enableTrace =
         queryOptions != null && Boolean.parseBoolean(queryOptions.get(CommonConstants.Broker.Request.TRACE));
 
@@ -91,7 +92,7 @@ public class BrokerReduceService extends BaseReduceService {
       }
     }
 
-    String tableName = brokerRequest.getQuerySource().getTableName();
+    String tableName = serverBrokerRequest.getQuerySource().getTableName();
     String rawTableName = TableNameBuilder.extractRawTableName(tableName);
 
     // Set execution statistics and Update broker metrics.
@@ -103,11 +104,23 @@ public class BrokerReduceService extends BaseReduceService {
       return brokerResponseNative;
     }
 
-    QueryContext queryContext = BrokerRequestToQueryContextConverter.convert(brokerRequest);
-    DataTableReducer dataTableReducer = ResultReducerFactory.getResultReducer(queryContext);
+    QueryContext serverQueryContext = BrokerRequestToQueryContextConverter.convert(serverBrokerRequest);
+    DataTableReducer dataTableReducer = ResultReducerFactory.getResultReducer(serverQueryContext);
     dataTableReducer.reduceAndSetResults(rawTableName, cachedDataSchema, dataTableMap, brokerResponseNative,
         new DataTableReducerContext(_reduceExecutorService, _maxReduceThreadsPerQuery, reduceTimeOutMs,
             _groupByTrimThreshold), brokerMetrics);
+    QueryContext queryContext;
+    if (brokerRequest == serverBrokerRequest) {
+      queryContext = serverQueryContext;
+    } else {
+      queryContext = BrokerRequestToQueryContextConverter.convert(brokerRequest);
+      GapfillUtils.GapfillType gapfillType = GapfillUtils.getGapfillType(queryContext);
+      if (gapfillType != null) {
+        GapfillProcessor gapfillProcessor = new GapfillProcessor(queryContext, gapfillType);
+        gapfillProcessor.process(brokerResponseNative);
+      }
+    }
+
     updateAlias(queryContext, brokerResponseNative);
     return brokerResponseNative;
   }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapfillFilterHandler.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapfillFilterHandler.java
new file mode 100644
index 0000000..919f99e
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapfillFilterHandler.java
@@ -0,0 +1,80 @@
+/**
+ * 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.util.HashMap;
+import java.util.Map;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FilterContext;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.query.reduce.filter.ColumnValueExtractor;
+import org.apache.pinot.core.query.reduce.filter.LiteralValueExtractor;
+import org.apache.pinot.core.query.reduce.filter.RowMatcher;
+import org.apache.pinot.core.query.reduce.filter.RowMatcherFactory;
+import org.apache.pinot.core.query.reduce.filter.ValueExtractor;
+import org.apache.pinot.core.query.reduce.filter.ValueExtractorFactory;
+import org.apache.pinot.core.util.GapfillUtils;
+
+
+/**
+ * Handler for Filter clause of GapFill.
+ */
+public class GapfillFilterHandler implements ValueExtractorFactory {
+  private final RowMatcher _rowMatcher;
+  private final DataSchema _dataSchema;
+  private final Map<String, Integer> _indexes;
+
+  public GapfillFilterHandler(FilterContext filter, DataSchema dataSchema) {
+    _dataSchema = dataSchema;
+    _indexes = new HashMap<>();
+    for (int i = 0; i < _dataSchema.size(); i++) {
+      // TODO: This won't work for certain aggregations because the column name in schema is not expression.toString().
+      // TODO: Please refer to {@link PostAggregationHandler} on how to handle the index for aggregation queries.
+      _indexes.put(_dataSchema.getColumnName(i), i);
+    }
+    _rowMatcher = RowMatcherFactory.getRowMatcher(filter, this);
+  }
+
+  /**
+   * Returns {@code true} if the given row matches the HAVING clause, {@code false} otherwise.
+   */
+  public boolean isMatch(Object[] row) {
+    return _rowMatcher.isMatch(row);
+  }
+
+  /**
+   * Returns a ValueExtractor based on the given expression.
+   */
+  @Override
+  public ValueExtractor getValueExtractor(ExpressionContext expression) {
+    expression = GapfillUtils.stripGapfill(expression);
+    if (expression.getType() == ExpressionContext.Type.LITERAL) {
+      // Literal
+      return new LiteralValueExtractor(expression.getLiteral());
+    }
+
+    if (expression.getType() == ExpressionContext.Type.IDENTIFIER) {
+      return new ColumnValueExtractor(_indexes.get(expression.getIdentifier()), _dataSchema);
+    } else {
+      // TODO: This does not handle transform properly (e.g. colA - colB where the gapfill selects colA and colB).
+      // TODO: This is handled within the PostAggregationValueExtractor, and we may also extract that out to be shared.
+      return new ColumnValueExtractor(_indexes.get(expression.getFunction().toString()), _dataSchema);
+    }
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapfillProcessor.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapfillProcessor.java
new file mode 100644
index 0000000..515c6a2
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GapfillProcessor.java
@@ -0,0 +1,477 @@
+/**
+ * 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.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.request.context.FunctionContext;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+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.core.common.BlockValSet;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunctionFactory;
+import org.apache.pinot.core.query.aggregation.function.CountAggregationFunction;
+import org.apache.pinot.core.query.aggregation.groupby.GroupByResultHolder;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.util.GapfillUtils;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
+
+
+/**
+ * Helper class to reduce and set gap fill results into the BrokerResponseNative
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GapfillProcessor {
+  private final QueryContext _queryContext;
+
+  private final int _limitForAggregatedResult;
+  private final DateTimeGranularitySpec _gapfillDateTimeGranularity;
+  private final DateTimeGranularitySpec _postGapfillDateTimeGranularity;
+  private final DateTimeFormatSpec _dateTimeFormatter;
+  private final long _startMs;
+  private final long _endMs;
+  private final long _gapfillTimeBucketSize;
+  private final long _postGapfillTimeBucketSize;
+  private final int _numOfTimeBuckets;
+  private final List<Integer> _groupByKeyIndexes;
+  private final Set<Key> _groupByKeys;
+  private final Map<Key, Object[]> _previousByGroupKey;
+  private final Map<String, ExpressionContext> _fillExpressions;
+  private final List<ExpressionContext> _timeSeries;
+  private final GapfillUtils.GapfillType _gapfillType;
+  private final int _timeBucketColumnIndex;
+  private final int _aggregationSize;
+  private int _limitForGapfilledResult;
+  private boolean[] _isGroupBySelections;
+  private int[] _sourceColumnIndexForResultSchema = null;
+
+  GapfillProcessor(QueryContext queryContext, GapfillUtils.GapfillType gapfillType) {
+    _queryContext = queryContext;
+    _gapfillType = gapfillType;
+    _limitForAggregatedResult = queryContext.getLimit();
+    if (_gapfillType == GapfillUtils.GapfillType.AGGREGATE_GAP_FILL
+        || _gapfillType == GapfillUtils.GapfillType.GAP_FILL) {
+      _limitForGapfilledResult = queryContext.getLimit();
+    } else {
+      _limitForGapfilledResult = queryContext.getSubquery().getLimit();
+    }
+
+    ExpressionContext gapFillSelection = GapfillUtils.getGapfillExpressionContext(queryContext, _gapfillType);
+    _timeBucketColumnIndex = GapfillUtils.findTimeBucketColumnIndex(queryContext, _gapfillType);
+
+    List<ExpressionContext> args = gapFillSelection.getFunction().getArguments();
+
+    _dateTimeFormatter = new DateTimeFormatSpec(args.get(1).getLiteral());
+    _gapfillDateTimeGranularity = new DateTimeGranularitySpec(args.get(4).getLiteral());
+    if (args.get(5).getLiteral() == null) {
+      _postGapfillDateTimeGranularity = _gapfillDateTimeGranularity;
+    } else {
+      _postGapfillDateTimeGranularity = new DateTimeGranularitySpec(args.get(5).getLiteral());
+    }
+    String start = args.get(2).getLiteral();
+    _startMs = truncate(_dateTimeFormatter.fromFormatToMillis(start));
+    String end = args.get(3).getLiteral();
+    _endMs = truncate(_dateTimeFormatter.fromFormatToMillis(end));
+    _gapfillTimeBucketSize = _gapfillDateTimeGranularity.granularityToMillis();
+    _postGapfillTimeBucketSize = _postGapfillDateTimeGranularity.granularityToMillis();
+    _numOfTimeBuckets = (int) ((_endMs - _startMs) / _gapfillTimeBucketSize);
+
+    _fillExpressions = GapfillUtils.getFillExpressions(gapFillSelection);
+
+    _aggregationSize = (int) (_postGapfillTimeBucketSize / _gapfillTimeBucketSize);
+
+    _previousByGroupKey = new HashMap<>();
+    _groupByKeyIndexes = new ArrayList<>();
+    _groupByKeys = new HashSet<>();
+
+    ExpressionContext timeseriesOn = GapfillUtils.getTimeSeriesOnExpressionContext(gapFillSelection);
+    _timeSeries = timeseriesOn.getFunction().getArguments();
+  }
+
+  private int findGapfillBucketIndex(long time) {
+    return (int) ((time - _startMs) / _gapfillTimeBucketSize);
+  }
+
+  private void replaceColumnNameWithAlias(DataSchema dataSchema) {
+    QueryContext queryContext;
+    if (_gapfillType == GapfillUtils.GapfillType.AGGREGATE_GAP_FILL_AGGREGATE) {
+      queryContext = _queryContext.getSubquery().getSubquery();
+    } else if (_gapfillType == GapfillUtils.GapfillType.GAP_FILL) {
+      queryContext = _queryContext;
+    } else {
+      queryContext = _queryContext.getSubquery();
+    }
+    List<String> aliasList = queryContext.getAliasList();
+    Map<String, String> columnNameToAliasMap = new HashMap<>();
+    for (int i = 0; i < aliasList.size(); i++) {
+      if (aliasList.get(i) != null) {
+        ExpressionContext selection = queryContext.getSelectExpressions().get(i);
+        if (GapfillUtils.isGapfill(selection)) {
+          selection = selection.getFunction().getArguments().get(0);
+        }
+        columnNameToAliasMap.put(selection.toString(), aliasList.get(i));
+      }
+    }
+    for (int i = 0; i < dataSchema.getColumnNames().length; i++) {
+      if (columnNameToAliasMap.containsKey(dataSchema.getColumnNames()[i])) {
+        dataSchema.getColumnNames()[i] = columnNameToAliasMap.get(dataSchema.getColumnNames()[i]);
+      }
+    }
+  }
+
+  /**
+   * Here are three things that happen
+   * 1. Sort the result sets from all pinot servers based on timestamp
+   * 2. Gapfill the data for missing entities per time bucket
+   * 3. Aggregate the dataset per time bucket.
+   */
+  public void process(BrokerResponseNative brokerResponseNative) {
+    DataSchema dataSchema = brokerResponseNative.getResultTable().getDataSchema();
+    DataSchema resultTableSchema = getResultTableDataSchema(dataSchema);
+    if (brokerResponseNative.getResultTable().getRows().isEmpty()) {
+      brokerResponseNative.setResultTable(new ResultTable(resultTableSchema, Collections.emptyList()));
+      return;
+    }
+
+    String[] columns = dataSchema.getColumnNames();
+
+    Map<String, Integer> indexes = new HashMap<>();
+    for (int i = 0; i < columns.length; i++) {
+      indexes.put(columns[i], i);
+    }
+
+    _isGroupBySelections = new boolean[dataSchema.getColumnDataTypes().length];
+
+    // The first one argument of timeSeries is time column. The left ones are defining entity.
+    for (ExpressionContext entityColum : _timeSeries) {
+      int index = indexes.get(entityColum.getIdentifier());
+      _isGroupBySelections[index] = true;
+    }
+
+    for (int i = 0; i < _isGroupBySelections.length; i++) {
+      if (_isGroupBySelections[i]) {
+        _groupByKeyIndexes.add(i);
+      }
+    }
+
+    List<Object[]>[] timeBucketedRawRows = putRawRowsIntoTimeBucket(brokerResponseNative.getResultTable().getRows());
+
+    replaceColumnNameWithAlias(dataSchema);
+
+    if (_queryContext.getAggregationFunctions() == null) {
+
+      Map<String, Integer> sourceColumnsIndexes = new HashMap<>();
+      for (int i = 0; i < dataSchema.getColumnNames().length; i++) {
+        sourceColumnsIndexes.put(dataSchema.getColumnName(i), i);
+      }
+      _sourceColumnIndexForResultSchema = new int[resultTableSchema.getColumnNames().length];
+      for (int i = 0; i < _sourceColumnIndexForResultSchema.length; i++) {
+        _sourceColumnIndexForResultSchema[i] = sourceColumnsIndexes.get(resultTableSchema.getColumnName(i));
+      }
+    }
+
+    List<Object[]> resultRows = gapFillAndAggregate(timeBucketedRawRows, resultTableSchema, dataSchema);
+    brokerResponseNative.setResultTable(new ResultTable(resultTableSchema, resultRows));
+  }
+
+  /**
+   * Constructs the DataSchema for the ResultTable.
+   */
+  private DataSchema getResultTableDataSchema(DataSchema dataSchema) {
+    if (_gapfillType == GapfillUtils.GapfillType.GAP_FILL) {
+      return dataSchema;
+    }
+
+    int numOfColumns = _queryContext.getSelectExpressions().size();
+    String[] columnNames = new String[numOfColumns];
+    ColumnDataType[] columnDataTypes = new ColumnDataType[numOfColumns];
+    for (int i = 0; i < numOfColumns; i++) {
+      ExpressionContext expressionContext = _queryContext.getSelectExpressions().get(i);
+      if (GapfillUtils.isGapfill(expressionContext)) {
+        expressionContext = expressionContext.getFunction().getArguments().get(0);
+      }
+      if (expressionContext.getType() != ExpressionContext.Type.FUNCTION) {
+        columnNames[i] = expressionContext.getIdentifier();
+        columnDataTypes[i] = ColumnDataType.STRING;
+      } else {
+        FunctionContext functionContext = expressionContext.getFunction();
+        AggregationFunction aggregationFunction =
+            AggregationFunctionFactory.getAggregationFunction(functionContext, _queryContext);
+        columnDataTypes[i] = aggregationFunction.getFinalResultColumnType();
+        columnNames[i] = functionContext.toString();
+      }
+    }
+    return new DataSchema(columnNames, columnDataTypes);
+  }
+
+  private Key constructGroupKeys(Object[] row) {
+    Object[] groupKeys = new Object[_groupByKeyIndexes.size()];
+    for (int i = 0; i < _groupByKeyIndexes.size(); i++) {
+      groupKeys[i] = row[_groupByKeyIndexes.get(i)];
+    }
+    return new Key(groupKeys);
+  }
+
+  private long truncate(long epoch) {
+    int sz = _gapfillDateTimeGranularity.getSize();
+    return epoch / sz * sz;
+  }
+
+  private List<Object[]> gapFillAndAggregate(List<Object[]>[] timeBucketedRawRows,
+      DataSchema dataSchemaForAggregatedResult, DataSchema dataSchema) {
+    List<Object[]> result = new ArrayList<>();
+
+    GapfillFilterHandler postGapfillFilterHandler = null;
+    if (_queryContext.getSubquery() != null && _queryContext.getFilter() != null) {
+      postGapfillFilterHandler = new GapfillFilterHandler(_queryContext.getFilter(), dataSchema);
+    }
+    GapfillFilterHandler postAggregateHavingFilterHandler = null;
+    if (_queryContext.getHavingFilter() != null) {
+      postAggregateHavingFilterHandler =
+          new GapfillFilterHandler(_queryContext.getHavingFilter(), dataSchemaForAggregatedResult);
+    }
+    long start = _startMs;
+    ColumnDataType[] resultColumnDataTypes = dataSchema.getColumnDataTypes();
+    List<Object[]> bucketedResult = new ArrayList<>();
+    for (long time = _startMs; time < _endMs; time += _gapfillTimeBucketSize) {
+      int index = findGapfillBucketIndex(time);
+      gapfill(time, bucketedResult, timeBucketedRawRows[index], dataSchema, postGapfillFilterHandler);
+      if (_queryContext.getAggregationFunctions() == null) {
+        for (Object[] row : bucketedResult) {
+          Object[] resultRow = new Object[_sourceColumnIndexForResultSchema.length];
+          for (int i = 0; i < _sourceColumnIndexForResultSchema.length; i++) {
+            resultRow[i] = row[_sourceColumnIndexForResultSchema[i]];
+          }
+          result.add(resultRow);
+        }
+        bucketedResult.clear();
+      } else if (index % _aggregationSize == _aggregationSize - 1) {
+        if (bucketedResult.size() > 0) {
+          Object timeCol;
+          if (resultColumnDataTypes[_timeBucketColumnIndex] == ColumnDataType.LONG) {
+            timeCol = Long.valueOf(_dateTimeFormatter.fromMillisToFormat(start));
+          } else {
+            timeCol = _dateTimeFormatter.fromMillisToFormat(start);
+          }
+          List<Object[]> aggregatedRows = aggregateGapfilledData(timeCol, bucketedResult, dataSchema);
+          for (Object[] aggregatedRow : aggregatedRows) {
+            if (postAggregateHavingFilterHandler == null || postAggregateHavingFilterHandler.isMatch(aggregatedRow)) {
+              result.add(aggregatedRow);
+            }
+            if (result.size() >= _limitForAggregatedResult) {
+              return result;
+            }
+          }
+          bucketedResult.clear();
+        }
+        start = time + _gapfillTimeBucketSize;
+      }
+    }
+    return result;
+  }
+
+  private void gapfill(long bucketTime, List<Object[]> bucketedResult, List<Object[]> rawRowsForBucket,
+      DataSchema dataSchema, GapfillFilterHandler postGapfillFilterHandler) {
+    ColumnDataType[] resultColumnDataTypes = dataSchema.getColumnDataTypes();
+    int numResultColumns = resultColumnDataTypes.length;
+    Set<Key> keys = new HashSet<>(_groupByKeys);
+
+    if (rawRowsForBucket != null) {
+      for (Object[] resultRow : rawRowsForBucket) {
+        for (int i = 0; i < resultColumnDataTypes.length; i++) {
+          resultRow[i] = resultColumnDataTypes[i].format(resultRow[i]);
+        }
+
+        long timeCol = _dateTimeFormatter.fromFormatToMillis(String.valueOf(resultRow[0]));
+        if (timeCol > bucketTime) {
+          break;
+        }
+        if (timeCol == bucketTime) {
+          if (postGapfillFilterHandler == null || postGapfillFilterHandler.isMatch(resultRow)) {
+            if (bucketedResult.size() >= _limitForGapfilledResult) {
+              _limitForGapfilledResult = 0;
+              break;
+            } else {
+              bucketedResult.add(resultRow);
+            }
+          }
+          Key key = constructGroupKeys(resultRow);
+          keys.remove(key);
+          _previousByGroupKey.put(key, resultRow);
+        }
+      }
+    }
+
+    for (Key key : keys) {
+      Object[] gapfillRow = new Object[numResultColumns];
+      int keyIndex = 0;
+      if (resultColumnDataTypes[_timeBucketColumnIndex] == ColumnDataType.LONG) {
+        gapfillRow[0] = Long.valueOf(_dateTimeFormatter.fromMillisToFormat(bucketTime));
+      } else {
+        gapfillRow[0] = _dateTimeFormatter.fromMillisToFormat(bucketTime);
+      }
+      for (int i = 1; i < _isGroupBySelections.length; i++) {
+        if (_isGroupBySelections[i]) {
+          gapfillRow[i] = key.getValues()[keyIndex++];
+        } else {
+          gapfillRow[i] = getFillValue(i, dataSchema.getColumnName(i), key, resultColumnDataTypes[i]);
+        }
+      }
+
+      if (postGapfillFilterHandler == null || postGapfillFilterHandler.isMatch(gapfillRow)) {
+        if (bucketedResult.size() >= _limitForGapfilledResult) {
+          break;
+        } else {
+          bucketedResult.add(gapfillRow);
+        }
+      }
+    }
+    if (_limitForGapfilledResult > _groupByKeys.size()) {
+      _limitForGapfilledResult -= _groupByKeys.size();
+    } else {
+      _limitForGapfilledResult = 0;
+    }
+  }
+
+  private List<Object[]> aggregateGapfilledData(Object timeCol, List<Object[]> bucketedRows, DataSchema dataSchema) {
+    List<ExpressionContext> groupbyExpressions = _queryContext.getGroupByExpressions();
+    Preconditions.checkArgument(groupbyExpressions != null, "No GroupBy Clause.");
+    Map<String, Integer> indexes = new HashMap<>();
+    for (int i = 0; i < dataSchema.getColumnNames().length; i++) {
+      indexes.put(dataSchema.getColumnName(i), i);
+    }
+
+    for (Object[] bucketedRow : bucketedRows) {
+      bucketedRow[_timeBucketColumnIndex] = timeCol;
+    }
+
+    Map<List<Object>, Integer> groupKeyIndexes = new HashMap<>();
+    int[] groupKeyArray = new int[bucketedRows.size()];
+    List<Object[]> aggregatedResult = new ArrayList<>();
+    for (int i = 0; i < bucketedRows.size(); i++) {
+      Object[] bucketedRow = bucketedRows.get(i);
+      List<Object> groupKey = new ArrayList<>(groupbyExpressions.size());
+      for (ExpressionContext groupbyExpression : groupbyExpressions) {
+        int columnIndex = indexes.get(groupbyExpression.toString());
+        groupKey.add(bucketedRow[columnIndex]);
+      }
+      if (groupKeyIndexes.containsKey(groupKey)) {
+        groupKeyArray[i] = groupKeyIndexes.get(groupKey);
+      } else {
+        // create the new groupBy Result row and fill the group by key
+        groupKeyArray[i] = groupKeyIndexes.size();
+        groupKeyIndexes.put(groupKey, groupKeyIndexes.size());
+        Object[] row = new Object[_queryContext.getSelectExpressions().size()];
+        for (int j = 0; j < _queryContext.getSelectExpressions().size(); j++) {
+          ExpressionContext expressionContext = _queryContext.getSelectExpressions().get(j);
+          if (expressionContext.getType() != ExpressionContext.Type.FUNCTION) {
+            row[j] = bucketedRow[indexes.get(expressionContext.toString())];
+          }
+        }
+        aggregatedResult.add(row);
+      }
+    }
+
+    Map<ExpressionContext, BlockValSet> blockValSetMap = new HashMap<>();
+    for (int i = 1; i < dataSchema.getColumnNames().length; i++) {
+      blockValSetMap.put(ExpressionContext.forIdentifier(dataSchema.getColumnName(i)),
+          new RowBasedBlockValSet(dataSchema.getColumnDataType(i), bucketedRows, i));
+    }
+
+    for (int i = 0; i < _queryContext.getSelectExpressions().size(); i++) {
+      ExpressionContext expressionContext = _queryContext.getSelectExpressions().get(i);
+      if (expressionContext.getType() == ExpressionContext.Type.FUNCTION) {
+        FunctionContext functionContext = expressionContext.getFunction();
+        AggregationFunction aggregationFunction =
+            AggregationFunctionFactory.getAggregationFunction(functionContext, _queryContext);
+        GroupByResultHolder groupByResultHolder =
+            aggregationFunction.createGroupByResultHolder(groupKeyIndexes.size(), groupKeyIndexes.size());
+        if (aggregationFunction instanceof CountAggregationFunction) {
+          aggregationFunction.aggregateGroupBySV(bucketedRows.size(), groupKeyArray, groupByResultHolder,
+              new HashMap<ExpressionContext, BlockValSet>());
+        } else {
+          aggregationFunction
+              .aggregateGroupBySV(bucketedRows.size(), groupKeyArray, groupByResultHolder, blockValSetMap);
+        }
+        for (int j = 0; j < groupKeyIndexes.size(); j++) {
+          Object[] row = aggregatedResult.get(j);
+          row[i] = aggregationFunction.extractGroupByResult(groupByResultHolder, j);
+          row[i] = aggregationFunction.extractFinalResult(row[i]);
+        }
+      }
+    }
+    return aggregatedResult;
+  }
+
+  private Object getFillValue(int columnIndex, String columnName, Object key, ColumnDataType dataType) {
+    ExpressionContext expressionContext = _fillExpressions.get(columnName);
+    if (expressionContext != null && expressionContext.getFunction() != null && GapfillUtils
+        .isFill(expressionContext)) {
+      List<ExpressionContext> args = expressionContext.getFunction().getArguments();
+      if (args.get(1).getLiteral() == null) {
+        throw new UnsupportedOperationException("Wrong Sql.");
+      }
+      GapfillUtils.FillType fillType = GapfillUtils.FillType.valueOf(args.get(1).getLiteral());
+      if (fillType == GapfillUtils.FillType.FILL_DEFAULT_VALUE) {
+        // TODO: may fill the default value from sql in the future.
+        return GapfillUtils.getDefaultValue(dataType);
+      } else if (fillType == GapfillUtils.FillType.FILL_PREVIOUS_VALUE) {
+        Object[] row = _previousByGroupKey.get(key);
+        if (row != null) {
+          return row[columnIndex];
+        } else {
+          return GapfillUtils.getDefaultValue(dataType);
+        }
+      } else {
+        throw new UnsupportedOperationException("unsupported fill type.");
+      }
+    } else {
+      return GapfillUtils.getDefaultValue(dataType);
+    }
+  }
+
+  /**
+   * Merge all result tables from different pinot servers and sort the rows based on timebucket.
+   */
+  private List<Object[]>[] putRawRowsIntoTimeBucket(List<Object[]> rows) {
+    List<Object[]>[] bucketedItems = new List[_numOfTimeBuckets];
+
+    for (Object[] row : rows) {
+      long timeBucket = _dateTimeFormatter.fromFormatToMillis(String.valueOf(row[_timeBucketColumnIndex]));
+      int index = findGapfillBucketIndex(timeBucket);
+      if (bucketedItems[index] == null) {
+        bucketedItems[index] = new ArrayList<>();
+      }
+      bucketedItems[index].add(row);
+      _groupByKeys.add(constructGroupKeys(row));
+    }
+    return bucketedItems;
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/HavingFilterHandler.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/HavingFilterHandler.java
index c4fe45f..885a8f1 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/HavingFilterHandler.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/HavingFilterHandler.java
@@ -18,25 +18,19 @@
  */
 package org.apache.pinot.core.query.reduce;
 
-import java.sql.Timestamp;
-import java.util.List;
 import org.apache.pinot.common.request.context.FilterContext;
-import org.apache.pinot.common.request.context.predicate.Predicate;
-import org.apache.pinot.core.operator.filter.predicate.PredicateEvaluator;
-import org.apache.pinot.core.operator.filter.predicate.PredicateEvaluatorProvider;
-import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.core.query.reduce.filter.RowMatcher;
+import org.apache.pinot.core.query.reduce.filter.RowMatcherFactory;
 
 
 /**
  * Handler for HAVING clause.
  */
 public class HavingFilterHandler {
-  private final PostAggregationHandler _postAggregationHandler;
   private final RowMatcher _rowMatcher;
 
   public HavingFilterHandler(FilterContext havingFilter, PostAggregationHandler postAggregationHandler) {
-    _postAggregationHandler = postAggregationHandler;
-    _rowMatcher = getRowMatcher(havingFilter);
+    _rowMatcher = RowMatcherFactory.getRowMatcher(havingFilter, postAggregationHandler);
   }
 
   /**
@@ -45,121 +39,4 @@ public class HavingFilterHandler {
   public boolean isMatch(Object[] row) {
     return _rowMatcher.isMatch(row);
   }
-
-  /**
-   * Helper method to construct a RowMatcher based on the given filter.
-   */
-  private RowMatcher getRowMatcher(FilterContext filter) {
-    switch (filter.getType()) {
-      case AND:
-        return new AndRowMatcher(filter.getChildren());
-      case OR:
-        return new OrRowMatcher(filter.getChildren());
-      case PREDICATE:
-        return new PredicateRowMatcher(filter.getPredicate());
-      default:
-        throw new IllegalStateException();
-    }
-  }
-
-  /**
-   * Filter matcher for the row.
-   */
-  private interface RowMatcher {
-
-    /**
-     * Returns {@code true} if the given row matches the filter, {@code false} otherwise.
-     */
-    boolean isMatch(Object[] row);
-  }
-
-  /**
-   * AND filter matcher.
-   */
-  private class AndRowMatcher implements RowMatcher {
-    RowMatcher[] _childMatchers;
-
-    AndRowMatcher(List<FilterContext> childFilters) {
-      int numChildren = childFilters.size();
-      _childMatchers = new RowMatcher[numChildren];
-      for (int i = 0; i < numChildren; i++) {
-        _childMatchers[i] = getRowMatcher(childFilters.get(i));
-      }
-    }
-
-    @Override
-    public boolean isMatch(Object[] row) {
-      for (RowMatcher childMatcher : _childMatchers) {
-        if (!childMatcher.isMatch(row)) {
-          return false;
-        }
-      }
-      return true;
-    }
-  }
-
-  /**
-   * OR filter matcher.
-   */
-  private class OrRowMatcher implements RowMatcher {
-    RowMatcher[] _childMatchers;
-
-    OrRowMatcher(List<FilterContext> childFilters) {
-      int numChildren = childFilters.size();
-      _childMatchers = new RowMatcher[numChildren];
-      for (int i = 0; i < numChildren; i++) {
-        _childMatchers[i] = getRowMatcher(childFilters.get(i));
-      }
-    }
-
-    @Override
-    public boolean isMatch(Object[] row) {
-      for (RowMatcher childMatcher : _childMatchers) {
-        if (childMatcher.isMatch(row)) {
-          return true;
-        }
-      }
-      return false;
-    }
-  }
-
-  /**
-   * Predicate matcher.
-   */
-  private class PredicateRowMatcher implements RowMatcher {
-    PostAggregationHandler.ValueExtractor _valueExtractor;
-    DataType _valueType;
-    PredicateEvaluator _predicateEvaluator;
-
-    PredicateRowMatcher(Predicate predicate) {
-      _valueExtractor = _postAggregationHandler.getValueExtractor(predicate.getLhs());
-      _valueType = _valueExtractor.getColumnDataType().toDataType();
-      _predicateEvaluator = PredicateEvaluatorProvider.getPredicateEvaluator(predicate, null, _valueType);
-    }
-
-    @Override
-    public boolean isMatch(Object[] row) {
-      Object value = _valueExtractor.extract(row);
-      switch (_valueType) {
-        case INT:
-          return _predicateEvaluator.applySV((int) value);
-        case LONG:
-          return _predicateEvaluator.applySV((long) value);
-        case FLOAT:
-          return _predicateEvaluator.applySV((float) value);
-        case DOUBLE:
-          return _predicateEvaluator.applySV((double) value);
-        case BOOLEAN:
-          return _predicateEvaluator.applySV((boolean) value ? 1 : 0);
-        case TIMESTAMP:
-          return _predicateEvaluator.applySV(((Timestamp) value).getTime());
-        case STRING:
-          return _predicateEvaluator.applySV((String) value);
-        case BYTES:
-          return _predicateEvaluator.applySV((byte[]) value);
-        default:
-          throw new IllegalStateException();
-      }
-    }
-  }
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/PostAggregationHandler.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/PostAggregationHandler.java
index d1a3cbf..7fb878c 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/PostAggregationHandler.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/PostAggregationHandler.java
@@ -30,6 +30,10 @@ import org.apache.pinot.common.request.context.RequestContextUtils;
 import org.apache.pinot.common.utils.DataSchema;
 import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
 import org.apache.pinot.core.query.postaggregation.PostAggregationFunction;
+import org.apache.pinot.core.query.reduce.filter.ColumnValueExtractor;
+import org.apache.pinot.core.query.reduce.filter.LiteralValueExtractor;
+import org.apache.pinot.core.query.reduce.filter.ValueExtractor;
+import org.apache.pinot.core.query.reduce.filter.ValueExtractorFactory;
 import org.apache.pinot.core.query.request.context.QueryContext;
 import org.apache.pinot.core.util.GapfillUtils;
 
@@ -38,7 +42,7 @@ import org.apache.pinot.core.util.GapfillUtils;
  * The {@code PostAggregationHandler} handles the post-aggregation calculation as well as the column re-ordering for the
  * aggregation result.
  */
-public class PostAggregationHandler {
+public class PostAggregationHandler implements ValueExtractorFactory {
   private final Map<Pair<FunctionContext, FilterContext>, Integer> _filteredAggregationsIndexMap;
   private final int _numGroupByExpressions;
   private final Map<ExpressionContext, Integer> _groupByExpressionIndexMap;
@@ -101,6 +105,7 @@ public class PostAggregationHandler {
   /**
    * Returns a ValueExtractor based on the given expression.
    */
+  @Override
   public ValueExtractor getValueExtractor(ExpressionContext expression) {
     expression = GapfillUtils.stripGapfill(expression);
     if (expression.getType() == ExpressionContext.Type.LITERAL) {
@@ -111,23 +116,23 @@ public class PostAggregationHandler {
       Integer groupByExpressionIndex = _groupByExpressionIndexMap.get(expression);
       if (groupByExpressionIndex != null) {
         // Group-by expression
-        return new ColumnValueExtractor(groupByExpressionIndex);
+        return new ColumnValueExtractor(groupByExpressionIndex, _dataSchema);
       }
     }
     FunctionContext function = expression.getFunction();
-    Preconditions.checkState(function != null, "Failed to find SELECT expression: %s in the GROUP-BY clause",
-        expression);
+    Preconditions
+        .checkState(function != null, "Failed to find SELECT expression: %s in the GROUP-BY clause", expression);
     if (function.getType() == FunctionContext.Type.AGGREGATION) {
       // Aggregation function
       return new ColumnValueExtractor(
-          _filteredAggregationsIndexMap.get(Pair.of(function, null)) + _numGroupByExpressions);
+          _filteredAggregationsIndexMap.get(Pair.of(function, null)) + _numGroupByExpressions, _dataSchema);
     } else if (function.getType() == FunctionContext.Type.TRANSFORM && function.getFunctionName()
         .equalsIgnoreCase("filter")) {
       FunctionContext aggregation = function.getArguments().get(0).getFunction();
       ExpressionContext filterExpression = function.getArguments().get(1);
       FilterContext filter = RequestContextUtils.getFilter(filterExpression);
       return new ColumnValueExtractor(
-          _filteredAggregationsIndexMap.get(Pair.of(aggregation, filter)) + _numGroupByExpressions);
+          _filteredAggregationsIndexMap.get(Pair.of(aggregation, filter)) + _numGroupByExpressions, _dataSchema);
     } else {
       // Post-aggregation function
       return new PostAggregationValueExtractor(function);
@@ -135,79 +140,6 @@ public class PostAggregationHandler {
   }
 
   /**
-   * Value extractor for the post-aggregation function.
-   */
-  public interface ValueExtractor {
-
-    /**
-     * Returns the column name for the value extracted.
-     */
-    String getColumnName();
-
-    /**
-     * Returns the ColumnDataType of the value extracted.
-     */
-    ColumnDataType getColumnDataType();
-
-    /**
-     * Extracts the value from the given row.
-     */
-    Object extract(Object[] row);
-  }
-
-  /**
-   * Value extractor for a literal.
-   */
-  private static class LiteralValueExtractor implements ValueExtractor {
-    final String _literal;
-
-    LiteralValueExtractor(String literal) {
-      _literal = literal;
-    }
-
-    @Override
-    public String getColumnName() {
-      return '\'' + _literal + '\'';
-    }
-
-    @Override
-    public ColumnDataType getColumnDataType() {
-      return ColumnDataType.STRING;
-    }
-
-    @Override
-    public Object extract(Object[] row) {
-      return _literal;
-    }
-  }
-
-  /**
-   * Value extractor for a non-post-aggregation column (group-by expression or aggregation).
-   */
-  private class ColumnValueExtractor implements ValueExtractor {
-    final int _index;
-
-    ColumnValueExtractor(int index) {
-      _index = index;
-    }
-
-    @Override
-    public String getColumnName() {
-      return _dataSchema.getColumnName(_index);
-    }
-
-    @Override
-    public ColumnDataType getColumnDataType() {
-      return _dataSchema.getColumnDataType(_index);
-    }
-
-    @Override
-    public Object extract(Object[] row) {
-      return row[_index];
-    }
-  }
-
-  /**
    * Value extractor for a post-aggregation column.
    */
   private class PostAggregationValueExtractor implements ValueExtractor {
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/RowBasedBlockValSet.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/RowBasedBlockValSet.java
new file mode 100644
index 0000000..90a759d
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/RowBasedBlockValSet.java
@@ -0,0 +1,192 @@
+/**
+ * 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.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.BlockValSet;
+import org.apache.pinot.segment.spi.index.reader.Dictionary;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * When the data is retrieved from pinot servers and merged on the broker, it
+ * will become row based data. If we want to apply Transformation or Aggregation,
+ * we need {@link BlockValSet} format data. This class is used to provide
+ * {@link BlockValSet} interface wrapping around row based data.
+ *
+ * TODO: We need add support for BYTES and MV
+ */
+public class RowBasedBlockValSet implements BlockValSet {
+
+  private final FieldSpec.DataType _dataType;
+  private final List<Object[]> _rows;
+  private final int _columnIndex;
+
+  public RowBasedBlockValSet(DataSchema.ColumnDataType columnDataType, List<Object[]> rows,
+      int columnIndex) {
+    _dataType = columnDataType.toDataType();
+    _rows = rows;
+    _columnIndex = columnIndex;
+  }
+
+  @Override
+  public FieldSpec.DataType getValueType() {
+    return _dataType;
+  }
+
+  @Override
+  public boolean isSingleValue() {
+    return true;
+  }
+
+  @Nullable
+  @Override
+  public Dictionary getDictionary() {
+    return null;
+  }
+
+  @Override
+  public int[] getDictionaryIdsSV() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int[] getIntValuesSV() {
+    int length = _rows.size();
+    int[] values = new int[length];
+    if (_dataType.isNumeric()) {
+      for (int i = 0; i < length; i++) {
+        values[i] = ((Number) _rows.get(i)[_columnIndex]).intValue();
+      }
+    } else if (_dataType == FieldSpec.DataType.STRING) {
+      for (int i = 0; i < length; i++) {
+        values[i] = Integer.parseInt((String) _rows.get(i)[_columnIndex]);
+      }
+    } else {
+      throw new IllegalStateException("Cannot read int values from data type: " + _dataType);
+    }
+    return values;
+  }
+
+  @Override
+  public long[] getLongValuesSV() {
+    int length = _rows.size();
+    long[] values = new long[length];
+    if (_dataType.isNumeric()) {
+      for (int i = 0; i < length; i++) {
+        values[i] = ((Number) _rows.get(i)[_columnIndex]).longValue();
+      }
+    } else if (_dataType == FieldSpec.DataType.STRING) {
+      for (int i = 0; i < length; i++) {
+        values[i] = Long.parseLong((String) _rows.get(i)[_columnIndex]);
+      }
+    } else {
+      throw new IllegalStateException("Cannot read long values from data type: " + _dataType);
+    }
+    return values;
+  }
+
+  @Override
+  public float[] getFloatValuesSV() {
+    int length = _rows.size();
+    float[] values = new float[length];
+    if (_dataType.isNumeric()) {
+      for (int i = 0; i < length; i++) {
+        values[i] = ((Number) _rows.get(i)[_columnIndex]).floatValue();
+      }
+    } else if (_dataType == FieldSpec.DataType.STRING) {
+      for (int i = 0; i < length; i++) {
+        values[i] = Float.parseFloat((String) _rows.get(i)[_columnIndex]);
+      }
+    } else {
+      throw new IllegalStateException("Cannot read float values from data type: " + _dataType);
+    }
+    return values;
+  }
+
+  @Override
+  public double[] getDoubleValuesSV() {
+    int length = _rows.size();
+    double[] values = new double[length];
+    if (_dataType.isNumeric()) {
+      for (int i = 0; i < length; i++) {
+        values[i] = ((Number) _rows.get(i)[_columnIndex]).doubleValue();
+      }
+    } else if (_dataType == FieldSpec.DataType.STRING) {
+      for (int i = 0; i < length; i++) {
+        values[i] = Double.parseDouble((String) _rows.get(i)[_columnIndex]);
+      }
+    } else {
+      throw new IllegalStateException("Cannot read double values from data type: " + _dataType);
+    }
+    return values;
+  }
+
+  @Override
+  public String[] getStringValuesSV() {
+    int length = _rows.size();
+    String[] values = new String[length];
+    for (int i = 0; i < length; i++) {
+      values[i] = _rows.get(i)[_columnIndex].toString();
+    }
+    return values;
+  }
+
+  @Override
+  public byte[][] getBytesValuesSV() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int[][] getDictionaryIdsMV() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int[][] getIntValuesMV() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public long[][] getLongValuesMV() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public float[][] getFloatValuesMV() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public double[][] getDoubleValuesMV() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String[][] getStringValuesMV() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int[] getNumMVEntries() {
+    throw new UnsupportedOperationException();
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/AndRowMatcher.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/AndRowMatcher.java
new file mode 100644
index 0000000..a222ba8
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/AndRowMatcher.java
@@ -0,0 +1,48 @@
+/**
+ * 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.filter;
+
+import java.util.List;
+import org.apache.pinot.common.request.context.FilterContext;
+
+
+/**
+ * AND filter matcher.
+ */
+public class AndRowMatcher implements RowMatcher {
+  RowMatcher[] _childMatchers;
+
+  AndRowMatcher(List<FilterContext> childFilters, ValueExtractorFactory valueExtractorFactory) {
+    int numChildren = childFilters.size();
+    _childMatchers = new RowMatcher[numChildren];
+    for (int i = 0; i < numChildren; i++) {
+      _childMatchers[i] = RowMatcherFactory.getRowMatcher(childFilters.get(i), valueExtractorFactory);
+    }
+  }
+
+  @Override
+  public boolean isMatch(Object[] row) {
+    for (RowMatcher childMatcher : _childMatchers) {
+      if (!childMatcher.isMatch(row)) {
+        return false;
+      }
+    }
+    return true;
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/ColumnValueExtractor.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/ColumnValueExtractor.java
new file mode 100644
index 0000000..77a2c1e
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/ColumnValueExtractor.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.query.reduce.filter;
+
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+
+
+/**
+ * Value extractor for a non-post-aggregation column (group-by expression or aggregation).
+ */
+public class ColumnValueExtractor implements ValueExtractor {
+  private final int _index;
+  private final DataSchema _dataSchema;
+
+  public ColumnValueExtractor(int index, DataSchema dataSchema) {
+    _index = index;
+    _dataSchema = dataSchema;
+  }
+
+  @Override
+  public String getColumnName() {
+    return _dataSchema.getColumnName(_index);
+  }
+
+  @Override
+  public ColumnDataType getColumnDataType() {
+    return _dataSchema.getColumnDataType(_index);
+  }
+
+  @Override
+  public Object extract(Object[] row) {
+    return row[_index];
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/LiteralValueExtractor.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/LiteralValueExtractor.java
new file mode 100644
index 0000000..101a83a
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/LiteralValueExtractor.java
@@ -0,0 +1,48 @@
+/**
+ * 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.filter;
+
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+
+
+/**
+ * Value extractor for a literal.
+ */
+public class LiteralValueExtractor implements ValueExtractor {
+  final String _literal;
+
+  public LiteralValueExtractor(String literal) {
+    _literal = literal;
+  }
+
+  @Override
+  public String getColumnName() {
+    return '\'' + _literal + '\'';
+  }
+
+  @Override
+  public ColumnDataType getColumnDataType() {
+    return ColumnDataType.STRING;
+  }
+
+  @Override
+  public Object extract(Object[] row) {
+    return _literal;
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/OrRowMatcher.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/OrRowMatcher.java
new file mode 100644
index 0000000..dc47c6a
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/OrRowMatcher.java
@@ -0,0 +1,48 @@
+/**
+ * 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.filter;
+
+import java.util.List;
+import org.apache.pinot.common.request.context.FilterContext;
+
+
+/**
+ * OR filter matcher.
+ */
+public class OrRowMatcher implements RowMatcher {
+  RowMatcher[] _childMatchers;
+
+  public OrRowMatcher(List<FilterContext> childFilters, ValueExtractorFactory valueExtractorFactory) {
+    int numChildren = childFilters.size();
+    _childMatchers = new RowMatcher[numChildren];
+    for (int i = 0; i < numChildren; i++) {
+      _childMatchers[i] = RowMatcherFactory.getRowMatcher(childFilters.get(i), valueExtractorFactory);
+    }
+  }
+
+  @Override
+  public boolean isMatch(Object[] row) {
+    for (RowMatcher childMatcher : _childMatchers) {
+      if (childMatcher.isMatch(row)) {
+        return true;
+      }
+    }
+    return false;
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/PredicateRowMatcher.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/PredicateRowMatcher.java
new file mode 100644
index 0000000..4645ef5
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/PredicateRowMatcher.java
@@ -0,0 +1,66 @@
+/**
+ * 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.filter;
+
+import java.sql.Timestamp;
+import org.apache.pinot.common.request.context.predicate.Predicate;
+import org.apache.pinot.core.operator.filter.predicate.PredicateEvaluator;
+import org.apache.pinot.core.operator.filter.predicate.PredicateEvaluatorProvider;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+
+
+/**
+ * Predicate matcher.
+ */
+public class PredicateRowMatcher implements RowMatcher {
+  ValueExtractor _valueExtractor;
+  DataType _valueType;
+  PredicateEvaluator _predicateEvaluator;
+
+  public PredicateRowMatcher(Predicate predicate, ValueExtractor valueExtractor) {
+    _valueExtractor = valueExtractor;
+    _valueType = _valueExtractor.getColumnDataType().toDataType();
+    _predicateEvaluator = PredicateEvaluatorProvider.getPredicateEvaluator(predicate, null, _valueType);
+  }
+
+  @Override
+  public boolean isMatch(Object[] row) {
+    Object value = _valueExtractor.extract(row);
+    switch (_valueType) {
+      case INT:
+        return _predicateEvaluator.applySV((int) value);
+      case LONG:
+        return _predicateEvaluator.applySV((long) value);
+      case FLOAT:
+        return _predicateEvaluator.applySV((float) value);
+      case DOUBLE:
+        return _predicateEvaluator.applySV((double) value);
+      case BOOLEAN:
+        return _predicateEvaluator.applySV((boolean) value ? 1 : 0);
+      case TIMESTAMP:
+        return _predicateEvaluator.applySV(((Timestamp) value).getTime());
+      case STRING:
+        return _predicateEvaluator.applySV((String) value);
+      case BYTES:
+        return _predicateEvaluator.applySV((byte[]) value);
+      default:
+        throw new IllegalStateException();
+    }
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/RowMatcher.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/RowMatcher.java
new file mode 100644
index 0000000..8986366
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/RowMatcher.java
@@ -0,0 +1,29 @@
+/**
+ * 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.filter;
+
+/**
+ * Filter matcher for the rows.
+ */
+public interface RowMatcher {
+  /**
+   * Returns {@code true} if the given row matches the filter, {@code false} otherwise.
+   */
+  boolean isMatch(Object[] row);
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/RowMatcherFactory.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/RowMatcherFactory.java
new file mode 100644
index 0000000..206e7ab
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/RowMatcherFactory.java
@@ -0,0 +1,47 @@
+/**
+ * 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.filter;
+
+import org.apache.pinot.common.request.context.FilterContext;
+
+
+/**
+ * Factory for RowMatcher.
+ */
+public class RowMatcherFactory {
+  private RowMatcherFactory() {
+  }
+
+  /**
+   * Helper method to construct a RowMatcher based on the given filter.
+   */
+  public static RowMatcher getRowMatcher(FilterContext filter, ValueExtractorFactory valueExtractorFactory) {
+    switch (filter.getType()) {
+      case AND:
+        return new AndRowMatcher(filter.getChildren(), valueExtractorFactory);
+      case OR:
+        return new OrRowMatcher(filter.getChildren(), valueExtractorFactory);
+      case PREDICATE:
+        return new PredicateRowMatcher(filter.getPredicate(),
+            valueExtractorFactory.getValueExtractor(filter.getPredicate().getLhs()));
+      default:
+        throw new IllegalStateException();
+    }
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/ValueExtractor.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/ValueExtractor.java
new file mode 100644
index 0000000..74b329e
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/ValueExtractor.java
@@ -0,0 +1,42 @@
+/**
+ * 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.filter;
+
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+
+
+/**
+ * Value extractor for the post-aggregation function or pre-aggregation gap fill.
+ */
+public interface ValueExtractor {
+  /**
+   * Returns the column name for the value extracted.
+   */
+  String getColumnName();
+
+  /**
+   * Returns the ColumnDataType of the value extracted.
+   */
+  ColumnDataType getColumnDataType();
+
+  /**
+   * Extracts the value from the given row.
+   */
+  Object extract(Object[] row);
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/ValueExtractorFactory.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/ValueExtractorFactory.java
new file mode 100644
index 0000000..8aae73a
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/filter/ValueExtractorFactory.java
@@ -0,0 +1,35 @@
+/**
+ * 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.filter;
+
+import org.apache.pinot.common.request.context.ExpressionContext;
+
+
+/**
+ * Factory for {@link ValueExtractor}.
+ */
+public interface ValueExtractorFactory {
+  /**
+   * Create the {@link ValueExtractor} for specific column.
+   *
+   * @param expression - column expression
+   * @return {@link ValueExtractor}
+   */
+  ValueExtractor getValueExtractor(ExpressionContext expression);
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/request/context/QueryContext.java b/pinot-core/src/main/java/org/apache/pinot/core/query/request/context/QueryContext.java
index 3a396b6..58d2891 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/query/request/context/QueryContext.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/request/context/QueryContext.java
@@ -85,6 +85,7 @@ public class QueryContext {
   // Keep the BrokerRequest to make incremental changes
   // TODO: Remove it once the whole query engine is using the QueryContext
   private final BrokerRequest _brokerRequest;
+  private final QueryContext _subquery;
 
   private final Function<Class<?>, Map<?, ?>> _sharedValues = MemoizedClassAssociation.of(ConcurrentHashMap::new);
 
@@ -120,7 +121,7 @@ public class QueryContext {
       @Nullable FilterContext filter, @Nullable List<ExpressionContext> groupByExpressions,
       @Nullable FilterContext havingFilter, @Nullable List<OrderByExpressionContext> orderByExpressions, int limit,
       int offset, Map<String, String> queryOptions, @Nullable Map<String, String> debugOptions,
-      BrokerRequest brokerRequest) {
+      BrokerRequest brokerRequest, QueryContext subquery) {
     _tableName = tableName;
     _selectExpressions = selectExpressions;
     _aliasList = Collections.unmodifiableList(aliasList);
@@ -133,6 +134,7 @@ public class QueryContext {
     _queryOptions = queryOptions;
     _debugOptions = debugOptions;
     _brokerRequest = brokerRequest;
+    _subquery = subquery;
   }
 
   /**
@@ -188,6 +190,10 @@ public class QueryContext {
     return _orderByExpressions;
   }
 
+  public QueryContext getSubquery() {
+    return _subquery;
+  }
+
   /**
    * Returns the limit of the query.
    */
@@ -375,6 +381,7 @@ public class QueryContext {
     private Map<String, String> _queryOptions;
     private Map<String, String> _debugOptions;
     private BrokerRequest _brokerRequest;
+    private QueryContext _subquery;
 
     public Builder setTableName(String tableName) {
       _tableName = tableName;
@@ -436,6 +443,11 @@ public class QueryContext {
       return this;
     }
 
+    public Builder setSubquery(QueryContext subquery) {
+      _subquery = subquery;
+      return this;
+    }
+
     public QueryContext build() {
       // TODO: Add validation logic here
 
@@ -444,7 +456,7 @@ public class QueryContext {
       }
       QueryContext queryContext =
           new QueryContext(_tableName, _selectExpressions, _aliasList, _filter, _groupByExpressions, _havingFilter,
-              _orderByExpressions, _limit, _offset, _queryOptions, _debugOptions, _brokerRequest);
+              _orderByExpressions, _limit, _offset, _queryOptions, _debugOptions, _brokerRequest, _subquery);
 
       // Pre-calculate the aggregation functions and columns for the query
       generateAggregationFunctions(queryContext);
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/request/context/utils/BrokerRequestToQueryContextConverter.java b/pinot-core/src/main/java/org/apache/pinot/core/query/request/context/utils/BrokerRequestToQueryContextConverter.java
index 4f835c2..965c4bc 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/query/request/context/utils/BrokerRequestToQueryContextConverter.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/request/context/utils/BrokerRequestToQueryContextConverter.java
@@ -53,12 +53,19 @@ public class BrokerRequestToQueryContextConverter {
    * Converts the given {@link BrokerRequest} into a {@link QueryContext}.
    */
   public static QueryContext convert(BrokerRequest brokerRequest) {
-    return brokerRequest.getPinotQuery() != null ? convertSQL(brokerRequest) : convertPQL(brokerRequest);
+    if (brokerRequest.getPinotQuery() != null) {
+      QueryContext queryContext = convertSQL(brokerRequest.getPinotQuery(), brokerRequest);
+      return queryContext;
+    } else {
+      return convertPQL(brokerRequest);
+    }
   }
 
-  private static QueryContext convertSQL(BrokerRequest brokerRequest) {
-    PinotQuery pinotQuery = brokerRequest.getPinotQuery();
-
+  private static QueryContext convertSQL(PinotQuery pinotQuery, BrokerRequest brokerRequest) {
+    QueryContext subquery = null;
+    if (pinotQuery.getDataSource().getSubquery() != null) {
+      subquery = convertSQL(pinotQuery.getDataSource().getSubquery(), brokerRequest);
+    }
     // SELECT
     List<ExpressionContext> selectExpressions;
     List<Expression> selectList = pinotQuery.getSelectList();
@@ -147,7 +154,7 @@ public class BrokerRequestToQueryContextConverter {
         .setGroupByExpressions(groupByExpressions).setOrderByExpressions(orderByExpressions)
         .setHavingFilter(havingFilter).setLimit(pinotQuery.getLimit()).setOffset(pinotQuery.getOffset())
         .setQueryOptions(pinotQuery.getQueryOptions()).setDebugOptions(pinotQuery.getDebugOptions())
-        .setBrokerRequest(brokerRequest).build();
+        .setSubquery(subquery).setBrokerRequest(brokerRequest).build();
   }
 
   private static QueryContext convertPQL(BrokerRequest brokerRequest) {
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/util/GapfillUtils.java b/pinot-core/src/main/java/org/apache/pinot/core/util/GapfillUtils.java
index 55b618c..c40837e 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/util/GapfillUtils.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/util/GapfillUtils.java
@@ -18,8 +18,17 @@
  */
 package org.apache.pinot.core.util;
 
+import com.google.common.base.Preconditions;
 import java.io.Serializable;
-import org.apache.commons.lang3.StringUtils;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.common.request.DataSource;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.ExpressionType;
+import org.apache.pinot.common.request.PinotQuery;
+import org.apache.pinot.common.request.QuerySource;
 import org.apache.pinot.common.request.context.ExpressionContext;
 import org.apache.pinot.common.request.context.FunctionContext;
 import org.apache.pinot.common.utils.DataSchema;
@@ -31,7 +40,11 @@ import org.apache.pinot.core.query.request.context.QueryContext;
  */
 public class GapfillUtils {
   private static final String POST_AGGREGATE_GAP_FILL = "postaggregategapfill";
+  private static final String GAP_FILL = "gapfill";
+  private static final String AS = "as";
   private static final String FILL = "fill";
+  private static final String TIME_SERIES_ON = "timeserieson";
+  private static final int STARTING_INDEX_OF_OPTIONAL_ARGS_FOR_PRE_AGGREGATE_GAP_FILL = 5;
 
   private GapfillUtils() {
   }
@@ -42,8 +55,8 @@ public class GapfillUtils {
     }
 
     FunctionContext function = expression.getFunction();
-    String functionName = canonicalizeFunctionName(function.getFunctionName());
-    if (functionName.equals(POST_AGGREGATE_GAP_FILL) || functionName.equals(FILL)) {
+    String functionName = function.getFunctionName();
+    if (functionName.equals(POST_AGGREGATE_GAP_FILL) || functionName.equals(FILL) || functionName.equals(GAP_FILL)) {
       return function.getArguments().get(0);
     }
     return expression;
@@ -54,7 +67,7 @@ public class GapfillUtils {
       return false;
     }
 
-    return POST_AGGREGATE_GAP_FILL.equals(canonicalizeFunctionName(expressionContext.getFunction().getFunctionName()));
+    return POST_AGGREGATE_GAP_FILL.equals(expressionContext.getFunction().getFunctionName());
   }
 
   public static boolean isPostAggregateGapfill(QueryContext queryContext) {
@@ -71,12 +84,15 @@ public class GapfillUtils {
       return false;
     }
 
-    return FILL.equals(canonicalizeFunctionName(expressionContext.getFunction().getFunctionName()));
+    return FILL.equals(expressionContext.getFunction().getFunctionName());
   }
 
-  static public enum FillType {
-    FILL_DEFAULT_VALUE,
-    FILL_PREVIOUS_VALUE,
+  public static boolean isTimeSeriesOn(ExpressionContext expressionContext) {
+    if (expressionContext.getType() != ExpressionContext.Type.FUNCTION) {
+      return false;
+    }
+
+    return TIME_SERIES_ON.equals(expressionContext.getFunction().getFunctionName());
   }
 
   /**
@@ -116,7 +132,267 @@ public class GapfillUtils {
     }
   }
 
-  private static String canonicalizeFunctionName(String functionName) {
-    return StringUtils.remove(functionName, '_').toLowerCase();
+  public static boolean isGapfill(ExpressionContext expressionContext) {
+    if (expressionContext.getType() != ExpressionContext.Type.FUNCTION) {
+      return false;
+    }
+
+    return GAP_FILL.equals(expressionContext.getFunction().getFunctionName());
+  }
+
+  private static boolean isGapfill(QueryContext queryContext) {
+    for (ExpressionContext expressionContext : queryContext.getSelectExpressions()) {
+      if (isGapfill(expressionContext)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Get the gapfill type for queryContext. Also do the validation for gapfill request.
+   * @param queryContext
+   */
+  public static GapfillType getGapfillType(QueryContext queryContext) {
+    GapfillType gapfillType = null;
+    if (queryContext.getSubquery() == null) {
+      if (isGapfill(queryContext)) {
+        Preconditions.checkArgument(queryContext.getAggregationFunctions() == null,
+            "Aggregation and Gapfill can not be in the same sql statement.");
+        gapfillType = GapfillType.GAP_FILL;
+      }
+    } else if (isGapfill(queryContext)) {
+      Preconditions.checkArgument(queryContext.getSubquery().getAggregationFunctions() != null,
+          "Select and Gapfill should be in the same sql statement.");
+      Preconditions.checkArgument(queryContext.getSubquery().getSubquery() == null,
+          "There is no three levels nesting sql when the outer query is gapfill.");
+      gapfillType = GapfillType.AGGREGATE_GAP_FILL;
+    } else if (isGapfill(queryContext.getSubquery())) {
+      if (queryContext.getAggregationFunctions() == null) {
+        gapfillType = GapfillType.GAP_FILL_SELECT;
+      } else if (queryContext.getSubquery().getSubquery() == null) {
+        gapfillType = GapfillType.GAP_FILL_AGGREGATE;
+      } else {
+        Preconditions
+            .checkArgument(queryContext.getSubquery().getSubquery().getAggregationFunctions() != null,
+                "Select cannot happen before gapfill.");
+        gapfillType = GapfillType.AGGREGATE_GAP_FILL_AGGREGATE;
+      }
+    }
+
+    if (gapfillType == null) {
+      return gapfillType;
+    }
+
+    ExpressionContext gapFillSelection = GapfillUtils.getGapfillExpressionContext(queryContext, gapfillType);
+
+    Preconditions.checkArgument(gapFillSelection != null && gapFillSelection.getFunction() != null,
+        "Gapfill Expression should be function.");
+    List<ExpressionContext> args = gapFillSelection.getFunction().getArguments();
+    Preconditions.checkArgument(args.size() > 5, "PreAggregateGapFill 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.");
+
+    ExpressionContext timeseriesOn = GapfillUtils.getTimeSeriesOnExpressionContext(gapFillSelection);
+    Preconditions.checkArgument(timeseriesOn != null, "The TimeSeriesOn expressions should be specified.");
+
+    if (queryContext.getAggregationFunctions() == null) {
+      return gapfillType;
+    }
+
+    List<ExpressionContext> groupbyExpressions = queryContext.getGroupByExpressions();
+    Preconditions.checkArgument(groupbyExpressions != null, "No GroupBy Clause.");
+    List<ExpressionContext> innerSelections = queryContext.getSubquery().getSelectExpressions();
+    String timeBucketCol = null;
+    List<String> strAlias = queryContext.getSubquery().getAliasList();
+    for (int i = 0; i < innerSelections.size(); i++) {
+      ExpressionContext innerSelection = innerSelections.get(i);
+      if (GapfillUtils.isGapfill(innerSelection)) {
+        if (strAlias.get(i) != null) {
+          timeBucketCol = strAlias.get(i);
+        } else {
+          timeBucketCol = innerSelection.getFunction().getArguments().get(0).toString();
+        }
+        break;
+      }
+    }
+
+    Preconditions.checkArgument(timeBucketCol != null, "No Group By timebucket.");
+
+    boolean findTimeBucket = false;
+    for (ExpressionContext groupbyExp : groupbyExpressions) {
+      if (timeBucketCol.equals(groupbyExp.toString())) {
+        findTimeBucket = true;
+        break;
+      }
+    }
+
+    Preconditions.checkArgument(findTimeBucket, "No Group By timebucket.");
+    return gapfillType;
+  }
+
+  private static ExpressionContext findGapfillExpressionContext(QueryContext queryContext) {
+    for (ExpressionContext expressionContext : queryContext.getSelectExpressions()) {
+      if (isGapfill(expressionContext)) {
+        return expressionContext;
+      }
+    }
+    return null;
+  }
+
+  public static ExpressionContext getGapfillExpressionContext(QueryContext queryContext, GapfillType gapfillType) {
+    if (gapfillType == GapfillType.AGGREGATE_GAP_FILL || gapfillType == GapfillType.GAP_FILL) {
+      return findGapfillExpressionContext(queryContext);
+    } else if (gapfillType == GapfillType.GAP_FILL_AGGREGATE || gapfillType == GapfillType.AGGREGATE_GAP_FILL_AGGREGATE
+        || gapfillType == GapfillType.GAP_FILL_SELECT) {
+      return findGapfillExpressionContext(queryContext.getSubquery());
+    } else {
+      return null;
+    }
+  }
+
+  public static int findTimeBucketColumnIndex(QueryContext queryContext, GapfillType gapfillType) {
+    if (gapfillType == GapfillType.GAP_FILL_AGGREGATE
+        || gapfillType == GapfillType.GAP_FILL_SELECT
+        || gapfillType == GapfillType.AGGREGATE_GAP_FILL_AGGREGATE) {
+      queryContext = queryContext.getSubquery();
+    }
+    List<ExpressionContext> expressionContexts = queryContext.getSelectExpressions();
+    for (int i = 0; i < expressionContexts.size(); i++) {
+      if (isGapfill(expressionContexts.get(i))) {
+        return i;
+      }
+    }
+    return -1;
+  }
+
+  public static ExpressionContext getTimeSeriesOnExpressionContext(ExpressionContext gapFillSelection) {
+    List<ExpressionContext> args = gapFillSelection.getFunction().getArguments();
+    for (int i = STARTING_INDEX_OF_OPTIONAL_ARGS_FOR_PRE_AGGREGATE_GAP_FILL; i < args.size(); i++) {
+      if (GapfillUtils.isTimeSeriesOn(args.get(i))) {
+        return args.get(i);
+      }
+    }
+    return null;
+  }
+
+  public static Map<String, ExpressionContext> getFillExpressions(ExpressionContext gapFillSelection) {
+    Map<String, ExpressionContext> fillExpressions = new HashMap<>();
+    List<ExpressionContext> args = gapFillSelection.getFunction().getArguments();
+    for (int i = STARTING_INDEX_OF_OPTIONAL_ARGS_FOR_PRE_AGGREGATE_GAP_FILL; i < args.size(); i++) {
+      if (GapfillUtils.isFill(args.get(i))) {
+        ExpressionContext fillExpression = args.get(i);
+        fillExpressions.put(fillExpression.getFunction().getArguments().get(0).getIdentifier(), fillExpression);
+      }
+    }
+    return fillExpressions;
+  }
+
+  public static BrokerRequest stripGapfill(BrokerRequest brokerRequest) {
+    if (brokerRequest.getPinotQuery().getDataSource() == null) {
+      return brokerRequest;
+    }
+    PinotQuery pinotQuery = brokerRequest.getPinotQuery();
+    if (pinotQuery.getDataSource().getSubquery() == null && !hasGapfill(pinotQuery)) {
+      return brokerRequest;
+    }
+
+    // carry over the query options from original query to server query.
+    Map<String, String> queryOptions = pinotQuery.getQueryOptions();
+    Map<String, String> debugOptions = pinotQuery.getDebugOptions();
+
+    while (pinotQuery.getDataSource().getSubquery() != null) {
+      pinotQuery = pinotQuery.getDataSource().getSubquery();
+    }
+
+    BrokerRequest strippedBrokerRequest = stripGapfill(pinotQuery);
+    strippedBrokerRequest.getPinotQuery().setQueryOptions(queryOptions);
+    strippedBrokerRequest.getPinotQuery().setDebugOptions(debugOptions);
+    return strippedBrokerRequest;
+  }
+
+  private static boolean hasGapfill(PinotQuery pinotQuery) {
+    for (Expression select : pinotQuery.getSelectList()) {
+      if (select.getType() != ExpressionType.FUNCTION) {
+        continue;
+      }
+      if (GAP_FILL.equals(select.getFunctionCall().getOperator())) {
+        return true;
+      }
+      if (AS.equals(select.getFunctionCall().getOperator())
+          && select.getFunctionCall().getOperands().get(0).getType() == ExpressionType.FUNCTION
+          && GAP_FILL.equals(select.getFunctionCall().getOperands().get(0).getFunctionCall().getOperator())) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private static BrokerRequest stripGapfill(PinotQuery pinotQuery) {
+    PinotQuery copy = new PinotQuery(pinotQuery);
+    BrokerRequest brokerRequest = new BrokerRequest();
+    brokerRequest.setPinotQuery(copy);
+    // Set table name in broker request because it is used for access control, query routing etc.
+    DataSource dataSource = copy.getDataSource();
+    if (dataSource != null) {
+      QuerySource querySource = new QuerySource();
+      querySource.setTableName(dataSource.getTableName());
+      brokerRequest.setQuerySource(querySource);
+    }
+    List<Expression> selectList = copy.getSelectList();
+    for (int i = 0; i < selectList.size(); i++) {
+      Expression select = selectList.get(i);
+      if (select.getType() != ExpressionType.FUNCTION) {
+        continue;
+      }
+      if (GAP_FILL.equals(select.getFunctionCall().getOperator())) {
+        selectList.set(i, select.getFunctionCall().getOperands().get(0));
+        break;
+      }
+      if (AS.equals(select.getFunctionCall().getOperator())
+          && select.getFunctionCall().getOperands().get(0).getType() == ExpressionType.FUNCTION
+          && GAP_FILL.equals(select.getFunctionCall().getOperands().get(0).getFunctionCall().getOperator())) {
+        select.getFunctionCall().getOperands().set(0,
+            select.getFunctionCall().getOperands().get(0).getFunctionCall().getOperands().get(0));
+        break;
+      }
+    }
+
+    for (Expression orderBy : copy.getOrderByList()) {
+      if (orderBy.getType() != ExpressionType.FUNCTION) {
+        continue;
+      }
+      if (orderBy.getFunctionCall().getOperands().get(0).getType() == ExpressionType.FUNCTION
+          && GAP_FILL.equals(
+              orderBy.getFunctionCall().getOperands().get(0).getFunctionCall().getOperator())) {
+        orderBy.getFunctionCall().getOperands().set(0,
+            orderBy.getFunctionCall().getOperands().get(0).getFunctionCall().getOperands().get(0));
+        break;
+      }
+    }
+    return brokerRequest;
+  }
+
+  public enum GapfillType {
+    // one sql query with gapfill only
+    GAP_FILL,
+    // gapfill as subquery, the outer query may have the filter
+    GAP_FILL_SELECT,
+    // gapfill as subquery, the outer query has the aggregation
+    GAP_FILL_AGGREGATE,
+    // aggregation as subqery, the outer query is gapfill
+    AGGREGATE_GAP_FILL,
+    // aggegration as second nesting subquery, gapfill as first nesting subquery, different aggregation as outer query
+    AGGREGATE_GAP_FILL_AGGREGATE
+  }
+
+  public enum FillType {
+    FILL_DEFAULT_VALUE, FILL_PREVIOUS_VALUE,
   }
 }
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/query/reduce/GapfillFilterHandlerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/query/reduce/GapfillFilterHandlerTest.java
new file mode 100644
index 0000000..64b6d4b
--- /dev/null
+++ b/pinot-core/src/test/java/org/apache/pinot/core/query/reduce/GapfillFilterHandlerTest.java
@@ -0,0 +1,44 @@
+/**
+ * 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 org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.utils.QueryContextConverterUtils;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertTrue;
+
+
+public class GapfillFilterHandlerTest {
+
+  @Test
+  public void testFilter() {
+    QueryContext queryContext =
+        QueryContextConverterUtils.getQueryContextFromSQL("SELECT d1, d2 FROM testTable WHERE d2 > 5");
+    DataSchema dataSchema =
+        new DataSchema(new String[]{"d1", "d2"}, new ColumnDataType[]{ColumnDataType.INT, ColumnDataType.LONG});
+    GapfillFilterHandler gapfillFilterHandler = new GapfillFilterHandler(queryContext.getFilter(), dataSchema);
+    assertFalse(gapfillFilterHandler.isMatch(new Object[]{1, 5L}));
+    assertTrue(gapfillFilterHandler.isMatch(new Object[]{2, 10L}));
+    assertFalse(gapfillFilterHandler.isMatch(new Object[]{3, 3L}));
+  }
+}
diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java
index e283978..6a2cfc2 100644
--- a/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java
@@ -40,6 +40,7 @@ import org.apache.pinot.core.query.request.context.QueryContext;
 import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
 import org.apache.pinot.core.query.request.context.utils.QueryContextConverterUtils;
 import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.core.util.GapfillUtils;
 import org.apache.pinot.pql.parsers.Pql2Compiler;
 import org.apache.pinot.segment.spi.IndexSegment;
 import org.apache.pinot.spi.config.table.TableConfig;
@@ -161,7 +162,7 @@ public abstract class BaseQueriesTest {
       }
     }
     QueryContext queryContext = BrokerRequestToQueryContextConverter.convert(brokerRequest);
-    return getBrokerResponse(queryContext, planMaker);
+    return getBrokerResponse(queryContext, queryContext, planMaker);
   }
 
   /**
@@ -197,8 +198,15 @@ public abstract class BaseQueriesTest {
     }
     queryOptions.put(Request.QueryOptionKey.GROUP_BY_MODE, Request.SQL);
     queryOptions.put(Request.QueryOptionKey.RESPONSE_FORMAT, Request.SQL);
+    BrokerRequest serverBrokerRequest = GapfillUtils.stripGapfill(brokerRequest);
     QueryContext queryContext = BrokerRequestToQueryContextConverter.convert(brokerRequest);
-    return getBrokerResponse(queryContext, planMaker);
+    QueryContext serverQueryContext;
+    if (brokerRequest == serverBrokerRequest) {
+      serverQueryContext = queryContext;
+    } else {
+      serverQueryContext = BrokerRequestToQueryContextConverter.convert(serverBrokerRequest);
+    }
+    return getBrokerResponse(queryContext, serverQueryContext, planMaker);
   }
 
   /**
@@ -206,12 +214,13 @@ public abstract class BaseQueriesTest {
    * <p>Use this to test the whole flow from server to broker.
    * <p>The result should be equivalent to querying 4 identical index segments.
    */
-  private BrokerResponseNative getBrokerResponse(QueryContext queryContext, PlanMaker planMaker) {
+  private BrokerResponseNative getBrokerResponse(
+      QueryContext queryContext, QueryContext serverQueryContext, PlanMaker planMaker) {
     // Server side.
-    queryContext.setEndTimeMs(System.currentTimeMillis() + Server.DEFAULT_QUERY_EXECUTOR_TIMEOUT_MS);
-    Plan plan = planMaker.makeInstancePlan(getIndexSegments(), queryContext, EXECUTOR_SERVICE);
+    serverQueryContext.setEndTimeMs(System.currentTimeMillis() + Server.DEFAULT_QUERY_EXECUTOR_TIMEOUT_MS);
+    Plan plan = planMaker.makeInstancePlan(getIndexSegments(), serverQueryContext, EXECUTOR_SERVICE);
 
-    BrokerRequest brokerRequest = queryContext.getBrokerRequest();
+    BrokerRequest brokerRequest = serverQueryContext.getBrokerRequest();
     DataTable instanceResponse =
         brokerRequest != null && brokerRequest.getPinotQuery() != null && brokerRequest.getPinotQuery().isExplain()
             ? ServerQueryExecutorV1Impl.processExplainPlanQueries(plan) : plan.execute();
@@ -235,8 +244,8 @@ public abstract class BaseQueriesTest {
     }
 
     BrokerResponseNative brokerResponse =
-        brokerReduceService.reduceOnDataTable(queryContext.getBrokerRequest(), dataTableMap,
-            CommonConstants.Broker.DEFAULT_BROKER_TIMEOUT_MS, null);
+        brokerReduceService.reduceOnDataTable(queryContext.getBrokerRequest(), serverQueryContext.getBrokerRequest(),
+            dataTableMap, CommonConstants.Broker.DEFAULT_BROKER_TIMEOUT_MS, null);
     brokerReduceService.shutDown();
     return brokerResponse;
   }
@@ -262,7 +271,9 @@ public abstract class BaseQueriesTest {
     }
     queryOptions.put(Request.QueryOptionKey.GROUP_BY_MODE, Request.SQL);
     queryOptions.put(Request.QueryOptionKey.RESPONSE_FORMAT, Request.SQL);
+    BrokerRequest serverBrokerRequest = GapfillUtils.stripGapfill(brokerRequest);
     QueryContext queryContext = BrokerRequestToQueryContextConverter.convert(brokerRequest);
-    return getBrokerResponse(queryContext, planMaker);
+    QueryContext serverQueryContext = BrokerRequestToQueryContextConverter.convert(serverBrokerRequest);
+    return getBrokerResponse(queryContext, serverQueryContext, planMaker);
   }
 }
diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/DistinctQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/DistinctQueriesTest.java
index 8a9f813..0994481 100644
--- a/pinot-core/src/test/java/org/apache/pinot/queries/DistinctQueriesTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/queries/DistinctQueriesTest.java
@@ -1053,8 +1053,8 @@ public class DistinctQueriesTest extends BaseQueriesTest {
     dataTableMap.put(new ServerRoutingInstance("localhost", 1234, TableType.OFFLINE), instanceResponse0);
     dataTableMap.put(new ServerRoutingInstance("localhost", 1234, TableType.REALTIME), instanceResponse1);
     BrokerResponseNative brokerResponse =
-        brokerReduceService.reduceOnDataTable(queryContext.getBrokerRequest(), dataTableMap,
-            CommonConstants.Broker.DEFAULT_BROKER_TIMEOUT_MS, null);
+        brokerReduceService.reduceOnDataTable(queryContext.getBrokerRequest(), queryContext.getBrokerRequest(),
+            dataTableMap, CommonConstants.Broker.DEFAULT_BROKER_TIMEOUT_MS, null);
     brokerReduceService.shutDown();
     return brokerResponse;
   }
diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/GapfillQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/GapfillQueriesTest.java
new file mode 100644
index 0000000..bf0a28b
--- /dev/null
+++ b/pinot-core/src/test/java/org/apache/pinot/queries/GapfillQueriesTest.java
@@ -0,0 +1,3699 @@
+/**
+ * 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.List;
+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 Gapfill queries.
+ */
+// TODO: Item 1. table alias for subquery in next PR
+// TODO: Item 2. Deprecate PostAggregateGapfill implementation in next PR
+@SuppressWarnings("rawtypes")
+public class GapfillQueriesTest 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 int NUM_LOTS = 4;
+
+  private static final String IS_OCCUPIED_COLUMN = "isOccupied";
+  private static final String LEVEL_ID_COLUMN = "levelId";
+  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.INT)
+          .addSingleValueDimension(LOT_ID_COLUMN, DataType.STRING)
+          .addSingleValueDimension(LEVEL_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;
+  }
+
+  GenericRow createRow(String time, int levelId, int lotId, boolean isOccupied) {
+    DateTimeFormatSpec dateTimeFormatter =
+        new DateTimeFormatSpec("1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS");
+    GenericRow parkingRow = new GenericRow();
+    parkingRow.putValue(EVENT_TIME_COLUMN, dateTimeFormatter.fromFormatToMillis(time));
+    parkingRow.putValue(LEVEL_ID_COLUMN, "Level_" + levelId);
+    parkingRow.putValue(LOT_ID_COLUMN, "LotId_" + lotId);
+    parkingRow.putValue(IS_OCCUPIED_COLUMN, isOccupied);
+    return parkingRow;
+  }
+
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+
+    List<GenericRow> records = new ArrayList<>(NUM_LOTS * 2);
+    records.add(createRow("2021-11-07 04:11:00.000", 0, 0, true));
+    records.add(createRow("2021-11-07 04:21:00.000", 0, 0, true));
+    records.add(createRow("2021-11-07 04:31:00.000", 1, 0, true));
+    records.add(createRow("2021-11-07 05:17:00.000", 0, 1, true));
+    records.add(createRow("2021-11-07 05:37:00.000", 0, 1, true));
+    records.add(createRow("2021-11-07 05:47:00.000", 1, 2, true));
+    records.add(createRow("2021-11-07 06:25:00.000", 0, 2, true));
+    records.add(createRow("2021-11-07 06:35:00.000", 0, 2, true));
+    records.add(createRow("2021-11-07 06:36:00.000", 1, 1, true));
+    records.add(createRow("2021-11-07 07:44:00.000", 0, 3, true));
+    records.add(createRow("2021-11-07 07:46:00.000", 1, 3, true));
+    records.add(createRow("2021-11-07 07:54:00.000", 1, 3, true));
+    records.add(createRow("2021-11-07 08:44:00.000", 0, 2, false));
+    records.add(createRow("2021-11-07 08:44:00.000", 1, 2, false));
+    records.add(createRow("2021-11-07 09:31:00.000", 0, 3, false));
+    records.add(createRow("2021-11-07 09:31:00.000", 1, 3, false));
+    records.add(createRow("2021-11-07 10:17:00.000", 0, 0, false));
+    records.add(createRow("2021-11-07 10:33:00.000", 0, 0, false));
+    records.add(createRow("2021-11-07 10:33:00.000", 1, 0, false));
+    records.add(createRow("2021-11-07 11:54:00.000", 0, 1, false));
+    records.add(createRow("2021-11-07 11:57:00.000", 1, 1, false));
+
+    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 datetimeconvertGapfillTestSelectSelect() {
+    DateTimeFormatSpec dateTimeFormatter =
+        new DateTimeFormatSpec("1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT GapFill(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 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     levelId, lotId, isOccupied "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    int[][] expectedOccupiedSlotsCounts1 =
+        new int[][]{{6, 6}, {8, 4}, {10, 2}, {12, 0}, {6, 4}, {4, 6}, {2, 10}, {0, 10}};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    int index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      int ones = expectedOccupiedSlotsCounts1[i][0];
+      int zeros = expectedOccupiedSlotsCounts1[i][1];
+      int total = ones + zeros;
+      for (int k = 0; k < total; k++) {
+        String firstTimeCol = (String) gapFillRows1.get(index)[0];
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        if (gapFillRows1.get(index)[3].equals(1)) {
+          ones--;
+        } else {
+          zeros--;
+        }
+        index++;
+      }
+      Assert.assertEquals(ones, 0);
+      Assert.assertEquals(zeros, 0);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows1.size(), index);
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, levelId, lotId, isOccupied, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(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 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE isOccupied = 1 "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    int[] expectedOccupiedSlotsCounts2 = new int[]{6, 8, 10, 12, 6, 4, 2, 0};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      for (int k = 0; k < expectedOccupiedSlotsCounts2[i]; k++) {
+        String firstTimeCol = (String) gapFillRows2.get(index)[0];
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        Assert.assertEquals(gapFillRows2.get(index)[3], 1);
+        index++;
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows2.size(), index);
+  }
+
+  @Test
+  public void datetimeconvertGapfillTestAggregateSelect() {
+    DateTimeFormatSpec dateTimeFormatter =
+        new DateTimeFormatSpec("1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "GapFill(time_col, "
+        + "    '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+        + "    '2021-11-07 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "    FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)), levelId, lotId, occupied "
+        + "FROM ("
+        + "  SELECT DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "    '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS') AS time_col,"
+        + "     lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  GROUP BY time_col, levelId, lotId "
+        + "  LIMIT 200 "
+        + ") "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    int[][] expectedOccupiedSlotsCounts1 = new int[][]{{2, 6}, {4, 4}, {6, 2}, {8, 0}, {6, 2}, {4, 4}, {2, 6}, {0, 8}};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    int index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      int ones = expectedOccupiedSlotsCounts1[i][0];
+      int zeros = expectedOccupiedSlotsCounts1[i][1];
+      int total = ones + zeros;
+      for (int k = 0; k < total; k++) {
+        String firstTimeCol = (String) gapFillRows1.get(index)[0];
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        if (gapFillRows1.get(index)[3].equals(1)) {
+          ones--;
+        } else {
+          zeros--;
+        }
+        index++;
+      }
+      Assert.assertEquals(ones, 0);
+      Assert.assertEquals(zeros, 0);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows1.size(), index);
+
+    String gapfillQuery2 = "SELECT "
+        + "GapFill(time_col, "
+        + "    '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+        + "    '2021-11-07 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "    FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)), levelId, lotId, occupied "
+        + "FROM ("
+        + "  SELECT DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "    '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS') AS time_col,"
+        + "     lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  GROUP BY time_col, levelId, lotId "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE occupied = 1 "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    int[] expectedOccupiedSlotsCounts2 = new int[]{2, 4, 6, 8, 6, 4, 2};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      for (int k = 0; k < expectedOccupiedSlotsCounts2[i]; k++) {
+        String firstTimeCol = (String) gapFillRows2.get(index)[0];
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        Assert.assertEquals(gapFillRows2.get(index)[3], 1);
+        index++;
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows2.size(), index);
+  }
+
+  @Test
+  public void datetimeconvertGapfillTestGapfillAggregate() {
+    DateTimeFormatSpec dateTimeFormatter =
+        new DateTimeFormatSpec("1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    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, "
+        + "SUM(isOccupied) "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    BrokerResponseNative dateTimeConvertBrokerResponse = getBrokerResponseForSqlQuery(dataTimeConvertQuery);
+
+    ResultTable dateTimeConvertResultTable = dateTimeConvertBrokerResponse.getResultTable();
+    Assert.assertEquals(dateTimeConvertResultTable.getRows().size(), 8);
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT  GapFill(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 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{6, 8, 10, 12, 6, 4, 2, 0};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length);
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      String firstTimeCol = (String) gapFillRows1.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i], gapFillRows1.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT  GapFill(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 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE isOccupied = 1 "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCounts2 = new double[]{6, 8, 10, 12, 6, 4, 2};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCounts2.length);
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      String firstTimeCol = (String) gapFillRows2.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts2[i], gapFillRows2.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void datetimeconvertGapfillTestGapfillAggregateWithOptionalGroupBy() {
+    DateTimeFormatSpec dateTimeFormatter =
+        new DateTimeFormatSpec("1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, levelId, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT  GapFill(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 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col, levelId "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCountsForLevel11 = new double[]{4, 5, 6, 5, 3, 2, 1, 0};
+    double[] expectedOccupiedSlotsCountsForLevel21 = new double[]{2, 3, 4, 7, 3, 2, 1, 0};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCountsForLevel11.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    for (int i = 0; i < expectedOccupiedSlotsCountsForLevel11.length * 2; i += 2) {
+      String firstTimeCol = (String) gapFillRows1.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows1.get(i)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel11[i / 2], gapFillRows1.get(i)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows1.get(i)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel21[i / 2], gapFillRows1.get(i)[2]);
+      }
+      firstTimeCol = (String) gapFillRows1.get(i + 1)[0];
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows1.get(i + 1)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel11[i / 2], gapFillRows1.get(i + 1)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows1.get(i + 1)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel21[i / 2], gapFillRows1.get(i + 1)[2]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, levelId, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT  GapFill(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 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE isOccupied = 1 "
+        + " GROUP BY time_col, levelId "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCountsForLevel12 = new double[]{4, 5, 6, 5, 3, 2, 1};
+    double[] expectedOccupiedSlotsCountsForLevel22 = new double[]{2, 3, 4, 7, 3, 2, 1};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCountsForLevel12.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    for (int i = 0; i < expectedOccupiedSlotsCountsForLevel12.length * 2; i += 2) {
+      String firstTimeCol = (String) gapFillRows2.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows2.get(i)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel12[i / 2], gapFillRows2.get(i)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows2.get(i)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel22[i / 2], gapFillRows2.get(i)[2]);
+      }
+      firstTimeCol = (String) gapFillRows2.get(i + 1)[0];
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows2.get(i + 1)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel12[i / 2], gapFillRows2.get(i + 1)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows2.get(i + 1)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel22[i / 2], gapFillRows2.get(i + 1)[2]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void datetimeconvertGapfillTestGapfillAggregateWithHavingClause() {
+    DateTimeFormatSpec dateTimeFormatter =
+        new DateTimeFormatSpec("1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, levelId, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT  GapFill(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 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col, levelId "
+        + " HAVING occupied_slots_count > 0"
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCountsForLevel11 = new double[]{4, 5, 6, 5, 3, 2, 1};
+    double[] expectedOccupiedSlotsCountsForLevel21 = new double[]{2, 3, 4, 7, 3, 2, 1};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCountsForLevel11.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    for (int i = 0; i < expectedOccupiedSlotsCountsForLevel11.length * 2; i += 2) {
+      String firstTimeCol = (String) gapFillRows1.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows1.get(i)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel11[i / 2], gapFillRows1.get(i)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows1.get(i)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel21[i / 2], gapFillRows1.get(i)[2]);
+      }
+      firstTimeCol = (String) gapFillRows1.get(i + 1)[0];
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows1.get(i + 1)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel11[i / 2], gapFillRows1.get(i + 1)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows1.get(i + 1)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel21[i / 2], gapFillRows1.get(i + 1)[2]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void datetimeconvertGapfillTestAggregateAggregate() {
+    DateTimeFormatSpec dateTimeFormatter =
+        new DateTimeFormatSpec("1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, "
+        + "    '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+        + "    '2021-11-07 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "      '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS') AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{2, 4, 6, 8, 6, 4, 2, 0};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length);
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      String firstTimeCol = (String) gapFillRows1.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i], gapFillRows1.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, "
+        + "    '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+        + "    '2021-11-07 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "      '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS') AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE occupied = 1 "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCounts2 = new double[]{2, 4, 6, 8, 6, 4, 2};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCounts2.length);
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      String firstTimeCol = (String) gapFillRows2.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts2[i], gapFillRows2.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void datetimeconvertGapfillTestAggregateAggregateWithOptionalGroupBy() {
+    DateTimeFormatSpec dateTimeFormatter =
+        new DateTimeFormatSpec("1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, levelId, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, "
+        + "    '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+        + "    '2021-11-07 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "      '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS') AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col, levelId "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{1, 2, 3, 4, 3, 2, 1, 0};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i += 2) {
+      String firstTimeCol = (String) gapFillRows1.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      firstTimeCol = (String) gapFillRows1.get(i + 1)[0];
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, levelId, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, "
+        + "    '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+        + "    '2021-11-07 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "      '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS') AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE occupied = 1 "
+        + " GROUP BY time_col, levelId "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCounts2 = new double[]{1, 2, 3, 4, 3, 2, 1};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCounts2.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i += 2) {
+      String firstTimeCol = (String) gapFillRows2.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      firstTimeCol = (String) gapFillRows2.get(i + 1)[0];
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void datetimeconvertGapfillTestAggregateAggregateWithHavingClause() {
+    String gapfillQuery1 = "SELECT "
+        + "time_col, levelId, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, "
+        + "    '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+        + "    '2021-11-07 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)),"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "      '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS') AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col, levelId "
+        + " HAVING occupied_slots_count > 0"
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{1, 2, 3, 4, 3, 2, 1};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length * 2);
+    DateTimeFormatSpec dateTimeFormatter =
+        new DateTimeFormatSpec("1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+
+    long start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i += 2) {
+      String firstTimeCol = (String) gapFillRows1.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      firstTimeCol = (String) gapFillRows1.get(i + 1)[0];
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochHoursGapfillTestSelectSelect() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "  GapFill(ToEpochHours(eventTime), '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    int[][] expectedOccupiedSlotsCounts1 =
+        new int[][]{{6, 6}, {8, 4}, {10, 2}, {12, 0}, {6, 4}, {4, 6}, {2, 10}, {0, 10}};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    int index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      int ones = expectedOccupiedSlotsCounts1[i][0];
+      int zeros = expectedOccupiedSlotsCounts1[i][1];
+      int total = ones + zeros;
+      for (int k = 0; k < total; k++) {
+        String firstTimeCol = ((Long) (gapFillRows1.get(index)[0])).toString();
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        if (gapFillRows1.get(index)[1].equals(1)) {
+          ones--;
+        } else {
+          zeros--;
+        }
+        index++;
+      }
+      Assert.assertEquals(ones, 0);
+      Assert.assertEquals(zeros, 0);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows1.size(), index);
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, levelId, lotId, isOccupied, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(ToEpochHours(eventTime), '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE isOccupied = 1 "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    int[] expectedOccupiedSlotsCounts2 = new int[]{6, 8, 10, 12, 6, 4, 2, 0};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      for (int k = 0; k < expectedOccupiedSlotsCounts2[i]; k++) {
+        String firstTimeCol = ((Long) gapFillRows2.get(index)[0]).toString();
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        Assert.assertEquals(gapFillRows2.get(index)[3], 1);
+        index++;
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows2.size(), index);
+  }
+
+  @Test
+  public void toEpochHoursGapfillTestAggregateSelect() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "  GapFill(time_col, '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)),"
+        + "  levelId, lotId, occupied "
+        + "FROM ("
+        + "  SELECT ToEpochHours(eventTime) AS time_col,"
+        + "     lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  GROUP BY time_col, levelId, lotId "
+        + "  LIMIT 200 "
+        + ") "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    int[][] expectedOccupiedSlotsCounts1 = new int[][]{{2, 6}, {4, 4}, {6, 2}, {8, 0}, {6, 2}, {4, 4}, {2, 6}, {0, 8}};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    int index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      int ones = expectedOccupiedSlotsCounts1[i][0];
+      int zeros = expectedOccupiedSlotsCounts1[i][1];
+      int total = ones + zeros;
+      for (int k = 0; k < total; k++) {
+        String firstTimeCol = ((Long) gapFillRows1.get(index)[0]).toString();
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        if (gapFillRows1.get(index)[3].equals(1)) {
+          ones--;
+        } else {
+          zeros--;
+        }
+        index++;
+      }
+      Assert.assertEquals(ones, 0);
+      Assert.assertEquals(zeros, 0);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows1.size(), index);
+
+    String gapfillQuery2 = "SELECT "
+        + "  GapFill(time_col, '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)),"
+        + " levelId, lotId, occupied "
+        + "FROM ("
+        + "  SELECT ToEpochHours(eventTime) AS time_col,"
+        + "     lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  GROUP BY time_col, levelId, lotId "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE occupied = 1 "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    int[] expectedOccupiedSlotsCounts2 = new int[]{2, 4, 6, 8, 6, 4, 2};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      for (int k = 0; k < expectedOccupiedSlotsCounts2[i]; k++) {
+        String firstTimeCol = ((Long) gapFillRows2.get(index)[0]).toString();
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        Assert.assertEquals(gapFillRows2.get(index)[3], 1);
+        index++;
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows2.size(), index);
+  }
+
+  @Test
+  public void toEpochHoursGapfillTestGapfillAggregate() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(ToEpochHours(eventTime), '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "    isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{6, 8, 10, 12, 6, 4, 2, 0};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length);
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i], gapFillRows1.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(ToEpochHours(eventTime), '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE isOccupied = 1 "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCounts2 = new double[]{6, 8, 10, 12, 6, 4, 2};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCounts2.length);
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      String firstTimeCol = ((Long) gapFillRows2.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts2[i], gapFillRows2.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochHoursGapfillTestGapfillAggregateWithOptionalGroupBy() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, levelId, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(ToEpochHours(eventTime), '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col, levelId "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCountsForLevel11 = new double[]{4, 5, 6, 5, 3, 2, 1, 0};
+    double[] expectedOccupiedSlotsCountsForLevel21 = new double[]{2, 3, 4, 7, 3, 2, 1, 0};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCountsForLevel11.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    for (int i = 0; i < expectedOccupiedSlotsCountsForLevel11.length * 2; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows1.get(i)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel11[i / 2], gapFillRows1.get(i)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows1.get(i)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel21[i / 2], gapFillRows1.get(i)[2]);
+      }
+      firstTimeCol = ((Long) gapFillRows1.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows1.get(i + 1)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel11[i / 2], gapFillRows1.get(i + 1)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows1.get(i + 1)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel21[i / 2], gapFillRows1.get(i + 1)[2]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, levelId, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(ToEpochHours(eventTime), '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE isOccupied = 1 "
+        + " GROUP BY time_col, levelId "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCountsForLevel12 = new double[]{4, 5, 6, 5, 3, 2, 1};
+    double[] expectedOccupiedSlotsCountsForLevel22 = new double[]{2, 3, 4, 7, 3, 2, 1};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCountsForLevel12.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    for (int i = 0; i < expectedOccupiedSlotsCountsForLevel12.length * 2; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows2.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows2.get(i)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel12[i / 2], gapFillRows2.get(i)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows2.get(i)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel22[i / 2], gapFillRows2.get(i)[2]);
+      }
+      firstTimeCol = ((Long) gapFillRows2.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows2.get(i + 1)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel12[i / 2], gapFillRows2.get(i + 1)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows2.get(i + 1)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel22[i / 2], gapFillRows2.get(i + 1)[2]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochHoursGapfillTestGapfillAggregateWithHavingClause() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, levelId, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(ToEpochHours(eventTime), '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "    isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col, levelId "
+        + " HAVING occupied_slots_count > 0"
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCountsForLevel11 = new double[]{4, 5, 6, 5, 3, 2, 1};
+    double[] expectedOccupiedSlotsCountsForLevel21 = new double[]{2, 3, 4, 7, 3, 2, 1};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCountsForLevel11.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    for (int i = 0; i < expectedOccupiedSlotsCountsForLevel11.length * 2; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows1.get(i)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel11[i / 2], gapFillRows1.get(i)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows1.get(i)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel21[i / 2], gapFillRows1.get(i)[2]);
+      }
+      firstTimeCol = ((Long) gapFillRows1.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows1.get(i + 1)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel11[i / 2], gapFillRows1.get(i + 1)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows1.get(i + 1)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel21[i / 2], gapFillRows1.get(i + 1)[2]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochHoursGapfillTestAggregateAggregate() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)),"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT ToEpochHours(eventTime) AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{2, 4, 6, 8, 6, 4, 2, 0};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length);
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i], gapFillRows1.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT ToEpochHours(eventTime) AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE occupied = 1 "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCounts2 = new double[]{2, 4, 6, 8, 6, 4, 2};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCounts2.length);
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts2[i], gapFillRows2.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochHoursGapfillTestAggregateAggregateWithOptionalGroupBy() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, levelId, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + " FROM ("
+        + "  SELECT ToEpochHours(eventTime) AS time_col,"
+        + "     lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  GROUP BY time_col, levelId, lotId "
+        + "  LIMIT 200 "
+        + ") "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col, levelId "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{1, 2, 3, 4, 3, 2, 1, 0};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      firstTimeCol = ((Long) gapFillRows1.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT ToEpochHours(eventTime) AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE occupied = 1 "
+        + " GROUP BY time_col, levelId "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCounts2 = new double[]{1, 2, 3, 4, 3, 2, 1};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCounts2.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows2.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      firstTimeCol = ((Long) gapFillRows2.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochHoursGapfillTestAggregateAggregateWithHavingClause() {
+    String gapfillQuery1 = "SELECT "
+        + "time_col, levelId, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT ToEpochHours(eventTime) AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + ") "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col, levelId "
+        + " HAVING occupied_slots_count > 0"
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{1, 2, 3, 4, 3, 2, 1};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length * 2);
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+
+    long start = dateTimeFormatter.fromFormatToMillis("454516");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      firstTimeCol = ((Long) gapFillRows1.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochMinutesRoundedHoursGapfillTestSelectSelect() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:MINUTES:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT GapFill(ToEpochMinutesRounded(eventTime, 60), '1:MINUTES:EPOCH', "
+        + "    '27270960',  '27271440', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    int[][] expectedOccupiedSlotsCounts1 =
+        new int[][]{{6, 6}, {8, 4}, {10, 2}, {12, 0}, {6, 4}, {4, 6}, {2, 10}, {0, 10}};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("27270960");
+    int index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      int ones = expectedOccupiedSlotsCounts1[i][0];
+      int zeros = expectedOccupiedSlotsCounts1[i][1];
+      int total = ones + zeros;
+      for (int k = 0; k < total; k++) {
+        String firstTimeCol = ((Long) (gapFillRows1.get(index)[0])).toString();
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        if (gapFillRows1.get(index)[1].equals(1)) {
+          ones--;
+        } else {
+          zeros--;
+        }
+        index++;
+      }
+      Assert.assertEquals(ones, 0);
+      Assert.assertEquals(zeros, 0);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows1.size(), index);
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, levelId, lotId, isOccupied, time_col "
+        + "FROM ("
+        + "  SELECT  GapFill(ToEpochMinutesRounded(eventTime, 60), '1:MINUTES:EPOCH', "
+        + "    '27270960',  '27271440', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE isOccupied = 1 "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    int[] expectedOccupiedSlotsCounts2 = new int[]{6, 8, 10, 12, 6, 4, 2, 0};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("27270960");
+    index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      for (int k = 0; k < expectedOccupiedSlotsCounts2[i]; k++) {
+        String firstTimeCol = ((Long) gapFillRows2.get(index)[0]).toString();
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        Assert.assertEquals(gapFillRows2.get(index)[3], 1);
+        index++;
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows2.size(), index);
+  }
+
+  @Test
+  public void toEpochMinutesRoundedHoursGapfillTestAggregateSelect() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:MINUTES:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "GapFill(time_col, '1:MINUTES:EPOCH', "
+        + "   '27270960',  '27271440', '1:HOURS',"
+        + "   FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)), levelId, lotId, occupied "
+        + "FROM ("
+        + "  SELECT ToEpochMinutesRounded(eventTime, 60) AS time_col,"
+        + "     lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  GROUP BY time_col, levelId, lotId "
+        + "  LIMIT 200 "
+        + ") "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    int[][] expectedOccupiedSlotsCounts1 = new int[][]{{2, 6}, {4, 4}, {6, 2}, {8, 0}, {6, 2}, {4, 4}, {2, 6}, {0, 8}};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("27270960");
+    int index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      int ones = expectedOccupiedSlotsCounts1[i][0];
+      int zeros = expectedOccupiedSlotsCounts1[i][1];
+      int total = ones + zeros;
+      for (int k = 0; k < total; k++) {
+        String firstTimeCol = ((Long) gapFillRows1.get(index)[0]).toString();
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        if (gapFillRows1.get(index)[3].equals(1)) {
+          ones--;
+        } else {
+          zeros--;
+        }
+        index++;
+      }
+      Assert.assertEquals(ones, 0);
+      Assert.assertEquals(zeros, 0);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows1.size(), index);
+
+    String gapfillQuery2 = "SELECT "
+        + "GapFill(time_col, '1:MINUTES:EPOCH', "
+        + "   '27270960',  '27271440', '1:HOURS',"
+        + "   FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)), levelId, lotId, occupied "
+        + "FROM ("
+        + "  SELECT  ToEpochMinutesRounded(eventTime, 60) AS time_col,"
+        + "     lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  GROUP BY time_col, levelId, lotId "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE occupied = 1 "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    int[] expectedOccupiedSlotsCounts2 = new int[]{2, 4, 6, 8, 6, 4, 2};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("27270960");
+    index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      for (int k = 0; k < expectedOccupiedSlotsCounts2[i]; k++) {
+        String firstTimeCol = ((Long) gapFillRows2.get(index)[0]).toString();
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        Assert.assertEquals(gapFillRows2.get(index)[3], 1);
+        index++;
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows2.size(), index);
+  }
+
+  @Test
+  public void toEpochMinutesRoundedHoursGapfillTestGapfillAggregate() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:MINUTES:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT  GapFill(ToEpochMinutesRounded(eventTime, 60), '1:MINUTES:EPOCH', "
+        + "    '27270960',  '27271440', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{6, 8, 10, 12, 6, 4, 2, 0};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length);
+    start = dateTimeFormatter.fromFormatToMillis("27270960");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i], gapFillRows1.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT  GapFill(ToEpochMinutesRounded(eventTime, 60), '1:MINUTES:EPOCH', "
+        + "    '27270960',  '27271440', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE isOccupied = 1 "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCounts2 = new double[]{6, 8, 10, 12, 6, 4, 2};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCounts2.length);
+    start = dateTimeFormatter.fromFormatToMillis("27270960");
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      String firstTimeCol = ((Long) gapFillRows2.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts2[i], gapFillRows2.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochMinutesRoundedHoursGapfillTestGapfillAggregateWithOptionalGroupBy() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:MINUTES:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, levelId, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT  GapFill(ToEpochMinutesRounded(eventTime, 60), '1:MINUTES:EPOCH', "
+        + "    '27270960',  '27271440', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col, levelId "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCountsForLevel11 = new double[]{4, 5, 6, 5, 3, 2, 1, 0};
+    double[] expectedOccupiedSlotsCountsForLevel21 = new double[]{2, 3, 4, 7, 3, 2, 1, 0};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCountsForLevel11.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("27270960");
+    for (int i = 0; i < expectedOccupiedSlotsCountsForLevel11.length * 2; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows1.get(i)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel11[i / 2], gapFillRows1.get(i)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows1.get(i)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel21[i / 2], gapFillRows1.get(i)[2]);
+      }
+      firstTimeCol = ((Long) gapFillRows1.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows1.get(i + 1)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel11[i / 2], gapFillRows1.get(i + 1)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows1.get(i + 1)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel21[i / 2], gapFillRows1.get(i + 1)[2]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, levelId, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT  GapFill(ToEpochMinutesRounded(eventTime, 60), '1:MINUTES:EPOCH', "
+        + "    '27270960',  '27271440', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE isOccupied = 1 "
+        + " GROUP BY time_col, levelId "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCountsForLevel12 = new double[]{4, 5, 6, 5, 3, 2, 1};
+    double[] expectedOccupiedSlotsCountsForLevel22 = new double[]{2, 3, 4, 7, 3, 2, 1};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCountsForLevel12.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("27270960");
+    for (int i = 0; i < expectedOccupiedSlotsCountsForLevel12.length * 2; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows2.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows2.get(i)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel12[i / 2], gapFillRows2.get(i)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows2.get(i)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel22[i / 2], gapFillRows2.get(i)[2]);
+      }
+      firstTimeCol = ((Long) gapFillRows2.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows2.get(i + 1)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel12[i / 2], gapFillRows2.get(i + 1)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows2.get(i + 1)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel22[i / 2], gapFillRows2.get(i + 1)[2]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochMinutesRoundedHoursGapfillTestGapfillAggregateWithHavingClause() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:MINUTES:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, levelId, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT  GapFill(ToEpochMinutesRounded(eventTime, 60), '1:MINUTES:EPOCH', "
+        + "    '27270960',  '27271440', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col, levelId "
+        + " HAVING occupied_slots_count > 0"
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCountsForLevel11 = new double[]{4, 5, 6, 5, 3, 2, 1};
+    double[] expectedOccupiedSlotsCountsForLevel21 = new double[]{2, 3, 4, 7, 3, 2, 1};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCountsForLevel11.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("27270960");
+    for (int i = 0; i < expectedOccupiedSlotsCountsForLevel11.length * 2; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows1.get(i)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel11[i / 2], gapFillRows1.get(i)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows1.get(i)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel21[i / 2], gapFillRows1.get(i)[2]);
+      }
+      firstTimeCol = ((Long) gapFillRows1.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows1.get(i + 1)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel11[i / 2], gapFillRows1.get(i + 1)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows1.get(i + 1)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel21[i / 2], gapFillRows1.get(i + 1)[2]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochMinutesRoundedHoursGapfillTestAggregateAggregate() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:MINUTES:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT  GapFill(time_col, '1:MINUTES:EPOCH', "
+        + "    '27270960',  '27271440', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT ToEpochMinutesRounded(eventTime, 60) AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{2, 4, 6, 8, 6, 4, 2, 0};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length);
+    start = dateTimeFormatter.fromFormatToMillis("27270960");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i], gapFillRows1.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT  GapFill(time_col, '1:MINUTES:EPOCH', "
+        + "    '27270960',  '27271440', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT ToEpochMinutesRounded(eventTime, 60) time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE occupied = 1 "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCounts2 = new double[]{2, 4, 6, 8, 6, 4, 2};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCounts2.length);
+    start = dateTimeFormatter.fromFormatToMillis("27270960");
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts2[i], gapFillRows2.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochMinutesRoundedHoursGapfillTestAggregateAggregateWithOptionalGroupBy() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:MINUTES:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, levelId, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT  GapFill(time_col, '1:MINUTES:EPOCH', "
+        + "    '27270960',  '27271440', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT ToEpochMinutesRounded(eventTime, 60) AS time_col,"
+        + "     lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col, levelId "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{1, 2, 3, 4, 3, 2, 1, 0};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("27270960");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      firstTimeCol = ((Long) gapFillRows1.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT  GapFill(time_col, '1:MINUTES:EPOCH', "
+        + "    '27270960',  '27271440', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT ToEpochMinutesRounded(eventTime, 60) AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE occupied = 1 "
+        + " GROUP BY time_col, levelId "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCounts2 = new double[]{1, 2, 3, 4, 3, 2, 1};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCounts2.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("27270960");
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows2.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      firstTimeCol = ((Long) gapFillRows2.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochMinutesRoundedHoursGapfillTestAggregateAggregateWithHavingClause() {
+    String gapfillQuery1 = "SELECT "
+        + "time_col, levelId, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT  GapFill(time_col, '1:MINUTES:EPOCH', "
+        + "    '27270960',  '27271440', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT ToEpochMinutesRounded(eventTime, 60) time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col, levelId "
+        + " HAVING occupied_slots_count > 0"
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{1, 2, 3, 4, 3, 2, 1};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length * 2);
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:MINUTES:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+
+    long start = dateTimeFormatter.fromFormatToMillis("27270960");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      firstTimeCol = ((Long) gapFillRows1.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochMinutesBucketHoursGapfillTestSelectSelect() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "  GapFill(ToEpochMinutesBucket(eventTime, 60), '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    int[][] expectedOccupiedSlotsCounts1 =
+        new int[][]{{6, 6}, {8, 4}, {10, 2}, {12, 0}, {6, 4}, {4, 6}, {2, 10}, {0, 10}};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    int index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      int ones = expectedOccupiedSlotsCounts1[i][0];
+      int zeros = expectedOccupiedSlotsCounts1[i][1];
+      int total = ones + zeros;
+      for (int k = 0; k < total; k++) {
+        String firstTimeCol = ((Long) (gapFillRows1.get(index)[0])).toString();
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        if (gapFillRows1.get(index)[1].equals(1)) {
+          ones--;
+        } else {
+          zeros--;
+        }
+        index++;
+      }
+      Assert.assertEquals(ones, 0);
+      Assert.assertEquals(zeros, 0);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows1.size(), index);
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, levelId, lotId, isOccupied, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(ToEpochMinutesBucket(eventTime, 60), '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE isOccupied = 1 "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    int[] expectedOccupiedSlotsCounts2 = new int[]{6, 8, 10, 12, 6, 4, 2, 0};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      for (int k = 0; k < expectedOccupiedSlotsCounts2[i]; k++) {
+        String firstTimeCol = ((Long) gapFillRows2.get(index)[0]).toString();
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        Assert.assertEquals(gapFillRows2.get(index)[3], 1);
+        index++;
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows2.size(), index);
+  }
+
+  @Test
+  public void toEpochMinutesBucketHoursGapfillTestAggregateSelect() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "  GapFill(time_col, '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)),"
+        + " levelId, lotId, occupied "
+        + "FROM ("
+        + "  SELECT ToEpochMinutesBucket(eventTime, 60) AS time_col,"
+        + "     lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  GROUP BY time_col, levelId, lotId "
+        + "  LIMIT 200 "
+        + ") "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    int[][] expectedOccupiedSlotsCounts1 = new int[][]{{2, 6}, {4, 4}, {6, 2}, {8, 0}, {6, 2}, {4, 4}, {2, 6}, {0, 8}};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    int index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      int ones = expectedOccupiedSlotsCounts1[i][0];
+      int zeros = expectedOccupiedSlotsCounts1[i][1];
+      int total = ones + zeros;
+      for (int k = 0; k < total; k++) {
+        String firstTimeCol = ((Long) gapFillRows1.get(index)[0]).toString();
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        if (gapFillRows1.get(index)[3].equals(1)) {
+          ones--;
+        } else {
+          zeros--;
+        }
+        index++;
+      }
+      Assert.assertEquals(ones, 0);
+      Assert.assertEquals(zeros, 0);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows1.size(), index);
+
+    String gapfillQuery2 = "SELECT "
+        + "  GapFill(time_col, '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)),"
+        + " levelId, lotId, occupied "
+        + "FROM ("
+        + "  SELECT ToEpochMinutesBucket(eventTime, 60) AS time_col,"
+        + "     lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  GROUP BY time_col, levelId, lotId "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE occupied = 1 "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    int[] expectedOccupiedSlotsCounts2 = new int[]{2, 4, 6, 8, 6, 4, 2};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      for (int k = 0; k < expectedOccupiedSlotsCounts2[i]; k++) {
+        String firstTimeCol = ((Long) gapFillRows2.get(index)[0]).toString();
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        Assert.assertEquals(gapFillRows2.get(index)[3], 1);
+        index++;
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows2.size(), index);
+  }
+
+  @Test
+  public void toEpochMinutesBucketHoursGapfillTestGapfillAggregate() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(ToEpochMinutesBucket(eventTime, 60), '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{6, 8, 10, 12, 6, 4, 2, 0};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length);
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i], gapFillRows1.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(ToEpochMinutesBucket(eventTime, 60), '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE isOccupied = 1 "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCounts2 = new double[]{6, 8, 10, 12, 6, 4, 2};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCounts2.length);
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      String firstTimeCol = ((Long) gapFillRows2.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts2[i], gapFillRows2.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochMinutesBucketHoursGapfillTestGapfillAggregateWithOptionalGroupBy() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, levelId, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(ToEpochMinutesBucket(eventTime, 60), '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col, levelId "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCountsForLevel11 = new double[]{4, 5, 6, 5, 3, 2, 1, 0};
+    double[] expectedOccupiedSlotsCountsForLevel21 = new double[]{2, 3, 4, 7, 3, 2, 1, 0};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCountsForLevel11.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    for (int i = 0; i < expectedOccupiedSlotsCountsForLevel11.length * 2; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows1.get(i)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel11[i / 2], gapFillRows1.get(i)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows1.get(i)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel21[i / 2], gapFillRows1.get(i)[2]);
+      }
+      firstTimeCol = ((Long) gapFillRows1.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows1.get(i + 1)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel11[i / 2], gapFillRows1.get(i + 1)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows1.get(i + 1)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel21[i / 2], gapFillRows1.get(i + 1)[2]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, levelId, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(ToEpochMinutesBucket(eventTime, 60), '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE isOccupied = 1 "
+        + " GROUP BY time_col, levelId "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCountsForLevel12 = new double[]{4, 5, 6, 5, 3, 2, 1};
+    double[] expectedOccupiedSlotsCountsForLevel22 = new double[]{2, 3, 4, 7, 3, 2, 1};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCountsForLevel12.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    for (int i = 0; i < expectedOccupiedSlotsCountsForLevel12.length * 2; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows2.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows2.get(i)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel12[i / 2], gapFillRows2.get(i)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows2.get(i)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel22[i / 2], gapFillRows2.get(i)[2]);
+      }
+      firstTimeCol = ((Long) gapFillRows2.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows2.get(i + 1)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel12[i / 2], gapFillRows2.get(i + 1)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows2.get(i + 1)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel22[i / 2], gapFillRows2.get(i + 1)[2]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochMinutesBucketHoursGapfillTestGapfillAggregateWithHavingClause() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, levelId, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(ToEpochMinutesBucket(eventTime, 60), '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col, levelId "
+        + " HAVING occupied_slots_count > 0"
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCountsForLevel11 = new double[]{4, 5, 6, 5, 3, 2, 1};
+    double[] expectedOccupiedSlotsCountsForLevel21 = new double[]{2, 3, 4, 7, 3, 2, 1};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCountsForLevel11.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    for (int i = 0; i < expectedOccupiedSlotsCountsForLevel11.length * 2; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows1.get(i)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel11[i / 2], gapFillRows1.get(i)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows1.get(i)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel21[i / 2], gapFillRows1.get(i)[2]);
+      }
+      firstTimeCol = ((Long) gapFillRows1.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows1.get(i + 1)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel11[i / 2], gapFillRows1.get(i + 1)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows1.get(i + 1)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel21[i / 2], gapFillRows1.get(i + 1)[2]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochMinutesBucketHoursGapfillTestAggregateAggregate() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT ToEpochMinutesBucket(eventTime, 60) AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{2, 4, 6, 8, 6, 4, 2, 0};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length);
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i], gapFillRows1.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT ToEpochMinutesBucket(eventTime, 60) AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE occupied = 1 "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCounts2 = new double[]{2, 4, 6, 8, 6, 4, 2};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCounts2.length);
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts2[i], gapFillRows2.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochMinutesBucketHoursGapfillTestAggregateAggregateWithOptionalGroupBy() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, levelId, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT ToEpochMinutesBucket(eventTime, 60) AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col, levelId "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{1, 2, 3, 4, 3, 2, 1, 0};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      firstTimeCol = ((Long) gapFillRows1.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "      occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT ToEpochMinutesBucket(eventTime, 60) AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE occupied = 1 "
+        + " GROUP BY time_col, levelId "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCounts2 = new double[]{1, 2, 3, 4, 3, 2, 1};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCounts2.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("454516");
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows2.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      firstTimeCol = ((Long) gapFillRows2.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void toEpochMinutesBucketHoursGapfillTestAggregateAggregateWithHavingClause() {
+    String gapfillQuery1 = "SELECT "
+        + "time_col, levelId, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, '1:HOURS:EPOCH', "
+        + "    '454516',  '454524', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT ToEpochMinutesBucket(eventTime, 60) AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col, levelId "
+        + " HAVING occupied_slots_count > 0"
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{1, 2, 3, 4, 3, 2, 1};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length * 2);
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:HOURS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+
+    long start = dateTimeFormatter.fromFormatToMillis("454516");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      firstTimeCol = ((Long) gapFillRows1.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void dateTruncHoursGapfillTestSelectSelect() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:MILLISECONDS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "  GapFill(DATETRUNC('hour', eventTime, 'milliseconds'), '1:MILLISECONDS:EPOCH', "
+        + "    '1636257600000',  '1636286400000', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    int[][] expectedOccupiedSlotsCounts1 =
+        new int[][]{{6, 6}, {8, 4}, {10, 2}, {12, 0}, {6, 4}, {4, 6}, {2, 10}, {0, 10}};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("1636257600000");
+    int index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      int ones = expectedOccupiedSlotsCounts1[i][0];
+      int zeros = expectedOccupiedSlotsCounts1[i][1];
+      int total = ones + zeros;
+      for (int k = 0; k < total; k++) {
+        String firstTimeCol = ((Long) (gapFillRows1.get(index)[0])).toString();
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        if (gapFillRows1.get(index)[1].equals(1)) {
+          ones--;
+        } else {
+          zeros--;
+        }
+        index++;
+      }
+      Assert.assertEquals(ones, 0);
+      Assert.assertEquals(zeros, 0);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows1.size(), index);
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, levelId, lotId, isOccupied, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(DATETRUNC('hour', eventTime, 'milliseconds'), '1:MILLISECONDS:EPOCH', "
+        + "    '1636257600000',  '1636286400000', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE isOccupied = 1 "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    int[] expectedOccupiedSlotsCounts2 = new int[]{6, 8, 10, 12, 6, 4, 2, 0};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("1636257600000");
+    index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      for (int k = 0; k < expectedOccupiedSlotsCounts2[i]; k++) {
+        String firstTimeCol = ((Long) gapFillRows2.get(index)[0]).toString();
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        Assert.assertEquals(gapFillRows2.get(index)[3], 1);
+        index++;
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows2.size(), index);
+  }
+
+  @Test
+  public void dateTruncHoursGapfillTestAggregateSelect() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:MILLISECONDS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "  GapFill(time_col, '1:MILLISECONDS:EPOCH', "
+        + "    '1636257600000',  '1636286400000', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)),"
+        + " levelId, lotId, occupied "
+        + "FROM ("
+        + "  SELECT DATETRUNC('hour', eventTime, 'milliseconds') AS time_col,"
+        + "     lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  GROUP BY time_col, levelId, lotId "
+        + "  LIMIT 200 "
+        + ") "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    int[][] expectedOccupiedSlotsCounts1 = new int[][]{{2, 6}, {4, 4}, {6, 2}, {8, 0}, {6, 2}, {4, 4}, {2, 6}, {0, 8}};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("1636257600000");
+    int index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      int ones = expectedOccupiedSlotsCounts1[i][0];
+      int zeros = expectedOccupiedSlotsCounts1[i][1];
+      int total = ones + zeros;
+      for (int k = 0; k < total; k++) {
+        String firstTimeCol = ((Long) gapFillRows1.get(index)[0]).toString();
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        if (gapFillRows1.get(index)[3].equals(1)) {
+          ones--;
+        } else {
+          zeros--;
+        }
+        index++;
+      }
+      Assert.assertEquals(ones, 0);
+      Assert.assertEquals(zeros, 0);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows1.size(), index);
+
+    String gapfillQuery2 = "SELECT "
+        + "  GapFill(time_col, '1:MILLISECONDS:EPOCH', "
+        + "    '1636257600000',  '1636286400000', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)),"
+        + "  levelId, lotId, occupied "
+        + "FROM ("
+        + "  SELECT DATETRUNC('hour', eventTime, 'milliseconds') AS time_col,"
+        + "     lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  GROUP BY time_col, levelId, lotId "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE occupied = 1 "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    int[] expectedOccupiedSlotsCounts2 = new int[]{2, 4, 6, 8, 6, 4, 2};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("1636257600000");
+    index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      for (int k = 0; k < expectedOccupiedSlotsCounts2[i]; k++) {
+        String firstTimeCol = ((Long) gapFillRows2.get(index)[0]).toString();
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        Assert.assertEquals(gapFillRows2.get(index)[3], 1);
+        index++;
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows2.size(), index);
+  }
+
+  @Test
+  public void dateTruncHoursGapfillTestGapfillAggregate() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:MILLISECONDS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(DATETRUNC('hour', eventTime, 'milliseconds'), '1:MILLISECONDS:EPOCH', "
+        + "    '1636257600000',  '1636286400000', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{6, 8, 10, 12, 6, 4, 2, 0};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length);
+    start = dateTimeFormatter.fromFormatToMillis("1636257600000");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i], gapFillRows1.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(DATETRUNC('hour', eventTime, 'milliseconds'), '1:MILLISECONDS:EPOCH', "
+        + "    '1636257600000',  '1636286400000', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE isOccupied = 1 "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCounts2 = new double[]{6, 8, 10, 12, 6, 4, 2};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCounts2.length);
+    start = dateTimeFormatter.fromFormatToMillis("1636257600000");
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      String firstTimeCol = ((Long) gapFillRows2.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts2[i], gapFillRows2.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void dateTruncHoursGapfillTestGapfillAggregateWithOptionalGroupBy() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:MILLISECONDS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, levelId, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(DATETRUNC('hour', eventTime, 'milliseconds'), '1:MILLISECONDS:EPOCH', "
+        + "    '1636257600000',  '1636286400000', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col, levelId "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCountsForLevel11 = new double[]{4, 5, 6, 5, 3, 2, 1, 0};
+    double[] expectedOccupiedSlotsCountsForLevel21 = new double[]{2, 3, 4, 7, 3, 2, 1, 0};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCountsForLevel11.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("1636257600000");
+    for (int i = 0; i < expectedOccupiedSlotsCountsForLevel11.length * 2; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows1.get(i)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel11[i / 2], gapFillRows1.get(i)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows1.get(i)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel21[i / 2], gapFillRows1.get(i)[2]);
+      }
+      firstTimeCol = ((Long) gapFillRows1.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows1.get(i + 1)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel11[i / 2], gapFillRows1.get(i + 1)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows1.get(i + 1)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel21[i / 2], gapFillRows1.get(i + 1)[2]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, levelId, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(DATETRUNC('hour', eventTime, 'milliseconds'), '1:MILLISECONDS:EPOCH', "
+        + "    '1636257600000',  '1636286400000', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE isOccupied = 1 "
+        + " GROUP BY time_col, levelId "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCountsForLevel12 = new double[]{4, 5, 6, 5, 3, 2, 1};
+    double[] expectedOccupiedSlotsCountsForLevel22 = new double[]{2, 3, 4, 7, 3, 2, 1};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCountsForLevel12.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("1636257600000");
+    for (int i = 0; i < expectedOccupiedSlotsCountsForLevel12.length * 2; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows2.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows2.get(i)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel12[i / 2], gapFillRows2.get(i)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows2.get(i)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel22[i / 2], gapFillRows2.get(i)[2]);
+      }
+      firstTimeCol = ((Long) gapFillRows2.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows2.get(i + 1)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel12[i / 2], gapFillRows2.get(i + 1)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows2.get(i + 1)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel22[i / 2], gapFillRows2.get(i + 1)[2]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void dateTruncHoursGapfillTestGapfillAggregateWithHavingClause() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:MILLISECONDS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT time_col, levelId, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(DATETRUNC('hour', eventTime, 'milliseconds'), '1:MILLISECONDS:EPOCH', "
+        + "    '1636257600000',  '1636286400000', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col, levelId "
+        + " HAVING occupied_slots_count > 0"
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCountsForLevel11 = new double[]{4, 5, 6, 5, 3, 2, 1};
+    double[] expectedOccupiedSlotsCountsForLevel21 = new double[]{2, 3, 4, 7, 3, 2, 1};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCountsForLevel11.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("1636257600000");
+    for (int i = 0; i < expectedOccupiedSlotsCountsForLevel11.length * 2; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows1.get(i)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel11[i / 2], gapFillRows1.get(i)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows1.get(i)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel21[i / 2], gapFillRows1.get(i)[2]);
+      }
+      firstTimeCol = ((Long) gapFillRows1.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      if ("Level_0".equals(gapFillRows1.get(i + 1)[1])) {
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel11[i / 2], gapFillRows1.get(i + 1)[2]);
+      } else {
+        Assert.assertEquals("Level_1", gapFillRows1.get(i + 1)[1]);
+        Assert.assertEquals(expectedOccupiedSlotsCountsForLevel21[i / 2], gapFillRows1.get(i + 1)[2]);
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void dateTruncHoursGapfillTestAggregateAggregate() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:MILLISECONDS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, '1:MILLISECONDS:EPOCH', "
+        + "    '1636257600000',  '1636286400000', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)), "
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT DATETRUNC('hour', eventTime, 'milliseconds') AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{2, 4, 6, 8, 6, 4, 2, 0};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length);
+    start = dateTimeFormatter.fromFormatToMillis("1636257600000");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[1]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i], gapFillRows1.get(i)[0]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, '1:MILLISECONDS:EPOCH', "
+        + "    '1636257600000',  '1636286400000', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT DATETRUNC('hour', eventTime, 'milliseconds') AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE occupied = 1 "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCounts2 = new double[]{2, 4, 6, 8, 6, 4, 2};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCounts2.length);
+    start = dateTimeFormatter.fromFormatToMillis("1636257600000");
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      String firstTimeCol = ((Long) gapFillRows2.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts2[i], gapFillRows2.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void dateTruncHoursGapfillTestAggregateAggregateWithOptionalGroupBy() {
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:MILLISECONDS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, levelId, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, '1:MILLISECONDS:EPOCH', "
+        + "    '1636257600000',  '1636286400000', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)), "
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT DATETRUNC('hour', eventTime, 'milliseconds') AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col, levelId "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{1, 2, 3, 4, 3, 2, 1, 0};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("1636257600000");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      firstTimeCol = ((Long) gapFillRows1.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, levelId, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, '1:MILLISECONDS:EPOCH', "
+        + "    '1636257600000',  '1636286400000', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)), "
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT DATETRUNC('hour', eventTime, 'milliseconds') AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE occupied = 1 "
+        + " GROUP BY time_col, levelId "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCounts2 = new double[]{1, 2, 3, 4, 3, 2, 1};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCounts2.length * 2);
+    start = dateTimeFormatter.fromFormatToMillis("1636257600000");
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows2.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      firstTimeCol = ((Long) gapFillRows2.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void dateTruncHoursGapfillTestAggregateAggregateWithHavingClause() {
+    String gapfillQuery1 = "SELECT "
+        + "time_col, levelId, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, '1:MILLISECONDS:EPOCH', "
+        + "    '1636257600000',  '1636286400000', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)),"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT DATETRUNC('hour', eventTime, 'milliseconds') AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col, levelId "
+        + " HAVING occupied_slots_count > 0"
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{1, 2, 3, 4, 3, 2, 1};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length * 2);
+    DateTimeFormatSpec dateTimeFormatter = new DateTimeFormatSpec("1:MILLISECONDS:EPOCH");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+
+    long start = dateTimeFormatter.fromFormatToMillis("1636257600000");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i += 2) {
+      String firstTimeCol = ((Long) gapFillRows1.get(i)[0]).toString();
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      firstTimeCol = ((Long) gapFillRows1.get(i + 1)[0]).toString();
+      timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i / 2], gapFillRows1.get(i)[2]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void datetimeconvertGapfillTestWithMissingTimeSeries() {
+    String gapfillQuery = "SELECT "
+        + "time_col, levelId, lotId, isOccupied, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(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 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE')) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " LIMIT 200 ";
+
+    try {
+      getBrokerResponseForSqlQuery(gapfillQuery);
+      Assert.fail();
+    } catch (Exception ex) {
+      Assert.assertTrue(ex.getClass().getSimpleName().equals("IllegalArgumentException"));
+    }
+  }
+
+  @Test
+  public void datetimeconvertGapfillTestWithMissingGroupByTimeBucket() {
+    String gapfillQuery = "SELECT "
+        + "levelId, SUM(isOccupied) "
+        + "FROM ("
+        + "  SELECT GapFill(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 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY levelId"
+        + " LIMIT 200 ";
+
+    try {
+      getBrokerResponseForSqlQuery(gapfillQuery);
+      Assert.fail();
+    } catch (Exception ex) {
+      Assert.assertTrue(ex.getClass().getSimpleName().equals("IllegalArgumentException"));
+    }
+  }
+
+  @Test
+  public void datetimeconvertGapfillTestAggregateAggregateWithLimitTesting() {
+    DateTimeFormatSpec dateTimeFormatter =
+        new DateTimeFormatSpec("1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, "
+        + "    '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+        + "    '2021-11-07 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "      '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS') AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 40 "
+        + ") "
+        + " GROUP BY time_col "
+        + " LIMIT 40 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{2, 4, 6, 8, 6};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length);
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      String firstTimeCol = (String) gapFillRows1.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i], gapFillRows1.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, "
+        + "    '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+        + "    '2021-11-07 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "      '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS') AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 56 "
+        + ") "
+        + " GROUP BY time_col "
+        + " LIMIT 6 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCounts2 = new double[]{2, 4, 6, 8, 6, 4};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCounts2.length);
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      String firstTimeCol = (String) gapFillRows1.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i], gapFillRows1.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void datetimeconvertGapfillTestSelectSelectOrderBy() {
+    DateTimeFormatSpec dateTimeFormatter =
+        new DateTimeFormatSpec("1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "  GapFill(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 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     levelId, lotId, isOccupied "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "ORDER BY time_col, levelId "
+        + "LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    int[][] expectedOccupiedSlotsCounts1 =
+        new int[][]{{6, 6}, {8, 4}, {10, 2}, {12, 0}, {6, 4}, {4, 6}, {2, 10}, {0, 10}};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    int index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      int ones = expectedOccupiedSlotsCounts1[i][0];
+      int zeros = expectedOccupiedSlotsCounts1[i][1];
+      int total = ones + zeros;
+      for (int k = 0; k < total; k++) {
+        String firstTimeCol = (String) gapFillRows1.get(index)[0];
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        if (gapFillRows1.get(index)[3].equals(1)) {
+          ones--;
+        } else {
+          zeros--;
+        }
+        index++;
+      }
+      Assert.assertEquals(ones, 0);
+      Assert.assertEquals(zeros, 0);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows1.size(), index);
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, levelId, lotId, isOccupied, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(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 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  ORDER BY time_col, levelId DESC"
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE isOccupied = 1 "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    int[] expectedOccupiedSlotsCounts2 = new int[]{6, 8, 10, 12, 6, 4, 2, 0};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      for (int k = 0; k < expectedOccupiedSlotsCounts2[i]; k++) {
+        String firstTimeCol = (String) gapFillRows2.get(index)[0];
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        Assert.assertEquals(gapFillRows2.get(index)[3], 1);
+        index++;
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows2.size(), index);
+  }
+
+  @Test
+  public void datetimeconvertGapfillTestAggregateSelectOrderBy() {
+    DateTimeFormatSpec dateTimeFormatter =
+        new DateTimeFormatSpec("1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "GapFill(time_col, "
+        + "    '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+        + "    '2021-11-07 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "    FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)), levelId, lotId, occupied "
+        + "FROM ("
+        + "  SELECT DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "    '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS') AS time_col,"
+        + "     lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  GROUP BY time_col, levelId, lotId "
+        + "  ORDER BY time_col, levelId "
+        + "  LIMIT 200 "
+        + ") "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    int[][] expectedOccupiedSlotsCounts1 = new int[][]{{2, 6}, {4, 4}, {6, 2}, {8, 0}, {6, 2}, {4, 4}, {2, 6}, {0, 8}};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    int index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      int ones = expectedOccupiedSlotsCounts1[i][0];
+      int zeros = expectedOccupiedSlotsCounts1[i][1];
+      int total = ones + zeros;
+      for (int k = 0; k < total; k++) {
+        String firstTimeCol = (String) gapFillRows1.get(index)[0];
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        if (gapFillRows1.get(index)[3].equals(1)) {
+          ones--;
+        } else {
+          zeros--;
+        }
+        index++;
+      }
+      Assert.assertEquals(ones, 0);
+      Assert.assertEquals(zeros, 0);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows1.size(), index);
+
+    String gapfillQuery2 =
+        "SELECT "
+        + "GapFill(time_col, "
+        + "    '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+            + "    '2021-11-07 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+            + "    FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)), levelId, lotId, occupied "
+            + "FROM ("
+        + "  SELECT DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+            + "    '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS') AS time_col,"
+            + "     lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "  FROM parkingData "
+            + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+            + "  GROUP BY time_col, levelId, lotId "
+        + "  ORDER BY time_col, levelId "
+        + "  LIMIT 200 "
+        + ") "
+            + " WHERE occupied = 1 "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    int[] expectedOccupiedSlotsCounts2 = new int[]{2, 4, 6, 8, 6, 4, 2};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    index = 0;
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      for (int k = 0; k < expectedOccupiedSlotsCounts2[i]; k++) {
+        String firstTimeCol = (String) gapFillRows2.get(index)[0];
+        long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+        Assert.assertEquals(timeStamp, start);
+        Assert.assertEquals(gapFillRows2.get(index)[3], 1);
+        index++;
+      }
+      start += dateTimeGranularity.granularityToMillis();
+    }
+    Assert.assertEquals(gapFillRows2.size(), index);
+  }
+
+  @Test
+  public void datetimeconvertGapfillTestGapfillAggregateOrderBy() {
+    DateTimeFormatSpec dateTimeFormatter =
+        new DateTimeFormatSpec("1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    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, "
+        + "SUM(isOccupied) "
+        + "FROM parkingData "
+        + "WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "GROUP BY 1 "
+        + "ORDER BY 1 "
+        + "LIMIT 200";
+
+    BrokerResponseNative dateTimeConvertBrokerResponse = getBrokerResponseForSqlQuery(dataTimeConvertQuery);
+
+    ResultTable dateTimeConvertResultTable = dateTimeConvertBrokerResponse.getResultTable();
+    Assert.assertEquals(dateTimeConvertResultTable.getRows().size(), 8);
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT  GapFill(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 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  ORDER BY time_col, levelId "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{6, 8, 10, 12, 6, 4, 2, 0};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length);
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      String firstTimeCol = (String) gapFillRows1.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i], gapFillRows1.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, SUM(isOccupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT  GapFill(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 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "     FILL(isOccupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     isOccupied, lotId, levelId"
+        + "  FROM parkingData "
+        + "  WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "  ORDER BY time_col, levelId "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE isOccupied = 1 "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCounts2 = new double[]{6, 8, 10, 12, 6, 4, 2};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCounts2.length);
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      String firstTimeCol = (String) gapFillRows2.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts2[i], gapFillRows2.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void datetimeconvertGapfillTestAggregateAggregateOrderBy() {
+    DateTimeFormatSpec dateTimeFormatter =
+        new DateTimeFormatSpec("1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, "
+        + "    '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+        + "    '2021-11-07 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "      '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS') AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    ORDER BY time_col, levelId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    double[] expectedOccupiedSlotsCounts1 = new double[]{2, 4, 6, 8, 6, 4, 2, 0};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length);
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      String firstTimeCol = (String) gapFillRows1.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i], gapFillRows1.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, "
+        + "    '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+        + "    '2021-11-07 4:00:00.000',  '2021-11-07 12:00:00.000', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "      '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '1:HOURS') AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    ORDER BY time_col, levelId "
+        + "    LIMIT 200 "
+        + "  ) "
+        + "  LIMIT 200 "
+        + ") "
+        + " WHERE occupied = 1 "
+        + " GROUP BY time_col "
+        + " LIMIT 200 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCounts2 = new double[]{2, 4, 6, 8, 6, 4, 2};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCounts2.length);
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      String firstTimeCol = (String) gapFillRows2.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts2[i], gapFillRows2.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @Test
+  public void datetimeconvertGapfillTestAggregateAggregateWithTimeBucketAggregation() {
+    DateTimeFormatSpec dateTimeFormatter =
+        new DateTimeFormatSpec("1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS");
+    DateTimeGranularitySpec dateTimeGranularity = new DateTimeGranularitySpec("1:HOURS");
+    long start;
+
+    String gapfillQuery1 = "SELECT "
+        + "time_col, COUNT(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, "
+        + "    '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+        + "    '2021-11-07 4:00:00.000',  '2021-11-07 12:00:00.000', '5:MINUTES', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "      '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '5:MINUTES') AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 2000 "
+        + "  ) "
+        + "  LIMIT 2000 "
+        + ") "
+        + " WHERE occupied = 1 "
+        + " GROUP BY time_col "
+        + " LIMIT 2000 ";
+
+    BrokerResponseNative gapfillBrokerResponse1 = getBrokerResponseForSqlQuery(gapfillQuery1);
+
+    long [] expectedOccupiedSlotsCounts1 = new long[]{16, 36, 60, 79, 88, 60, 33, 21};
+    ResultTable gapFillResultTable1 = gapfillBrokerResponse1.getResultTable();
+    List<Object[]> gapFillRows1 = gapFillResultTable1.getRows();
+    Assert.assertEquals(gapFillRows1.size(), expectedOccupiedSlotsCounts1.length);
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    for (int i = 0; i < expectedOccupiedSlotsCounts1.length; i++) {
+      String firstTimeCol = (String) gapFillRows1.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts1[i], gapFillRows1.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+
+    String gapfillQuery2 = "SELECT "
+        + "time_col, SUM(occupied) as occupied_slots_count, time_col "
+        + "FROM ("
+        + "  SELECT GapFill(time_col, "
+        + "    '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', "
+        + "    '2021-11-07 4:00:00.000',  '2021-11-07 12:00:00.000', '5:MINUTES', '1:HOURS',"
+        + "     FILL(occupied, 'FILL_PREVIOUS_VALUE'), TIMESERIESON(levelId, lotId)) AS time_col,"
+        + "     occupied, lotId, levelId"
+        + "  FROM ("
+        + "    SELECT DATETIMECONVERT(eventTime, '1:MILLISECONDS:EPOCH', "
+        + "      '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss.SSS', '5:MINUTES') AS time_col,"
+        + "       lastWithTime(isOccupied, eventTime, 'INT') as occupied, lotId, levelId"
+        + "    FROM parkingData "
+        + "    WHERE eventTime >= 1636257600000 AND eventTime <= 1636286400000 "
+        + "    GROUP BY time_col, levelId, lotId "
+        + "    LIMIT 2000 "
+        + "  ) "
+        + "  LIMIT 2000 "
+        + ") "
+        + " WHERE occupied = 1 "
+        + " GROUP BY time_col "
+        + " LIMIT 2000 ";
+
+    BrokerResponseNative gapfillBrokerResponse2 = getBrokerResponseForSqlQuery(gapfillQuery2);
+
+    double[] expectedOccupiedSlotsCounts2 = new double[]{16, 36, 60, 79, 88, 60, 33, 21};
+    ResultTable gapFillResultTable2 = gapfillBrokerResponse2.getResultTable();
+    List<Object[]> gapFillRows2 = gapFillResultTable2.getRows();
+    Assert.assertEquals(gapFillRows2.size(), expectedOccupiedSlotsCounts2.length);
+    start = dateTimeFormatter.fromFormatToMillis("2021-11-07 04:00:00.000");
+    for (int i = 0; i < expectedOccupiedSlotsCounts2.length; i++) {
+      String firstTimeCol = (String) gapFillRows2.get(i)[0];
+      long timeStamp = dateTimeFormatter.fromFormatToMillis(firstTimeCol);
+      Assert.assertEquals(timeStamp, start);
+      Assert.assertEquals(expectedOccupiedSlotsCounts2[i], gapFillRows2.get(i)[1]);
+      start += dateTimeGranularity.granularityToMillis();
+    }
+  }
+
+  @AfterClass
+  public void tearDown()
+      throws IOException {
+    _indexSegment.destroy();
+    FileUtils.deleteDirectory(INDEX_DIR);
+  }
+}

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