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/06/25 15:11:38 UTC

[incubator-iotdb] branch feature_async_close_tsfile updated: rename (#205)

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

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


The following commit(s) were added to refs/heads/feature_async_close_tsfile by this push:
     new 18d08f8  rename (#205)
18d08f8 is described below

commit 18d08f8f3a42e7d68b6bde0f9186ae5815dce305
Author: RUI, LEI <33...@users.noreply.github.com>
AuthorDate: Tue Jun 25 23:11:34 2019 +0800

    rename (#205)
    
    * add annotations
---
 .../engine/bufferwrite/BufferWriteProcessor.java   |   4 +-
 .../db/engine/filenode/FileNodeProcessor.java      |   2 +-
 .../db/qp/executor/IQueryProcessExecutor.java      |   8 +-
 .../db/query/aggregation/AggregateFunction.java    |   4 +-
 .../db/query/aggregation/impl/CountAggrFunc.java   |   4 +-
 .../db/query/aggregation/impl/FirstAggrFunc.java   |   4 +-
 .../db/query/aggregation/impl/LastAggrFunc.java    |   4 +-
 .../db/query/aggregation/impl/MaxTimeAggrFunc.java |   6 +-
 .../query/aggregation/impl/MaxValueAggrFunc.java   |   4 +-
 .../db/query/aggregation/impl/MeanAggrFunc.java    |   4 +-
 .../db/query/aggregation/impl/MinTimeAggrFunc.java |   4 +-
 .../query/aggregation/impl/MinValueAggrFunc.java   |   4 +-
 ...ator.java => EngineDataSetWithValueFilter.java} |  24 ++--
 ...r.java => EngineDataSetWithoutValueFilter.java} |  28 ++--
 .../groupby/GroupByWithOnlyTimeFilterDataSet.java  |   6 +-
 .../groupby/GroupByWithValueFilterDataSet.java     |   6 +-
 .../db/query/executor/AggregateEngineExecutor.java |  26 ++--
 .../iotdb/db/query/executor/EngineExecutor.java    |  22 +--
 .../iotdb/db/query/executor/EngineQueryRouter.java |   6 +-
 .../db/query/executor/FillEngineExecutor.java      |   6 +-
 .../db/query/factory/ISeriesReaderFactory.java     |  65 +++++----
 .../db/query/factory/SeriesReaderFactoryImpl.java  | 160 ++++++++++-----------
 .../java/org/apache/iotdb/db/query/fill/IFill.java |   2 +-
 .../apache/iotdb/db/query/reader/IPointReader.java |   8 +-
 .../org/apache/iotdb/db/query/reader/IReader.java  |  36 -----
 ...derByTimeStamp.java => IReaderByTimeStamp.java} |   4 +-
 ...ilter.java => SeriesReaderWithValueFilter.java} |  11 +-
 ...er.java => SeriesReaderWithoutValueFilter.java} |  39 ++---
 .../iotdb/db/query/reader/mem/MemChunkReader.java  |   6 +-
 .../reader/mem/MemChunkReaderByTimestamp.java      |   4 +-
 ...Timestamp.java => SeriesReaderByTimestamp.java} |  15 +-
 .../db/query/reader/sequence/IterateReader.java    |  17 +--
 ...DataReaderV2.java => SequenceSeriesReader.java} |  52 +++----
 ...2.java => SequenceSeriesReaderByTimestamp.java} |  35 ++---
 ...FileReaderV2.java => UnSealedTsFileReader.java} |  35 ++---
 ...2.java => UnSealedTsFileReaderByTimestamp.java} |  25 ++--
 .../adapter/FileSeriesReaderAdapter.java           |   4 +-
 .../adapter/SeriesReaderByTimestampAdapter.java    |  20 +--
 ...EngineChunkReader.java => DiskChunkReader.java} |  13 +-
 ...estamp.java => DiskChunkReaderByTimestamp.java} |  14 +-
 .../UnsequenceSeriesReader.java}                   |  23 ++-
 .../db/query/timegenerator/EngineLeafNode.java     |   6 +-
 .../query/timegenerator/EngineNodeConstructor.java |   2 +-
 .../org/apache/iotdb/db/service/StartupChecks.java |   2 +-
 .../org/apache/iotdb/db/service/TSServiceImpl.java |   4 +-
 .../iotdb/db/integration/IoTDBAuthorizationIT.java |   2 +-
 .../iotdb/db/integration/IoTDBLargeDataIT.java     |   4 +-
 .../iotdb/db/integration/IoTDBMultiSeriesIT.java   |   4 +-
 .../iotdb/db/integration/IoTDBSeriesReaderIT.java  |   4 +-
 .../iotdb/db/integration/IoTDBTimeZoneIT.java      |   2 +-
 .../org/apache/iotdb/db/qp/plan/QPUpdateTest.java  |   2 +-
 ....java => EngineDataSetWithValueFilterTest.java} |   2 +-
 .../query/reader/FakedSeriesReaderByTimestamp.java |   4 +-
 ...ava => SeriesReaderWithoutValueFilterTest.java} |  10 +-
 .../reader/merge/SeriesMergeSortReaderTest.java    |   3 +-
 ...pTest.java => SeriesReaderByTimestampTest.java} |   7 +-
 ...erTest.java => UnsequenceSeriesReaderTest.java} |  17 +--
 .../query/reader/sequence/SeqDataReaderTest.java   |   4 +-
 .../reader/sequence/UnsealedSeqReaderTest.java     |   4 +-
 ....java => UnseqSeriesReaderByTimestampTest.java} |  12 +-
 ...erTest.java => UnsequenceSeriesReaderTest.java} |  10 +-
 .../writelog/recover/UnseqTsFileRecoverTest.java   |   9 +-
 .../query/dataset/DataSetWithTimeGenerator.java    |  16 +--
 .../query/dataset/DataSetWithoutTimeGenerator.java |   2 +-
 .../query/executor/ExecutorWithTimeGenerator.java  |   6 +-
 ...stamp.java => FileSeriesReaderByTimestamp.java} |   4 +-
 .../query/timegenerator/ReaderByTimestampTest.java |   4 +-
 67 files changed, 451 insertions(+), 459 deletions(-)

diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessor.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessor.java
index 35765e7..0f5fcd8 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessor.java
@@ -222,7 +222,7 @@ public class BufferWriteProcessor extends Processor {
           LOGGER.info("BufferWriteProcessor.insert step2 of SAFE cost: {}", start2);
           Map<MemTableWriteTimeCostType, long[]> map = MemTableWriteTimeCost.getInstance().getTimeCostMaps().get(Thread.currentThread().getName());
           for(MemTableWriteTimeCostType type: MemTableWriteTimeCostType.values()){
-            LOGGER.info("In BufferWriteProcessor.insert step2 of SAFE, {} cost {} ms, executeWithOutTimeGenerator {} times", type, map.get(type)[1], map.get(type)[0]);
+            LOGGER.info("In BufferWriteProcessor.insert step2 of SAFE, {} cost {} ms, executeWithoutValueFilter {} times", type, map.get(type)[1], map.get(type)[0]);
           }
         }
         checkMemThreshold4Flush(memUsage);
