You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by su...@apache.org on 2019/04/04 02:51:47 UTC

[incubator-iotdb] branch master updated: [IOTDB-47][IOTDB-54][IOTDB-59][IOTDB-60]Aggregate+GroupBy+Fill (#97)

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

suyue pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git


The following commit(s) were added to refs/heads/master by this push:
     new e0d4e33  [IOTDB-47][IOTDB-54][IOTDB-59][IOTDB-60]Aggregate+GroupBy+Fill (#97)
e0d4e33 is described below

commit e0d4e33517657c69871e1056b4dcb7f37c6eae84
Author: suyue <23...@qq.com>
AuthorDate: Thu Apr 4 10:51:43 2019 +0800

    [IOTDB-47][IOTDB-54][IOTDB-59][IOTDB-60]Aggregate+GroupBy+Fill (#97)
    
    * realize reader for aggregation
    
    * realize aggregate function interface
    
    * add aggregation in qp
    
    * add test
    
    * optimization last and max_time aggregate query that has only time filter or no filter
    
    * add aggregate function with value filter
    
    * modify for pr review suggestion
    
    * fix bug of IOTDB-54
    
    * group by
    
    * add groupby test and fix groupBy bug
    
    * fill query
    
    * modify pr review
    
    * provide unified query resource control interface in QueryResourceManager.
    
    * bug fix: multiple queries in the same statement will cause resource leak.
    
    * optimization of query with value filter by push down filter
---
 .../db/engine/querycontext/ReadOnlyMemChunk.java   |   7 -
 .../java/org/apache/iotdb/db/metadata/MTree.java   |   2 +-
 .../iotdb/db/qp/executor/OverflowQPExecutor.java   |  44 +-
 .../iotdb/db/qp/executor/QueryProcessExecutor.java |  99 ++-
 .../iotdb/db/qp/physical/sys/MetadataPlan.java     |   2 +-
 .../iotdb/db/qp/strategy/PhysicalGenerator.java    |   1 +
 .../db/query/aggregation/AggreFuncFactory.java     |  79 ++
 .../db/query/aggregation/AggreResultData.java      | 192 +++++
 .../db/query/aggregation/AggregateFunction.java    | 135 ++++
 .../db/query/aggregation/impl/CountAggrFunc.java   | 155 ++++
 .../db/query/aggregation/impl/FirstAggrFunc.java   | 154 ++++
 .../db/query/aggregation/impl/LastAggrFunc.java    | 151 ++++
 .../db/query/aggregation/impl/MaxTimeAggrFunc.java | 135 ++++
 .../query/aggregation/impl/MaxValueAggrFunc.java   | 170 ++++
 .../db/query/aggregation/impl/MeanAggrFunc.java    | 163 ++++
 .../db/query/aggregation/impl/MinTimeAggrFunc.java | 156 ++++
 .../query/aggregation/impl/MinValueAggrFunc.java   | 166 ++++
 .../db/query/aggregation/impl/SumAggrFunc.java}    |  66 +-
 .../iotdb/db/query/context/QueryContext.java       |  17 +
 ...edFilePathsManager.java => JobFileManager.java} |  78 +-
 .../db/query/control/QueryDataSourceManager.java   |  53 --
 ...TokenManager.java => QueryResourceManager.java} |  86 +-
 .../query/dataset/AggreResultDataPointReader.java  |  52 ++
 .../dataset/EngineDataSetWithTimeGenerator.java    |   7 +-
 .../dataset/EngineDataSetWithoutTimeGenerator.java |  16 +-
 .../db/query/executor/AggregateEngineExecutor.java | 330 ++++++++
 .../executor/EngineExecutorWithTimeGenerator.java  |  55 +-
 .../EngineExecutorWithoutTimeGenerator.java        |  41 +-
 .../iotdb/db/query/executor/EngineQueryRouter.java | 180 ++++-
 .../db/query/executor/FillEngineExecutor.java      |  91 +++
 .../executor/groupby/GroupByEngineDataSet.java     | 167 ++++
 .../GroupByWithOnlyTimeFilterDataSetDataSet.java   | 314 +++++++
 .../GroupByWithValueFilterDataSetDataSet.java      | 160 ++++
 .../db/query/factory/SeriesReaderFactory.java      | 129 ++-
 .../java/org/apache/iotdb/db/query/fill/IFill.java |  79 +-
 .../org/apache/iotdb/db/query/fill/LinearFill.java |  90 ++-
 .../apache/iotdb/db/query/fill/PreviousFill.java   |  36 +-
 .../iotdb/db/query/reader/AllDataReader.java       | 129 +++
 ...eaderByTimeStamp.java => IAggregateReader.java} |  16 +-
 .../iotdb/db/query/reader/IBatchReader.java}       |  61 +-
 .../iotdb/db/query/reader/IPointReader.java}       |  57 +-
 .../org/apache/iotdb/db/query/reader/IReader.java  |  14 +-
 ...nkReaderWithFilter.java => MemChunkReader.java} |  57 +-
 .../reader/mem/MemChunkReaderByTimestamp.java      |  37 +-
 .../reader/mem/MemChunkReaderWithoutFilter.java    |  70 --
 .../reader/merge/EngineReaderByTimeStamp.java      |   8 +-
 .../db/query/reader/merge/PriorityMergeReader.java |  56 +-
 .../merge/PriorityMergeReaderByTimestamp.java      |  60 +-
 .../query/reader/sequence/SealedTsFilesReader.java | 151 ++--
 .../sequence/SealedTsFilesReaderByTimestamp.java   | 136 ++++
 .../query/reader/sequence/SequenceDataReader.java  | 106 ++-
 .../sequence/SequenceDataReaderByTimestamp.java    | 108 +++
 .../reader/sequence/UnSealedTsFileReader.java      |  65 +-
 .../sequence/UnSealedTsFilesReaderByTimestamp.java |  64 ++
 .../query/reader/unsequence/EngineChunkReader.java |  39 +-
 .../unsequence/EngineChunkReaderByTimestamp.java   |  77 ++
 .../db/query/timegenerator/EngineLeafNode.java     |  11 +-
 .../query/timegenerator/EngineNodeConstructor.java |  22 +-
 .../query/timegenerator/EngineTimeGenerator.java   |  10 +-
 .../org/apache/iotdb/db/rescon/package-info.java}  |  53 +-
 .../java/org/apache/iotdb/db/service/IoTDB.java    |   1 -
 .../org/apache/iotdb/db/service/TSServiceImpl.java |  76 +-
 .../java/org/apache/iotdb/db/utils/QueryUtils.java |   3 +
 .../apache/iotdb/db/utils/TimeValuePairUtils.java  |  26 +
 .../org/apache/iotdb/db/utils/TsPrimitiveType.java |   2 +-
 .../engine/modification/DeletionFileNodeTest.java  |  25 +-
 .../db/engine/modification/DeletionQueryTest.java  |  16 +-
 .../apache/iotdb/db/integration/IOTDBFillIT.java   | 351 ++++++++
 .../iotdb/db/integration/IOTDBGroupByIT.java       | 532 ++++++++++++
 .../iotdb/db/integration/IoTDBAggregationIT.java   | 550 +++++++++++++
 .../integration/IoTDBAggregationLargeDataIT.java   | 900 +++++++++++++++++++++
 .../integration/IoTDBAggregationSmallDataIT.java   | 757 +++++++++++++++++
 .../iotdb/db/integration/IoTDBCompleteIT.java      |  38 +
 .../db/integration/IoTDBEngineTimeGeneratorIT.java |  20 +-
 .../iotdb/db/integration/IoTDBLargeDataIT.java     |   4 -
 .../db/integration/IoTDBSequenceDataQueryIT.java   |  23 +-
 .../iotdb/db/integration/IoTDBSeriesReaderIT.java  |  29 +-
 .../apache/iotdb/db/qp/plan/PhysicalPlanTest.java  |   2 +-
 .../org/apache/iotdb/db/qp/plan/QPUpdateTest.java  |  31 +-
 .../apache/iotdb/db/qp/utils/MemIntQpExecutor.java |  27 +-
 .../db/query/control/FileReaderManagerTest.java    |  11 +-
 ...agerTest.java => QueryResourceManagerTest.java} |   2 +-
 .../query/executor/GroupByEngineDataSetTest.java   | 153 ++++
 .../iotdb/db/query/reader/AllDataReaderTest.java   |  76 ++
 .../iotdb/db/query/reader/FakedIBatchPoint.java    | 108 +++
 .../iotdb/db/query/reader/FakedIPointReader.java   |  74 ++
 .../query/reader/FakedSeriesReaderByTimestamp.java |  76 ++
 .../merge/PriorityMergeReaderByTimestampTest.java  |  72 +-
 .../reader/merge/PriorityMergeReaderTest.java      |  28 +-
 .../reader/merge/SeriesMergeSortReaderTest.java    |  27 +-
 .../SequenceDataReaderByTimestampTest.java         |  91 +++
 .../apache/iotdb/db/utils/EnvironmentUtils.java    |  10 +-
 .../org/apache/iotdb/jdbc/IoTDBQueryResultSet.java |   8 +-
 .../java/org/apache/iotdb/jdbc/IoTDBStatement.java |   9 +-
 service-rpc/src/main/thrift/rpc.thrift             |   2 +
 .../apache/iotdb/tsfile/read/common/BatchData.java |  97 ++-
 .../iotdb/tsfile/read/filter/basic/Filter.java     |   8 +
 .../tsfile/read/filter/operator/AndFilter.java     |   6 +
 .../iotdb/tsfile/read/filter/operator/Eq.java      |  14 +
 .../iotdb/tsfile/read/filter/operator/Gt.java      |  17 +-
 .../iotdb/tsfile/read/filter/operator/GtEq.java    |  17 +-
 .../iotdb/tsfile/read/filter/operator/Lt.java      |  14 +
 .../iotdb/tsfile/read/filter/operator/LtEq.java    |  14 +
 .../iotdb/tsfile/read/filter/operator/NotEq.java   |  14 +
 .../tsfile/read/filter/operator/NotFilter.java     |   5 +
 .../tsfile/read/filter/operator/OrFilter.java      |   6 +
 .../tsfile/read/query/dataset/QueryDataSet.java    |   7 +
 .../tsfile/read/reader/chunk/ChunkReader.java      |  54 +-
 .../read/reader/series/FileSeriesReader.java       |  45 +-
 .../reader/series/SeriesReaderByTimestamp.java     |  33 +-
 .../read/filter/MinTimeMaxTimeFilterTest.java      |  52 +-
 .../iotdb/tsfile/read/filter/OperatorTest.java     |   1 +
 112 files changed, 8701 insertions(+), 1128 deletions(-)

diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/querycontext/ReadOnlyMemChunk.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/querycontext/ReadOnlyMemChunk.java
index d42eef7..5086c6e 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/querycontext/ReadOnlyMemChunk.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/querycontext/ReadOnlyMemChunk.java
@@ -48,13 +48,6 @@ public class ReadOnlyMemChunk implements TimeValuePairSorter {
   /**
    * init by TSDataType and TimeValuePairSorter.
    */
-//  public ReadOnlyMemChunk(TSDataType dataType, TimeValuePairSorter memSeries) {
-//    this(dataType, memSeries, Collections.emptyMap());
-//  }
-
-  /**
-   * init by TSDataType and TimeValuePairSorter.
-   */
   public ReadOnlyMemChunk(TSDataType dataType, TimeValuePairSorter memSeries, Map<String, String> props) {
     this.dataType = dataType;
     this.memSeries = memSeries;
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/metadata/MTree.java b/iotdb/src/main/java/org/apache/iotdb/db/metadata/MTree.java
index 6444b4a..2c7e5b0 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/metadata/MTree.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/metadata/MTree.java
@@ -970,7 +970,7 @@ public class MTree implements Serializable {
     if (node.isLeaf()) {
       if (nodes.length <= idx) {
         String nodePath = parent + node;
-        List<String> tsRow = new ArrayList<>(4);// get [name,storage group,dataType,encoding]
+        List<String> tsRow = new ArrayList<>(4);// get [name,storage group,resultDataType,encoding]
         tsRow.add(nodePath);
         MeasurementSchema measurementSchema = node.getSchema();
         tsRow.add(node.getDataFileName());
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/OverflowQPExecutor.java b/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/OverflowQPExecutor.java
index 061bb47..d477e2f 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/OverflowQPExecutor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/OverflowQPExecutor.java
@@ -51,8 +51,11 @@ import org.apache.iotdb.db.qp.physical.sys.AuthorPlan;
 import org.apache.iotdb.db.qp.physical.sys.LoadDataPlan;
 import org.apache.iotdb.db.qp.physical.sys.MetadataPlan;
 import org.apache.iotdb.db.qp.physical.sys.PropertyPlan;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.fill.IFill;
 import org.apache.iotdb.db.utils.AuthUtils;
 import org.apache.iotdb.db.utils.LoadDataUtils;
+import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
 import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
@@ -176,22 +179,26 @@ public class OverflowQPExecutor extends QueryProcessExecutor {
   }
 
   @Override
-  public QueryDataSet aggregate(List<Pair<Path, String>> aggres, IExpression expression)
-      throws ProcessorException {
-    throw new ProcessorException("not support");
+  public QueryDataSet aggregate(List<Path> paths, List<String> aggres, IExpression expression,
+      QueryContext context)
+      throws ProcessorException, FileNodeManagerException, QueryFilterOptimizationException,
+      PathErrorException, IOException {
+    return queryRouter.aggregate(paths, aggres, expression, context);
   }
 
-  // @Override
-  // public QueryDataSet fill(List<Path> fillPaths, long queryTime, Map<TSDataType, IFill> fillTypes) throws
-  // ProcessorException, IOException, PathErrorException {
-  // return queryEngine.fill(fillPaths, queryTime, fillTypes);
-  // }
+  @Override
+  public QueryDataSet fill(List<Path> fillPaths, long queryTime, Map<TSDataType, IFill> fillTypes,
+      QueryContext context)
+      throws ProcessorException, IOException, PathErrorException, FileNodeManagerException {
+    return queryRouter.fill(fillPaths, queryTime, fillTypes, context);
+  }
 
   @Override
-  public QueryDataSet groupBy(List<Pair<Path, String>> aggres, IExpression expression, long unit,
-      long origin,
-      List<Pair<Long, Long>> intervals, int fetchSize) throws ProcessorException {
-    throw new ProcessorException("not support");
+  public QueryDataSet groupBy(List<Path> paths, List<String> aggres, IExpression expression,
+      long unit, long origin, List<Pair<Long, Long>> intervals, QueryContext context)
+      throws ProcessorException, FileNodeManagerException, QueryFilterOptimizationException,
+      PathErrorException, IOException {
+    return queryRouter.groupBy(paths, aggres, expression, unit, origin, intervals, context);
   }
 
   @Override
@@ -502,7 +509,8 @@ public class OverflowQPExecutor extends QueryProcessExecutor {
           // optimize the speed of adding timeseries
           String fileNodePath = mManager.getFileNameByPath(path.getFullPath());
           // the two map is stored in the storage group node
-          Map<String, MeasurementSchema> schemaMap = mManager.getSchemaMapForOneFileNode(fileNodePath);
+          Map<String, MeasurementSchema> schemaMap = mManager
+              .getSchemaMapForOneFileNode(fileNodePath);
           Map<String, Integer> numSchemaMap = mManager.getNumSchemaMapForOneFileNode(fileNodePath);
           String lastNode = path.getMeasurement();
           boolean isNewMeasurement = true;
@@ -514,8 +522,8 @@ public class OverflowQPExecutor extends QueryProcessExecutor {
               if (!columnSchema.getType().equals(dataType)
                   || !columnSchema.getEncodingType().equals(encoding)) {
                 throw new ProcessorException(String.format(
-                    "The dataType or encoding of the last node %s is conflicting in the storage group %s",
-                    lastNode, fileNodePath));
+                    "The resultDataType or encoding of the last node %s is conflicting "
+                        + "in the storage group %s", lastNode, fileNodePath));
               }
               mManager.addPathToMTree(path.getFullPath(), dataType, encoding, compressor, props);
               numSchemaMap.put(lastNode, numSchemaMap.get(lastNode) + 1);
@@ -529,7 +537,8 @@ public class OverflowQPExecutor extends QueryProcessExecutor {
               if (isNewMeasurement) {
                 // add time series to schema
                 fileNodeManager.addTimeSeries(path, dataType, encoding, compressor, props);
-                //TODO fileNodeManager.addTimeSeries(path, dataType, encoding, compressor, encodingArgs);
+                //TODO fileNodeManager.addTimeSeries(
+                //TODO path, resultDataType, encoding, compressor, encodingArgs);
               }
               // fileNodeManager.closeOneFileNode(namespacePath);
             } catch (FileNodeManagerException e) {
@@ -627,8 +636,7 @@ public class OverflowQPExecutor extends QueryProcessExecutor {
   /**
    * Delete all data of time series in pathList.
    *
-   * @param pathList
-   *            deleted paths
+   * @param pathList deleted paths
    */
   private void deleteDataOfTimeSeries(List<String> pathList) throws ProcessorException {
     for (String p : pathList) {
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/QueryProcessExecutor.java b/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/QueryProcessExecutor.java
index b735617..60f97f7 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/QueryProcessExecutor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/QueryProcessExecutor.java
@@ -22,14 +22,21 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import org.apache.iotdb.db.exception.FileNodeManagerException;
 import org.apache.iotdb.db.exception.PathErrorException;
 import org.apache.iotdb.db.exception.ProcessorException;
 import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.AggregationPlan;
+import org.apache.iotdb.db.qp.physical.crud.FillQueryPlan;
+import org.apache.iotdb.db.qp.physical.crud.GroupByPlan;
 import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.executor.EngineQueryRouter;
+import org.apache.iotdb.db.query.fill.IFill;
+import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.read.expression.IExpression;
@@ -40,18 +47,40 @@ import org.apache.iotdb.tsfile.utils.Pair;
 public abstract class QueryProcessExecutor {
 
   protected ThreadLocal<Integer> fetchSize = new ThreadLocal<>();
-  private EngineQueryRouter queryRouter = new EngineQueryRouter();
+  protected EngineQueryRouter queryRouter = new EngineQueryRouter();
 
   public QueryProcessExecutor() {
   }
 
-  public QueryDataSet processQuery(PhysicalPlan plan) throws IOException, FileNodeManagerException {
-    QueryPlan queryPlan = (QueryPlan) plan;
+  /**
+   * process query plan of qp layer, construct queryDataSet.
+   * @param queryPlan QueryPlan
+   * @return QueryDataSet
+   */
+  public QueryDataSet processQuery(QueryPlan queryPlan, QueryContext context)
+      throws IOException, FileNodeManagerException, PathErrorException,
+      QueryFilterOptimizationException, ProcessorException {
 
     QueryExpression queryExpression = QueryExpression.create().setSelectSeries(queryPlan.getPaths())
         .setExpression(queryPlan.getExpression());
+    if (queryPlan instanceof GroupByPlan) {
+      GroupByPlan groupByPlan = (GroupByPlan) queryPlan;
+      return groupBy(groupByPlan.getPaths(), groupByPlan.getAggregations(),
+          groupByPlan.getExpression(), groupByPlan.getUnit(), groupByPlan.getOrigin(),
+          groupByPlan.getIntervals(), context);
+    }
 
-    return queryRouter.query(queryExpression);
+    if (queryPlan instanceof AggregationPlan) {
+      return aggregate(queryPlan.getPaths(), queryPlan.getAggregations(),
+          ((AggregationPlan) queryPlan).getExpression(), context);
+    }
+
+    if (queryPlan instanceof FillQueryPlan) {
+      FillQueryPlan fillQueryPlan = (FillQueryPlan) queryPlan;
+      return fill(queryPlan.getPaths(), fillQueryPlan.getQueryTime(),
+          fillQueryPlan.getFillType(), context);
+    }
+    return queryRouter.query(queryExpression, context);
   }
 
   public abstract TSDataType getSeriesType(Path fullPath) throws PathErrorException;
@@ -73,25 +102,26 @@ public abstract class QueryProcessExecutor {
     this.fetchSize.set(fetchSize);
   }
 
-  public abstract QueryDataSet aggregate(List<Pair<Path, String>> aggres, IExpression expression)
-      throws ProcessorException, IOException, PathErrorException;
+  public abstract QueryDataSet aggregate(List<Path> paths, List<String> aggres,
+      IExpression expression, QueryContext context) throws ProcessorException, IOException,
+      PathErrorException, FileNodeManagerException, QueryFilterOptimizationException;
+
+  public abstract QueryDataSet groupBy(List<Path> paths, List<String> aggres,
+      IExpression expression, long unit, long origin, List<Pair<Long, Long>> intervals,
+      QueryContext context) throws ProcessorException, IOException, PathErrorException,
+      FileNodeManagerException, QueryFilterOptimizationException;
 
-  public abstract QueryDataSet groupBy(List<Pair<Path, String>> aggres, IExpression expression,
-      long unit,
-      long origin, List<Pair<Long, Long>> intervals, int fetchSize)
-      throws ProcessorException, IOException, PathErrorException;
+  public abstract QueryDataSet fill(List<Path> fillPaths, long queryTime, Map<TSDataType,
+      IFill> fillTypes, QueryContext context)
+      throws ProcessorException, IOException, PathErrorException, FileNodeManagerException;
 
   /**
    * executeWithGlobalTimeFilter update command and return whether the operator is successful.
    *
-   * @param path
-   *            : update series seriesPath
-   * @param startTime
-   *            start time in update command
-   * @param endTime
-   *            end time in update command
-   * @param value
-   *            - in type of string
+   * @param path : update series seriesPath
+   * @param startTime start time in update command
+   * @param endTime end time in update command
+   * @param value - in type of string
    * @return - whether the operator is successful.
    */
   public abstract boolean update(Path path, long startTime, long endTime, String value)
@@ -100,10 +130,8 @@ public abstract class QueryProcessExecutor {
   /**
    * executeWithGlobalTimeFilter delete command and return whether the operator is successful.
    *
-   * @param paths
-   *            : delete series paths
-   * @param deleteTime
-   *            end time in delete command
+   * @param paths : delete series paths
+   * @param deleteTime end time in delete command
    * @return - whether the operator is successful.
    */
   public boolean delete(List<Path> paths, long deleteTime) throws ProcessorException {
@@ -137,10 +165,8 @@ public abstract class QueryProcessExecutor {
   /**
    * executeWithGlobalTimeFilter delete command and return whether the operator is successful.
    *
-   * @param path
-   *            : delete series seriesPath
-   * @param deleteTime
-   *            end time in delete command
+   * @param path : delete series seriesPath
+   * @param deleteTime end time in delete command
    * @return - whether the operator is successful.
    */
   protected abstract boolean delete(Path path, long deleteTime) throws ProcessorException;
@@ -148,12 +174,9 @@ public abstract class QueryProcessExecutor {
   /**
    * insert a single value. Only used in test
    *
-   * @param path
-   *            seriesPath to be inserted
-   * @param insertTime
-   *            - it's time point but not a range
-   * @param value
-   *            value to be inserted
+   * @param path seriesPath to be inserted
+   * @param insertTime - it's time point but not a range
+   * @param value value to be inserted
    * @return - Operate Type.
    */
   public abstract int insert(Path path, long insertTime, String value) throws ProcessorException;
@@ -161,14 +184,10 @@ public abstract class QueryProcessExecutor {
   /**
    * executeWithGlobalTimeFilter insert command and return whether the operator is successful.
    *
-   * @param deviceId
-   *            deviceId to be inserted
-   * @param insertTime
-   *            - it's time point but not a range
-   * @param measurementList
-   *            measurements to be inserted
-   * @param insertValues
-   *            values to be inserted
+   * @param deviceId deviceId to be inserted
+   * @param insertTime - it's time point but not a range
+   * @param measurementList measurements to be inserted
+   * @param insertValues values to be inserted
    * @return - Operate Type.
    */
   public abstract int multiInsert(String deviceId, long insertTime, List<String> measurementList,
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/qp/physical/sys/MetadataPlan.java b/iotdb/src/main/java/org/apache/iotdb/db/qp/physical/sys/MetadataPlan.java
index f612165..82b9404 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/qp/physical/sys/MetadataPlan.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/qp/physical/sys/MetadataPlan.java
@@ -114,7 +114,7 @@ public class MetadataPlan extends PhysicalPlan {
 
   @Override
   public String toString() {
-    String ret = String.format("seriesPath: %s\ndataType: %s\nencoding: %s\nnamespace type: %s\nargs: ", path, dataType, encoding, namespaceType);
+    String ret = String.format("seriesPath: %s\nresultDataType: %s\nencoding: %s\nnamespace type: %s\nargs: ", path, dataType, encoding, namespaceType);
     StringBuilder stringBuilder = new StringBuilder(ret.length()+50);
     stringBuilder.append(ret);
     for (Map.Entry prop : props.entrySet()) {
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java b/iotdb/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
index 2caac94..290d777 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
@@ -242,6 +242,7 @@ public class PhysicalGenerator {
       ((GroupByPlan) queryPlan).setUnit(queryOperator.getUnit());
       ((GroupByPlan) queryPlan).setOrigin(queryOperator.getOrigin());
       ((GroupByPlan) queryPlan).setIntervals(queryOperator.getIntervals());
+      ((GroupByPlan) queryPlan).setAggregations(queryOperator.getSelectOperator().getAggregations());
     } else if (queryOperator.isFill()) {
       queryPlan = new FillQueryPlan();
       FilterOperator timeFilter = queryOperator.getFilterOperator();
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/AggreFuncFactory.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/AggreFuncFactory.java
new file mode 100644
index 0000000..c2d1883
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/AggreFuncFactory.java
@@ -0,0 +1,79 @@
+/**
+ * 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.iotdb.db.query.aggregation;
+
+import org.apache.iotdb.db.exception.ProcessorException;
+import org.apache.iotdb.db.query.aggregation.impl.CountAggrFunc;
+import org.apache.iotdb.db.query.aggregation.impl.FirstAggrFunc;
+import org.apache.iotdb.db.query.aggregation.impl.LastAggrFunc;
+import org.apache.iotdb.db.query.aggregation.impl.MaxTimeAggrFunc;
+import org.apache.iotdb.db.query.aggregation.impl.MaxValueAggrFunc;
+import org.apache.iotdb.db.query.aggregation.impl.MeanAggrFunc;
+import org.apache.iotdb.db.query.aggregation.impl.MinTimeAggrFunc;
+import org.apache.iotdb.db.query.aggregation.impl.MinValueAggrFunc;
+import org.apache.iotdb.db.query.aggregation.impl.SumAggrFunc;
+import org.apache.iotdb.tsfile.common.constant.StatisticConstant;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+/**
+ * Easy factory pattern to build AggregateFunction.
+ */
+public class AggreFuncFactory {
+
+  private AggreFuncFactory() {
+  }
+
+  /**
+   * construct AggregateFunction using factory pattern.
+   *
+   * @param aggrFuncName function name.
+   * @param dataType data type.
+   */
+  public static AggregateFunction getAggrFuncByName(String aggrFuncName, TSDataType dataType)
+      throws ProcessorException {
+    if (aggrFuncName == null) {
+      throw new ProcessorException("AggregateFunction Name must not be null");
+    }
+
+    switch (aggrFuncName.toLowerCase()) {
+      case StatisticConstant.MIN_TIME:
+        return new MinTimeAggrFunc();
+      case StatisticConstant.MAX_TIME:
+        return new MaxTimeAggrFunc();
+      case StatisticConstant.MIN_VALUE:
+        return new MinValueAggrFunc(dataType);
+      case StatisticConstant.MAX_VALUE:
+        return new MaxValueAggrFunc(dataType);
+      case StatisticConstant.COUNT:
+        return new CountAggrFunc();
+      case StatisticConstant.MEAN:
+        return new MeanAggrFunc(dataType);
+      case StatisticConstant.FIRST:
+        return new FirstAggrFunc(dataType);
+      case StatisticConstant.SUM:
+        return new SumAggrFunc(dataType);
+      case StatisticConstant.LAST:
+        return new LastAggrFunc(dataType);
+      default:
+        throw new ProcessorException(
+            "aggregate does not support " + aggrFuncName + " function.");
+    }
+  }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/AggreResultData.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/AggreResultData.java
new file mode 100644
index 0000000..0efae01
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/AggreResultData.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.iotdb.db.query.aggregation;
+
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Binary;
+
+public class AggreResultData {
+
+  private long timestamp;
+  private TSDataType dataType;
+
+  private boolean booleanRet;
+  private int intRet;
+  private long longRet;
+  private float floatRet;
+  private double doubleRet;
+  private Binary binaryRet;
+
+  private boolean isSetValue;
+  private boolean isSetTime;
+
+  public AggreResultData(TSDataType dataType) {
+    this.dataType = dataType;
+    this.isSetTime = false;
+    this.isSetValue = false;
+  }
+
+  public void reSet() {
+    isSetValue = false;
+    isSetTime = false;
+  }
+
+  public void putTimeAndValue(long timestamp, Object v) {
+    setTimestamp(timestamp);
+    setAnObject((Comparable<?>) v);
+  }
+
+  public Object getValue() {
+    switch (dataType) {
+      case BOOLEAN:
+        return booleanRet;
+      case DOUBLE:
+        return doubleRet;
+      case TEXT:
+        return binaryRet;
+      case FLOAT:
+        return floatRet;
+      case INT32:
+        return intRet;
+      case INT64:
+        return longRet;
+      default:
+        throw new UnSupportedDataTypeException(String.valueOf(dataType));
+    }
+  }
+
+  /**
+   * set an object.
+   *
+   * @param v object value
+   */
+  public void setAnObject(Comparable<?> v) {
+    isSetValue = true;
+    switch (dataType) {
+      case BOOLEAN:
+        booleanRet = (Boolean) v;
+        break;
+      case DOUBLE:
+        doubleRet = (Double) v;
+        break;
+      case TEXT:
+        binaryRet = (Binary) v;
+        break;
+      case FLOAT:
+        floatRet = (Float) v;
+        break;
+      case INT32:
+        intRet = (Integer) v;
+        break;
+      case INT64:
+        longRet = (Long) v;
+        break;
+      default:
+        throw new UnSupportedDataTypeException(String.valueOf(dataType));
+    }
+  }
+
+  public long getTimestamp() {
+    return timestamp;
+  }
+
+  public void setTimestamp(long timestamp) {
+    isSetTime = true;
+    this.timestamp = timestamp;
+  }
+
+  public TSDataType getDataType() {
+    return dataType;
+  }
+
+  public boolean isBooleanRet() {
+    return booleanRet;
+  }
+
+  public void setBooleanRet(boolean booleanRet) {
+    this.isSetValue = true;
+    this.booleanRet = booleanRet;
+  }
+
+  public int getIntRet() {
+    return intRet;
+  }
+
+  public void setIntRet(int intRet) {
+    this.isSetValue = true;
+    this.intRet = intRet;
+  }
+
+  public long getLongRet() {
+    return longRet;
+  }
+
+  public void setLongRet(long longRet) {
+    this.isSetValue = true;
+    this.longRet = longRet;
+  }
+
+  public float getFloatRet() {
+    return floatRet;
+  }
+
+  public void setFloatRet(float floatRet) {
+    this.isSetValue = true;
+    this.floatRet = floatRet;
+  }
+
+  public double getDoubleRet() {
+    return doubleRet;
+  }
+
+  public void setDoubleRet(double doubleRet) {
+    this.isSetValue = true;
+    this.doubleRet = doubleRet;
+  }
+
+  public Binary getBinaryRet() {
+    return binaryRet;
+  }
+
+  public void setBinaryRet(Binary binaryRet) {
+    this.isSetValue = true;
+    this.binaryRet = binaryRet;
+  }
+
+  public boolean isSetValue() {
+    return isSetValue;
+  }
+
+  public boolean isSetTime() {
+    return isSetTime;
+  }
+
+  public AggreResultData deepCopy() {
+    AggreResultData aggreResultData = new AggreResultData(this.dataType);
+    if (isSetValue) {
+      aggreResultData.setAnObject((Comparable<?>) this.getValue());
+    }
+    if (isSetTime) {
+      aggreResultData.setTimestamp(this.getTimestamp());
+    }
+    return aggreResultData;
+  }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/AggregateFunction.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/AggregateFunction.java
new file mode 100644
index 0000000..a9bdba2
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/AggregateFunction.java
@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.aggregation;
+
+import java.io.IOException;
+import org.apache.iotdb.db.exception.ProcessorException;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+public abstract class AggregateFunction {
+
+  protected AggreResultData resultData;
+  private TSDataType resultDataType;
+
+  /**
+   * construct.
+   *
+   * @param dataType result data type.
+   */
+  public AggregateFunction(TSDataType dataType) {
+    this.resultDataType = dataType;
+    this.resultData = new AggreResultData(dataType);
+  }
+
+  public abstract void init();
+
+  public abstract AggreResultData getResult();
+
+  /**
+   * <p>
+   * Calculate the aggregation using <code>PageHeader</code>.
+   * </p>
+   *
+   * @param pageHeader <code>PageHeader</code>
+   */
+  public abstract void calculateValueFromPageHeader(PageHeader pageHeader)
+      throws ProcessorException;
+
+  /**
+   * <p>
+   * Could not calculate using <method>calculateValueFromPageHeader</method> directly. Calculate the
+   * aggregation according to all decompressed data in this page.
+   * </p>
+   *
+   * @param dataInThisPage the data in the DataPage
+   * @param unsequenceReader unsequence data reader
+   * @throws IOException TsFile data read exception
+   * @throws ProcessorException wrong aggregation method parameter
+   */
+  public abstract void calculateValueFromPageData(BatchData dataInThisPage,
+      IPointReader unsequenceReader) throws IOException, ProcessorException;
+
+  /**
+   * <p>
+   * Could not calculate using <method>calculateValueFromPageHeader</method> directly. Calculate the
+   * aggregation according to all decompressed data in this page.
+   * </p>
+   *
+   * @param dataInThisPage the data in the DataPage
+   * @param unsequenceReader unsequence data reader
+   * @param bound the time upper bounder of data in unsequence data reader
+   * @throws IOException TsFile data read exception
+   * @throws ProcessorException wrong aggregation method parameter
+   */
+  public abstract void calculateValueFromPageData(BatchData dataInThisPage,
+      IPointReader unsequenceReader, long bound) throws IOException, ProcessorException;
+
+  /**
+   * <p>
+   * Calculate the aggregation with data in unsequenceReader.
+   * </p>
+   *
+   * @param unsequenceReader unsequence data reader
+   */
+  public abstract void calculateValueFromUnsequenceReader(IPointReader unsequenceReader)
+      throws IOException, ProcessorException;
+
+  /**
+   * <p>
+   * Calculate the aggregation with data whose timestamp is less than bound in unsequenceReader.
+   * </p>
+   *
+   * @param unsequenceReader unsequence data reader
+   * @param bound the time upper bounder of data in unsequence data reader
+   * @throws IOException TsFile data read exception
+   */
+  public abstract void calculateValueFromUnsequenceReader(IPointReader unsequenceReader, long bound)
+      throws IOException, ProcessorException;
+
+  /**
+   * <p>
+   * This method is calculate the aggregation using the common timestamps of cross series filter.
+   * </p>
+   *
+   * @throws IOException TsFile data read error
+   */
+  public abstract void calcAggregationUsingTimestamps(long[] timestamps, int length,
+      EngineReaderByTimeStamp dataReader) throws IOException;
+
+  /**
+   * Judge if aggregation results have been calculated. In other words, if the aggregated result
+   * does not need to compute the remaining data, it returns true.
+   *
+   * @return If the aggregation result has been calculated return true, else return false.
+   */
+  public abstract boolean isCalculatedAggregationResult();
+
+  /**
+   * Return data type of aggregation function result data.
+   */
+  public TSDataType getResultDataType() {
+    return resultDataType;
+  }
+
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/CountAggrFunc.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/CountAggrFunc.java
new file mode 100644
index 0000000..020aaf1
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/CountAggrFunc.java
@@ -0,0 +1,155 @@
+/**
+ * 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.iotdb.db.query.aggregation.impl;
+
+import java.io.IOException;
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CountAggrFunc extends AggregateFunction {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(CountAggrFunc.class);
+
+  public CountAggrFunc() {
+    super(TSDataType.INT64);
+  }
+
+  @Override
+  public void init() {
+    resultData.reSet();
+    resultData.setTimestamp(0);
+    resultData.setLongRet(0);
+  }
+
+  @Override
+  public AggreResultData getResult() {
+    return resultData;
+  }
+
+  @Override
+  public void calculateValueFromPageHeader(PageHeader pageHeader) {
+    if (LOGGER.isDebugEnabled()) {
+      LOGGER.debug("PageHeader>>>>>>>>>>>>num of rows:{}, minTimeStamp:{}, maxTimeStamp{}",
+          pageHeader.getNumOfValues(), pageHeader.getMinTimestamp(), pageHeader.getMaxTimestamp());
+    }
+    long preValue = resultData.getLongRet();
+    preValue += pageHeader.getNumOfValues();
+    resultData.setLongRet(preValue);
+
+  }
+
+  @Override
+  public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader)
+      throws IOException {
+    calculateValueFromPageData(dataInThisPage, unsequenceReader, false, 0);
+  }
+
+  @Override
+  public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader,
+      long bound) throws IOException {
+    calculateValueFromPageData(dataInThisPage, unsequenceReader, true, bound);
+  }
+
+  private void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader,
+      boolean hasBound, long bound) throws IOException {
+    int cnt = 0;
+    while (dataInThisPage.hasNext() && unsequenceReader.hasNext()) {
+      long minTimestamp = Math
+          .min(dataInThisPage.currentTime(), unsequenceReader.current().getTimestamp());
+      if (hasBound && minTimestamp >= bound) {
+        break;
+      }
+      if (dataInThisPage.currentTime() == unsequenceReader.current().getTimestamp()) {
+        dataInThisPage.next();
+        unsequenceReader.next();
+      } else if (dataInThisPage.currentTime() < unsequenceReader.current().getTimestamp()) {
+        dataInThisPage.next();
+      } else {
+        unsequenceReader.next();
+      }
+      cnt++;
+    }
+
+    while (dataInThisPage.hasNext()) {
+      if (hasBound && dataInThisPage.currentTime() >= bound) {
+        break;
+      }
+      dataInThisPage.next();
+      cnt++;
+    }
+    long preValue = resultData.getLongRet();
+    preValue += cnt;
+    resultData.setLongRet(preValue);
+  }
+
+  @Override
+  public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader)
+      throws IOException {
+    int cnt = 0;
+    while (unsequenceReader.hasNext()) {
+      unsequenceReader.next();
+      cnt++;
+    }
+    long preValue = resultData.getLongRet();
+    preValue += cnt;
+    resultData.setLongRet(preValue);
+  }
+
+  @Override
+  public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader, long bound)
+      throws IOException {
+    int cnt = 0;
+    while (unsequenceReader.hasNext() && unsequenceReader.current().getTimestamp() < bound) {
+      unsequenceReader.next();
+      cnt++;
+    }
+    long preValue = resultData.getLongRet();
+    preValue += cnt;
+    resultData.setLongRet(preValue);
+  }
+
+  @Override
+  public void calcAggregationUsingTimestamps(long[] timestamps, int length,
+      EngineReaderByTimeStamp dataReader) throws IOException {
+    int cnt = 0;
+    for (int i = 0; i < length; i++) {
+      Object value = dataReader.getValueInTimestamp(timestamps[i]);
+      if (value != null) {
+        cnt++;
+      }
+    }
+
+    long preValue = resultData.getLongRet();
+    preValue += cnt;
+    resultData.setLongRet(preValue);
+  }
+
+  @Override
+  public boolean isCalculatedAggregationResult() {
+    return false;
+  }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/FirstAggrFunc.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/FirstAggrFunc.java
new file mode 100644
index 0000000..3e8bb05
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/FirstAggrFunc.java
@@ -0,0 +1,154 @@
+/**
+ * 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.iotdb.db.query.aggregation.impl;
+
+import java.io.IOException;
+import org.apache.iotdb.db.exception.ProcessorException;
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+public class FirstAggrFunc extends AggregateFunction {
+
+  public FirstAggrFunc(TSDataType dataType) {
+    super(dataType);
+  }
+
+  @Override
+  public void init() {
+    resultData.reSet();
+  }
+
+  @Override
+  public AggreResultData getResult() {
+    return resultData;
+  }
+
+  @Override
+  public void calculateValueFromPageHeader(PageHeader pageHeader) throws ProcessorException {
+    if (resultData.isSetTime()) {
+      return;
+    }
+
+    Object firstVal = pageHeader.getStatistics().getFirst();
+    if (firstVal == null) {
+      throw new ProcessorException("PageHeader contains no FIRST value");
+    }
+    resultData.putTimeAndValue(0, firstVal);
+  }
+
+  @Override
+  public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader)
+      throws IOException {
+    if (resultData.isSetTime()) {
+      return;
+    }
+    if (dataInThisPage.hasNext() && unsequenceReader.hasNext()) {
+      if (dataInThisPage.currentTime() >= unsequenceReader.current().getTimestamp()) {
+        resultData.putTimeAndValue(0, unsequenceReader.current().getValue().getValue());
+        unsequenceReader.next();
+        return;
+      } else {
+        resultData.putTimeAndValue(0, dataInThisPage.currentValue());
+        dataInThisPage.next();
+        return;
+      }
+    }
+
+    if (dataInThisPage.hasNext()) {
+      resultData.putTimeAndValue(0, dataInThisPage.currentValue());
+    }
+  }
+
+  @Override
+  public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader,
+      long bound) throws IOException {
+    if (resultData.isSetTime()) {
+      return;
+    }
+    if (dataInThisPage.hasNext() && unsequenceReader.hasNext()) {
+      if (dataInThisPage.currentTime() >= unsequenceReader.current().getTimestamp()) {
+        if (unsequenceReader.current().getTimestamp() < bound) {
+          resultData.putTimeAndValue(0, unsequenceReader.current().getValue().getValue());
+          unsequenceReader.next();
+          return;
+        }
+      } else {
+        if (dataInThisPage.currentTime() < bound) {
+          resultData.putTimeAndValue(0, dataInThisPage.currentValue());
+          dataInThisPage.next();
+          return;
+        }
+      }
+    }
+
+    if (dataInThisPage.hasNext() && dataInThisPage.currentTime() < bound) {
+      resultData.putTimeAndValue(0, dataInThisPage.currentValue());
+      dataInThisPage.next();
+    }
+  }
+
+  @Override
+  public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader)
+      throws IOException {
+    if (resultData.isSetTime()) {
+      return;
+    }
+    if (unsequenceReader.hasNext()) {
+      resultData.putTimeAndValue(0, unsequenceReader.current().getValue().getValue());
+    }
+  }
+
+  @Override
+  public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader, long bound)
+      throws IOException {
+    if (resultData.isSetTime()) {
+      return;
+    }
+    if (unsequenceReader.hasNext() && unsequenceReader.current().getTimestamp() < bound) {
+      resultData.putTimeAndValue(0, unsequenceReader.current().getValue().getValue());
+    }
+  }
+
+  @Override
+  public void calcAggregationUsingTimestamps(long[] timestamps, int length,
+      EngineReaderByTimeStamp dataReader) throws IOException {
+    if (resultData.isSetTime()) {
+      return;
+    }
+
+    for (int i = 0; i < length; i++) {
+      Object value = dataReader.getValueInTimestamp(timestamps[i]);
+      if (value != null) {
+        resultData.putTimeAndValue(0, value);
+        break;
+      }
+    }
+  }
+
+  @Override
+  public boolean isCalculatedAggregationResult() {
+    return resultData.isSetTime();
+  }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/LastAggrFunc.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/LastAggrFunc.java
new file mode 100644
index 0000000..8c12728
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/LastAggrFunc.java
@@ -0,0 +1,151 @@
+/**
+ * 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.iotdb.db.query.aggregation.impl;
+
+import java.io.IOException;
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.db.utils.TimeValuePair;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+public class LastAggrFunc extends AggregateFunction {
+
+  public LastAggrFunc(TSDataType dataType) {
+    super(dataType);
+  }
+
+  @Override
+  public void init() {
+    resultData.reSet();
+  }
+
+  @Override
+  public AggreResultData getResult() {
+    if (resultData.isSetTime()) {
+      resultData.setTimestamp(0);
+    }
+    return resultData;
+  }
+
+  @Override
+  public void calculateValueFromPageHeader(PageHeader pageHeader) {
+    Object lastVal = pageHeader.getStatistics().getLast();
+    updateLastResult(pageHeader.getMaxTimestamp(), lastVal);
+  }
+
+  @Override
+  public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader)
+      throws IOException {
+    calculateValueFromPageData(dataInThisPage, unsequenceReader, Long.MAX_VALUE);
+  }
+
+  @Override
+  public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader,
+      long bound) throws IOException {
+    long time = -1;
+    Object lastVal = null;
+    while (dataInThisPage.hasNext() && dataInThisPage.currentTime() < bound) {
+      time = dataInThisPage.currentTime();
+      lastVal = dataInThisPage.currentValue();
+      dataInThisPage.next();
+    }
+
+    while (unsequenceReader.hasNext()) {
+      if (unsequenceReader.current().getTimestamp() < time) {
+        unsequenceReader.next();
+      } else if (unsequenceReader.current().getTimestamp() == time) {
+        lastVal = unsequenceReader.current().getValue().getValue();
+        unsequenceReader.next();
+      } else {
+        break;
+      }
+    }
+
+    // has inited lastVal and time in the batch(dataInThisPage).
+    if (time != -1) {
+      updateLastResult(time, lastVal);
+    }
+  }
+
+  @Override
+  public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader)
+      throws IOException {
+    TimeValuePair pair = null;
+    while (unsequenceReader.hasNext()) {
+      pair = unsequenceReader.next();
+    }
+
+    if (pair != null) {
+      updateLastResult(pair.getTimestamp(), pair.getValue().getValue());
+    }
+
+  }
+
+  @Override
+  public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader, long bound)
+      throws IOException {
+    TimeValuePair pair = null;
+    while (unsequenceReader.hasNext() && unsequenceReader.current().getTimestamp() < bound) {
+      pair = unsequenceReader.next();
+    }
+
+    if (pair != null) {
+      updateLastResult(pair.getTimestamp(), pair.getValue().getValue());
+    }
+  }
+
+  @Override
+  public void calcAggregationUsingTimestamps(long[] timestamps, int length,
+      EngineReaderByTimeStamp dataReader) throws IOException {
+
+    long time = -1;
+    Object lastVal = null;
+    for (int i = 0; i < length; i++) {
+      Object value = dataReader.getValueInTimestamp(timestamps[i]);
+      if (value != null) {
+        time = timestamps[i];
+        lastVal = value;
+      }
+    }
+    if (time != -1) {
+      updateLastResult(time, lastVal);
+    }
+  }
+
+  @Override
+  public boolean isCalculatedAggregationResult() {
+    return false;
+  }
+
+  private void updateLastResult(long time, Object value) {
+    if (!resultData.isSetTime()) {
+      resultData.putTimeAndValue(time, value);
+    } else {
+      if (time >= resultData.getTimestamp()) {
+        resultData.putTimeAndValue(time, value);
+      }
+    }
+  }
+
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MaxTimeAggrFunc.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MaxTimeAggrFunc.java
new file mode 100644
index 0000000..73b8f1c
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MaxTimeAggrFunc.java
@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.aggregation.impl;
+
+import java.io.IOException;
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.db.utils.TimeValuePair;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+public class MaxTimeAggrFunc extends AggregateFunction {
+
+  public MaxTimeAggrFunc() {
+    super(TSDataType.INT64);
+  }
+
+  @Override
+  public void init() {
+    resultData.reSet();
+  }
+
+  @Override
+  public AggreResultData getResult() {
+    return resultData;
+  }
+
+  @Override
+  public void calculateValueFromPageHeader(PageHeader pageHeader) {
+    long maxTimestamp = pageHeader.getMaxTimestamp();
+    updateMaxTimeResult(0, maxTimestamp);
+  }
+
+  @Override
+  public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader) {
+
+    int maxIndex = dataInThisPage.length() - 1;
+    if (maxIndex < 0) {
+      return;
+    }
+    long time = dataInThisPage.getTimeByIndex(maxIndex);
+    updateMaxTimeResult(0, time);
+  }
+
+  @Override
+  public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader,
+      long bound) {
+    long time = -1;
+    while (dataInThisPage.hasNext()) {
+      if (dataInThisPage.currentTime() < bound) {
+        time = dataInThisPage.currentTime();
+        dataInThisPage.next();
+      } else {
+        break;
+      }
+    }
+    if (time != -1) {
+      updateMaxTimeResult(0, time);
+    }
+  }
+
+  @Override
+  public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader)
+      throws IOException {
+    TimeValuePair pair = null;
+    while (unsequenceReader.hasNext()) {
+      pair = unsequenceReader.next();
+    }
+    if (pair != null) {
+      updateMaxTimeResult(0, pair.getTimestamp());
+    }
+  }
+
+  @Override
+  public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader, long bound)
+      throws IOException {
+    TimeValuePair pair = null;
+    while (unsequenceReader.hasNext() && unsequenceReader.current().getTimestamp() < bound) {
+      pair = unsequenceReader.next();
+    }
+    if (pair != null) {
+      updateMaxTimeResult(0, pair.getTimestamp());
+    }
+  }
+
+  //TODO Consider how to reverse order in dataReader(EngineReaderByTimeStamp)
+  @Override
+  public void calcAggregationUsingTimestamps(long[] timestamps, int length,
+      EngineReaderByTimeStamp dataReader) throws IOException {
+    long time = -1;
+    for (int i = 0; i < length; i++) {
+      Object value = dataReader.getValueInTimestamp(timestamps[i]);
+      if (value != null) {
+        time = timestamps[i];
+      }
+    }
+
+    if (time == -1) {
+      return;
+    }
+    updateMaxTimeResult(0, time);
+  }
+
+  @Override
+  public boolean isCalculatedAggregationResult() {
+    return false;
+  }
+
+  private void updateMaxTimeResult(long time, long value) {
+    if (!resultData.isSetValue() || value >= resultData.getLongRet()) {
+      resultData.setTimestamp(time);
+      resultData.setLongRet(value);
+    }
+  }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MaxValueAggrFunc.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MaxValueAggrFunc.java
new file mode 100644
index 0000000..8e2d2a7
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MaxValueAggrFunc.java
@@ -0,0 +1,170 @@
+/**
+ * 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.iotdb.db.query.aggregation.impl;
+
+import java.io.IOException;
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+public class MaxValueAggrFunc extends AggregateFunction {
+
+  public MaxValueAggrFunc(TSDataType dataType) {
+    super(dataType);
+  }
+
+  @Override
+  public void init() {
+    resultData.reSet();
+  }
+
+  @Override
+  public AggreResultData getResult() {
+    return resultData;
+  }
+
+  @Override
+  public void calculateValueFromPageHeader(PageHeader pageHeader) {
+    Comparable<Object> maxVal = (Comparable<Object>) pageHeader.getStatistics().getMax();
+    updateResult(maxVal);
+  }
+
+  @Override
+  public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader)
+      throws IOException {
+    Comparable<Object> maxVal = null;
+    Object tmpVal = null;
+    while (dataInThisPage.hasNext() && unsequenceReader.hasNext()) {
+      if (dataInThisPage.currentTime() < unsequenceReader.current().getTimestamp()) {
+        tmpVal = dataInThisPage.currentValue();
+        dataInThisPage.next();
+      } else if (dataInThisPage.currentTime() > unsequenceReader.current().getTimestamp()) {
+        tmpVal = unsequenceReader.current().getValue().getValue();
+        unsequenceReader.next();
+      } else {
+        tmpVal = unsequenceReader.current().getValue().getValue();
+        dataInThisPage.next();
+        unsequenceReader.next();
+      }
+
+      if (maxVal == null || maxVal.compareTo(tmpVal) < 0) {
+        maxVal = (Comparable<Object>) tmpVal;
+      }
+    }
+
+    while (dataInThisPage.hasNext()) {
+      if (maxVal == null || maxVal.compareTo(dataInThisPage.currentValue()) < 0) {
+        maxVal = (Comparable<Object>) dataInThisPage.currentValue();
+      }
+      dataInThisPage.next();
+    }
+    updateResult(maxVal);
+  }
+
+  @Override
+  public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader,
+      long bound) throws IOException {
+    Object tmpVal = null;
+    while (dataInThisPage.hasNext() && unsequenceReader.hasNext()) {
+      long time = Math.min(dataInThisPage.currentTime(), unsequenceReader.current().getTimestamp());
+      if (time >= bound) {
+        break;
+      }
+
+      if (dataInThisPage.currentTime() == time) {
+        tmpVal = dataInThisPage.currentValue();
+        dataInThisPage.next();
+      }
+
+      if (unsequenceReader.current().getTimestamp() == time) {
+        tmpVal = unsequenceReader.current().getValue().getValue();
+        unsequenceReader.next();
+      }
+      updateResult((Comparable<Object>) tmpVal);
+    }
+
+    while (dataInThisPage.hasNext() && dataInThisPage.currentTime() < bound) {
+      updateResult((Comparable<Object>) dataInThisPage.currentValue());
+      dataInThisPage.next();
+    }
+  }
+
+  @Override
+  public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader)
+      throws IOException {
+    Comparable<Object> maxVal = null;
+    while (unsequenceReader.hasNext()) {
+      if (maxVal == null
+          || maxVal.compareTo(unsequenceReader.current().getValue().getValue()) < 0) {
+        maxVal = (Comparable<Object>) unsequenceReader.current().getValue().getValue();
+      }
+      unsequenceReader.next();
+    }
+    updateResult(maxVal);
+  }
+
+  @Override
+  public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader, long bound)
+      throws IOException {
+    Comparable<Object> maxVal = null;
+    while (unsequenceReader.hasNext() && unsequenceReader.current().getTimestamp() < bound) {
+      if (maxVal == null
+          || maxVal.compareTo(unsequenceReader.current().getValue().getValue()) < 0) {
+        maxVal = (Comparable<Object>) unsequenceReader.current().getValue().getValue();
+      }
+      unsequenceReader.next();
+    }
+    updateResult(maxVal);
+  }
+
+  @Override
+  public void calcAggregationUsingTimestamps(long[] timestamps, int length,
+      EngineReaderByTimeStamp dataReader) throws IOException {
+    Comparable<Object> maxVal = null;
+    for (int i = 0; i < length; i++) {
+      Object value = dataReader.getValueInTimestamp(timestamps[i]);
+      if (value == null) {
+        continue;
+      }
+      if (maxVal == null || maxVal.compareTo(value) < 0) {
+        maxVal = (Comparable<Object>) value;
+      }
+    }
+    updateResult(maxVal);
+  }
+
+  @Override
+  public boolean isCalculatedAggregationResult() {
+    return false;
+  }
+
+  private void updateResult(Comparable<Object> maxVal) {
+    if (maxVal == null) {
+      return;
+    }
+    if (!resultData.isSetValue() || maxVal.compareTo(resultData.getValue()) > 0) {
+      resultData.putTimeAndValue(0, maxVal);
+    }
+  }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MeanAggrFunc.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MeanAggrFunc.java
new file mode 100644
index 0000000..da6cc95
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MeanAggrFunc.java
@@ -0,0 +1,163 @@
+/**
+ * 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.iotdb.db.query.aggregation.impl;
+
+import java.io.IOException;
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.db.utils.TimeValuePair;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+public class MeanAggrFunc extends AggregateFunction {
+
+  protected double sum = 0.0;
+  private int cnt = 0;
+  private TSDataType seriesDataType;
+
+  public MeanAggrFunc(TSDataType seriesDataType) {
+    super(TSDataType.DOUBLE);
+    this.seriesDataType = seriesDataType;
+  }
+
+  @Override
+  public void init() {
+    resultData.reSet();
+    sum = 0.0;
+    cnt = 0;
+  }
+
+  @Override
+  public AggreResultData getResult() {
+    if (cnt > 0) {
+      resultData.setTimestamp(0);
+      resultData.setDoubleRet(sum / cnt);
+    }
+    return resultData;
+  }
+
+  @Override
+  public void calculateValueFromPageHeader(PageHeader pageHeader) {
+    sum += pageHeader.getStatistics().getSum();
+    cnt += pageHeader.getNumOfValues();
+  }
+
+  @Override
+  public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader)
+      throws IOException {
+    calculateValueFromPageData(dataInThisPage, unsequenceReader, false, 0);
+  }
+
+  @Override
+  public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader,
+      long bound) throws IOException {
+    calculateValueFromPageData(dataInThisPage, unsequenceReader, true, bound);
+  }
+
+  private void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader,
+      boolean hasBound, long bound) throws IOException {
+    while (dataInThisPage.hasNext() && unsequenceReader.hasNext()) {
+      Object sumVal = null;
+      long time = Math.min(dataInThisPage.currentTime(), unsequenceReader.current().getTimestamp());
+      if (hasBound && time >= bound) {
+        break;
+      }
+      if (dataInThisPage.currentTime() < unsequenceReader.current().getTimestamp()) {
+        sumVal = dataInThisPage.currentValue();
+        dataInThisPage.next();
+      } else if (dataInThisPage.currentTime() == unsequenceReader.current().getTimestamp()) {
+        sumVal = unsequenceReader.current().getValue().getValue();
+        dataInThisPage.next();
+        unsequenceReader.next();
+      } else {
+        sumVal = unsequenceReader.current().getValue().getValue();
+        unsequenceReader.next();
+      }
+      updateMean(seriesDataType, sumVal);
+    }
+
+    while (dataInThisPage.hasNext()) {
+      if (hasBound && dataInThisPage.currentTime() >= bound) {
+        break;
+      }
+      updateMean(seriesDataType, dataInThisPage.currentValue());
+      dataInThisPage.next();
+    }
+  }
+
+  private void updateMean(TSDataType type, Object sumVal) throws IOException {
+    switch (type) {
+      case INT32:
+        sum += (int) sumVal;
+        break;
+      case INT64:
+        sum += (long) sumVal;
+        break;
+      case FLOAT:
+        sum += (float) sumVal;
+        break;
+      case DOUBLE:
+        sum += (double) sumVal;
+        break;
+      case TEXT:
+      case BOOLEAN:
+      default:
+        throw new IOException("Unsupported data type in aggregation MEAN : " + type);
+    }
+    cnt++;
+  }
+
+  @Override
+  public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader)
+      throws IOException {
+    while (unsequenceReader.hasNext()) {
+      TimeValuePair pair = unsequenceReader.next();
+      updateMean(seriesDataType, pair.getValue().getValue());
+    }
+  }
+
+  @Override
+  public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader, long bound)
+      throws IOException {
+    while (unsequenceReader.hasNext() && unsequenceReader.current().getTimestamp() < bound) {
+      TimeValuePair pair = unsequenceReader.next();
+      updateMean(seriesDataType, pair.getValue().getValue());
+    }
+  }
+
+  @Override
+  public void calcAggregationUsingTimestamps(long[] timestamps, int length,
+      EngineReaderByTimeStamp dataReader) throws IOException {
+    for (int i = 0; i < length; i++) {
+      Object value = dataReader.getValueInTimestamp(timestamps[i]);
+      if (value != null) {
+        updateMean(seriesDataType, value);
+      }
+    }
+  }
+
+  @Override
+  public boolean isCalculatedAggregationResult() {
+    return false;
+  }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MinTimeAggrFunc.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MinTimeAggrFunc.java
new file mode 100644
index 0000000..f083812
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MinTimeAggrFunc.java
@@ -0,0 +1,156 @@
+/**
+ * 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.iotdb.db.query.aggregation.impl;
+
+import java.io.IOException;
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+public class MinTimeAggrFunc extends AggregateFunction {
+
+  public MinTimeAggrFunc() {
+    super(TSDataType.INT64);
+  }
+
+  @Override
+  public void init() {
+    resultData.reSet();
+  }
+
+  @Override
+  public AggreResultData getResult() {
+    return resultData;
+  }
+
+  @Override
+  public void calculateValueFromPageHeader(PageHeader pageHeader) {
+    if (resultData.isSetValue()) {
+      return;
+    }
+    long time = pageHeader.getMinTimestamp();
+    resultData.putTimeAndValue(0, time);
+  }
+
+  @Override
+  public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader)
+      throws IOException {
+    if (resultData.isSetValue()) {
+      return;
+    }
+
+    if (dataInThisPage.hasNext() && unsequenceReader.hasNext()) {
+      if (dataInThisPage.currentTime() < unsequenceReader.current().getTimestamp()) {
+        resultData.setTimestamp(0);
+        resultData.setLongRet(dataInThisPage.currentTime());
+      } else {
+        resultData.setTimestamp(0);
+        resultData.setLongRet(unsequenceReader.current().getTimestamp());
+      }
+      return;
+    }
+
+    if (dataInThisPage.hasNext()) {
+      resultData.setTimestamp(0);
+      resultData.setLongRet(dataInThisPage.currentTime());
+    }
+  }
+
+  @Override
+  public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader,
+      long bound) throws IOException {
+    if (resultData.isSetValue()) {
+      return;
+    }
+
+    if (dataInThisPage.hasNext() && unsequenceReader.hasNext()) {
+      if (dataInThisPage.currentTime() < unsequenceReader.current().getTimestamp()) {
+        if (dataInThisPage.currentTime() >= bound) {
+          return;
+        }
+        resultData.setTimestamp(0);
+        resultData.setLongRet(dataInThisPage.currentTime());
+      } else {
+        if (unsequenceReader.current().getTimestamp() >= bound) {
+          return;
+        }
+        resultData.setTimestamp(0);
+        resultData.setLongRet(unsequenceReader.current().getTimestamp());
+      }
+      return;
+    }
+
+    if (dataInThisPage.hasNext() && dataInThisPage.currentTime() < bound) {
+      resultData.setTimestamp(0);
+      resultData.setLongRet(dataInThisPage.currentTime());
+      dataInThisPage.next();
+    }
+  }
+
+  @Override
+  public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader)
+      throws IOException {
+    if (resultData.isSetValue()) {
+      return;
+    }
+    if (unsequenceReader.hasNext()) {
+      resultData.setTimestamp(0);
+      resultData.setLongRet(unsequenceReader.current().getTimestamp());
+    }
+  }
+
+  @Override
+  public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader, long bound)
+      throws IOException {
+    if (resultData.isSetValue()) {
+      return;
+    }
+    if (unsequenceReader.hasNext() && unsequenceReader.current().getTimestamp() < bound) {
+      resultData.setTimestamp(0);
+      resultData.setLongRet(unsequenceReader.current().getTimestamp());
+    }
+  }
+
+  @Override
+  public void calcAggregationUsingTimestamps(long[] timestamps, int length,
+      EngineReaderByTimeStamp dataReader) throws IOException {
+    if (resultData.isSetValue()) {
+      return;
+    }
+    for (int i = 0; i < length; i++) {
+      Object value = dataReader.getValueInTimestamp(timestamps[i]);
+      if (value != null) {
+        resultData.setTimestamp(0);
+        resultData.setLongRet(timestamps[i]);
+        return;
+      }
+    }
+  }
+
+  @Override
+  public boolean isCalculatedAggregationResult() {
+    return resultData.isSetValue();
+  }
+
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MinValueAggrFunc.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MinValueAggrFunc.java
new file mode 100644
index 0000000..ad64fb3
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MinValueAggrFunc.java
@@ -0,0 +1,166 @@
+/**
+ * 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.iotdb.db.query.aggregation.impl;
+
+import java.io.IOException;
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+public class MinValueAggrFunc extends AggregateFunction {
+
+  public MinValueAggrFunc(TSDataType dataType) {
+    super(dataType);
+  }
+
+  @Override
+  public void init() {
+    resultData.reSet();
+  }
+
+  @Override
+  public AggreResultData getResult() {
+    return resultData;
+  }
+
+  @Override
+  public void calculateValueFromPageHeader(PageHeader pageHeader) {
+    Comparable<Object> minVal = (Comparable<Object>) pageHeader.getStatistics().getMin();
+    updateResult(minVal);
+  }
+
+  @Override
+  public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader)
+      throws IOException {
+    while (dataInThisPage.hasNext() && unsequenceReader.hasNext()) {
+      if (dataInThisPage.currentTime() < unsequenceReader.current().getTimestamp()) {
+        updateResult((Comparable<Object>) dataInThisPage.currentValue());
+        dataInThisPage.next();
+      } else if (dataInThisPage.currentTime() == unsequenceReader.current().getTimestamp()) {
+        updateResult((Comparable<Object>) unsequenceReader.current().getValue().getValue());
+        dataInThisPage.next();
+        unsequenceReader.next();
+      } else {
+        updateResult((Comparable<Object>) unsequenceReader.current().getValue().getValue());
+        unsequenceReader.next();
+      }
+    }
+
+    Comparable<Object> minVal = null;
+    while (dataInThisPage.hasNext()) {
+      if (minVal == null
+          || minVal.compareTo(dataInThisPage.currentValue()) > 0) {
+        minVal = (Comparable<Object>) dataInThisPage.currentValue();
+      }
+      dataInThisPage.next();
+    }
+    updateResult(minVal);
+  }
+
+  @Override
+  public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader,
+      long bound) throws IOException {
+    while (dataInThisPage.hasNext() && unsequenceReader.hasNext()) {
+      long time = Math.min(dataInThisPage.currentTime(), unsequenceReader.current().getTimestamp());
+      if (time >= bound) {
+        break;
+      }
+
+      if (dataInThisPage.currentTime() == time) {
+        updateResult((Comparable<Object>) dataInThisPage.currentValue());
+        dataInThisPage.next();
+      }
+
+      if (unsequenceReader.current().getTimestamp() == time) {
+        updateResult((Comparable<Object>) unsequenceReader.current().getValue().getValue());
+        unsequenceReader.next();
+      }
+
+    }
+
+    while (dataInThisPage.hasNext() && dataInThisPage.currentTime() < bound) {
+      updateResult((Comparable<Object>) dataInThisPage.currentValue());
+      dataInThisPage.next();
+    }
+  }
+
+  @Override
+  public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader)
+      throws IOException {
+    Comparable<Object> minVal = null;
+    while (unsequenceReader.hasNext()) {
+      if (minVal == null
+          || minVal.compareTo(unsequenceReader.current().getValue().getValue()) > 0) {
+        minVal = (Comparable<Object>) unsequenceReader.current().getValue().getValue();
+      }
+      unsequenceReader.next();
+    }
+    updateResult(minVal);
+  }
+
+  @Override
+  public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader, long bound)
+      throws IOException {
+    Comparable<Object> minVal = null;
+    while (unsequenceReader.hasNext() && unsequenceReader.current().getTimestamp() < bound) {
+      if (minVal == null
+          || minVal.compareTo(unsequenceReader.current().getValue().getValue()) > 0) {
+        minVal = (Comparable<Object>) unsequenceReader.current().getValue().getValue();
+      }
+      unsequenceReader.next();
+    }
+    updateResult(minVal);
+  }
+
+  @Override
+  public void calcAggregationUsingTimestamps(long[] timestamps, int length,
+      EngineReaderByTimeStamp dataReader) throws IOException {
+    Comparable<Object> minVal = null;
+    for (int i = 0; i < length; i++) {
+      Object value = dataReader.getValueInTimestamp(timestamps[i]);
+      if (value == null) {
+        continue;
+      }
+      if (minVal == null || minVal.compareTo(value) > 0) {
+        minVal = (Comparable<Object>) value;
+      }
+    }
+    updateResult(minVal);
+  }
+
+  @Override
+  public boolean isCalculatedAggregationResult() {
+    return false;
+  }
+
+  private void updateResult(Comparable<Object> minVal) {
+    if (minVal == null) {
+      return;
+    }
+    if (!resultData.isSetValue() || minVal.compareTo(resultData.getValue()) < 0) {
+      resultData.putTimeAndValue(0, minVal);
+    }
+  }
+
+}
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/SumAggrFunc.java
similarity index 64%
copy from iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java
copy to iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/SumAggrFunc.java
index 3ac1540..3bb8d7d 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/SumAggrFunc.java
@@ -1,29 +1,37 @@
-/**
- * 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.iotdb.db.query.control;
-
-import org.junit.Test;
-
-public class QueryTokenManagerTest {
-
-  @Test
-  public void test() {
-    //TODO
-  }
-}
+/**
+ * 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.iotdb.db.query.aggregation.impl;
+
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public class SumAggrFunc extends MeanAggrFunc {
+
+  public SumAggrFunc(TSDataType seriesDataType) {
+    super(seriesDataType);
+  }
+
+  @Override
+  public AggreResultData getResult() {
+    resultData.setDoubleRet(sum);
+    resultData.setTimestamp(0);
+    return resultData;
+  }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java b/iotdb/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java
index 2a7769a..cfb06ad 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java
@@ -44,6 +44,15 @@ public class QueryContext {
    */
   private Map<String, List<Modification>> fileModCache = new HashMap<>();
 
+  private long jobId;
+
+  public QueryContext() {
+  }
+
+  public QueryContext(long jobId) {
+    this.jobId = jobId;
+  }
+
   /**
    * Find the modifications of timeseries 'path' in 'modFile'. If they are not in the cache, read
    * them from 'modFile' and put then into the cache.
@@ -75,4 +84,12 @@ public class QueryContext {
 
     return pathModifications;
   }
+
+  public long getJobId() {
+    return jobId;
+  }
+
+  public void setJobId(long jobId) {
+    this.jobId = jobId;
+  }
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/control/OpenedFilePathsManager.java b/iotdb/src/main/java/org/apache/iotdb/db/query/control/JobFileManager.java
similarity index 55%
rename from iotdb/src/main/java/org/apache/iotdb/db/query/control/OpenedFilePathsManager.java
rename to iotdb/src/main/java/org/apache/iotdb/db/query/control/JobFileManager.java
index bc6ee38..84e93c6 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/control/OpenedFilePathsManager.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/control/JobFileManager.java
@@ -27,45 +27,35 @@ import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
 
 /**
  * <p>
- * Singleton pattern, to manage all query tokens. Each jdbc request has an unique job id, in this jdbc request,
- * OpenedFilePathsManager manage all the opened files, and store in the set of current job id.
+ * JobFileManager records the paths of files that every query job uses for QueryResourceManager.
+ * <p>
  */
-public class OpenedFilePathsManager {
-
-  /**
-   * Each jdbc request has an unique jod id, job id is stored in thread local variable jobIdContainer.
-   */
-  private ThreadLocal<Long> jobIdContainer;
+public class JobFileManager {
 
   /**
    * Map<jobId, Set<filePaths>>
    */
-  private ConcurrentHashMap<Long, Set<String>> closedFilePathsMap;
-  private ConcurrentHashMap<Long, Set<String>> unclosedFilePathsMap;
+  private ConcurrentHashMap<Long, Set<String>> sealedFilePathsMap;
+  private ConcurrentHashMap<Long, Set<String>> unsealedFilePathsMap;
 
-  private OpenedFilePathsManager() {
-    jobIdContainer = new ThreadLocal<>();
-    closedFilePathsMap = new ConcurrentHashMap<>();
-    unclosedFilePathsMap = new ConcurrentHashMap<>();
-  }
-
-  public static OpenedFilePathsManager getInstance() {
-    return OpenedFilePathsManagerHelper.INSTANCE;
+  public JobFileManager() {
+    sealedFilePathsMap = new ConcurrentHashMap<>();
+    unsealedFilePathsMap = new ConcurrentHashMap<>();
   }
 
   /**
-   * Set job id for current request thread. When a query request is created firstly, this method must be invoked.
+   * Set job id for current request thread. When a query request is created firstly,
+   * this method must be invoked.
    */
-  public void setJobIdForCurrentRequestThread(long jobId) {
-    jobIdContainer.set(jobId);
-    closedFilePathsMap.put(jobId, new HashSet<>());
-    unclosedFilePathsMap.put(jobId, new HashSet<>());
+  public void addJobId(long jobId) {
+    sealedFilePathsMap.computeIfAbsent(jobId, x -> new HashSet<>());
+    unsealedFilePathsMap.computeIfAbsent(jobId, x -> new HashSet<>());
   }
 
   /**
-   * Add the unique file paths to closedFilePathsMap and unclosedFilePathsMap.
+   * Add the unique file paths to sealedFilePathsMap and unsealedFilePathsMap.
    */
-  public void addUsedFilesForCurrentRequestThread(long jobId, QueryDataSource dataSource) {
+  public void addUsedFilesForGivenJob(long jobId, QueryDataSource dataSource) {
     for (TsFileResource tsFileResource : dataSource.getSeqDataSource().getSealedTsFiles()) {
       String sealedFilePath = tsFileResource.getFilePath();
       addFilePathToMap(jobId, sealedFilePath, true);
@@ -88,42 +78,30 @@ public class OpenedFilePathsManager {
    * Whenever the jdbc request is closed normally or abnormally, this method must be invoked. All file paths used by
    * this jdbc request must be cleared and thus the usage reference must be decreased.
    */
-  public void removeUsedFilesForCurrentRequestThread() {
-    if (jobIdContainer.get() != null) {
-      long jobId = jobIdContainer.get();
-      jobIdContainer.remove();
-
-      for (String filePath : closedFilePathsMap.get(jobId)) {
-        FileReaderManager.getInstance().decreaseFileReaderReference(filePath, true);
-      }
-      closedFilePathsMap.remove(jobId);
-      for (String filePath : unclosedFilePathsMap.get(jobId)) {
+  void removeUsedFilesForGivenJob(long jobId) {
+      for (String filePath : sealedFilePathsMap.get(jobId)) {
         FileReaderManager.getInstance().decreaseFileReaderReference(filePath, false);
       }
-      unclosedFilePathsMap.remove(jobId);
-    }
+      sealedFilePathsMap.remove(jobId);
+      for (String filePath : unsealedFilePathsMap.get(jobId)) {
+        FileReaderManager.getInstance().decreaseFileReaderReference(filePath, true);
+      }
+      unsealedFilePathsMap.remove(jobId);
   }
 
   /**
    * Increase the usage reference of filePath of job id. Before the invoking of this method,
    * <code>this.setJobIdForCurrentRequestThread</code> has been invoked,
-   * so <code>closedFilePathsMap.get(jobId)</code> or <code>unclosedFilePathsMap.get(jobId)</code>
+   * so <code>sealedFilePathsMap.get(jobId)</code> or <code>unsealedFilePathsMap.get(jobId)</code>
    * must not return null.
    */
-  public void addFilePathToMap(long jobId, String filePath, boolean isClosed) {
-    ConcurrentHashMap<Long, Set<String>> pathMap = !isClosed ? unclosedFilePathsMap :
-        closedFilePathsMap;
+  void addFilePathToMap(long jobId, String filePath, boolean isSealed) {
+    ConcurrentHashMap<Long, Set<String>> pathMap = !isSealed ? unsealedFilePathsMap :
+        sealedFilePathsMap;
+    //TODO this is not an atomic operation, is there concurrent problem?
     if (!pathMap.get(jobId).contains(filePath)) {
       pathMap.get(jobId).add(filePath);
-      FileReaderManager.getInstance().increaseFileReaderReference(filePath, isClosed);
-    }
-  }
-
-  private static class OpenedFilePathsManagerHelper {
-    private static final OpenedFilePathsManager INSTANCE = new OpenedFilePathsManager();
-
-    private OpenedFilePathsManagerHelper() {
-
+      FileReaderManager.getInstance().increaseFileReaderReference(filePath, isSealed);
     }
   }
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/control/QueryDataSourceManager.java b/iotdb/src/main/java/org/apache/iotdb/db/query/control/QueryDataSourceManager.java
deleted file mode 100644
index f3fd1f3..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/control/QueryDataSourceManager.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * 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.iotdb.db.query.control;
-
-import org.apache.iotdb.db.engine.filenode.FileNodeManager;
-import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
-import org.apache.iotdb.db.exception.FileNodeManagerException;
-import org.apache.iotdb.db.query.context.QueryContext;
-import org.apache.iotdb.tsfile.read.common.Path;
-import org.apache.iotdb.tsfile.read.expression.impl.SingleSeriesExpression;
-
-/**
- * <p>
- * This class is used to get query data source of a given path. See the component of
- * <code>QueryDataSource</code>
- */
-public class QueryDataSourceManager {
-
-  private static FileNodeManager fileNodeManager = FileNodeManager.getInstance();
-
-  private QueryDataSourceManager() {
-  }
-
-  public static QueryDataSource getQueryDataSource(long jobId, Path selectedPath,
-      QueryContext context)
-      throws FileNodeManagerException {
-
-    SingleSeriesExpression singleSeriesExpression = new SingleSeriesExpression(selectedPath, null);
-    QueryDataSource queryDataSource = fileNodeManager.query(singleSeriesExpression, context);
-
-    // add used files to current thread request cached map
-    OpenedFilePathsManager.getInstance()
-        .addUsedFilesForCurrentRequestThread(jobId, queryDataSource);
-
-    return queryDataSource;
-  }
-}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/control/QueryTokenManager.java b/iotdb/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java
similarity index 64%
rename from iotdb/src/main/java/org/apache/iotdb/db/query/control/QueryTokenManager.java
rename to iotdb/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java
index e304675..a535d87 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/control/QueryTokenManager.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java
@@ -24,8 +24,11 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
 import org.apache.iotdb.db.engine.filenode.FileNodeManager;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
 import org.apache.iotdb.db.exception.FileNodeManagerException;
+import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.read.expression.ExpressionType;
 import org.apache.iotdb.tsfile.read.expression.IBinaryExpression;
@@ -34,18 +37,17 @@ import org.apache.iotdb.tsfile.read.expression.impl.SingleSeriesExpression;
 
 /**
  * <p>
- * Singleton pattern, to manage all query tokens. Each jdbc query request can query multiple series,
- * in the processing of querying different device id, the <code>FileNodeManager.getInstance().
- * beginQuery</code> and <code>FileNodeManager.getInstance().endQuery</code> must be invoked in the
- * beginning and ending of jdbc request.
+ * QueryResourceManager manages resource (file streams) used by each query job, and assign Ids to the jobs.
+ * During the life cycle of a query, the following methods must be called in strict order:
+ * 1. assignJobId - get an Id for the new job.
+ * 2. beginQueryOfGivenQueryPaths - remind FileNodeManager that some files are being used
+ * 3. (if using filter)beginQueryOfGivenExpression
+ *     - remind FileNodeManager that some files are being used
+ * 4. getQueryDataSource - open files for the job or reuse existing readers.
+ * 5. endQueryForGivenJob - release the resource used by this job.
  * </p>
  */
-public class QueryTokenManager {
-
-  /**
-   * Each jdbc request has unique jod id, job id is stored in thread local variable jobContainer.
-   */
-  private ThreadLocal<Long> jobContainer;
+public class QueryResourceManager {
 
   /**
    * Map&lt;jobId, Map&lt;deviceId, List&lt;token&gt;&gt;&gt;.
@@ -72,35 +74,41 @@ public class QueryTokenManager {
    * <code>FileNodeManager.getInstance().beginQuery(device_2)</code> will be invoked again, it
    * returns result token `3` and `4` .
    *
-   * <code>FileNodeManager.getInstance().endQuery(device_1, 1)</code> and
-   * <code>FileNodeManager.getInstance().endQuery(device_2, 2)</code> must be invoked no matter how
+   * <code>FileNodeManager.getInstance().endQueryForGivenJob(device_1, 1)</code> and
+   * <code>FileNodeManager.getInstance().endQueryForGivenJob(device_2, 2)</code> must be invoked no matter how
    * query process Q1 exits normally or abnormally. So is Q2,
-   * <code>FileNodeManager.getInstance().endQuery(device_1, 3)</code> and
-   * <code>FileNodeManager.getInstance().endQuery(device_2, 4)</code> must be invoked
+   * <code>FileNodeManager.getInstance().endQueryForGivenJob(device_1, 3)</code> and
+   * <code>FileNodeManager.getInstance().endQueryForGivenJob(device_2, 4)</code> must be invoked
    *
    * Last but no least, to ensure the correctness of write process and query process of IoTDB,
    * <code>FileNodeManager.getInstance().beginQuery()</code> and
-   * <code>FileNodeManager.getInstance().endQuery()</code> must be executed rightly.
+   * <code>FileNodeManager.getInstance().endQueryForGivenJob()</code> must be executed rightly.
    * </p>
    */
   private ConcurrentHashMap<Long, ConcurrentHashMap<String, List<Integer>>> queryTokensMap;
+  private JobFileManager filePathsManager;
+  private AtomicLong maxJobId;
+
 
-  private QueryTokenManager() {
-    jobContainer = new ThreadLocal<>();
+  private QueryResourceManager() {
     queryTokensMap = new ConcurrentHashMap<>();
+    filePathsManager = new JobFileManager();
+    maxJobId = new AtomicLong(0);
   }
 
-  public static QueryTokenManager getInstance() {
+  public static QueryResourceManager getInstance() {
     return QueryTokenManagerHelper.INSTANCE;
   }
 
   /**
-   * Set job id for current request thread. When a query request is created firstly, this method
+   * Assign a jobId for a new query job. When a query request is created firstly, this method
    * must be invoked.
    */
-  public void setJobIdForCurrentRequestThread(long jobId) {
-    jobContainer.set(jobId);
-    queryTokensMap.put(jobId, new ConcurrentHashMap<>());
+  public long assignJobId() {
+    long jobId = maxJobId.incrementAndGet();
+    queryTokensMap.computeIfAbsent(jobId, x -> new ConcurrentHashMap<>());
+    filePathsManager.addJobId(jobId);
+    return jobId;
   }
 
   /**
@@ -132,22 +140,37 @@ public class QueryTokenManager {
     }
   }
 
+  public QueryDataSource getQueryDataSource(Path selectedPath,
+      QueryContext context)
+      throws FileNodeManagerException {
+
+    SingleSeriesExpression singleSeriesExpression = new SingleSeriesExpression(selectedPath, null);
+    QueryDataSource queryDataSource = FileNodeManager.getInstance()
+        .query(singleSeriesExpression, context);
+
+    // add used files to current thread request cached map
+    filePathsManager.addUsedFilesForGivenJob(context.getJobId(), queryDataSource);
+
+    return queryDataSource;
+  }
+
   /**
    * Whenever the jdbc request is closed normally or abnormally, this method must be invoked. All
    * query tokens created by this jdbc request must be cleared.
    */
-  public void endQueryForCurrentRequestThread() throws FileNodeManagerException {
-    if (jobContainer.get() != null) {
-      long jobId = jobContainer.get();
-      jobContainer.remove();
-
+  public void endQueryForGivenJob(long jobId) throws FileNodeManagerException {
+    if (queryTokensMap.get(jobId) == null) {
+      // no resource need to be released.
+      return;
+    }
       for (Map.Entry<String, List<Integer>> entry : queryTokensMap.get(jobId).entrySet()) {
         for (int token : entry.getValue()) {
           FileNodeManager.getInstance().endQuery(entry.getKey(), token);
         }
       }
       queryTokensMap.remove(jobId);
-    }
+    // remove usage of opened file paths of current thread
+    filePathsManager.removeUsedFilesForGivenJob(jobId);
   }
 
   private void getUniquePaths(IExpression expression, Set<String> deviceIdSet) {
@@ -161,15 +184,12 @@ public class QueryTokenManager {
   }
 
   private void putQueryTokenForCurrentRequestThread(long jobId, String deviceId, int queryToken) {
-    if (!queryTokensMap.get(jobId).containsKey(deviceId)) {
-      queryTokensMap.get(jobId).put(deviceId, new ArrayList<>());
-    }
-    queryTokensMap.get(jobId).get(deviceId).add(queryToken);
+    queryTokensMap.get(jobId).computeIfAbsent(deviceId, x -> new ArrayList<>()).add(queryToken);
   }
 
   private static class QueryTokenManagerHelper {
 
-    private static final QueryTokenManager INSTANCE = new QueryTokenManager();
+    private static final QueryResourceManager INSTANCE = new QueryResourceManager();
 
     private QueryTokenManagerHelper() {
     }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/AggreResultDataPointReader.java b/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/AggreResultDataPointReader.java
new file mode 100644
index 0000000..c2edab0
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/AggreResultDataPointReader.java
@@ -0,0 +1,52 @@
+/**
+ * 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.iotdb.db.query.dataset;
+
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.utils.TimeValuePair;
+import org.apache.iotdb.db.utils.TimeValuePairUtils;
+
+public class AggreResultDataPointReader implements IPointReader {
+
+  private AggreResultData aggreResultData;
+
+  public AggreResultDataPointReader(AggreResultData aggreResultData) {
+    this.aggreResultData = aggreResultData;
+  }
+
+  @Override
+  public boolean hasNext() {
+    return aggreResultData.isSetValue();
+  }
+
+  @Override
+  public TimeValuePair next() {
+    TimeValuePair timeValuePair = TimeValuePairUtils.getCurrentTimeValuePair(aggreResultData);
+    aggreResultData.reSet();
+    return timeValuePair;
+  }
+
+  @Override
+  public TimeValuePair current() {
+    return TimeValuePairUtils.getCurrentTimeValuePair(aggreResultData);
+  }
+
+  @Override
+  public void close() {
+    // batch data doesn't need to close.
+  }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGenerator.java b/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGenerator.java
index 20d93e8..cdaa0fb 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGenerator.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGenerator.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.util.List;
 import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
 import org.apache.iotdb.db.query.timegenerator.EngineTimeGenerator;
-import org.apache.iotdb.db.utils.TsPrimitiveType;
 import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.common.Field;
@@ -62,11 +61,11 @@ public class EngineDataSetWithTimeGenerator extends QueryDataSet {
     RowRecord rowRecord = new RowRecord(timestamp);
     for (int i = 0; i < readers.size(); i++) {
       EngineReaderByTimeStamp reader = readers.get(i);
-      TsPrimitiveType tsPrimitiveType = reader.getValueInTimestamp(timestamp);
-      if (tsPrimitiveType == null) {
+      Object value = reader.getValueInTimestamp(timestamp);
+      if (value == null) {
         rowRecord.addField(new Field(null));
       } else {
-        rowRecord.addField(getField(tsPrimitiveType.getValue(), dataTypes.get(i)));
+        rowRecord.addField(getField(value, dataTypes.get(i)));
       }
     }
 
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithoutTimeGenerator.java b/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithoutTimeGenerator.java
index 4c0ccb7..73fc71f 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithoutTimeGenerator.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithoutTimeGenerator.java
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.query.dataset;
 
 import java.io.IOException;
@@ -23,7 +24,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.PriorityQueue;
 import java.util.Set;
-import org.apache.iotdb.db.query.reader.IReader;
+import org.apache.iotdb.db.query.reader.IPointReader;
 import org.apache.iotdb.db.utils.TimeValuePair;
 import org.apache.iotdb.db.utils.TsPrimitiveType;
 import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
@@ -34,11 +35,11 @@ import org.apache.iotdb.tsfile.read.common.RowRecord;
 import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
 
 /**
- * TODO implement this class as TsFile DataSetWithoutTimeGenerator
+ * TODO implement this class as TsFile DataSetWithoutTimeGenerator.
  */
 public class EngineDataSetWithoutTimeGenerator extends QueryDataSet {
 
-  private List<IReader> readers;
+  private List<IPointReader> readers;
 
   private TimeValuePair[] cacheTimeValueList;
 
@@ -55,7 +56,7 @@ public class EngineDataSetWithoutTimeGenerator extends QueryDataSet {
    * @throws IOException IOException
    */
   public EngineDataSetWithoutTimeGenerator(List<Path> paths, List<TSDataType> dataTypes,
-      List<IReader> readers)
+      List<IPointReader> readers)
       throws IOException {
     super(paths, dataTypes);
     this.readers = readers;
@@ -68,7 +69,7 @@ public class EngineDataSetWithoutTimeGenerator extends QueryDataSet {
     cacheTimeValueList = new TimeValuePair[readers.size()];
 
     for (int i = 0; i < readers.size(); i++) {
-      IReader reader = readers.get(i);
+      IPointReader reader = readers.get(i);
       if (reader.hasNext()) {
         TimeValuePair timeValuePair = reader.next();
         cacheTimeValueList[i] = timeValuePair;
@@ -89,7 +90,7 @@ public class EngineDataSetWithoutTimeGenerator extends QueryDataSet {
     RowRecord record = new RowRecord(minTime);
 
     for (int i = 0; i < readers.size(); i++) {
-      IReader reader = readers.get(i);
+      IPointReader reader = readers.get(i);
       if (cacheTimeValueList[i] == null) {
         record.addField(new Field(null));
       } else {
@@ -109,6 +110,9 @@ public class EngineDataSetWithoutTimeGenerator extends QueryDataSet {
   }
 
   private Field getField(TsPrimitiveType tsPrimitiveType, TSDataType dataType) {
+    if (tsPrimitiveType == null) {
+      return new Field(null);
+    }
     Field field = new Field(dataType);
     switch (dataType) {
       case INT32:
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/AggregateEngineExecutor.java b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/AggregateEngineExecutor.java
new file mode 100644
index 0000000..ba76c2f
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/AggregateEngineExecutor.java
@@ -0,0 +1,330 @@
+/**
+ * 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.iotdb.db.query.executor;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.exception.FileNodeManagerException;
+import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.exception.ProcessorException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.query.aggregation.AggreFuncFactory;
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+import org.apache.iotdb.db.query.aggregation.impl.LastAggrFunc;
+import org.apache.iotdb.db.query.aggregation.impl.MaxTimeAggrFunc;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.dataset.AggreResultDataPointReader;
+import org.apache.iotdb.db.query.dataset.EngineDataSetWithoutTimeGenerator;
+import org.apache.iotdb.db.query.factory.SeriesReaderFactory;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.db.query.reader.merge.PriorityMergeReader;
+import org.apache.iotdb.db.query.reader.sequence.SequenceDataReader;
+import org.apache.iotdb.db.query.timegenerator.EngineTimeGenerator;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.expression.IExpression;
+import org.apache.iotdb.tsfile.read.expression.impl.GlobalTimeExpression;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+
+public class AggregateEngineExecutor {
+
+  private List<Path> selectedSeries;
+  private List<String> aggres;
+  private IExpression expression;
+
+  /**
+   * aggregation batch calculation size.
+   **/
+  private int aggregateFetchSize;
+
+  /**
+   * constructor.
+   */
+  public AggregateEngineExecutor(List<Path> selectedSeries, List<String> aggres,
+      IExpression expression) {
+    this.selectedSeries = selectedSeries;
+    this.aggres = aggres;
+    this.expression = expression;
+    this.aggregateFetchSize = 10 * IoTDBDescriptor.getInstance().getConfig().getFetchSize();
+  }
+
+  /**
+   * execute aggregate function with only time filter or no filter.
+   *
+   * @param context query context
+   */
+  public QueryDataSet executeWithOutTimeGenerator(QueryContext context)
+      throws FileNodeManagerException, IOException, PathErrorException, ProcessorException {
+    Filter timeFilter = null;
+    if (expression != null) {
+      timeFilter = ((GlobalTimeExpression) expression).getFilter();
+    }
+    QueryResourceManager
+        .getInstance().beginQueryOfGivenQueryPaths(context.getJobId(), selectedSeries);
+
+    List<SequenceDataReader> readersOfSequenceData = new ArrayList<>();
+    List<IPointReader> readersOfUnSequenceData = new ArrayList<>();
+    List<AggregateFunction> aggregateFunctions = new ArrayList<>();
+    for (int i = 0; i < selectedSeries.size(); i++) {
+      // construct AggregateFunction
+      TSDataType tsDataType = MManager.getInstance()
+          .getSeriesType(selectedSeries.get(i).getFullPath());
+      AggregateFunction function = AggreFuncFactory.getAggrFuncByName(aggres.get(i), tsDataType);
+      function.init();
+      aggregateFunctions.add(function);
+
+      QueryDataSource queryDataSource = QueryResourceManager.getInstance()
+          .getQueryDataSource(selectedSeries.get(i), context);
+
+      // sequence reader for sealed tsfile, unsealed tsfile, memory
+      SequenceDataReader sequenceReader;
+      if (function instanceof MaxTimeAggrFunc || function instanceof LastAggrFunc) {
+        sequenceReader = new SequenceDataReader(queryDataSource.getSeqDataSource(), timeFilter,
+            context, true);
+      } else {
+        sequenceReader = new SequenceDataReader(queryDataSource.getSeqDataSource(), timeFilter,
+            context, false);
+      }
+
+      // unseq reader for all chunk groups in unSeqFile, memory
+      PriorityMergeReader unSeqMergeReader = SeriesReaderFactory.getInstance()
+          .createUnSeqMergeReader(queryDataSource.getOverflowSeriesDataSource(), timeFilter);
+      readersOfSequenceData.add(sequenceReader);
+      readersOfUnSequenceData.add(unSeqMergeReader);
+    }
+    List<AggreResultData> aggreResultDataList = new ArrayList<>();
+    //TODO use multi-thread
+    for (int i = 0; i < selectedSeries.size(); i++) {
+      AggreResultData aggreResultData = aggregateWithOutTimeGenerator(aggregateFunctions.get(i),
+          readersOfSequenceData.get(i), readersOfUnSequenceData.get(i), timeFilter);
+      aggreResultDataList.add(aggreResultData);
+    }
+    return constructDataSet(aggreResultDataList);
+  }
+
+  /**
+   * calculation aggregate result with only time filter or no filter for one series.
+   *
+   * @param function aggregate function
+   * @param sequenceReader sequence data reader
+   * @param unSequenceReader unsequence data reader
+   * @param filter time filter or null
+   * @return one series aggregate result data
+   */
+  private AggreResultData aggregateWithOutTimeGenerator(AggregateFunction function,
+      SequenceDataReader sequenceReader, IPointReader unSequenceReader, Filter filter)
+      throws IOException, ProcessorException {
+    if (function instanceof MaxTimeAggrFunc || function instanceof LastAggrFunc) {
+      return handleLastMaxTimeWithOutTimeGenerator(function, sequenceReader, unSequenceReader,
+          filter);
+    }
+
+    while (sequenceReader.hasNext()) {
+      PageHeader pageHeader = sequenceReader.nextPageHeader();
+      // judge if overlap with unsequence data
+      if (canUseHeader(function, pageHeader, unSequenceReader, filter)) {
+        // cal by pageHeader
+        function.calculateValueFromPageHeader(pageHeader);
+        sequenceReader.skipPageData();
+      } else {
+        // cal by pageData
+        function.calculateValueFromPageData(sequenceReader.nextBatch(), unSequenceReader);
+      }
+
+      if (function.isCalculatedAggregationResult()) {
+        return function.getResult();
+      }
+    }
+
+    // cal with unsequence data
+    if (unSequenceReader.hasNext()) {
+      function.calculateValueFromUnsequenceReader(unSequenceReader);
+    }
+    return function.getResult();
+  }
+
+  /**
+   * determine whether pageHeader can be used to compute aggregation results.
+   */
+  private boolean canUseHeader(AggregateFunction function, PageHeader pageHeader,
+      IPointReader unSequenceReader, Filter filter)
+      throws IOException, ProcessorException {
+    // if page data is memory data.
+    if (pageHeader == null) {
+      return false;
+    }
+
+    long minTime = pageHeader.getMinTimestamp();
+    long maxTime = pageHeader.getMaxTimestamp();
+
+    // If there are points in the page that do not satisfy the time filter,
+    // page header cannot be used to calculate.
+    if (filter != null && !filter.containStartEndTime(minTime, maxTime)) {
+      return false;
+    }
+
+    // cal unsequence data with timestamps between pages.
+    function.calculateValueFromUnsequenceReader(unSequenceReader, minTime);
+
+    return !(unSequenceReader.hasNext() && unSequenceReader.current().getTimestamp() <= maxTime);
+
+  }
+
+  /**
+   * handle last and max_time aggregate function with only time filter or no filter.
+   *
+   * @param function aggregate function
+   * @param sequenceReader sequence data reader
+   * @param unSequenceReader unsequence data reader
+   * @return BatchData-aggregate result
+   */
+  private AggreResultData handleLastMaxTimeWithOutTimeGenerator(AggregateFunction function,
+      SequenceDataReader sequenceReader, IPointReader unSequenceReader, Filter timeFilter)
+      throws IOException, ProcessorException {
+    long lastBatchTimeStamp = Long.MIN_VALUE;
+    boolean isChunkEnd = false;
+    while (sequenceReader.hasNext()) {
+      PageHeader pageHeader = sequenceReader.nextPageHeader();
+      // judge if overlap with unsequence data
+      if (canUseHeader(function, pageHeader, unSequenceReader, timeFilter)) {
+        // cal by pageHeader
+        function.calculateValueFromPageHeader(pageHeader);
+        sequenceReader.skipPageData();
+
+        if (lastBatchTimeStamp > pageHeader.getMinTimestamp()) {
+          // the chunk is end.
+          isChunkEnd = true;
+        } else {
+          // current page and last page are in the same chunk.
+          lastBatchTimeStamp = pageHeader.getMinTimestamp();
+        }
+      } else {
+        // cal by pageData
+        BatchData batchData = sequenceReader.nextBatch();
+        if (lastBatchTimeStamp > batchData.currentTime()) {
+          // the chunk is end.
+          isChunkEnd = true;
+        } else {
+          // current page and last page are in the same chunk.
+          lastBatchTimeStamp = batchData.currentTime();
+        }
+        function.calculateValueFromPageData(batchData, unSequenceReader);
+      }
+      if (isChunkEnd) {
+        break;
+      }
+    }
+
+    // cal with unsequence data
+    if (unSequenceReader.hasNext()) {
+      function.calculateValueFromUnsequenceReader(unSequenceReader);
+    }
+    return function.getResult();
+  }
+
+
+  /**
+   * execute aggregate function with value filter.
+   *
+   * @param context query context.
+   */
+  public QueryDataSet executeWithTimeGenerator(QueryContext context)
+      throws FileNodeManagerException, PathErrorException, IOException, ProcessorException {
+    QueryResourceManager
+        .getInstance().beginQueryOfGivenQueryPaths(context.getJobId(), selectedSeries);
+    QueryResourceManager.getInstance().beginQueryOfGivenExpression(context.getJobId(), expression);
+
+    EngineTimeGenerator timestampGenerator = new EngineTimeGenerator(expression, context);
+    List<EngineReaderByTimeStamp> readersOfSelectedSeries = SeriesReaderFactory
+        .getByTimestampReadersOfSelectedPaths(selectedSeries, context);
+
+    List<AggregateFunction> aggregateFunctions = new ArrayList<>();
+    for (int i = 0; i < selectedSeries.size(); i++) {
+      TSDataType type = MManager.getInstance().getSeriesType(selectedSeries.get(i).getFullPath());
+      AggregateFunction function = AggreFuncFactory.getAggrFuncByName(aggres.get(i), type);
+      function.init();
+      aggregateFunctions.add(function);
+    }
+    List<AggreResultData> batchDataList = aggregateWithTimeGenerator(aggregateFunctions,
+        timestampGenerator,
+        readersOfSelectedSeries);
+    return constructDataSet(batchDataList);
+  }
+
+  /**
+   * calculation aggregate result with value filter.
+   */
+  private List<AggreResultData> aggregateWithTimeGenerator(
+      List<AggregateFunction> aggregateFunctions,
+      EngineTimeGenerator timestampGenerator,
+      List<EngineReaderByTimeStamp> readersOfSelectedSeries)
+      throws IOException {
+
+    while (timestampGenerator.hasNext()) {
+
+      // generate timestamps for aggregate
+      long[] timeArray = new long[aggregateFetchSize];
+      int timeArrayLength = 0;
+      for (int cnt = 0; cnt < aggregateFetchSize; cnt++) {
+        if (!timestampGenerator.hasNext()) {
+          break;
+        }
+        timeArray[timeArrayLength++] = timestampGenerator.next();
+      }
+
+      // cal part of aggregate result
+      for (int i = 0; i < readersOfSelectedSeries.size(); i++) {
+        aggregateFunctions.get(i).calcAggregationUsingTimestamps(timeArray, timeArrayLength,
+            readersOfSelectedSeries.get(i));
+      }
+    }
+
+    List<AggreResultData> aggreResultDataArrayList = new ArrayList<>();
+    for (AggregateFunction function : aggregateFunctions) {
+      aggreResultDataArrayList.add(function.getResult());
+    }
+    return aggreResultDataArrayList;
+  }
+
+  /**
+   * using aggregate result data list construct QueryDataSet.
+   *
+   * @param aggreResultDataList aggregate result data list
+   */
+  private QueryDataSet constructDataSet(List<AggreResultData> aggreResultDataList)
+      throws IOException {
+    List<TSDataType> dataTypes = new ArrayList<>();
+    List<IPointReader> resultDataPointReaders = new ArrayList<>();
+    for (AggreResultData resultData : aggreResultDataList) {
+      dataTypes.add(resultData.getDataType());
+      resultDataPointReaders.add(new AggreResultDataPointReader(resultData));
+    }
+    return new EngineDataSetWithoutTimeGenerator(selectedSeries, dataTypes, resultDataPointReaders);
+  }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithTimeGenerator.java b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithTimeGenerator.java
index 13f0053..3b40187 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithTimeGenerator.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithTimeGenerator.java
@@ -16,24 +16,20 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.query.executor;
 
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
 import org.apache.iotdb.db.exception.FileNodeManagerException;
 import org.apache.iotdb.db.exception.PathErrorException;
 import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.query.context.QueryContext;
-import org.apache.iotdb.db.query.control.QueryDataSourceManager;
-import org.apache.iotdb.db.query.control.QueryTokenManager;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
 import org.apache.iotdb.db.query.dataset.EngineDataSetWithTimeGenerator;
 import org.apache.iotdb.db.query.factory.SeriesReaderFactory;
 import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
-import org.apache.iotdb.db.query.reader.merge.PriorityMergeReader;
-import org.apache.iotdb.db.query.reader.merge.PriorityMergeReaderByTimestamp;
-import org.apache.iotdb.db.query.reader.sequence.SequenceDataReader;
 import org.apache.iotdb.db.query.timegenerator.EngineTimeGenerator;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.common.Path;
@@ -46,10 +42,8 @@ import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
 public class EngineExecutorWithTimeGenerator {
 
   private QueryExpression queryExpression;
-  private long jobId;
 
-  EngineExecutorWithTimeGenerator(long jobId, QueryExpression queryExpression) {
-    this.jobId = jobId;
+  EngineExecutorWithTimeGenerator(QueryExpression queryExpression) {
     this.queryExpression = queryExpression;
   }
 
@@ -62,17 +56,17 @@ public class EngineExecutorWithTimeGenerator {
    */
   public QueryDataSet execute(QueryContext context) throws FileNodeManagerException {
 
-    QueryTokenManager.getInstance()
-        .beginQueryOfGivenQueryPaths(jobId, queryExpression.getSelectedSeries());
-    QueryTokenManager.getInstance()
-        .beginQueryOfGivenExpression(jobId, queryExpression.getExpression());
+    QueryResourceManager.getInstance()
+        .beginQueryOfGivenQueryPaths(context.getJobId(), queryExpression.getSelectedSeries());
+    QueryResourceManager.getInstance()
+        .beginQueryOfGivenExpression(context.getJobId(), queryExpression.getExpression());
 
     EngineTimeGenerator timestampGenerator;
     List<EngineReaderByTimeStamp> readersOfSelectedSeries;
     try {
-      timestampGenerator = new EngineTimeGenerator(jobId, queryExpression.getExpression(), context);
-      readersOfSelectedSeries = getReadersOfSelectedPaths(queryExpression.getSelectedSeries(),
-          context);
+      timestampGenerator = new EngineTimeGenerator(queryExpression.getExpression(), context);
+      readersOfSelectedSeries = SeriesReaderFactory
+          .getByTimestampReadersOfSelectedPaths(queryExpression.getSelectedSeries(), context);
     } catch (IOException ex) {
       throw new FileNodeManagerException(ex);
     }
@@ -92,33 +86,4 @@ public class EngineExecutorWithTimeGenerator {
         readersOfSelectedSeries);
   }
 
-  private List<EngineReaderByTimeStamp> getReadersOfSelectedPaths(List<Path> paths,
-      QueryContext context)
-      throws IOException, FileNodeManagerException {
-
-    List<EngineReaderByTimeStamp> readersOfSelectedSeries = new ArrayList<>();
-
-    for (Path path : paths) {
-
-      QueryDataSource queryDataSource = QueryDataSourceManager.getQueryDataSource(jobId, path,
-          context);
-
-      PriorityMergeReaderByTimestamp mergeReaderByTimestamp = new PriorityMergeReaderByTimestamp();
-
-      // reader for sequence data
-      SequenceDataReader tsFilesReader = new SequenceDataReader(queryDataSource.getSeqDataSource(),
-          null, context);
-      mergeReaderByTimestamp.addReaderWithPriority(tsFilesReader, 1);
-
-      // reader for unSequence data
-      PriorityMergeReader unSeqMergeReader = SeriesReaderFactory.getInstance()
-          .createUnSeqMergeReader(queryDataSource.getOverflowSeriesDataSource(), null);
-      mergeReaderByTimestamp.addReaderWithPriority(unSeqMergeReader, 2);
-
-      readersOfSelectedSeries.add(mergeReaderByTimestamp);
-    }
-
-    return readersOfSelectedSeries;
-  }
-
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithoutTimeGenerator.java b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithoutTimeGenerator.java
index 11c94bc..2d07796 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithoutTimeGenerator.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithoutTimeGenerator.java
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.query.executor;
 
 import java.io.IOException;
@@ -26,11 +27,11 @@ import org.apache.iotdb.db.exception.FileNodeManagerException;
 import org.apache.iotdb.db.exception.PathErrorException;
 import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.query.context.QueryContext;
-import org.apache.iotdb.db.query.control.QueryDataSourceManager;
-import org.apache.iotdb.db.query.control.QueryTokenManager;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
 import org.apache.iotdb.db.query.dataset.EngineDataSetWithoutTimeGenerator;
 import org.apache.iotdb.db.query.factory.SeriesReaderFactory;
-import org.apache.iotdb.db.query.reader.IReader;
+import org.apache.iotdb.db.query.reader.AllDataReader;
+import org.apache.iotdb.db.query.reader.IPointReader;
 import org.apache.iotdb.db.query.reader.merge.PriorityMergeReader;
 import org.apache.iotdb.db.query.reader.sequence.SequenceDataReader;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -46,10 +47,8 @@ import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
 public class EngineExecutorWithoutTimeGenerator {
 
   private QueryExpression queryExpression;
-  private long jobId;
 
-  public EngineExecutorWithoutTimeGenerator(long jobId, QueryExpression queryExpression) {
-    this.jobId = jobId;
+  public EngineExecutorWithoutTimeGenerator(QueryExpression queryExpression) {
     this.queryExpression = queryExpression;
   }
 
@@ -61,15 +60,15 @@ public class EngineExecutorWithoutTimeGenerator {
 
     Filter timeFilter = ((GlobalTimeExpression) queryExpression.getExpression()).getFilter();
 
-    List<IReader> readersOfSelectedSeries = new ArrayList<>();
+    List<IPointReader> readersOfSelectedSeries = new ArrayList<>();
     List<TSDataType> dataTypes = new ArrayList<>();
 
-    QueryTokenManager.getInstance()
-        .beginQueryOfGivenQueryPaths(jobId, queryExpression.getSelectedSeries());
+    QueryResourceManager.getInstance()
+        .beginQueryOfGivenQueryPaths(context.getJobId(), queryExpression.getSelectedSeries());
 
     for (Path path : queryExpression.getSelectedSeries()) {
 
-      QueryDataSource queryDataSource = QueryDataSourceManager.getQueryDataSource(jobId, path,
+      QueryDataSource queryDataSource = QueryResourceManager.getInstance().getQueryDataSource(path,
           context);
 
       // add data type
@@ -79,14 +78,11 @@ public class EngineExecutorWithoutTimeGenerator {
         throw new FileNodeManagerException(e);
       }
 
-      PriorityMergeReader priorityReader = new PriorityMergeReader();
-
       // sequence reader for one sealed tsfile
       SequenceDataReader tsFilesReader;
       try {
         tsFilesReader = new SequenceDataReader(queryDataSource.getSeqDataSource(),
             timeFilter, context);
-        priorityReader.addReaderWithPriority(tsFilesReader, PriorityMergeReader.LOW_PRIORITY);
       } catch (IOException e) {
         throw new FileNodeManagerException(e);
       }
@@ -96,12 +92,12 @@ public class EngineExecutorWithoutTimeGenerator {
       try {
         unSeqMergeReader = SeriesReaderFactory.getInstance()
             .createUnSeqMergeReader(queryDataSource.getOverflowSeriesDataSource(), timeFilter);
-        priorityReader.addReaderWithPriority(unSeqMergeReader, PriorityMergeReader.HIGH_PRIORITY);
       } catch (IOException e) {
         throw new FileNodeManagerException(e);
       }
 
-      readersOfSelectedSeries.add(priorityReader);
+      // merge sequence data with unsequence data.
+      readersOfSelectedSeries.add(new AllDataReader(tsFilesReader, unSeqMergeReader));
     }
 
     try {
@@ -118,15 +114,15 @@ public class EngineExecutorWithoutTimeGenerator {
   public QueryDataSet executeWithoutFilter(QueryContext context)
       throws FileNodeManagerException {
 
-    List<IReader> readersOfSelectedSeries = new ArrayList<>();
+    List<IPointReader> readersOfSelectedSeries = new ArrayList<>();
     List<TSDataType> dataTypes = new ArrayList<>();
 
-    QueryTokenManager.getInstance()
-        .beginQueryOfGivenQueryPaths(jobId, queryExpression.getSelectedSeries());
+    QueryResourceManager.getInstance()
+        .beginQueryOfGivenQueryPaths(context.getJobId(), queryExpression.getSelectedSeries());
 
     for (Path path : queryExpression.getSelectedSeries()) {
 
-      QueryDataSource queryDataSource = QueryDataSourceManager.getQueryDataSource(jobId, path,
+      QueryDataSource queryDataSource = QueryResourceManager.getInstance().getQueryDataSource(path,
           context);
 
       // add data type
@@ -136,14 +132,11 @@ public class EngineExecutorWithoutTimeGenerator {
         throw new FileNodeManagerException(e);
       }
 
-      PriorityMergeReader priorityReader = new PriorityMergeReader();
-
       // sequence insert data
       SequenceDataReader tsFilesReader;
       try {
         tsFilesReader = new SequenceDataReader(queryDataSource.getSeqDataSource(),
             null, context);
-        priorityReader.addReaderWithPriority(tsFilesReader, 1);
       } catch (IOException e) {
         throw new FileNodeManagerException(e);
       }
@@ -153,12 +146,12 @@ public class EngineExecutorWithoutTimeGenerator {
       try {
         unSeqMergeReader = SeriesReaderFactory.getInstance()
             .createUnSeqMergeReader(queryDataSource.getOverflowSeriesDataSource(), null);
-        priorityReader.addReaderWithPriority(unSeqMergeReader, 2);
       } catch (IOException e) {
         throw new FileNodeManagerException(e);
       }
 
-      readersOfSelectedSeries.add(priorityReader);
+      // merge sequence data with unsequence data.
+      readersOfSelectedSeries.add(new AllDataReader(tsFilesReader, unSeqMergeReader));
     }
 
     try {
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineQueryRouter.java b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineQueryRouter.java
index c9327f4..96ad5ad 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineQueryRouter.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineQueryRouter.java
@@ -16,22 +16,32 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.query.executor;
 
-import static org.apache.iotdb.tsfile.read.expression.ExpressionType.GLOBAL_TIME;
+package org.apache.iotdb.db.query.executor;
 
 import java.io.IOException;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
 import org.apache.iotdb.db.exception.FileNodeManagerException;
 import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.exception.ProcessorException;
 import org.apache.iotdb.db.query.context.QueryContext;
-import org.apache.iotdb.db.query.control.OpenedFilePathsManager;
-import org.apache.iotdb.db.query.control.QueryTokenManager;
+import org.apache.iotdb.db.query.executor.groupby.GroupByWithOnlyTimeFilterDataSetDataSet;
+import org.apache.iotdb.db.query.executor.groupby.GroupByWithValueFilterDataSetDataSet;
+import org.apache.iotdb.db.query.fill.IFill;
 import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.expression.ExpressionType;
 import org.apache.iotdb.tsfile.read.expression.IExpression;
 import org.apache.iotdb.tsfile.read.expression.QueryExpression;
+import org.apache.iotdb.tsfile.read.expression.impl.BinaryExpression;
+import org.apache.iotdb.tsfile.read.expression.impl.GlobalTimeExpression;
 import org.apache.iotdb.tsfile.read.expression.util.ExpressionOptimizer;
+import org.apache.iotdb.tsfile.read.filter.TimeFilter;
 import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+import org.apache.iotdb.tsfile.utils.Pair;
 
 /**
  * Query entrance class of IoTDB query process. All query clause will be transformed to physical
@@ -39,40 +49,26 @@ import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
  */
 public class EngineQueryRouter {
 
-  /**
-   * Each unique jdbc request(query, aggregation or others job) has an unique job id. This job id
-   * will always be maintained until the request is closed. In each job, the unique file will be
-   * only opened once to avoid too many opened files error.
-   */
-  private AtomicLong jobIdGenerator = new AtomicLong();
+
 
   /**
    * execute physical plan.
    */
-  public QueryDataSet query(QueryExpression queryExpression)
+  public QueryDataSet query(QueryExpression queryExpression, QueryContext context)
       throws FileNodeManagerException {
 
-    long nextJobId = getNextJobId();
-    QueryTokenManager.getInstance().setJobIdForCurrentRequestThread(nextJobId);
-    OpenedFilePathsManager.getInstance().setJobIdForCurrentRequestThread(nextJobId);
-
-    QueryContext context = new QueryContext();
-
     if (queryExpression.hasQueryFilter()) {
       try {
         IExpression optimizedExpression = ExpressionOptimizer.getInstance()
             .optimize(queryExpression.getExpression(), queryExpression.getSelectedSeries());
         queryExpression.setExpression(optimizedExpression);
 
-        if (optimizedExpression.getType() == GLOBAL_TIME) {
+        if (optimizedExpression.getType() == ExpressionType.GLOBAL_TIME) {
           EngineExecutorWithoutTimeGenerator engineExecutor =
-              new EngineExecutorWithoutTimeGenerator(
-
-                  nextJobId, queryExpression);
+              new EngineExecutorWithoutTimeGenerator(queryExpression);
           return engineExecutor.executeWithGlobalTimeFilter(context);
         } else {
           EngineExecutorWithTimeGenerator engineExecutor = new EngineExecutorWithTimeGenerator(
-              nextJobId,
               queryExpression);
           return engineExecutor.execute(context);
         }
@@ -82,13 +78,145 @@ public class EngineQueryRouter {
       }
     } else {
       EngineExecutorWithoutTimeGenerator engineExecutor = new EngineExecutorWithoutTimeGenerator(
-          nextJobId,
           queryExpression);
       return engineExecutor.executeWithoutFilter(context);
     }
   }
 
-  private synchronized long getNextJobId() {
-    return jobIdGenerator.incrementAndGet();
+  /**
+   * execute aggregation query.
+   */
+  public QueryDataSet aggregate(List<Path> selectedSeries, List<String> aggres,
+      IExpression expression, QueryContext context) throws QueryFilterOptimizationException,
+      FileNodeManagerException, IOException, PathErrorException, ProcessorException {
+
+    if (expression != null) {
+      IExpression optimizedExpression = ExpressionOptimizer.getInstance()
+          .optimize(expression, selectedSeries);
+      AggregateEngineExecutor engineExecutor = new AggregateEngineExecutor(
+          selectedSeries, aggres, optimizedExpression);
+      if (optimizedExpression.getType() == ExpressionType.GLOBAL_TIME) {
+        return engineExecutor.executeWithOutTimeGenerator(context);
+      } else {
+        return engineExecutor.executeWithTimeGenerator(context);
+      }
+    } else {
+      AggregateEngineExecutor engineExecutor = new AggregateEngineExecutor(
+          selectedSeries, aggres, null);
+      return engineExecutor.executeWithOutTimeGenerator(context);
+    }
   }
+
+  /**
+   * execute groupBy query.
+   *
+   * @param selectedSeries select path list
+   * @param aggres aggregation name list
+   * @param expression filter expression
+   * @param unit time granularity for interval partitioning, unit is ms.
+   * @param origin the datum time point for interval division is divided into a time interval for
+   * each TimeUnit time from this point forward and backward.
+   * @param intervals time intervals, closed interval.
+   */
+  public QueryDataSet groupBy(List<Path> selectedSeries, List<String> aggres,
+      IExpression expression, long unit, long origin, List<Pair<Long, Long>> intervals,
+      QueryContext context)
+      throws ProcessorException, QueryFilterOptimizationException, FileNodeManagerException,
+      PathErrorException, IOException {
+
+    long nextJobId = context.getJobId();
+
+    // check the legitimacy of intervals
+    for (Pair<Long, Long> pair : intervals) {
+      if (!(pair.left > 0 && pair.right > 0)) {
+        throw new ProcessorException(
+            String.format("Time interval<%d, %d> must be greater than 0.", pair.left, pair.right));
+      }
+      if (pair.right < pair.left) {
+        throw new ProcessorException(String.format(
+            "Interval starting time must be greater than the interval ending time, "
+                + "found error interval<%d, %d>", pair.left, pair.right));
+      }
+    }
+    // merge intervals
+    List<Pair<Long, Long>> mergedIntervalList = mergeInterval(intervals);
+
+    // construct groupBy intervals filter
+    BinaryExpression intervalFilter = null;
+    for (Pair<Long, Long> pair : mergedIntervalList) {
+      BinaryExpression pairFilter = BinaryExpression
+          .and(new GlobalTimeExpression(TimeFilter.gtEq(pair.left)),
+              new GlobalTimeExpression(TimeFilter.ltEq(pair.right)));
+      if (intervalFilter != null) {
+        intervalFilter = BinaryExpression.or(intervalFilter, pairFilter);
+      } else {
+        intervalFilter = pairFilter;
+      }
+    }
+
+    // merge interval filter and filtering conditions after where statements
+    if (expression == null) {
+      expression = intervalFilter;
+    } else {
+      expression = BinaryExpression.and(expression, intervalFilter);
+    }
+
+    IExpression optimizedExpression = ExpressionOptimizer.getInstance()
+        .optimize(expression, selectedSeries);
+    if (optimizedExpression.getType() == ExpressionType.GLOBAL_TIME) {
+      GroupByWithOnlyTimeFilterDataSetDataSet groupByEngine = new GroupByWithOnlyTimeFilterDataSetDataSet(
+          nextJobId, selectedSeries, unit, origin, mergedIntervalList);
+      groupByEngine.initGroupBy(context, aggres, optimizedExpression);
+      return groupByEngine;
+    } else {
+      GroupByWithValueFilterDataSetDataSet groupByEngine = new GroupByWithValueFilterDataSetDataSet(
+          nextJobId,
+          selectedSeries, unit, origin, mergedIntervalList);
+      groupByEngine.initGroupBy(context, aggres, optimizedExpression);
+      return groupByEngine;
+    }
+  }
+
+  /**
+   * execute fill query.
+   *
+   * @param fillPaths select path list
+   * @param queryTime timestamp
+   * @param fillType type IFill map
+   */
+  public QueryDataSet fill(List<Path> fillPaths, long queryTime, Map<TSDataType, IFill> fillType,
+      QueryContext context)
+      throws FileNodeManagerException, PathErrorException, IOException {
+
+    long nextJobId = context.getJobId();
+
+    FillEngineExecutor fillEngineExecutor = new FillEngineExecutor(nextJobId, fillPaths, queryTime,
+        fillType);
+    return fillEngineExecutor.execute(context);
+  }
+
+  /**
+   * sort intervals by start time and merge overlapping intervals.
+   *
+   * @param intervals time interval
+   */
+  private List<Pair<Long, Long>> mergeInterval(List<Pair<Long, Long>> intervals) {
+    // sort by interval start time.
+    intervals.sort(((o1, o2) -> (int) (o1.left - o2.left)));
+
+    LinkedList<Pair<Long, Long>> merged = new LinkedList<>();
+    for (Pair<Long, Long> interval : intervals) {
+      // if the list of merged intervals is empty or
+      // if the current interval does not overlap with the previous, simply append it.
+      if (merged.isEmpty() || merged.getLast().right < interval.left) {
+        merged.add(interval);
+      } else {
+        // otherwise, there is overlap, so we merge the current and previous intervals.
+        merged.getLast().right = Math.max(merged.getLast().right, interval.right);
+      }
+    }
+    return merged;
+  }
+
+
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/FillEngineExecutor.java b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/FillEngineExecutor.java
new file mode 100644
index 0000000..83c5fa9
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/FillEngineExecutor.java
@@ -0,0 +1,91 @@
+/**
+ * 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.iotdb.db.query.executor;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.exception.FileNodeManagerException;
+import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.dataset.EngineDataSetWithoutTimeGenerator;
+import org.apache.iotdb.db.query.fill.IFill;
+import org.apache.iotdb.db.query.fill.PreviousFill;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+
+public class FillEngineExecutor {
+
+  private long jobId;
+  private List<Path> selectedSeries;
+  private long queryTime;
+  private Map<TSDataType, IFill> typeIFillMap;
+
+  public FillEngineExecutor(long jobId, List<Path> selectedSeries, long queryTime,
+      Map<TSDataType, IFill> typeIFillMap) {
+    this.jobId = jobId;
+    this.selectedSeries = selectedSeries;
+    this.queryTime = queryTime;
+    this.typeIFillMap = typeIFillMap;
+  }
+
+  /**
+   * execute fill.
+   *
+   * @param context query context
+   */
+  public QueryDataSet execute(QueryContext context)
+      throws FileNodeManagerException, PathErrorException, IOException {
+    QueryResourceManager.getInstance().beginQueryOfGivenQueryPaths(jobId, selectedSeries);
+
+    List<IFill> fillList = new ArrayList<>();
+    List<TSDataType> dataTypeList = new ArrayList<>();
+    for (Path path : selectedSeries) {
+      QueryDataSource queryDataSource = QueryResourceManager.getInstance()
+          .getQueryDataSource(path, context);
+      TSDataType dataType = MManager.getInstance().getSeriesType(path.getFullPath());
+      dataTypeList.add(dataType);
+      IFill fill = null;
+      if (!typeIFillMap.containsKey(dataType)) {
+        fill = new PreviousFill(dataType, queryTime, 0);
+      } else {
+        fill = typeIFillMap.get(dataType).copy(path);
+      }
+      fill.setDataType(dataType);
+      fill.setQueryTime(queryTime);
+      fill.constructReaders(queryDataSource, context);
+      fillList.add(fill);
+    }
+
+    List<IPointReader> readers = new ArrayList<>();
+    for (IFill fill : fillList) {
+      readers.add(fill.getFillResult());
+    }
+
+    return new EngineDataSetWithoutTimeGenerator(selectedSeries, dataTypeList, readers);
+  }
+
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/groupby/GroupByEngineDataSet.java b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/groupby/GroupByEngineDataSet.java
new file mode 100644
index 0000000..2ccecf5
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/groupby/GroupByEngineDataSet.java
@@ -0,0 +1,167 @@
+/**
+ * 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.iotdb.db.query.executor.groupby;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.exception.ProcessorException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.query.aggregation.AggreFuncFactory;
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.Field;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+public abstract class GroupByEngineDataSet extends QueryDataSet {
+
+  protected long jobId;
+  protected List<Path> selectedSeries;
+  private long unit;
+  private long origin;
+  private List<Pair<Long, Long>> mergedIntervals;
+
+  protected long startTime;
+  protected long endTime;
+  private int usedIndex;
+  protected List<AggregateFunction> functions;
+  protected boolean hasCachedTimeInterval;
+
+  /**
+   * groupBy query.
+   */
+  public GroupByEngineDataSet(long jobId, List<Path> paths, long unit, long origin,
+      List<Pair<Long, Long>> mergedIntervals) {
+    super(paths);
+    this.jobId = jobId;
+    this.selectedSeries = paths;
+    this.unit = unit;
+    this.origin = origin;
+    this.mergedIntervals = mergedIntervals;
+    this.functions = new ArrayList<>();
+
+    // init group by time partition
+    this.usedIndex = 0;
+    this.hasCachedTimeInterval = false;
+    this.endTime = -1;
+  }
+
+  protected void initAggreFuction(List<String> aggres)
+      throws PathErrorException, ProcessorException {
+
+    List<TSDataType> types = new ArrayList<>();
+    // construct AggregateFunctions
+    for (int i = 0; i < paths.size(); i++) {
+      TSDataType tsDataType = MManager.getInstance()
+          .getSeriesType(selectedSeries.get(i).getFullPath());
+      AggregateFunction function = AggreFuncFactory.getAggrFuncByName(aggres.get(i), tsDataType);
+      function.init();
+      functions.add(function);
+      types.add(function.getResultDataType());
+    }
+    super.setDataTypes(types);
+  }
+
+  @Override
+  public boolean hasNext() {
+    // has cached
+    if (hasCachedTimeInterval) {
+      return true;
+    }
+
+    // end
+    if (usedIndex >= mergedIntervals.size()) {
+      return false;
+    }
+
+    // skip the intervals in coverage of last time-partition
+    while (usedIndex < mergedIntervals.size() && mergedIntervals.get(usedIndex).right < endTime) {
+      usedIndex++;
+    }
+    if (usedIndex >= mergedIntervals.size()) {
+      return false;
+    }
+
+    // initialize the start-end time of next interval
+    if (endTime < mergedIntervals.get(usedIndex).left) {
+      // interval start time
+      startTime = mergedIntervals.get(usedIndex).left;
+      if (origin > startTime) {
+        endTime = origin - (origin - startTime) / unit * unit;
+      } else {
+        endTime = origin + (startTime - origin) / unit * unit + unit;
+      }
+      hasCachedTimeInterval = true;
+      return true;
+    }
+
+    // current interval is not covered yet
+    if (endTime <= mergedIntervals.get(usedIndex).right) {
+      startTime = endTime;
+      endTime += unit;
+      hasCachedTimeInterval = true;
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * this method is only used in the test class to get the next time partition.
+   */
+  public Pair<Long, Long> nextTimePartition() {
+    hasCachedTimeInterval = false;
+    return new Pair<>(startTime, endTime);
+  }
+
+  protected Field getField(AggreResultData aggreResultData) {
+    if (!aggreResultData.isSetValue()) {
+      return new Field(null);
+    }
+    Field field = new Field(aggreResultData.getDataType());
+    switch (aggreResultData.getDataType()) {
+      case INT32:
+        field.setIntV(aggreResultData.getIntRet());
+        break;
+      case INT64:
+        field.setLongV(aggreResultData.getLongRet());
+        break;
+      case FLOAT:
+        field.setFloatV(aggreResultData.getFloatRet());
+        break;
+      case DOUBLE:
+        field.setDoubleV(aggreResultData.getDoubleRet());
+        break;
+      case BOOLEAN:
+        field.setBoolV(aggreResultData.isBooleanRet());
+        break;
+      case TEXT:
+        field.setBinaryV(aggreResultData.getBinaryRet());
+        break;
+      default:
+        throw new UnSupportedDataTypeException("UnSupported: " + aggreResultData.getDataType());
+    }
+    return field;
+  }
+
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/groupby/GroupByWithOnlyTimeFilterDataSetDataSet.java b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/groupby/GroupByWithOnlyTimeFilterDataSetDataSet.java
new file mode 100644
index 0000000..e82b567
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/groupby/GroupByWithOnlyTimeFilterDataSetDataSet.java
@@ -0,0 +1,314 @@
+/**
+ * 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.iotdb.db.query.executor.groupby;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.exception.FileNodeManagerException;
+import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.exception.ProcessorException;
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.factory.SeriesReaderFactory;
+import org.apache.iotdb.db.query.reader.IAggregateReader;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.PriorityMergeReader;
+import org.apache.iotdb.db.query.reader.sequence.SequenceDataReader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.Field;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.expression.IExpression;
+import org.apache.iotdb.tsfile.read.expression.impl.GlobalTimeExpression;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+public class GroupByWithOnlyTimeFilterDataSetDataSet extends GroupByEngineDataSet {
+
+  protected List<IPointReader> unSequenceReaderList;
+  protected List<IAggregateReader> sequenceReaderList;
+  private List<BatchData> batchDataList;
+  private List<Boolean> hasCachedSequenceDataList;
+  private Filter timeFilter;
+
+  /**
+   * constructor.
+   */
+  public GroupByWithOnlyTimeFilterDataSetDataSet(long jobId, List<Path> paths, long unit,
+      long origin, List<Pair<Long, Long>> mergedIntervals) {
+    super(jobId, paths, unit, origin, mergedIntervals);
+    this.unSequenceReaderList = new ArrayList<>();
+    this.sequenceReaderList = new ArrayList<>();
+    this.timeFilter = null;
+    this.hasCachedSequenceDataList = new ArrayList<>();
+    this.batchDataList = new ArrayList<>();
+    for (int i = 0; i < paths.size(); i++) {
+      hasCachedSequenceDataList.add(false);
+      batchDataList.add(null);
+    }
+  }
+
+  /**
+   * init reader and aggregate function.
+   */
+  public void initGroupBy(QueryContext context, List<String> aggres, IExpression expression)
+      throws FileNodeManagerException, PathErrorException, ProcessorException, IOException {
+    initAggreFuction(aggres);
+    // init reader
+    QueryResourceManager.getInstance().beginQueryOfGivenQueryPaths(jobId, selectedSeries);
+    if (expression != null) {
+      timeFilter = ((GlobalTimeExpression) expression).getFilter();
+    }
+    for (int i = 0; i < selectedSeries.size(); i++) {
+      QueryDataSource queryDataSource = QueryResourceManager.getInstance()
+          .getQueryDataSource(selectedSeries.get(i), context);
+
+      // sequence reader for sealed tsfile, unsealed tsfile, memory
+      SequenceDataReader sequenceReader = new SequenceDataReader(queryDataSource.getSeqDataSource(),
+          timeFilter, context, false);
+
+      // unseq reader for all chunk groups in unSeqFile, memory
+      PriorityMergeReader unSeqMergeReader = SeriesReaderFactory.getInstance()
+          .createUnSeqMergeReader(queryDataSource.getOverflowSeriesDataSource(), timeFilter);
+
+      sequenceReaderList.add(sequenceReader);
+      unSequenceReaderList.add(unSeqMergeReader);
+    }
+
+  }
+
+  @Override
+  public RowRecord next() throws IOException {
+    if (!hasCachedTimeInterval) {
+      throw new IOException("need to call hasNext() before calling next() "
+          + "in GroupByWithOnlyTimeFilterDataSetDataSet.");
+    }
+    hasCachedTimeInterval = false;
+    RowRecord record = new RowRecord(startTime);
+    for (int i = 0; i < functions.size(); i++) {
+      AggreResultData res = null;
+      try {
+        res = nextSeries(i);
+      } catch (ProcessorException e) {
+        throw new IOException(e);
+      }
+      if (res == null) {
+        record.addField(new Field(null));
+      } else {
+        record.addField(getField(res));
+      }
+    }
+    return record;
+  }
+
+  /**
+   * calculate the group by result of the series indexed by idx.
+   *
+   * @param idx series id
+   */
+  private AggreResultData nextSeries(int idx) throws IOException, ProcessorException {
+    IPointReader unsequenceReader = unSequenceReaderList.get(idx);
+    IAggregateReader sequenceReader = sequenceReaderList.get(idx);
+    AggregateFunction function = functions.get(idx);
+    function.init();
+
+    // skip the points with timestamp less than startTime
+    skipBeforeStartTimeData(idx, sequenceReader, unsequenceReader);
+
+    // cal group by in batch data
+    boolean finishCheckSequenceData = calGroupByInBatchData(idx, function, unsequenceReader);
+    if (finishCheckSequenceData) {
+      // check unsequence data
+      function.calculateValueFromUnsequenceReader(unsequenceReader, endTime);
+      return function.getResult().deepCopy();
+    }
+
+    // continue checking sequence data
+    while (sequenceReader.hasNext()) {
+      PageHeader pageHeader = sequenceReader.nextPageHeader();
+
+      // memory data
+      if (pageHeader == null) {
+        batchDataList.set(idx, sequenceReader.nextBatch());
+        hasCachedSequenceDataList.set(idx, true);
+        finishCheckSequenceData = calGroupByInBatchData(idx, function, unsequenceReader);
+      } else {
+        // page data
+        long minTime = pageHeader.getMinTimestamp();
+        long maxTime = pageHeader.getMaxTimestamp();
+        // no point in sequence data with a timestamp less than endTime
+        if (minTime >= endTime) {
+          finishCheckSequenceData = true;
+        } else if (canUseHeader(minTime, maxTime, unsequenceReader, function)) {
+          // cal using page header
+          function.calculateValueFromPageHeader(pageHeader);
+          sequenceReader.skipPageData();
+        } else {
+          // cal using page data
+          batchDataList.set(idx, sequenceReader.nextBatch());
+          hasCachedSequenceDataList.set(idx, true);
+          finishCheckSequenceData = calGroupByInBatchData(idx, function, unsequenceReader);
+        }
+
+        if (finishCheckSequenceData) {
+          break;
+        }
+      }
+    }
+    // cal using unsequence data
+    function.calculateValueFromUnsequenceReader(unsequenceReader, endTime);
+    return function.getResult().deepCopy();
+  }
+
+  /**
+   * calculate groupBy's result in batch data.
+   *
+   * @param idx series index
+   * @param function aggregate function of the series
+   * @param unsequenceReader unsequence reader of the series
+   * @return if all sequential data been computed
+   */
+  private boolean calGroupByInBatchData(int idx, AggregateFunction function,
+      IPointReader unsequenceReader)
+      throws IOException, ProcessorException {
+    BatchData batchData = batchDataList.get(idx);
+    boolean hasCachedSequenceData = hasCachedSequenceDataList.get(idx);
+    boolean finishCheckSequenceData = false;
+    // there was unprocessed data in last batch
+    if (hasCachedSequenceData && batchData.hasNext()) {
+      function.calculateValueFromPageData(batchData, unsequenceReader, endTime);
+    }
+
+    if (hasCachedSequenceData && batchData.hasNext()) {
+      finishCheckSequenceData = true;
+    } else {
+      hasCachedSequenceData = false;
+    }
+    batchDataList.set(idx, batchData);
+    hasCachedSequenceDataList.set(idx, hasCachedSequenceData);
+    return finishCheckSequenceData;
+  }
+
+  /**
+   * skip the points with timestamp less than startTime.
+   *
+   * @param idx the index of series
+   * @param sequenceReader sequence Reader
+   * @param unsequenceReader unsequence Reader
+   * @throws IOException exception when reading file
+   */
+  private void skipBeforeStartTimeData(int idx, IAggregateReader sequenceReader,
+      IPointReader unsequenceReader)
+      throws IOException {
+
+    // skip the unsequenceReader points with timestamp less than startTime
+    skipPointInUnsequenceData(unsequenceReader);
+
+    // skip the cached batch data points with timestamp less than startTime
+    if (skipPointInBatchData(idx)) {
+      return;
+    }
+
+    // skip the points in sequenceReader data whose timestamp are less than startTime
+    while (sequenceReader.hasNext()) {
+      PageHeader pageHeader = sequenceReader.nextPageHeader();
+      // memory data
+      if (pageHeader == null) {
+        batchDataList.set(idx, sequenceReader.nextBatch());
+        hasCachedSequenceDataList.set(idx, true);
+        if (skipPointInBatchData(idx)) {
+          return;
+        }
+      } else {
+        // page data
+
+        // timestamps of all points in the page are less than startTime
+        if (pageHeader.getMaxTimestamp() < startTime) {
+          sequenceReader.skipPageData();
+          continue;
+        } else if (pageHeader.getMinTimestamp() >= startTime) {
+          // timestamps of all points in the page are greater or equal to startTime, needn't to skip
+          return;
+        }
+        // the page has overlap with startTime
+        batchDataList.set(idx, sequenceReader.nextBatch());
+        hasCachedSequenceDataList.set(idx, true);
+        if (skipPointInBatchData(idx)) {
+          return;
+        }
+      }
+    }
+  }
+
+  /**
+   * skip points in unsequence reader whose timestamp is less than startTime.
+   *
+   * @param unsequenceReader unsequence reader
+   */
+  private void skipPointInUnsequenceData(IPointReader unsequenceReader) throws IOException {
+    while (unsequenceReader.hasNext() && unsequenceReader.current().getTimestamp() < startTime) {
+      unsequenceReader.next();
+    }
+  }
+
+  /**
+   * skip points in batch data whose timestamp is less than startTime.
+   *
+   * @param idx series index
+   * @return whether has next in batch data
+   */
+  private boolean skipPointInBatchData(int idx) {
+    BatchData batchData = batchDataList.get(idx);
+    boolean hasCachedSequenceData = hasCachedSequenceDataList.get(idx);
+    if (!hasCachedSequenceData) {
+      return false;
+    }
+
+    // skip the cached batch data points with timestamp less than startTime
+    while (batchData.hasNext() && batchData.currentTime() < startTime) {
+      batchData.next();
+    }
+    batchDataList.set(idx, batchData);
+    if (batchData.hasNext()) {
+      return true;
+    } else {
+      hasCachedSequenceDataList.set(idx, false);
+      return false;
+    }
+  }
+
+  private boolean canUseHeader(long minTime, long maxTime, IPointReader unSequenceReader,
+      AggregateFunction function)
+      throws IOException, ProcessorException {
+    if (timeFilter != null && !timeFilter.containStartEndTime(minTime, maxTime)) {
+      return false;
+    }
+
+    // cal unsequence data with timestamps between pages.
+    function.calculateValueFromUnsequenceReader(unSequenceReader, minTime);
+
+    return !(unSequenceReader.hasNext() && unSequenceReader.current().getTimestamp() <= maxTime);
+  }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/groupby/GroupByWithValueFilterDataSetDataSet.java b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/groupby/GroupByWithValueFilterDataSetDataSet.java
new file mode 100644
index 0000000..7a6b3d2
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/groupby/GroupByWithValueFilterDataSetDataSet.java
@@ -0,0 +1,160 @@
+/**
+ * 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.iotdb.db.query.executor.groupby;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.FileNodeManagerException;
+import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.exception.ProcessorException;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.factory.SeriesReaderFactory;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.db.query.timegenerator.EngineTimeGenerator;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.expression.IExpression;
+import org.apache.iotdb.tsfile.read.query.timegenerator.TimeGenerator;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+public class GroupByWithValueFilterDataSetDataSet extends GroupByEngineDataSet {
+
+  private List<EngineReaderByTimeStamp> allDataReaderList;
+  private TimeGenerator timestampGenerator;
+  /**
+   * cached timestamp for next group by partition.
+   */
+  private long timestamp;
+  /**
+   * if this object has cached timestamp for next group by partition.
+   */
+  private boolean hasCachedTimestamp;
+
+  /**
+   * group by batch calculation size.
+   */
+  private int timeStampFetchSize;
+
+  /**
+   * constructor.
+   */
+  public GroupByWithValueFilterDataSetDataSet(long jobId, List<Path> paths, long unit, long origin,
+      List<Pair<Long, Long>> mergedIntervals) {
+    super(jobId, paths, unit, origin, mergedIntervals);
+    this.allDataReaderList = new ArrayList<>();
+    this.timeStampFetchSize = 10 * IoTDBDescriptor.getInstance().getConfig().getFetchSize();
+  }
+
+  /**
+   * init reader and aggregate function.
+   */
+  public void initGroupBy(QueryContext context, List<String> aggres, IExpression expression)
+      throws FileNodeManagerException, PathErrorException, ProcessorException, IOException {
+    initAggreFuction(aggres);
+
+    QueryResourceManager.getInstance().beginQueryOfGivenExpression(context.getJobId(), expression);
+    QueryResourceManager
+        .getInstance().beginQueryOfGivenQueryPaths(context.getJobId(), selectedSeries);
+    this.timestampGenerator = new EngineTimeGenerator(expression, context);
+    this.allDataReaderList = SeriesReaderFactory
+        .getByTimestampReadersOfSelectedPaths(selectedSeries, context);
+  }
+
+  @Override
+  public RowRecord next() throws IOException {
+    if (!hasCachedTimeInterval) {
+      throw new IOException("need to call hasNext() before calling next()"
+          + " in GroupByWithOnlyTimeFilterDataSetDataSet.");
+    }
+    hasCachedTimeInterval = false;
+    for (AggregateFunction function : functions) {
+      function.init();
+    }
+
+    long[] timestampArray = new long[timeStampFetchSize];
+    int timeArrayLength = 0;
+    if (hasCachedTimestamp) {
+      if (timestamp < endTime) {
+        hasCachedTimestamp = false;
+        timestampArray[timeArrayLength++] = timestamp;
+      } else {
+        return constructRowRecord();
+      }
+    }
+
+    while (timestampGenerator.hasNext()) {
+      // construct timestamp array
+      timeArrayLength = constructTimeArrayForOneCal(timestampArray, timeArrayLength);
+
+      // cal result using timestamp array
+      for (int i = 0; i < selectedSeries.size(); i++) {
+        functions.get(i).calcAggregationUsingTimestamps(
+            timestampArray, timeArrayLength, allDataReaderList.get(i));
+      }
+
+      timeArrayLength = 0;
+      // judge if it's end
+      if (timestamp >= endTime) {
+        hasCachedTimestamp = true;
+        break;
+      }
+    }
+
+    if (timeArrayLength > 0) {
+      // cal result using timestamp array
+      for (int i = 0; i < selectedSeries.size(); i++) {
+        functions.get(i).calcAggregationUsingTimestamps(
+            timestampArray, timeArrayLength, allDataReaderList.get(i));
+      }
+    }
+    return constructRowRecord();
+  }
+
+  /**
+   * construct an array of timestamps for one batch of a group by partition calculating.
+   *
+   * @param timestampArray timestamp array
+   * @param timeArrayLength the current length of timestamp array
+   * @return time array length
+   */
+  private int constructTimeArrayForOneCal(long[] timestampArray, int timeArrayLength)
+      throws IOException {
+    for (int cnt = 1; cnt < timeStampFetchSize && timestampGenerator.hasNext(); cnt++) {
+      timestamp = timestampGenerator.next();
+      if (timestamp < endTime) {
+        timestampArray[timeArrayLength++] = timestamp;
+      } else {
+        hasCachedTimestamp = true;
+        break;
+      }
+    }
+    return timeArrayLength;
+  }
+
+  private RowRecord constructRowRecord() {
+    RowRecord record = new RowRecord(startTime);
+    functions.forEach(function -> record.addField(getField(function.getResult())));
+    return record;
+  }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java b/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java
index 42e81f2..b731fc5 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java
@@ -20,24 +20,36 @@
 package org.apache.iotdb.db.query.factory;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 import org.apache.iotdb.db.engine.filenode.TsFileResource;
 import org.apache.iotdb.db.engine.modification.Modification;
 import org.apache.iotdb.db.engine.querycontext.OverflowInsertFile;
 import org.apache.iotdb.db.engine.querycontext.OverflowSeriesDataSource;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.exception.FileNodeManagerException;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.control.FileReaderManager;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.reader.AllDataReader;
+import org.apache.iotdb.db.query.reader.IBatchReader;
+import org.apache.iotdb.db.query.reader.IPointReader;
 import org.apache.iotdb.db.query.reader.IReader;
-import org.apache.iotdb.db.query.reader.mem.MemChunkReaderWithFilter;
-import org.apache.iotdb.db.query.reader.mem.MemChunkReaderWithoutFilter;
+import org.apache.iotdb.db.query.reader.mem.MemChunkReader;
+import org.apache.iotdb.db.query.reader.mem.MemChunkReaderByTimestamp;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
 import org.apache.iotdb.db.query.reader.merge.PriorityMergeReader;
+import org.apache.iotdb.db.query.reader.merge.PriorityMergeReaderByTimestamp;
 import org.apache.iotdb.db.query.reader.sequence.SealedTsFilesReader;
+import org.apache.iotdb.db.query.reader.sequence.SequenceDataReaderByTimestamp;
 import org.apache.iotdb.db.query.reader.unsequence.EngineChunkReader;
+import org.apache.iotdb.db.query.reader.unsequence.EngineChunkReaderByTimestamp;
 import org.apache.iotdb.db.utils.QueryUtils;
 import org.apache.iotdb.tsfile.common.constant.StatisticConstant;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
 import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
 import org.apache.iotdb.tsfile.read.common.Chunk;
+import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.read.controller.ChunkLoaderImpl;
 import org.apache.iotdb.tsfile.read.controller.MetadataQuerier;
 import org.apache.iotdb.tsfile.read.controller.MetadataQuerierByFileImpl;
@@ -45,6 +57,7 @@ import org.apache.iotdb.tsfile.read.expression.impl.SingleSeriesExpression;
 import org.apache.iotdb.tsfile.read.filter.DigestForFilter;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
+import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReaderByTimestamp;
 import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReaderWithFilter;
 import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReaderWithoutFilter;
 import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReader;
@@ -110,15 +123,9 @@ public class SeriesReaderFactory {
 
     // add reader for MemTable
     if (overflowSeriesDataSource.hasRawChunk()) {
-      if (filter != null) {
-        unSeqMergeReader.addReaderWithPriority(
-            new MemChunkReaderWithFilter(overflowSeriesDataSource.getReadableMemChunk(), filter),
-            priorityValue);
-      } else {
-        unSeqMergeReader.addReaderWithPriority(
-            new MemChunkReaderWithoutFilter(overflowSeriesDataSource.getReadableMemChunk()),
-            priorityValue);
-      }
+      unSeqMergeReader.addReaderWithPriority(
+          new MemChunkReader(overflowSeriesDataSource.getReadableMemChunk(), filter),
+          priorityValue);
     }
 
     // TODO add external sort when needed
@@ -141,22 +148,24 @@ public class SeriesReaderFactory {
         singleSeriesExpression,
         tsFileResource.getFilePath());
 
-    PriorityMergeReader priorityMergeReader = new PriorityMergeReader();
-
-    // Sequence reader
-    IReader seriesInTsFileReader = createSealedTsFileReaderForMerge(tsFileResource,
+    // sequence reader
+    IBatchReader seriesInTsFileReader = createSealedTsFileReaderForMerge(tsFileResource,
         singleSeriesExpression, context);
-    priorityMergeReader.addReaderWithPriority(seriesInTsFileReader, 1);
 
-    // UnSequence merge reader
-    IReader unSeqMergeReader = createUnSeqMergeReader(overflowSeriesDataSource,
+    // unSequence merge reader
+    IPointReader unSeqMergeReader = createUnSeqMergeReader(overflowSeriesDataSource,
         singleSeriesExpression.getFilter());
-    priorityMergeReader.addReaderWithPriority(unSeqMergeReader, 2);
+    if (!seriesInTsFileReader.hasNext()) {
+      // only have unsequence data.
+      return unSeqMergeReader;
+    } else {
+      // merge sequence data with unsequence data.
+      return new AllDataReader(seriesInTsFileReader, unSeqMergeReader);
+    }
 
-    return priorityMergeReader;
   }
 
-  private IReader createSealedTsFileReaderForMerge(TsFileResource fileNode,
+  private IBatchReader createSealedTsFileReaderForMerge(TsFileResource fileNode,
       SingleSeriesExpression singleSeriesExpression,
       QueryContext context)
       throws IOException {
@@ -178,6 +187,84 @@ public class SeriesReaderFactory {
     return new SealedTsFilesReader(seriesInTsFileReader, context);
   }
 
+  /**
+   * construct ByTimestampReader, include sequential data and unsequential data.
+   *
+   * @param paths selected series path
+   * @param context query context
+   * @return the list of EngineReaderByTimeStamp
+   */
+  public static List<EngineReaderByTimeStamp> getByTimestampReadersOfSelectedPaths(
+      List<Path> paths, QueryContext context) throws IOException, FileNodeManagerException {
+
+    List<EngineReaderByTimeStamp> readersOfSelectedSeries = new ArrayList<>();
+
+    for (Path path : paths) {
+
+      QueryDataSource queryDataSource = QueryResourceManager.getInstance().getQueryDataSource(path,
+          context);
+
+      PriorityMergeReaderByTimestamp mergeReaderByTimestamp = new PriorityMergeReaderByTimestamp();
+
+      // reader for sequence data
+      SequenceDataReaderByTimestamp tsFilesReader = new SequenceDataReaderByTimestamp(
+          queryDataSource.getSeqDataSource(), context);
+      mergeReaderByTimestamp.addReaderWithPriority(tsFilesReader, 1);
+
+      // reader for unSequence data
+      PriorityMergeReaderByTimestamp unSeqMergeReader = SeriesReaderFactory.getInstance()
+          .createUnSeqMergeReaderByTimestamp(queryDataSource.getOverflowSeriesDataSource());
+      mergeReaderByTimestamp.addReaderWithPriority(unSeqMergeReader, 2);
+
+      readersOfSelectedSeries.add(mergeReaderByTimestamp);
+    }
+
+    return readersOfSelectedSeries;
+  }
+
+  /**
+   * This method is used to create unsequence insert reader by timestamp for IoTDB request, such as
+   * query, aggregation and groupby request.
+   */
+  public PriorityMergeReaderByTimestamp createUnSeqMergeReaderByTimestamp(
+      OverflowSeriesDataSource overflowSeriesDataSource)
+      throws IOException {
+
+    PriorityMergeReaderByTimestamp unSeqMergeReader = new PriorityMergeReaderByTimestamp();
+
+    int priorityValue = 1;
+
+    for (OverflowInsertFile overflowInsertFile : overflowSeriesDataSource
+        .getOverflowInsertFileList()) {
+
+      // store only one opened file stream into manager, to avoid too many opened files
+      TsFileSequenceReader unClosedTsFileReader = FileReaderManager.getInstance()
+          .get(overflowInsertFile.getFilePath(), false);
+
+      ChunkLoaderImpl chunkLoader = new ChunkLoaderImpl(unClosedTsFileReader);
+
+      for (ChunkMetaData chunkMetaData : overflowInsertFile.getChunkMetaDataList()) {
+
+        Chunk chunk = chunkLoader.getChunk(chunkMetaData);
+        ChunkReaderByTimestamp chunkReader = new ChunkReaderByTimestamp(chunk);
+
+        unSeqMergeReader
+            .addReaderWithPriority(new EngineChunkReaderByTimestamp(chunkReader), priorityValue);
+        priorityValue++;
+      }
+    }
+
+    // add reader for MemTable
+    if (overflowSeriesDataSource.hasRawChunk()) {
+      unSeqMergeReader.addReaderWithPriority(
+          new MemChunkReaderByTimestamp(overflowSeriesDataSource.getReadableMemChunk()),
+          priorityValue);
+    }
+
+    // TODO add external sort when needed
+    return unSeqMergeReader;
+  }
+
   private static class SeriesReaderFactoryHelper {
 
     private static final SeriesReaderFactory INSTANCE = new SeriesReaderFactory();
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/fill/IFill.java b/iotdb/src/main/java/org/apache/iotdb/db/query/fill/IFill.java
index 7ddce7a..d64b49a 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/fill/IFill.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/fill/IFill.java
@@ -16,18 +16,30 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.query.fill;
 
-import org.apache.iotdb.db.exception.ProcessorException;
+import java.io.IOException;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.factory.SeriesReaderFactory;
+import org.apache.iotdb.db.query.reader.AllDataReader;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.PriorityMergeReader;
+import org.apache.iotdb.db.query.reader.sequence.SequenceDataReader;
+import org.apache.iotdb.db.utils.TimeValuePair;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.read.common.BatchData;
 import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.filter.TimeFilter;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 
 public abstract class IFill {
 
   long queryTime;
   TSDataType dataType;
 
+  IPointReader allDataReader;
+
   public IFill(TSDataType dataType, long queryTime) {
     this.dataType = dataType;
     this.queryTime = queryTime;
@@ -38,7 +50,24 @@ public abstract class IFill {
 
   public abstract IFill copy(Path path);
 
-  public abstract BatchData getFillResult() throws ProcessorException;
+  public abstract void constructReaders(QueryDataSource queryDataSource, QueryContext context)
+      throws IOException;
+
+  void constructReaders(QueryDataSource queryDataSource, QueryContext context, long beforeRange)
+      throws IOException {
+    Filter timeFilter = constructFilter(beforeRange);
+    // sequence reader for sealed tsfile, unsealed tsfile, memory
+    SequenceDataReader sequenceReader = new SequenceDataReader(queryDataSource.getSeqDataSource(),
+        timeFilter, context, false);
+
+    // unseq reader for all chunk groups in unSeqFile, memory
+    PriorityMergeReader unSeqMergeReader = SeriesReaderFactory.getInstance()
+        .createUnSeqMergeReader(queryDataSource.getOverflowSeriesDataSource(), timeFilter);
+
+    allDataReader = new AllDataReader(sequenceReader, unSeqMergeReader);
+  }
+
+  public abstract IPointReader getFillResult() throws IOException;
 
   public TSDataType getDataType() {
     return this.dataType;
@@ -48,11 +77,47 @@ public abstract class IFill {
     this.dataType = dataType;
   }
 
-  public long getQueryTime() {
-    return this.queryTime;
-  }
-
   public void setQueryTime(long queryTime) {
     this.queryTime = queryTime;
   }
+
+  private Filter constructFilter(long beforeRange) {
+    // if the fill time range is not set, beforeRange will be set to -1.
+    if (beforeRange == -1) {
+      return null;
+    }
+    return TimeFilter.gtEq(queryTime - beforeRange);
+  }
+
+  class TimeValuePairPointReader implements IPointReader {
+
+    private boolean isUsed;
+    private TimeValuePair pair;
+
+    public TimeValuePairPointReader(TimeValuePair pair) {
+      this.pair = pair;
+      this.isUsed = (pair == null);
+    }
+
+    @Override
+    public boolean hasNext() {
+      return !isUsed;
+    }
+
+    @Override
+    public TimeValuePair next() {
+      isUsed = true;
+      return pair;
+    }
+
+    @Override
+    public TimeValuePair current() {
+      return pair;
+    }
+
+    @Override
+    public void close() {
+      // no need to close
+    }
+  }
 }
\ No newline at end of file
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/fill/LinearFill.java b/iotdb/src/main/java/org/apache/iotdb/db/query/fill/LinearFill.java
index 764f7c7..dc46082 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/fill/LinearFill.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/fill/LinearFill.java
@@ -16,18 +16,23 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.query.fill;
 
-import org.apache.iotdb.db.exception.ProcessorException;
+import java.io.IOException;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.exception.UnSupportedFillTypeException;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.utils.TimeValuePair;
+import org.apache.iotdb.db.utils.TsPrimitiveType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.read.common.BatchData;
 import org.apache.iotdb.tsfile.read.common.Path;
 
 public class LinearFill extends IFill {
 
   private long beforeRange;
   private long afterRange;
-  private BatchData result;
 
   public LinearFill(long beforeRange, long afterRange) {
     this.beforeRange = beforeRange;
@@ -42,7 +47,6 @@ public class LinearFill extends IFill {
     super(dataType, queryTime);
     this.beforeRange = beforeRange;
     this.afterRange = afterRange;
-    result = new BatchData(dataType, true, true);
   }
 
   public long getBeforeRange() {
@@ -67,7 +71,81 @@ public class LinearFill extends IFill {
   }
 
   @Override
-  public BatchData getFillResult() throws ProcessorException {
-    return result;
+  public void constructReaders(QueryDataSource queryDataSource, QueryContext context)
+      throws IOException {
+    super.constructReaders(queryDataSource, context, beforeRange);
+  }
+
+  @Override
+  public IPointReader getFillResult() throws IOException {
+    TimeValuePair beforePair = null;
+    TimeValuePair afterPair = null;
+    while (allDataReader.hasNext()) {
+      afterPair = allDataReader.next();
+      if (afterPair.getTimestamp() <= queryTime) {
+        beforePair = afterPair;
+      } else {
+        break;
+      }
+    }
+
+    if (beforePair == null || beforePair.getTimestamp() == queryTime) {
+      return new TimeValuePairPointReader(beforePair);
+    }
+
+    // if afterRange equals -1, this means that there is no time-bound filling.
+    if (afterRange == -1) {
+      return new TimeValuePairPointReader(average(beforePair, afterPair));
+    }
+
+    if (afterPair.getTimestamp() > queryTime + afterRange || afterPair.getTimestamp() < queryTime) {
+      return new TimeValuePairPointReader(new TimeValuePair(queryTime, null));
+    }
+    return new TimeValuePairPointReader(average(beforePair, afterPair));
+  }
+
+  // returns the average of two points
+  private TimeValuePair average(TimeValuePair beforePair, TimeValuePair afterPair) {
+    double totalTimeLength = (double) afterPair.getTimestamp() - beforePair.getTimestamp();
+    double beforeTimeLength = (double) (queryTime - beforePair.getTimestamp());
+    switch (dataType) {
+      case INT32:
+        int startIntValue = beforePair.getValue().getInt();
+        int endIntValue = afterPair.getValue().getInt();
+        int fillIntValue =
+            startIntValue + (int) ((double) (endIntValue - startIntValue) / totalTimeLength
+                * beforeTimeLength);
+        beforePair.setValue(TsPrimitiveType.getByType(TSDataType.INT32, fillIntValue));
+        break;
+      case INT64:
+        long startLongValue = beforePair.getValue().getLong();
+        long endLongValue = afterPair.getValue().getLong();
+        long fillLongValue =
+            startLongValue + (long) ((double) (endLongValue - startLongValue) / totalTimeLength
+                * beforeTimeLength);
+        beforePair.setValue(TsPrimitiveType.getByType(TSDataType.INT64, fillLongValue));
+        break;
+      case FLOAT:
+        float startFloatValue = beforePair.getValue().getFloat();
+        float endFloatValue = afterPair.getValue().getFloat();
+        float fillFloatValue =
+            startFloatValue + (float) ((endFloatValue - startFloatValue) / totalTimeLength
+                * beforeTimeLength);
+        beforePair.setValue(TsPrimitiveType.getByType(TSDataType.FLOAT, fillFloatValue));
+        break;
+      case DOUBLE:
+        double startDoubleValue = beforePair.getValue().getDouble();
+        double endDoubleValue = afterPair.getValue().getDouble();
+        double fillDoubleValue =
+            startDoubleValue + ((endDoubleValue - startDoubleValue) / totalTimeLength
+                * beforeTimeLength);
+        beforePair.setValue(TsPrimitiveType.getByType(TSDataType.DOUBLE, fillDoubleValue));
+        break;
+      default:
+        throw new UnSupportedFillTypeException("Unsupported linear fill data type : " + dataType);
+
+    }
+    beforePair.setTimestamp(queryTime);
+    return beforePair;
   }
 }
\ No newline at end of file
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/fill/PreviousFill.java b/iotdb/src/main/java/org/apache/iotdb/db/query/fill/PreviousFill.java
index 1c130b9..b75fb4f 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/fill/PreviousFill.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/fill/PreviousFill.java
@@ -18,21 +18,21 @@
  */
 package org.apache.iotdb.db.query.fill;
 
-import org.apache.iotdb.db.exception.ProcessorException;
+import java.io.IOException;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.utils.TimeValuePair;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.read.common.BatchData;
 import org.apache.iotdb.tsfile.read.common.Path;
 
 public class PreviousFill extends IFill {
 
   private long beforeRange;
 
-  private BatchData result;
-
   public PreviousFill(TSDataType dataType, long queryTime, long beforeRange) {
     super(dataType, queryTime);
     this.beforeRange = beforeRange;
-    result = new BatchData(dataType, true, true);
   }
 
   public PreviousFill(long beforeRange) {
@@ -44,12 +44,34 @@ public class PreviousFill extends IFill {
     return new PreviousFill(dataType, queryTime, beforeRange);
   }
 
+  @Override
+  public void constructReaders(QueryDataSource queryDataSource, QueryContext context)
+      throws IOException {
+    super.constructReaders(queryDataSource, context, beforeRange);
+  }
+
   public long getBeforeRange() {
     return beforeRange;
   }
 
   @Override
-  public BatchData getFillResult() throws ProcessorException {
-    return result;
+  public IPointReader getFillResult() throws IOException {
+    TimeValuePair beforePair = null;
+    TimeValuePair cachedPair = null;
+    while (allDataReader.hasNext()) {
+      cachedPair = allDataReader.next();
+      if (cachedPair.getTimestamp() <= queryTime) {
+        beforePair = cachedPair;
+      } else {
+        break;
+      }
+    }
+
+    if (beforePair != null) {
+      beforePair.setTimestamp(queryTime);
+    } else {
+      beforePair = new TimeValuePair(queryTime, null);
+    }
+    return new TimeValuePairPointReader(beforePair);
   }
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/AllDataReader.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/AllDataReader.java
new file mode 100644
index 0000000..6eaa4bb
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/AllDataReader.java
@@ -0,0 +1,129 @@
+/**
+ * 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.iotdb.db.query.reader;
+
+import java.io.IOException;
+import org.apache.iotdb.db.utils.TimeValuePair;
+import org.apache.iotdb.db.utils.TimeValuePairUtils;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+public class AllDataReader implements IPointReader {
+
+  private IBatchReader batchReader;
+  private IPointReader pointReader;
+
+  private boolean hasCachedBatchData;
+  private BatchData batchData;
+
+  /**
+   * merge sequence reader, unsequence reader.
+   */
+  public AllDataReader(IBatchReader batchReader, IPointReader pointReader) {
+    this.batchReader = batchReader;
+    this.pointReader = pointReader;
+
+    this.hasCachedBatchData = false;
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (hasNextInBatchDataOrBatchReader()) {
+      return true;
+    }
+    // has value in pointReader
+    return pointReader != null && pointReader.hasNext();
+  }
+
+  @Override
+  public TimeValuePair next() throws IOException {
+
+    // has next in both batch reader and point reader
+    if (hasNextInBothReader()) {
+      long timeInPointReader = pointReader.current().getTimestamp();
+      long timeInBatchData = batchData.currentTime();
+      if (timeInPointReader > timeInBatchData) {
+        TimeValuePair timeValuePair = TimeValuePairUtils.getCurrentTimeValuePair(batchData);
+        batchData.next();
+        return timeValuePair;
+      } else if (timeInPointReader == timeInBatchData) {
+        batchData.next();
+        return pointReader.next();
+      } else {
+        return pointReader.next();
+      }
+    }
+
+    // only has next in batch reader
+    if (hasNextInBatchDataOrBatchReader()) {
+      TimeValuePair timeValuePair = TimeValuePairUtils.getCurrentTimeValuePair(batchData);
+      batchData.next();
+      return timeValuePair;
+    }
+
+    // only has next in point reader
+    if (pointReader != null && pointReader.hasNext()) {
+      return pointReader.next();
+    }
+    return null;
+  }
+
+  /**
+   * judge if has next in both batch record and pointReader.
+   */
+  private boolean hasNextInBothReader() throws IOException {
+    if (!hasNextInBatchDataOrBatchReader()) {
+      return false;
+    }
+    return pointReader != null && pointReader.hasNext();
+  }
+
+  /**
+   * judge if has next in batch record, either in batch data or in batch reader.
+   */
+  private boolean hasNextInBatchDataOrBatchReader() throws IOException {
+    // has value in batchData
+    if (hasCachedBatchData && batchData.hasNext()) {
+      return true;
+    } else {
+      hasCachedBatchData = false;
+    }
+
+    // has value in batchReader
+    while (batchReader.hasNext()) {
+      batchData = batchReader.nextBatch();
+      if (batchData.hasNext()) {
+        hasCachedBatchData = true;
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public TimeValuePair current() throws IOException {
+    throw new IOException("current() in AllDataReader is an empty method.");
+  }
+
+  @Override
+  public void close() throws IOException {
+    batchReader.close();
+    pointReader.close();
+  }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IAggregateReader.java
similarity index 66%
copy from iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java
copy to iotdb/src/main/java/org/apache/iotdb/db/query/reader/IAggregateReader.java
index 8ddf0ad..edc985f 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IAggregateReader.java
@@ -16,17 +16,19 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.query.reader.merge;
+
+package org.apache.iotdb.db.query.reader;
 
 import java.io.IOException;
-import org.apache.iotdb.db.query.reader.IReader;
-import org.apache.iotdb.db.utils.TsPrimitiveType;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
 
-public interface EngineReaderByTimeStamp extends IReader {
+public interface IAggregateReader extends IBatchReader {
 
   /**
-   * Given a timestamp, the reader is supposed to return the corresponding value in the timestamp.
-   * If no value in this timestamp, null will be returned.
+   * Returns meta-information of batch data. If batch data comes from memory, return null. If batch
+   * data comes from page data, return pageHeader.
    */
-  TsPrimitiveType getValueInTimestamp(long timestamp) throws IOException;
+  PageHeader nextPageHeader() throws IOException;
+
+  void skipPageData() throws IOException;
 }
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IBatchReader.java
similarity index 74%
copy from iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java
copy to iotdb/src/main/java/org/apache/iotdb/db/query/reader/IBatchReader.java
index 3ac1540..6ca07c2 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IBatchReader.java
@@ -1,29 +1,32 @@
-/**
- * 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.iotdb.db.query.control;
-
-import org.junit.Test;
-
-public class QueryTokenManagerTest {
-
-  @Test
-  public void test() {
-    //TODO
-  }
-}
+/**
+ * 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.iotdb.db.query.reader;
+
+import java.io.IOException;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+public interface IBatchReader {
+
+  boolean hasNext() throws IOException;
+
+  BatchData nextBatch() throws IOException;
+
+  void close() throws IOException;
+}
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IPointReader.java
similarity index 79%
copy from iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java
copy to iotdb/src/main/java/org/apache/iotdb/db/query/reader/IPointReader.java
index 3ac1540..2970e1c 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IPointReader.java
@@ -1,29 +1,28 @@
-/**
- * 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.iotdb.db.query.control;
-
-import org.junit.Test;
-
-public class QueryTokenManagerTest {
-
-  @Test
-  public void test() {
-    //TODO
-  }
-}
+/**
+ * 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.iotdb.db.query.reader;
+
+import java.io.IOException;
+import org.apache.iotdb.db.utils.TimeValuePair;
+
+public interface IPointReader extends IReader {
+
+  TimeValuePair current() throws IOException;
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IReader.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IReader.java
index 6a574d6..2982a97 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IReader.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IReader.java
@@ -16,15 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.query.reader;
 
 import java.io.IOException;
 import org.apache.iotdb.db.utils.TimeValuePair;
-import org.apache.iotdb.tsfile.read.common.BatchData;
 
 /**
- * Vital read interface. Batch method is used to increase query speed. Getting a batch of data
- * a time is faster than getting one point a time.
+ * Vital read interface. Batch method is used to increase query speed. Getting a batch of data a
+ * time is faster than getting one point a time.
  */
 public interface IReader {
 
@@ -32,13 +32,5 @@ public interface IReader {
 
   TimeValuePair next() throws IOException;
 
-  void skipCurrentTimeValuePair() throws IOException;
-
   void close() throws IOException;
-
-  boolean hasNextBatch();
-
-  BatchData nextBatch();
-
-  BatchData currentBatch();
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderWithFilter.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReader.java
similarity index 56%
rename from iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderWithFilter.java
rename to iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReader.java
index 1f21a84..8c3e9cf 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderWithFilter.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReader.java
@@ -16,26 +16,36 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.query.reader.mem;
 
 import java.util.Iterator;
-import org.apache.iotdb.db.engine.memtable.TimeValuePairSorter;
 import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
-import org.apache.iotdb.db.query.reader.IReader;
+import org.apache.iotdb.db.query.reader.IAggregateReader;
+import org.apache.iotdb.db.query.reader.IBatchReader;
+import org.apache.iotdb.db.query.reader.IPointReader;
 import org.apache.iotdb.db.utils.TimeValuePair;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.common.BatchData;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 
-public class MemChunkReaderWithFilter implements IReader {
+public class MemChunkReader implements IPointReader, IBatchReader, IAggregateReader {
 
   private Iterator<TimeValuePair> timeValuePairIterator;
   private Filter filter;
   private boolean hasCachedTimeValuePair;
   private TimeValuePair cachedTimeValuePair;
 
-  public MemChunkReaderWithFilter(ReadOnlyMemChunk readableChunk, Filter filter) {
+  private TSDataType dataType;
+
+  /**
+   * memory data reader.
+   */
+  public MemChunkReader(ReadOnlyMemChunk readableChunk, Filter filter) {
     timeValuePairIterator = readableChunk.getIterator();
     this.filter = filter;
+    this.dataType = readableChunk.getDataType();
   }
 
   @Override
@@ -45,7 +55,8 @@ public class MemChunkReaderWithFilter implements IReader {
     }
     while (timeValuePairIterator.hasNext()) {
       TimeValuePair timeValuePair = timeValuePairIterator.next();
-      if (filter.satisfy(timeValuePair.getTimestamp(), timeValuePair.getValue().getValue())) {
+      if (filter == null || filter
+          .satisfy(timeValuePair.getTimestamp(), timeValuePair.getValue().getValue())) {
         hasCachedTimeValuePair = true;
         cachedTimeValuePair = timeValuePair;
         break;
@@ -65,27 +76,45 @@ public class MemChunkReaderWithFilter implements IReader {
   }
 
   @Override
-  public void skipCurrentTimeValuePair() {
-    next();
+  public TimeValuePair current() {
+    if (!hasCachedTimeValuePair) {
+      cachedTimeValuePair = timeValuePairIterator.next();
+      hasCachedTimeValuePair = true;
+    }
+    return cachedTimeValuePair;
   }
 
   @Override
-  public void close() {
-    // Do nothing because mem chunk reader will not open files
+  public BatchData nextBatch() {
+    BatchData batchData = new BatchData(dataType, true);
+    if (hasCachedTimeValuePair) {
+      hasCachedTimeValuePair = false;
+      batchData.putTime(cachedTimeValuePair.getTimestamp());
+      batchData.putAnObject(cachedTimeValuePair.getValue().getValue());
+    }
+    while (timeValuePairIterator.hasNext()) {
+      TimeValuePair timeValuePair = timeValuePairIterator.next();
+      if (filter == null || filter
+          .satisfy(timeValuePair.getTimestamp(), timeValuePair.getValue().getValue())) {
+        batchData.putTime(timeValuePair.getTimestamp());
+        batchData.putAnObject(timeValuePair.getValue().getValue());
+      }
+    }
+    return batchData;
   }
 
   @Override
-  public boolean hasNextBatch() {
-    return false;
+  public void close() {
+    // Do nothing because mem chunk reader will not open files
   }
 
   @Override
-  public BatchData nextBatch() {
+  public PageHeader nextPageHeader() {
     return null;
   }
 
   @Override
-  public BatchData currentBatch() {
-    return null;
+  public void skipPageData() {
+    nextBatch();
   }
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderByTimestamp.java
index dbd6beb..9827f5e 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderByTimestamp.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderByTimestamp.java
@@ -16,15 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.query.reader.mem;
 
-import java.io.IOException;
 import java.util.Iterator;
 import org.apache.iotdb.db.engine.memtable.TimeValuePairSorter;
 import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
 import org.apache.iotdb.db.utils.TimeValuePair;
-import org.apache.iotdb.db.utils.TsPrimitiveType;
-import org.apache.iotdb.tsfile.read.common.BatchData;
 
 public class MemChunkReaderByTimestamp implements EngineReaderByTimeStamp {
 
@@ -44,8 +42,7 @@ public class MemChunkReaderByTimestamp implements EngineReaderByTimeStamp {
     return timeValuePairIterator.hasNext();
   }
 
-  @Override
-  public TimeValuePair next() throws IOException {
+  private TimeValuePair next() {
     if (hasCachedTimeValuePair) {
       hasCachedTimeValuePair = false;
       return cachedTimeValuePair;
@@ -54,25 +51,15 @@ public class MemChunkReaderByTimestamp implements EngineReaderByTimeStamp {
     }
   }
 
-  @Override
-  public void skipCurrentTimeValuePair() throws IOException {
-    next();
-  }
-
-  @Override
-  public void close() {
-    // Do nothing because mem chunk reader will not open files
-  }
-
   // TODO consider change timeValuePairIterator to List structure, and use binary search instead of
   // sequential search
   @Override
-  public TsPrimitiveType getValueInTimestamp(long timestamp) throws IOException {
+  public Object getValueInTimestamp(long timestamp) {
     while (hasNext()) {
       TimeValuePair timeValuePair = next();
       long time = timeValuePair.getTimestamp();
       if (time == timestamp) {
-        return timeValuePair.getValue();
+        return timeValuePair.getValue().getValue();
       } else if (time > timestamp) {
         hasCachedTimeValuePair = true;
         cachedTimeValuePair = timeValuePair;
@@ -81,20 +68,4 @@ public class MemChunkReaderByTimestamp implements EngineReaderByTimeStamp {
     }
     return null;
   }
-
-  @Override
-  public boolean hasNextBatch() {
-    return false;
-  }
-
-  @Override
-  public BatchData nextBatch() {
-    return null;
-  }
-
-  @Override
-  public BatchData currentBatch() {
-    return null;
-  }
-
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderWithoutFilter.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderWithoutFilter.java
deleted file mode 100644
index bae7496..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderWithoutFilter.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/**
- * 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.iotdb.db.query.reader.mem;
-
-import java.util.Iterator;
-import org.apache.iotdb.db.engine.memtable.TimeValuePairSorter;
-import org.apache.iotdb.db.query.reader.IReader;
-import org.apache.iotdb.db.utils.TimeValuePair;
-import org.apache.iotdb.tsfile.read.common.BatchData;
-
-// TODO merge MemChunkReaderWithoutFilter and MemChunkReaderWithFilter to one class
-public class MemChunkReaderWithoutFilter implements IReader {
-
-  private Iterator<TimeValuePair> timeValuePairIterator;
-
-  public MemChunkReaderWithoutFilter(TimeValuePairSorter readableChunk) {
-    timeValuePairIterator = readableChunk.getIterator();
-  }
-
-  @Override
-  public boolean hasNext() {
-    return timeValuePairIterator.hasNext();
-  }
-
-  @Override
-  public TimeValuePair next() {
-    return timeValuePairIterator.next();
-  }
-
-  @Override
-  public void skipCurrentTimeValuePair() {
-    next();
-  }
-
-  @Override
-  public void close() {
-    // Do nothing because mem chunk reader will not open files
-  }
-
-  @Override
-  public boolean hasNextBatch() {
-    return false;
-  }
-
-  @Override
-  public BatchData nextBatch() {
-    return null;
-  }
-
-  @Override
-  public BatchData currentBatch() {
-    return null;
-  }
-}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java
index 8ddf0ad..65a8849 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java
@@ -19,14 +19,14 @@
 package org.apache.iotdb.db.query.reader.merge;
 
 import java.io.IOException;
-import org.apache.iotdb.db.query.reader.IReader;
-import org.apache.iotdb.db.utils.TsPrimitiveType;
 
-public interface EngineReaderByTimeStamp extends IReader {
+public interface EngineReaderByTimeStamp {
 
   /**
    * Given a timestamp, the reader is supposed to return the corresponding value in the timestamp.
    * If no value in this timestamp, null will be returned.
    */
-  TsPrimitiveType getValueInTimestamp(long timestamp) throws IOException;
+  Object getValueInTimestamp(long timestamp) throws IOException;
+
+  boolean hasNext() throws IOException;
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReader.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReader.java
index e100e50..e76d142 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReader.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReader.java
@@ -16,36 +16,34 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.query.reader.merge;
 
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.PriorityQueue;
-import org.apache.iotdb.db.query.reader.IReader;
+import org.apache.iotdb.db.query.reader.IPointReader;
 import org.apache.iotdb.db.utils.TimeValuePair;
-import org.apache.iotdb.tsfile.read.common.BatchData;
 
 /**
  * <p>
- * Usage:
- * (1) merge multiple chunk group readers in the unsequence file
- * (2)merge sequence reader, unsequence reader and mem reader
+ * Usage: (1) merge multiple chunk group readers in the unsequence file.
  * </p>
  */
-public class PriorityMergeReader implements IReader {
+public class PriorityMergeReader implements IPointReader {
 
   public static final int LOW_PRIORITY = 1;
   public static final int HIGH_PRIORITY = 2;
 
-  private List<IReader> readerList = new ArrayList<>();
+  private List<IPointReader> readerList = new ArrayList<>();
   private List<Integer> priorityList = new ArrayList<>();
   private PriorityQueue<Element> heap = new PriorityQueue<>();
 
   /**
-   * The bigger the priority value is, the higher the priority of this reader is
+   * The bigger the priority value is, the higher the priority of this reader is.
    */
-  public void addReaderWithPriority(IReader reader, int priority) throws IOException {
+  public void addReaderWithPriority(IPointReader reader, int priority) throws IOException {
     if (reader.hasNext()) {
       heap.add(new Element(readerList.size(), reader.next(), priority));
     }
@@ -65,11 +63,16 @@ public class PriorityMergeReader implements IReader {
     return top.timeValuePair;
   }
 
+  @Override
+  public TimeValuePair current() {
+    return heap.peek().timeValuePair;
+  }
+
   private void updateHeap(Element top) throws IOException {
     while (!heap.isEmpty() && heap.peek().timeValuePair.getTimestamp() == top.timeValuePair
         .getTimestamp()) {
       Element e = heap.poll();
-      IReader reader = readerList.get(e.index);
+      IPointReader reader = readerList.get(e.index);
       if (reader.hasNext()) {
         heap.add(new Element(e.index, reader.next(), priorityList.get(e.index)));
       }
@@ -77,35 +80,12 @@ public class PriorityMergeReader implements IReader {
   }
 
   @Override
-  public void skipCurrentTimeValuePair() throws IOException {
-    if (hasNext()) {
-      next();
-    }
-  }
-
-  @Override
   public void close() throws IOException {
-    for (IReader reader : readerList) {
+    for (IPointReader reader : readerList) {
       reader.close();
     }
   }
 
-  @Override
-  public boolean hasNextBatch() {
-    // TODO
-    return false;
-  }
-
-  @Override
-  public BatchData nextBatch() {
-    return null;
-  }
-
-  @Override
-  public BatchData currentBatch() {
-    return null;
-  }
-
   protected class Element implements Comparable<Element> {
 
     int index;
@@ -133,11 +113,11 @@ public class PriorityMergeReader implements IReader {
     }
 
     @Override
-    public boolean equals(Object o){
-      if (o instanceof Element){
+    public boolean equals(Object o) {
+      if (o instanceof Element) {
         Element element = (Element) o;
         if (this.timeValuePair.getTimestamp() == element.timeValuePair.getTimestamp()
-        && this.priority.equals(element.priority)){
+            && this.priority.equals(element.priority)) {
           return true;
         }
       }
@@ -145,7 +125,7 @@ public class PriorityMergeReader implements IReader {
     }
 
     @Override
-    public int hashCode(){
+    public int hashCode() {
       return (int) (timeValuePair.getTimestamp() * 31 + priority.hashCode());
     }
   }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java
index 244291a..0f42e87 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java
@@ -16,44 +16,54 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.query.reader.merge;
 
 import java.io.IOException;
-import org.apache.iotdb.db.utils.TimeValuePair;
-import org.apache.iotdb.db.utils.TsPrimitiveType;
+import java.util.ArrayList;
+import java.util.List;
 
 /**
- * TODO the process of PriorityMergeReaderByTimestamp can be optimized.
+ * <p>
+ * Usage: Get value in timestamp by sorting time-value pair in multiple readers with time and
+ * priority. (1) merge multiple chunk group readers in the unsequence file (2)merge sequence reader,
+ * unsequence reader and mem reader
+ * </p>
  */
-public class PriorityMergeReaderByTimestamp extends PriorityMergeReader implements
-    EngineReaderByTimeStamp {
+public class PriorityMergeReaderByTimestamp implements EngineReaderByTimeStamp {
+
+  private List<EngineReaderByTimeStamp> readerList = new ArrayList<>();
+  private List<Integer> priorityList = new ArrayList<>();
 
-  private boolean hasCachedTimeValuePair;
-  private TimeValuePair cachedTimeValuePair;
+  /**
+   * This function doesn't sort reader by priority. So you have to call this function in order of
+   * reader priority from small to large.
+   */
+  public void addReaderWithPriority(EngineReaderByTimeStamp reader, int priority) {
+    readerList.add(reader);
+    priorityList.add(priority);
+  }
 
   @Override
-  public TsPrimitiveType getValueInTimestamp(long timestamp) throws IOException {
-
-    if (hasCachedTimeValuePair) {
-      if (cachedTimeValuePair.getTimestamp() == timestamp) {
-        hasCachedTimeValuePair = false;
-        return cachedTimeValuePair.getValue();
-      } else if (cachedTimeValuePair.getTimestamp() > timestamp) {
-        return null;
+  public Object getValueInTimestamp(long timestamp) throws IOException {
+    Object value = null;
+    for (int i = readerList.size() - 1; i >= 0; i--) {
+      value = readerList.get(i).getValueInTimestamp(timestamp);
+      if (value != null) {
+        return value;
       }
     }
+    return value;
+  }
 
-    while (hasNext()) {
-      cachedTimeValuePair = next();
-      if (cachedTimeValuePair.getTimestamp() == timestamp) {
-        hasCachedTimeValuePair = false;
-        return cachedTimeValuePair.getValue();
-      } else if (cachedTimeValuePair.getTimestamp() > timestamp) {
-        hasCachedTimeValuePair = true;
-        return null;
+  @Override
+  public boolean hasNext() throws IOException {
+    for (int i = readerList.size() - 1; i >= 0; i--) {
+      if (readerList.get(i).hasNext()) {
+        return true;
       }
     }
-
-    return null;
+    return false;
   }
+
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReader.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReader.java
index d83af0f..075fa08 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReader.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReader.java
@@ -21,15 +21,16 @@ package org.apache.iotdb.db.query.reader.sequence;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import org.apache.iotdb.db.engine.filenode.TsFileResource;
 import org.apache.iotdb.db.engine.modification.Modification;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.control.FileReaderManager;
-import org.apache.iotdb.db.query.reader.IReader;
+import org.apache.iotdb.db.query.reader.IAggregateReader;
+import org.apache.iotdb.db.query.reader.IBatchReader;
 import org.apache.iotdb.db.utils.QueryUtils;
-import org.apache.iotdb.db.utils.TimeValuePair;
-import org.apache.iotdb.db.utils.TimeValuePairUtils;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
 import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
 import org.apache.iotdb.tsfile.read.common.BatchData;
@@ -42,37 +43,50 @@ import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReader;
 import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderWithFilter;
 import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderWithoutFilter;
 
-public class SealedTsFilesReader implements IReader {
+public class SealedTsFilesReader implements IBatchReader, IAggregateReader {
 
   private Path seriesPath;
   private List<TsFileResource> sealedTsFiles;
-  private int usedIntervalFileIndex;
+  private int indexOfNextTsFileResource;
   private FileSeriesReader seriesReader;
   private Filter filter;
-  private BatchData data;
-  private boolean hasCachedData;
   private QueryContext context;
+  private boolean isReverse;
 
+  /**
+   * init with seriesPath, sealedTsFiles, filter, context.
+   *
+   * @param seriesPath path
+   * @param sealedTsFiles sealed file list
+   * @param filter null if no filter
+   * @param isReverse true-traverse chunks from behind forward, false-traverse chunks from front to
+   * back.
+   */
   public SealedTsFilesReader(Path seriesPath, List<TsFileResource> sealedTsFiles, Filter filter,
-      QueryContext context) {
-    this(seriesPath, sealedTsFiles, context);
+      QueryContext context, boolean isReverse) {
+    this(seriesPath, sealedTsFiles, context, isReverse);
     this.filter = filter;
-
   }
 
   /**
    * init with seriesPath and sealedTsFiles.
    */
   public SealedTsFilesReader(Path seriesPath, List<TsFileResource> sealedTsFiles,
-      QueryContext context) {
+      QueryContext context, boolean isReverse) {
+    if (isReverse) {
+      Collections.reverse(sealedTsFiles);
+    }
     this.seriesPath = seriesPath;
     this.sealedTsFiles = sealedTsFiles;
-    this.usedIntervalFileIndex = 0;
+    this.indexOfNextTsFileResource = 0;
     this.seriesReader = null;
-    this.hasCachedData = false;
     this.context = context;
+    this.isReverse = isReverse;
   }
 
+  /**
+   * init with seriesReader and queryContext.
+   */
   public SealedTsFilesReader(FileSeriesReader seriesReader, QueryContext context) {
     this.seriesReader = seriesReader;
     sealedTsFiles = new ArrayList<>();
@@ -81,109 +95,63 @@ public class SealedTsFilesReader implements IReader {
 
   @Override
   public boolean hasNext() throws IOException {
-    if (hasCachedData) {
+
+    // try to get next batch data from current reader
+    if (seriesReader != null && seriesReader.hasNextBatch()) {
       return true;
     }
 
-    while (!hasCachedData) {
-      boolean flag = false;
-
-      // try to get next time value pair from current batch data
-      if (data != null && data.hasNext()) {
-        hasCachedData = true;
-        return true;
-      }
-
-      // try to get next batch data from current reader
-      if (seriesReader != null && seriesReader.hasNextBatch()) {
-        data = seriesReader.nextBatch();
-        if (data.hasNext()) {
-          hasCachedData = true;
-          return true;
-        } else {
-          flag = true;
-        }
-      }
-
+    // init until reach a satisfied reader
+    while (indexOfNextTsFileResource < sealedTsFiles.size()) {
       // try to get next batch data from next reader
-      while (!flag && usedIntervalFileIndex < sealedTsFiles.size()) {
-        // init until reach a satisfied reader
-        if (seriesReader == null || !seriesReader.hasNextBatch()) {
-          TsFileResource fileNode = sealedTsFiles.get(usedIntervalFileIndex++);
-          if (singleTsFileSatisfied(fileNode)) {
-            initSingleTsFileReader(fileNode, context);
-          } else {
-            flag = true;
-          }
-        }
-        if (!flag && seriesReader.hasNextBatch()) {
-          data = seriesReader.nextBatch();
-
-          // notice that, data maybe an empty batch data, so an examination must exist
-          if (data.hasNext()) {
-            hasCachedData = true;
-            return true;
-          }
-        }
+      TsFileResource tsfile = sealedTsFiles.get(indexOfNextTsFileResource++);
+      if (singleTsFileSatisfied(tsfile)) {
+        initSingleTsFileReader(tsfile, context);
+      } else {
+        continue;
       }
 
-      if (!flag || data == null || !data.hasNext()) {
-        break;
+      if (seriesReader.hasNextBatch()) {
+        return true;
       }
     }
 
     return false;
   }
 
-  @Override
-  public TimeValuePair next() {
-    TimeValuePair timeValuePair = TimeValuePairUtils.getCurrentTimeValuePair(data);
-    data.next();
-    hasCachedData = false;
-    return timeValuePair;
-  }
-
-  @Override
-  public void skipCurrentTimeValuePair() {
-    next();
-  }
-
-  @Override
-  public void close() throws IOException {
-    if (seriesReader != null) {
-      seriesReader.close();
-    }
-  }
-
-  private boolean singleTsFileSatisfied(TsFileResource fileNode) {
+  private boolean singleTsFileSatisfied(TsFileResource tsfile) {
 
     if (filter == null) {
       return true;
     }
 
-    long startTime = fileNode.getStartTime(seriesPath.getDevice());
-    long endTime = fileNode.getEndTime(seriesPath.getDevice());
+    long startTime = tsfile.getStartTime(seriesPath.getDevice());
+    long endTime = tsfile.getEndTime(seriesPath.getDevice());
     return filter.satisfyStartEndTime(startTime, endTime);
   }
 
-  private void initSingleTsFileReader(TsFileResource fileNode, QueryContext context)
+  private void initSingleTsFileReader(TsFileResource tsfile, QueryContext context)
       throws IOException {
 
     // to avoid too many opened files
     TsFileSequenceReader tsFileReader = FileReaderManager.getInstance()
-        .get(fileNode.getFilePath(), true);
+        .get(tsfile.getFilePath(), true);
 
     MetadataQuerierByFileImpl metadataQuerier = new MetadataQuerierByFileImpl(tsFileReader);
     List<ChunkMetaData> metaDataList = metadataQuerier.getChunkMetaDataList(seriesPath);
 
-    List<Modification> pathModifications = context.getPathModifications(fileNode.getModFile(),
+    List<Modification> pathModifications = context.getPathModifications(tsfile.getModFile(),
         seriesPath.getFullPath());
-    if (pathModifications.size() > 0) {
+    if (!pathModifications.isEmpty()) {
       QueryUtils.modifyChunkMetaData(metaDataList, pathModifications);
     }
 
     ChunkLoader chunkLoader = new ChunkLoaderImpl(tsFileReader);
 
+    if (isReverse) {
+      Collections.reverse(metaDataList);
+    }
+
     if (filter == null) {
       seriesReader = new FileSeriesReaderWithoutFilter(chunkLoader, metaDataList);
     } else {
@@ -192,17 +160,24 @@ public class SealedTsFilesReader implements IReader {
   }
 
   @Override
-  public boolean hasNextBatch() {
-    return false;
+  public BatchData nextBatch() throws IOException {
+    return seriesReader.nextBatch();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (seriesReader != null) {
+      seriesReader.close();
+    }
   }
 
   @Override
-  public BatchData nextBatch() {
-    return null;
+  public PageHeader nextPageHeader() throws IOException {
+    return seriesReader.nextPageHeader();
   }
 
   @Override
-  public BatchData currentBatch() {
-    return null;
+  public void skipPageData() {
+    seriesReader.skipPageData();
   }
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java
new file mode 100644
index 0000000..e58583c
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java
@@ -0,0 +1,136 @@
+/**
+ * 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.iotdb.db.query.reader.sequence;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.iotdb.db.engine.filenode.TsFileResource;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.FileReaderManager;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.db.utils.QueryUtils;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.controller.ChunkLoader;
+import org.apache.iotdb.tsfile.read.controller.ChunkLoaderImpl;
+import org.apache.iotdb.tsfile.read.controller.MetadataQuerierByFileImpl;
+import org.apache.iotdb.tsfile.read.reader.series.SeriesReaderByTimestamp;
+
+public class SealedTsFilesReaderByTimestamp implements EngineReaderByTimeStamp {
+
+  private Path seriesPath;
+  private List<TsFileResource> sealedTsFiles;
+  private int nextIntervalFileIndex;
+  private SeriesReaderByTimestamp seriesReader;
+  private QueryContext context;
+
+  /**
+   * init with seriesPath and sealedTsFiles.
+   */
+  public SealedTsFilesReaderByTimestamp(Path seriesPath, List<TsFileResource> sealedTsFiles,
+      QueryContext context) {
+    this.seriesPath = seriesPath;
+    this.sealedTsFiles = sealedTsFiles;
+    this.nextIntervalFileIndex = 0;
+    this.seriesReader = null;
+    this.context = context;
+  }
+
+  @Override
+  public Object getValueInTimestamp(long timestamp) throws IOException {
+    Object value = null;
+    if (seriesReader != null) {
+      value = seriesReader.getValueInTimestamp(timestamp);
+      if (value != null || seriesReader.hasNext()) {
+        return value;
+      }
+    }
+    constructReader(timestamp);
+    if (seriesReader != null) {
+      value = seriesReader.getValueInTimestamp(timestamp);
+      if (value != null || seriesReader.hasNext()) {
+        return value;
+      }
+    }
+
+    return value;
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (seriesReader != null && seriesReader.hasNext()) {
+      return true;
+    }
+    while (nextIntervalFileIndex < sealedTsFiles.size()) {
+      initSingleTsFileReader(sealedTsFiles.get(nextIntervalFileIndex), context);
+      nextIntervalFileIndex++;
+      if (seriesReader.hasNext()) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * construct reader with the file that might overlap this timestamp.
+   */
+  private void constructReader(long timestamp) throws IOException {
+    while (nextIntervalFileIndex < sealedTsFiles.size()) {
+      if (singleTsFileSatisfied(sealedTsFiles.get(nextIntervalFileIndex), timestamp)) {
+        initSingleTsFileReader(sealedTsFiles.get(nextIntervalFileIndex), context);
+        nextIntervalFileIndex++;
+        return;
+      }
+      nextIntervalFileIndex++;
+    }
+  }
+
+  /**
+   * Judge whether the file should be skipped.
+   */
+  private boolean singleTsFileSatisfied(TsFileResource fileNode, long timestamp) {
+    long endTime = fileNode.getEndTime(seriesPath.getDevice());
+    return endTime >= timestamp;
+  }
+
+  private void initSingleTsFileReader(TsFileResource fileNode, QueryContext context)
+      throws IOException {
+
+    // to avoid too many opened files
+    TsFileSequenceReader tsFileReader = FileReaderManager.getInstance()
+        .get(fileNode.getFilePath(), true);
+
+    MetadataQuerierByFileImpl metadataQuerier = new MetadataQuerierByFileImpl(tsFileReader);
+    List<ChunkMetaData> metaDataList = metadataQuerier.getChunkMetaDataList(seriesPath);
+
+    List<Modification> pathModifications = context.getPathModifications(fileNode.getModFile(),
+        seriesPath.getFullPath());
+    if (!pathModifications.isEmpty()) {
+      QueryUtils.modifyChunkMetaData(metaDataList, pathModifications);
+    }
+    ChunkLoader chunkLoader = new ChunkLoaderImpl(tsFileReader);
+
+    seriesReader = new SeriesReaderByTimestamp(chunkLoader, metaDataList);
+
+  }
+
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReader.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReader.java
index e4ca98f..b691d7f 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReader.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReader.java
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.query.reader.sequence;
 
 import java.io.IOException;
@@ -23,10 +24,10 @@ import java.util.ArrayList;
 import java.util.List;
 import org.apache.iotdb.db.engine.querycontext.GlobalSortedSeriesDataSource;
 import org.apache.iotdb.db.query.context.QueryContext;
-import org.apache.iotdb.db.query.reader.IReader;
-import org.apache.iotdb.db.query.reader.mem.MemChunkReaderWithFilter;
-import org.apache.iotdb.db.query.reader.mem.MemChunkReaderWithoutFilter;
-import org.apache.iotdb.db.utils.TimeValuePair;
+import org.apache.iotdb.db.query.reader.IAggregateReader;
+import org.apache.iotdb.db.query.reader.IBatchReader;
+import org.apache.iotdb.db.query.reader.mem.MemChunkReader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
 import org.apache.iotdb.tsfile.read.common.BatchData;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 
@@ -36,49 +37,77 @@ import org.apache.iotdb.tsfile.read.filter.basic.Filter;
  * in MemTable.
  * </p>
  */
-public class SequenceDataReader implements IReader {
+public class SequenceDataReader implements IBatchReader, IAggregateReader {
 
-  private List<IReader> seriesReaders;
+  private List<IAggregateReader> seriesReaders;
   private boolean curReaderInitialized;
   private int nextSeriesReaderIndex;
-  private IReader currentSeriesReader;
+  private IAggregateReader currentSeriesReader;
 
   /**
-   * init with globalSortedSeriesDataSource and filter.
+   * init with globalSortedSeriesDataSource, filter, context and isReverse.
+   *
+   * @param sources data source
+   * @param filter null if no filter
+   * @param context query context
+   * @param isReverse true-traverse chunks from behind forward, false-traverse chunks from front to
+   * back.
    */
   public SequenceDataReader(GlobalSortedSeriesDataSource sources, Filter filter,
-      QueryContext context)
-      throws IOException {
+      QueryContext context, boolean isReverse) throws IOException {
     seriesReaders = new ArrayList<>();
 
     curReaderInitialized = false;
     nextSeriesReaderIndex = 0;
 
-    // add reader for sealed TsFiles
-    if (sources.hasSealedTsFiles()) {
-      seriesReaders.add(
-          new SealedTsFilesReader(sources.getSeriesPath(), sources.getSealedTsFiles(), filter,
-              context));
-    }
+    if (isReverse) {
+      // add data in memTable
+      if (sources.hasRawSeriesChunk()) {
+        seriesReaders.add(new MemChunkReader(sources.getReadableChunk(), filter));
+      }
 
-    // add reader for unSealed TsFile
-    if (sources.hasUnsealedTsFile()) {
-      seriesReaders.add(new UnSealedTsFileReader(sources.getUnsealedTsFile(), filter));
-    }
+      // add reader for unSealed TsFile
+      if (sources.hasUnsealedTsFile()) {
+        seriesReaders.add(new UnSealedTsFileReader(sources.getUnsealedTsFile(), filter, isReverse));
+      }
+
+      // add reader for sealed TsFiles
+      if (sources.hasSealedTsFiles()) {
+        seriesReaders.add(
+            new SealedTsFilesReader(sources.getSeriesPath(), sources.getSealedTsFiles(), filter,
+                context, isReverse));
+      }
+    } else {
+      // add reader for sealed TsFiles
+      if (sources.hasSealedTsFiles()) {
+        seriesReaders.add(
+            new SealedTsFilesReader(sources.getSeriesPath(), sources.getSealedTsFiles(), filter,
+                context, isReverse));
+      }
 
-    // add data in memTable
-    if (sources.hasRawSeriesChunk()) {
-      if (filter == null) {
-        seriesReaders.add(new MemChunkReaderWithoutFilter(sources.getReadableChunk()));
-      } else {
-        seriesReaders.add(new MemChunkReaderWithFilter(sources.getReadableChunk(), filter));
+      // add reader for unSealed TsFile
+      if (sources.hasUnsealedTsFile()) {
+        seriesReaders.add(new UnSealedTsFileReader(sources.getUnsealedTsFile(), filter, isReverse));
+      }
+
+      // add data in memTable
+      if (sources.hasRawSeriesChunk()) {
+        seriesReaders.add(new MemChunkReader(sources.getReadableChunk(), filter));
       }
     }
+  }
 
+  /**
+   * init with globalSortedSeriesDataSource, filter, context and isReverse.
+   */
+  public SequenceDataReader(GlobalSortedSeriesDataSource sources, Filter filter,
+      QueryContext context) throws IOException {
+    this(sources, filter, context, false);
   }
 
   @Override
   public boolean hasNext() throws IOException {
+
     if (curReaderInitialized && currentSeriesReader.hasNext()) {
       return true;
     } else {
@@ -96,35 +125,24 @@ public class SequenceDataReader implements IReader {
   }
 
   @Override
-  public TimeValuePair next() throws IOException {
-    return currentSeriesReader.next();
-  }
-
-  @Override
-  public void skipCurrentTimeValuePair() throws IOException {
-    next();
-  }
-
-  @Override
   public void close() throws IOException {
-    for (IReader seriesReader : seriesReaders) {
+    for (IBatchReader seriesReader : seriesReaders) {
       seriesReader.close();
     }
   }
 
   @Override
-  public boolean hasNextBatch() {
-    return false;
+  public BatchData nextBatch() throws IOException {
+    return currentSeriesReader.nextBatch();
   }
 
   @Override
-  public BatchData nextBatch() {
-    return null;
+  public PageHeader nextPageHeader() throws IOException {
+    return currentSeriesReader.nextPageHeader();
   }
 
   @Override
-  public BatchData currentBatch() {
-    return null;
+  public void skipPageData() throws IOException {
+    currentSeriesReader.skipPageData();
   }
-
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestamp.java
new file mode 100644
index 0000000..a5fdc44
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestamp.java
@@ -0,0 +1,108 @@
+/**
+ * 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.iotdb.db.query.reader.sequence;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.iotdb.db.engine.querycontext.GlobalSortedSeriesDataSource;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.reader.mem.MemChunkReaderByTimestamp;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+
+public class SequenceDataReaderByTimestamp implements EngineReaderByTimeStamp {
+
+  private List<EngineReaderByTimeStamp> seriesReaders;
+  private int nextSeriesReaderIndex;
+  private EngineReaderByTimeStamp currentSeriesReader;
+
+  /**
+   * init with globalSortedSeriesDataSource and filter.
+   */
+  public SequenceDataReaderByTimestamp(GlobalSortedSeriesDataSource sources, QueryContext context)
+      throws IOException {
+    seriesReaders = new ArrayList<>();
+
+    nextSeriesReaderIndex = 0;
+
+    // add reader for sealed TsFiles
+    if (sources.hasSealedTsFiles()) {
+      seriesReaders.add(
+          new SealedTsFilesReaderByTimestamp(sources.getSeriesPath(), sources.getSealedTsFiles(),
+              context));
+    }
+
+    // add reader for unSealed TsFile
+    if (sources.hasUnsealedTsFile()) {
+      seriesReaders.add(new UnSealedTsFilesReaderByTimestamp(sources.getUnsealedTsFile()));
+    }
+
+    // add data in memTable
+    if (sources.hasRawSeriesChunk()) {
+      seriesReaders.add(new MemChunkReaderByTimestamp(sources.getReadableChunk()));
+    }
+
+  }
+
+  /**
+   * This method is used only in unit test.
+   */
+  public SequenceDataReaderByTimestamp(List<EngineReaderByTimeStamp> seriesReaders) {
+    this.seriesReaders = seriesReaders;
+    nextSeriesReaderIndex = 0;
+  }
+
+
+  @Override
+  public Object getValueInTimestamp(long timestamp) throws IOException {
+    Object value = null;
+    if (currentSeriesReader != null) {
+      value = currentSeriesReader.getValueInTimestamp(timestamp);
+      if (value != null || currentSeriesReader.hasNext()) {
+        return value;
+      }
+    }
+
+    while (nextSeriesReaderIndex < seriesReaders.size()) {
+      currentSeriesReader = seriesReaders.get(nextSeriesReaderIndex++);
+      if (currentSeriesReader != null) {
+        value = currentSeriesReader.getValueInTimestamp(timestamp);
+        if (value != null || currentSeriesReader.hasNext()) {
+          return value;
+        }
+      }
+    }
+    return value;
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currentSeriesReader != null && currentSeriesReader.hasNext()) {
+      return true;
+    }
+    while (nextSeriesReaderIndex < seriesReaders.size()) {
+      currentSeriesReader = seriesReaders.get(nextSeriesReaderIndex++);
+      if (currentSeriesReader != null && currentSeriesReader.hasNext()) {
+        return true;
+      }
+    }
+    return false;
+  }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReader.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReader.java
index 3c02930..d4e75e6 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReader.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReader.java
@@ -16,14 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.query.reader.sequence;
 
 import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
 import org.apache.iotdb.db.engine.querycontext.UnsealedTsFile;
 import org.apache.iotdb.db.query.control.FileReaderManager;
-import org.apache.iotdb.db.query.reader.IReader;
-import org.apache.iotdb.db.utils.TimeValuePair;
-import org.apache.iotdb.db.utils.TimeValuePairUtils;
+import org.apache.iotdb.db.query.reader.IAggregateReader;
+import org.apache.iotdb.db.query.reader.IBatchReader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
 import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
 import org.apache.iotdb.tsfile.read.common.BatchData;
 import org.apache.iotdb.tsfile.read.common.Path;
@@ -34,58 +38,43 @@ import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReader;
 import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderWithFilter;
 import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderWithoutFilter;
 
-public class UnSealedTsFileReader implements IReader {
+public class UnSealedTsFileReader implements IBatchReader, IAggregateReader {
 
   protected Path seriesPath;
   private FileSeriesReader unSealedReader;
-  private BatchData data;
 
   /**
    * Construct funtion for UnSealedTsFileReader.
    *
    * @param unsealedTsFile -param to initial
    * @param filter -filter
+   * @param isReverse true-traverse chunks from behind forward; false-traverse chunks from front to
+   * back;
    */
-  public UnSealedTsFileReader(UnsealedTsFile unsealedTsFile, Filter filter) throws IOException {
+  public UnSealedTsFileReader(UnsealedTsFile unsealedTsFile, Filter filter, boolean isReverse)
+      throws IOException {
 
     TsFileSequenceReader unClosedTsFileReader = FileReaderManager.getInstance()
         .get(unsealedTsFile.getFilePath(),
             false);
     ChunkLoader chunkLoader = new ChunkLoaderImpl(unClosedTsFileReader);
 
+    List<ChunkMetaData> metaDataList = unsealedTsFile.getChunkMetaDataList();
+    //reverse chunk metadata list if traversing chunks from behind forward
+    if (isReverse && metaDataList != null && !metaDataList.isEmpty()) {
+      Collections.reverse(metaDataList);
+    }
+
     if (filter == null) {
-      unSealedReader = new FileSeriesReaderWithoutFilter(chunkLoader,
-          unsealedTsFile.getChunkMetaDataList());
+      unSealedReader = new FileSeriesReaderWithoutFilter(chunkLoader, metaDataList);
     } else {
-      unSealedReader = new FileSeriesReaderWithFilter(chunkLoader,
-          unsealedTsFile.getChunkMetaDataList(),
-          filter);
+      unSealedReader = new FileSeriesReaderWithFilter(chunkLoader, metaDataList, filter);
     }
-
   }
 
   @Override
   public boolean hasNext() throws IOException {
-    if (data == null || !data.hasNext()) {
-      if (!unSealedReader.hasNextBatch()) {
-        return false;
-      }
-      data = unSealedReader.nextBatch();
-    }
-
-    return data.hasNext();
-  }
-
-  @Override
-  public TimeValuePair next() {
-    TimeValuePair timeValuePair = TimeValuePairUtils.getCurrentTimeValuePair(data);
-    data.next();
-    return timeValuePair;
-  }
-
-  @Override
-  public void skipCurrentTimeValuePair() {
-    data.next();
+    return unSealedReader.hasNextBatch();
   }
 
   @Override
@@ -96,17 +85,17 @@ public class UnSealedTsFileReader implements IReader {
   }
 
   @Override
-  public boolean hasNextBatch() {
-    return false;
+  public BatchData nextBatch() throws IOException {
+    return unSealedReader.nextBatch();
   }
 
   @Override
-  public BatchData nextBatch() {
-    return null;
+  public PageHeader nextPageHeader() throws IOException {
+    return unSealedReader.nextPageHeader();
   }
 
   @Override
-  public BatchData currentBatch() {
-    return null;
+  public void skipPageData() {
+    unSealedReader.skipPageData();
   }
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFilesReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFilesReaderByTimestamp.java
new file mode 100644
index 0000000..4aebdb8
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFilesReaderByTimestamp.java
@@ -0,0 +1,64 @@
+/**
+ * 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.iotdb.db.query.reader.sequence;
+
+import java.io.IOException;
+import org.apache.iotdb.db.engine.querycontext.UnsealedTsFile;
+import org.apache.iotdb.db.query.control.FileReaderManager;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.controller.ChunkLoader;
+import org.apache.iotdb.tsfile.read.controller.ChunkLoaderImpl;
+import org.apache.iotdb.tsfile.read.reader.series.SeriesReaderByTimestamp;
+
+public class UnSealedTsFilesReaderByTimestamp implements EngineReaderByTimeStamp {
+
+  protected Path seriesPath;
+  private SeriesReaderByTimestamp unSealedReader;
+
+  /**
+   * Construct funtion for UnSealedTsFileReader.
+   *
+   * @param unsealedTsFile -param to initial
+   */
+  public UnSealedTsFilesReaderByTimestamp(UnsealedTsFile unsealedTsFile) throws IOException {
+
+    TsFileSequenceReader unClosedTsFileReader = FileReaderManager.getInstance()
+        .get(unsealedTsFile.getFilePath(),
+            false);
+    ChunkLoader chunkLoader = new ChunkLoaderImpl(unClosedTsFileReader);
+    unSealedReader = new SeriesReaderByTimestamp(chunkLoader,
+        unsealedTsFile.getChunkMetaDataList());
+
+
+  }
+
+  @Override
+  public Object getValueInTimestamp(long timestamp) throws IOException {
+    return unSealedReader.getValueInTimestamp(timestamp);
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    return unSealedReader.hasNext();
+  }
+
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReader.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReader.java
index e49e968..2465cbd 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReader.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReader.java
@@ -16,17 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.query.reader.unsequence;
 
 import java.io.IOException;
-import org.apache.iotdb.db.query.reader.IReader;
+import org.apache.iotdb.db.query.reader.IPointReader;
 import org.apache.iotdb.db.utils.TimeValuePair;
 import org.apache.iotdb.db.utils.TimeValuePairUtils;
 import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
 import org.apache.iotdb.tsfile.read.common.BatchData;
 import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
 
-public class EngineChunkReader implements IReader {
+public class EngineChunkReader implements IPointReader {
 
   private ChunkReader chunkReader;
   private BatchData data;
@@ -44,15 +45,16 @@ public class EngineChunkReader implements IReader {
 
   @Override
   public boolean hasNext() throws IOException {
-    if (data == null || !data.hasNext()) {
-      if (chunkReader.hasNextBatch()) {
-        data = chunkReader.nextBatch();
-      } else {
-        return false;
+    if (data != null && data.hasNext()) {
+      return true;
+    }
+    while (chunkReader.hasNextBatch()) {
+      data = chunkReader.nextBatch();
+      if (data.hasNext()) {
+        return true;
       }
     }
-
-    return data.hasNext();
+    return false;
   }
 
   @Override
@@ -63,8 +65,8 @@ public class EngineChunkReader implements IReader {
   }
 
   @Override
-  public void skipCurrentTimeValuePair() {
-    next();
+  public TimeValuePair current() {
+    return TimeValuePairUtils.getCurrentTimeValuePair(data);
   }
 
   @Override
@@ -72,19 +74,4 @@ public class EngineChunkReader implements IReader {
     this.chunkReader.close();
     this.unClosedTsFileReader.close();
   }
-
-  @Override
-  public boolean hasNextBatch() {
-    return false;
-  }
-
-  @Override
-  public BatchData nextBatch() {
-    return null;
-  }
-
-  @Override
-  public BatchData currentBatch() {
-    return null;
-  }
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReaderByTimestamp.java
new file mode 100644
index 0000000..c6f22a4
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReaderByTimestamp.java
@@ -0,0 +1,77 @@
+/**
+ * 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.iotdb.db.query.reader.unsequence;
+
+import java.io.IOException;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReaderByTimestamp;
+
+public class EngineChunkReaderByTimestamp implements EngineReaderByTimeStamp {
+
+  private ChunkReaderByTimestamp chunkReader;
+  private BatchData data;
+
+  public EngineChunkReaderByTimestamp(ChunkReaderByTimestamp chunkReader) {
+    this.chunkReader = chunkReader;
+  }
+
+  /**
+   * get value with time equals timestamp. If there is no such point, return null.
+   */
+  @Override
+  public Object getValueInTimestamp(long timestamp) throws IOException {
+
+    if (!hasNext()) {
+      return null;
+    }
+
+    while (data != null) {
+      Object value = data.getValueInTimestamp(timestamp);
+      if (value != null) {
+        return value;
+      }
+      if (data.hasNext()) {
+        return null;
+      } else {
+        chunkReader.setCurrentTimestamp(timestamp);
+        if (chunkReader.hasNextBatch()) {
+          data = chunkReader.nextBatch();
+        } else {
+          return null;
+        }
+      }
+    }
+
+    return null;
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (data != null && data.hasNext()) {
+      return true;
+    }
+    if (chunkReader != null && chunkReader.hasNextBatch()) {
+      data = chunkReader.nextBatch();
+      return true;
+    }
+    return false;
+  }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineLeafNode.java b/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineLeafNode.java
index 292ca0e..3d4b11c 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineLeafNode.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineLeafNode.java
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.query.timegenerator;
 
 import java.io.IOException;
@@ -45,16 +46,6 @@ public class EngineLeafNode implements Node {
   }
 
   /**
-   * check if current time of current batch is equals to input time.
-   */
-  public boolean currentTimeIs(long time) {
-    if (!reader.currentBatch().hasNext()) {
-      return false;
-    }
-    return reader.currentBatch().currentTime() == time;
-  }
-
-  /**
    * check if current value is equals to input value.
    */
   public Object currentValue(long time) {
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineNodeConstructor.java b/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineNodeConstructor.java
index 0cb02c8..4ffe62b 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineNodeConstructor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineNodeConstructor.java
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.query.timegenerator;
 
 import static org.apache.iotdb.tsfile.read.expression.ExpressionType.AND;
@@ -26,8 +27,9 @@ import java.io.IOException;
 import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
 import org.apache.iotdb.db.exception.FileNodeManagerException;
 import org.apache.iotdb.db.query.context.QueryContext;
-import org.apache.iotdb.db.query.control.QueryDataSourceManager;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
 import org.apache.iotdb.db.query.factory.SeriesReaderFactory;
+import org.apache.iotdb.db.query.reader.AllDataReader;
 import org.apache.iotdb.db.query.reader.IReader;
 import org.apache.iotdb.db.query.reader.merge.PriorityMergeReader;
 import org.apache.iotdb.db.query.reader.sequence.SequenceDataReader;
@@ -42,10 +44,8 @@ import org.apache.iotdb.tsfile.read.query.timegenerator.node.OrNode;
 
 public class EngineNodeConstructor {
 
-  private long jobId;
 
-  public EngineNodeConstructor(long jobId) {
-    this.jobId = jobId;
+  public EngineNodeConstructor() {
   }
 
   /**
@@ -87,24 +87,26 @@ public class EngineNodeConstructor {
       QueryContext context)
       throws IOException, FileNodeManagerException {
 
-    QueryDataSource queryDataSource = QueryDataSourceManager.getQueryDataSource(jobId,
+    QueryDataSource queryDataSource = QueryResourceManager.getInstance().getQueryDataSource(
         singleSeriesExpression.getSeriesPath(), context);
 
     Filter filter = singleSeriesExpression.getFilter();
 
-    PriorityMergeReader priorityReader = new PriorityMergeReader();
-
     // reader for all sequence data
     SequenceDataReader tsFilesReader = new SequenceDataReader(queryDataSource.getSeqDataSource(),
         filter, context);
-    priorityReader.addReaderWithPriority(tsFilesReader, 1);
 
     // reader for all unSequence data
     PriorityMergeReader unSeqMergeReader = SeriesReaderFactory.getInstance()
         .createUnSeqMergeReader(queryDataSource.getOverflowSeriesDataSource(), filter);
-    priorityReader.addReaderWithPriority(unSeqMergeReader, 2);
 
-    return priorityReader;
+    if (!tsFilesReader.hasNext()) {
+      //only have unsequence data.
+      return unSeqMergeReader;
+    } else {
+      //merge sequence data with unsequence data.
+      return new AllDataReader(tsFilesReader, unSeqMergeReader);
+    }
   }
 
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineTimeGenerator.java b/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineTimeGenerator.java
index a151677..350ea6f 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineTimeGenerator.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineTimeGenerator.java
@@ -34,20 +34,18 @@ public class EngineTimeGenerator implements TimeGenerator {
 
   private IExpression expression;
   private Node operatorNode;
-  private long jobId;
 
   /**
    * Constructor of EngineTimeGenerator.
    */
-  public EngineTimeGenerator(long jobId, IExpression expression, QueryContext context)
-      throws IOException, FileNodeManagerException {
-    this.jobId = jobId;
+  public EngineTimeGenerator(IExpression expression, QueryContext context)
+      throws FileNodeManagerException {
     this.expression = expression;
     initNode(context);
   }
 
-  private void initNode(QueryContext context) throws IOException, FileNodeManagerException {
-    EngineNodeConstructor engineNodeConstructor = new EngineNodeConstructor(jobId);
+  private void initNode(QueryContext context) throws FileNodeManagerException {
+    EngineNodeConstructor engineNodeConstructor = new EngineNodeConstructor();
     this.operatorNode = engineNodeConstructor.construct(expression, context);
   }
 
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java b/iotdb/src/main/java/org/apache/iotdb/db/rescon/package-info.java
similarity index 81%
copy from iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java
copy to iotdb/src/main/java/org/apache/iotdb/db/rescon/package-info.java
index 3ac1540..96c83c9 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/rescon/package-info.java
@@ -1,29 +1,24 @@
-/**
- * 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.iotdb.db.query.control;
-
-import org.junit.Test;
-
-public class QueryTokenManagerTest {
-
-  @Test
-  public void test() {
-    //TODO
-  }
-}
+/**
+ * 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.
+ */
+
+/**
+ * rescon means Resource Control, classes in this package provide global control over various
+ * resources shared in IoTDB.
+ */
+package org.apache.iotdb.db.rescon;
\ No newline at end of file
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/service/IoTDB.java b/iotdb/src/main/java/org/apache/iotdb/db/service/IoTDB.java
index c63ec74..d85f056 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/service/IoTDB.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/service/IoTDB.java
@@ -105,7 +105,6 @@ public class IoTDB implements IoTDBMBean {
     registerManager.register(CloseMergeService.getInstance());
     registerManager.register(StatMonitor.getInstance());
     registerManager.register(BasicMemController.getInstance());
-    registerManager.register(FileReaderManager.getInstance());
     registerManager.register(SyncServerManager.getInstance());
 
     JMXService.registerMBean(getInstance(), mbeanName);
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java b/iotdb/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
index c5fd6f7..a68f586 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
@@ -49,8 +49,8 @@ import org.apache.iotdb.db.qp.logical.Operator;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
 import org.apache.iotdb.db.qp.physical.sys.AuthorPlan;
-import org.apache.iotdb.db.query.control.OpenedFilePathsManager;
-import org.apache.iotdb.db.query.control.QueryTokenManager;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
 import org.apache.iotdb.service.rpc.thrift.ServerProperties;
 import org.apache.iotdb.service.rpc.thrift.TSCancelOperationReq;
 import org.apache.iotdb.service.rpc.thrift.TSCancelOperationResp;
@@ -79,6 +79,7 @@ import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneResp;
 import org.apache.iotdb.service.rpc.thrift.TS_SessionHandle;
 import org.apache.iotdb.service.rpc.thrift.TS_Status;
 import org.apache.iotdb.service.rpc.thrift.TS_StatusCode;
+import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
 import org.apache.thrift.TException;
@@ -104,6 +105,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   private ThreadLocal<HashMap<String, QueryDataSet>> queryRet = new ThreadLocal<>();
   private ThreadLocal<ZoneId> zoneIds = new ThreadLocal<>();
   private IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private ThreadLocal<Map<Long, QueryContext>> contextMapLocal = new ThreadLocal<>();
 
   public TSServiceImpl() throws IOException {
     // do nothing because there is no need
@@ -184,11 +186,8 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   public TSCloseOperationResp closeOperation(TSCloseOperationReq req) throws TException {
     LOGGER.info("{}: receive close operation", IoTDBConstant.GLOBAL_DB_NAME);
     try {
-      // end query for all the query tokens created by current thread
-      QueryTokenManager.getInstance().endQueryForCurrentRequestThread();
 
-      // remove usage of opened file paths of current thread
-      OpenedFilePathsManager.getInstance().removeUsedFilesForCurrentRequestThread();
+      releaseQueryResource(req);
 
       clearAllStatusForCurrentRequest();
     } catch (FileNodeManagerException e) {
@@ -197,6 +196,22 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     return new TSCloseOperationResp(new TS_Status(TS_StatusCode.SUCCESS_STATUS));
   }
 
+  private void releaseQueryResource(TSCloseOperationReq req) throws FileNodeManagerException {
+    Map<Long, QueryContext> contextMap = contextMapLocal.get();
+    if (contextMap == null) {
+      return;
+    }
+    if(req == null || req.queryId == -1) {
+      // end query for all the query tokens created by current thread
+      for (QueryContext context : contextMap.values()) {
+        QueryResourceManager.getInstance().endQueryForGivenJob(context.getJobId());
+      }
+    } else {
+      QueryResourceManager.getInstance()
+          .endQueryForGivenJob(contextMap.remove(req.queryId).getJobId());
+    }
+  }
+
   private void clearAllStatusForCurrentRequest() {
     if (this.queryRet.get() != null) {
       this.queryRet.get().clear();
@@ -206,7 +221,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     }
   }
 
-  private TS_Status getErrorStatus(String message){
+  private TS_Status getErrorStatus(String message) {
     TS_Status status = new TS_Status(TS_StatusCode.ERROR_STATUS);
     status.setErrorMessage(message);
     return status;
@@ -230,7 +245,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
           resp.setShowTimeseriesList(showTimeseriesList);
         } catch (PathErrorException e) {
           status = getErrorStatus(
-                  String.format("Failed to fetch timeseries %s's metadata because: %s",
+              String.format("Failed to fetch timeseries %s's metadata because: %s",
                   req.getColumnPath(), e));
           resp.setStatus(status);
           return resp;
@@ -239,8 +254,8 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
               .error(String.format("Failed to fetch timeseries %s's metadata", req.getColumnPath()),
                   outOfMemoryError);
           status = getErrorStatus(
-                  String.format("Failed to fetch timeseries %s's metadata because: %s",
-                          req.getColumnPath(), outOfMemoryError));
+              String.format("Failed to fetch timeseries %s's metadata because: %s",
+                  req.getColumnPath(), outOfMemoryError));
           resp.setStatus(status);
           return resp;
         }
@@ -251,14 +266,15 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
           Set<String> storageGroups = MManager.getInstance().getAllStorageGroup();
           resp.setShowStorageGroups(storageGroups);
         } catch (PathErrorException e) {
-          status = getErrorStatus(String.format("Failed to fetch storage groups' metadata because: %s", e));
+          status = getErrorStatus(
+              String.format("Failed to fetch storage groups' metadata because: %s", e));
           resp.setStatus(status);
           return resp;
         } catch (OutOfMemoryError outOfMemoryError) { // TODO OOME
           LOGGER.error("Failed to fetch storage groups' metadata", outOfMemoryError);
           status = getErrorStatus(
-                  String.format("Failed to fetch storage groups' metadata because: %s",
-                          outOfMemoryError));
+              String.format("Failed to fetch storage groups' metadata because: %s",
+                  outOfMemoryError));
           resp.setStatus(status);
           return resp;
         }
@@ -270,7 +286,8 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
           metadataInJson = MManager.getInstance().getMetadataInString();
         } catch (OutOfMemoryError outOfMemoryError) { // TODO OOME
           LOGGER.error("Failed to fetch all metadata in json", outOfMemoryError);
-          status = getErrorStatus(String.format("Failed to fetch all metadata in json because: %s", outOfMemoryError));
+          status = getErrorStatus(
+              String.format("Failed to fetch all metadata in json because: %s", outOfMemoryError));
           resp.setStatus(status);
           return resp;
         }
@@ -295,7 +312,8 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
           return resp;
         } catch (OutOfMemoryError outOfMemoryError) { // TODO OOME
           LOGGER.error("Failed to get delta object map", outOfMemoryError);
-          status = getErrorStatus(String.format("Failed to get delta object map because: %s", outOfMemoryError));
+          status = getErrorStatus(
+              String.format("Failed to get delta object map because: %s", outOfMemoryError));
           break;
         }
         status = new TS_Status(TS_StatusCode.SUCCESS_STATUS);
@@ -317,7 +335,8 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
         try {
           resp.setColumnsList(MManager.getInstance().getPaths(req.getColumnPath()));
         } catch (PathErrorException e) {
-          status = getErrorStatus(String.format("Failed to fetch %s's all columns because: %s", req.getColumnPath(), e));
+          status = getErrorStatus(String
+              .format("Failed to fetch %s's all columns because: %s", req.getColumnPath(), e));
           resp.setStatus(status);
           return resp;
         } catch (OutOfMemoryError outOfMemoryError) { // TODO OOME
@@ -587,10 +606,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
       int fetchSize = req.getFetch_size();
       QueryDataSet queryDataSet;
       if (!queryRet.get().containsKey(statement)) {
-        PhysicalPlan physicalPlan = queryStatus.get().get(statement);
-        processor.getExecutor().setFetchSize(fetchSize);
-        queryDataSet = processor.getExecutor().processQuery(physicalPlan);
-        queryRet.get().put(statement, queryDataSet);
+        queryDataSet = createNewDataSet(statement, fetchSize, req);
       } else {
         queryDataSet = queryRet.get().get(statement);
       }
@@ -610,6 +626,26 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     }
   }
 
+  private QueryDataSet createNewDataSet(String statement, int fetchSize, TSFetchResultsReq req)
+      throws PathErrorException, QueryFilterOptimizationException, FileNodeManagerException,
+      ProcessorException, IOException {
+    PhysicalPlan physicalPlan = queryStatus.get().get(statement);
+    processor.getExecutor().setFetchSize(fetchSize);
+
+    QueryContext context = new QueryContext(QueryResourceManager.getInstance().assignJobId());
+    Map<Long, QueryContext> contextMap = contextMapLocal.get();
+    if (contextMap == null) {
+      contextMap = new HashMap<>();
+      contextMapLocal.set(contextMap);
+    }
+    contextMap.put(req.queryId, context);
+
+    QueryDataSet queryDataSet = processor.getExecutor().processQuery((QueryPlan) physicalPlan,
+        context);
+    queryRet.get().put(statement, queryDataSet);
+    return queryDataSet;
+  }
+
   @Override
   public TSExecuteStatementResp executeUpdateStatement(TSExecuteStatementReq req)
       throws TException {
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java b/iotdb/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java
index 784abc0..8f6fe21 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java
@@ -33,6 +33,9 @@ public class QueryUtils {
   /**
    * modifyChunkMetaData iterates the chunkMetaData and applies all available modifications on it to
    * generate a ModifiedChunkMetadata.
+   * <br/>
+   * the caller should guarantee that chunkMetaData and modifications refer to the same time series
+   * paths.
    * @param chunkMetaData the original chunkMetaData.
    * @param modifications all possible modifications.
    */
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/utils/TimeValuePairUtils.java b/iotdb/src/main/java/org/apache/iotdb/db/utils/TimeValuePairUtils.java
index 241222a..72fb735 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/utils/TimeValuePairUtils.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/utils/TimeValuePairUtils.java
@@ -18,6 +18,7 @@
  */
 package org.apache.iotdb.db.utils;
 
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
 import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.read.common.BatchData;
 
@@ -51,4 +52,29 @@ public class TimeValuePairUtils {
         throw new UnSupportedDataTypeException(String.valueOf(data.getDataType()));
     }
   }
+
+  /**
+   * get given data's current (time,value) pair.
+   *
+   * @param data -AggreResultData
+   * @return -given data's (time,value) pair
+   */
+  public static TimeValuePair getCurrentTimeValuePair(AggreResultData data) {
+    switch (data.getDataType()) {
+      case INT32:
+        return new TimeValuePair(data.getTimestamp(), new TsPrimitiveType.TsInt(data.getIntRet()));
+      case INT64:
+        return new TimeValuePair(data.getTimestamp(), new TsPrimitiveType.TsLong(data.getLongRet()));
+      case FLOAT:
+        return new TimeValuePair(data.getTimestamp(), new TsPrimitiveType.TsFloat(data.getFloatRet()));
+      case DOUBLE:
+        return new TimeValuePair(data.getTimestamp(), new TsPrimitiveType.TsDouble(data.getDoubleRet()));
+      case TEXT:
+        return new TimeValuePair(data.getTimestamp(), new TsPrimitiveType.TsBinary(data.getBinaryRet()));
+      case BOOLEAN:
+        return new TimeValuePair(data.getTimestamp(), new TsPrimitiveType.TsBoolean(data.isBooleanRet()));
+      default:
+        throw new UnSupportedDataTypeException(String.valueOf(data.getDataType()));
+    }
+  }
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/utils/TsPrimitiveType.java b/iotdb/src/main/java/org/apache/iotdb/db/utils/TsPrimitiveType.java
index ea43c28..d77704e 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/utils/TsPrimitiveType.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/utils/TsPrimitiveType.java
@@ -26,7 +26,7 @@ import org.apache.iotdb.tsfile.utils.Binary;
 public abstract class TsPrimitiveType implements Serializable {
 
   /**
-   * get tsPrimitiveType by dataType.
+   * get tsPrimitiveType by resultDataType.
    *
    * @param dataType -given TsDataType
    * @param v -
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java
index 35bfc84..03d3e57 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java
@@ -20,6 +20,8 @@
 package org.apache.iotdb.db.engine.modification;
 
 import static junit.framework.TestCase.assertTrue;
+import static org.apache.iotdb.db.utils.EnvironmentUtils.TEST_QUERY_CONTEXT;
+import static org.apache.iotdb.db.utils.EnvironmentUtils.TEST_QUERY_JOB_ID;
 import static org.junit.Assert.assertEquals;
 
 import java.io.File;
@@ -35,8 +37,9 @@ import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
 import org.apache.iotdb.db.exception.FileNodeManagerException;
 import org.apache.iotdb.db.exception.MetadataArgsErrorException;
 import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.exception.StartupException;
 import org.apache.iotdb.db.metadata.MManager;
-import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.db.utils.TimeValuePair;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
@@ -67,7 +70,9 @@ public class DeletionFileNodeTest {
 
   @Before
   public void setup() throws MetadataArgsErrorException,
-      PathErrorException, IOException, FileNodeManagerException {
+      PathErrorException, IOException, FileNodeManagerException, StartupException {
+    EnvironmentUtils.envSetUp();
+
     MManager.getInstance().setStorageLevelToMTree(processorName);
     for (int i = 0; i < 10; i++) {
       MManager.getInstance().addPathToMTree(processorName + "." + measurements[i], dataType,
@@ -103,8 +108,10 @@ public class DeletionFileNodeTest {
 
     SingleSeriesExpression expression = new SingleSeriesExpression(new Path(processorName,
         measurements[5]), null);
-    QueryContext context = new QueryContext();
-    QueryDataSource dataSource = FileNodeManager.getInstance().query(expression, context);
+    QueryResourceManager.getInstance().beginQueryOfGivenExpression(TEST_QUERY_JOB_ID, expression);
+    QueryDataSource dataSource = QueryResourceManager.getInstance()
+        .getQueryDataSource(expression.getSeriesPath(), TEST_QUERY_CONTEXT);
+
     Iterator<TimeValuePair> timeValuePairs =
         dataSource.getSeqDataSource().getReadableChunk().getIterator();
     int count = 0;
@@ -113,6 +120,7 @@ public class DeletionFileNodeTest {
       count++;
     }
     assertEquals(50, count);
+    QueryResourceManager.getInstance().endQueryForGivenJob(TEST_QUERY_JOB_ID);
   }
 
   @Test
@@ -185,8 +193,11 @@ public class DeletionFileNodeTest {
 
     SingleSeriesExpression expression = new SingleSeriesExpression(new Path(processorName,
         measurements[5]), null);
-    QueryContext context = new QueryContext();
-    QueryDataSource dataSource = FileNodeManager.getInstance().query(expression, context);
+
+    QueryResourceManager.getInstance().beginQueryOfGivenExpression(TEST_QUERY_JOB_ID, expression);
+    QueryDataSource dataSource = QueryResourceManager.getInstance()
+        .getQueryDataSource(expression.getSeriesPath(), TEST_QUERY_CONTEXT);
+
     Iterator<TimeValuePair> timeValuePairs =
         dataSource.getOverflowSeriesDataSource().getReadableMemChunk().getIterator();
     int count = 0;
@@ -195,6 +206,8 @@ public class DeletionFileNodeTest {
       count++;
     }
     assertEquals(50, count);
+
+    QueryResourceManager.getInstance().endQueryForGivenJob(TEST_QUERY_JOB_ID);
   }
 
   @Test
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java
index c3e7dc3..3e9bb99 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java
@@ -19,6 +19,7 @@
 
 package org.apache.iotdb.db.engine.modification;
 
+import static org.apache.iotdb.db.utils.EnvironmentUtils.TEST_QUERY_CONTEXT;
 import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
@@ -30,6 +31,7 @@ import org.apache.iotdb.db.engine.memcontrol.BasicMemController.UsageLevel;
 import org.apache.iotdb.db.exception.FileNodeManagerException;
 import org.apache.iotdb.db.exception.MetadataArgsErrorException;
 import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.exception.StartupException;
 import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.query.executor.EngineQueryRouter;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
@@ -63,7 +65,9 @@ public class DeletionQueryTest {
 
   @Before
   public void setup() throws MetadataArgsErrorException,
-      PathErrorException, IOException, FileNodeManagerException {
+      PathErrorException, IOException, FileNodeManagerException, StartupException {
+    EnvironmentUtils.envSetUp();
+
     MManager.getInstance().setStorageLevelToMTree(processorName);
     for (int i = 0; i < 10; i++) {
       MManager.getInstance().addPathToMTree(processorName + "." + measurements[i], dataType,
@@ -103,7 +107,7 @@ public class DeletionQueryTest {
     pathList.add(new Path(processorName, measurements[5]));
 
     QueryExpression queryExpression = QueryExpression.create(pathList, null);
-    QueryDataSet dataSet = router.query(queryExpression);
+    QueryDataSet dataSet = router.query(queryExpression, TEST_QUERY_CONTEXT);
 
     int count = 0;
     while (dataSet.hasNext()) {
@@ -134,7 +138,7 @@ public class DeletionQueryTest {
     pathList.add(new Path(processorName, measurements[5]));
 
     QueryExpression queryExpression = QueryExpression.create(pathList, null);
-    QueryDataSet dataSet = router.query(queryExpression);
+    QueryDataSet dataSet = router.query(queryExpression, TEST_QUERY_CONTEXT);
 
     int count = 0;
     while (dataSet.hasNext()) {
@@ -176,7 +180,7 @@ public class DeletionQueryTest {
     pathList.add(new Path(processorName, measurements[5]));
 
     QueryExpression queryExpression = QueryExpression.create(pathList, null);
-    QueryDataSet dataSet = router.query(queryExpression);
+    QueryDataSet dataSet = router.query(queryExpression, TEST_QUERY_CONTEXT);
 
     int count = 0;
     while (dataSet.hasNext()) {
@@ -218,7 +222,7 @@ public class DeletionQueryTest {
     pathList.add(new Path(processorName, measurements[5]));
 
     QueryExpression queryExpression = QueryExpression.create(pathList, null);
-    QueryDataSet dataSet = router.query(queryExpression);
+    QueryDataSet dataSet = router.query(queryExpression, TEST_QUERY_CONTEXT);
 
     int count = 0;
     while (dataSet.hasNext()) {
@@ -282,7 +286,7 @@ public class DeletionQueryTest {
     pathList.add(new Path(processorName, measurements[5]));
 
     QueryExpression queryExpression = QueryExpression.create(pathList, null);
-    QueryDataSet dataSet = router.query(queryExpression);
+    QueryDataSet dataSet = router.query(queryExpression, TEST_QUERY_CONTEXT);
 
     int count = 0;
     while (dataSet.hasNext()) {
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IOTDBFillIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IOTDBFillIT.java
new file mode 100644
index 0000000..261cc17
--- /dev/null
+++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IOTDBFillIT.java
@@ -0,0 +1,351 @@
+/**
+ * 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.iotdb.db.integration;
+
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.jdbc.Config;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class IOTDBFillIT {
+
+  private static IoTDB daemon;
+
+  private static String[] dataSet1 = new String[]{
+      "SET STORAGE GROUP TO root.ln.wf01.wt01",
+      "CREATE TIMESERIES root.ln.wf01.wt01.status WITH DATATYPE=BOOLEAN, ENCODING=PLAIN",
+      "CREATE TIMESERIES root.ln.wf01.wt01.temperature WITH DATATYPE=DOUBLE, ENCODING=PLAIN",
+      "CREATE TIMESERIES root.ln.wf01.wt01.hardware WITH DATATYPE=INT32, ENCODING=PLAIN",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(1, 1.1, false, 11)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(2, 2.2, true, 22)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(3, 3.3, false, 33 )",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(4, 4.4, false, 44)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(5, 5.5, false, 55)",
+      "flush",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(100, 100.1, false, 110)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(150, 200.2, true, 220)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(200, 300.3, false, 330 )",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(250, 400.4, false, 440)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(300, 500.5, false, 550)",
+      "flush",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(10, 10.1, false, 110)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(20, 20.2, true, 220)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(30, 30.3, false, 330 )",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(40, 40.4, false, 440)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(50, 50.5, false, 550)",
+      "flush",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(500, 100.1, false, 110)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(510, 200.2, true, 220)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(520, 300.3, false, 330 )",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(530, 400.4, false, 440)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(540, 500.5, false, 550)",
+      "flush",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(580, 100.1, false, 110)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(590, 200.2, true, 220)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(600, 300.3, false, 330 )",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(610, 400.4, false, 440)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(620, 500.5, false, 550)",
+  };
+
+  private static final String TIMESTAMP_STR = "Time";
+  private static final String TEMPERATURE_STR = "root.ln.wf01.wt01.temperature";
+  private static final String STATUS_STR = "root.ln.wf01.wt01.status";
+  private static final String HARDWARE_STR = "root.ln.wf01.wt01.hardware";
+
+  @Before
+  public void setUp() throws Exception {
+    EnvironmentUtils.closeStatMonitor();
+    EnvironmentUtils.closeMemControl();
+    daemon = IoTDB.getInstance();
+    daemon.active();
+    EnvironmentUtils.envSetUp();
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    prepareData();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    daemon.stop();
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void LinearFillTest() throws SQLException {
+    String[] retArray1 = new String[]{
+        "3,3.3,false,33",
+        "70,70.34,false,374",
+        "70,null,null,null",
+        "625,null,false,null"
+    };
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute("select temperature,status, hardware from "
+          + "root.ln.wf01.wt01 where time = 3 "
+          + "Fill(int32[linear, 5ms, 5ms], double[linear, 5ms, 5ms], boolean[previous, 5ms])");
+
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(TEMPERATURE_STR)
+            + "," + resultSet.getString(STATUS_STR) + "," + resultSet.getString(HARDWARE_STR);
+        System.out.println(ans);
+        Assert.assertEquals(retArray1[cnt], ans);
+        cnt++;
+      }
+      statement.close();
+
+      statement = connection.createStatement();
+      hasResultSet = statement.execute("select temperature,status, hardware "
+          + "from root.ln.wf01.wt01 where time = 70 Fill(int32[linear, 500ms, 500ms], "
+          + "double[linear, 500ms, 500ms], boolean[previous, 500ms])");
+
+      Assert.assertTrue(hasResultSet);
+      resultSet = statement.getResultSet();
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(TEMPERATURE_STR)
+            + "," + resultSet.getString(STATUS_STR) + "," + resultSet.getString(HARDWARE_STR);
+        Assert.assertEquals(retArray1[cnt], ans);
+        cnt++;
+        System.out.println(ans);
+      }
+      statement.close();
+
+      statement = connection.createStatement();
+      hasResultSet = statement.execute("select temperature,status, hardware "
+          + "from root.ln.wf01.wt01 where time = 70 "
+          + "Fill(int32[linear, 25ms, 25ms], double[linear, 25ms, 25ms], boolean[previous, 5ms])");
+
+      Assert.assertTrue(hasResultSet);
+      resultSet = statement.getResultSet();
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(TEMPERATURE_STR)
+            + "," + resultSet.getString(STATUS_STR) + "," + resultSet.getString(HARDWARE_STR);
+        Assert.assertEquals(retArray1[cnt], ans);
+        cnt++;
+        System.out.println(ans);
+      }
+      statement.close();
+
+      statement = connection.createStatement();
+      hasResultSet = statement.execute("select temperature,status, hardware "
+          + "from root.ln.wf01.wt01 where time = 625 "
+          + "Fill(int32[linear, 25ms, 25ms], double[linear, 25ms, 25ms], boolean[previous, 5ms])");
+
+      Assert.assertTrue(hasResultSet);
+      resultSet = statement.getResultSet();
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(TEMPERATURE_STR)
+            + "," + resultSet.getString(STATUS_STR) + "," + resultSet.getString(HARDWARE_STR);
+        System.out.println(cnt + " " + ans);
+        Assert.assertEquals(retArray1[cnt], ans);
+        cnt++;
+      }
+      statement.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  @Test
+  public void PreviousFillTest() throws SQLException {
+    String[] retArray1 = new String[]{
+        "3,3.3,false,33",
+        "70,50.5,false,550",
+        "70,null,null,null"
+    };
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute("select temperature,status, hardware "
+          + "from root.ln.wf01.wt01 where time = 3 "
+          + "Fill(int32[previous, 5ms], double[previous, 5ms], boolean[previous, 5ms])");
+
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(TEMPERATURE_STR)
+            + "," + resultSet.getString(STATUS_STR) + "," + resultSet.getString(HARDWARE_STR);
+        Assert.assertEquals(retArray1[cnt], ans);
+        cnt++;
+      }
+      statement.close();
+
+      statement = connection.createStatement();
+      hasResultSet = statement.execute("select temperature,status, hardware "
+          + "from root.ln.wf01.wt01 where time = 70 "
+          + "Fill(int32[previous, 500ms], double[previous, 500ms], boolean[previous, 500ms])");
+
+      Assert.assertTrue(hasResultSet);
+      resultSet = statement.getResultSet();
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(TEMPERATURE_STR)
+            + "," + resultSet.getString(STATUS_STR) + "," + resultSet.getString(HARDWARE_STR);
+        Assert.assertEquals(retArray1[cnt], ans);
+        cnt++;
+        System.out.println(ans);
+      }
+      statement.close();
+
+      statement = connection.createStatement();
+      hasResultSet = statement.execute("select temperature,status, hardware "
+          + "from root.ln.wf01.wt01 where time = 70 "
+          + "Fill(int32[previous, 15ms], double[previous, 15ms], boolean[previous, 5ms])");
+
+      Assert.assertTrue(hasResultSet);
+      resultSet = statement.getResultSet();
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(TEMPERATURE_STR)
+            + "," + resultSet.getString(STATUS_STR) + "," + resultSet.getString(HARDWARE_STR);
+        Assert.assertEquals(retArray1[cnt], ans);
+        cnt++;
+        System.out.println(ans);
+      }
+      statement.close();
+      Assert.assertEquals(retArray1.length, cnt);
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  @Test
+  public void EmptyTimeRangeFillTest() throws SQLException {
+    String[] retArray1 = new String[]{
+        "3,3.3,false,33",
+        "70,70.34,false,374"
+    };
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute("select temperature,status, hardware "
+          + "from root.ln.wf01.wt01 where time = 3 "
+          + "Fill(int32[linear], double[linear], boolean[previous])");
+
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(TEMPERATURE_STR)
+            + "," + resultSet.getString(STATUS_STR) + "," + resultSet.getString(HARDWARE_STR);
+        Assert.assertEquals(retArray1[cnt], ans);
+        cnt++;
+      }
+      statement.close();
+
+      statement = connection.createStatement();
+      hasResultSet = statement.execute("select temperature,status, hardware "
+          + "from root.ln.wf01.wt01 where time = 70 "
+          + "Fill(int32[linear], double[linear], boolean[previous])");
+
+      Assert.assertTrue(hasResultSet);
+      resultSet = statement.getResultSet();
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(TEMPERATURE_STR)
+            + "," + resultSet.getString(STATUS_STR) + "," + resultSet.getString(HARDWARE_STR);
+        Assert.assertEquals(retArray1[cnt], ans);
+        cnt++;
+        System.out.println(ans);
+      }
+      statement.close();
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  private void prepareData() throws SQLException {
+    Connection connection = null;
+    try {
+      connection = DriverManager
+          .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root",
+              "root");
+      Statement statement = connection.createStatement();
+
+      for (String sql : dataSet1) {
+        statement.execute(sql);
+      }
+      statement.close();
+
+    } catch (Exception e) {
+      e.printStackTrace();
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+}
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IOTDBGroupByIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IOTDBGroupByIT.java
new file mode 100644
index 0000000..7d68679
--- /dev/null
+++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IOTDBGroupByIT.java
@@ -0,0 +1,532 @@
+/**
+ * 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.iotdb.db.integration;
+
+import static org.apache.iotdb.db.integration.Constant.count;
+import static org.apache.iotdb.db.integration.Constant.first;
+import static org.apache.iotdb.db.integration.Constant.last;
+import static org.apache.iotdb.db.integration.Constant.max_time;
+import static org.apache.iotdb.db.integration.Constant.max_value;
+import static org.apache.iotdb.db.integration.Constant.mean;
+import static org.apache.iotdb.db.integration.Constant.min_time;
+import static org.apache.iotdb.db.integration.Constant.min_value;
+import static org.apache.iotdb.db.integration.Constant.sum;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.jdbc.Config;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class IOTDBGroupByIT {
+
+  private static IoTDB daemon;
+
+  private static String[] dataSet1 = new String[]{
+      "SET STORAGE GROUP TO root.ln.wf01.wt01",
+      "CREATE TIMESERIES root.ln.wf01.wt01.status WITH DATATYPE=BOOLEAN, ENCODING=PLAIN",
+      "CREATE TIMESERIES root.ln.wf01.wt01.temperature WITH DATATYPE=DOUBLE, ENCODING=PLAIN",
+      "CREATE TIMESERIES root.ln.wf01.wt01.hardware WITH DATATYPE=INT32, ENCODING=PLAIN",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(1, 1.1, false, 11)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(2, 2.2, true, 22)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(3, 3.3, false, 33 )",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(4, 4.4, false, 44)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(5, 5.5, false, 55)",
+      "flush",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(100, 100.1, false, 110)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(150, 200.2, true, 220)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(200, 300.3, false, 330 )",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(250, 400.4, false, 440)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(300, 500.5, false, 550)",
+      "flush",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(10, 10.1, false, 110)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(20, 20.2, true, 220)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(30, 30.3, false, 330 )",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(40, 40.4, false, 440)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(50, 50.5, false, 550)",
+      "flush",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(500, 100.1, false, 110)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(510, 200.2, true, 220)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(520, 300.3, false, 330 )",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(530, 400.4, false, 440)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(540, 500.5, false, 550)",
+      "flush",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(580, 100.1, false, 110)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(590, 200.2, true, 220)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(600, 300.3, false, 330 )",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(610, 400.4, false, 440)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(620, 500.5, false, 550)",
+  };
+
+  private static final String TIMESTAMP_STR = "Time";
+
+  @Before
+  public void setUp() throws Exception {
+    EnvironmentUtils.closeStatMonitor();
+    EnvironmentUtils.closeMemControl();
+    daemon = IoTDB.getInstance();
+    daemon.active();
+    EnvironmentUtils.envSetUp();
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    prepareData();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    daemon.stop();
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void countSumMeanTest() throws SQLException {
+    String[] retArray1 = new String[]{
+        "2,1,4.4,4.4",
+        "5,3,35.8,11.933333333333332",
+        "25,1,30.3,30.3",
+        "50,1,50.5,50.5",
+        "65,0,0.0,null",
+        "85,1,100.1,100.1",
+        "105,0,0.0,null",
+        "125,0,0.0,null",
+        "145,1,200.2,200.2",
+        "310,0,0.0,null"
+    };
+    String[] retArray2 = new String[]{
+        "2,2,7.7,3.85",
+        "5,3,35.8,11.933333333333332",
+        "25,1,30.3,30.3",
+        "50,1,50.5,50.5",
+        "65,0,0.0,null",
+        "85,1,100.1,100.1",
+        "105,0,0.0,null",
+        "125,0,0.0,null",
+        "145,1,200.2,200.2",
+        "310,0,0.0,null"
+    };
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute(
+          "select count(temperature), sum(temperature), mean(temperature) from "
+              + "root.ln.wf01.wt01 where time > 3 "
+              + "GROUP BY (20ms, 5,[2,30], [35,37], [50, 160], [310, 314])");
+
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+            .getString(count("root.ln.wf01.wt01.temperature")) + "," +
+            resultSet.getString(sum("root.ln.wf01.wt01.temperature")) + "," + resultSet
+            .getString(mean("root.ln.wf01.wt01.temperature"));
+        Assert.assertEquals(retArray1[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(retArray1.length, cnt);
+      statement.close();
+
+      statement = connection.createStatement();
+      hasResultSet = statement.execute(
+          "select count(temperature), sum(temperature), mean(temperature) from "
+              + "root.ln.wf01.wt01 where temperature > 3 "
+              + "GROUP BY (20ms, 5,[2,30], [35,37], [50, 160], [310, 314])");
+
+      Assert.assertTrue(hasResultSet);
+      resultSet = statement.getResultSet();
+      cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+            .getString(count("root.ln.wf01.wt01.temperature")) + "," +
+            resultSet.getString(sum("root.ln.wf01.wt01.temperature")) + "," + resultSet
+            .getString(mean("root.ln.wf01.wt01.temperature"));
+        Assert.assertEquals(retArray2[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(retArray2.length, cnt);
+      statement.close();
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  @Test
+  public void maxMinValeTimeTest() throws SQLException {
+    String[] retArray1 = new String[]{
+        "2,4.4,4.4,4,4",
+        "5,20.2,5.5,20,5",
+        "25,30.3,30.3,30,30",
+        "50,50.5,50.5,50,50",
+        "65,null,null,null,null",
+        "85,100.1,100.1,100,100",
+        "105,null,null,null,null",
+        "125,null,null,null,null",
+        "145,200.2,200.2,150,150",
+        "310,null,null,null,null"
+    };
+    String[] retArray2 = new String[]{
+        "2,4.4,3.3,4,3",
+        "5,20.2,5.5,20,5",
+        "25,30.3,30.3,30,30",
+        "50,50.5,50.5,50,50",
+        "65,null,null,null,null",
+        "85,100.1,100.1,100,100",
+        "105,null,null,null,null",
+        "125,null,null,null,null",
+        "145,200.2,200.2,150,150",
+        "310,null,null,null,null"
+    };
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute(
+          "select max_value(temperature), min_value(temperature), max_time(temperature), "
+              + "min_time(temperature) from root.ln.wf01.wt01 where time > 3 "
+              + "GROUP BY (20ms, 5,[2,30], [35,37], [50, 160], [310, 314])");
+
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+            .getString(max_value("root.ln.wf01.wt01.temperature"))
+            + "," + resultSet.getString(min_value("root.ln.wf01.wt01.temperature")) + ","
+            + resultSet.getString(max_time("root.ln.wf01.wt01.temperature"))
+            + "," + resultSet.getString(min_time("root.ln.wf01.wt01.temperature"));
+        Assert.assertEquals(retArray1[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(retArray1.length, cnt);
+      statement.close();
+
+      statement = connection.createStatement();
+      hasResultSet = statement.execute(
+          "select max_value(temperature), min_value(temperature), max_time(temperature), "
+              + "min_time(temperature) from root.ln.wf01.wt01 where temperature > 3 "
+              + "GROUP BY (20ms, 5,[2,30], [35,37], [50, 160], [310, 314])");
+
+      Assert.assertTrue(hasResultSet);
+      resultSet = statement.getResultSet();
+      cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+            .getString(max_value("root.ln.wf01.wt01.temperature"))
+            + "," + resultSet.getString(min_value("root.ln.wf01.wt01.temperature")) + ","
+            + resultSet.getString(max_time("root.ln.wf01.wt01.temperature"))
+            + "," + resultSet.getString(min_time("root.ln.wf01.wt01.temperature"));
+        Assert.assertEquals(retArray2[cnt], ans);
+        cnt++;
+        //System.out.println(ans);
+      }
+      Assert.assertEquals(retArray2.length, cnt);
+      statement.close();
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  @Test
+  public void firstLastTest() throws SQLException {
+    String[] retArray1 = new String[]{
+        "2,4.4,4.4",
+        "5,20.2,5.5",
+        "25,30.3,30.3",
+        "50,50.5,50.5",
+        "65,null,null",
+        "85,100.1,100.1",
+        "105,null,null",
+        "125,null,null",
+        "145,200.2,200.2",
+        "310,null,null"
+    };
+    String[] retArray2 = new String[]{
+        "2,4.4,3.3",
+        "5,20.2,5.5",
+        "25,30.3,30.3",
+        "50,50.5,50.5",
+        "65,null,null",
+        "85,100.1,100.1",
+        "105,null,null",
+        "125,null,null",
+        "145,200.2,200.2",
+        "310,null,null"
+    };
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute(
+          "select last(temperature), first(temperature) from root.ln.wf01.wt01 where time > 3 "
+              + "GROUP BY (20ms, 5,[2,30], [35,37], [50, 160], [310, 314])");
+
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+            .getString(last("root.ln.wf01.wt01.temperature"))
+            + "," + resultSet.getString(first("root.ln.wf01.wt01.temperature"));
+        System.out.println(ans);
+        Assert.assertEquals(retArray1[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(retArray1.length, cnt);
+      statement.close();
+
+      statement = connection.createStatement();
+      hasResultSet = statement.execute(
+          "select first(temperature), last(temperature) from root.ln.wf01.wt01 "
+              + "where temperature > 3 "
+              + "GROUP BY (20ms, 5,[2,30], [35,37], [50, 160], [310, 314])");
+
+      Assert.assertTrue(hasResultSet);
+      resultSet = statement.getResultSet();
+      cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+            .getString(last("root.ln.wf01.wt01.temperature"))
+            + "," + resultSet.getString(first("root.ln.wf01.wt01.temperature"));
+        System.out.println(ans);
+        Assert.assertEquals(retArray2[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(retArray2.length, cnt);
+      statement.close();
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  @Test
+  public void largeIntervalTest() throws SQLException {
+    String[] retArray1 = new String[]{
+        "2,4.4,4,20,4",
+        "30,30.3,16,610,30",
+        "620,500.5,1,620,620"
+    };
+    String[] retArray2 = new String[]{
+        "2,3.3,5,20,3",
+        "30,30.3,16,610,30",
+        "620,500.5,1,620,620"
+    };
+
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute(
+          "select min_value(temperature), count(temperature), max_time(temperature), "
+              + "min_time(temperature) from root.ln.wf01.wt01 where time > 3 GROUP BY "
+              + "(590ms, 30, [2, 30], [30, 120], [100, 120], [123, 125], [155, 550], [540, 680])");
+
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+            .getString(min_value("root.ln.wf01.wt01.temperature"))
+            + "," + resultSet.getString(count("root.ln.wf01.wt01.temperature")) + "," +
+            resultSet.getString(max_time("root.ln.wf01.wt01.temperature"))
+            + "," + resultSet.getString(min_time("root.ln.wf01.wt01.temperature"));
+        Assert.assertEquals(retArray1[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(retArray1.length, cnt);
+      statement.close();
+
+      statement = connection.createStatement();
+      hasResultSet = statement.execute(
+          "select min_value(temperature), count (temperature), max_time(temperature), "
+              + "min_time(temperature) from root.ln.wf01.wt01 where temperature > 3 GROUP BY "
+              + "(590ms, 30, [2, 30], [30, 120], [100, 120], [123, 125], [155, 550],[540, 680])");
+
+      Assert.assertTrue(hasResultSet);
+      resultSet = statement.getResultSet();
+      cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+            .getString(min_value("root.ln.wf01.wt01.temperature"))
+            + "," + resultSet.getString(count("root.ln.wf01.wt01.temperature")) + ","
+            + resultSet.getString(max_time("root.ln.wf01.wt01.temperature"))
+            + "," + resultSet.getString(min_time("root.ln.wf01.wt01.temperature"));
+        Assert.assertEquals(retArray2[cnt], ans);
+        cnt++;
+        //System.out.println(ans);
+      }
+      Assert.assertEquals(retArray2.length, cnt);
+      statement.close();
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  @Test
+  public void smallPartitionTest() throws SQLException {
+    String[] retArray1 = new String[]{
+        "50,100.1,50.5,150.6",
+        "615,500.5,500.5,500.5"
+
+    };
+    String[] retArray2 = new String[]{
+        "50,100.1,50.5,150.6",
+        "585,null,null,0.0",
+        "590,500.5,200.2,700.7"
+    };
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute(
+          "select last(temperature), first(temperature), sum(temperature) from "
+              + "root.ln.wf01.wt01 where time > 3 "
+              + "GROUP BY (80ms, 30,[50,100], [615, 650])");
+
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+            .getString(last("root.ln.wf01.wt01.temperature"))
+            + "," + resultSet.getString(first("root.ln.wf01.wt01.temperature")) + ","
+            + resultSet.getString(sum("root.ln.wf01.wt01.temperature"));
+        System.out.println(ans);
+        Assert.assertEquals(retArray1[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(retArray1.length, cnt);
+      statement.close();
+
+      statement = connection.createStatement();
+      hasResultSet = statement.execute(
+          "select first(temperature), last(temperature), sum(temperature) from "
+              + "root.ln.wf01.wt01 where temperature > 3 "
+              + "GROUP BY (80ms, 30,[50,100], [585,590], [615, 650])");
+
+      Assert.assertTrue(hasResultSet);
+      resultSet = statement.getResultSet();
+      cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+            .getString(last("root.ln.wf01.wt01.temperature"))
+            + "," + resultSet.getString(first("root.ln.wf01.wt01.temperature")) + ","
+            + resultSet.getString(sum("root.ln.wf01.wt01.temperature"));
+        System.out.println(ans);
+        Assert.assertEquals(retArray2[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(retArray2.length, cnt);
+      statement.close();
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  public void prepareData() throws SQLException {
+    Connection connection = null;
+    try {
+      connection = DriverManager
+          .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root",
+              "root");
+      Statement statement = connection.createStatement();
+
+      for (String sql : dataSet1) {
+        statement.execute(sql);
+      }
+      statement.close();
+
+    } catch (Exception e) {
+      e.printStackTrace();
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+}
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAggregationIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAggregationIT.java
new file mode 100644
index 0000000..47ff18c
--- /dev/null
+++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAggregationIT.java
@@ -0,0 +1,550 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.integration;
+
+import static org.apache.iotdb.db.integration.Constant.count;
+import static org.apache.iotdb.db.integration.Constant.first;
+import static org.apache.iotdb.db.integration.Constant.last;
+import static org.apache.iotdb.db.integration.Constant.max_time;
+import static org.apache.iotdb.db.integration.Constant.max_value;
+import static org.apache.iotdb.db.integration.Constant.mean;
+import static org.apache.iotdb.db.integration.Constant.min_time;
+import static org.apache.iotdb.db.integration.Constant.min_value;
+import static org.apache.iotdb.db.integration.Constant.sum;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.jdbc.Config;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class IoTDBAggregationIT {
+
+  private static IoTDB daemon;
+
+  private static String[] creationSqls = new String[]{
+      "SET STORAGE GROUP TO root.vehicle.d0",
+      "SET STORAGE GROUP TO root.vehicle.d1",
+
+      "CREATE TIMESERIES root.vehicle.d0.s0 WITH DATATYPE=INT32, ENCODING=RLE",
+      "CREATE TIMESERIES root.vehicle.d0.s1 WITH DATATYPE=INT64, ENCODING=RLE",
+      "CREATE TIMESERIES root.vehicle.d0.s2 WITH DATATYPE=FLOAT, ENCODING=RLE",
+      "CREATE TIMESERIES root.vehicle.d0.s3 WITH DATATYPE=TEXT, ENCODING=PLAIN"
+  };
+
+  private static String[] dataSet2 = new String[]{
+      "SET STORAGE GROUP TO root.ln.wf01.wt01",
+      "CREATE TIMESERIES root.ln.wf01.wt01.status WITH DATATYPE=BOOLEAN, ENCODING=PLAIN",
+      "CREATE TIMESERIES root.ln.wf01.wt01.temperature WITH DATATYPE=FLOAT, ENCODING=PLAIN",
+      "CREATE TIMESERIES root.ln.wf01.wt01.hardware WITH DATATYPE=INT32, ENCODING=PLAIN",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(1, 1.1, false, 11)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(2, 2.2, true, 22)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(3, 3.3, false, 33 )",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(4, 4.4, false, 44)",
+      "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+          + "values(5, 5.5, false, 55)"
+  };
+
+  private String insertTemplate = "INSERT INTO root.vehicle.d0(timestamp,s0,s1,s2,s3)"
+      + " VALUES(%d,%d,%d,%f,%s)";
+
+  private static final String TIMESTAMP_STR = "Time";
+  private final String d0s0 = "root.vehicle.d0.s0";
+  private final String d0s1 = "root.vehicle.d0.s1";
+  private final String d0s2 = "root.vehicle.d0.s2";
+  private final String d0s3 = "root.vehicle.d0.s3";
+  private static final String TEMPERATURE_STR = "root.ln.wf01.wt01.temperature";
+
+  @Before
+  public void setUp() throws Exception {
+    EnvironmentUtils.closeStatMonitor();
+    EnvironmentUtils.closeMemControl();
+    daemon = IoTDB.getInstance();
+    daemon.active();
+    EnvironmentUtils.envSetUp();
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    prepareData();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    daemon.stop();
+    EnvironmentUtils.cleanEnv();
+  }
+
+  //add test for part of points in page don't satisfy filter
+  //details in: https://issues.apache.org/jira/projects/IOTDB/issues/IOTDB-54
+  @Test
+  public void test() throws SQLException {
+    String[] retArray = new String[]{
+        "0,2",
+        "0,4",
+        "0,3"
+    };
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute(
+          "select count(temperature) from root.ln.wf01.wt01 where time > 3");
+
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," +
+            resultSet.getString(count(TEMPERATURE_STR));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+
+      statement = connection.createStatement();
+      hasResultSet = statement.execute(
+          "select min_time(temperature) from root.ln.wf01.wt01 where time > 3");
+
+      Assert.assertTrue(hasResultSet);
+      resultSet = statement.getResultSet();
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," +
+            resultSet.getString(min_time(TEMPERATURE_STR));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(2, cnt);
+      statement.close();
+
+      statement = connection.createStatement();
+      hasResultSet = statement.execute(
+          "select min_time(temperature) from root.ln.wf01.wt01 where temperature > 3");
+
+      Assert.assertTrue(hasResultSet);
+      resultSet = statement.getResultSet();
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," +
+            resultSet.getString(min_time(TEMPERATURE_STR));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(3, cnt);
+      statement.close();
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  @Test
+  public void countTest() throws SQLException {
+    String[] retArray = new String[]{
+        "0,2001,2001,2001,2001",
+        "0,7500,7500,7500,7500"
+    };
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute("select count(s0),count(s1),count(s2),count(s3) " +
+          "from root.vehicle.d0 where time >= 6000 and time <= 9000");
+
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(count(d0s0))
+            + "," + resultSet.getString(count(d0s1)) + "," + resultSet.getString(count(d0s2))
+            + "," + resultSet.getString(count(d0s3));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+
+      statement = connection.createStatement();
+      hasResultSet = statement.execute("select count(s0),count(s1),count(s2),count(s3) " +
+          "from root.vehicle.d0");
+
+      Assert.assertTrue(hasResultSet);
+      resultSet = statement.getResultSet();
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(count(d0s0))
+            + "," + resultSet.getString(count(d0s1)) + "," + resultSet.getString(count(d0s2))
+            + "," + resultSet.getString(count(d0s3));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(2, cnt);
+      statement.close();
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  @Test
+  public void firstTest() throws SQLException {
+    String[] retArray = new String[]{
+        "0,2000,2000,2000.0,2000",
+        "0,500,500,500.0,500"
+    };
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute("select first(s0),first(s1),first(s2),first(s3) " +
+          "from root.vehicle.d0 where time >= 1500 and time <= 9000");
+
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(first(d0s0))
+            + "," + resultSet.getString(first(d0s1)) + "," + resultSet.getString(first(d0s2))
+            + "," + resultSet.getString(first(d0s3));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+
+      statement = connection.createStatement();
+      hasResultSet = statement.execute("select first(s0),first(s1),first(s2),first(s3) " +
+          "from root.vehicle.d0");
+
+      Assert.assertTrue(hasResultSet);
+      resultSet = statement.getResultSet();
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(first(d0s0))
+            + "," + resultSet.getString(first(d0s1)) + "," + resultSet.getString(first(d0s2))
+            + "," + resultSet.getString(first(d0s3));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(2, cnt);
+      statement.close();
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  @Test
+  public void lastTest() throws SQLException {
+    String[] retArray = new String[]{
+        "0,8499,8499.0",
+        "0,1499,1499.0",
+        "0,2200,2200.0"
+    };
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute("select last(s0),last(s2) " +
+          "from root.vehicle.d0 where time >= 1500 and time < 9000");
+
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(last(d0s0))
+            + "," + resultSet.getString(last(d0s2));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+
+      statement = connection.createStatement();
+      hasResultSet = statement.execute("select last(s0),last(s2) " +
+          "from root.vehicle.d0 where time <= 1600");
+
+      Assert.assertTrue(hasResultSet);
+      resultSet = statement.getResultSet();
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(last(d0s0))
+            + "," + resultSet.getString(last(d0s2));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(2, cnt);
+      statement.close();
+
+      statement = connection.createStatement();
+      hasResultSet = statement.execute("select last(s0),last(s2) " +
+          "from root.vehicle.d0 where time <= 2200");
+
+      Assert.assertTrue(hasResultSet);
+      resultSet = statement.getResultSet();
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(last(d0s0))
+            + "," + resultSet.getString(last(d0s2));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(3, cnt);
+      statement.close();
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  @Test
+  public void maxminTimeTest() throws SQLException {
+    String[] retArray = new String[]{
+        "0,8499,500",
+        "0,2499,2000"
+    };
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute("select max_time(s0),min_time(s2) " +
+          "from root.vehicle.d0 where time >= 100 and time < 9000");
+
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(max_time(d0s0))
+            + "," + resultSet.getString(min_time(d0s2));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+
+      statement = connection.createStatement();
+      hasResultSet = statement.execute("select max_time(s0),min_time(s2) " +
+          "from root.vehicle.d0 where time <= 2500 and time > 1800");
+
+      Assert.assertTrue(hasResultSet);
+      resultSet = statement.getResultSet();
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(max_time(d0s0))
+            + "," + resultSet.getString(min_time(d0s2));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(2, cnt);
+      statement.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  @Test
+  public void maxminValueTest() throws SQLException {
+    String[] retArray = new String[]{
+        "0,8499,500.0",
+        "0,2499,500.0"
+    };
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute("select max_value(s0),min_value(s2) " +
+          "from root.vehicle.d0 where time >= 100 and time < 9000");
+
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(max_value(d0s0))
+            + "," + resultSet.getString(min_value(d0s2));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+
+      statement = connection.createStatement();
+      hasResultSet = statement.execute("select max_value(s0),min_value(s2) " +
+          "from root.vehicle.d0 where time < 2500");
+
+      Assert.assertTrue(hasResultSet);
+      resultSet = statement.getResultSet();
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(max_value(d0s0))
+            + "," + resultSet.getString(min_value(d0s2));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(2, cnt);
+      statement.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  @Test
+  public void meanSumTest() throws SQLException {
+    String[] retArray = new String[]{
+        "0,1.4508E7,7250.374812593703",
+        "0,626750.0,1250.998003992016"
+    };
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute("select sum(s0),mean(s2)" +
+          "from root.vehicle.d0 where time >= 6000 and time <= 9000");
+
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(sum(d0s0))
+            + "," + resultSet.getString(mean(d0s2));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+
+      statement = connection.createStatement();
+      hasResultSet = statement.execute("select sum(s0),mean(s2)" +
+          "from root.vehicle.d0 where time >= 1000 and time <= 2000");
+
+      Assert.assertTrue(hasResultSet);
+      resultSet = statement.getResultSet();
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(sum(d0s0))
+            + "," + resultSet.getString(mean(d0s2));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(2, cnt);
+      statement.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  private void prepareData() throws SQLException {
+    Connection connection = null;
+    try {
+      connection = DriverManager
+          .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root",
+              "root");
+      Statement statement = connection.createStatement();
+      for (String sql : creationSqls) {
+        statement.execute(sql);
+      }
+
+      for (String sql : dataSet2) {
+        statement.execute(sql);
+      }
+      statement.close();
+
+      statement = connection.createStatement();
+      // prepare BufferWrite file
+      for (int i = 5000; i < 7000; i++) {
+        statement.execute(String.format(insertTemplate, i, i, i, (double) i, "\'" + i + "\'"));
+      }
+      statement.execute("flush");
+      for (int i = 7500; i < 8500; i++) {
+        statement.execute(String.format(insertTemplate, i, i, i, (double) i, "\'" + i + "\'"));
+      }
+      statement.execute("flush");
+      // prepare Unseq-File
+      for (int i = 500; i < 1500; i++) {
+        statement.execute(String.format(insertTemplate, i, i, i, (double) i, "\'" + i + "\'"));
+      }
+      statement.execute("flush");
+      for (int i = 3000; i < 6500; i++) {
+        statement.execute(String.format(insertTemplate, i, i, i, (double) i, "\'" + i + "\'"));
+      }
+      statement.execute("merge");
+
+      // prepare BufferWrite cache
+      for (int i = 9000; i < 10000; i++) {
+        statement.execute(String.format(insertTemplate, i, i, i, (double) i, "\'" + i + "\'"));
+      }
+      // prepare Overflow cache
+      for (int i = 2000; i < 2500; i++) {
+        statement.execute(String.format(insertTemplate, i, i, i, (double) i, "\'" + i + "\'"));
+      }
+      statement.close();
+
+    } catch (Exception e) {
+      e.printStackTrace();
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+}
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAggregationLargeDataIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAggregationLargeDataIT.java
new file mode 100644
index 0000000..d0f782d
--- /dev/null
+++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAggregationLargeDataIT.java
@@ -0,0 +1,900 @@
+/**
+ * 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.iotdb.db.integration;
+
+import static org.apache.iotdb.db.integration.Constant.count;
+import static org.apache.iotdb.db.integration.Constant.first;
+import static org.apache.iotdb.db.integration.Constant.max_time;
+import static org.apache.iotdb.db.integration.Constant.max_value;
+import static org.apache.iotdb.db.integration.Constant.mean;
+import static org.apache.iotdb.db.integration.Constant.min_value;
+import static org.apache.iotdb.db.integration.Constant.sum;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.jdbc.Config;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class IoTDBAggregationLargeDataIT {
+
+  private static final String TIMESTAMP_STR = "Time";
+  private final String d0s0 = "root.vehicle.d0.s0";
+  private final String d0s1 = "root.vehicle.d0.s1";
+  private final String d0s2 = "root.vehicle.d0.s2";
+  private final String d0s3 = "root.vehicle.d0.s3";
+  private final String d0s4 = "root.vehicle.d0.s4";
+
+  private static String[] createSql = new String[]{
+      "SET STORAGE GROUP TO root.vehicle",
+      "CREATE TIMESERIES root.vehicle.d1.s0 WITH DATATYPE=INT32, ENCODING=RLE",
+      "CREATE TIMESERIES root.vehicle.d0.s2 WITH DATATYPE=FLOAT, ENCODING=RLE",
+      "CREATE TIMESERIES root.vehicle.d0.s3 WITH DATATYPE=TEXT, ENCODING=PLAIN",
+      "CREATE TIMESERIES root.vehicle.d0.s0 WITH DATATYPE=INT32, ENCODING=RLE",
+      "CREATE TIMESERIES root.vehicle.d0.s1 WITH DATATYPE=INT64, ENCODING=RLE",
+      "CREATE TIMESERIES root.vehicle.d0.s4 WITH DATATYPE=BOOLEAN, ENCODING=PLAIN",
+  };
+
+  private static String[] insertSql = new String[]{
+      "insert into root.vehicle.d0(timestamp,s0) values(1,101)",
+      "insert into root.vehicle.d0(timestamp,s0) values(2,198)",
+      "insert into root.vehicle.d0(timestamp,s0) values(100,99)",
+      "insert into root.vehicle.d0(timestamp,s0) values(101,99)",
+      "insert into root.vehicle.d0(timestamp,s0) values(102,80)",
+      "insert into root.vehicle.d0(timestamp,s0) values(103,99)",
+      "insert into root.vehicle.d0(timestamp,s0) values(104,90)",
+      "insert into root.vehicle.d0(timestamp,s0) values(105,99)",
+      "insert into root.vehicle.d0(timestamp,s0) values(106,99)",
+      "insert into root.vehicle.d0(timestamp,s0) values(2,10000)",
+      "insert into root.vehicle.d0(timestamp,s0) values(50,10000)",
+      "insert into root.vehicle.d0(timestamp,s0) values(1000,22222)",
+      "insert into root.vehicle.d0(timestamp,s0) values(106,199)",
+      "DELETE FROM root.vehicle.d0.s0 WHERE time < 104",
+
+      "insert into root.vehicle.d0(timestamp,s1) values(1,1101)",
+      "insert into root.vehicle.d0(timestamp,s1) values(2,198)",
+      "insert into root.vehicle.d0(timestamp,s1) values(100,199)",
+      "insert into root.vehicle.d0(timestamp,s1) values(101,199)",
+      "insert into root.vehicle.d0(timestamp,s1) values(102,180)",
+      "insert into root.vehicle.d0(timestamp,s1) values(103,199)",
+      "insert into root.vehicle.d0(timestamp,s1) values(104,190)",
+      "insert into root.vehicle.d0(timestamp,s1) values(105,199)",
+      "insert into root.vehicle.d0(timestamp,s1) values(2,40000)",
+      "insert into root.vehicle.d0(timestamp,s1) values(50,50000)",
+      "insert into root.vehicle.d0(timestamp,s1) values(1000,55555)",
+
+      "insert into root.vehicle.d0(timestamp,s2) values(1000,55555)",
+      "insert into root.vehicle.d0(timestamp,s2) values(2,2.22)",
+      "insert into root.vehicle.d0(timestamp,s2) values(3,3.33)",
+      "insert into root.vehicle.d0(timestamp,s2) values(4,4.44)",
+      "insert into root.vehicle.d0(timestamp,s2) values(102,10.00)",
+      "insert into root.vehicle.d0(timestamp,s2) values(105,11.11)",
+      "insert into root.vehicle.d0(timestamp,s2) values(1000,1000.11)",
+
+      "insert into root.vehicle.d0(timestamp,s3) values(60,'aaaaa')",
+      "insert into root.vehicle.d0(timestamp,s3) values(70,'bbbbb')",
+      "insert into root.vehicle.d0(timestamp,s3) values(80,'ccccc')",
+      "insert into root.vehicle.d0(timestamp,s3) values(101,'ddddd')",
+      "insert into root.vehicle.d0(timestamp,s3) values(102,'fffff')",
+
+      "insert into root.vehicle.d1(timestamp,s0) values(1,999)",
+      "insert into root.vehicle.d1(timestamp,s0) values(1000,888)",
+
+      "insert into root.vehicle.d0(timestamp,s4) values(100, false)",
+      "insert into root.vehicle.d0(timestamp,s4) values(100, true)",
+  };
+
+  private IoTDB daemon;
+
+  @Before
+  public void setUp() throws Exception {
+    EnvironmentUtils.closeStatMonitor();
+    EnvironmentUtils.closeMemControl();
+    daemon = IoTDB.getInstance();
+    daemon.active();
+    EnvironmentUtils.envSetUp();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    daemon.stop();
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void test() throws ClassNotFoundException, SQLException {
+    insertSQL();
+
+    Connection connection = DriverManager.
+        getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+    lastAggreWithSingleFilterTest();
+    meanAggreWithSingleFilterTest();
+    sumAggreWithSingleFilterTest();
+    firstAggreWithSingleFilterTest();
+    countAggreWithSingleFilterTest();
+    minMaxTimeAggreWithSingleFilterTest();
+    minValueAggreWithSingleFilterTest();
+    maxValueAggreWithSingleFilterTest();
+
+    lastAggreWithMultiFilterTest();
+    countAggreWithMultiFilterTest();
+    minTimeAggreWithMultiFilterTest();
+    maxTimeAggreWithMultiFilterTest();
+    minValueAggreWithMultiFilterTest();
+    maxValueAggreWithMultiFilterTest();
+    meanAggreWithMultiFilterTest();
+    sumAggreWithMultiFilterTest();
+    firstAggreWithMultiFilterTest();
+    connection.close();
+
+  }
+
+  private void lastAggreWithMultiFilterTest() throws SQLException {
+    String[] retArray = new String[]{
+        "0,9,39,63.0,E,true"
+    };
+
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute(
+          "select last(s0),last(s1),last(s2),last(s3),last(s4)" +
+              " from root.vehicle.d0 where s1 >= 0");
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," +
+            resultSet.getString(Constant.last(d0s0)) + "," + resultSet
+            .getString(Constant.last(d0s1))
+            + "," + resultSet.getString(Constant.last(d0s2)) + "," +
+            resultSet.getString(Constant.last(d0s3)) + "," + resultSet
+            .getString(Constant.last(d0s4));
+        // System.out.println("============ " + ans);
+        Assert.assertEquals(ans, retArray[cnt]);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  private void lastAggreWithSingleFilterTest() throws SQLException {
+    String[] retArray = new String[]{
+        "0,9,39,63.0"
+    };
+
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute("select last(s0),last(s1),last(s2)" +
+          " from root.vehicle.d0 where s1 >= 0");
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," +
+            resultSet.getString(Constant.last(d0s0)) + ","
+            + resultSet.getString(Constant.last(d0s1)) + "," +
+            resultSet.getString(Constant.last(d0s2));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  private void sumAggreWithSingleFilterTest() throws SQLException {
+    String[] retArray = new String[]{
+        "0,55061.0,156752.0,20254"
+    };
+
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute("select sum(s0),sum(s1),sum(s2)" +
+          " from root.vehicle.d0 where s1 >= 0");
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(sum(d0s0)) + ","
+            + resultSet.getString(sum(d0s1)) + "," + Math
+            .round(resultSet.getDouble(Constant.sum(d0s2)));
+        Assert.assertEquals(ans, retArray[cnt]);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  private void firstAggreWithSingleFilterTest() throws SQLException {
+    String[] retArray = new String[]{
+        "0,90,1101,2.22,ddddd,true"
+    };
+
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute("select first(s0),first(s1),first(s2),first(s3),"
+          + "first(s4) from root.vehicle.d0 where s1 >= 0");
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," +
+            resultSet.getString(Constant.first(d0s0)) + ","
+            + resultSet.getString(Constant.first(d0s1)) + "," +
+            resultSet.getString(Constant.first(d0s2))
+            + "," + resultSet.getString(Constant.first(d0s3)) + "," +
+            resultSet.getString(Constant.first(d0s4));
+        Assert.assertEquals(ans, retArray[cnt]);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  private void meanAggreWithSingleFilterTest() throws SQLException {
+    String[] retArray = new String[]{
+        "0,75,212,28"
+    };
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute(
+          "select mean(s0),mean(s1),mean(s2) from root.vehicle.d0 where s1 >= 0");
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans =
+            resultSet.getString(TIMESTAMP_STR) + "," + Math.round(resultSet.getDouble(mean(d0s0)))
+                + "," + Math.round(resultSet.getDouble(mean(d0s1))) + "," +
+                Math.round(resultSet.getDouble(mean(d0s2)));
+        //System.out.println("!!!!!============ " + ans);
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  private void countAggreWithSingleFilterTest() throws SQLException {
+    String[] retArray = new String[]{
+        "0,733,740,734"
+    };
+
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute(
+          "select count(s0),count(s1),count(s2) from root.vehicle.d0 where s1 >= 0");
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(count(d0s0))
+            + "," + resultSet.getString(count(d0s1)) + "," + resultSet.getString(count(d0s2));
+        //System.out.println("!!!!!============ " + ans);
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  private void minMaxTimeAggreWithSingleFilterTest() throws ClassNotFoundException, SQLException {
+    String[] retArray = new String[]{
+        "0,104,1,2,101,100"
+    };
+
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute(
+          "select min_time(s0),min_time(s1),min_time(s2),min_time(s3),min_time(s4)" +
+              " from root.vehicle.d0 where s1 >= 0");
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," +
+            resultSet.getString(Constant.min_time(d0s0)) + "," +
+            resultSet.getString(Constant.min_time(d0s1)) + "," +
+            resultSet.getString(Constant.min_time(d0s2)) +
+            "," + resultSet.getString(Constant.min_time(d0s3)) +
+            "," + resultSet.getString(Constant.min_time(d0s4));
+        Assert.assertEquals(ans, retArray[cnt]);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+
+      retArray = new String[]{
+          "0,3999,3999,3999,3599,100"
+      };
+      statement = connection.createStatement();
+      hasResultSet = statement.execute("select max_time(s0),max_time(s1),max_time(s2),"
+          + "max_time(s3),max_time(s4) from root.vehicle.d0 " +
+          "where s1 < 50000 and s1 != 100");
+      Assert.assertTrue(hasResultSet);
+      resultSet = statement.getResultSet();
+      cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(max_time(d0s0))
+            + "," + resultSet.getString(max_time(d0s1)) + "," + resultSet.getString(max_time(d0s2))
+            + "," + resultSet.getString(max_time(d0s3)) + "," + resultSet.getString(max_time(d0s4));
+        Assert.assertEquals(ans, retArray[cnt]);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  private void minValueAggreWithSingleFilterTest() throws ClassNotFoundException, SQLException {
+    String[] retArray = new String[]{
+        "0,0,0,0.0,B,true"
+    };
+
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    Connection connection = null;
+    try {
+      connection = DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute("select min_value(s0),min_value(s1),min_value(s2),"
+          + "min_value(s3),min_value(s4) from root.vehicle.d0 " +
+          "where s1 < 50000 and s1 != 100");
+
+      if (hasResultSet) {
+        ResultSet resultSet = statement.getResultSet();
+        int cnt = 0;
+        while (resultSet.next()) {
+          String ans =
+              resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(min_value(d0s0))
+                  + "," + resultSet.getString(min_value(d0s1)) + "," + resultSet
+                  .getString(min_value(d0s2))
+                  + "," + resultSet.getString(min_value(d0s3)) + "," + resultSet
+                  .getString(min_value(d0s4));
+          //System.out.println("============ " + ans);
+          Assert.assertEquals(ans, retArray[cnt]);
+          cnt++;
+        }
+        Assert.assertEquals(1, cnt);
+      }
+      statement.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  private void maxValueAggreWithSingleFilterTest() throws ClassNotFoundException, SQLException {
+    String[] retArray = new String[]{
+        "0,99,40000,122.0,fffff,true"
+    };
+
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute("select max_value(s0),max_value(s1),max_value(s2),"
+          + "max_value(s3),max_value(s4) from root.vehicle.d0 " +
+          "where s1 < 50000 and s1 != 100");
+
+      if (hasResultSet) {
+        ResultSet resultSet = statement.getResultSet();
+        int cnt = 0;
+        while (resultSet.next()) {
+          String ans =
+              resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(max_value(d0s0))
+                  + "," + resultSet.getString(max_value(d0s1)) + "," + resultSet
+                  .getString(max_value(d0s2))
+                  + "," + resultSet.getString(max_value(d0s3)) + "," + resultSet
+                  .getString(max_value(d0s4));
+          //System.out.println("============ " + ans);
+          Assert.assertEquals(ans, retArray[cnt]);
+          cnt++;
+        }
+        Assert.assertEquals(1, cnt);
+      }
+      statement.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  private void meanAggreWithMultiFilterTest() throws ClassNotFoundException, SQLException {
+    String[] retArray = new String[]{
+        "0,55061.0,733,75,212,28"
+    };
+
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute("select sum(s0),count(s0),mean(s0),mean(s1),"
+          + "mean(s2) from root.vehicle.d0 " +
+          "where s1 >= 0 or s2 < 10");
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans =
+            resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(sum(d0s0)) + "," +
+                resultSet.getString(count(d0s0)) + "," + Math.round(resultSet.getDouble(mean(d0s0)))
+                + "," +
+                Math.round(resultSet.getDouble(mean(d0s1))) + "," + Math
+                .round(resultSet.getDouble(mean(d0s2)));
+        //System.out.println("!!!!!============ " + ans);
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  private void sumAggreWithMultiFilterTest() throws ClassNotFoundException, SQLException {
+    String[] retArray = new String[]{
+        "0,55061.0,156752.0,20262"
+    };
+
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute("select sum(s0),sum(s1),sum(s2) from root.vehicle.d0"
+          + " where s1 >= 0 or s2 < 10");
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(sum(d0s0))
+            + "," + resultSet.getString(sum(d0s1)) + "," + Math
+            .round(resultSet.getDouble(sum(d0s2)));
+        //String ans = resultSet.getString(sum(d0s3));
+        //System.out.println("!!!!!============ " + ans);
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  private void firstAggreWithMultiFilterTest() throws ClassNotFoundException, SQLException {
+    String[] retArray = new String[]{
+        "0,90,1101,2.22,ddddd,true"
+    };
+
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute("select first(s0),first(s1),first(s2),first(s3),"
+          + "first(s4) from root.vehicle.d0 where s1 >= 0 or s2 < 10");
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(first(d0s0))
+            + "," + resultSet.getString(first(d0s1)) + "," + resultSet.getString(first(d0s2))
+            + "," + resultSet.getString(first(d0s3)) + "," + resultSet.getString(first(d0s4));
+        //String ans = resultSet.getString(first(d0s3));
+        //System.out.println("!!!!!============ " + ans);
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  private void countAggreWithMultiFilterTest() throws ClassNotFoundException, SQLException {
+    String[] retArray = new String[]{
+        "0,733,740,736,482,1"
+    };
+
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute("select count(s0),count(s1),count(s2),count(s3),"
+          + "count(s4) from root.vehicle.d0 where s1 >= 0 or s2 < 10");
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(count(d0s0))
+            + "," + resultSet.getString(count(d0s1)) + "," + resultSet.getString(count(d0s2))
+            + "," + resultSet.getString(count(d0s3)) + "," + resultSet.getString(count(d0s4));
+        //String ans = resultSet.getString(count(d0s3));
+        //System.out.println("!!!!!============ " + ans);
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  private void minTimeAggreWithMultiFilterTest() throws ClassNotFoundException, SQLException {
+    String[] retArray = new String[]{
+        "0,104,1,2,101,100"
+    };
+
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute("select min_time(s0),min_time(s1),min_time(s2),"
+          + "min_time(s3),min_time(s4) from root.vehicle.d0 where s1 >= 0");
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," +
+            resultSet.getString(Constant.min_time(d0s0)) + "," +
+            resultSet.getString(Constant.min_time(d0s1)) + "," +
+            resultSet.getString(Constant.min_time(d0s2)) + "," +
+            resultSet.getString(Constant.min_time(d0s3)) + "," +
+            resultSet.getString(Constant.min_time(d0s4));
+        // System.out.println("============ " + ans);
+        Assert.assertEquals(ans, retArray[cnt]);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  private void maxTimeAggreWithMultiFilterTest() throws ClassNotFoundException, SQLException {
+    String[] retArray = new String[]{
+        "0,3999,3999,3999,3599,100"
+    };
+
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute("select max_time(s0),max_time(s1),max_time(s2),"
+          + "max_time(s3),max_time(s4) from root.vehicle.d0 " +
+          "where s1 < 50000 and s1 != 100");
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(max_time(d0s0))
+            + "," + resultSet.getString(max_time(d0s1)) + "," + resultSet.getString(max_time(d0s2))
+            + "," + resultSet.getString(max_time(d0s3)) + "," + resultSet.getString(max_time(d0s4));
+        //System.out.println("============ " + ans);
+        Assert.assertEquals(ans, retArray[cnt]);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  private void minValueAggreWithMultiFilterTest() throws ClassNotFoundException, SQLException {
+    String[] retArray = new String[]{
+        "0,0,0,0.0,B,true"
+    };
+
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute("select min_value(s0),min_value(s1),min_value(s2),"
+          + "min_value(s3),min_value(s4) from root.vehicle.d0 " +
+          "where s1 < 50000 and s1 != 100");
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(min_value(d0s0))
+            + "," + resultSet.getString(min_value(d0s1)) + "," + resultSet
+            .getString(min_value(d0s2))
+            + "," + resultSet.getString(min_value(d0s3)) + "," + resultSet
+            .getString(min_value(d0s4));
+        //System.out.println("============ " + ans);
+        Assert.assertEquals(ans, retArray[cnt]);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  private void maxValueAggreWithMultiFilterTest() throws ClassNotFoundException, SQLException {
+    String[] retArray = new String[]{
+        "0,99,40000,122.0,fffff,true"
+    };
+
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute("select max_value(s0),max_value(s1),max_value(s2),"
+          + "max_value(s3),max_value(s4) from root.vehicle.d0 where s1 < 50000 and s1 != 100");
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(max_value(d0s0))
+            + "," + resultSet.getString(max_value(d0s1)) + "," + resultSet
+            .getString(max_value(d0s2))
+            + "," + resultSet.getString(max_value(d0s3)) + "," + resultSet
+            .getString(max_value(d0s4));
+        Assert.assertEquals(ans, retArray[cnt]);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  private static String[] stringValue = new String[]{"A", "B", "C", "D", "E"};
+
+  public static void insertSQL() throws ClassNotFoundException, SQLException {
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    Connection connection = null;
+    double d0s0sum = 0.0, d0s1sum = 0.0, d0s2sum = 0.0;
+    int cnt = 0;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      for (String sql : createSql) {
+        statement.execute(sql);
+      }
+
+      // insert large amount of data
+      for (int time = 3000; time < 3600; time++) {
+        if (time % 5 == 0) {
+          continue;
+        }
+
+        String sql = String.format("insert into root.vehicle.d0(timestamp,s0) values(%s,%s)",
+            time, time % 100);
+        statement.execute(sql);
+        sql = String.format("insert into root.vehicle.d0(timestamp,s1) values(%s,%s)",
+            time, time % 17);
+        statement.execute(sql);
+        sql = String.format("insert into root.vehicle.d0(timestamp,s2) values(%s,%s)",
+            time, time % 22);
+        statement.execute(sql);
+        sql = String.format("insert into root.vehicle.d0(timestamp,s3) values(%s,'%s')",
+            time, stringValue[time % 5]);
+        statement.execute(sql);
+        cnt++;
+        d0s0sum += time % 100;
+        d0s1sum += time % 17;
+        d0s2sum += time % 22;
+      }
+
+      statement.execute("flush");
+      System.out.println("cnt = " + cnt);
+
+      // insert large amount of data
+      for (int time = 3700; time < 4000; time++) {
+        if (time % 6 == 0) {
+          continue;
+        }
+
+        String sql = String.format("insert into root.vehicle.d0(timestamp,s0) values(%s,%s)",
+            time, time % 70);
+        statement.execute(sql);
+        sql = String.format("insert into root.vehicle.d0(timestamp,s1) values(%s,%s)",
+            time, time % 40);
+        statement.execute(sql);
+        sql = String.format("insert into root.vehicle.d0(timestamp,s2) values(%s,%s)",
+            time, time % 123);
+        statement.execute(sql);
+
+        cnt++;
+        d0s0sum += time % 70;
+        d0s1sum += time % 40;
+        d0s2sum += time % 123;
+      }
+
+      statement.execute("merge");
+
+      System.out.println("large insert cnt = " + cnt);
+      System.out
+          .println("d0s0sum = " + d0s0sum + "; d0s1sum = " + d0s1sum + "; d0s2sum = " + d0s2sum);
+      for (String sql : insertSql) {
+        statement.execute(sql);
+      }
+
+      statement.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+}
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAggregationSmallDataIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAggregationSmallDataIT.java
new file mode 100644
index 0000000..b9608e0
--- /dev/null
+++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAggregationSmallDataIT.java
@@ -0,0 +1,757 @@
+/**
+ * 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.iotdb.db.integration;
+
+import static org.apache.iotdb.db.integration.Constant.count;
+import static org.apache.iotdb.db.integration.Constant.first;
+import static org.apache.iotdb.db.integration.Constant.last;
+import static org.apache.iotdb.db.integration.Constant.max_time;
+import static org.apache.iotdb.db.integration.Constant.max_value;
+import static org.apache.iotdb.db.integration.Constant.mean;
+import static org.apache.iotdb.db.integration.Constant.min_time;
+import static org.apache.iotdb.db.integration.Constant.min_value;
+import static org.apache.iotdb.db.integration.Constant.sum;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.jdbc.Config;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Multiple aggregation with filter test.
+ */
+public class IoTDBAggregationSmallDataIT {
+
+  private static final String TIMESTAMP_STR = "Time";
+  private final String d0s0 = "root.vehicle.d0.s0";
+  private final String d0s1 = "root.vehicle.d0.s1";
+  private final String d0s2 = "root.vehicle.d0.s2";
+  private final String d0s3 = "root.vehicle.d0.s3";
+  private final String d0s4 = "root.vehicle.d0.s4";
+  private final String d1s0 = "root.vehicle.d1.s0";
+  private final String d1s1 = "root.vehicle.d1.s1";
+
+  private static String[] sqls = new String[]{
+      "SET STORAGE GROUP TO root.vehicle",
+      "CREATE TIMESERIES root.vehicle.d1.s0 WITH DATATYPE=INT32, ENCODING=RLE",
+      "CREATE TIMESERIES root.vehicle.d1.s1 WITH DATATYPE=INT64, ENCODING=RLE",
+
+      "CREATE TIMESERIES root.vehicle.d0.s0 WITH DATATYPE=INT32, ENCODING=RLE",
+      "CREATE TIMESERIES root.vehicle.d0.s1 WITH DATATYPE=INT64, ENCODING=RLE",
+      "CREATE TIMESERIES root.vehicle.d0.s2 WITH DATATYPE=FLOAT, ENCODING=RLE",
+      "CREATE TIMESERIES root.vehicle.d0.s3 WITH DATATYPE=TEXT, ENCODING=PLAIN",
+      "CREATE TIMESERIES root.vehicle.d0.s4 WITH DATATYPE=BOOLEAN, ENCODING=PLAIN",
+
+      "insert into root.vehicle.d0(timestamp,s0) values(1,101)",
+      "insert into root.vehicle.d0(timestamp,s0) values(2,198)",
+      "insert into root.vehicle.d0(timestamp,s0) values(100,99)",
+      "insert into root.vehicle.d0(timestamp,s0) values(101,99)",
+      "insert into root.vehicle.d0(timestamp,s0) values(102,80)",
+      "insert into root.vehicle.d0(timestamp,s0) values(103,99)",
+      "insert into root.vehicle.d0(timestamp,s0) values(104,90)",
+      "insert into root.vehicle.d0(timestamp,s0) values(105,99)",
+      "insert into root.vehicle.d0(timestamp,s0) values(106,99)",
+      "insert into root.vehicle.d0(timestamp,s0) values(2,10000)",
+      "insert into root.vehicle.d0(timestamp,s0) values(50,10000)",
+      "insert into root.vehicle.d0(timestamp,s0) values(1000,22222)",
+      "insert into root.vehicle.d0(timestamp,s0) values(106,199)",
+      "DELETE FROM root.vehicle.d0.s0 WHERE time < 104",
+
+      "insert into root.vehicle.d0(timestamp,s1) values(1,1101)",
+      "insert into root.vehicle.d0(timestamp,s1) values(2,198)",
+      "insert into root.vehicle.d0(timestamp,s1) values(100,199)",
+      "insert into root.vehicle.d0(timestamp,s1) values(101,199)",
+      "insert into root.vehicle.d0(timestamp,s1) values(102,180)",
+      "insert into root.vehicle.d0(timestamp,s1) values(103,199)",
+      "insert into root.vehicle.d0(timestamp,s1) values(104,190)",
+      "insert into root.vehicle.d0(timestamp,s1) values(105,199)",
+      "insert into root.vehicle.d0(timestamp,s1) values(2,40000)",
+      "insert into root.vehicle.d0(timestamp,s1) values(50,50000)",
+      "insert into root.vehicle.d0(timestamp,s1) values(1000,55555)",
+
+      "insert into root.vehicle.d0(timestamp,s2) values(1000,55555)",
+      "insert into root.vehicle.d0(timestamp,s2) values(2,2.22)",
+      "insert into root.vehicle.d0(timestamp,s2) values(3,3.33)",
+      "insert into root.vehicle.d0(timestamp,s2) values(4,4.44)",
+      "insert into root.vehicle.d0(timestamp,s2) values(102,10.00)",
+      "insert into root.vehicle.d0(timestamp,s2) values(105,11.11)",
+      "insert into root.vehicle.d0(timestamp,s2) values(1000,1000.11)",
+
+      "insert into root.vehicle.d0(timestamp,s3) values(60,'aaaaa')",
+      "insert into root.vehicle.d0(timestamp,s3) values(70,'bbbbb')",
+      "insert into root.vehicle.d0(timestamp,s3) values(80,'ccccc')",
+      "insert into root.vehicle.d0(timestamp,s3) values(101,'ddddd')",
+      "insert into root.vehicle.d0(timestamp,s3) values(102,'fffff')",
+
+      "insert into root.vehicle.d1(timestamp,s0) values(1,999)",
+      "insert into root.vehicle.d1(timestamp,s0) values(1000,888)",
+
+      "insert into root.vehicle.d0(timestamp,s1) values(2000-01-01T08:00:00+08:00, 100)",
+      "insert into root.vehicle.d0(timestamp,s3) values(2000-01-01T08:00:00+08:00, 'good')",
+
+      "insert into root.vehicle.d0(timestamp,s4) values(100, false)",
+      "insert into root.vehicle.d0(timestamp,s4) values(100, true)"
+  };
+
+  private IoTDB daemon;
+
+  @Before
+  public void setUp() throws Exception {
+    EnvironmentUtils.closeStatMonitor();
+    EnvironmentUtils.closeMemControl();
+    daemon = IoTDB.getInstance();
+    daemon.active();
+    EnvironmentUtils.envSetUp();
+
+    Thread.sleep(5000);
+    insertSQL();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    daemon.stop();
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void countOnlyTimeFilterTest() throws ClassNotFoundException, SQLException {
+    String[] retArray = new String[]{
+        "0,3,7,4,5,1"
+    };
+
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute(
+          "select count(s0),count(s1),count(s2),count(s3),count(s4) " +
+              "from root.vehicle.d0 where time >= 3 and time <= 106");
+
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(count(d0s0))
+            + "," + resultSet.getString(count(d0s1)) + "," + resultSet.getString(count(d0s2))
+            + "," + resultSet.getString(count(d0s3)) + "," + resultSet.getString(count(d0s4));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    } finally {
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+
+  @Test
+  public void functionsNoFilterTest() throws ClassNotFoundException, SQLException {
+    String[] retArray = new String[]{
+        "0,4,0,6,1",
+        "0,22222,null,good",
+        "0,90,null,aaaaa",
+        "0,22222,null,good",
+        "0,22610.0,0.0"
+    };
+
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    Connection connection = null;
+    try {
+      connection = DriverManager.
+          getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+
+      //select count(d0.s0),count(d1.s1),count(d0.s3),count(d0.s4) from root.vehicle
+      Statement statement = connection.createStatement();
+      boolean hasResultSet = statement.execute(
+          "select count(d0.s0),count(d1.s1),count(d0.s3),count(d0.s4) from root.vehicle");
+
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(count(d0s0))
+            + "," + resultSet.getString(count(d1s1)) + "," + resultSet.getString(count(d0s3))
+            + "," + resultSet.getString(count(d0s4));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(1, cnt);
+      statement.close();
+
+      //select max_value(d0.s0),max_value(d1.s1),max_value(d0.s3) from root.vehicle
+      statement = connection.createStatement();
+      hasResultSet = statement.execute(
+          "select max_value(d0.s0),max_value(d1.s1),max_value(d0.s3) from root.vehicle");
+      resultSet = statement.getResultSet();
+      Assert.assertTrue(hasResultSet);
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(max_value(d0s0))
+            + "," + resultSet.getString(max_value(d1s1)) + ","
+            + resultSet.getString(max_value(d0s3));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(2, cnt);
+      statement.close();
+
+      //select first(d0.s0),first(d1.s1),first(d0.s3) from root.vehicle
+      statement = connection.createStatement();
+      hasResultSet = statement.execute(
+          "select first(d0.s0),first(d1.s1),first(d0.s3) from root.vehicle");
+      resultSet = statement.getResultSet();
+      Assert.assertTrue(hasResultSet);
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(first(d0s0))
+            + "," + resultSet.getString(first(d1s1)) + "," + resultSet.getString(first(d0s3));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(3, cnt);
+      statement.close();
+
+      //select last(d0.s0),last(d1.s1),last(d0.s3) from root.vehicle
+      statement = connection.createStatement();
+      hasResultSet = statement.execute(
+          "select last(d0.s0),last(d1.s1),last(d0.s3) from root.vehicle");
+      resultSet = statement.getResultSet();
+      Assert.assertTrue(hasResultSet);
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(last(d0s0))
+            + "," + resultSet.getString(last(d1s1)) + "," + resultSet.getString(last(d0s3));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(4, cnt);
+      statement.close();
+
+      //select sum(d0.s0),sum(d1.s1),sum(d0.s3) from root.vehicle
+      statement = connection.createStatement();
+      hasResultSet = statement.execute("select sum(d0.s0),sum(d1.s1) from root.vehicle");
+      resultSet = statement.getResultSet();
+      Assert.assertTrue(hasResultSet);
+      while (resultSet.next()) {
+        String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(sum(d0s0))
+            + "," + resultSet.getString(sum(d1s1));
+        Assert.assertEquals(retArray[cnt], ans);
+        cnt++;
+      }
+      Assert.assertEquals(5, cnt);
+      statement.close();
+
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
... 2932 lines suppressed ...