@@ -241,7 +241,7 @@ public class BufferWriteProcessor extends Processor {
           LOGGER.info("BufferWriteProcessor.insert step2 of WARNING cost: {}", start2);
           Map<MemTableWriteTimeCostType, long[]> map = MemTableWriteTimeCost.getInstance().getTimeCostMaps().get(Thread.currentThread().getName());
           for(MemTableWriteTimeCostType type: MemTableWriteTimeCostType.values()){
-            LOGGER.info("In BufferWriteProcessor.insert step2 of WARNING, {} cost {} ms, executeWithOutTimeGenerator {} times", type, map.get(type)[1], map.get(type)[0]);
+            LOGGER.info("In BufferWriteProcessor.insert step2 of WARNING, {} cost {} ms, executeWithoutValueFilter {} times", type, map.get(type)[1], map.get(type)[0]);
           }
         }
         try {
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeProcessor.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeProcessor.java
index f639014..0a3f604 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeProcessor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeProcessor.java
@@ -456,7 +456,7 @@
 //  }
 //
 //  /**
-//   * executeWithOutTimeGenerator filenode recovery.
+//   * executeWithoutValueFilter filenode recovery.
 //   */
 //  public void recover() throws FileNodeProcessorException {
 //    // restore sequential files
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/IQueryProcessExecutor.java b/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/IQueryProcessExecutor.java
index 16bca0e..56710b2 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/IQueryProcessExecutor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/IQueryProcessExecutor.java
@@ -78,7 +78,7 @@ public interface IQueryProcessExecutor {
       throws ProcessorException, IOException, PathErrorException, FileNodeManagerException;
 
   /**
-   * executeWithOutTimeGenerator update command and return whether the operator is successful.
+   * executeWithoutValueFilter update command and return whether the operator is successful.
    *
    * @param path : update series seriesPath
    * @param startTime start time in update command
@@ -90,7 +90,7 @@ public interface IQueryProcessExecutor {
       throws ProcessorException;
 
   /**
-   * executeWithOutTimeGenerator delete command and return whether the operator is successful.
+   * executeWithoutValueFilter delete command and return whether the operator is successful.
    *
    * @param deletePlan physical delete plan
    * @return - whether the operator is successful.
@@ -98,7 +98,7 @@ public interface IQueryProcessExecutor {
   boolean delete(DeletePlan deletePlan) throws ProcessorException;
 
   /**
-   * executeWithOutTimeGenerator delete command and return whether the operator is successful.
+   * executeWithoutValueFilter delete command and return whether the operator is successful.
    *
    * @param path : delete series seriesPath
    * @param deleteTime end time in delete command
@@ -107,7 +107,7 @@ public interface IQueryProcessExecutor {
   boolean delete(Path path, long deleteTime) throws ProcessorException;
 
   /**
-   * executeWithOutTimeGenerator insert command and return whether the operator is successful.
+   * executeWithoutValueFilter insert command and return whether the operator is successful.
    *
    * @return - Operate Type.
    */
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
index 6c11cc3..0c9be4f 100644
--- 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
@@ -22,7 +22,7 @@ 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.db.query.reader.IReaderByTimeStamp;
 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;
@@ -115,7 +115,7 @@ public abstract class AggregateFunction {
    * @throws IOException TsFile data cloneList error
    */
   public abstract void calcAggregationUsingTimestamps(long[] timestamps, int length,
-      EngineReaderByTimeStamp dataReader) throws IOException;
+      IReaderByTimeStamp dataReader) throws IOException;
 
   /**
    * Judge if aggregation results have been calculated. In other words, if the aggregated result
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
index 99ef06c..55a4630 100644
--- 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
@@ -23,7 +23,7 @@ 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.query.reader.IReaderByTimeStamp;
 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;
@@ -134,7 +134,7 @@ public class CountAggrFunc extends AggregateFunction {
 
   @Override
   public void calcAggregationUsingTimestamps(long[] timestamps, int length,
-      EngineReaderByTimeStamp dataReader) throws IOException {
+      IReaderByTimeStamp dataReader) throws IOException {
     int cnt = 0;
     for (int i = 0; i < length; i++) {
       Object value = dataReader.getValueInTimestamp(timestamps[i]);
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
index 0688303..7ae5ae0 100644
--- 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
@@ -24,7 +24,7 @@ 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.db.query.reader.IReaderByTimeStamp;
 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;
@@ -133,7 +133,7 @@ public class FirstAggrFunc extends AggregateFunction {
 
   @Override
   public void calcAggregationUsingTimestamps(long[] timestamps, int length,
-      EngineReaderByTimeStamp dataReader) throws IOException {
+      IReaderByTimeStamp dataReader) throws IOException {
     if (resultData.isSetTime()) {
       return;
     }
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
index b8ea98b..c0ec6d9 100644
--- 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
@@ -23,7 +23,7 @@ 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.query.reader.IReaderByTimeStamp;
 import org.apache.iotdb.db.utils.TimeValuePair;
 import org.apache.iotdb.tsfile.file.header.PageHeader;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -117,7 +117,7 @@ public class LastAggrFunc extends AggregateFunction {
 
   @Override
   public void calcAggregationUsingTimestamps(long[] timestamps, int length,
-      EngineReaderByTimeStamp dataReader) throws IOException {
+      IReaderByTimeStamp dataReader) throws IOException {
 
     long time = -1;
     Object lastVal = null;
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
index ba3460b..caae764 100644
--- 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
@@ -23,7 +23,7 @@ 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.query.reader.IReaderByTimeStamp;
 import org.apache.iotdb.db.utils.TimeValuePair;
 import org.apache.iotdb.tsfile.file.header.PageHeader;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -103,10 +103,10 @@ public class MaxTimeAggrFunc extends AggregateFunction {
     }
   }
 
-  //TODO Consider how to reverse order in dataReader(EngineReaderByTimeStamp)
+  //TODO Consider how to reverse order in dataReader(IReaderByTimeStamp)
   @Override
   public void calcAggregationUsingTimestamps(long[] timestamps, int length,
-      EngineReaderByTimeStamp dataReader) throws IOException {
+      IReaderByTimeStamp dataReader) throws IOException {
     long time = -1;
     for (int i = 0; i < length; i++) {
       Object value = dataReader.getValueInTimestamp(timestamps[i]);
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
index 40bea9c..4b77250 100644
--- 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
@@ -23,7 +23,7 @@ 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.query.reader.IReaderByTimeStamp;
 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;
@@ -140,7 +140,7 @@ public class MaxValueAggrFunc extends AggregateFunction {
 
   @Override
   public void calcAggregationUsingTimestamps(long[] timestamps, int length,
-      EngineReaderByTimeStamp dataReader) throws IOException {
+      IReaderByTimeStamp dataReader) throws IOException {
     Comparable<Object> maxVal = null;
     for (int i = 0; i < length; i++) {
       Object value = dataReader.getValueInTimestamp(timestamps[i]);
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
index 5139c0b..6461d52 100644
--- 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
@@ -23,7 +23,7 @@ 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.query.reader.IReaderByTimeStamp;
 import org.apache.iotdb.db.utils.TimeValuePair;
 import org.apache.iotdb.tsfile.file.header.PageHeader;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -149,7 +149,7 @@ public class MeanAggrFunc extends AggregateFunction {
 
   @Override
   public void calcAggregationUsingTimestamps(long[] timestamps, int length,
-      EngineReaderByTimeStamp dataReader) throws IOException {
+      IReaderByTimeStamp dataReader) throws IOException {
     for (int i = 0; i < length; i++) {
       Object value = dataReader.getValueInTimestamp(timestamps[i]);
       if (value != null) {
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
index 278fde2..b27f68f 100644
--- 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
@@ -23,7 +23,7 @@ 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.query.reader.IReaderByTimeStamp;
 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;
@@ -134,7 +134,7 @@ public class MinTimeAggrFunc extends AggregateFunction {
 
   @Override
   public void calcAggregationUsingTimestamps(long[] timestamps, int length,
-      EngineReaderByTimeStamp dataReader) throws IOException {
+      IReaderByTimeStamp dataReader) throws IOException {
     if (resultData.isSetValue()) {
       return;
     }
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
index 137f8fe..6c3df3d 100644
--- 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
@@ -23,7 +23,7 @@ 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.query.reader.IReaderByTimeStamp;
 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;
@@ -135,7 +135,7 @@ public class MinValueAggrFunc extends AggregateFunction {
 
   @Override
   public void calcAggregationUsingTimestamps(long[] timestamps, int length,
-      EngineReaderByTimeStamp dataReader) throws IOException {
+      IReaderByTimeStamp dataReader) throws IOException {
     Comparable<Object> minVal = null;
     for (int i = 0; i < length; i++) {
       Object value = dataReader.getValueInTimestamp(timestamps[i]);
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/EngineDataSetWithValueFilter.java
similarity index 78%
rename from iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGenerator.java
rename to iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithValueFilter.java
index 2f86407..ff789ca 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/EngineDataSetWithValueFilter.java
@@ -20,7 +20,7 @@ package org.apache.iotdb.db.query.dataset;
 
 import java.io.IOException;
 import java.util.List;
-import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.db.query.reader.IReaderByTimeStamp;
 import org.apache.iotdb.db.query.timegenerator.EngineTimeGenerator;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.common.Field;
@@ -28,26 +28,26 @@ import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.read.common.RowRecord;
 import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
 
-public class EngineDataSetWithTimeGenerator extends QueryDataSet {
+public class EngineDataSetWithValueFilter extends QueryDataSet {
 
   private EngineTimeGenerator timeGenerator;
-  private List<EngineReaderByTimeStamp> readers;
+  private List<IReaderByTimeStamp> seriesReaderByTimestampList;
   private boolean hasCachedRowRecord;
   private RowRecord cachedRowRecord;
 
   /**
-   * constructor of EngineDataSetWithTimeGenerator.
+   * constructor of EngineDataSetWithValueFilter.
    *
    * @param paths paths in List structure
    * @param dataTypes time series data type
    * @param timeGenerator EngineTimeGenerator object
-   * @param readers readers in List(EngineReaderByTimeStamp) structure
+   * @param readers readers in List(IReaderByTimeStamp) structure
    */
-  public EngineDataSetWithTimeGenerator(List<Path> paths, List<TSDataType> dataTypes,
-      EngineTimeGenerator timeGenerator, List<EngineReaderByTimeStamp> readers) {
+  public EngineDataSetWithValueFilter(List<Path> paths, List<TSDataType> dataTypes,
+                                      EngineTimeGenerator timeGenerator, List<IReaderByTimeStamp> readers) {
     super(paths, dataTypes);
     this.timeGenerator = timeGenerator;
-    this.readers = readers;
+    this.seriesReaderByTimestampList = readers;
   }
 
   @Override
@@ -77,8 +77,8 @@ public class EngineDataSetWithTimeGenerator extends QueryDataSet {
       boolean hasField = false;
       long timestamp = timeGenerator.next();
       RowRecord rowRecord = new RowRecord(timestamp);
-      for (int i = 0; i < readers.size(); i++) {
-        EngineReaderByTimeStamp reader = readers.get(i);
+      for (int i = 0; i < seriesReaderByTimestampList.size(); i++) {
+        IReaderByTimeStamp reader = seriesReaderByTimestampList.get(i);
         Object value = reader.getValueInTimestamp(timestamp);
         if (value == null) {
           rowRecord.addField(new Field(null));
@@ -100,7 +100,7 @@ public class EngineDataSetWithTimeGenerator extends QueryDataSet {
     return timeGenerator;
   }
 
-  public List<EngineReaderByTimeStamp> getReaders() {
-    return readers;
+  public List<IReaderByTimeStamp> getReaders() {
+    return seriesReaderByTimestampList;
   }
 }
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/EngineDataSetWithoutValueFilter.java
similarity index 81%
rename from iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithoutTimeGenerator.java
rename to iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithoutValueFilter.java
index bc9bb08..413ec8f 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/EngineDataSetWithoutValueFilter.java
@@ -37,9 +37,9 @@ import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
 /**
  * TODO implement this class as TsFile DataSetWithoutTimeGenerator.
  */
-public class EngineDataSetWithoutTimeGenerator extends QueryDataSet {
+public class EngineDataSetWithoutValueFilter extends QueryDataSet {
 
-  private List<IPointReader> readers;
+  private List<IPointReader> seriesReaderWithoutValueFilterList;
 
   private TimeValuePair[] cacheTimeValueList;
 
@@ -48,28 +48,28 @@ public class EngineDataSetWithoutTimeGenerator extends QueryDataSet {
   private Set<Long> timeSet;
 
   /**
-   * constructor of EngineDataSetWithoutTimeGenerator.
+   * constructor of EngineDataSetWithoutValueFilter.
    *
    * @param paths paths in List structure
    * @param dataTypes time series data type
-   * @param readers readers in List(IReader) structure
+   * @param readers readers in List(IPointReader) structure
    * @throws IOException IOException
    */
-  public EngineDataSetWithoutTimeGenerator(List<Path> paths, List<TSDataType> dataTypes,
-      List<IPointReader> readers)
+  public EngineDataSetWithoutValueFilter(List<Path> paths, List<TSDataType> dataTypes,
+                                         List<IPointReader> readers)
       throws IOException {
     super(paths, dataTypes);
-    this.readers = readers;
+    this.seriesReaderWithoutValueFilterList = readers;
     initHeap();
   }
 
   private void initHeap() throws IOException {
     timeSet = new HashSet<>();
     timeHeap = new PriorityQueue<>();
-    cacheTimeValueList = new TimeValuePair[readers.size()];
+    cacheTimeValueList = new TimeValuePair[seriesReaderWithoutValueFilterList.size()];
 
-    for (int i = 0; i < readers.size(); i++) {
-      IPointReader reader = readers.get(i);
+    for (int i = 0; i < seriesReaderWithoutValueFilterList.size(); i++) {
+      IPointReader reader = seriesReaderWithoutValueFilterList.get(i);
       if (reader.hasNext()) {
         TimeValuePair timeValuePair = reader.next();
         cacheTimeValueList[i] = timeValuePair;
@@ -89,14 +89,14 @@ public class EngineDataSetWithoutTimeGenerator extends QueryDataSet {
 
     RowRecord record = new RowRecord(minTime);
 
-    for (int i = 0; i < readers.size(); i++) {
-      IPointReader reader = readers.get(i);
+    for (int i = 0; i < seriesReaderWithoutValueFilterList.size(); i++) {
+      IPointReader reader = seriesReaderWithoutValueFilterList.get(i);
       if (cacheTimeValueList[i] == null) {
         record.addField(new Field(null));
       } else {
         if (cacheTimeValueList[i].getTimestamp() == minTime) {
           record.addField(getField(cacheTimeValueList[i].getValue(), dataTypes.get(i)));
-          if (readers.get(i).hasNext()) {
+          if (seriesReaderWithoutValueFilterList.get(i).hasNext()) {
             cacheTimeValueList[i] = reader.next();
             timeHeapPut(cacheTimeValueList[i].getTimestamp());
           }
@@ -156,6 +156,6 @@ public class EngineDataSetWithoutTimeGenerator extends QueryDataSet {
   }
 
   public List<IPointReader> getReaders() {
-    return readers;
+    return seriesReaderWithoutValueFilterList;
   }
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithOnlyTimeFilterDataSet.java b/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithOnlyTimeFilterDataSet.java
index fb19694..87f81cb 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithOnlyTimeFilterDataSet.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithOnlyTimeFilterDataSet.java
@@ -30,7 +30,7 @@ import org.apache.iotdb.db.query.control.QueryResourceManager;
 import org.apache.iotdb.db.query.factory.SeriesReaderFactoryImpl;
 import org.apache.iotdb.db.query.reader.IAggregateReader;
 import org.apache.iotdb.db.query.reader.IPointReader;
-import org.apache.iotdb.db.query.reader.sequence.SequenceDataReaderV2;
+import org.apache.iotdb.db.query.reader.sequence.SequenceSeriesReader;
 import org.apache.iotdb.tsfile.file.header.PageHeader;
 import org.apache.iotdb.tsfile.read.common.BatchData;
 import org.apache.iotdb.tsfile.read.common.Field;
@@ -86,13 +86,13 @@ public class GroupByWithOnlyTimeFilterDataSet extends GroupByEngineDataSet {
           .getQueryDataSourceV2(path, context);
 
       // sequence reader for sealed tsfile, unsealed tsfile, memory
-      SequenceDataReaderV2 sequenceReader = new SequenceDataReaderV2(
+      SequenceSeriesReader sequenceReader = new SequenceSeriesReader(
           queryDataSource.getSeriesPath(), queryDataSource.getSeqResources(), timeFilter, context,
           false);
 
       // unseq reader for all chunk groups in unSeqFile, memory
       IPointReader unSeqMergeReader = SeriesReaderFactoryImpl.getInstance()
-              .createUnSeqReader(queryDataSource.getSeriesPath(), queryDataSource.getUnseqResources(), context,
+              .createUnseqSeriesReader(queryDataSource.getSeriesPath(), queryDataSource.getUnseqResources(), context,
                       timeFilter);
 
       sequenceReaderList.add(sequenceReader);
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithValueFilterDataSet.java b/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithValueFilterDataSet.java
index 356a726..29e0e15 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithValueFilterDataSet.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithValueFilterDataSet.java
@@ -30,7 +30,7 @@ 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.SeriesReaderFactoryImpl;
-import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.db.query.reader.IReaderByTimeStamp;
 import org.apache.iotdb.db.query.timegenerator.EngineTimeGenerator;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.read.common.RowRecord;
@@ -40,7 +40,7 @@ import org.apache.iotdb.tsfile.utils.Pair;
 
 public class GroupByWithValueFilterDataSet extends GroupByEngineDataSet {
 
-  private List<EngineReaderByTimeStamp> allDataReaderList;
+  private List<IReaderByTimeStamp> allDataReaderList;
   private TimeGenerator timestampGenerator;
   /**
    * cached timestamp for next group by partition.
@@ -78,7 +78,7 @@ public class GroupByWithValueFilterDataSet extends GroupByEngineDataSet {
         .getInstance().beginQueryOfGivenQueryPaths(context.getJobId(), selectedSeries);
     this.timestampGenerator = new EngineTimeGenerator(expression, context);
     this.allDataReaderList = SeriesReaderFactoryImpl.getInstance()
-        .createByTimestampReaders(selectedSeries, context);
+        .createSeriesReadersByTimestamp(selectedSeries, context);
   }
 
   @Override
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
index a0b4859..246fcdc 100644
--- 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
@@ -32,13 +32,13 @@ 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.dataset.EngineDataSetWithoutValueFilter;
 import org.apache.iotdb.db.query.factory.AggreFuncFactory;
 import org.apache.iotdb.db.query.factory.SeriesReaderFactoryImpl;
 import org.apache.iotdb.db.query.reader.IAggregateReader;
 import org.apache.iotdb.db.query.reader.IPointReader;
-import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
-import org.apache.iotdb.db.query.reader.sequence.SequenceDataReaderV2;
+import org.apache.iotdb.db.query.reader.IReaderByTimeStamp;
+import org.apache.iotdb.db.query.reader.sequence.SequenceSeriesReader;
 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;
@@ -76,7 +76,7 @@ public class AggregateEngineExecutor {
   }
 
   /**
-   * executeWithOutTimeGenerator aggregate function with only time filter or no filter.
+   * executeWithoutValueFilter aggregate function with only time filter or no filter.
    *
    * @param context query context
    */
@@ -104,18 +104,18 @@ public class AggregateEngineExecutor {
           .getQueryDataSourceV2(selectedSeries.get(i), context);
 
       // sequence reader for sealed tsfile, unsealed tsfile, memory
-      SequenceDataReaderV2 sequenceReader;
+      SequenceSeriesReader sequenceReader;
       if (function instanceof MaxTimeAggrFunc || function instanceof LastAggrFunc) {
-        sequenceReader = new SequenceDataReaderV2(queryDataSource.getSeriesPath(),
+        sequenceReader = new SequenceSeriesReader(queryDataSource.getSeriesPath(),
             queryDataSource.getSeqResources(), timeFilter, context, true);
       } else {
-        sequenceReader = new SequenceDataReaderV2(queryDataSource.getSeriesPath(),
+        sequenceReader = new SequenceSeriesReader(queryDataSource.getSeriesPath(),
             queryDataSource.getSeqResources(), timeFilter, context, false);
       }
 
       // unseq reader for all chunk groups in unSeqFile, memory
       IPointReader unSeqMergeReader = SeriesReaderFactoryImpl.getInstance()
-              .createUnSeqReader(queryDataSource.getSeriesPath(), queryDataSource.getUnseqResources(), context,
+              .createUnseqSeriesReader(queryDataSource.getSeriesPath(), queryDataSource.getUnseqResources(), context,
                       timeFilter);
 
       readersOfSequenceData.add(sequenceReader);
@@ -253,7 +253,7 @@ public class AggregateEngineExecutor {
 
 
   /**
-   * executeWithOutTimeGenerator aggregate function with value filter.
+   * executeWithoutValueFilter aggregate function with value filter.
    *
    * @param context query context.
    */
@@ -264,8 +264,8 @@ public class AggregateEngineExecutor {
     QueryResourceManager.getInstance().beginQueryOfGivenExpression(context.getJobId(), expression);
 
     EngineTimeGenerator timestampGenerator = new EngineTimeGenerator(expression, context);
-    List<EngineReaderByTimeStamp> readersOfSelectedSeries = SeriesReaderFactoryImpl.getInstance()
-        .createByTimestampReaders(selectedSeries, context);
+    List<IReaderByTimeStamp> readersOfSelectedSeries = SeriesReaderFactoryImpl.getInstance()
+        .createSeriesReadersByTimestamp(selectedSeries, context);
 
     List<AggregateFunction> aggregateFunctions = new ArrayList<>();
     for (int i = 0; i < selectedSeries.size(); i++) {
@@ -286,7 +286,7 @@ public class AggregateEngineExecutor {
   private List<AggreResultData> aggregateWithTimeGenerator(
       List<AggregateFunction> aggregateFunctions,
       EngineTimeGenerator timestampGenerator,
-      List<EngineReaderByTimeStamp> readersOfSelectedSeries)
+      List<IReaderByTimeStamp> readersOfSelectedSeries)
       throws IOException {
 
     while (timestampGenerator.hasNext()) {
@@ -328,6 +328,6 @@ public class AggregateEngineExecutor {
       dataTypes.add(resultData.getDataType());
       resultDataPointReaders.add(new AggreResultDataPointReader(resultData));
     }
-    return new EngineDataSetWithoutTimeGenerator(selectedSeries, dataTypes, resultDataPointReaders);
+    return new EngineDataSetWithoutValueFilter(selectedSeries, dataTypes, resultDataPointReaders);
   }
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutor.java b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutor.java
index 362ad7b..2898fc7 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutor.java
@@ -26,11 +26,11 @@ 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.EngineDataSetWithTimeGenerator;
-import org.apache.iotdb.db.query.dataset.EngineDataSetWithoutTimeGenerator;
+import org.apache.iotdb.db.query.dataset.EngineDataSetWithValueFilter;
+import org.apache.iotdb.db.query.dataset.EngineDataSetWithoutValueFilter;
 import org.apache.iotdb.db.query.factory.SeriesReaderFactoryImpl;
 import org.apache.iotdb.db.query.reader.IPointReader;
-import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.db.query.reader.IReaderByTimeStamp;
 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;
@@ -53,7 +53,7 @@ public class EngineExecutor {
   /**
    * without filter or with global time filter.
    */
-  public QueryDataSet executeWithOutTimeGenerator(QueryContext context)
+  public QueryDataSet executeWithoutValueFilter(QueryContext context)
       throws FileNodeManagerException, IOException {
 
     Filter timeFilter = null;
@@ -77,12 +77,12 @@ public class EngineExecutor {
       }
 
       IPointReader reader = SeriesReaderFactoryImpl.getInstance()
-          .createReaderWithOptGlobalTimeFilter(path, timeFilter, context);
+          .createSeriesReaderWithoutValueFilter(path, timeFilter, context);
       readersOfSelectedSeries.add(reader);
     }
 
     try {
-      return new EngineDataSetWithoutTimeGenerator(queryExpression.getSelectedSeries(), dataTypes,
+      return new EngineDataSetWithoutValueFilter(queryExpression.getSelectedSeries(), dataTypes,
           readersOfSelectedSeries);
     } catch (IOException e) {
       throw new FileNodeManagerException(e);
@@ -90,12 +90,12 @@ public class EngineExecutor {
   }
 
   /**
-   * executeWithOutTimeGenerator query.
+   * executeWithoutValueFilter query.
    *
    * @return QueryDataSet object
    * @throws FileNodeManagerException FileNodeManagerException
    */
-  public QueryDataSet executeWithTimeGenerator(QueryContext context) throws FileNodeManagerException, IOException {
+  public QueryDataSet executeWithValueFilter(QueryContext context) throws FileNodeManagerException, IOException {
 
     QueryResourceManager.getInstance()
         .beginQueryOfGivenQueryPaths(context.getJobId(), queryExpression.getSelectedSeries());
@@ -103,11 +103,11 @@ public class EngineExecutor {
         .beginQueryOfGivenExpression(context.getJobId(), queryExpression.getExpression());
 
     EngineTimeGenerator timestampGenerator;
-    List<EngineReaderByTimeStamp> readersOfSelectedSeries;
+    List<IReaderByTimeStamp> readersOfSelectedSeries;
 
     timestampGenerator = new EngineTimeGenerator(queryExpression.getExpression(), context);
     readersOfSelectedSeries = SeriesReaderFactoryImpl.getInstance()
-        .createByTimestampReaders(queryExpression.getSelectedSeries(), context);
+        .createSeriesReadersByTimestamp(queryExpression.getSelectedSeries(), context);
 
     List<TSDataType> dataTypes = new ArrayList<>();
 
@@ -119,7 +119,7 @@ public class EngineExecutor {
       }
 
     }
-    return new EngineDataSetWithTimeGenerator(queryExpression.getSelectedSeries(), dataTypes,
+    return new EngineDataSetWithValueFilter(queryExpression.getSelectedSeries(), dataTypes,
         timestampGenerator,
         readersOfSelectedSeries);
   }
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 a2257d5..f474cae 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
@@ -61,9 +61,9 @@ public class EngineQueryRouter implements IEngineQueryRouter {
         EngineExecutor engineExecutor =
             new EngineExecutor(queryExpression);
         if (optimizedExpression.getType() == ExpressionType.GLOBAL_TIME) {
-          return engineExecutor.executeWithOutTimeGenerator(context);
+          return engineExecutor.executeWithoutValueFilter(context);
         } else {
-          return engineExecutor.executeWithTimeGenerator(context);
+          return engineExecutor.executeWithValueFilter(context);
         }
 
       } catch (QueryFilterOptimizationException | IOException e) {
@@ -73,7 +73,7 @@ public class EngineQueryRouter implements IEngineQueryRouter {
       EngineExecutor engineExecutor = new EngineExecutor(
           queryExpression);
       try {
-        return engineExecutor.executeWithOutTimeGenerator(context);
+        return engineExecutor.executeWithoutValueFilter(context);
       } catch (IOException e) {
         throw new FileNodeManagerException(e);
       }
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
index ac54f87..33980dd 100644
--- 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
@@ -28,7 +28,7 @@ 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.dataset.EngineDataSetWithoutValueFilter;
 import org.apache.iotdb.db.query.fill.IFill;
 import org.apache.iotdb.db.query.fill.PreviousFill;
 import org.apache.iotdb.db.query.reader.IPointReader;
@@ -52,7 +52,7 @@ public class FillEngineExecutor {
   }
 
   /**
-   * executeWithOutTimeGenerator fill.
+   * executeWithoutValueFilter fill.
    *
    * @param context query context
    */
@@ -82,7 +82,7 @@ public class FillEngineExecutor {
       readers.add(fill.getFillResult());
     }
 
-    return new EngineDataSetWithoutTimeGenerator(selectedSeries, dataTypeList, readers);
+    return new EngineDataSetWithoutValueFilter(selectedSeries, dataTypeList, readers);
   }
 
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/factory/ISeriesReaderFactory.java b/iotdb/src/main/java/org/apache/iotdb/db/query/factory/ISeriesReaderFactory.java
index ec0d412..e92b8e0 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/factory/ISeriesReaderFactory.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/factory/ISeriesReaderFactory.java
@@ -18,66 +18,73 @@
  */
 package org.apache.iotdb.db.query.factory;
 
-import java.io.IOException;
-import java.util.List;
 import org.apache.iotdb.db.engine.filenodeV2.TsFileResourceV2;
 import org.apache.iotdb.db.exception.FileNodeManagerException;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.reader.IPointReader;
-import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.db.query.reader.IReaderByTimeStamp;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 
+import java.io.IOException;
+import java.util.List;
+
 /**
- * This class defines the interface of construct readers for different data source. Note that, job
+ * This class defines the interface of constructing readers for different data source. Note that, job
  * id equals -1 meant that this method is used for IoTDB merge process, it's no need to maintain the
  * opened file stream.
  */
 public interface ISeriesReaderFactory {
 
 
-  /**
-   * This method is used to read all unsequence data for IoTDB request, such as query, aggregation
+  /*
+   * This method is used to read unsequence data for IoTDB request, such as query, aggregation
    * and groupby request.
+   *
+   * @param seriesPath the path of the time series to be read
+   * @param unseqResources unsequence data in the seriesPath
+   * @param context query context
+   * @param filter It can a combination of time and value filter or null.
+   * @return unsequence series reader
    */
-  IPointReader createUnSeqReader(Path seriesPath, List<TsFileResourceV2> unSeqResources,
-      QueryContext context,
-      Filter filter) throws IOException;
+  IPointReader createUnseqSeriesReader(Path seriesPath, List<TsFileResourceV2> unseqResources,
+                                       QueryContext context,
+                                       Filter filter) throws IOException;
 
 
   /**
-   * construct ByTimestampReader, including sequential data and unsequential data.
+   * construct a list of SeriesReaderByTimestamp, including sequence data and unsequence data.
    *
-   * @param paths selected series path
+   * @param paths   the paths of the time series to be read
    * @param context query context
-   * @return the list of EngineReaderByTimeStamp
+   * @return a list of IReaderByTimeStamp
    */
-  List<EngineReaderByTimeStamp> createByTimestampReaders(List<Path> paths,
-      QueryContext context) throws FileNodeManagerException, IOException;
+  List<IReaderByTimeStamp> createSeriesReadersByTimestamp(List<Path> paths,
+                                                          QueryContext context) throws FileNodeManagerException, IOException;
 
   /**
-   * construct IPointReader with <br>only time filter or no filter</br>, including sequential data
-   * and unsequential data. This reader won't filter the result of merged sequential data and
-   * unsequential data reader.
+   * construct IPointReader with <b>only time filter or no filter</b>, including sequence data
+   * and unsequence data. This reader won't filter the result of merged sequence data and
+   * unsequence data reader.
    *
-   * @param path selected series path
+   * @param path       the path of the time series to be read
    * @param timeFilter time filter or null
-   * @param context query context
+   * @param context    query context
    * @return data reader including seq and unseq data source.
    */
-  IPointReader createReaderWithOptGlobalTimeFilter(Path path, Filter timeFilter,
-      QueryContext context) throws FileNodeManagerException, IOException;
+  IPointReader createSeriesReaderWithoutValueFilter(Path path, Filter timeFilter,
+                                                    QueryContext context) throws FileNodeManagerException, IOException;
 
   /**
-   * construct IPointReader with <br>value filter</br>, include sequential data and unsequential
-   * data. This reader will filter the result of merged sequential data and unsequential data
-   * reader, so if only has time filter please call createReaderWithOptGlobalTimeFilter().
+   * construct IPointReader with <b>value filter</b>, include sequence data and unsequence
+   * data. This reader will filter the result of merged sequence data and unsequence data
+   * reader, so if only time filter is used, please call createSeriesReaderWithoutValueFilter().
    *
-   * @param path selected series path
-   * @param filter time filter or null
+   * @param path    selected series path
+   * @param filter  filter that contains at least a value filter
    * @param context query context
-   * @return data reader including seq and unseq data source.
+   * @return data reader including sequence and unsequence data source.
    */
-  IPointReader createReaderWithValueFilter(Path path, Filter filter, QueryContext context)
-      throws FileNodeManagerException, IOException;
+  IPointReader createSeriesReaderWithValueFilter(Path path, Filter filter, QueryContext context)
+          throws FileNodeManagerException, IOException;
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactoryImpl.java b/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactoryImpl.java
index d5f7145..b6c1949 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactoryImpl.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactoryImpl.java
@@ -18,9 +18,6 @@
  */
 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.filenodeV2.TsFileResourceV2;
 import org.apache.iotdb.db.engine.modification.Modification;
 import org.apache.iotdb.db.engine.querycontext.QueryDataSourceV2;
@@ -29,18 +26,18 @@ import org.apache.iotdb.db.exception.ProcessorException;
 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.AllDataReaderWithOptGlobalTimeFilter;
-import org.apache.iotdb.db.query.reader.AllDataReaderWithValueFilter;
 import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.SeriesReaderWithValueFilter;
+import org.apache.iotdb.db.query.reader.SeriesReaderWithoutValueFilter;
 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.SequenceDataReaderByTimestampV2;
-import org.apache.iotdb.db.query.reader.sequence.SequenceDataReaderV2;
-import org.apache.iotdb.db.query.reader.unsequence.EngineChunkReader;
-import org.apache.iotdb.db.query.reader.unsequence.EngineChunkReaderByTimestamp;
+import org.apache.iotdb.db.query.reader.IReaderByTimeStamp;
+import org.apache.iotdb.db.query.reader.merge.SeriesReaderByTimestamp;
+import org.apache.iotdb.db.query.reader.unsequence.UnsequenceSeriesReader;
+import org.apache.iotdb.db.query.reader.sequence.SequenceSeriesReader;
+import org.apache.iotdb.db.query.reader.sequence.SequenceSeriesReaderByTimestamp;
+import org.apache.iotdb.db.query.reader.unsequence.DiskChunkReader;
+import org.apache.iotdb.db.query.reader.unsequence.DiskChunkReaderByTimestamp;
 import org.apache.iotdb.db.utils.QueryUtils;
 import org.apache.iotdb.tsfile.common.constant.StatisticConstant;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
@@ -58,6 +55,10 @@ import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReaderWithoutFilter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
 public class SeriesReaderFactoryImpl implements ISeriesReaderFactory {
 
   private static final Logger logger = LoggerFactory.getLogger(SeriesReaderFactoryImpl.class);
@@ -70,18 +71,18 @@ public class SeriesReaderFactoryImpl implements ISeriesReaderFactory {
   }
 
   @Override
-  public IPointReader createUnSeqReader(Path seriesPath, List<TsFileResourceV2> unSeqResources,
-      QueryContext context,
-      Filter filter) throws IOException {
-    PriorityMergeReader unSeqMergeReader = new PriorityMergeReader();
+  public IPointReader createUnseqSeriesReader(Path seriesPath, List<TsFileResourceV2> unseqResources,
+                                              QueryContext context,
+                                              Filter filter) throws IOException {
+    UnsequenceSeriesReader unseqMergeReader = new UnsequenceSeriesReader();
 
     int priorityValue = 1;
 
-    for (TsFileResourceV2 tsFileResourceV2 : unSeqResources) {
+    for (TsFileResourceV2 tsFileResourceV2 : unseqResources) {
 
       // store only one opened file stream into manager, to avoid too many opened files
       TsFileSequenceReader tsFileReader = FileReaderManager.getInstance()
-          .get(tsFileResourceV2.getFile().getPath(), tsFileResourceV2.isClosed());
+              .get(tsFileResourceV2.getFile().getPath(), tsFileResourceV2.isClosed());
 
       // get modified chunk metadatas
       List<ChunkMetaData> metaDataList;
@@ -90,8 +91,8 @@ public class SeriesReaderFactoryImpl implements ISeriesReaderFactory {
         metaDataList = metadataQuerier.getChunkMetaDataList(seriesPath);
         // mod
         List<Modification> pathModifications = context
-            .getPathModifications(tsFileResourceV2.getModFile(),
-                seriesPath.getFullPath());
+                .getPathModifications(tsFileResourceV2.getModFile(),
+                        seriesPath.getFullPath());
         if (!pathModifications.isEmpty()) {
           QueryUtils.modifyChunkMetaData(metaDataList, pathModifications);
         }
@@ -100,15 +101,15 @@ public class SeriesReaderFactoryImpl implements ISeriesReaderFactory {
       }
 
       // add readers for chunks
-      // TODO 下面这段对chunkmetadata过滤考虑复用
+      // TODO future advancement: decrease the duplicated code
       ChunkLoaderImpl chunkLoader = new ChunkLoaderImpl(tsFileReader);
       for (ChunkMetaData chunkMetaData : metaDataList) {
 
         DigestForFilter digest = new DigestForFilter(chunkMetaData.getStartTime(),
-            chunkMetaData.getEndTime(),
-            chunkMetaData.getDigest().getStatistics().get(StatisticConstant.MIN_VALUE),
-            chunkMetaData.getDigest().getStatistics().get(StatisticConstant.MAX_VALUE),
-            chunkMetaData.getTsDataType());
+                chunkMetaData.getEndTime(),
+                chunkMetaData.getDigest().getStatistics().get(StatisticConstant.MIN_VALUE),
+                chunkMetaData.getDigest().getStatistics().get(StatisticConstant.MAX_VALUE),
+                chunkMetaData.getTsDataType());
 
         if (filter != null && !filter.satisfy(digest)) {
           continue;
@@ -116,35 +117,35 @@ public class SeriesReaderFactoryImpl implements ISeriesReaderFactory {
 
         Chunk chunk = chunkLoader.getChunk(chunkMetaData);
         ChunkReader chunkReader = filter != null ? new ChunkReaderWithFilter(chunk, filter)
-            : new ChunkReaderWithoutFilter(chunk);
+                : new ChunkReaderWithoutFilter(chunk);
 
-        unSeqMergeReader.addReaderWithPriority(new EngineChunkReader(chunkReader), priorityValue);
+        unseqMergeReader.addReaderWithPriority(new DiskChunkReader(chunkReader), priorityValue);
 
         priorityValue++;
       }
 
       // add reader for MemTable
       if (!tsFileResourceV2.isClosed()) {
-        unSeqMergeReader.addReaderWithPriority(
-            new MemChunkReader(tsFileResourceV2.getReadOnlyMemChunk(), filter), priorityValue++);
+        unseqMergeReader.addReaderWithPriority(
+                new MemChunkReader(tsFileResourceV2.getReadOnlyMemChunk(), filter), priorityValue++);
       }
     }
 
     // TODO add external sort when needed
-    return unSeqMergeReader;
+    return unseqMergeReader;
   }
 
-  private PriorityMergeReaderByTimestamp createUnSeqByTimestampReader(Path seriesPath,
-      List<TsFileResourceV2> unSeqResources, QueryContext context) throws IOException {
-    PriorityMergeReaderByTimestamp unSeqMergeReader = new PriorityMergeReaderByTimestamp();
+  private SeriesReaderByTimestamp createUnseqSeriesReaderByTimestamp(Path seriesPath,
+                                                                     List<TsFileResourceV2> unseqResources, QueryContext context) throws IOException {
+    SeriesReaderByTimestamp unseqMergeReader = new SeriesReaderByTimestamp();
 
     int priorityValue = 1;
 
-    for (TsFileResourceV2 tsFileResourceV2 : unSeqResources) {
+    for (TsFileResourceV2 tsFileResourceV2 : unseqResources) {
 
       // store only one opened file stream into manager, to avoid too many opened files
       TsFileSequenceReader tsFileReader = FileReaderManager.getInstance()
-          .get(tsFileResourceV2.getFile().getPath(), tsFileResourceV2.isClosed());
+              .get(tsFileResourceV2.getFile().getPath(), tsFileResourceV2.isClosed());
 
       List<ChunkMetaData> metaDataList;
       if (tsFileResourceV2.isClosed()) {
@@ -152,8 +153,8 @@ public class SeriesReaderFactoryImpl implements ISeriesReaderFactory {
         metaDataList = metadataQuerier.getChunkMetaDataList(seriesPath);
         // mod
         List<Modification> pathModifications = context
-            .getPathModifications(tsFileResourceV2.getModFile(),
-                seriesPath.getFullPath());
+                .getPathModifications(tsFileResourceV2.getModFile(),
+                        seriesPath.getFullPath());
         if (!pathModifications.isEmpty()) {
           QueryUtils.modifyChunkMetaData(metaDataList, pathModifications);
         }
@@ -168,51 +169,50 @@ public class SeriesReaderFactoryImpl implements ISeriesReaderFactory {
         Chunk chunk = chunkLoader.getChunk(chunkMetaData);
         ChunkReaderByTimestamp chunkReader = new ChunkReaderByTimestamp(chunk);
 
-        unSeqMergeReader.addReaderWithPriority(new EngineChunkReaderByTimestamp(chunkReader),
-            priorityValue);
+        unseqMergeReader.addReaderWithPriority(new DiskChunkReaderByTimestamp(chunkReader),
+                priorityValue);
 
         priorityValue++;
       }
 
       // add reader for MemTable
       if (!tsFileResourceV2.isClosed()) {
-        unSeqMergeReader.addReaderWithPriority(
-            new MemChunkReaderByTimestamp(tsFileResourceV2.getReadOnlyMemChunk()), priorityValue++);
+        unseqMergeReader.addReaderWithPriority(
+                new MemChunkReaderByTimestamp(tsFileResourceV2.getReadOnlyMemChunk()), priorityValue++);
       }
     }
 
     // TODO add external sort when needed
-    return unSeqMergeReader;
+    return unseqMergeReader;
   }
 
   @Override
-  public List<EngineReaderByTimeStamp> createByTimestampReaders(List<Path> paths,
-      QueryContext context) throws FileNodeManagerException, IOException {
-    List<EngineReaderByTimeStamp> readersOfSelectedSeries = new ArrayList<>();
+  public List<IReaderByTimeStamp> createSeriesReadersByTimestamp(List<Path> paths,
+                                                                 QueryContext context) throws FileNodeManagerException, IOException {
+    List<IReaderByTimeStamp> readersOfSelectedSeries = new ArrayList<>();
 
     for (Path path : paths) {
 
       QueryDataSourceV2 queryDataSource = null;
       try {
         queryDataSource = QueryResourceManager.getInstance()
-            .getQueryDataSourceV2(path,
-                context);
+                .getQueryDataSourceV2(path,
+                        context);
       } catch (ProcessorException e) {
         throw new FileNodeManagerException(e);
       }
 
-      PriorityMergeReaderByTimestamp mergeReaderByTimestamp = new PriorityMergeReaderByTimestamp();
+      SeriesReaderByTimestamp mergeReaderByTimestamp = new SeriesReaderByTimestamp();
 
       // reader for sequence data
-      SequenceDataReaderByTimestampV2 tsFilesReader = new SequenceDataReaderByTimestampV2(path,
-          queryDataSource.getSeqResources(), context);
+      SequenceSeriesReaderByTimestamp tsFilesReader = new SequenceSeriesReaderByTimestamp(path,
+              queryDataSource.getSeqResources(), context);
       mergeReaderByTimestamp.addReaderWithPriority(tsFilesReader, 1);
 
-      // reader for unSequence data
-      //TODO add create unseq reader
-      PriorityMergeReaderByTimestamp unSeqMergeReader = createUnSeqByTimestampReader(path,
-          queryDataSource.getUnseqResources(), context);
-      mergeReaderByTimestamp.addReaderWithPriority(unSeqMergeReader, 2);
+      // reader for unsequence data
+      SeriesReaderByTimestamp unseqMergeReader = createUnseqSeriesReaderByTimestamp(path,
+              queryDataSource.getUnseqResources(), context);
+      mergeReaderByTimestamp.addReaderWithPriority(unseqMergeReader, 2);
 
       readersOfSelectedSeries.add(mergeReaderByTimestamp);
     }
@@ -221,62 +221,62 @@ public class SeriesReaderFactoryImpl implements ISeriesReaderFactory {
   }
 
   @Override
-  public IPointReader createReaderWithOptGlobalTimeFilter(Path path, Filter timeFilter,
-      QueryContext context)
-      throws FileNodeManagerException, IOException {
+  public IPointReader createSeriesReaderWithoutValueFilter(Path path, Filter timeFilter,
+                                                           QueryContext context)
+          throws FileNodeManagerException, IOException {
     QueryDataSourceV2 queryDataSource = null;
     try {
       queryDataSource = QueryResourceManager.getInstance()
-          .getQueryDataSourceV2(path, context);
+              .getQueryDataSourceV2(path, context);
     } catch (ProcessorException e) {
       throw new FileNodeManagerException(e);
     }
 
     // sequence reader for one sealed tsfile
-    SequenceDataReaderV2 tsFilesReader;
+    SequenceSeriesReader tsFilesReader;
 
-    tsFilesReader = new SequenceDataReaderV2(queryDataSource.getSeriesPath(),
-        queryDataSource.getSeqResources(),
-        timeFilter, context);
+    tsFilesReader = new SequenceSeriesReader(queryDataSource.getSeriesPath(),
+            queryDataSource.getSeqResources(),
+            timeFilter, context);
 
-    // unseq reader for all chunk groups in unSeqFile
-    IPointReader unSeqMergeReader = null;
-    unSeqMergeReader = createUnSeqReader(path, queryDataSource.getUnseqResources(), context,
-        timeFilter);
+    // unseq reader for all chunk groups in unseqFile
+    IPointReader unseqMergeReader = null;
+    unseqMergeReader = createUnseqSeriesReader(path, queryDataSource.getUnseqResources(), context,
+            timeFilter);
 
     if (!tsFilesReader.hasNext()) {
       //only have unsequence data.
-      return unSeqMergeReader;
+      return unseqMergeReader;
     } else {
       //merge sequence data with unsequence data.
-      return new AllDataReaderWithOptGlobalTimeFilter(tsFilesReader, unSeqMergeReader);
+      return new SeriesReaderWithoutValueFilter(tsFilesReader, unseqMergeReader);
     }
   }
 
   @Override
-  public IPointReader createReaderWithValueFilter(Path path, Filter filter, QueryContext context)
-      throws FileNodeManagerException, IOException {
+  public IPointReader createSeriesReaderWithValueFilter(Path path, Filter filter, QueryContext context)
+          throws FileNodeManagerException, IOException {
     QueryDataSourceV2 queryDataSource = null;
     try {
       queryDataSource = QueryResourceManager.getInstance()
-          .getQueryDataSourceV2(path, context);
+              .getQueryDataSourceV2(path, context);
     } catch (ProcessorException e) {
       throw new FileNodeManagerException(e);
     }
 
     // sequence reader for one sealed tsfile
-    SequenceDataReaderV2 tsFilesReader;
+    SequenceSeriesReader tsFilesReader;
 
-    tsFilesReader = new SequenceDataReaderV2(queryDataSource.getSeriesPath(),
-        queryDataSource.getSeqResources(),
-        filter, context);
+    tsFilesReader = new SequenceSeriesReader(queryDataSource.getSeriesPath(),
+            queryDataSource.getSeqResources(),
+            filter, context);
 
-    // unseq reader for all chunk groups in unSeqFile. Filter for unSeqMergeReader is null, because
+    // unseq reader for all chunk groups in unseqFile. Filter for unseqMergeReader is null, because
     // we won't push down filter in unsequence data source.
-    IPointReader unSeqMergeReader;
-    unSeqMergeReader = createUnSeqReader(path, queryDataSource.getUnseqResources(), context, null);
+    IPointReader unseqMergeReader;
+    unseqMergeReader = createUnseqSeriesReader(path, queryDataSource.getUnseqResources(), context, null);
 
-    return new AllDataReaderWithValueFilter(tsFilesReader, unSeqMergeReader, filter);
+    return new SeriesReaderWithValueFilter(tsFilesReader, unseqMergeReader, filter);
   }
 
   private static class SeriesReaderFactoryHelper {
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 b5b2283..f6a0d80 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
@@ -54,7 +54,7 @@ public abstract class IFill {
       throws IOException, FileNodeManagerException {
     Filter timeFilter = constructFilter(beforeRange);
     allDataReader = SeriesReaderFactoryImpl.getInstance()
-        .createReaderWithOptGlobalTimeFilter(path, timeFilter, context);
+        .createSeriesReaderWithoutValueFilter(path, timeFilter, context);
   }
 
   public abstract IPointReader getFillResult() throws IOException;
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IPointReader.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IPointReader.java
index 2970e1c..3e61008 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IPointReader.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IPointReader.java
@@ -22,7 +22,13 @@ package org.apache.iotdb.db.query.reader;
 import java.io.IOException;
 import org.apache.iotdb.db.utils.TimeValuePair;
 
-public interface IPointReader extends IReader {
+public interface IPointReader{
+
+  boolean hasNext() throws IOException;
+
+  TimeValuePair next() throws IOException;
 
   TimeValuePair current() throws IOException;
+
+  void close() 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
deleted file mode 100644
index d0603f5..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IReader.java
+++ /dev/null
@@ -1,36 +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;
-
-import java.io.IOException;
-import org.apache.iotdb.db.utils.TimeValuePair;
-
-/**
- * Vital cloneList 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 {
-
-  boolean hasNext() throws IOException;
-
-  TimeValuePair next() throws IOException;
-
-  void close() throws IOException;
-}
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/IReaderByTimeStamp.java
similarity index 93%
rename from iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java
rename to iotdb/src/main/java/org/apache/iotdb/db/query/reader/IReaderByTimeStamp.java
index 93daff8..b72dd93 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/IReaderByTimeStamp.java
@@ -16,11 +16,11 @@
  * 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;
 
-public interface EngineReaderByTimeStamp {
+public interface IReaderByTimeStamp {
 
   /**
    * Given a timestamp, the reader is supposed to return the corresponding value in the timestamp.
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/AllDataReaderWithValueFilter.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/SeriesReaderWithValueFilter.java
similarity index 87%
rename from iotdb/src/main/java/org/apache/iotdb/db/query/reader/AllDataReaderWithValueFilter.java
rename to iotdb/src/main/java/org/apache/iotdb/db/query/reader/SeriesReaderWithValueFilter.java
index 45bc9f6..7931f89 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/AllDataReaderWithValueFilter.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/SeriesReaderWithValueFilter.java
@@ -19,14 +19,15 @@
 
 package org.apache.iotdb.db.query.reader;
 
-import java.io.IOException;
 import org.apache.iotdb.db.utils.TimeValuePair;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 
+import java.io.IOException;
+
 /**
  * A value filter reader for read data source, including sequence data and unsequence data.
  */
-public class AllDataReaderWithValueFilter extends AllDataReaderWithOptGlobalTimeFilter {
+public class SeriesReaderWithValueFilter extends SeriesReaderWithoutValueFilter {
 
   private Filter filter;
   private boolean hasCachedValue;
@@ -35,9 +36,9 @@ public class AllDataReaderWithValueFilter extends AllDataReaderWithOptGlobalTime
   /**
    * merge sequence reader, unsequence reader.
    */
-  public AllDataReaderWithValueFilter(IBatchReader batchReader, IPointReader pointReader,
-      Filter filter) {
-    super(batchReader, pointReader);
+  public SeriesReaderWithValueFilter(IBatchReader seqSeriesReader, IPointReader unseqSeriesReader,
+                                     Filter filter) {
+    super(seqSeriesReader, unseqSeriesReader);
     this.filter = filter;
   }
 
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/AllDataReaderWithOptGlobalTimeFilter.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/SeriesReaderWithoutValueFilter.java
similarity index 75%
rename from iotdb/src/main/java/org/apache/iotdb/db/query/reader/AllDataReaderWithOptGlobalTimeFilter.java
rename to iotdb/src/main/java/org/apache/iotdb/db/query/reader/SeriesReaderWithoutValueFilter.java
index 10b28fb..dd76c2f 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/AllDataReaderWithOptGlobalTimeFilter.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/SeriesReaderWithoutValueFilter.java
@@ -19,19 +19,20 @@
 
 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;
 
+import java.io.IOException;
+
 /**
  * It used to merge sequence data and unsequence data with <br>only time filter or no
  * filter.</br>
  */
-public class AllDataReaderWithOptGlobalTimeFilter implements IPointReader {
+public class SeriesReaderWithoutValueFilter implements IPointReader {
 
-  private IBatchReader batchReader;
-  private IPointReader pointReader;
+  private IBatchReader seqSeriesReader;
+  private IPointReader unseqSeriesReader;
 
   private boolean hasCachedBatchData;
   private BatchData batchData;
@@ -39,9 +40,9 @@ public class AllDataReaderWithOptGlobalTimeFilter implements IPointReader {
   /**
    * merge sequence reader, unsequence reader.
    */
-  public AllDataReaderWithOptGlobalTimeFilter(IBatchReader batchReader, IPointReader pointReader) {
-    this.batchReader = batchReader;
-    this.pointReader = pointReader;
+  public SeriesReaderWithoutValueFilter(IBatchReader seqSeriesReader, IPointReader unseqSeriesReader) {
+    this.seqSeriesReader = seqSeriesReader;
+    this.unseqSeriesReader = unseqSeriesReader;
 
     this.hasCachedBatchData = false;
   }
@@ -52,7 +53,7 @@ public class AllDataReaderWithOptGlobalTimeFilter implements IPointReader {
       return true;
     }
     // has value in pointReader
-    return pointReader != null && pointReader.hasNext();
+    return unseqSeriesReader != null && unseqSeriesReader.hasNext();
   }
 
   @Override
@@ -60,7 +61,7 @@ public class AllDataReaderWithOptGlobalTimeFilter implements IPointReader {
 
     // has next in both batch reader and point reader
     if (hasNextInBothReader()) {
-      long timeInPointReader = pointReader.current().getTimestamp();
+      long timeInPointReader = unseqSeriesReader.current().getTimestamp();
       long timeInBatchData = batchData.currentTime();
       if (timeInPointReader > timeInBatchData) {
         TimeValuePair timeValuePair = TimeValuePairUtils.getCurrentTimeValuePair(batchData);
@@ -68,9 +69,9 @@ public class AllDataReaderWithOptGlobalTimeFilter implements IPointReader {
         return timeValuePair;
       } else if (timeInPointReader == timeInBatchData) {
         batchData.next();
-        return pointReader.next();
+        return unseqSeriesReader.next();
       } else {
-        return pointReader.next();
+        return unseqSeriesReader.next();
       }
     }
 
@@ -82,8 +83,8 @@ public class AllDataReaderWithOptGlobalTimeFilter implements IPointReader {
     }
 
     // only has next in point reader
-    if (pointReader != null && pointReader.hasNext()) {
-      return pointReader.next();
+    if (unseqSeriesReader != null && unseqSeriesReader.hasNext()) {
+      return unseqSeriesReader.next();
     }
     return null;
   }
@@ -95,7 +96,7 @@ public class AllDataReaderWithOptGlobalTimeFilter implements IPointReader {
     if (!hasNextInBatchDataOrBatchReader()) {
       return false;
     }
-    return pointReader != null && pointReader.hasNext();
+    return unseqSeriesReader != null && unseqSeriesReader.hasNext();
   }
 
   /**
@@ -110,8 +111,8 @@ public class AllDataReaderWithOptGlobalTimeFilter implements IPointReader {
     }
 
     // has value in batchReader
-    while (batchReader != null && batchReader.hasNext()) {
-      batchData = batchReader.nextBatch();
+    while (seqSeriesReader != null && seqSeriesReader.hasNext()) {
+      batchData = seqSeriesReader.nextBatch();
       if (batchData.hasNext()) {
         hasCachedBatchData = true;
         return true;
@@ -122,12 +123,12 @@ public class AllDataReaderWithOptGlobalTimeFilter implements IPointReader {
 
   @Override
   public TimeValuePair current() throws IOException {
-    throw new IOException("current() in AllDataReaderWithOptGlobalTimeFilter is an empty method.");
+    throw new IOException("current() in SeriesReaderWithoutValueFilter is an empty method.");
   }
 
   @Override
   public void close() throws IOException {
-    batchReader.close();
-    pointReader.close();
+    seqSeriesReader.close();
+    unseqSeriesReader.close();
   }
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReader.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReader.java
index 8c3e9cf..1efc08e 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReader.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReader.java
@@ -19,10 +19,8 @@
 
 package org.apache.iotdb.db.query.reader.mem;
 
-import java.util.Iterator;
 import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
 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;
@@ -30,7 +28,9 @@ 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 MemChunkReader implements IPointReader, IBatchReader, IAggregateReader {
+import java.util.Iterator;
+
+public class MemChunkReader implements IPointReader, IAggregateReader {
 
   private Iterator<TimeValuePair> timeValuePairIterator;
   private Filter filter;
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 9827f5e..e88ec3c 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
@@ -21,10 +21,10 @@ 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.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.db.query.reader.IReaderByTimeStamp;
 import org.apache.iotdb.db.utils.TimeValuePair;
 
-public class MemChunkReaderByTimestamp implements EngineReaderByTimeStamp {
+public class MemChunkReaderByTimestamp implements IReaderByTimeStamp {
 
   private Iterator<TimeValuePair> timeValuePairIterator;
   private boolean hasCachedTimeValuePair;
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/SeriesReaderByTimestamp.java
similarity index 76%
rename from iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java
rename to iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/SeriesReaderByTimestamp.java
index 0f42e87..faf8f1c 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/SeriesReaderByTimestamp.java
@@ -19,6 +19,8 @@
 
 package org.apache.iotdb.db.query.reader.merge;
 
+import org.apache.iotdb.db.query.reader.IReaderByTimeStamp;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -26,20 +28,21 @@ import java.util.List;
 /**
  * <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
+ * priority. (1) merge multiple chunk group readers in the unsequence file, e.g.,
+ * SeriesReaderFactoryImpl.createUnseqSeriesReaderByTimestamp; (2)merge sequence reader and
+ * unsequence reader, e.g., SeriesReaderFactoryImpl.createSeriesReaderByTimestamp.
  * </p>
  */
-public class PriorityMergeReaderByTimestamp implements EngineReaderByTimeStamp {
+public class SeriesReaderByTimestamp implements IReaderByTimeStamp {
 
-  private List<EngineReaderByTimeStamp> readerList = new ArrayList<>();
+  private List<IReaderByTimeStamp> readerList = new ArrayList<>();
   private List<Integer> priorityList = new ArrayList<>();
 
   /**
-   * This function doesn't sort reader by priority. So you have to call this function in order of
+   * NOTE: 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) {
+  public void addReaderWithPriority(IReaderByTimeStamp reader, int priority) {
     readerList.add(reader);
     priorityList.add(priority);
   }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/IterateReader.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/IterateReader.java
index d15e3a4..2e345ac 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/IterateReader.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/IterateReader.java
@@ -18,26 +18,27 @@
  */
 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.query.reader.IAggregateReader;
 import org.apache.iotdb.db.query.reader.IBatchReader;
 import org.apache.iotdb.tsfile.file.header.PageHeader;
 import org.apache.iotdb.tsfile.read.common.BatchData;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
 /**
  * get data sequentially from the reader list.
  */
 public class IterateReader implements IAggregateReader {
 
-  protected List<IAggregateReader> seriesReaders;
+  protected List<IAggregateReader> seqResourceSeriesReaderList;
   protected boolean curReaderInitialized;
   protected int nextSeriesReaderIndex;
   protected IAggregateReader currentSeriesReader;
 
   public IterateReader() {
-    this.seriesReaders = new ArrayList<>();
+    this.seqResourceSeriesReaderList = new ArrayList<>();
     this.curReaderInitialized = false;
     this.nextSeriesReaderIndex = 0;
   }
@@ -51,8 +52,8 @@ public class IterateReader implements IAggregateReader {
       curReaderInitialized = false;
     }
 
-    while (nextSeriesReaderIndex < seriesReaders.size()) {
-      currentSeriesReader = seriesReaders.get(nextSeriesReaderIndex++);
+    while (nextSeriesReaderIndex < seqResourceSeriesReaderList.size()) {
+      currentSeriesReader = seqResourceSeriesReaderList.get(nextSeriesReaderIndex++);
       if (currentSeriesReader.hasNext()) {
         curReaderInitialized = true;
         return true;
@@ -63,7 +64,7 @@ public class IterateReader implements IAggregateReader {
 
   @Override
   public void close() throws IOException {
-    for (IBatchReader seriesReader : seriesReaders) {
+    for (IBatchReader seriesReader : seqResourceSeriesReaderList) {
       seriesReader.close();
     }
   }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderV2.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceSeriesReader.java
similarity index 77%
rename from iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderV2.java
rename to iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceSeriesReader.java
index 1ba307e..9c689eb 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderV2.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceSeriesReader.java
@@ -18,19 +18,15 @@
  */
 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.filenodeV2.TsFileResourceV2;
 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.IAggregateReader;
-import org.apache.iotdb.db.query.reader.adapter.FileSeriesReaderAdapter;
+import org.apache.iotdb.db.query.reader.sequence.adapter.FileSeriesReaderAdapter;
 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.BatchData;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.read.controller.ChunkLoader;
 import org.apache.iotdb.tsfile.read.controller.ChunkLoaderImpl;
@@ -40,11 +36,15 @@ 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;
 
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
 /**
- * batch reader of data in: 1) sealed tsfile. 2) unsealed tsfile, which include data in disk of
+ * batch reader of data in: 1) sealed tsfile. 2) unsealed tsfile, which include data on disk of
  * unsealed file and in memtables that will be flushing to unsealed tsfile.
  */
-public class SequenceDataReaderV2 extends IterateReader {
+public class SequenceSeriesReader extends IterateReader {
 
   private Path seriesPath;
   /**
@@ -56,14 +56,14 @@ public class SequenceDataReaderV2 extends IterateReader {
   /**
    * init with globalSortedSeriesDataSource, filter, context and isReverse.
    *
-   * @param seriesPath data source
+   * @param seriesPath   data source
    * @param seqResources null if no filter
-   * @param context query context
-   * @param isReverse true-traverse chunks from behind forward, false-traverse chunks from front to
-   * back.
+   * @param context      query context
+   * @param isReverse    true-traverse chunks from behind forward, false-traverse chunks from front to
+   *                     back.
    */
-  public SequenceDataReaderV2(Path seriesPath, List<TsFileResourceV2> seqResources,
-      Filter timeFilter, QueryContext context, boolean isReverse) throws IOException {
+  public SequenceSeriesReader(Path seriesPath, List<TsFileResourceV2> seqResources,
+                              Filter timeFilter, QueryContext context, boolean isReverse) throws IOException {
     super();
     this.seriesPath = seriesPath;
     this.enableReverse = isReverse;
@@ -72,10 +72,10 @@ public class SequenceDataReaderV2 extends IterateReader {
     }
     for (TsFileResourceV2 tsFileResource : seqResources) {
       if (tsFileResource.isClosed()) {
-        constructSealedTsFileReader(tsFileResource, timeFilter, context, seriesReaders);
+        constructSealedTsFileReader(tsFileResource, timeFilter, context, seqResourceSeriesReaderList);
       } else {
-        seriesReaders.add(
-            new UnSealedTsFileReaderV2(tsFileResource, timeFilter, enableReverse));
+        seqResourceSeriesReaderList.add(
+                new UnSealedTsFileReader(tsFileResource, timeFilter, enableReverse));
       }
     }
   }
@@ -83,17 +83,17 @@ public class SequenceDataReaderV2 extends IterateReader {
   /**
    * traverse chunks from front to back.
    */
-  public SequenceDataReaderV2(Path seriesPath, List<TsFileResourceV2> seqResources,
-      Filter timeFilter, QueryContext context) throws IOException {
+  public SequenceSeriesReader(Path seriesPath, List<TsFileResourceV2> seqResources,
+                              Filter timeFilter, QueryContext context) throws IOException {
     this(seriesPath, seqResources, timeFilter, context, false);
   }
 
   private void constructSealedTsFileReader(TsFileResourceV2 tsFileResource, Filter filter,
-      QueryContext context, List<IAggregateReader> readerList)
-      throws IOException {
+                                           QueryContext context, List<IAggregateReader> seqResourceSeriesReaderList)
+          throws IOException {
     if (singleTsFileSatisfied(tsFileResource, filter)) {
-      readerList.add(
-          new FileSeriesReaderAdapter(initSealedTsFileReader(tsFileResource, filter, context)));
+      seqResourceSeriesReaderList.add(
+              new FileSeriesReaderAdapter(initSealedTsFileReader(tsFileResource, filter, context)));
     }
 
   }
@@ -116,18 +116,18 @@ public class SequenceDataReaderV2 extends IterateReader {
   }
 
   private FileSeriesReader initSealedTsFileReader(TsFileResourceV2 tsfile, Filter filter,
-      QueryContext context)
-      throws IOException {
+                                                  QueryContext context)
+          throws IOException {
 
     // to avoid too many opened files
     TsFileSequenceReader tsFileReader = FileReaderManager.getInstance()
-        .get(tsfile.getFile().getPath(), true);
+            .get(tsfile.getFile().getPath(), true);
 
     MetadataQuerierByFileImpl metadataQuerier = new MetadataQuerierByFileImpl(tsFileReader);
     List<ChunkMetaData> metaDataList = metadataQuerier.getChunkMetaDataList(seriesPath);
 
     List<Modification> pathModifications = context.getPathModifications(tsfile.getModFile(),
-        seriesPath.getFullPath());
+            seriesPath.getFullPath());
     if (!pathModifications.isEmpty()) {
       QueryUtils.modifyChunkMetaData(metaDataList, pathModifications);
     }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestampV2.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceSeriesReaderByTimestamp.java
similarity index 83%
rename from iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestampV2.java
rename to iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceSeriesReaderByTimestamp.java
index e38728c..ef3dcfd 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestampV2.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceSeriesReaderByTimestamp.java
@@ -19,14 +19,12 @@
 
 package org.apache.iotdb.db.query.reader.sequence;
 
-import java.io.IOException;
-import java.util.List;
 import org.apache.iotdb.db.engine.filenodeV2.TsFileResourceV2;
 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.adapter.SeriesReaderByTimestampAdapter;
-import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.db.query.reader.IReaderByTimeStamp;
+import org.apache.iotdb.db.query.reader.sequence.adapter.SeriesReaderByTimestampAdapter;
 import org.apache.iotdb.db.utils.QueryUtils;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
 import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
@@ -34,26 +32,29 @@ 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;
+import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderByTimestamp;
+
+import java.io.IOException;
+import java.util.List;
 
 /**
- * EngineReaderByTimeStamp of data in: 1) sealed tsfile. 2) unsealed tsfile, which include data in disk of
+ * IReaderByTimeStamp of data in: 1) sealed tsfile. 2) unsealed tsfile, which include data on disk of
  * unsealed file and in memtables that will be flushing to unsealed tsfile.
  */
-public class SequenceDataReaderByTimestampV2 implements EngineReaderByTimeStamp {
+public class SequenceSeriesReaderByTimestamp implements IReaderByTimeStamp {
 
   protected Path seriesPath;
   private List<TsFileResourceV2> tsFileResourceV2List;
   private int nextIntervalFileIndex;
-  protected EngineReaderByTimeStamp seriesReader;
+  protected IReaderByTimeStamp seriesReader;
   private QueryContext context;
 
   /**
    * init with seriesPath and tsfile list which include sealed tsfile and unseadled tsfile.
    */
-  public SequenceDataReaderByTimestampV2(Path seriesPath,
-      List<TsFileResourceV2> tsFileResourceV2List,
-      QueryContext context) {
+  public SequenceSeriesReaderByTimestamp(Path seriesPath,
+                                         List<TsFileResourceV2> tsFileResourceV2List,
+                                         QueryContext context) {
     this.seriesPath = seriesPath;
     this.tsFileResourceV2List = tsFileResourceV2List;
     this.nextIntervalFileIndex = 0;
@@ -128,28 +129,28 @@ public class SequenceDataReaderByTimestampV2 implements EngineReaderByTimeStamp
   }
 
   private void initUnSealedTsFileReader(TsFileResourceV2 tsFile)
-      throws IOException {
-    seriesReader = new UnSealedTsFileReaderByTimestampV2(tsFile);
+          throws IOException {
+    seriesReader = new UnSealedTsFileReaderByTimestamp(tsFile);
   }
 
   private void initSealedTsFileReader(TsFileResourceV2 fileNode, QueryContext context)
-      throws IOException {
+          throws IOException {
 
     // to avoid too many opened files
     TsFileSequenceReader tsFileReader = FileReaderManager.getInstance()
-        .get(fileNode.getFile().getPath(), true);
+            .get(fileNode.getFile().getPath(), true);
 
     MetadataQuerierByFileImpl metadataQuerier = new MetadataQuerierByFileImpl(tsFileReader);
     List<ChunkMetaData> metaDataList = metadataQuerier.getChunkMetaDataList(seriesPath);
 
     List<Modification> pathModifications = context.getPathModifications(fileNode.getModFile(),
-        seriesPath.getFullPath());
+            seriesPath.getFullPath());
     if (!pathModifications.isEmpty()) {
       QueryUtils.modifyChunkMetaData(metaDataList, pathModifications);
     }
     ChunkLoader chunkLoader = new ChunkLoaderImpl(tsFileReader);
 
     seriesReader = new SeriesReaderByTimestampAdapter(
-        new SeriesReaderByTimestamp(chunkLoader, metaDataList));
+            new FileSeriesReaderByTimestamp(chunkLoader, metaDataList));
   }
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReaderV2.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReader.java
similarity index 74%
rename from iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReaderV2.java
rename to iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReader.java
index c4b337f..facc83f 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReaderV2.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReader.java
@@ -19,13 +19,10 @@
 
 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.filenodeV2.TsFileResourceV2;
 import org.apache.iotdb.db.query.control.FileReaderManager;
-import org.apache.iotdb.db.query.reader.adapter.FileSeriesReaderAdapter;
 import org.apache.iotdb.db.query.reader.mem.MemChunkReader;
+import org.apache.iotdb.db.query.reader.sequence.adapter.FileSeriesReaderAdapter;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
 import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
 import org.apache.iotdb.tsfile.read.controller.ChunkLoader;
@@ -35,24 +32,28 @@ 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;
 
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
 /**
- * include data in one closing bufferWriteProcessfor or working bufferWriteProcessfor: 1) the data
- * in unseal tsfile part which has been flushed to disk 2) the data in flushing memtable list
+ * batch reader of data in: 1) the data in unseal tsfile part which has been flushed to disk;
+ * 2) the data in flushing memtable list.
  */
-public class UnSealedTsFileReaderV2 extends IterateReader {
+public class UnSealedTsFileReader extends IterateReader {
 
   /**
    * 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;
+   * @param filter         -filter
+   * @param isReverse      true-traverse chunks from behind forward; false-traverse chunks from front to
+   *                       back;
    */
-  public UnSealedTsFileReaderV2(TsFileResourceV2 unsealedTsFile, Filter filter, boolean isReverse)
-      throws IOException {
+  public UnSealedTsFileReader(TsFileResourceV2 unsealedTsFile, Filter filter, boolean isReverse)
+          throws IOException {
     TsFileSequenceReader unClosedTsFileReader = FileReaderManager.getInstance()
-        .get(unsealedTsFile.getFile().getPath(), false);
+            .get(unsealedTsFile.getFile().getPath(), false);
     ChunkLoader chunkLoader = new ChunkLoaderImpl(unClosedTsFileReader);
 
     List<ChunkMetaData> metaDataList = unsealedTsFile.getChunkMetaDatas();
@@ -72,11 +73,11 @@ public class UnSealedTsFileReaderV2 extends IterateReader {
     // data in flushing memtable
     MemChunkReader memChunkReader = new MemChunkReader(unsealedTsFile.getReadOnlyMemChunk(), filter);
     if (isReverse) {
-      seriesReaders.add(memChunkReader);
-      seriesReaders.add(new FileSeriesReaderAdapter(unSealedReader));
+      seqResourceSeriesReaderList.add(memChunkReader);
+      seqResourceSeriesReaderList.add(new FileSeriesReaderAdapter(unSealedReader));
     } else {
-      seriesReaders.add(new FileSeriesReaderAdapter(unSealedReader));
-      seriesReaders.add(memChunkReader);
+      seqResourceSeriesReaderList.add(new FileSeriesReaderAdapter(unSealedReader));
+      seqResourceSeriesReaderList.add(memChunkReader);
 
     }
   }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReaderByTimestampV2.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReaderByTimestamp.java
similarity index 77%
rename from iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReaderByTimestampV2.java
rename to iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReaderByTimestamp.java
index fd76d5e..3ea6af6 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReaderByTimestampV2.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReaderByTimestamp.java
@@ -19,32 +19,33 @@
 
 package org.apache.iotdb.db.query.reader.sequence;
 
-import java.io.IOException;
 import org.apache.iotdb.db.engine.filenodeV2.TsFileResourceV2;
 import org.apache.iotdb.db.query.control.FileReaderManager;
+import org.apache.iotdb.db.query.reader.IReaderByTimeStamp;
 import org.apache.iotdb.db.query.reader.mem.MemChunkReaderByTimestamp;
-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;
+import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderByTimestamp;
+
+import java.io.IOException;
 
 /**
- * include data in one closing bufferWriteProcessfor or working bufferWriteProcessfor: 1) the data
- * in unseal tsfile part which has been flushed to disk 2) the data in flushing memtable list
+ * IReaderByTimeStamp of data in: 1) the data in unseal tsfile part which has been flushed to disk;
+ * 2) the data in flushing memtable list.
  */
-public class UnSealedTsFileReaderByTimestampV2 implements EngineReaderByTimeStamp {
+public class UnSealedTsFileReaderByTimestamp implements IReaderByTimeStamp {
 
   protected Path seriesPath;
   /**
    * reader the data of unseal tsfile part which has been flushed to disk
    */
-  private SeriesReaderByTimestamp unSealedReader;
+  private FileSeriesReaderByTimestamp unSealedReader;
   /**
    * reader of the data in flushing memtable list
    */
-  private EngineReaderByTimeStamp memSeriesReader;
+  private IReaderByTimeStamp memSeriesReader;
   /**
    * whether unSealedReader has been used. True if current reader is memSeriesReader,
    * false if current reader is unSealedReader.
@@ -56,12 +57,12 @@ public class UnSealedTsFileReaderByTimestampV2 implements EngineReaderByTimeStam
    *
    * @param tsFileResource -unclosed tsfile resource
    */
-  public UnSealedTsFileReaderByTimestampV2(TsFileResourceV2 tsFileResource) throws IOException {
+  public UnSealedTsFileReaderByTimestamp(TsFileResourceV2 tsFileResource) throws IOException {
     TsFileSequenceReader unClosedTsFileReader = FileReaderManager.getInstance()
-        .get(tsFileResource.getFile().getPath(), false);
+            .get(tsFileResource.getFile().getPath(), false);
     ChunkLoader chunkLoader = new ChunkLoaderImpl(unClosedTsFileReader);
-    unSealedReader = new SeriesReaderByTimestamp(chunkLoader,
-        tsFileResource.getChunkMetaDatas());
+    unSealedReader = new FileSeriesReaderByTimestamp(chunkLoader,
+            tsFileResource.getChunkMetaDatas());
 
     memSeriesReader = new MemChunkReaderByTimestamp(tsFileResource.getReadOnlyMemChunk());
     unSealedReaderEnded = false;
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/adapter/FileSeriesReaderAdapter.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/adapter/FileSeriesReaderAdapter.java
similarity index 92%
rename from iotdb/src/main/java/org/apache/iotdb/db/query/reader/adapter/FileSeriesReaderAdapter.java
rename to iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/adapter/FileSeriesReaderAdapter.java
index 2f98b47..c3bd49d 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/adapter/FileSeriesReaderAdapter.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/adapter/FileSeriesReaderAdapter.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.query.reader.adapter;
+package org.apache.iotdb.db.query.reader.sequence.adapter;
 
 import java.io.IOException;
 import org.apache.iotdb.db.query.reader.IAggregateReader;
@@ -25,7 +25,7 @@ import org.apache.iotdb.tsfile.read.common.BatchData;
 import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReader;
 
 /**
- *  FileSeriesReader to IAggregateReader adapter.
+ *  This class is used to convert FileSeriesReader (in TsFile) to IAggregateReader.
  */
 public class FileSeriesReaderAdapter implements IAggregateReader {
 
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/adapter/SeriesReaderByTimestampAdapter.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/adapter/SeriesReaderByTimestampAdapter.java
similarity index 58%
rename from iotdb/src/main/java/org/apache/iotdb/db/query/reader/adapter/SeriesReaderByTimestampAdapter.java
rename to iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/adapter/SeriesReaderByTimestampAdapter.java
index 0409d4a..cd8866c 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/adapter/SeriesReaderByTimestampAdapter.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/adapter/SeriesReaderByTimestampAdapter.java
@@ -17,30 +17,30 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.query.reader.adapter;
+package org.apache.iotdb.db.query.reader.sequence.adapter;
 
 import java.io.IOException;
-import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
-import org.apache.iotdb.tsfile.read.reader.series.SeriesReaderByTimestamp;
+import org.apache.iotdb.db.query.reader.IReaderByTimeStamp;
+import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderByTimestamp;
 
 /**
- * SeriesReaderByTimestamp to EngineReaderByTimeStamp adapter.
+ * This class is used to convert FileSeriesReaderByTimestamp (in TsFile) to IReaderByTimeStamp.
  */
-public class SeriesReaderByTimestampAdapter implements EngineReaderByTimeStamp {
+public class SeriesReaderByTimestampAdapter implements IReaderByTimeStamp {
 
-  private SeriesReaderByTimestamp seriesReaderByTimestamp;
+  private FileSeriesReaderByTimestamp fileSeriesReaderByTimestamp;
 
-  public SeriesReaderByTimestampAdapter(SeriesReaderByTimestamp seriesReaderByTimestamp) {
-    this.seriesReaderByTimestamp = seriesReaderByTimestamp;
+  public SeriesReaderByTimestampAdapter(FileSeriesReaderByTimestamp fileSeriesReaderByTimestamp) {
+    this.fileSeriesReaderByTimestamp = fileSeriesReaderByTimestamp;
   }
 
   @Override
   public Object getValueInTimestamp(long timestamp) throws IOException {
-    return seriesReaderByTimestamp.getValueInTimestamp(timestamp);
+    return fileSeriesReaderByTimestamp.getValueInTimestamp(timestamp);
   }
 
   @Override
   public boolean hasNext() throws IOException {
-    return seriesReaderByTimestamp.hasNext();
+    return fileSeriesReaderByTimestamp.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/DiskChunkReader.java
similarity index 89%
rename from iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReader.java
rename to iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/DiskChunkReader.java
index 04a1fce..e989d73 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/DiskChunkReader.java
@@ -16,23 +16,26 @@
  * 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.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 IPointReader {
+import java.io.IOException;
+
+/**
+ * IPointReader of data in a chunk.
+ * Note that currently this class is only used by unsequence query logic.
+ */
+public class DiskChunkReader implements IPointReader {
 
   private ChunkReader chunkReader;
   private BatchData data;
 
-  public EngineChunkReader(ChunkReader chunkReader) {
+  public DiskChunkReader(ChunkReader chunkReader) {
     this.chunkReader = chunkReader;
   }
 
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/DiskChunkReaderByTimestamp.java
similarity index 86%
rename from iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReaderByTimestamp.java
rename to iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/DiskChunkReaderByTimestamp.java
index c6f22a4..f1372cb 100644
--- 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/DiskChunkReaderByTimestamp.java
@@ -16,20 +16,24 @@
  * 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.db.query.reader.IReaderByTimeStamp;
 import org.apache.iotdb.tsfile.read.common.BatchData;
 import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReaderByTimestamp;
 
-public class EngineChunkReaderByTimestamp implements EngineReaderByTimeStamp {
+import java.io.IOException;
+
+/**
+ * IReaderByTimeStamp of data in a chunk.
+ * Note that currently this class is only used by unsequence query logic.
+ */
+public class DiskChunkReaderByTimestamp implements IReaderByTimeStamp {
 
   private ChunkReaderByTimestamp chunkReader;
   private BatchData data;
 
-  public EngineChunkReaderByTimestamp(ChunkReaderByTimestamp chunkReader) {
+  public DiskChunkReaderByTimestamp(ChunkReaderByTimestamp chunkReader) {
     this.chunkReader = chunkReader;
   }
 
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/unsequence/UnsequenceSeriesReader.java
similarity index 82%
rename from iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReader.java
rename to iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/UnsequenceSeriesReader.java
index 251bd9d..0f40f37 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/unsequence/UnsequenceSeriesReader.java
@@ -16,8 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
-package org.apache.iotdb.db.query.reader.merge;
+package org.apache.iotdb.db.query.reader.unsequence;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -26,14 +25,14 @@ import java.util.PriorityQueue;
 import org.apache.iotdb.db.query.reader.IPointReader;
 import org.apache.iotdb.db.utils.TimeValuePair;
 
-/**
- * <p>
- * Usage: (1) merge multiple chunk group readers in the unsequence file.
- * </p>
+ /**
+  * This class is used to read unsequence data in: 1) sealed tsfile resources,
+  * 2) unsealed tsfile resources, which include data on disk and in memtables
+  * that will be flushing to unsealed tsfile resources.
  */
-public class PriorityMergeReader implements IPointReader {
+public class UnsequenceSeriesReader implements IPointReader {
 
-  private List<IPointReader> readerList = new ArrayList<>();
+  private List<IPointReader> unseqResourceSeriesReaderList = new ArrayList<>();
   private List<Integer> priorityList = new ArrayList<>();
   private PriorityQueue<Element> heap = new PriorityQueue<>();
 
@@ -42,9 +41,9 @@ public class PriorityMergeReader implements IPointReader {
    */
   public void addReaderWithPriority(IPointReader reader, int priority) throws IOException {
     if (reader.hasNext()) {
-      heap.add(new Element(readerList.size(), reader.next(), priority));
+      heap.add(new Element(unseqResourceSeriesReaderList.size(), reader.next(), priority));
     }
-    readerList.add(reader);
+    unseqResourceSeriesReaderList.add(reader);
     priorityList.add(priority);
   }
 
@@ -69,7 +68,7 @@ public class PriorityMergeReader implements IPointReader {
     while (!heap.isEmpty() && heap.peek().timeValuePair.getTimestamp() == top.timeValuePair
         .getTimestamp()) {
       Element e = heap.poll();
-      IPointReader reader = readerList.get(e.index);
+      IPointReader reader = unseqResourceSeriesReaderList.get(e.index);
       if (reader.hasNext()) {
         heap.add(new Element(e.index, reader.next(), priorityList.get(e.index)));
       }
@@ -78,7 +77,7 @@ public class PriorityMergeReader implements IPointReader {
 
   @Override
   public void close() throws IOException {
-    for (IPointReader reader : readerList) {
+    for (IPointReader reader : unseqResourceSeriesReaderList) {
       reader.close();
     }
   }
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 3d4b11c..26b72ab 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
@@ -20,18 +20,18 @@
 package org.apache.iotdb.db.query.timegenerator;
 
 import java.io.IOException;
-import org.apache.iotdb.db.query.reader.IReader;
+import org.apache.iotdb.db.query.reader.IPointReader;
 import org.apache.iotdb.tsfile.read.common.BatchData;
 import org.apache.iotdb.tsfile.read.query.timegenerator.node.Node;
 import org.apache.iotdb.tsfile.read.query.timegenerator.node.NodeType;
 
 public class EngineLeafNode implements Node {
 
-  private IReader reader;
+  private IPointReader reader;
 
   private BatchData data = null;
 
-  public EngineLeafNode(IReader reader) {
+  public EngineLeafNode(IPointReader reader) {
     this.reader = reader;
   }
 
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 b95b217..5ce7deb 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
@@ -52,7 +52,7 @@ public class EngineNodeConstructor extends AbstractNodeConstructor {
         Path path = ((SingleSeriesExpression) expression).getSeriesPath();
         return new EngineLeafNode(
             SeriesReaderFactoryImpl.getInstance()
-                .createReaderWithValueFilter(path, filter, context));
+                .createSeriesReaderWithValueFilter(path, filter, context));
       } catch (IOException e) {
         throw new FileNodeManagerException(e);
       }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/service/StartupChecks.java b/iotdb/src/main/java/org/apache/iotdb/db/service/StartupChecks.java
index f64b317..31e6d7c 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/service/StartupChecks.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/service/StartupChecks.java
@@ -84,7 +84,7 @@ public class StartupChecks {
   }
 
   /**
-   * executeWithOutTimeGenerator every pretests.
+   * executeWithoutValueFilter every pretests.
    */
   public void verify() throws StartupException {
     for (StartupCheck check : preChecks) {
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 6fe08c8..8106a20 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
@@ -399,7 +399,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   }
 
   /**
-   * Judge whether the statement is ADMIN COMMAND and if true, executeWithOutTimeGenerator it.
+   * Judge whether the statement is ADMIN COMMAND and if true, executeWithoutValueFilter it.
    *
    * @param statement command
    * @return true if the statement is ADMIN COMMAND
@@ -467,7 +467,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
           }
         } catch (Exception e) {
           String errMessage = String.format(
-              "Fail to generate physcial plan and executeWithOutTimeGenerator for statement "
+              "Fail to generate physcial plan and executeWithoutValueFilter for statement "
                   + "%s beacuse %s",
               statement, e.getMessage());
           LOGGER.warn("Error occurred when executing {}", statement, e);
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAuthorizationIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAuthorizationIT.java
index 1633738..ce2df28 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAuthorizationIT.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAuthorizationIT.java
@@ -563,7 +563,7 @@ public class IoTDBAuthorizationIT {
     userStmt.execute("CREATE TIMESERIES root.a.b WITH DATATYPE=INT32,ENCODING=PLAIN");
     userStmt.execute("CREATE TIMESERIES root.a.c WITH DATATYPE=INT32,ENCODING=PLAIN");
     userStmt.execute("INSERT INTO root.a(timestamp,b,c) VALUES (1,100,1000)");
-    // userStmt.executeWithOutTimeGenerator("DELETE FROM root.a.b WHERE TIME <= 1000000000");
+    // userStmt.executeWithoutValueFilter("DELETE FROM root.a.b WHERE TIME <= 1000000000");
     userStmt.execute("SELECT * FROM root");
     userStmt.getResultSet().close();
 
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBLargeDataIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBLargeDataIT.java
index 12d67f9..9c5902f 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBLargeDataIT.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBLargeDataIT.java
@@ -136,7 +136,7 @@ public class IoTDBLargeDataIT {
       }
 
       statement.execute("flush");
-      // statement.executeWithOutTimeGenerator("merge");
+      // statement.executeWithoutValueFilter("merge");
 
       Thread.sleep(5000);
 
@@ -208,7 +208,7 @@ public class IoTDBLargeDataIT {
       }
 
       // overflow delete
-      // statement.executeWithOutTimeGenerator("DELETE FROM root.vehicle.d0.s1 WHERE time < 3200");
+      // statement.executeWithoutValueFilter("DELETE FROM root.vehicle.d0.s1 WHERE time < 3200");
 
       statement.close();
     } catch (Exception e) {
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java
index ad5b971..67dc511 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java
@@ -137,7 +137,7 @@ public class IoTDBMultiSeriesIT {
       }
 
       statement.execute("flush");
-      // statement.executeWithOutTimeGenerator("merge");
+      // statement.executeWithoutValueFilter("merge");
 
       Thread.sleep(5000);
 
@@ -209,7 +209,7 @@ public class IoTDBMultiSeriesIT {
       }
 
       // overflow delete
-      // statement.executeWithOutTimeGenerator("DELETE FROM root.vehicle.d0.s1 WHERE time < 3200");
+      // statement.executeWithoutValueFilter("DELETE FROM root.vehicle.d0.s1 WHERE time < 3200");
 
       statement.close();
     } catch (Exception e) {
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java
index 00db7c8..5424ef6 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java
@@ -138,7 +138,7 @@ public class IoTDBSeriesReaderIT {
         statement.execute(sql);
       }
 
-      // statement.executeWithOutTimeGenerator("flush");
+      // statement.executeWithoutValueFilter("flush");
 
       // insert large amount of data time range : 13700 ~ 24000
       for (int time = 13700; time < 24000; time++) {
@@ -226,7 +226,7 @@ public class IoTDBSeriesReaderIT {
       }
 
       // overflow update
-      // statement.executeWithOutTimeGenerator("UPDATE root.vehicle SET d0.s1 = 11111111 WHERE time > 23000 and time < 100100");
+      // statement.executeWithoutValueFilter("UPDATE root.vehicle SET d0.s1 = 11111111 WHERE time > 23000 and time < 100100");
 
       statement.close();
     } catch (Exception e) {
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBTimeZoneIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBTimeZoneIT.java
index c17517b..eb68eaf 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBTimeZoneIT.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBTimeZoneIT.java
@@ -67,7 +67,7 @@ public class IoTDBTimeZoneIT {
   }
 
   /**
-   * // executeWithOutTimeGenerator in cli-tool SET STORAGE GROUP TO root.timezone CREATE TIMESERIES root.timezone.tz1 WITH DATATYPE =
+   * // executeWithoutValueFilter in cli-tool SET STORAGE GROUP TO root.timezone CREATE TIMESERIES root.timezone.tz1 WITH DATATYPE =
    * INT32, ENCODING = PLAIN set time_zone=+08:00 insert into root.timezone(timestamp,tz1) values(1514779200000,1)
    * insert into root.timezone(timestamp,tz1) values(2018-1-1T12:00:01,2) insert into root.timezone(timestamp,tz1)
    * values(2018-1-1T12:00:02+08:00,3) insert into root.timezone(timestamp,tz1) values(2018-1-1T12:00:03+09:00,4)
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/qp/plan/QPUpdateTest.java b/iotdb/src/test/java/org/apache/iotdb/db/qp/plan/QPUpdateTest.java
index 2f1b209..cf0b3da 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/qp/plan/QPUpdateTest.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/qp/plan/QPUpdateTest.java
@@ -188,7 +188,7 @@ public class QPUpdateTest {
     String sqlStr = "insert into root.qp_update_test.device_1 (timestamp, sensor_1, sensor_2) values (13, 50, 40)";
     PhysicalPlan plan1 = processor.parseSQLToPhysicalPlan(sqlStr);
 
-    // executeWithOutTimeGenerator insert
+    // executeWithoutValueFilter insert
     boolean upRet = processor.getExecutor().processNonQuery(plan1);
     assertTrue(upRet);
 
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGeneratorTest.java b/iotdb/src/test/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithValueFilterTest.java
similarity index 99%
rename from iotdb/src/test/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGeneratorTest.java
rename to iotdb/src/test/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithValueFilterTest.java
index b53e4d8..4aa1eed 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGeneratorTest.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithValueFilterTest.java
@@ -33,7 +33,7 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-public class EngineDataSetWithTimeGeneratorTest {
+public class EngineDataSetWithValueFilterTest {
 
   private QueryProcessExecutor queryExecutor = new OverflowQPExecutor();
   private QueryProcessor processor = new QueryProcessor(queryExecutor);
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/FakedSeriesReaderByTimestamp.java b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/FakedSeriesReaderByTimestamp.java
index 52eda0f..24891b0 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/FakedSeriesReaderByTimestamp.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/FakedSeriesReaderByTimestamp.java
@@ -22,12 +22,12 @@ package org.apache.iotdb.db.query.reader;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
-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.file.metadata.enums.TSDataType;
 
-public class FakedSeriesReaderByTimestamp implements EngineReaderByTimeStamp {
+public class FakedSeriesReaderByTimestamp implements IReaderByTimeStamp {
 
   private Iterator<TimeValuePair> iterator;
   private boolean hasCachedTimeValuePair = false;
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/AllDataReaderWithOptGlobalTimeFilterTest.java b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/SeriesReaderWithoutValueFilterTest.java
similarity index 88%
rename from iotdb/src/test/java/org/apache/iotdb/db/query/reader/AllDataReaderWithOptGlobalTimeFilterTest.java
rename to iotdb/src/test/java/org/apache/iotdb/db/query/reader/SeriesReaderWithoutValueFilterTest.java
index eed9285..76e1748 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/AllDataReaderWithOptGlobalTimeFilterTest.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/SeriesReaderWithoutValueFilterTest.java
@@ -24,18 +24,18 @@ import org.apache.iotdb.db.utils.TimeValuePair;
 import org.junit.Assert;
 import org.junit.Test;
 
-public class AllDataReaderWithOptGlobalTimeFilterTest {
+public class SeriesReaderWithoutValueFilterTest {
 
-  private AllDataReaderWithOptGlobalTimeFilter reader1;
-  private AllDataReaderWithOptGlobalTimeFilter reader2;
+  private SeriesReaderWithoutValueFilter reader1;
+  private SeriesReaderWithoutValueFilter reader2;
 
   private void init() {
     IBatchReader batchReader1 = new FakedIBatchPoint(100, 1000, 7, 11);
     IBatchReader batchReader2 = new FakedIBatchPoint(100, 1000, 7, 11);
     IPointReader pointReader = new FakedIPointReader(20, 500, 11, 19);
 
-    reader1 = new AllDataReaderWithOptGlobalTimeFilter(batchReader1, pointReader);
-    reader2 = new AllDataReaderWithOptGlobalTimeFilter(batchReader2, null);
+    reader1 = new SeriesReaderWithoutValueFilter(batchReader1, pointReader);
+    reader2 = new SeriesReaderWithoutValueFilter(batchReader2, null);
   }
 
   @Test
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/SeriesMergeSortReaderTest.java b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/SeriesMergeSortReaderTest.java
index e16c071..522a29d 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/SeriesMergeSortReaderTest.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/SeriesMergeSortReaderTest.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.query.reader.merge;
 
 import java.io.IOException;
 import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.unsequence.UnsequenceSeriesReader;
 import org.apache.iotdb.db.utils.TimeValuePair;
 import org.apache.iotdb.db.utils.TsPrimitiveType;
 import org.junit.Assert;
@@ -51,7 +52,7 @@ public class SeriesMergeSortReaderTest {
   }
 
   private void test(long[] retTimestamp, long[] retValue, long[]... sources) throws IOException {
-    PriorityMergeReader seriesMergeSortReader = new PriorityMergeReader();
+    UnsequenceSeriesReader seriesMergeSortReader = new UnsequenceSeriesReader();
     for (int i = 0; i < sources.length; i++) {
       seriesMergeSortReader.addReaderWithPriority(
           new FakedSeriesReader(sources[i], i + 1), i + 1);
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/SeriesReaderByTimestampTest.java
similarity index 96%
rename from iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java
rename to iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/SeriesReaderByTimestampTest.java
index 4185f9d..726f43c 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/SeriesReaderByTimestampTest.java
@@ -24,13 +24,14 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Random;
 import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.IReaderByTimeStamp;
 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.junit.Assert;
 import org.junit.Test;
 
-public class PriorityMergeReaderByTimestampTest {
+public class SeriesReaderByTimestampTest {
 
   @Test
   public void test() throws IOException {
@@ -41,7 +42,7 @@ public class PriorityMergeReaderByTimestampTest {
     FakedPrioritySeriesReaderByTimestamp reader3 = new FakedPrioritySeriesReaderByTimestamp(1080,
         200, 13, 31);
 
-    PriorityMergeReaderByTimestamp priorityReader = new PriorityMergeReaderByTimestamp();
+    SeriesReaderByTimestamp priorityReader = new SeriesReaderByTimestamp();
     priorityReader.addReaderWithPriority(reader1, 1);
     priorityReader.addReaderWithPriority(reader2, 2);
     priorityReader.addReaderWithPriority(reader3, 3);
@@ -92,7 +93,7 @@ public class PriorityMergeReaderByTimestampTest {
 
   }
 
-  public static class FakedPrioritySeriesReaderByTimestamp implements EngineReaderByTimeStamp,
+  public static class FakedPrioritySeriesReaderByTimestamp implements IReaderByTimeStamp,
       IPointReader {
 
     private Iterator<TimeValuePair> iterator;
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderTest.java b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/UnsequenceSeriesReaderTest.java
similarity index 84%
rename from iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderTest.java
rename to iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/UnsequenceSeriesReaderTest.java
index 8974c0a..9c19597 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderTest.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/UnsequenceSeriesReaderTest.java
@@ -23,6 +23,7 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.unsequence.UnsequenceSeriesReader;
 import org.apache.iotdb.db.utils.TimeValuePair;
 import org.apache.iotdb.db.utils.TsPrimitiveType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -30,9 +31,9 @@ import org.junit.Assert;
 import org.junit.Test;
 
 /**
- * Test {@code PriorityMergeReader}
+ * Test {@code UnsequenceSeriesReader}
  */
-public class PriorityMergeReaderTest {
+public class UnsequenceSeriesReaderTest {
 
   @Test
   public void test() throws IOException {
@@ -40,14 +41,14 @@ public class PriorityMergeReaderTest {
     FakedPrioritySeriesReader reader2 = new FakedPrioritySeriesReader(150, 60, 6, 19);
     FakedPrioritySeriesReader reader3 = new FakedPrioritySeriesReader(180, 50, 7, 31);
 
-    PriorityMergeReader priorityMergeReader = new PriorityMergeReader();
-    priorityMergeReader.addReaderWithPriority(reader1, 3);
-    priorityMergeReader.addReaderWithPriority(reader2, 2);
-    priorityMergeReader.addReaderWithPriority(reader3, 1);
+    UnsequenceSeriesReader unsequenceSeriesReader = new UnsequenceSeriesReader();
+    unsequenceSeriesReader.addReaderWithPriority(reader1, 3);
+    unsequenceSeriesReader.addReaderWithPriority(reader2, 2);
+    unsequenceSeriesReader.addReaderWithPriority(reader3, 1);
 
     int cnt = 0;
-    while (priorityMergeReader.hasNext()) {
-      TimeValuePair timeValuePair = priorityMergeReader.next();
+    while (unsequenceSeriesReader.hasNext()) {
+      TimeValuePair timeValuePair = unsequenceSeriesReader.next();
       long time = timeValuePair.getTimestamp();
       long value = timeValuePair.getValue().getLong();
 
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/sequence/SeqDataReaderTest.java b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/sequence/SeqDataReaderTest.java
index 831e192..9731b4c 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/sequence/SeqDataReaderTest.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/sequence/SeqDataReaderTest.java
@@ -34,7 +34,7 @@ public class SeqDataReaderTest extends ReaderTestHelper {
   public void testSeqReader() throws IOException, FileNodeProcessorException {
     QueryDataSourceV2 queryDataSource = fileNodeProcessorV2.query(deviceId, measurementId);
     Path path = new Path(deviceId, measurementId);
-    SequenceDataReaderV2 readerV2 = new SequenceDataReaderV2(path,
+    SequenceSeriesReader readerV2 = new SequenceSeriesReader(path,
         queryDataSource.getSeqResources(), null,
         EnvironmentUtils.TEST_QUERY_CONTEXT);
     long time = 999;
@@ -53,7 +53,7 @@ public class SeqDataReaderTest extends ReaderTestHelper {
   public void testSeqByTimestampReader() throws IOException, FileNodeProcessorException {
     QueryDataSourceV2 queryDataSource = fileNodeProcessorV2.query(deviceId, measurementId);
     Path path = new Path(deviceId, measurementId);
-    SequenceDataReaderByTimestampV2 readerV2 = new SequenceDataReaderByTimestampV2(path,
+    SequenceSeriesReaderByTimestamp readerV2 = new SequenceSeriesReaderByTimestamp(path,
         queryDataSource.getSeqResources(), EnvironmentUtils.TEST_QUERY_CONTEXT);
 
     for (int time = 1000; time <= 3020; time += 10) {
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/sequence/UnsealedSeqReaderTest.java b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/sequence/UnsealedSeqReaderTest.java
index 8c13ec0..ffc80d4 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/sequence/UnsealedSeqReaderTest.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/sequence/UnsealedSeqReaderTest.java
@@ -34,7 +34,7 @@ public class UnsealedSeqReaderTest extends ReaderTestHelper {
     QueryDataSourceV2 queryDataSource = fileNodeProcessorV2.query(deviceId, measurementId);
     TsFileResourceV2 resourceV2 = queryDataSource.getSeqResources().get(0);
     Assert.assertEquals(false, resourceV2.isClosed());
-    UnSealedTsFileReaderV2 readerV2 = new UnSealedTsFileReaderV2(resourceV2, null, false);
+    UnSealedTsFileReader readerV2 = new UnSealedTsFileReader(resourceV2, null, false);
     long time = 999;
     while (readerV2.hasNext()) {
       BatchData batchData = readerV2.nextBatch();
@@ -52,7 +52,7 @@ public class UnsealedSeqReaderTest extends ReaderTestHelper {
     QueryDataSourceV2 queryDataSource = fileNodeProcessorV2.query(deviceId, measurementId);
     TsFileResourceV2 resourceV2 = queryDataSource.getSeqResources().get(0);
     Assert.assertEquals(false, resourceV2.isClosed());
-    UnSealedTsFileReaderByTimestampV2 readerV2 = new UnSealedTsFileReaderByTimestampV2(
+    UnSealedTsFileReaderByTimestamp readerV2 = new UnSealedTsFileReaderByTimestamp(
         resourceV2);
 
     for (int time = 1000; time <= 3020; time += 10) {
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/unsequence/CreateByTimestampReadersOfSelectedPathsTest.java b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/unsequence/UnseqSeriesReaderByTimestampTest.java
similarity index 89%
rename from iotdb/src/test/java/org/apache/iotdb/db/query/reader/unsequence/CreateByTimestampReadersOfSelectedPathsTest.java
rename to iotdb/src/test/java/org/apache/iotdb/db/query/reader/unsequence/UnseqSeriesReaderByTimestampTest.java
index 56611e1..7ba5b63 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/unsequence/CreateByTimestampReadersOfSelectedPathsTest.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/unsequence/UnseqSeriesReaderByTimestampTest.java
@@ -23,7 +23,7 @@ import org.apache.iotdb.db.engine.filenodeV2.FileNodeManagerV2;
 import org.apache.iotdb.db.exception.FileNodeManagerException;
 import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
 import org.apache.iotdb.db.query.factory.SeriesReaderFactoryImpl;
-import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.db.query.reader.IReaderByTimeStamp;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.common.Path;
@@ -38,7 +38,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
-public class CreateByTimestampReadersOfSelectedPathsTest {
+public class UnseqSeriesReaderByTimestampTest {
     private String systemDir = "data/info";
     private String deviceId = "root.vehicle.d0";
     private String measurementId = "s0";
@@ -56,7 +56,7 @@ public class CreateByTimestampReadersOfSelectedPathsTest {
     }
 
     @Test
-    public void testUnSeqReaderWithoutFilter() throws IOException, FileNodeManagerException {
+    public void testUnseqSeriesReaderByTimestamp() throws IOException, FileNodeManagerException {
         // write
         for (int j = 1; j <= 10; j++) {
             TSRecord record = new TSRecord(j, deviceId);
@@ -79,10 +79,10 @@ public class CreateByTimestampReadersOfSelectedPathsTest {
         // query
         List<Path> paths = new ArrayList<>();
         paths.add(new Path(deviceId, measurementId));
-        List<EngineReaderByTimeStamp> readers = SeriesReaderFactoryImpl.getInstance().
-            createByTimestampReaders(paths, EnvironmentUtils.TEST_QUERY_CONTEXT);
+        List<IReaderByTimeStamp> readers = SeriesReaderFactoryImpl.getInstance().
+                createSeriesReadersByTimestamp(paths, EnvironmentUtils.TEST_QUERY_CONTEXT);
         Assert.assertEquals(1, readers.size());
-        EngineReaderByTimeStamp reader = readers.get(0);
+        IReaderByTimeStamp reader = readers.get(0);
 
         for (long time = 1; time <= 10; time++) {
             // NOTE that the timestamps should be in be in strictly increasing order.
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/unsequence/UnseqReaderTest.java b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/unsequence/UnsequenceSeriesReaderTest.java
similarity index 90%
rename from iotdb/src/test/java/org/apache/iotdb/db/query/reader/unsequence/UnseqReaderTest.java
rename to iotdb/src/test/java/org/apache/iotdb/db/query/reader/unsequence/UnsequenceSeriesReaderTest.java
index 12aa3db..86ecf81 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/unsequence/UnseqReaderTest.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/unsequence/UnsequenceSeriesReaderTest.java
@@ -33,7 +33,7 @@ import org.junit.Test;
 
 import java.io.IOException;
 
-public class UnseqReaderTest extends ReaderTestHelper {
+public class UnsequenceSeriesReaderTest extends ReaderTestHelper {
     @Override
     protected void insertData() {
         for (int j = 1; j <= 10; j++) {
@@ -50,10 +50,10 @@ public class UnseqReaderTest extends ReaderTestHelper {
     }
 
     @Test
-    public void testUnSeqReaderWithGlobalTimeFilter() throws IOException, FileNodeProcessorException {
+    public void testUnseqSeriesReaderWithGlobalTimeFilter() throws IOException, FileNodeProcessorException {
         Path path = new Path(deviceId, measurementId);
         QueryDataSourceV2 queryDataSource = fileNodeProcessorV2.query(deviceId, measurementId);
-        IPointReader reader = SeriesReaderFactoryImpl.getInstance().createUnSeqReader(path,
+        IPointReader reader = SeriesReaderFactoryImpl.getInstance().createUnseqSeriesReader(path,
                 queryDataSource.getUnseqResources(), EnvironmentUtils.TEST_QUERY_CONTEXT, TimeFilter.eq(4));
         int cnt = 0;
         while (reader.hasNext()) {
@@ -66,10 +66,10 @@ public class UnseqReaderTest extends ReaderTestHelper {
     }
 
     @Test
-    public void testUnSeqReaderWithoutFilter() throws IOException, FileNodeProcessorException {
+    public void testUnseqSeriesReaderWithoutFilter() throws IOException, FileNodeProcessorException {
         Path path = new Path(deviceId, measurementId);
         QueryDataSourceV2 queryDataSource = fileNodeProcessorV2.query(deviceId, measurementId);
-        IPointReader reader = SeriesReaderFactoryImpl.getInstance().createUnSeqReader(path,
+        IPointReader reader = SeriesReaderFactoryImpl.getInstance().createUnseqSeriesReader(path,
                 queryDataSource.getUnseqResources(), EnvironmentUtils.TEST_QUERY_CONTEXT, null);
         int cnt = 0;
         while (reader.hasNext()) {
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/writelog/recover/UnseqTsFileRecoverTest.java b/iotdb/src/test/java/org/apache/iotdb/db/writelog/recover/UnseqTsFileRecoverTest.java
index 8f49300..6faf9d6 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/writelog/recover/UnseqTsFileRecoverTest.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/writelog/recover/UnseqTsFileRecoverTest.java
@@ -24,14 +24,13 @@ import static org.junit.Assert.assertEquals;
 import java.io.File;
 import java.io.IOException;
 import org.apache.commons.io.FileUtils;
-import org.apache.iotdb.db.engine.filenode.TsFileResource;
 import org.apache.iotdb.db.engine.filenodeV2.TsFileResourceV2;
 import org.apache.iotdb.db.engine.querycontext.OverflowInsertFile;
 import org.apache.iotdb.db.engine.version.VersionController;
 import org.apache.iotdb.db.exception.ProcessorException;
 import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
-import org.apache.iotdb.db.query.reader.merge.PriorityMergeReader;
-import org.apache.iotdb.db.query.reader.unsequence.EngineChunkReader;
+import org.apache.iotdb.db.query.reader.unsequence.UnsequenceSeriesReader;
+import org.apache.iotdb.db.query.reader.unsequence.DiskChunkReader;
 import org.apache.iotdb.db.utils.TimeValuePair;
 import org.apache.iotdb.db.writelog.manager.MultiFileLogNodeManager;
 import org.apache.iotdb.db.writelog.node.WriteLogNode;
@@ -140,7 +139,7 @@ public class UnseqTsFileRecoverTest {
     OverflowInsertFile overflowInsertFile = new OverflowInsertFile(tsF.getPath(),
         metadataQuerier.getChunkMetaDataList(path));
 
-    PriorityMergeReader unSeqMergeReader = new PriorityMergeReader();
+    UnsequenceSeriesReader unSeqMergeReader = new UnsequenceSeriesReader();
     int priorityValue = 1;
 
     for (ChunkMetaData chunkMetaData : overflowInsertFile.getChunkMetaDataList()) {
@@ -148,7 +147,7 @@ public class UnseqTsFileRecoverTest {
       ChunkReader chunkReader = new ChunkReaderWithoutFilter(chunk);
 
       unSeqMergeReader
-          .addReaderWithPriority(new EngineChunkReader(chunkReader),
+          .addReaderWithPriority(new DiskChunkReader(chunkReader),
               priorityValue);
       priorityValue++;
     }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/dataset/DataSetWithTimeGenerator.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/dataset/DataSetWithTimeGenerator.java
index 8659c40..88310e0 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/dataset/DataSetWithTimeGenerator.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/dataset/DataSetWithTimeGenerator.java
@@ -20,14 +20,12 @@ package org.apache.iotdb.tsfile.read.query.dataset;
 
 import java.io.IOException;
 import java.util.List;
-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.common.RowRecord;
 import org.apache.iotdb.tsfile.read.query.timegenerator.TimeGenerator;
-import org.apache.iotdb.tsfile.read.reader.series.SeriesReaderByTimestamp;
-import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderByTimestamp;
 
 /**
  * query processing: (1) generate time by series that has filter (2) get value of series that does
@@ -36,7 +34,7 @@ import org.apache.iotdb.tsfile.utils.Binary;
 public class DataSetWithTimeGenerator extends QueryDataSet {
 
   private TimeGenerator timeGenerator;
-  private List<SeriesReaderByTimestamp> readers;
+  private List<FileSeriesReaderByTimestamp> readers;
   private List<Boolean> cached;
 
   /**
@@ -46,11 +44,11 @@ public class DataSetWithTimeGenerator extends QueryDataSet {
    * @param cached cached boolean in List(boolean) structure
    * @param dataTypes TSDataTypes in List structure
    * @param timeGenerator TimeGenerator object
-   * @param readers readers in List(SeriesReaderByTimestamp) structure
+   * @param readers readers in List(FileSeriesReaderByTimestamp) structure
    */
   public DataSetWithTimeGenerator(List<Path> paths, List<Boolean> cached,
       List<TSDataType> dataTypes,
-      TimeGenerator timeGenerator, List<SeriesReaderByTimestamp> readers) {
+      TimeGenerator timeGenerator, List<FileSeriesReaderByTimestamp> readers) {
     super(paths, dataTypes);
     this.cached = cached;
     this.timeGenerator = timeGenerator;
@@ -77,8 +75,8 @@ public class DataSetWithTimeGenerator extends QueryDataSet {
       }
 
       // get value from series reader without filter
-      SeriesReaderByTimestamp seriesReaderByTimestamp = readers.get(i);
-      Object value = seriesReaderByTimestamp.getValueInTimestamp(timestamp);
+      FileSeriesReaderByTimestamp fileSeriesReaderByTimestamp = readers.get(i);
+      Object value = fileSeriesReaderByTimestamp.getValueInTimestamp(timestamp);
       rowRecord.addField(getField(value, dataTypes.get(i)));
     }
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/dataset/DataSetWithoutTimeGenerator.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/dataset/DataSetWithoutTimeGenerator.java
index fd2346a..7c6342f 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/dataset/DataSetWithoutTimeGenerator.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/dataset/DataSetWithoutTimeGenerator.java
@@ -55,7 +55,7 @@ public class DataSetWithoutTimeGenerator extends QueryDataSet {
    *
    * @param paths paths in List structure
    * @param dataTypes TSDataTypes in List structure
-   * @param readers readers in List(SeriesReaderByTimestamp) structure
+   * @param readers readers in List(FileSeriesReaderByTimestamp) structure
    * @throws IOException IOException
    */
   public DataSetWithoutTimeGenerator(List<Path> paths, List<TSDataType> dataTypes,
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/executor/ExecutorWithTimeGenerator.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/executor/ExecutorWithTimeGenerator.java
index 6114ce6..5bd6892 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/executor/ExecutorWithTimeGenerator.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/executor/ExecutorWithTimeGenerator.java
@@ -35,7 +35,7 @@ import org.apache.iotdb.tsfile.read.expression.impl.SingleSeriesExpression;
 import org.apache.iotdb.tsfile.read.query.dataset.DataSetWithTimeGenerator;
 import org.apache.iotdb.tsfile.read.query.timegenerator.TimeGenerator;
 import org.apache.iotdb.tsfile.read.query.timegenerator.TimeGeneratorImpl;
-import org.apache.iotdb.tsfile.read.reader.series.SeriesReaderByTimestamp;
+import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderByTimestamp;
 
 public class ExecutorWithTimeGenerator implements QueryExecutor {
 
@@ -65,7 +65,7 @@ public class ExecutorWithTimeGenerator implements QueryExecutor {
     // the size of hasFilter is equal to selectedPathList, if a series has a filter, it is true,
     // otherwise false
     List<Boolean> cached = removeFilteredPaths(expression, selectedPathList);
-    List<SeriesReaderByTimestamp> readersOfSelectedSeries = new ArrayList<>();
+    List<FileSeriesReaderByTimestamp> readersOfSelectedSeries = new ArrayList<>();
     List<TSDataType> dataTypes = new ArrayList<>();
 
     Iterator<Boolean> cachedIterator = cached.iterator();
@@ -81,7 +81,7 @@ public class ExecutorWithTimeGenerator implements QueryExecutor {
           readersOfSelectedSeries.add(null);
           continue;
         }
-        SeriesReaderByTimestamp seriesReader = new SeriesReaderByTimestamp(chunkLoader,
+        FileSeriesReaderByTimestamp seriesReader = new FileSeriesReaderByTimestamp(chunkLoader,
             chunkMetaDataList);
         readersOfSelectedSeries.add(seriesReader);
       } else {
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/SeriesReaderByTimestamp.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReaderByTimestamp.java
similarity index 96%
rename from tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/SeriesReaderByTimestamp.java
rename to tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReaderByTimestamp.java
index 61d9e9b..90e31a9 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/SeriesReaderByTimestamp.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReaderByTimestamp.java
@@ -34,7 +34,7 @@ import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReaderByTimestamp;
  * a series with given timestamps.
  * </p>
  */
-public class SeriesReaderByTimestamp {
+public class FileSeriesReaderByTimestamp {
 
   protected ChunkLoader chunkLoader;
   protected List<ChunkMetaData> chunkMetaDataList;
@@ -47,7 +47,7 @@ public class SeriesReaderByTimestamp {
   /**
    * init with chunkLoader and chunkMetaDataList.
    */
-  public SeriesReaderByTimestamp(ChunkLoader chunkLoader, List<ChunkMetaData> chunkMetaDataList) {
+  public FileSeriesReaderByTimestamp(ChunkLoader chunkLoader, List<ChunkMetaData> chunkMetaDataList) {
     this.chunkLoader = chunkLoader;
     this.chunkMetaDataList = chunkMetaDataList;
     currentTimestamp = Long.MIN_VALUE;
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/query/timegenerator/ReaderByTimestampTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/query/timegenerator/ReaderByTimestampTest.java
index 5f1ecd5..ab2f462 100644
--- a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/query/timegenerator/ReaderByTimestampTest.java
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/query/timegenerator/ReaderByTimestampTest.java
@@ -31,7 +31,7 @@ import org.apache.iotdb.tsfile.read.controller.ChunkLoaderImpl;
 import org.apache.iotdb.tsfile.read.controller.MetadataQuerierByFileImpl;
 import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReader;
 import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderWithoutFilter;
-import org.apache.iotdb.tsfile.read.reader.series.SeriesReaderByTimestamp;
+import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderByTimestamp;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -87,7 +87,7 @@ public class ReaderByTimestampTest {
     long startTimestamp = System.currentTimeMillis();
     count = 0;
 
-    SeriesReaderByTimestamp seriesReaderFromSingleFileByTimestamp = new SeriesReaderByTimestamp(
+    FileSeriesReaderByTimestamp seriesReaderFromSingleFileByTimestamp = new FileSeriesReaderByTimestamp(
         seriesChunkLoader,
         chunkMetaDataList);