You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by hx...@apache.org on 2021/01/21 04:06:10 UTC

[iotdb] branch master updated: [IOTDB-1081] New TsFile Format (V3) (#2184)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new cf9b35d  [IOTDB-1081] New TsFile Format (V3) (#2184)
cf9b35d is described below

commit cf9b35d5499e64e69ecd7c985e0c389e826fb041
Author: Jackie Tien <Ja...@foxmail.com>
AuthorDate: Thu Jan 21 12:05:48 2021 +0800

    [IOTDB-1081] New TsFile Format (V3) (#2184)
    
    Co-authored-by: HTHou <hh...@outlook.com>
    Co-authored-by: Zesong Sun <sz...@mails.tsinghua.edu.cn>
    Co-authored-by: Qi Yu <yu...@gmail.com>
    Co-authored-by: wshao08 <ko...@163.com>
---
 .../main/java/org/apache/iotdb/JDBCExample.java    |   25 +-
 .../apache/iotdb/tsfile/TsFileSequenceRead.java    |   62 +-
 .../apache/iotdb/hadoop/fileSystem/HDFSInput.java  |   11 +
 .../apache/iotdb/hadoop/fileSystem/HDFSOutput.java |    5 +
 .../iotdb/hadoop/tsfile/record/HDFSTSRecord.java   |    4 +-
 server/file-changelists/TsFileMods-changelist.md   |   11 +
 .../system.properties-changelist.md                |    9 +
 .../resources/tools/upgrade/config.properties      |   27 -
 .../resources/tools/upgrade/offline-upgrade.bat    |   67 --
 .../resources/tools/upgrade/offline-upgrade.sh     |   47 -
 .../org/apache/iotdb/db/conf/IoTDBConfigCheck.java |  199 +++-
 .../apache/iotdb/db/engine/cache/ChunkCache.java   |    5 +-
 .../level/LevelCompactionTsFileManagement.java     |    9 +-
 .../merge/selector/MaxFileMergeFileSelector.java   |    4 +-
 .../io/LocalTextModificationAccessor.java          |    8 +-
 .../engine/storagegroup/StorageGroupProcessor.java |  147 +--
 .../db/engine/storagegroup/TsFileResource.java     |   50 +-
 .../iotdb/db/engine/upgrade/UpgradeTask.java       |  107 +-
 .../iotdb/db/metadata/logfile/MLogWriter.java      |    6 +-
 .../iotdb/db/qp/physical/crud/InsertRowPlan.java   |    4 +-
 .../db/qp/physical/crud/InsertTabletPlan.java      |    4 +-
 .../db/query/aggregation/AggregateResult.java      |    2 +-
 .../db/query/aggregation/impl/AvgAggrResult.java   |   12 +-
 .../db/query/aggregation/impl/SumAggrResult.java   |   10 +-
 .../iotdb/db/query/control/FileReaderManager.java  |   16 +-
 .../FixLengthIExternalSortFileDeserializer.java    |    2 +-
 .../chunk/metadata/DiskChunkMetadataLoader.java    |    5 +-
 .../org/apache/iotdb/db/service/UpgradeSevice.java |    3 -
 .../apache/iotdb/db/tools/TsFileSketchTool.java    |   36 +-
 .../db/tools/upgrade/TsFileOnlineUpgradeTool.java  |  451 +++------
 .../org/apache/iotdb/db/utils/FileLoaderUtils.java |   12 +-
 .../java/org/apache/iotdb/db/utils/QueryUtils.java |    9 +
 .../org/apache/iotdb/db/utils/UpgradeUtils.java    |  112 ++-
 .../engine/merge/MaxFileMergeFileSelectorTest.java |    4 +-
 .../merge/MaxSeriesMergeFileSelectorTest.java      |    8 +-
 .../db/integration/IoTDBNewTsFileCompactionIT.java | 1018 ++++++++++++++++++++
 .../db/qp/physical/InsertTabletMultiPlanTest.java  |    2 +-
 .../iotdb/db/qp/physical/InsertTabletPlanTest.java |    2 +-
 .../db/writelog/recover/SeqTsFileRecoverTest.java  |    2 +-
 .../iotdb/session/IoTDBSessionIteratorIT.java      |    2 +-
 .../apache/iotdb/spark/tsfile/HDFSInputTest.java   |    4 +-
 tsfile/format-changelist.md                        |    4 +-
 .../iotdb/tsfile/common/conf/TSFileConfig.java     |    6 +-
 .../iotdb/tsfile/encoding/decoder/Decoder.java     |    8 +-
 .../tsfile/encoding/decoder/FloatDecoder.java      |   11 +-
 .../tsfile/encoding/decoder/IntRleDecoder.java     |   11 +-
 .../tsfile/encoding/decoder/LongRleDecoder.java    |   11 +-
 .../tsfile/encoding/decoder/PlainDecoder.java      |   21 +-
 .../iotdb/tsfile/encoding/decoder/RleDecoder.java  |   15 +-
 .../tsfile/encoding/encoder/FloatEncoder.java      |    6 +-
 .../tsfile/encoding/encoder/IntRleEncoder.java     |    8 +-
 .../tsfile/encoding/encoder/LongRleEncoder.java    |    6 +-
 .../tsfile/encoding/encoder/PlainEncoder.java      |   57 +-
 .../iotdb/tsfile/encoding/encoder/RleEncoder.java  |   20 +-
 .../tsfile/encoding/encoder/TSEncodingBuilder.java |   13 +-
 .../org/apache/iotdb/tsfile/file/MetaMarker.java   |   17 +-
 .../iotdb/tsfile/file/footer/ChunkGroupFooter.java |  156 ---
 .../iotdb/tsfile/file/header/ChunkGroupHeader.java |  117 +++
 .../iotdb/tsfile/file/header/ChunkHeader.java      |  151 +--
 .../iotdb/tsfile/file/header/PageHeader.java       |   54 +-
 .../iotdb/tsfile/file/header/package-info.java     |   30 -
 .../iotdb/tsfile/file/metadata/ChunkMetadata.java  |   35 +-
 .../file/metadata/MetadataIndexConstructor.java    |    5 +-
 .../tsfile/file/metadata/MetadataIndexEntry.java   |    4 +-
 .../tsfile/file/metadata/MetadataIndexNode.java    |   12 +-
 .../tsfile/file/metadata/TimeseriesMetadata.java   |   36 +-
 .../iotdb/tsfile/file/metadata/TsFileMetadata.java |   45 +-
 .../file/metadata/enums/CompressionType.java       |   64 +-
 .../file/metadata/enums/MetadataIndexNodeType.java |   57 +-
 .../tsfile/file/metadata/enums/TSDataType.java     |  113 +--
 .../tsfile/file/metadata/enums/TSEncoding.java     |   94 +-
 .../file/metadata/statistics/BinaryStatistics.java |   13 +-
 .../metadata/statistics/BooleanStatistics.java     |   48 +-
 .../file/metadata/statistics/DoubleStatistics.java |   23 +-
 .../file/metadata/statistics/FloatStatistics.java  |   23 +-
 .../metadata/statistics/IntegerStatistics.java     |   48 +-
 .../file/metadata/statistics/LongStatistics.java   |   31 +-
 .../file/metadata/statistics/Statistics.java       |   22 +-
 .../iotdb/tsfile/read/TsFileSequenceReader.java    |  404 ++++----
 .../apache/iotdb/tsfile/read/common/BatchData.java |    2 +-
 .../org/apache/iotdb/tsfile/read/common/Chunk.java |   99 +-
 .../read/controller/CachedChunkLoaderImpl.java     |    3 +-
 .../iotdb/tsfile/read/reader/LocalTsFileInput.java |   20 +
 .../iotdb/tsfile/read/reader/TsFileInput.java      |    5 +
 .../tsfile/read/reader/chunk/ChunkReader.java      |  105 +-
 .../iotdb/tsfile/read/reader/page/PageReader.java  |   14 +-
 .../tsfile/utils/ReadWriteForEncodingUtils.java    |   96 +-
 .../iotdb/tsfile/utils/ReadWriteIOUtils.java       |  103 +-
 .../v1/file/metadata/ChunkGroupMetaDataV1.java     |  118 ---
 .../tsfile/v1/file/metadata/ChunkMetadataV1.java   |  131 ---
 .../v1/file/metadata/TimeseriesMetadataForV1.java  |   42 -
 .../v1/file/metadata/TsDeviceMetadataIndexV1.java  |   78 --
 .../v1/file/metadata/TsDeviceMetadataV1.java       |   87 --
 .../iotdb/tsfile/v1/file/metadata/TsDigestV1.java  |   75 --
 .../tsfile/v1/file/metadata/TsFileMetadataV1.java  |  106 --
 .../metadata/statistics/BinaryStatisticsV1.java    |   84 --
 .../metadata/statistics/BooleanStatisticsV1.java   |   80 --
 .../metadata/statistics/DoubleStatisticsV1.java    |   79 --
 .../metadata/statistics/FloatStatisticsV1.java     |   79 --
 .../metadata/statistics/IntegerStatisticsV1.java   |   79 --
 .../file/metadata/statistics/LongStatisticsV1.java |   80 --
 .../v1/file/metadata/statistics/StatisticsV1.java  |  225 -----
 .../iotdb/tsfile/v1/file/utils/HeaderUtils.java    |  141 ---
 .../tsfile/v1/read/TsFileSequenceReaderForV1.java  |  409 --------
 .../tsfile/v2/file/footer/ChunkGroupFooterV2.java  |   86 ++
 .../iotdb/tsfile/v2/file/header/ChunkHeaderV2.java |  108 +++
 .../iotdb/tsfile/v2/file/header/PageHeaderV2.java  |   51 +
 .../tsfile/v2/file/metadata/ChunkMetadataV2.java   |   52 +
 .../file/metadata/MetadataIndexEntryV2.java}       |   22 +-
 .../v2/file/metadata/MetadataIndexNodeV2.java      |   47 +
 .../v2/file/metadata/TimeseriesMetadataV2.java     |   42 +
 .../tsfile/v2/file/metadata/TsFileMetadataV2.java  |   75 ++
 .../v2/file/metadata/statistics/StatisticsV2.java  |  104 ++
 .../tsfile/v2/read/TsFileSequenceReaderForV2.java  |  581 +++++++++++
 .../tsfile/v2/read/reader/page/PageReaderV2.java   |   85 ++
 .../apache/iotdb/tsfile/write/TsFileWriter.java    |    2 +-
 .../iotdb/tsfile/write/chunk/ChunkWriterImpl.java  |   78 +-
 .../iotdb/tsfile/write/chunk/IChunkWriter.java     |    6 -
 .../apache/iotdb/tsfile/write/page/PageWriter.java |   30 +-
 .../tsfile/write/schema/MeasurementSchema.java     |   38 +-
 .../write/writer/ForceAppendTsFileWriter.java      |    3 -
 .../tsfile/write/writer/LocalTsFileOutput.java     |    5 +
 .../write/writer/RestorableTsFileIOWriter.java     |    1 -
 .../iotdb/tsfile/write/writer/TsFileIOWriter.java  |   54 +-
 .../iotdb/tsfile/write/writer/TsFileOutput.java    |    8 +
 .../tsfile/encoding/decoder/IntRleDecoderTest.java |   83 +-
 .../encoding/decoder/LongRleDecoderTest.java       |   17 +-
 .../iotdb/tsfile/file/header/PageHeaderTest.java   |    2 +-
 .../metadata/statistics/DoubleStatisticsTest.java  |   10 +-
 .../metadata/statistics/FloatStatisticsTest.java   |   11 +-
 .../metadata/statistics/IntegerStatisticsTest.java |   10 +-
 .../metadata/statistics/LongStatisticsTest.java    |   15 +-
 .../iotdb/tsfile/file/metadata/utils/Utils.java    |   36 +-
 .../iotdb/tsfile/read/GetAllDevicesTest.java       |   14 +-
 .../tsfile/read/TsFileSequenceReaderTest.java      |   66 +-
 .../iotdb/tsfile/read/reader/PageReaderTest.java   |   54 +-
 .../utils/ReadWriteForEncodingUtilsTest.java       |   54 ++
 .../iotdb/tsfile/write/TsFileIOWriterTest.java     |   20 +-
 .../iotdb/tsfile/write/writer/PageWriterTest.java  |   54 +-
 .../write/writer/RestorableTsFileIOWriterTest.java |   15 +-
 140 files changed, 4555 insertions(+), 4046 deletions(-)

diff --git a/example/jdbc/src/main/java/org/apache/iotdb/JDBCExample.java b/example/jdbc/src/main/java/org/apache/iotdb/JDBCExample.java
index f7efb38..a547c51 100644
--- a/example/jdbc/src/main/java/org/apache/iotdb/JDBCExample.java
+++ b/example/jdbc/src/main/java/org/apache/iotdb/JDBCExample.java
@@ -28,6 +28,7 @@ import java.sql.SQLException;
 import java.sql.Statement;
 
 public class JDBCExample {
+
   public static void main(String[] args) throws ClassNotFoundException, SQLException {
     Class.forName("org.apache.iotdb.jdbc.IoTDBDriver");
     try (Connection connection = DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
@@ -38,15 +39,18 @@ public class JDBCExample {
 
       try {
         statement.execute("SET STORAGE GROUP TO root.sg1");
-        statement.execute("CREATE TIMESERIES root.sg1.d1.s1 WITH DATATYPE=INT64, ENCODING=RLE, COMPRESSOR=SNAPPY");
-        statement.execute("CREATE TIMESERIES root.sg1.d1.s2 WITH DATATYPE=INT64, ENCODING=RLE, COMPRESSOR=SNAPPY");
-        statement.execute("CREATE TIMESERIES root.sg1.d1.s3 WITH DATATYPE=INT64, ENCODING=RLE, COMPRESSOR=SNAPPY");
+        statement.execute(
+            "CREATE TIMESERIES root.sg1.d1.s1 WITH DATATYPE=INT64, ENCODING=RLE, COMPRESSOR=SNAPPY");
+        statement.execute(
+            "CREATE TIMESERIES root.sg1.d1.s2 WITH DATATYPE=INT64, ENCODING=RLE, COMPRESSOR=SNAPPY");
+        statement.execute(
+            "CREATE TIMESERIES root.sg1.d1.s3 WITH DATATYPE=INT64, ENCODING=RLE, COMPRESSOR=SNAPPY");
       } catch (IoTDBSQLException e) {
         System.out.println(e.getMessage());
       }
 
       for (int i = 0; i <= 100; i++) {
-        statement.addBatch("insert into root.sg1.d1(timestamp, s1, s2, s3) values("+ i + "," + 1 + "," + 1 + "," + 1 + ")");
+        statement.addBatch(prepareInsertStatment(i));
       }
       statement.executeBatch();
       statement.clearBatch();
@@ -55,10 +59,11 @@ public class JDBCExample {
       outputResult(resultSet);
       resultSet = statement.executeQuery("select count(*) from root");
       outputResult(resultSet);
-      resultSet = statement.executeQuery("select count(*) from root where time >= 1 and time <= 100 group by ([0, 100), 20ms, 20ms)");
+      resultSet = statement.executeQuery(
+          "select count(*) from root where time >= 1 and time <= 100 group by ([0, 100), 20ms, 20ms)");
       outputResult(resultSet);
-    } catch (IoTDBSQLException e){
-        System.out.println(e.getMessage());
+    } catch (IoTDBSQLException e) {
+      System.out.println(e.getMessage());
     }
   }
 
@@ -85,4 +90,10 @@ public class JDBCExample {
       System.out.println("--------------------------\n");
     }
   }
+  
+  private static String prepareInsertStatment(int time) {
+    return "insert into root.sg1.d1(timestamp, s1, s2, s3) values(" + time + "," + 1 + "," + 1 + ","
+        + 1 + ")";
+  }
+
 }
\ No newline at end of file
diff --git a/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java b/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java
index da5bae6..7fca807 100644
--- a/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java
+++ b/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java
@@ -26,7 +26,7 @@ import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.encoding.decoder.Decoder;
 import org.apache.iotdb.tsfile.file.MetaMarker;
-import org.apache.iotdb.tsfile.file.footer.ChunkGroupFooter;
+import org.apache.iotdb.tsfile.file.header.ChunkGroupHeader;
 import org.apache.iotdb.tsfile.file.header.ChunkHeader;
 import org.apache.iotdb.tsfile.file.header.PageHeader;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
@@ -39,67 +39,77 @@ import org.apache.iotdb.tsfile.read.reader.page.PageReader;
 
 public class TsFileSequenceRead {
 
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+  @SuppressWarnings({"squid:S3776", "squid:S106"}) // Suppress high Cognitive Complexity and Standard outputs warning
   public static void main(String[] args) throws IOException {
     String filename = "test.tsfile";
     if (args.length >= 1) {
       filename = args[0];
     }
     try (TsFileSequenceReader reader = new TsFileSequenceReader(filename)) {
-      System.out.println("file length: " + FSFactoryProducer.getFSFactory().getFile(filename).length());
+      System.out
+          .println("file length: " + FSFactoryProducer.getFSFactory().getFile(filename).length());
       System.out.println("file magic head: " + reader.readHeadMagic());
       System.out.println("file magic tail: " + reader.readTailMagic());
       System.out.println("Level 1 metadata position: " + reader.getFileMetadataPos());
       System.out.println("Level 1 metadata size: " + reader.getFileMetadataSize());
       // Sequential reading of one ChunkGroup now follows this order:
-      // first SeriesChunks (headers and data) in one ChunkGroup, then the CHUNK_GROUP_FOOTER
+      // first the CHUNK_GROUP_HEADER, then SeriesChunks (headers and data) in one ChunkGroup
       // Because we do not know how many chunks a ChunkGroup may have, we should read one byte (the marker) ahead and
       // judge accordingly.
-      reader.position((long) TSFileConfig.MAGIC_STRING.getBytes().length + TSFileConfig.VERSION_NUMBER
-              .getBytes().length);
+      reader.position((long) TSFileConfig.MAGIC_STRING.getBytes().length + 1);
       System.out.println("[Chunk Group]");
       System.out.println("position: " + reader.position());
       byte marker;
       while ((marker = reader.readMarker()) != MetaMarker.SEPARATOR) {
         switch (marker) {
           case MetaMarker.CHUNK_HEADER:
+          case MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER:
             System.out.println("\t[Chunk]");
+            System.out.println("\tchunk type: " + marker);
             System.out.println("\tposition: " + reader.position());
-            ChunkHeader header = reader.readChunkHeader();
+            ChunkHeader header = reader.readChunkHeader(marker);
             System.out.println("\tMeasurement: " + header.getMeasurementID());
             Decoder defaultTimeDecoder = Decoder.getDecoderByType(
-                    TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder()),
-                    TSDataType.INT64);
+                TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder()),
+                TSDataType.INT64);
             Decoder valueDecoder = Decoder
-                    .getDecoderByType(header.getEncodingType(), header.getDataType());
-            for (int j = 0; j < header.getNumOfPages(); j++) {
+                .getDecoderByType(header.getEncodingType(), header.getDataType());
+            int dataSize = header.getDataSize();
+            while (dataSize > 0) {
               valueDecoder.reset();
               System.out.println("\t\t[Page]\n \t\tPage head position: " + reader.position());
-              PageHeader pageHeader = reader.readPageHeader(header.getDataType());
+              PageHeader pageHeader = reader.readPageHeader(header.getDataType(),
+                  header.getChunkType() == MetaMarker.CHUNK_HEADER);
               System.out.println("\t\tPage data position: " + reader.position());
-              System.out.println("\t\tpoints in the page: " + pageHeader.getNumOfValues());
               ByteBuffer pageData = reader.readPage(pageHeader, header.getCompressionType());
               System.out
-                      .println("\t\tUncompressed page data size: " + pageHeader.getUncompressedSize());
+                  .println("\t\tUncompressed page data size: " + pageHeader.getUncompressedSize());
               PageReader reader1 = new PageReader(pageData, header.getDataType(), valueDecoder,
-                      defaultTimeDecoder, null);
+                  defaultTimeDecoder, null);
               BatchData batchData = reader1.getAllSatisfiedPageData();
+              if (header.getChunkType() == MetaMarker.CHUNK_HEADER) {
+                System.out.println("\t\tpoints in the page: " + pageHeader.getNumOfValues());
+              } else {
+                System.out.println("\t\tpoints in the page: " + batchData.length());
+              }
               while (batchData.hasCurrent()) {
                 System.out.println(
-                        "\t\t\ttime, value: " + batchData.currentTime() + ", " + batchData
-                                .currentValue());
+                    "\t\t\ttime, value: " + batchData.currentTime() + ", " + batchData
+                        .currentValue());
                 batchData.next();
               }
+              dataSize -= pageHeader.getSerializedPageSize();
             }
             break;
-          case MetaMarker.CHUNK_GROUP_FOOTER:
-            System.out.println("Chunk Group Footer position: " + reader.position());
-            ChunkGroupFooter chunkGroupFooter = reader.readChunkGroupFooter();
-            System.out.println("device: " + chunkGroupFooter.getDeviceID());
+          case MetaMarker.CHUNK_GROUP_HEADER:
+            System.out.println("Chunk Group Header position: " + reader.position());
+            ChunkGroupHeader chunkGroupHeader = reader.readChunkGroupHeader();
+            System.out.println("device: " + chunkGroupHeader.getDeviceID());
             break;
-          case MetaMarker.VERSION:
-            long version = reader.readVersion();
-            System.out.println("version: " + version);
+          case MetaMarker.OPERATION_INDEX_RANGE:
+            reader.readPlanIndex();
+            System.out.println("minPlanIndex: " + reader.getMinPlanIndex());
+            System.out.println("maxPlanIndex: " + reader.getMaxPlanIndex());
             break;
           default:
             MetaMarker.handleUnexpectedMarker(marker);
@@ -108,8 +118,8 @@ public class TsFileSequenceRead {
       System.out.println("[Metadata]");
       for (String device : reader.getAllDevices()) {
         Map<String, List<ChunkMetadata>> seriesMetaData = reader.readChunkMetadataInDevice(device);
-        System.out.println(String
-                .format("\t[Device]Device %s, Number of Measurements %d", device, seriesMetaData.size()));
+        System.out.printf("\t[Device]Device %s, Number of Measurements %d%n", device,
+            seriesMetaData.size());
         for (Map.Entry<String, List<ChunkMetadata>> serie : seriesMetaData.entrySet()) {
           System.out.println("\t\tMeasurement:" + serie.getKey());
           for (ChunkMetadata chunkMetadata : serie.getValue()) {
diff --git a/hadoop/src/main/java/org/apache/iotdb/hadoop/fileSystem/HDFSInput.java b/hadoop/src/main/java/org/apache/iotdb/hadoop/fileSystem/HDFSInput.java
index 0b7da82..7e6df31 100644
--- a/hadoop/src/main/java/org/apache/iotdb/hadoop/fileSystem/HDFSInput.java
+++ b/hadoop/src/main/java/org/apache/iotdb/hadoop/fileSystem/HDFSInput.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.iotdb.tsfile.read.reader.TsFileInput;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 public class HDFSInput implements TsFileInput {
 
@@ -126,4 +127,14 @@ public class HDFSInput implements TsFileInput {
   public int readInt() throws IOException {
     throw new UnsupportedOperationException();
   }
+
+  @Override
+  public synchronized String readVarIntString(long position) throws IOException {
+    long srcPosition = fsDataInputStream.getPos();
+
+    fsDataInputStream.seek(position);
+    String res = ReadWriteIOUtils.readVarIntString(fsDataInputStream);
+    fsDataInputStream.seek(srcPosition);
+    return res;
+  }
 }
diff --git a/hadoop/src/main/java/org/apache/iotdb/hadoop/fileSystem/HDFSOutput.java b/hadoop/src/main/java/org/apache/iotdb/hadoop/fileSystem/HDFSOutput.java
index aec1a58..2920311 100644
--- a/hadoop/src/main/java/org/apache/iotdb/hadoop/fileSystem/HDFSOutput.java
+++ b/hadoop/src/main/java/org/apache/iotdb/hadoop/fileSystem/HDFSOutput.java
@@ -61,6 +61,11 @@ public class HDFSOutput implements TsFileOutput {
     fsDataOutputStream.write(b);
   }
 
+  @Override
+  public void write(byte b) throws IOException {
+    fsDataOutputStream.write(b);
+  }
+
   public void write(ByteBuffer b) throws IOException {
     throw new UnsupportedOperationException("Unsupported operation.");
   }
diff --git a/hadoop/src/main/java/org/apache/iotdb/hadoop/tsfile/record/HDFSTSRecord.java b/hadoop/src/main/java/org/apache/iotdb/hadoop/tsfile/record/HDFSTSRecord.java
index f996094..a5d380e 100644
--- a/hadoop/src/main/java/org/apache/iotdb/hadoop/tsfile/record/HDFSTSRecord.java
+++ b/hadoop/src/main/java/org/apache/iotdb/hadoop/tsfile/record/HDFSTSRecord.java
@@ -100,7 +100,7 @@ public class HDFSTSRecord implements Writable {
     out.write(deviceId.getBytes(StandardCharsets.UTF_8));
     out.writeInt(dataPointList.size());
     for (DataPoint dataPoint : dataPointList) {
-      out.writeShort(dataPoint.getType().serialize());
+      out.write(dataPoint.getType().serialize());
       out.writeInt(dataPoint.getMeasurementId().getBytes(StandardCharsets.UTF_8).length);
       out.write(dataPoint.getMeasurementId().getBytes(StandardCharsets.UTF_8));
       switch (dataPoint.getType()) {
@@ -139,7 +139,7 @@ public class HDFSTSRecord implements Writable {
     List<DataPoint> dataPoints = new ArrayList<>(len);
 
     for (int i = 0; i < len; i++) {
-      TSDataType dataType = TSDataType.deserialize(in.readShort());
+      TSDataType dataType = TSDataType.deserialize(in.readByte());
       int lenOfMeasurementId = in.readInt();
       byte[] c = new byte[lenOfMeasurementId];
       in.readFully(c);
diff --git a/server/file-changelists/TsFileMods-changelist.md b/server/file-changelists/TsFileMods-changelist.md
index 4c942e2..08dd78f 100644
--- a/server/file-changelists/TsFileMods-changelist.md
+++ b/server/file-changelists/TsFileMods-changelist.md
@@ -19,6 +19,17 @@
 
 -->
 
+# 0.11.x -> 0.12.x
+
+Last updated on 2021-1-15 by Haonan Hou.
+## Update:
+| Latest Changes                     |
+| ---------------------------------- |
+| Remove versionInfo in Tsfile and get rid of versions in memtable |
+| Change version in mods file to TsFile offset |
+
+No change
+
 # 0.10.x -> 0.11.x
 
 Last updated on 2020-11-25 by Haonan Hou.
diff --git a/server/file-changelists/system.properties-changelist.md b/server/file-changelists/system.properties-changelist.md
index 0c4a148..c0f306b 100644
--- a/server/file-changelists/system.properties-changelist.md
+++ b/server/file-changelists/system.properties-changelist.md
@@ -18,6 +18,15 @@
     under the License.
 
 -->
+# 0.11.x -> 0.12.x
+
+Last updated on 2021-1-18 by Haonan Hou
+
+## Update:
+| Latest Changes                     |
+| ---------------------------------- |
+| Change iotdb_version=0.11.x to 0.12.x  |
+
 # 0.10.x -> 0.11.x
 
 Last updated on 2020-11-5 by Haonan Hou
diff --git a/server/src/assembly/resources/tools/upgrade/config.properties b/server/src/assembly/resources/tools/upgrade/config.properties
deleted file mode 100644
index 7de016b..0000000
--- a/server/src/assembly/resources/tools/upgrade/config.properties
+++ /dev/null
@@ -1,27 +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.
-#
-
-# how many thread will be set up to perform offline upgrade tasks
-upgrade_thread_num=10
-# Used to specify the data dirs that need to be upgraded
-# Commas could be used to separate the folder paths if there are more than one data dir that needs to be upgraded
-old_version_data_dirs=/Users/tianyu/3sjar/data
-# Used to specify the upgrading data dirs
-# It is worth noting that the length of the old_version_data_dirs and new_version_data_dirs parameters should be equal.
-new_version_data_dirs=/Users/tianyu/3sjar/data1
\ No newline at end of file
diff --git a/server/src/assembly/resources/tools/upgrade/offline-upgrade.bat b/server/src/assembly/resources/tools/upgrade/offline-upgrade.bat
deleted file mode 100644
index 1561e5d..0000000
--- a/server/src/assembly/resources/tools/upgrade/offline-upgrade.bat
+++ /dev/null
@@ -1,67 +0,0 @@
-@REM
-@REM Licensed to the Apache Software Foundation (ASF) under one
-@REM or more contributor license agreements.  See the NOTICE file
-@REM distributed with this work for additional information
-@REM regarding copyright ownership.  The ASF licenses this file
-@REM to you under the Apache License, Version 2.0 (the
-@REM "License"); you may not use this file except in compliance
-@REM with the License.  You may obtain a copy of the License at
-@REM
-@REM     http://www.apache.org/licenses/LICENSE-2.0
-@REM
-@REM Unless required by applicable law or agreed to in writing,
-@REM software distributed under the License is distributed on an
-@REM "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-@REM KIND, either express or implied.  See the License for the
-@REM specific language governing permissions and limitations
-@REM under the License.
-@REM
-
-
-@echo off
-if "%OS%" == "Windows_NT" setlocal
-
-pushd %~dp0..\..
-if NOT DEFINED IOTDB_HOME set IOTDB_HOME=%CD%
-popd
-
-set IOTDB_CONF=%IOTDB_HOME%\conf
-
-if NOT DEFINED MAIN_CLASS set MAIN_CLASS=org.apache.iotdb.db.tools.upgrade.OfflineUpgradeTool
-if NOT DEFINED JAVA_HOME goto :err
-
-@REM -----------------------------------------------------------------------------
-@REM JVM Opts we'll use in legacy run or installation
-set JAVA_OPTS=-ea^
- -Dlogback.configurationFile="%IOTDB_CONF%\logback-tool.xml"^
- -DIOTDB_HOME=%IOTDB_HOME%
-
-@REM ***** CLASSPATH library setting *****
-@REM Ensure that any user defined CLASSPATH variables are not used on startup
-set CLASSPATH="%IOTDB_HOME%\lib"
-
-@REM For each jar in the IOTDB_HOME lib directory call append to build the CLASSPATH variable.
-for %%i in ("%IOTDB_HOME%\lib\iotdb-*.jar") do call :append "%%i"
-goto okClasspath
-
-:append
-set CLASSPATH=%CLASSPATH%;%1
-goto :eof
-
-@REM -----------------------------------------------------------------------------
-:okClasspath
-
-"%JAVA_HOME%\bin\java" %JAVA_OPTS% %JAVA_OPTS% -cp "%CLASSPATH%" %MAIN_CLASS% %*
-
-goto finally
-
-
-:err
-echo JAVA_HOME environment variable must be set!
-pause
-
-
-@REM -----------------------------------------------------------------------------
-:finally
-
-ENDLOCAL
\ No newline at end of file
diff --git a/server/src/assembly/resources/tools/upgrade/offline-upgrade.sh b/server/src/assembly/resources/tools/upgrade/offline-upgrade.sh
deleted file mode 100755
index 1be56a1..0000000
--- a/server/src/assembly/resources/tools/upgrade/offline-upgrade.sh
+++ /dev/null
@@ -1,47 +0,0 @@
-#!/bin/sh
-#
-# 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.
-#
-
-if [ -z "${IOTDB_HOME}" ]; then
-  export IOTDB_HOME="$(cd "`dirname "$0"`"/../..; pwd)"
-fi
-
-IOTDB_CONF=${IOTDB_HOME}/conf
-
-CLASSPATH=""
-for f in ${IOTDB_HOME}/lib/*.jar; do
-  CLASSPATH=${CLASSPATH}":"$f
-done
-
-MAIN_CLASS=org.apache.iotdb.db.tools.upgrade.OfflineUpgradeTool
-
-if [ -n "$JAVA_HOME" ]; then
-    for java in "$JAVA_HOME"/bin/amd64/java "$JAVA_HOME"/bin/java; do
-        if [ -x "$java" ]; then
-            JAVA="$java"
-            break
-        fi
-    done
-else
-    JAVA=java
-fi
-
-iotdb_parms="-Dlogback.configurationFile=${IOTDB_CONF}/logback-tool.xml"
-
-exec "$JAVA" $iotdb_parms -cp "$CLASSPATH" "$MAIN_CLASS" "$@"
\ No newline at end of file
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java
index f19db87..00e76e1 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java
@@ -25,19 +25,25 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.nio.file.Files;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
+import java.util.Set;
+
 import org.apache.commons.io.FileUtils;
 import org.apache.iotdb.db.conf.directories.DirectoryManager;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.modification.ModificationFile;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.utils.FilePathUtils;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
 import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -47,10 +53,13 @@ public class IoTDBConfigCheck {
 
   private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
 
+  private FSFactory fsFactory = FSFactoryProducer.getFSFactory();
+
   // this file is located in data/system/schema/system.properties
   // If user delete folder "data", system.properties can reset.
   private static final String PROPERTIES_FILE_NAME = "system.properties";
   private static final String SCHEMA_DIR = config.getSchemaDir();
+  private static final String SYSTEM_DIR = config.getSystemDir();
   private static final String WAL_DIR = config.getWalDir();
 
   private File propertiesFile;
@@ -189,31 +198,32 @@ public class IoTDBConfigCheck {
             inputStream, TSFileConfig.STRING_CHARSET)) {
       properties.load(inputStreamReader);
     }
-    // check whether upgrading from <=v0.10 to v0.12
-    if (!properties.containsKey(IOTDB_VERSION_STRING) ||
-      properties.getProperty(IOTDB_VERSION_STRING).startsWith("0.10")) {
-      logger.error("DO NOT UPGRADE IoTDB from v0.10 or lower version to v0.12!"
-          + " Please upgrade to v0.11 first");
+    // check whether upgrading from <=v0.9 to v0.12
+    if (!properties.containsKey(IOTDB_VERSION_STRING)) {
+      logger.error("DO NOT UPGRADE IoTDB from v0.9 or lower version to v0.12!"
+          + " Please upgrade to v0.10 first");
       System.exit(-1);
     }
-
-    // check whether upgrading from v0.11 to v0.12
-    if (properties.getProperty(IOTDB_VERSION_STRING).startsWith("0.11")) {
-      logger.info("Upgrading IoTDB from v0.11 to v0.12, checking files...");
+    // check whether upgrading from v0.10 or v0.11 to v0.12
+    String versionString = properties.getProperty(IOTDB_VERSION_STRING);
+    if (versionString.startsWith("0.10") || versionString.startsWith("0.11")) {
+      logger.info("Upgrading IoTDB from {} to {}, checking files...",
+          versionString, IoTDBConstant.VERSION);
       checkUnClosedTsFileV2();
+      moveTsFileV2();
+      moveVersionFile();
+      logger.info("checking files successful");
+      MLogWriter.upgradeMLog();
+      logger.info("Mlog upgraded!");
+      logger.info("Start upgrading Version-2 TsFiles...");
       upgradePropertiesFile();
-      logger.info("Upgrade to IoTDB v0.12 successfully!");
     }
-
-    MLogWriter.upgradeMLog();
-
     checkProperties();
   }
 
   /**
-   * upgrade 0.11 properties to 0.12 properties
+   * upgrade 0.10 or 0.11 properties to 0.12 properties
    */
-  @SuppressWarnings("unused")
   private void upgradePropertiesFile()
       throws IOException {
     // create an empty tmpPropertiesFile
@@ -314,14 +324,13 @@ public class IoTDBConfigCheck {
   }
 
   /**
-   * ensure all TsFiles are closed in 0.11 when starting 0.12
+   * ensure all TsFiles are closed when starting 0.12
    */
-  @SuppressWarnings("unused")
   private void checkUnClosedTsFileV2() {
     if (SystemFileFactory.INSTANCE.getFile(WAL_DIR).isDirectory()
       && SystemFileFactory.INSTANCE.getFile(WAL_DIR).list().length != 0) {
-      logger.error("Unclosed Version-2 TsFile detected, please run 'flush' on v0.11 IoTDB"
-        + " before upgrading to v0.12");
+      logger.error("Unclosed Version-2 TsFile detected, please stop insertion, then run 'flush' "
+          + "on v0.10 or v0.11 IoTDB before upgrading to v0.12");
       System.exit(-1);
     }
     checkUnClosedTsFileV2InFolders(DirectoryManager.getInstance().getAllSequenceFileFolders());
@@ -330,7 +339,7 @@ public class IoTDBConfigCheck {
 
   private void checkUnClosedTsFileV2InFolders(List<String> folders) {
     for (String baseDir : folders) {
-      File fileFolder = FSFactoryProducer.getFSFactory().getFile(baseDir);
+      File fileFolder = fsFactory.getFile(baseDir);
       if (!fileFolder.isDirectory()) {
         continue;
       }
@@ -342,14 +351,150 @@ public class IoTDBConfigCheck {
           if (!partitionDir.isDirectory()) {
             continue;
           }
-          File[] tsfiles = FSFactoryProducer.getFSFactory()
-            .listFilesBySuffix(partitionDir.toString(), TsFileConstant.TSFILE_SUFFIX);
-          File[] resources = FSFactoryProducer.getFSFactory()
-            .listFilesBySuffix(partitionDir.toString(), TsFileResource.RESOURCE_SUFFIX);
+          File[] tsfiles = fsFactory
+              .listFilesBySuffix(partitionDir.toString(), TsFileConstant.TSFILE_SUFFIX);
+          File[] resources = fsFactory
+              .listFilesBySuffix(partitionDir.toString(), TsFileResource.RESOURCE_SUFFIX);
           if (tsfiles.length != resources.length) {
-            logger.error("Unclosed Version-2 TsFile detected, please run 'flush' on v0.10 IoTDB"
-              + " before upgrading to v0.11");
-            System.exit(-1);
+            File[] zeroLevelTsFiles = fsFactory
+                .listFilesBySuffix(partitionDir.toString(), "0" + TsFileConstant.TSFILE_SUFFIX);
+            File[] zeroLevelResources = fsFactory
+                .listFilesBySuffix(partitionDir.toString(), "0" + TsFileResource.RESOURCE_SUFFIX);
+            if (zeroLevelTsFiles.length != zeroLevelResources.length) {
+              logger.error("Unclosed Version-2 TsFile detected, please stop insertion, then run 'flush' "
+                  + "on v0.11 IoTDB before upgrading to v0.12");
+              System.exit(-1);
+            }
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * If upgrading from v0.11.2 to v0.12, there may be some unsealed merging files.
+   * We have to delete these files before upgrading.
+   */
+  private void deleteMergingTsFiles(File[] tsfiles, File[] resources) {
+    Set<String> resourcesSet = new HashSet<>();
+    for (File resource : resources) {
+      resourcesSet.add(resource.getName());
+    }
+    for (File tsfile : tsfiles) {
+      if (!resourcesSet.contains(tsfile.getName() + TsFileResource.RESOURCE_SUFFIX)) {
+        try {
+          Files.delete(tsfile.toPath());
+        } catch (Exception e) {
+          logger.error("Failed to delete merging tsfile {} ", tsfile, e);
+          System.exit(-1);
+        }
+      }
+    }
+  }
+
+  private void moveTsFileV2() {
+    moveFileToUpgradeFolder(DirectoryManager.getInstance().getAllSequenceFileFolders());
+    moveFileToUpgradeFolder(DirectoryManager.getInstance().getAllUnSequenceFileFolders());
+    logger.info("Move version-2 TsFile successfully");
+  }
+
+  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+  private void moveFileToUpgradeFolder(List<String> folders) {
+    for (String baseDir : folders) {
+      File fileFolder = fsFactory.getFile(baseDir);
+      if (!fileFolder.isDirectory()) {
+        continue;
+      }
+      for (File storageGroup : fileFolder.listFiles()) {
+        if (!storageGroup.isDirectory()) {
+          continue;
+        }
+        for (File partitionDir : storageGroup.listFiles()) {
+          if (!partitionDir.isDirectory()) {
+            continue;
+          }
+          File[] oldTsfileArray = fsFactory
+              .listFilesBySuffix(partitionDir.getAbsolutePath(), TsFileConstant.TSFILE_SUFFIX);
+          File[] oldResourceFileArray = fsFactory
+              .listFilesBySuffix(partitionDir.getAbsolutePath(), TsFileResource.RESOURCE_SUFFIX);
+          File[] oldModificationFileArray = fsFactory
+              .listFilesBySuffix(partitionDir.getAbsolutePath(), ModificationFile.FILE_SUFFIX);
+          deleteMergingTsFiles(oldTsfileArray, oldResourceFileArray);
+          // move the old files to upgrade folder if exists
+          if (oldTsfileArray.length +
+              oldResourceFileArray.length +
+              oldModificationFileArray.length != 0) {
+            // create upgrade directory if not exist
+            File upgradeFolder = fsFactory.getFile(
+                partitionDir, IoTDBConstant.UPGRADE_FOLDER_NAME);
+            if (upgradeFolder.mkdirs()) {
+              logger.info("Upgrade Directory {} doesn't exist, create it",
+                  upgradeFolder.getPath());
+            } else if (!upgradeFolder.exists()) {
+              logger.error("Create upgrade Directory {} failed",
+                  upgradeFolder.getPath());
+            }
+            // move .tsfile to upgrade folder
+            for (File file : oldTsfileArray) {
+              if (!file.renameTo(fsFactory.getFile(upgradeFolder, file.getName()))) {
+                logger.error("Failed to move tsfile {} to upgrade folder", file);
+              }
+            }
+            // move .resource to upgrade folder
+            for (File file : oldResourceFileArray) {
+              if (!file.renameTo(fsFactory.getFile(upgradeFolder, file.getName()))) {
+                logger.error("Failed to move resource {} to upgrade folder", file);
+              }
+            }
+            // move .mods to upgrade folder
+            for (File file : oldModificationFileArray) {
+              if (!file.renameTo(fsFactory.getFile(upgradeFolder, file.getName()))) {
+                logger.error("Failed to move mod file {} to upgrade folder", file);
+              }
+            }
+          }
+        }
+      }
+    }
+  }
+
+  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+  private void moveVersionFile() {
+    File sgDir = SystemFileFactory.INSTANCE.getFile(
+        FilePathUtils.regularizePath(SYSTEM_DIR) + "storage_groups");
+    if (sgDir.isDirectory()) {
+      for (File sg : sgDir.listFiles()) {
+        if (!sg.isDirectory()) {
+          continue;
+        }
+        for (File partition : sg.listFiles()) {
+          if (!partition.isDirectory()) {
+            continue;
+          }
+          File virtualSg = SystemFileFactory.INSTANCE.getFile(sg, "0");
+          if (!virtualSg.exists()) {
+            virtualSg.mkdir();
+          }
+          File newPartition =  SystemFileFactory.INSTANCE
+              .getFile(virtualSg, partition.getName());
+          if (!newPartition.exists()) {
+            newPartition.mkdir();
+          }
+          for (File versionFile : partition.listFiles()) {
+            if (versionFile.isDirectory()) {
+              continue;
+            }
+            if (!versionFile.renameTo(SystemFileFactory.INSTANCE
+                .getFile(newPartition, versionFile.getName()))) {
+              logger.error("Rename {} failed", versionFile);
+            }
+          }
+          if (partition.listFiles().length == 0) {
+            try {
+              Files.delete(partition.toPath());
+            } catch (IOException e) {
+              logger.error("Delete {} failed", partition);
+            }
           }
         }
       }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/cache/ChunkCache.java b/server/src/main/java/org/apache/iotdb/db/engine/cache/ChunkCache.java
index 6fccb8b..c621a26 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/cache/ChunkCache.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/cache/ChunkCache.java
@@ -87,7 +87,7 @@ public class ChunkCache {
     if (!CACHE_ENABLE) {
       Chunk chunk = reader.readMemChunk(chunkMetaData);
       return new Chunk(chunk.getHeader(), chunk.getData().duplicate(),
-          chunk.getDeleteIntervalList());
+          chunk.getDeleteIntervalList(), chunkMetaData.getStatistics());
     }
 
     cacheRequestNum.incrementAndGet();
@@ -121,7 +121,8 @@ public class ChunkCache {
     if (config.isDebugOn()) {
       DEBUG_LOGGER.info("get chunk from cache whose meta data is: " + chunkMetaData);
     }
-    return new Chunk(chunk.getHeader(), chunk.getData().duplicate(), chunk.getDeleteIntervalList());
+    return new Chunk(chunk.getHeader(), chunk.getData().duplicate(), chunk.getDeleteIntervalList(),
+        chunkMetaData.getStatistics());
   }
 
   private void printCacheLog(boolean isHit) {
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/level/LevelCompactionTsFileManagement.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/level/LevelCompactionTsFileManagement.java
index 940c841..ec1ebdd 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/level/LevelCompactionTsFileManagement.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/level/LevelCompactionTsFileManagement.java
@@ -20,7 +20,6 @@
 package org.apache.iotdb.db.engine.compaction.level;
 
 import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SEPARATOR;
-import static org.apache.iotdb.db.engine.compaction.no.NoCompactionTsFileManagement.compareFileName;
 import static org.apache.iotdb.db.engine.compaction.utils.CompactionLogger.COMPACTION_LOG_NAME;
 import static org.apache.iotdb.db.engine.compaction.utils.CompactionLogger.SOURCE_NAME;
 import static org.apache.iotdb.db.engine.compaction.utils.CompactionLogger.TARGET_NAME;
@@ -43,13 +42,14 @@ import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.cache.ChunkMetadataCache;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.engine.compaction.TsFileManagement;
 import org.apache.iotdb.db.engine.compaction.utils.CompactionLogAnalyzer;
 import org.apache.iotdb.db.engine.compaction.utils.CompactionLogger;
 import org.apache.iotdb.db.engine.compaction.utils.CompactionUtils;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.query.control.FileReaderManager;
+import org.apache.iotdb.db.utils.TestOnly;
 import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
 import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
 import org.slf4j.Logger;
@@ -690,4 +690,9 @@ public class LevelCompactionTsFileManagement extends TsFileManagement {
     logger.error("cannot get tsfile resource path: {}", filePath);
     throw new IOException();
   }
+
+  @TestOnly
+  public Map<Long, List<SortedSet<TsFileResource>>> getSequenceTsFileResources() {
+    return sequenceTsFileResources;
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/selector/MaxFileMergeFileSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/merge/selector/MaxFileMergeFileSelector.java
index 58c62c0..b85d894 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/selector/MaxFileMergeFileSelector.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/merge/selector/MaxFileMergeFileSelector.java
@@ -31,7 +31,6 @@ import org.apache.iotdb.db.engine.merge.manage.MergeResource;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.MergeException;
 import org.apache.iotdb.db.utils.MergeUtils;
-import org.apache.iotdb.db.utils.UpgradeUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -150,8 +149,7 @@ public class MaxFileMergeFileSelector implements IMergeFileSelector {
       // select next unseq files
       TsFileResource unseqFile = resource.getUnseqFiles().get(unseqIndex);
 
-      if (seqSelectedNum != resource.getSeqFiles().size() && !UpgradeUtils
-          .isNeedUpgrade(unseqFile)) {
+      if (seqSelectedNum != resource.getSeqFiles().size()) {
         selectOverlappedSeqFiles(unseqFile);
       }
       boolean isClosed = checkClosed(unseqFile);
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessor.java b/server/src/main/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessor.java
index abc8c41..384860f 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessor.java
@@ -131,7 +131,7 @@ public class LocalTextModificationAccessor implements ModificationReader, Modifi
   /**
    * Decode a range deletion record. E.g. "DELETION,root.ln.wf01.wt01.temperature,111,100,300"
    * the index of field endTimestamp is length - 1, startTimestamp is length - 2,
-   * versionNum is length - 3. Fields in index range [1, length -3) all belong
+   * TsFile offset is length - 3. Fields in index range [1, length -3) all belong
    * to a timeseries path in case when the path contains comma.
    */
   private static Deletion decodeDeletion(String[] fields) throws IOException {
@@ -142,9 +142,9 @@ public class LocalTextModificationAccessor implements ModificationReader, Modifi
     String path = "";
     long startTimestamp;
     long endTimestamp;
-    long versionNum;
+    long tsFileOffset;
     try {
-      versionNum = Long.parseLong(fields[fields.length - 3]);
+      tsFileOffset = Long.parseLong(fields[fields.length - 3]);
     } catch (NumberFormatException e) {
       return decodePointDeletion(fields);
     }
@@ -158,7 +158,7 @@ public class LocalTextModificationAccessor implements ModificationReader, Modifi
     try {
       String[] pathArray = Arrays.copyOfRange(fields, 1, fields.length - 3);
       path = String.join(SEPARATOR, pathArray);
-      return new Deletion(new PartialPath(path), versionNum, startTimestamp, endTimestamp);
+      return new Deletion(new PartialPath(path), tsFileOffset, startTimestamp, endTimestamp);
     } catch (IllegalPathException e) {
       throw new IOException("Invalid series path: " + path);
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
index a1ff15d..7d3f114 100755
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
@@ -25,6 +25,7 @@ import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFF
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.Files;
 import java.nio.ByteBuffer;
 import java.nio.MappedByteBuffer;
 import java.util.ArrayList;
@@ -43,6 +44,7 @@ import java.util.Set;
 import java.util.TreeMap;
 import java.util.concurrent.Executors;
 import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReadWriteLock;
@@ -66,6 +68,8 @@ import org.apache.iotdb.db.engine.modification.Deletion;
 import org.apache.iotdb.db.engine.modification.ModificationFile;
 import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
 import org.apache.iotdb.db.engine.storagegroup.timeindex.DeviceTimeIndex;
+import org.apache.iotdb.db.engine.upgrade.UpgradeCheckStatus;
+import org.apache.iotdb.db.engine.upgrade.UpgradeLog;
 import org.apache.iotdb.db.engine.version.SimpleFileVersionController;
 import org.apache.iotdb.db.engine.version.VersionController;
 import org.apache.iotdb.db.exception.BatchProcessException;
@@ -88,10 +92,10 @@ import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.control.QueryFileManager;
 import org.apache.iotdb.db.service.IoTDB;
-import org.apache.iotdb.db.service.UpgradeSevice;
 import org.apache.iotdb.db.utils.CopyOnReadLinkedList;
 import org.apache.iotdb.db.utils.MmapUtil;
 import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.db.utils.UpgradeUtils;
 import org.apache.iotdb.db.writelog.recover.TsFileRecoverPerformer;
 import org.apache.iotdb.rpc.RpcUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
@@ -131,7 +135,6 @@ public class StorageGroupProcessor {
 
   public static final String MERGING_MODIFICATION_FILE_NAME = "merge.mods";
   private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
-  private static final String FAIL_TO_UPGRADE_FOLDER = "Failed to move {} to upgrade folder";
   private static final Logger DEBUG_LOGGER = LoggerFactory.getLogger("QUERY_DEBUG");
 
   /**
@@ -186,6 +189,8 @@ public class StorageGroupProcessor {
   private List<TsFileResource> upgradeUnseqFileList = new LinkedList<>();
 
   private CopyOnReadLinkedList<TsFileProcessor> closingUnSequenceTsFileProcessor = new CopyOnReadLinkedList<>();
+
+  private AtomicInteger upgradeFileCount = new AtomicInteger();
   /*
    * time partition id -> map, which contains
    * device -> global latest timestamp of each device latestTimeForEachDevice caches non-flushed
@@ -412,6 +417,10 @@ public class StorageGroupProcessor {
       List<TsFileResource> oldUnseqTsFiles = unseqTsFilesPair.right;
       upgradeUnseqFileList.addAll(oldUnseqTsFiles);
 
+      if (upgradeSeqFileList.size() + upgradeUnseqFileList.size() != 0) {
+        upgradeFileCount.set(upgradeSeqFileList.size() + upgradeUnseqFileList.size());
+      }
+
       // split by partition so that we can find the last file of each partition and decide to
       // close it or not
       Map<Long, List<TsFileResource>> partitionTmpSeqTsFiles = splitResourcesByPartition(
@@ -452,6 +461,14 @@ public class StorageGroupProcessor {
         long partitionNum = resource.getTimePartition();
         updatePartitionFileVersion(partitionNum, resource.getVersion());
       }
+      for (TsFileResource resource : upgradeSeqFileList) {
+        long partitionNum = resource.getTimePartition();
+        updatePartitionFileVersion(partitionNum, resource.getVersion());
+      }
+      for (TsFileResource resource : upgradeUnseqFileList) {
+        long partitionNum = resource.getTimePartition();
+        updatePartitionFileVersion(partitionNum, resource.getVersion());
+      }
       updateLatestFlushedTime();
     } catch (IOException | MetadataException e) {
       throw new StorageGroupProcessorException(e);
@@ -517,11 +534,11 @@ public class StorageGroupProcessor {
             .put(deviceId, endTime);
         globalLatestFlushedTimeForEachDevice.put(deviceId, endTime);
 
-        // set all the covered partition's LatestFlushedTime to Long.MAX_VALUE
+        // set all the covered partition's LatestFlushedTime
         long partitionId = StorageEngine.getTimePartition(resource.getStartTime(deviceId));
         while (partitionId <= endTimePartitionId) {
           partitionLatestFlushedTimeForEachDevice.computeIfAbsent(partitionId, l -> new HashMap<>())
-              .put(deviceId, Long.MAX_VALUE);
+              .put(deviceId, endTime);
           if (!timePartitionIdVersionControllerMap.containsKey(partitionId)) {
             File directory = SystemFileFactory.INSTANCE
                 .getFile(storageGroupSysDir, String.valueOf(partitionId));
@@ -563,57 +580,13 @@ public class StorageGroupProcessor {
       // the process was interrupted before the merged files could be named
       continueFailedRenames(fileFolder, MERGE_SUFFIX);
 
-      File[] oldTsfileArray = fsFactory
-          .listFilesBySuffix(fileFolder.getAbsolutePath(), TSFILE_SUFFIX);
-      File[] oldResourceFileArray = fsFactory
-          .listFilesBySuffix(fileFolder.getAbsolutePath(), TsFileResource.RESOURCE_SUFFIX);
-      File[] oldModificationFileArray = fsFactory
-          .listFilesBySuffix(fileFolder.getAbsolutePath(), ModificationFile.FILE_SUFFIX);
-      File upgradeFolder = fsFactory.getFile(fileFolder, IoTDBConstant.UPGRADE_FOLDER_NAME);
-      // move the old files to upgrade folder if exists
-      if (oldTsfileArray.length != 0 || oldResourceFileArray.length != 0) {
-        // create upgrade directory if not exist
-        if (upgradeFolder.mkdirs()) {
-          logger.info("Upgrade Directory {} doesn't exist, create it",
-              upgradeFolder.getPath());
-        } else if (!upgradeFolder.exists()) {
-          logger.error("Create upgrade Directory {} failed",
-              upgradeFolder.getPath());
-        }
-        // move .tsfile to upgrade folder
-        for (File file : oldTsfileArray) {
-          if (!file.renameTo(fsFactory.getFile(upgradeFolder, file.getName()))) {
-            logger.error(FAIL_TO_UPGRADE_FOLDER, file);
-          }
-        }
-        // move .resource to upgrade folder
-        for (File file : oldResourceFileArray) {
-          if (!file.renameTo(fsFactory.getFile(upgradeFolder, file.getName()))) {
-            logger.error(FAIL_TO_UPGRADE_FOLDER, file);
-          }
-        }
-        // move .mods to upgrade folder
-        for (File file : oldModificationFileArray) {
-          if (!file.renameTo(fsFactory.getFile(upgradeFolder, file.getName()))) {
-            logger.error(FAIL_TO_UPGRADE_FOLDER, file);
-          }
-        }
-
-        Collections.addAll(upgradeFiles,
-            fsFactory.listFilesBySuffix(upgradeFolder.getAbsolutePath(), TSFILE_SUFFIX));
-      }
-      // if already move old files to upgradeFolder 
-      else if (upgradeFolder.exists()) {
-        Collections.addAll(upgradeFiles,
-            fsFactory.listFilesBySuffix(upgradeFolder.getAbsolutePath(), TSFILE_SUFFIX));
-      }
-
       File[] subFiles = fileFolder.listFiles();
       if (subFiles != null) {
         for (File partitionFolder : subFiles) {
           if (!partitionFolder.isDirectory()) {
             logger.warn("{} is not a directory.", partitionFolder.getAbsolutePath());
-          } else if (!partitionFolder.getName().equals(IoTDBConstant.UPGRADE_FOLDER_NAME)) {
+          } 
+          else if (!partitionFolder.getName().equals(IoTDBConstant.UPGRADE_FOLDER_NAME)) {
             // some TsFileResource may be being persisted when the system crashed, try recovering such
             // resources
             continueFailedRenames(partitionFolder, TEMP_SUFFIX);
@@ -625,6 +598,11 @@ public class StorageGroupProcessor {
             Collections.addAll(tsFiles,
                 fsFactory.listFilesBySuffix(partitionFolder.getAbsolutePath(), TSFILE_SUFFIX));
           }
+          else {
+            // collect old TsFiles for upgrading
+            Collections.addAll(upgradeFiles,
+                fsFactory.listFilesBySuffix(partitionFolder.getAbsolutePath(), TSFILE_SUFFIX));
+          }
         }
       }
 
@@ -638,7 +616,7 @@ public class StorageGroupProcessor {
       TsFileResource fileResource = new TsFileResource(f);
       fileResource.setClosed(true);
       // make sure the flush command is called before IoTDB is down.
-      fileResource.deserialize();
+      fileResource.deserializeFromOldFile();
       upgradeRet.add(fileResource);
     }
     return new Pair<>(ret, upgradeRet);
@@ -1566,6 +1544,11 @@ public class StorageGroupProcessor {
    */
   public void delete(PartialPath path, long startTime, long endTime, long planIndex)
       throws IOException {
+    // If there are still some old version tsfiles, the delete won't succeeded.
+    if (upgradeFileCount.get() != 0) {
+      throw new IOException("Delete failed. "
+          + "Please do not delete until the old files upgraded.");
+    }
     // TODO: how to avoid partial deletion?
     // FIXME: notice that if we may remove a SGProcessor out of memory, we need to close all opened
     //mod files in mergingModification, sequenceFileList, and unsequenceFileList
@@ -1816,7 +1799,7 @@ public class StorageGroupProcessor {
    * @return total num of the tsfiles which need to be upgraded in the storage group
    */
   public int countUpgradeFiles() {
-    return upgradeSeqFileList.size() + upgradeUnseqFileList.size();
+    return upgradeFileCount.get();
   }
 
   public void upgrade() {
@@ -1841,23 +1824,19 @@ public class StorageGroupProcessor {
               resource.getEndTime(device))
       );
     }
-    insertLock.writeLock().lock();
-    tsFileManagement.writeLock();
-    try {
-      if (tsFileResource.isSeq()) {
-        tsFileManagement.addAll(upgradedResources, true);
-        upgradeSeqFileList.remove(tsFileResource);
-      } else {
-        tsFileManagement.addAll(upgradedResources, false);
-        upgradeUnseqFileList.remove(tsFileResource);
+    upgradeFileCount.getAndAdd(-1);
+    // load all upgraded resources in this sg to tsFileManagement
+    if (upgradeFileCount.get() == 0) {
+      insertLock.writeLock().lock();  
+      tsFileManagement.writeLock();
+      try {
+        loadUpgradedResources(upgradeSeqFileList, true);
+        loadUpgradedResources(upgradeUnseqFileList, false);
+      } finally {
+        tsFileManagement.writeUnlock(); 
+        insertLock.writeLock().unlock();
       }
-    } finally {
-      tsFileManagement.writeUnlock();
-      insertLock.writeLock().unlock();
-    }
-
-    // after upgrade complete, update partitionLatestFlushedTimeForEachDevice
-    if (countUpgradeFiles() == 0) {
+      // after upgrade complete, update partitionLatestFlushedTimeForEachDevice
       for (Entry<Long, Map<String, Long>> entry : newlyFlushedPartitionLatestFlushedTimeForEachDevice
           .entrySet()) {
         long timePartitionId = entry.getKey();
@@ -1872,10 +1851,38 @@ public class StorageGroupProcessor {
           }
         }
       }
-      if (StorageEngine.getInstance().countUpgradeFiles() == 0) {
-        UpgradeSevice.getINSTANCE().stop();
+    }
+  }
+
+  private void loadUpgradedResources(List<TsFileResource> resources, boolean isseq) {
+    if (resources.isEmpty()) {
+      return;
+    }
+    for (TsFileResource resource : resources) {
+      try {
+        UpgradeUtils.moveUpgradedFiles(resource);
+        tsFileManagement.addAll(resource.getUpgradedResources(), isseq); 
+        // delete old TsFile and resource
+        resource.delete();
+        Files.deleteIfExists(fsFactory
+            .getFile(resource.getTsFile().toPath() + ModificationFile.FILE_SUFFIX).toPath());
+        UpgradeLog.writeUpgradeLogFile(
+            resource.getTsFile().getAbsolutePath() + "," + UpgradeCheckStatus.UPGRADE_SUCCESS);
+      } catch (IOException e) {
+        logger.error("Unable to load {}, caused by ", resource, e);
+      }
+    }
+    // delete upgrade folder when it is empty
+    if (resources.get(0).getTsFile().getParentFile().isDirectory()
+        && resources.get(0).getTsFile().getParentFile().listFiles().length == 0) {
+      try {
+        Files.delete(resources.get(0).getTsFile().getParentFile().toPath());
+      } catch (IOException e) {
+        logger.error("Delete upgrade folder {} failed, caused by ",
+            resources.get(0).getTsFile().getParentFile(), e);
       }
     }
+    resources.clear();
   }
 
   public void merge(boolean fullMerge) {
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
index 0519851..82f31f3 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
@@ -26,6 +26,7 @@ import java.nio.file.FileAlreadyExistsException;
 import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
@@ -44,7 +45,6 @@ import org.apache.iotdb.db.exception.PartitionViolationException;
 import org.apache.iotdb.db.service.UpgradeSevice;
 import org.apache.iotdb.db.utils.FilePathUtils;
 import org.apache.iotdb.db.utils.TestOnly;
-import org.apache.iotdb.db.utils.UpgradeUtils;
 import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
 import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
@@ -119,13 +119,13 @@ public class TsFileResource {
   private FSFactory fsFactory = FSFactoryProducer.getFSFactory();
 
   /**
-   * generated upgraded TsFile ResourceList used for upgrading v0.9.x/v1 -> 0.10/v2
+   * generated upgraded TsFile ResourceList used for upgrading v0.11.x/v2 -> 0.12/v3
    */
   private List<TsFileResource> upgradedResources;
 
   /**
-   * load upgraded TsFile Resources to storage group processor used for upgrading v0.9.x/v1 ->
-   * 0.10/v2
+   * load upgraded TsFile Resources to storage group processor used for upgrading v0.11.x/v2 ->
+   * 0.12/v3
    */
   private UpgradeTsFileResourceCallBack upgradeTsFileResourceCallBack;
 
@@ -283,7 +283,45 @@ public class TsFileResource {
       timeIndex = TimeIndexLevel.valueOf(timeIndexType).getTimeIndex().deserialize(inputStream);
       maxPlanIndex = ReadWriteIOUtils.readLong(inputStream);
       minPlanIndex = ReadWriteIOUtils.readLong(inputStream);
+      if (inputStream.available() > 0) {
+        String modFileName = ReadWriteIOUtils.readString(inputStream);
+        if (modFileName != null) {
+          File modF = new File(file.getParentFile(), modFileName);
+          modFile = new ModificationFile(modF.getPath());
+        }
+      }
+    }
+  }
 
+  public void deserializeFromOldFile() throws IOException {
+    try (InputStream inputStream = fsFactory.getBufferedInputStream(
+        file + RESOURCE_SUFFIX)) {
+      // deserialize old TsfileResource
+      int size = ReadWriteIOUtils.readInt(inputStream);
+      Map<String, Integer> deviceMap = new HashMap<>();
+      long[] startTimesArray = new long[size];
+      long[] endTimesArray = new long[size];
+      for (int i = 0; i < size; i++) {
+        String path = ReadWriteIOUtils.readString(inputStream);
+        long time = ReadWriteIOUtils.readLong(inputStream);
+        deviceMap.put(path, i);
+        startTimesArray[i] = time;
+      }
+      size = ReadWriteIOUtils.readInt(inputStream);
+      for (int i = 0; i < size; i++) {
+        ReadWriteIOUtils.readString(inputStream); // String path
+        long time = ReadWriteIOUtils.readLong(inputStream);
+        endTimesArray[i] = time;
+      }
+      timeIndexType = (byte) 1;
+      timeIndex = new DeviceTimeIndex(deviceMap, startTimesArray, endTimesArray);
+      if (inputStream.available() > 0) {
+        int versionSize = ReadWriteIOUtils.readInt(inputStream);
+        for (int i = 0; i < versionSize; i++) {
+          // historicalVersions
+          ReadWriteIOUtils.readLong(inputStream);
+        }
+      }
       if (inputStream.available() > 0) {
         String modFileName = ReadWriteIOUtils.readString(inputStream);
         if (modFileName != null) {
@@ -420,9 +458,7 @@ public class TsFileResource {
   }
 
   void doUpgrade() {
-    if (UpgradeUtils.isNeedUpgrade(this)) {
-      UpgradeSevice.getINSTANCE().submitUpgradeTask(new UpgradeTask(this));
-    }
+    UpgradeSevice.getINSTANCE().submitUpgradeTask(new UpgradeTask(this));
   }
 
   public void removeModFile() throws IOException {
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/upgrade/UpgradeTask.java b/server/src/main/java/org/apache/iotdb/db/engine/upgrade/UpgradeTask.java
index 015cc01..deb8e27 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/upgrade/UpgradeTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/upgrade/UpgradeTask.java
@@ -18,20 +18,15 @@
  */
 package org.apache.iotdb.db.engine.upgrade;
 
-import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
-
 import java.io.File;
 import java.io.IOException;
-import java.nio.file.Files;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.iotdb.db.concurrent.WrappedRunnable;
-import org.apache.iotdb.db.conf.IoTDBConstant;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.modification.ModificationFile;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.service.UpgradeSevice;
 import org.apache.iotdb.db.tools.upgrade.TsFileOnlineUpgradeTool;
+import org.apache.iotdb.db.utils.UpgradeUtils;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
 import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
 import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
@@ -43,7 +38,6 @@ public class UpgradeTask extends WrappedRunnable {
   private TsFileResource upgradeResource;
   private static final Logger logger = LoggerFactory.getLogger(UpgradeTask.class);
   private static final String COMMA_SEPERATOR = ",";
-  private static final int maxLevelNum = IoTDBDescriptor.getInstance().getConfig().getSeqLevelNum();
 
   private FSFactory fsFactory = FSFactoryProducer.getFSFactory();
 
@@ -54,63 +48,17 @@ public class UpgradeTask extends WrappedRunnable {
   @Override
   public void runMayThrow() {
     try {
-      List<TsFileResource> upgradedResources = generateUpgradedFiles();
-      upgradeResource.writeLock();
       String oldTsfilePath = upgradeResource.getTsFile().getAbsolutePath();
-      String oldModificationFilePath = oldTsfilePath + ModificationFile.FILE_SUFFIX;
+      List<TsFileResource> upgradedResources;
+      if (!UpgradeUtils.isUpgradedFileGenerated(oldTsfilePath)) {
+        upgradedResources = generateUpgradedFiles();
+      }
+      else {
+        upgradedResources = findUpgradedFiles();
+      }
+      upgradeResource.writeLock();
       try {
-        // delete old TsFile and resource
-        upgradeResource.delete();
-        File modificationFile = FSFactoryProducer.getFSFactory().getFile(oldModificationFilePath);
-        // move upgraded TsFiles and modificationFile to their own partition directories
-        for (TsFileResource upgradedResource : upgradedResources) {
-          File upgradedFile = upgradedResource.getTsFile();
-          long partition = upgradedResource.getTimePartition();
-          String storageGroupPath = upgradedFile.getParentFile().getParentFile().getParent();
-          File partitionDir = FSFactoryProducer.getFSFactory()
-              .getFile(storageGroupPath, partition + "");
-          if (!partitionDir.exists()) {
-            partitionDir.mkdir();
-          }
-          FSFactoryProducer.getFSFactory().moveFile(upgradedFile,
-              FSFactoryProducer.getFSFactory().getFile(partitionDir, upgradedFile.getName()));
-          upgradedResource.setFile(
-              FSFactoryProducer.getFSFactory().getFile(partitionDir, upgradedFile.getName()));
-          // copy mods file to partition directories
-          if (modificationFile.exists()) {
-            Files.copy(modificationFile.toPath(),
-                FSFactoryProducer.getFSFactory().getFile(partitionDir, upgradedFile.getName()
-                    + ModificationFile.FILE_SUFFIX).toPath());
-          }
-          upgradedResource.serialize();
-          // delete tmp partition folder when it is empty
-          if (upgradedFile.getParentFile().isDirectory()
-              && upgradedFile.getParentFile().listFiles().length == 0) {
-            Files.delete(upgradedFile.getParentFile().toPath());
-          }
-          // rename all files to 0 level
-          upgradedFile = upgradedResource.getTsFile();
-          File zeroMergeVersionFile = getMaxMergeVersionFile(upgradedFile);
-          fsFactory.moveFile(upgradedFile, zeroMergeVersionFile);
-          fsFactory.moveFile(
-              fsFactory.getFile(upgradedFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX),
-              fsFactory
-                  .getFile(
-                      zeroMergeVersionFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX));
-          upgradedResource.setFile(upgradedFile);
-        }
-        // delete old modificationFile 
-        if (modificationFile.exists()) {
-          Files.delete(modificationFile.toPath());
-        }
-        // delete upgrade folder when it is empty
-        if (upgradeResource.getTsFile().getParentFile().isDirectory()
-            && upgradeResource.getTsFile().getParentFile().listFiles().length == 0) {
-          Files.delete(upgradeResource.getTsFile().getParentFile().toPath());
-        }
         upgradeResource.setUpgradedResources(upgradedResources);
-        UpgradeLog.writeUpgradeLogFile(
-            oldTsfilePath + COMMA_SEPERATOR + UpgradeCheckStatus.UPGRADE_SUCCESS);
         upgradeResource.getUpgradeTsFileResourceCallBack().call(upgradeResource);
       } finally {
         upgradeResource.writeUnlock();
@@ -118,37 +66,50 @@ public class UpgradeTask extends WrappedRunnable {
       UpgradeSevice.setCntUpgradeFileNum(UpgradeSevice.getCntUpgradeFileNum() - 1);
       logger.info("Upgrade completes, file path:{} , the remaining upgraded file num: {}",
           oldTsfilePath, UpgradeSevice.getCntUpgradeFileNum());
+      if (UpgradeSevice.getCntUpgradeFileNum() == 0) {
+        UpgradeSevice.getINSTANCE().stop();
+        logger.info("All files upgraded successfully! ");
+      }
     } catch (Exception e) {
       logger.error("meet error when upgrade file:{}", upgradeResource.getTsFile().getAbsolutePath(),
           e);
     }
   }
 
-  private List<TsFileResource> generateUpgradedFiles() throws WriteProcessException {
+  private List<TsFileResource> generateUpgradedFiles() 
+      throws IOException, WriteProcessException {
     upgradeResource.readLock();
     String oldTsfilePath = upgradeResource.getTsFile().getAbsolutePath();
     List<TsFileResource> upgradedResources = new ArrayList<>();
     UpgradeLog.writeUpgradeLogFile(
         oldTsfilePath + COMMA_SEPERATOR + UpgradeCheckStatus.BEGIN_UPGRADE_FILE);
     try {
-      TsFileOnlineUpgradeTool.upgradeOneTsfile(oldTsfilePath, upgradedResources);
+      TsFileOnlineUpgradeTool.upgradeOneTsfile(upgradeResource, upgradedResources);
       UpgradeLog.writeUpgradeLogFile(
           oldTsfilePath + COMMA_SEPERATOR + UpgradeCheckStatus.AFTER_UPGRADE_FILE);
-    } catch (IOException e) {
-      logger
-          .error("generate upgrade file failed, the file to be upgraded:{}", oldTsfilePath, e);
     } finally {
       upgradeResource.readUnlock();
     }
     return upgradedResources;
   }
 
-  private File getMaxMergeVersionFile(File seqFile) {
-    String[] splits = seqFile.getName().replace(TSFILE_SUFFIX, "")
-        .split(IoTDBConstant.FILE_NAME_SEPARATOR);
-    return fsFactory.getFile(seqFile.getParentFile(),
-        splits[0] + IoTDBConstant.FILE_NAME_SEPARATOR + splits[1]
-            + IoTDBConstant.FILE_NAME_SEPARATOR + (maxLevelNum - 1) + TSFILE_SUFFIX);
+  private List<TsFileResource> findUpgradedFiles() throws IOException {
+    upgradeResource.readLock();
+    List<TsFileResource> upgradedResources = new ArrayList<>();
+    try {
+      File upgradeFolder = upgradeResource.getTsFile().getParentFile();
+      for (File tempPartitionDir : upgradeFolder.listFiles()) {
+        if (tempPartitionDir.isDirectory() && 
+            fsFactory.getFile(tempPartitionDir, upgradeResource.getTsFile().getName()).exists()) {
+          TsFileResource resource = new TsFileResource(
+              fsFactory.getFile(tempPartitionDir, upgradeResource.getTsFile().getName()));
+          resource.deserialize();
+          upgradedResources.add(resource);
+        }
+      }
+    } finally {
+      upgradeResource.readUnlock();
+    }
+    return upgradedResources;
   }
-
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogWriter.java b/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogWriter.java
index e733813..812885d 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogWriter.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogWriter.java
@@ -330,9 +330,9 @@ public class MLogWriter implements AutoCloseable {
         }
 
         CreateTimeSeriesPlan plan = new CreateTimeSeriesPlan(new PartialPath(args[1]),
-          TSDataType.deserialize(Short.parseShort(args[2])),
-          TSEncoding.deserialize(Short.parseShort(args[3])),
-          CompressionType.deserialize(Short.parseShort(args[4])), props, null, null, alias);
+          TSDataType.deserialize((byte) Short.parseShort(args[2])),
+          TSEncoding.deserialize((byte) Short.parseShort(args[3])),
+          CompressionType.deserialize((byte) Short.parseShort(args[4])), props, null, null, alias);
 
         plan.setTagOffset(offset);
         createTimeseries(plan);
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowPlan.java
index ec32f5f..81d81e8 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowPlan.java
@@ -50,7 +50,7 @@ import org.slf4j.LoggerFactory;
 public class InsertRowPlan extends InsertPlan {
 
   private static final Logger logger = LoggerFactory.getLogger(InsertRowPlan.class);
-  private static final short TYPE_RAW_STRING = -1;
+  private static final byte TYPE_RAW_STRING = -1;
 
   private long time;
   private Object[] values;
@@ -376,7 +376,7 @@ public class InsertRowPlan extends InsertPlan {
     for (int i = 0; i < measurements.length; i++) {
       // types are not determined, the situation mainly occurs when the plan uses string values
       // and is forwarded to other nodes
-      short typeNum = ReadWriteIOUtils.readShort(buffer);
+      byte typeNum = (byte) ReadWriteIOUtils.read(buffer);
       if (typeNum == TYPE_RAW_STRING) {
         values[i] = ReadWriteIOUtils.readString(buffer);
         continue;
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertTabletPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertTabletPlan.java
index 7cbb426..ba258e8 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertTabletPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertTabletPlan.java
@@ -170,7 +170,7 @@ public class InsertTabletPlan extends InsertPlan {
         continue;
       }
       TSDataType dataType = dataTypes[i];
-      stream.writeShort(dataType.serialize());
+      stream.write(dataType.serialize());
     }
   }
 
@@ -412,7 +412,7 @@ public class InsertTabletPlan extends InsertPlan {
 
     this.dataTypes = new TSDataType[measurementSize];
     for (int i = 0; i < measurementSize; i++) {
-      dataTypes[i] = TSDataType.deserialize(buffer.getShort());
+      dataTypes[i] = TSDataType.deserialize(buffer.get());
     }
 
     int rows = buffer.getInt();
diff --git a/server/src/main/java/org/apache/iotdb/db/query/aggregation/AggregateResult.java b/server/src/main/java/org/apache/iotdb/db/query/aggregation/AggregateResult.java
index ef01070..915f4f3 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/aggregation/AggregateResult.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/aggregation/AggregateResult.java
@@ -109,7 +109,7 @@ public abstract class AggregateResult {
 
   public static AggregateResult deserializeFrom(ByteBuffer buffer) {
     AggregationType aggregationType = AggregationType.deserialize(buffer);
-    TSDataType dataType = TSDataType.deserialize(buffer.getShort());
+    TSDataType dataType = TSDataType.deserialize(buffer.get());
     boolean ascending = ReadWriteIOUtils.readBool(buffer);
     AggregateResult aggregateResult = AggregateResultFactory
         .getAggrResultByType(aggregationType, dataType, ascending);
diff --git a/server/src/main/java/org/apache/iotdb/db/query/aggregation/impl/AvgAggrResult.java b/server/src/main/java/org/apache/iotdb/db/query/aggregation/impl/AvgAggrResult.java
index fea3095..8fb122e 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/aggregation/impl/AvgAggrResult.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/aggregation/impl/AvgAggrResult.java
@@ -28,6 +28,8 @@ import org.apache.iotdb.db.query.reader.series.IReaderByTimestamp;
 import org.apache.iotdb.tsfile.exception.filter.StatisticsClassException;
 import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.statistics.BooleanStatistics;
+import org.apache.iotdb.tsfile.file.metadata.statistics.IntegerStatistics;
 import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
 import org.apache.iotdb.tsfile.read.common.BatchData;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
@@ -69,8 +71,14 @@ public class AvgAggrResult extends AggregateResult {
       throw new StatisticsClassException("Binary statistics does not support: avg");
     }
     cnt += statistics.getCount();
+    double sum;
+    if (statistics instanceof IntegerStatistics || statistics instanceof BooleanStatistics) {
+      sum = statistics.getSumLongValue();
+    } else {
+      sum = statistics.getSumDoubleValue();
+    }
     avg = avg * ((double) preCnt / cnt) + ((double) statistics.getCount() / cnt)
-        * statistics.getSumValue() / statistics.getCount();
+        * sum / statistics.getCount();
   }
 
   @Override
@@ -144,7 +152,7 @@ public class AvgAggrResult extends AggregateResult {
 
   @Override
   protected void deserializeSpecificFields(ByteBuffer buffer) {
-    this.seriesDataType = TSDataType.deserialize(buffer.getShort());
+    this.seriesDataType = TSDataType.deserialize(buffer.get());
     this.avg = buffer.getDouble();
     this.cnt = buffer.getLong();
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/query/aggregation/impl/SumAggrResult.java b/server/src/main/java/org/apache/iotdb/db/query/aggregation/impl/SumAggrResult.java
index 099a016..02f8826 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/aggregation/impl/SumAggrResult.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/aggregation/impl/SumAggrResult.java
@@ -27,6 +27,8 @@ import org.apache.iotdb.db.query.aggregation.AggregationType;
 import org.apache.iotdb.db.query.reader.series.IReaderByTimestamp;
 import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.statistics.BooleanStatistics;
+import org.apache.iotdb.tsfile.file.metadata.statistics.IntegerStatistics;
 import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
 import org.apache.iotdb.tsfile.read.common.BatchData;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
@@ -50,7 +52,11 @@ public class SumAggrResult extends AggregateResult {
   @Override
   public void updateResultFromStatistics(Statistics statistics) {
     double preValue = getDoubleValue();
-    preValue += statistics.getSumValue();
+    if (statistics instanceof IntegerStatistics || statistics instanceof BooleanStatistics) {
+      preValue += statistics.getSumLongValue();
+    } else {
+      preValue += statistics.getSumDoubleValue();
+    }
     setDoubleValue(preValue);
   }
 
@@ -118,7 +124,7 @@ public class SumAggrResult extends AggregateResult {
 
   @Override
   protected void deserializeSpecificFields(ByteBuffer buffer) {
-    seriesDataType = TSDataType.deserialize(buffer.getShort());
+    seriesDataType = TSDataType.deserialize(buffer.get());
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/query/control/FileReaderManager.java b/server/src/main/java/org/apache/iotdb/db/query/control/FileReaderManager.java
index 2b60229..f2a1617 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/control/FileReaderManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/control/FileReaderManager.java
@@ -33,7 +33,7 @@ import org.apache.iotdb.db.service.ServiceType;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
 import org.apache.iotdb.tsfile.read.UnClosedTsFileReader;
-import org.apache.iotdb.tsfile.v1.read.TsFileSequenceReaderForV1;
+import org.apache.iotdb.tsfile.v2.read.TsFileSequenceReaderForV2;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -167,15 +167,13 @@ public class FileReaderManager implements IService {
       }
       else {
         tsFileReader = new TsFileSequenceReader(filePath);
-        switch (tsFileReader.readVersionNumber()) {
-          case TSFileConfig.VERSION_NUMBER_V1:
-            tsFileReader.close();
-            tsFileReader = new TsFileSequenceReaderForV1(filePath);
-            break;
-          case TSFileConfig.VERSION_NUMBER:
-            break;
-          default:
+        if (tsFileReader.readVersionNumber() != TSFileConfig.VERSION_NUMBER) {
+          tsFileReader.close();
+          tsFileReader = new TsFileSequenceReaderForV2(filePath);
+          if (!((TsFileSequenceReaderForV2) tsFileReader).readVersionNumberV2()
+              .equals(TSFileConfig.VERSION_NUMBER_V2)) {
             throw new IOException("The version of this TsFile is not corrent. ");
+          }
         }
       }
       readerMap.put(filePath, tsFileReader);
diff --git a/server/src/main/java/org/apache/iotdb/db/query/externalsort/serialize/impl/FixLengthIExternalSortFileDeserializer.java b/server/src/main/java/org/apache/iotdb/db/query/externalsort/serialize/impl/FixLengthIExternalSortFileDeserializer.java
index b87a852..ed8b4d1 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/externalsort/serialize/impl/FixLengthIExternalSortFileDeserializer.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/externalsort/serialize/impl/FixLengthIExternalSortFileDeserializer.java
@@ -83,7 +83,7 @@ public class FixLengthIExternalSortFileDeserializer implements IExternalSortFile
   }
 
   private TSDataType readHeader() throws IOException {
-    return TSDataType.deserialize(ReadWriteIOUtils.readShort(inputStream));
+    return TSDataType.deserialize(ReadWriteIOUtils.readByte(inputStream));
   }
 
   private void setReader(TSDataType type) {
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/metadata/DiskChunkMetadataLoader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/metadata/DiskChunkMetadataLoader.java
index ec13354..7144419 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/metadata/DiskChunkMetadataLoader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/metadata/DiskChunkMetadataLoader.java
@@ -63,8 +63,11 @@ public class DiskChunkMetadataLoader implements IChunkMetadataLoader {
         .satisfyStartEndTime(chunkMetaData.getStartTime(), chunkMetaData.getEndTime()))
         || chunkMetaData.getStartTime() > chunkMetaData.getEndTime());
 
+    // For chunkMetadata from old TsFile, do not set version
     for (ChunkMetadata metadata : chunkMetadataList) {
-      metadata.setVersion(resource.getVersion());
+      if (!metadata.isFromOldTsFile()) {
+        metadata.setVersion(resource.getVersion());
+      }
     }
     return chunkMetadataList;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/service/UpgradeSevice.java b/server/src/main/java/org/apache/iotdb/db/service/UpgradeSevice.java
index b7d1b89..863855b 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/UpgradeSevice.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/UpgradeSevice.java
@@ -71,9 +71,6 @@ public class UpgradeSevice implements IService {
     if (upgradeThreadPool != null) {
       upgradeThreadPool.shutdownNow();
       logger.info("Waiting for upgrade task pool to shut down");
-      while (!upgradeThreadPool.isTerminated()) {
-        // wait
-      }
       upgradeThreadPool = null;
       logger.info("Upgrade service stopped");
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/tools/TsFileSketchTool.java b/server/src/main/java/org/apache/iotdb/db/tools/TsFileSketchTool.java
index f48fb35..8423ce0 100644
--- a/server/src/main/java/org/apache/iotdb/db/tools/TsFileSketchTool.java
+++ b/server/src/main/java/org/apache/iotdb/db/tools/TsFileSketchTool.java
@@ -27,7 +27,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
-import org.apache.iotdb.tsfile.file.footer.ChunkGroupFooter;
+import org.apache.iotdb.tsfile.file.header.ChunkGroupHeader;
 import org.apache.iotdb.tsfile.file.metadata.ChunkGroupMetadata;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
 import org.apache.iotdb.tsfile.file.metadata.MetadataIndexEntry;
@@ -75,13 +75,21 @@ public class TsFileSketchTool {
                 String.format("%20d", TSFileConfig.MAGIC_STRING.getBytes().length)
                         + "|\t[version number] "
                         + reader.readVersionNumber());
+        long nextChunkGroupHeaderPos = (long) TSFileConfig.MAGIC_STRING.getBytes().length 
+            + Byte.BYTES;
         // ChunkGroup begins
         for (ChunkGroupMetadata chunkGroupMetadata : allChunkGroupMetadata) {
           printlnBoth(pw, str1.toString() + "\t[Chunk Group] of " + chunkGroupMetadata.getDevice() +
                   ", num of Chunks:" + chunkGroupMetadata.getChunkMetadataList().size());
+          // chunkGroupHeader begins
+          printlnBoth(pw, String.format("%20s", nextChunkGroupHeaderPos) + "|\t[Chunk Group Header]");
+          ChunkGroupHeader chunkGroupHeader = reader.readChunkGroupHeader(nextChunkGroupHeaderPos, false);
+          printlnBoth(pw, String.format("%20s", "") + "|\t\t[marker] 0");
+          printlnBoth(pw,
+                  String.format("%20s", "") + "|\t\t[deviceID] " + chunkGroupHeader.getDeviceID());
           // chunk begins
-          long chunkEndPos = 0;
           for (ChunkMetadata chunkMetadata : chunkGroupMetadata.getChunkMetadataList()) {
+            Chunk chunk = reader.readMemChunk(chunkMetadata);
             printlnBoth(pw,
                     String.format("%20d", chunkMetadata.getOffsetOfChunkHeader()) + "|\t[Chunk] of "
                             + chunkMetadata.getMeasurementUid() + ", numOfPoints:" + chunkMetadata
@@ -89,24 +97,11 @@ public class TsFileSketchTool {
                             + chunkMetadata.getEndTime() + "], tsDataType:" + chunkMetadata.getDataType()
                             + ", \n" + String.format("%20s", "") + " \t" + chunkMetadata.getStatistics());
             printlnBoth(pw, String.format("%20s", "") + "|\t\t[marker] 1");
-            printlnBoth(pw, String.format("%20s", "") + "|\t\t[ChunkHeader]");
-            Chunk chunk = reader.readMemChunk(chunkMetadata);
-            printlnBoth(pw,
-                    String.format("%20s", "") + "|\t\t" + chunk.getHeader().getNumOfPages() + " pages");
-            chunkEndPos =
+            nextChunkGroupHeaderPos =
                     chunkMetadata.getOffsetOfChunkHeader() + chunk.getHeader().getSerializedSize()
-                            + chunk.getHeader().getDataSize();
+                            + chunk.getHeader().getDataSize() - 1;
           }
-          // chunkGroupFooter begins
-          printlnBoth(pw, String.format("%20s", chunkEndPos) + "|\t[Chunk Group Footer]");
-          ChunkGroupFooter chunkGroupFooter = reader.readChunkGroupFooter(chunkEndPos, false);
-          printlnBoth(pw, String.format("%20s", "") + "|\t\t[marker] 0");
-          printlnBoth(pw,
-                  String.format("%20s", "") + "|\t\t[deviceID] " + chunkGroupFooter.getDeviceID());
-          printlnBoth(pw,
-                  String.format("%20s", "") + "|\t\t[dataSize] " + chunkGroupFooter.getDataSize());
-          printlnBoth(pw, String.format("%20s", "") + "|\t\t[num of chunks] " + chunkGroupFooter
-                  .getNumberOfChunks());
+
           printlnBoth(pw, str1.toString() + "\t[Chunk Group] of "
                   + chunkGroupMetadata.getDevice() + " ends");
         }
@@ -150,11 +145,6 @@ public class TsFileSketchTool {
         printlnBoth(pw,
                 String.format("%20s", "") + "|\t\t" + tsFileMetaData.getMetadataIndex().getChildren()
                         .size() + " key&TsMetadataIndex");
-        printlnBoth(pw,
-                String.format("%20s", "") + "|\t\t[totalChunkNum] " + tsFileMetaData.getTotalChunkNum());
-        printlnBoth(pw,
-                String.format("%20s", "") + "|\t\t[invalidChunkNum] " + tsFileMetaData
-                        .getInvalidChunkNum());
 
         // bloom filter
         BloomFilter bloomFilter = tsFileMetaData.getBloomFilter();
diff --git a/server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java b/server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java
index 307210c..adbb22e 100644
--- a/server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java
+++ b/server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java
@@ -23,36 +23,33 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.modification.Deletion;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.modification.ModificationFile;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
-import org.apache.iotdb.tsfile.compress.IUnCompressor;
 import org.apache.iotdb.tsfile.encoding.decoder.Decoder;
 import org.apache.iotdb.tsfile.exception.write.PageException;
 import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
 import org.apache.iotdb.tsfile.file.MetaMarker;
-import org.apache.iotdb.tsfile.file.footer.ChunkGroupFooter;
+import org.apache.iotdb.tsfile.file.header.ChunkGroupHeader;
 import org.apache.iotdb.tsfile.file.header.ChunkHeader;
 import org.apache.iotdb.tsfile.file.header.PageHeader;
 import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
-import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
 import org.apache.iotdb.tsfile.read.common.BatchData;
-import org.apache.iotdb.tsfile.read.reader.TsFileInput;
-import org.apache.iotdb.tsfile.read.reader.page.PageReader;
 import org.apache.iotdb.tsfile.utils.Binary;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-import org.apache.iotdb.tsfile.v1.file.metadata.TsDeviceMetadataIndexV1;
-import org.apache.iotdb.tsfile.v1.file.metadata.TsDeviceMetadataV1;
-import org.apache.iotdb.tsfile.v1.file.metadata.TsFileMetadataV1;
-import org.apache.iotdb.tsfile.v1.file.utils.HeaderUtils;
+import org.apache.iotdb.tsfile.v2.read.TsFileSequenceReaderForV2;
+import org.apache.iotdb.tsfile.v2.read.reader.page.PageReaderV2;
 import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
 import org.apache.iotdb.tsfile.write.chunk.IChunkWriter;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
@@ -64,347 +61,206 @@ public class TsFileOnlineUpgradeTool implements AutoCloseable {
 
   private static final Logger logger = LoggerFactory.getLogger(TsFileOnlineUpgradeTool.class);
 
-  private TsFileInput tsFileInput;
-  private long fileMetadataPos;
-  private int fileMetadataSize;
-  private ByteBuffer markerBuffer = ByteBuffer.allocate(Byte.BYTES);
+  private TsFileSequenceReaderForV2 reader;
+  private File oldTsFile;
+  private List<Modification> oldModification;
+  private Iterator<Modification> modsIterator;
+  // new tsFile writer -> list of new modification
+  private Map<TsFileIOWriter, ModificationFile> fileModificationMap;
+  private Deletion currentMod;
   private Decoder defaultTimeDecoder = Decoder.getDecoderByType(
       TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder()),
       TSDataType.INT64);
   private Decoder valueDecoder;
-  protected String file;
 
-  // PartitionId -> TsFileIOWriter 
+  // PartitionId -> TsFileIOWriter
   private Map<Long, TsFileIOWriter> partitionWriterMap;
 
   /**
    * Create a file reader of the given file. The reader will read the tail of the file to get the
-   * file metadata size.Then the reader will skip the first TSFileConfig.OLD_MAGIC_STRING.length()
+   * file metadata size. Then the reader will skip the first TSFileConfig.OLD_MAGIC_STRING.length()
    * bytes of the file for preparing reading real data.
    *
    * @param file the data file
    * @throws IOException If some I/O error occurs
    */
-  public TsFileOnlineUpgradeTool(String file) throws IOException {
-    this(file, true);
-  }
-
-  /**
-   * construct function for TsfileOnlineUpgradeTool.
-   *
-   * @param file -given file name
-   * @param loadMetadataSize -load meta data size
-   */
-  public TsFileOnlineUpgradeTool(String file, boolean loadMetadataSize) throws IOException {
-    this.file = file;
-    tsFileInput = FSFactoryProducer.getFileInputFactory().getTsFileInput(file);
+  public TsFileOnlineUpgradeTool(TsFileResource resourceToBeUpgraded) throws IOException {
+    oldTsFile = resourceToBeUpgraded.getTsFile();
+    String file = oldTsFile.getAbsolutePath();
+    reader = new TsFileSequenceReaderForV2(file);
     partitionWriterMap = new HashMap<>();
-    try {
-      if (loadMetadataSize) {
-        loadMetadataSize();
-      }
-    } catch (Exception e) {
-      tsFileInput.close();
-      throw e;
+    if (FSFactoryProducer.getFSFactory().getFile(file +
+        ModificationFile.FILE_SUFFIX).exists()) {
+      oldModification = (List<Modification>) resourceToBeUpgraded.getModFile().getModifications();
+      modsIterator = oldModification.iterator();
+      fileModificationMap = new HashMap<>();
     }
   }
 
   /**
-   * upgrade a single tsfile
+   * upgrade a single TsFile
    *
    * @param tsFileName old version tsFile's absolute path
    * @param upgradedResources new version tsFiles' resources
    */
-  public static void upgradeOneTsfile(String tsFileName, List<TsFileResource> upgradedResources)
+  public static void upgradeOneTsfile(TsFileResource resourceToBeUpgraded, List<TsFileResource> upgradedResources)
       throws IOException, WriteProcessException {
-    try (TsFileOnlineUpgradeTool updater = new TsFileOnlineUpgradeTool(tsFileName)) {
+    try (TsFileOnlineUpgradeTool updater = new TsFileOnlineUpgradeTool(resourceToBeUpgraded)) {
       updater.upgradeFile(upgradedResources);
     }
   }
 
-  /**
-   *
-   */
-  public void loadMetadataSize() throws IOException {
-    ByteBuffer metadataSize = ByteBuffer.allocate(Integer.BYTES);
-    tsFileInput.read(metadataSize,
-        tsFileInput.size() - TSFileConfig.MAGIC_STRING.getBytes().length - Integer.BYTES);
-    metadataSize.flip();
-    // read file metadata size and position
-    fileMetadataSize = ReadWriteIOUtils.readInt(metadataSize);
-    fileMetadataPos =
-        tsFileInput.size() - TSFileConfig.MAGIC_STRING.getBytes().length - Integer.BYTES
-            - fileMetadataSize;
-    // skip the magic header
-    position(TSFileConfig.MAGIC_STRING.length());
-  }
-
-  public String readTailMagic() throws IOException {
-    long totalSize = tsFileInput.size();
-
-    ByteBuffer magicStringBytes = ByteBuffer.allocate(TSFileConfig.MAGIC_STRING.length());
-    tsFileInput.read(magicStringBytes, totalSize - TSFileConfig.MAGIC_STRING.length());
-    magicStringBytes.flip();
-    return new String(magicStringBytes.array());
-  }
-
-  /**
-   * whether the file is a complete TsFile: only if the head magic and tail magic string exists.
-   */
-  public boolean isComplete() throws IOException {
-    return tsFileInput.size() >= TSFileConfig.MAGIC_STRING.length() * 2 && readTailMagic()
-        .equals(readHeadMagic());
-  }
-
-  /**
-   * this function does not modify the position of the file reader.
-   */
-  public String readHeadMagic() throws IOException {
-    return readHeadMagic(false);
-  }
-
-  /**
-   * @param movePosition whether move the position of the file reader after reading the magic header
-   * to the end of the magic head string.
-   */
-  public String readHeadMagic(boolean movePosition) throws IOException {
-    ByteBuffer magicStringBytes = ByteBuffer.allocate(TSFileConfig.MAGIC_STRING.length());
-    if (movePosition) {
-      tsFileInput.position(0);
-      tsFileInput.read(magicStringBytes);
-    } else {
-      tsFileInput.read(magicStringBytes, 0);
-    }
-    magicStringBytes.flip();
-    return new String(magicStringBytes.array());
-  }
-
-  /**
-   * this function reads version number and checks compatibility of TsFile.
-   */
-  public String readVersionNumber() throws IOException {
-    ByteBuffer versionNumberBytes = ByteBuffer
-        .allocate(TSFileConfig.VERSION_NUMBER.getBytes().length);
-    tsFileInput.position(TSFileConfig.MAGIC_STRING.getBytes().length);
-    tsFileInput.read(versionNumberBytes);
-    versionNumberBytes.flip();
-    return new String(versionNumberBytes.array());
-  }
-
-  /**
-   * this function does not modify the position of the file reader.
-   */
-  public TsFileMetadataV1 readFileMetadata() throws IOException {
-    return TsFileMetadataV1.deserializeFrom(readData(fileMetadataPos, fileMetadataSize));
-  }
-
-  /**
-   * this function does not modify the position of the file reader.
-   */
-  public TsDeviceMetadataV1 readTsDeviceMetaData(TsDeviceMetadataIndexV1 index) throws IOException {
-    return TsDeviceMetadataV1.deserializeFrom(readData(index.getOffset(), index.getLen()));
-  }
-
-  /**
-   * read data from current position of the input, and deserialize it to a CHUNK_GROUP_FOOTER. <br>
-   * This method is not threadsafe.
-   *
-   * @return a CHUNK_GROUP_FOOTER
-   * @throws IOException io error
-   */
-  public ChunkGroupFooter readChunkGroupFooter() throws IOException {
-    return ChunkGroupFooter.deserializeFrom(tsFileInput.wrapAsInputStream(), true);
-  }
-
-  /**
-   * read data from current position of the input, and deserialize it to a CHUNK_HEADER. <br> This
-   * method is not threadsafe.
-   *
-   * @return a CHUNK_HEADER
-   * @throws IOException io error
-   */
-  public ChunkHeader readChunkHeader() throws IOException {
-    return HeaderUtils.deserializeChunkHeaderV1(tsFileInput.wrapAsInputStream(), true);
-  }
-
-  /**
-   * not thread safe.
-   *
-   * @param type given tsfile data type
-   */
-  public PageHeader readPageHeader(TSDataType type) throws IOException {
-    return HeaderUtils.deserializePageHeaderV1(tsFileInput.wrapAsInputStream(), type);
-  }
-
-  public ByteBuffer readPage(PageHeader header, CompressionType type)
-      throws IOException {
-    ByteBuffer buffer = readData(-1, header.getCompressedSize());
-    IUnCompressor unCompressor = IUnCompressor.getUnCompressor(type);
-    ByteBuffer uncompressedBuffer = ByteBuffer.allocate(header.getUncompressedSize());
-    if (type == CompressionType.UNCOMPRESSED) {
-      return buffer;
-    }
-    unCompressor.uncompress(buffer.array(), buffer.position(), buffer.remaining(),
-        uncompressedBuffer.array(),
-        0);
-    return uncompressedBuffer;
-  }
-
-  public ByteBuffer readCompressedPage(PageHeader header) throws IOException {
-    return readData(-1, header.getCompressedSize());
-  }
-
-  public long position() throws IOException {
-    return tsFileInput.position();
-  }
-
-  public void position(long offset) throws IOException {
-    tsFileInput.position(offset);
-  }
-
-  /**
-   * read one byte from the input. <br> this method is not thread safe
-   */
-  public byte readMarker() throws IOException {
-    markerBuffer.clear();
-    if (ReadWriteIOUtils.readAsPossible(tsFileInput, markerBuffer) == 0) {
-      throw new IOException("reach the end of the file.");
-    }
-    markerBuffer.flip();
-    return markerBuffer.get();
-  }
-
-  public byte readMarker(long position) throws IOException {
-    return readData(position, Byte.BYTES).get();
-  }
-
-  @Override
   public void close() throws IOException {
-    this.tsFileInput.close();
-  }
-
-  public String getFileName() {
-    return this.file;
-  }
-
-  /**
-   * read data from tsFileInput, from the current position (if position = -1), or the given
-   * position. <br> if position = -1, the tsFileInput's position will be changed to the current
-   * position + real data size that been read. Other wise, the tsFileInput's position is not
-   * changed.
-   *
-   * @param position the start position of data in the tsFileInput, or the current position if
-   * position = -1
-   * @param size the size of data that want to read
-   * @return data that been read.
-   */
-  private ByteBuffer readData(long position, int size) throws IOException {
-    ByteBuffer buffer = ByteBuffer.allocate(size);
-    if (position == -1) {
-      if (ReadWriteIOUtils.readAsPossible(tsFileInput, buffer) != size) {
-        throw new IOException("reach the end of the data");
-      }
-    } else {
-      if (ReadWriteIOUtils.readAsPossible(tsFileInput, buffer, position, size) != size) {
-        throw new IOException("reach the end of the data");
-      }
-    }
-    buffer.flip();
-    return buffer;
+    this.reader.close();
   }
 
   /**
-   * upgrade file and resource
+   * upgrade file resource
    *
    * @throws IOException, WriteProcessException
    */
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  public void upgradeFile(List<TsFileResource> upgradedResources)
+  @SuppressWarnings({ "squid:S3776", "deprecation" }) // Suppress high Cognitive Complexity warning
+  private void upgradeFile(List<TsFileResource> upgradedResources)
       throws IOException, WriteProcessException {
-    File oldTsFile = FSFactoryProducer.getFSFactory().getFile(this.file);
 
-    // check if the old TsFile has correct header 
-    if (!fileCheck(oldTsFile)) {
+    // check if the old TsFile has correct header
+    if (!fileCheck()) {
       return;
     }
 
+    int headerLength = TSFileConfig.MAGIC_STRING.getBytes().length +
+        TSFileConfig.VERSION_NUMBER_V2.getBytes().length;
+    reader.position(headerLength);
     // start to scan chunks and chunkGroups
     boolean newChunkGroup = true;
-    int chunkGroupCount = 0;
     List<List<PageHeader>> pageHeadersInChunkGroup = new ArrayList<>();
     List<List<ByteBuffer>> pageDataInChunkGroup = new ArrayList<>();
-    List<List<Boolean>> pagePartitionInfoInChunkGroup = new ArrayList<>();
+    List<List<Boolean>> needToDecodeInfoInChunkGroup = new ArrayList<>();
     byte marker;
     List<MeasurementSchema> measurementSchemaList = new ArrayList<>();
     try {
-      while ((marker = this.readMarker()) != MetaMarker.SEPARATOR) {
+      while ((marker = reader.readMarker()) != MetaMarker.SEPARATOR) {
         switch (marker) {
           case MetaMarker.CHUNK_HEADER:
             // this is the first chunk of a new ChunkGroup.
             if (newChunkGroup) {
               newChunkGroup = false;
             }
-            ChunkHeader header = this.readChunkHeader();
-            MeasurementSchema measurementSchema = new MeasurementSchema(header.getMeasurementID(),
+            ChunkHeader header = reader.readChunkHeader();
+            MeasurementSchema measurementSchema = new MeasurementSchema(
+                header.getMeasurementID(),
                 header.getDataType(),
                 header.getEncodingType(),
                 header.getCompressionType());
             measurementSchemaList.add(measurementSchema);
+            TSDataType dataType = header.getDataType();
+            TSEncoding encoding = header.getEncodingType();
             List<PageHeader> pageHeadersInChunk = new ArrayList<>();
             List<ByteBuffer> dataInChunk = new ArrayList<>();
-            List<Boolean> pagePartitionInfo = new ArrayList<>();
+            List<Boolean> needToDecodeInfo = new ArrayList<>();
             for (int j = 0; j < header.getNumOfPages(); j++) {
-              PageHeader pageHeader = readPageHeader(header.getDataType());
-              boolean pageInSamePartition = checkIfPageInSameTimePartition(pageHeader);
-              pagePartitionInfo.add(pageInSamePartition);
-              ByteBuffer pageData = pageInSamePartition ?
-                  readCompressedPage(pageHeader)
-                  : readPage(pageHeader, header.getCompressionType());
+              // a new Page
+              PageHeader pageHeader = reader.readPageHeader(dataType);
+              boolean needToDecode = 
+                  checkIfNeedToDecode(dataType, encoding, pageHeader);
+              needToDecodeInfo.add(needToDecode);
+              ByteBuffer pageData = !needToDecode
+                  ? reader.readCompressedPage(pageHeader) 
+                  : reader.readPage(pageHeader, header.getCompressionType());
               pageHeadersInChunk.add(pageHeader);
               dataInChunk.add(pageData);
             }
             pageHeadersInChunkGroup.add(pageHeadersInChunk);
             pageDataInChunkGroup.add(dataInChunk);
-            pagePartitionInfoInChunkGroup.add(pagePartitionInfo);
+            needToDecodeInfoInChunkGroup.add(needToDecodeInfo);
             break;
-          case MetaMarker.CHUNK_GROUP_FOOTER:
-            // this is the footer of a ChunkGroup.
-            ChunkGroupFooter chunkGroupFooter = this.readChunkGroupFooter();
+          case MetaMarker.CHUNK_GROUP_HEADER:
+            // this is the footer of a ChunkGroup in TsFileV2.
+            ChunkGroupHeader chunkGroupFooter = reader.readChunkGroupFooter();
             String deviceID = chunkGroupFooter.getDeviceID();
-            rewrite(oldTsFile, deviceID, measurementSchemaList, pageHeadersInChunkGroup,
-                pageDataInChunkGroup, pagePartitionInfoInChunkGroup);
-
+            rewrite(deviceID, measurementSchemaList, pageHeadersInChunkGroup,
+                pageDataInChunkGroup, needToDecodeInfoInChunkGroup);
             pageHeadersInChunkGroup.clear();
             pageDataInChunkGroup.clear();
             measurementSchemaList.clear();
-            pagePartitionInfoInChunkGroup.clear();
+            needToDecodeInfoInChunkGroup.clear();
             newChunkGroup = true;
-            chunkGroupCount++;
             break;
-
+          case MetaMarker.VERSION:
+            long version = reader.readVersion();
+            // convert old Modification to new 
+            if (oldModification != null && modsIterator.hasNext()) {
+              if (currentMod == null) {
+                currentMod = (Deletion) modsIterator.next();
+              }
+              if (currentMod.getFileOffset() <= version) {
+                for (Entry<TsFileIOWriter, ModificationFile> entry 
+                    : fileModificationMap.entrySet()) {
+                  TsFileIOWriter tsFileIOWriter = entry.getKey();
+                  ModificationFile newMods = entry.getValue();
+                  newMods.write(new Deletion(currentMod.getPath(), 
+                      tsFileIOWriter.getFile().length(),
+                      currentMod.getStartTime(),
+                      currentMod.getEndTime()));
+                }
+                currentMod = null;
+              }
+            }
+            // write plan indices for ending memtable
+            for (TsFileIOWriter tsFileIOWriter : partitionWriterMap.values()) { 
+              tsFileIOWriter.writePlanIndices(); 
+            }
+            break;
           default:
             // the disk file is corrupted, using this file may be dangerous
-            logger.error("Unrecognized marker detected, this file may be corrupted");
-            return;
+            throw new IOException("Unrecognized marker detected, "
+                + "this file may be corrupted");
         }
       }
       // close upgraded tsFiles and generate resources for them
       for (TsFileIOWriter tsFileIOWriter : partitionWriterMap.values()) {
         upgradedResources.add(endFileAndGenerateResource(tsFileIOWriter));
       }
+      // write the remain modification for new file
+      if (oldModification != null) {
+        while (currentMod != null || modsIterator.hasNext()) {
+          if (currentMod == null) {
+            currentMod = (Deletion) modsIterator.next();
+          }
+          for (Entry<TsFileIOWriter, ModificationFile> entry 
+              : fileModificationMap.entrySet()) {
+            TsFileIOWriter tsFileIOWriter = entry.getKey();
+            ModificationFile newMods = entry.getValue();
+            newMods.write(new Deletion(currentMod.getPath(), 
+                tsFileIOWriter.getFile().length(),
+                currentMod.getStartTime(),
+                currentMod.getEndTime()));
+          }
+          currentMod = null;
+        }
+      }
     } catch (IOException e2) {
-      logger.info("TsFile upgrade process cannot proceed at position {} after {} chunk groups "
-          + "recovered, because : {}", this.position(), chunkGroupCount, e2.getMessage());
+      throw new IOException("TsFile upgrade process cannot proceed at position " +
+          reader.position() + "because: " + e2.getMessage());
     } finally {
-      if (tsFileInput != null) {
-        tsFileInput.close();
+      if (reader != null) {
+        reader.close();
       }
     }
   }
 
-  private boolean checkIfPageInSameTimePartition(PageHeader pageHeader) {
-    return StorageEngine.getTimePartition(pageHeader.getStartTime())
-        == StorageEngine.getTimePartition(pageHeader.getEndTime());
+  /**
+   * Due to TsFile version-3 changed the serialize way of integer in TEXT data and 
+   * INT32 data with PLAIN encoding, and also add a sum statistic for BOOLEAN data,
+   * these types of data need to decode to points and rewrite in new TsFile.
+   */
+  private boolean checkIfNeedToDecode(TSDataType dataType, TSEncoding encoding,
+      PageHeader pageHeader) {
+    return dataType == TSDataType.BOOLEAN ||
+        dataType == TSDataType.TEXT ||
+        (dataType == TSDataType.INT32 && encoding == TSEncoding.PLAIN) ||
+        StorageEngine.getTimePartition(pageHeader.getStartTime()) 
+        != StorageEngine.getTimePartition(pageHeader.getEndTime());
   }
 
   /**
@@ -412,25 +268,26 @@ public class TsFileOnlineUpgradeTool implements AutoCloseable {
    * this case, we have to decode the data to points, and then rewrite the data points to different
    * chunkWriters, finally write chunks to their own upgraded TsFiles
    */
-  private void rewrite(File oldTsFile, String deviceId, List<MeasurementSchema> schemas,
+  private void rewrite(String deviceId, List<MeasurementSchema> schemas,
       List<List<PageHeader>> pageHeadersInChunkGroup, List<List<ByteBuffer>> dataInChunkGroup,
-                       List<List<Boolean>> pagePartitionInfoInChunkGroup)
+      List<List<Boolean>> needToDecodeInfoInChunkGroup)
       throws IOException, PageException {
     Map<Long, Map<MeasurementSchema, ChunkWriterImpl>> chunkWritersInChunkGroup = new HashMap<>();
     for (int i = 0; i < schemas.size(); i++) {
       MeasurementSchema schema = schemas.get(i);
       List<ByteBuffer> pageDataInChunk = dataInChunkGroup.get(i);
       List<PageHeader> pageHeadersInChunk = pageHeadersInChunkGroup.get(i);
-      List<Boolean> pagePartitionInfo = pagePartitionInfoInChunkGroup.get(i);
+      List<Boolean> needToDecodeInfoInChunk = needToDecodeInfoInChunkGroup.get(i);
       valueDecoder = Decoder
           .getDecoderByType(schema.getEncodingType(), schema.getType());
+      boolean isOnlyOnePageChunk = pageDataInChunk.size() == 1;
       for (int j = 0; j < pageDataInChunk.size(); j++) {
-        if (Boolean.TRUE.equals(pagePartitionInfo.get(j))) {
-          writePageInSamePartitionToFile(oldTsFile, schema, pageHeadersInChunk.get(j),
-              pageDataInChunk.get(j), chunkWritersInChunkGroup);
-        } else {
-          writePageInDifferentPartitionsToFiles(oldTsFile, schema, pageDataInChunk.get(j),
+        if (Boolean.TRUE.equals(needToDecodeInfoInChunk.get(j))) {
+          decodeAndWritePageInToFiles(oldTsFile, schema, pageDataInChunk.get(j),
               chunkWritersInChunkGroup);
+        } else {
+          writePageInToFile(oldTsFile, schema, pageHeadersInChunk.get(j),
+              pageDataInChunk.get(j), chunkWritersInChunkGroup, isOnlyOnePageChunk);
         }
       }
     }
@@ -456,14 +313,18 @@ public class TsFileOnlineUpgradeTool implements AutoCloseable {
           if (!partitionDir.exists()) {
             partitionDir.mkdirs();
           }
-          File newFile = FSFactoryProducer.getFSFactory().getFile(oldTsFile.getParent()
-              + File.separator + partition + File.separator + oldTsFile.getName());
+          File newFile = FSFactoryProducer.getFSFactory().getFile(partitionDir 
+              + File.separator + oldTsFile.getName());
           try {
             if (!newFile.createNewFile()) {
               logger.error("The TsFile {} has been created ", newFile);
               return null;
             }
-            return new TsFileIOWriter(newFile);
+            TsFileIOWriter writer = new TsFileIOWriter(newFile);
+            if (oldModification != null) {
+              fileModificationMap.put(writer, new ModificationFile(newFile + ModificationFile.FILE_SUFFIX));
+            }
+            return writer;
           } catch (IOException e) {
             logger.error("Create new TsFile {} failed ", newFile);
             return null;
@@ -472,10 +333,11 @@ public class TsFileOnlineUpgradeTool implements AutoCloseable {
     );
   }
 
-  private void writePageInSamePartitionToFile(File oldTsFile, MeasurementSchema schema,
+  private void writePageInToFile(File oldTsFile, MeasurementSchema schema,
       PageHeader pageHeader,
       ByteBuffer pageData,
-      Map<Long, Map<MeasurementSchema, ChunkWriterImpl>> chunkWritersInChunkGroup)
+      Map<Long, Map<MeasurementSchema, ChunkWriterImpl>> chunkWritersInChunkGroup,
+      boolean isOnlyOnePageChunk)
       throws PageException {
     long partitionId = StorageEngine.getTimePartition(pageHeader.getStartTime());
     getOrDefaultTsFileIOWriter(oldTsFile, partitionId);
@@ -483,17 +345,17 @@ public class TsFileOnlineUpgradeTool implements AutoCloseable {
         .getOrDefault(partitionId, new HashMap<>());
     ChunkWriterImpl chunkWriter = chunkWriters
         .getOrDefault(schema, new ChunkWriterImpl(schema));
-    chunkWriter.writePageHeaderAndDataIntoBuff(pageData, pageHeader);
+    chunkWriter.writePageHeaderAndDataIntoBuff(pageData, pageHeader, isOnlyOnePageChunk);
     chunkWriters.put(schema, chunkWriter);
     chunkWritersInChunkGroup.put(partitionId, chunkWriters);
   }
 
-  private void writePageInDifferentPartitionsToFiles(File oldTsFile, MeasurementSchema schema,
+  private void decodeAndWritePageInToFiles(File oldTsFile, MeasurementSchema schema,
       ByteBuffer pageData,
       Map<Long, Map<MeasurementSchema, ChunkWriterImpl>> chunkWritersInChunkGroup)
       throws IOException {
     valueDecoder.reset();
-    PageReader pageReader = new PageReader(pageData, schema.getType(), valueDecoder,
+    PageReaderV2 pageReader = new PageReaderV2(pageData, schema.getType(), valueDecoder,
         defaultTimeDecoder, null);
     BatchData batchData = pageReader.getAllSatisfiedPageData();
     while (batchData.hasCurrent()) {
@@ -538,32 +400,22 @@ public class TsFileOnlineUpgradeTool implements AutoCloseable {
   /**
    * check if the file to be upgraded has correct magic strings and version number
    */
-  private boolean fileCheck(File oldTsFile) throws IOException {
-    long fileSize;
-    if (!oldTsFile.exists()) {
-      logger.error("the file to be updated does not exist, file path: {}", oldTsFile.getPath());
-      return false;
-    } else {
-      fileSize = oldTsFile.length();
-    }
+  private boolean fileCheck() throws IOException {
 
-    String magic = readHeadMagic(true);
+    String magic = reader.readHeadMagic();
     if (!magic.equals(TSFileConfig.MAGIC_STRING)) {
-      logger.error("the file's MAGIC STRING is incorrect, file path: {}", oldTsFile.getPath());
+      logger.error("the file's MAGIC STRING is incorrect, file path: {}", reader.getFileName());
       return false;
     }
 
-    String versionNumber = readVersionNumber();
-    if (!versionNumber.equals(TSFileConfig.VERSION_NUMBER_V1)) {
-      logger.error("the file's Version Number is incorrect, file path: {}", oldTsFile.getPath());
+    String versionNumber = reader.readVersionNumberV2();
+    if (!versionNumber.equals(TSFileConfig.VERSION_NUMBER_V2)) {
+      logger.error("the file's Version Number is incorrect, file path: {}", reader.getFileName());
       return false;
     }
 
-    if (fileSize == TSFileConfig.MAGIC_STRING.length()) {
-      logger.error("the file only contains magic string, file path: {}", oldTsFile.getPath());
-      return false;
-    } else if (!readTailMagic().equals(TSFileConfig.MAGIC_STRING)) {
-      logger.error("the file cannot upgrade, file path: {}", oldTsFile.getPath());
+    if (!reader.readTailMagic().equals(TSFileConfig.MAGIC_STRING)) {
+      logger.error("the file is not closed correctly, file path: {}", reader.getFileName());
       return false;
     }
     return true;
@@ -584,6 +436,7 @@ public class TsFileOnlineUpgradeTool implements AutoCloseable {
       }
     }
     tsFileResource.setClosed(true);
+    tsFileResource.serialize();
     return tsFileResource;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/FileLoaderUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/FileLoaderUtils.java
index ba9e0ec..88240ce 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/FileLoaderUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/FileLoaderUtils.java
@@ -82,12 +82,13 @@ public class FileLoaderUtils {
   }
 
   /**
-   * @param resource TsFile
+   * @param resource   TsFile
    * @param seriesPath Timeseries path
    * @param allSensors measurements queried at the same time of this device
-   * @param filter any filter, only used to check time range
+   * @param filter     any filter, only used to check time range
    */
-  public static TimeseriesMetadata loadTimeSeriesMetadata(TsFileResource resource, PartialPath seriesPath,
+  public static TimeseriesMetadata loadTimeSeriesMetadata(TsFileResource resource,
+      PartialPath seriesPath,
       QueryContext context, Filter filter, Set<String> allSensors) throws IOException {
     TimeseriesMetadata timeSeriesMetadata;
     if (resource.isClosed()) {
@@ -140,7 +141,7 @@ public class FileLoaderUtils {
    * load all page readers in one chunk that satisfying the timeFilter
    *
    * @param chunkMetaData the corresponding chunk metadata
-   * @param timeFilter it should be a TimeFilter instead of a ValueFilter
+   * @param timeFilter    it should be a TimeFilter instead of a ValueFilter
    */
   public static List<IPageReader> loadPageReaderList(ChunkMetadata chunkMetaData, Filter timeFilter)
       throws IOException {
@@ -154,7 +155,8 @@ public class FileLoaderUtils {
       chunkReader = new MemChunkReader(memChunkLoader.getChunk(), timeFilter);
     } else {
       Chunk chunk = chunkLoader.loadChunk(chunkMetaData);
-      chunkReader = new ChunkReader(chunk, timeFilter, chunkMetaData.isFromOldTsFile());
+      chunk.setFromOldFile(chunkMetaData.isFromOldTsFile());
+      chunkReader = new ChunkReader(chunk, timeFilter);
       chunkReader.hasNextSatisfiedPage();
     }
     return chunkReader.loadPageReaderList();
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java
index 9265083..4981b1c 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java
@@ -48,6 +48,15 @@ public class QueryUtils {
     for (int metaIndex = 0; metaIndex < chunkMetaData.size(); metaIndex++) {
       ChunkMetadata metaData = chunkMetaData.get(metaIndex);
       for (Modification modification : modifications) {
+        // When the chunkMetadata come from an old TsFile, the method modification.getFileOffset() 
+        // is gerVersionNum actually. In this case, we compare the versions of modification and
+        // mataData to determine whether need to do modify.
+        if (metaData.isFromOldTsFile()) {
+          if (modification.getFileOffset() > metaData.getVersion()) {
+            doModifyChunkMetaData(modification, metaData);
+          }
+          continue;
+        }
         // The case modification.getFileOffset() == metaData.getOffsetOfChunkHeader()
         // is not supposed to exist as getFileOffset() is offset containing full chunk,
         // while getOffsetOfChunkHeader() returns the chunk header offset
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/UpgradeUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/UpgradeUtils.java
index c213f64..e16a118 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/UpgradeUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/UpgradeUtils.java
@@ -24,15 +24,19 @@ import java.io.FileReader;
 import java.io.IOException;
 import java.nio.file.Files;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.iotdb.db.engine.modification.ModificationFile;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.engine.upgrade.UpgradeCheckStatus;
 import org.apache.iotdb.db.engine.upgrade.UpgradeLog;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
-import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
+import org.apache.iotdb.tsfile.v2.read.TsFileSequenceReaderForV2;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -45,6 +49,9 @@ public class UpgradeUtils {
   private static final ReadWriteLock cntUpgradeFileLock = new ReentrantReadWriteLock();
   private static final ReadWriteLock upgradeLogLock = new ReentrantReadWriteLock();
 
+  private static FSFactory fsFactory = FSFactoryProducer.getFSFactory();
+
+  private static Map<String, Integer> upgradeRecoverMap = new HashMap<>();
   public static ReadWriteLock getCntUpgradeFileLock() {
     return cntUpgradeFileLock;
   }
@@ -66,12 +73,15 @@ public class UpgradeUtils {
     } finally {
       tsFileResource.readUnlock();
     }
-    try (TsFileSequenceReader tsFileSequenceReader = new TsFileSequenceReader(
+    tsFileResource.readLock();
+    try (TsFileSequenceReaderForV2 tsFileSequenceReader = new TsFileSequenceReaderForV2(
         tsFileResource.getTsFile().getAbsolutePath())) {
-      if (tsFileSequenceReader.readVersionNumber().equals(TSFileConfig.VERSION_NUMBER_V1)) {
+      String versionNumber = tsFileSequenceReader.readVersionNumberV2();
+      if (versionNumber.equals(TSFileConfig.VERSION_NUMBER_V2) 
+          || versionNumber.equals(TSFileConfig.VERSION_NUMBER_V1)) {
         return true;
       }
-    } catch (Exception e) {
+    } catch (IOException e) {
       logger.error("meet error when judge whether file needs to be upgraded, the file's path:{}",
           tsFileResource.getTsFile().getAbsolutePath(), e);
     } finally {
@@ -80,21 +90,53 @@ public class UpgradeUtils {
     return false;
   }
 
-  /**
-   * Since one old TsFile may be upgraded to multiple upgraded files, 
-   * this method is for getting the name of one of the upgraded file. 
-   * 
-   * @param upgradeResource TsFile resource to be upgraded
-   * @return name of upgraded file
-   * 
-   */
-  public static String getOneUpgradedFileName(TsFileResource upgradeResource)
-      throws IOException {
-    upgradeResource.deserialize();
-    long firstPartitionId = upgradeResource.getTimePartition();
-    File oldTsFile = upgradeResource.getTsFile();
-    return oldTsFile.getParent()
-        + File.separator + firstPartitionId + File.separator+ oldTsFile.getName();
+  public static void moveUpgradedFiles(TsFileResource resource) throws IOException {
+    List<TsFileResource> upgradedResources = resource.getUpgradedResources();
+    for (TsFileResource upgradedResource : upgradedResources) {
+      File upgradedFile = upgradedResource.getTsFile();
+      long partition = upgradedResource.getTimePartition();
+      String virtualStorageGroupDir = upgradedFile.getParentFile().getParentFile().getParent();
+      File partitionDir = fsFactory.getFile(virtualStorageGroupDir, String.valueOf(partition));
+      if (!partitionDir.exists()) {
+        partitionDir.mkdir();
+      }
+      // move upgraded TsFile
+      fsFactory.moveFile(upgradedFile,
+          fsFactory.getFile(partitionDir, upgradedFile.getName()));
+      // delete generated temp resource
+      Files.delete(fsFactory
+          .getFile(upgradedResource.getTsFile().toPath() + TsFileResource.RESOURCE_SUFFIX).toPath());
+      // move upgraded mods file
+      File newModsFile = fsFactory
+          .getFile(upgradedResource.getTsFile().toPath() + ModificationFile.FILE_SUFFIX);
+      if (newModsFile.exists()) {
+        fsFactory.moveFile(newModsFile,
+            fsFactory.getFile(partitionDir, newModsFile.getName()));
+      }
+      // re-serialize upgraded resource to correct place
+      upgradedResource.setFile(
+          fsFactory.getFile(partitionDir, upgradedFile.getName()));
+      if (newModsFile.exists()) {
+        upgradedResource.getModFile();
+      }
+      upgradedResource.serialize();
+      // delete tmp partition folder when it is empty
+      File tmpPartitionDir = upgradedFile.getParentFile();
+      if (tmpPartitionDir.isDirectory() && tmpPartitionDir.listFiles().length == 0) {
+        Files.delete(tmpPartitionDir.toPath());
+      }
+      // delete upgrade folder when it is empty
+      File upgradeDir = tmpPartitionDir.getParentFile();
+      if (upgradeDir.isDirectory() && upgradeDir.listFiles().length == 0) {
+        Files.delete(upgradeDir.toPath());
+      }
+    }
+  }
+
+  public static boolean isUpgradedFileGenerated(String oldFileName) {
+    return upgradeRecoverMap.containsKey(oldFileName) 
+        && upgradeRecoverMap.get(oldFileName) == UpgradeCheckStatus.AFTER_UPGRADE_FILE
+        .getCheckStatusCode();
   }
 
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
@@ -103,7 +145,6 @@ public class UpgradeUtils {
       try (BufferedReader upgradeLogReader = new BufferedReader(
           new FileReader(
               FSFactoryProducer.getFSFactory().getFile(UpgradeLog.getUpgradeLogPath())))) {
-        Map<String, Integer> upgradeRecoverMap = new HashMap<>();
         String line = null;
         while ((line = upgradeLogReader.readLine()) != null) {
           String oldFileName = line.split(COMMA_SEPERATOR)[0];
@@ -127,40 +168,11 @@ public class UpgradeUtils {
                   if (generatedFile.getName().equals(FSFactoryProducer.getFSFactory()
                       .getFile(key).getName())) {
                     Files.delete(generatedFile.toPath());
+                    Files.deleteIfExists(new File(generatedFile + ModificationFile.FILE_SUFFIX).toPath());
                   }
                 }
               }
             }
-          } else if (upgradeRecoverMap.get(key) == UpgradeCheckStatus.AFTER_UPGRADE_FILE
-              .getCheckStatusCode()) {
-            String upgradedFileName = getOneUpgradedFileName(new TsFileResource(
-                FSFactoryProducer.getFSFactory().getFile(key)));
-            if (FSFactoryProducer.getFSFactory().getFile(key).exists() && FSFactoryProducer
-                .getFSFactory().getFile(upgradedFileName).exists()) {
-              // if both old tsfile and upgrade file exists, delete the old tsfile and resource
-              Files.delete(FSFactoryProducer.getFSFactory().getFile(key).toPath());
-              Files.delete(FSFactoryProducer.getFSFactory().getFile(key 
-                  + TsFileResource.RESOURCE_SUFFIX).toPath());
-            } 
-            // move the upgrade files and resources to their own partition directories
-            File upgradeDir = FSFactoryProducer.getFSFactory().getFile(key)
-                .getParentFile();
-            String storageGroupPath = upgradeDir.getParent();
-            File[] partitionDirs = upgradeDir.listFiles();
-            for (File partitionDir : partitionDirs) {
-              if (partitionDir.isDirectory()) {
-                String partitionId = partitionDir.getName();
-                File destPartitionDir = FSFactoryProducer.getFSFactory().getFile(storageGroupPath, partitionId);
-                if (!destPartitionDir.exists()) {
-                  destPartitionDir.mkdir();
-                }
-                File[] generatedFiles = partitionDir.listFiles();
-                for (File generatedFile : generatedFiles) {
-                  FSFactoryProducer.getFSFactory().moveFile(generatedFile, 
-                      FSFactoryProducer.getFSFactory().getFile(destPartitionDir, generatedFile.getName()));
-                }
-              }
-            }
           }
         }
       } catch (IOException e) {
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java
index 1e50590..7036f75 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java
@@ -78,8 +78,8 @@ public class MaxFileMergeFileSelectorTest extends MergeTest {
     List[] result = mergeFileSelector.select();
     List<TsFileResource> seqSelected = result[0];
     List<TsFileResource> unseqSelected = result[1];
-    assertEquals(seqResources.subList(0, 3), seqSelected);
-    assertEquals(unseqResources.subList(0, 3), unseqSelected);
+    assertEquals(seqResources.subList(0, 4), seqSelected);
+    assertEquals(unseqResources.subList(0, 4), unseqSelected);
     resource.clear();
   }
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxSeriesMergeFileSelectorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxSeriesMergeFileSelectorTest.java
index 2a876d1..2e89ee5 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxSeriesMergeFileSelectorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxSeriesMergeFileSelectorTest.java
@@ -85,8 +85,8 @@ public class MaxSeriesMergeFileSelectorTest extends MergeTest {
     List[] result = mergeFileSelector.select();
     List<TsFileResource> seqSelected = result[0];
     List<TsFileResource> unseqSelected = result[1];
-    assertEquals(seqResources.subList(0, 3), seqSelected);
-    assertEquals(unseqResources.subList(0, 3), unseqSelected);
+    assertEquals(seqResources.subList(0, 4), seqSelected);
+    assertEquals(unseqResources.subList(0, 4), unseqSelected);
     assertEquals(MaxSeriesMergeFileSelector.MAX_SERIES_NUM,
         mergeFileSelector.getConcurrentMergeNum());
     resource.clear();
@@ -100,8 +100,8 @@ public class MaxSeriesMergeFileSelectorTest extends MergeTest {
     List[] result = mergeFileSelector.select();
     List<TsFileResource> seqSelected = result[0];
     List<TsFileResource> unseqSelected = result[1];
-    assertEquals(seqResources.subList(0, 1), seqSelected);
-    assertEquals(unseqResources.subList(0, 1), unseqSelected);
+    assertEquals(seqResources.subList(0, 2), seqSelected);
+    assertEquals(unseqResources.subList(0, 2), unseqSelected);
     resource.clear();
   }
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBNewTsFileCompactionIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBNewTsFileCompactionIT.java
new file mode 100644
index 0000000..e2ebf0b
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBNewTsFileCompactionIT.java
@@ -0,0 +1,1018 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.integration;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.concurrent.TimeUnit;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.level.LevelCompactionTsFileManagement;
+import org.apache.iotdb.db.engine.storagegroup.StorageGroupProcessor;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.jdbc.Config;
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class IoTDBNewTsFileCompactionIT {
+
+  private int prevSeqLevelFileNum;
+  private int prevSeqLevelNum;
+  private int prevMergePagePointNumber;
+  private int preMaxNumberOfPointsInPage;
+  private CompactionStrategy preCompactionStrategy;
+  private PartialPath storageGroupPath;
+  // the unit is ns
+  private static final long MAX_WAIT_TIME_FOR_MERGE = 5_000_000_000L;
+  private static final float FLOAT_DELTA = 0.00001f;
+
+  @Before
+  public void setUp() throws Exception {
+    EnvironmentUtils.closeStatMonitor();
+    prevSeqLevelFileNum = IoTDBDescriptor.getInstance().getConfig().getSeqFileNumInEachLevel();
+    prevSeqLevelNum = IoTDBDescriptor.getInstance().getConfig().getSeqLevelNum();
+    prevMergePagePointNumber = IoTDBDescriptor.getInstance().getConfig()
+        .getMergePagePointNumberThreshold();
+    preMaxNumberOfPointsInPage = TSFileDescriptor.getInstance().getConfig()
+        .getMaxNumberOfPointsInPage();
+    preCompactionStrategy = IoTDBDescriptor.getInstance().getConfig().getCompactionStrategy();
+    storageGroupPath = new PartialPath("root.sg1");
+    IoTDBDescriptor.getInstance().getConfig().setSeqFileNumInEachLevel(2);
+    IoTDBDescriptor.getInstance().getConfig().setSeqLevelNum(2);
+    IoTDBDescriptor.getInstance().getConfig().setMergePagePointNumberThreshold(1);
+    TSFileDescriptor.getInstance().getConfig().setMaxNumberOfPointsInPage(1);
+    IoTDBDescriptor.getInstance().getConfig()
+        .setCompactionStrategy(CompactionStrategy.LEVEL_COMPACTION);
+    EnvironmentUtils.envSetUp();
+    Class.forName(Config.JDBC_DRIVER_NAME);
+
+    try (Connection connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+
+      statement.execute("SET STORAGE GROUP TO root.sg1");
+    }
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    EnvironmentUtils.cleanEnv();
+    IoTDBDescriptor.getInstance().getConfig().setSeqFileNumInEachLevel(prevSeqLevelFileNum);
+    IoTDBDescriptor.getInstance().getConfig().setSeqLevelNum(prevSeqLevelNum);
+    IoTDBDescriptor.getInstance().getConfig()
+        .setMergePagePointNumberThreshold(prevMergePagePointNumber);
+    TSFileDescriptor.getInstance().getConfig()
+        .setMaxNumberOfPointsInPage(preMaxNumberOfPointsInPage);
+    IoTDBDescriptor.getInstance().getConfig().setCompactionStrategy(preCompactionStrategy);
+  }
+
+  /**
+   * first file has only one page for each chunk and only one chunk for each time series
+   * second file has only one page for each chunk and only one chunk for each time series
+   */
+  @Test
+  public void test1() throws SQLException {
+    String[][] retArray = {
+        {"1", "1"},
+        {"2", "2"}
+    };
+    int preAvgSeriesPointNumberThreshold = IoTDBDescriptor.getInstance().getConfig()
+        .getAvgSeriesPointNumberThreshold();
+    try (Connection connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(10000);
+
+      // first file
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(1, 1)");
+      statement.execute("FLUSH");
+
+      // second file
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(2, 2)");
+      statement.execute("FLUSH");
+
+      assertTrue(waitForMergeFinish());
+
+      int cnt;
+      try (ResultSet resultSet = statement.executeQuery("SELECT s1 FROM root.sg1.d1")) {
+        cnt = 0;
+        while (resultSet.next()) {
+          long time = resultSet.getLong("Time");
+          float s1 = resultSet.getFloat("root.sg1.d1.s1");
+          assertEquals(Long.parseLong(retArray[cnt][0]), time);
+          assertEquals(Float.parseFloat(retArray[cnt][1]), s1, FLOAT_DELTA);
+          cnt++;
+        }
+      }
+      assertEquals(retArray.length, cnt);
+    } catch (StorageEngineException | InterruptedException e) {
+      e.printStackTrace();
+      fail();
+    } finally {
+      IoTDBDescriptor.getInstance().getConfig()
+          .setAvgSeriesPointNumberThreshold(preAvgSeriesPointNumberThreshold);
+    }
+  }
+
+  /**
+   * first file has only one page for each chunk and two chunks for each time series
+   * second file has only one page for each chunk and only one chunk for each time series
+   */
+  @Test
+  public void test2() throws SQLException {
+    String[][] retArray = {
+        {"1", "1"},
+        {"2", "2"},
+        {"3", "3"}
+    };
+    int preAvgSeriesPointNumberThreshold = IoTDBDescriptor.getInstance().getConfig()
+        .getAvgSeriesPointNumberThreshold();
+    try (Connection connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(1);
+
+      // first file
+      // two chunks
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(1, 1)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(2, 2)");
+      statement.execute("FLUSH");
+
+      // second file
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(3, 3)");
+      statement.execute("FLUSH");
+
+      assertTrue(waitForMergeFinish());
+
+      int cnt;
+      try (ResultSet resultSet = statement.executeQuery("SELECT s1 FROM root.sg1.d1")) {
+        cnt = 0;
+        while (resultSet.next()) {
+          long time = resultSet.getLong("Time");
+          float s1 = resultSet.getFloat("root.sg1.d1.s1");
+          assertEquals(Long.parseLong(retArray[cnt][0]), time);
+          assertEquals(Float.parseFloat(retArray[cnt][1]), s1, FLOAT_DELTA);
+          cnt++;
+        }
+      }
+      assertEquals(retArray.length, cnt);
+    } catch (StorageEngineException | InterruptedException e) {
+      e.printStackTrace();
+      fail();
+    } finally {
+      IoTDBDescriptor.getInstance().getConfig()
+          .setAvgSeriesPointNumberThreshold(preAvgSeriesPointNumberThreshold);
+    }
+  }
+
+  /**
+   * first file has two pages for each chunk and only one chunk for each time series
+   * second file has only one page for each chunk and only one chunk for each time series
+   */
+  @Test
+  public void test3() throws SQLException {
+    String[][] retArray = {
+        {"1", "1"},
+        {"2", "2"},
+        {"3", "3"},
+    };
+    int preAvgSeriesPointNumberThreshold = IoTDBDescriptor.getInstance().getConfig()
+        .getAvgSeriesPointNumberThreshold();
+    try (Connection connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(10000);
+
+      // first file
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(1, 1)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(2, 2)");
+      statement.execute("FLUSH");
+
+      // second file
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(3, 3)");
+      statement.execute("FLUSH");
+
+      assertTrue(waitForMergeFinish());
+
+      int cnt;
+      try (ResultSet resultSet = statement.executeQuery("SELECT s1 FROM root.sg1.d1")) {
+        cnt = 0;
+        while (resultSet.next()) {
+          long time = resultSet.getLong("Time");
+          float s1 = resultSet.getFloat("root.sg1.d1.s1");
+          assertEquals(Long.parseLong(retArray[cnt][0]), time);
+          assertEquals(Float.parseFloat(retArray[cnt][1]), s1, FLOAT_DELTA);
+          cnt++;
+        }
+      }
+      assertEquals(retArray.length, cnt);
+    } catch (StorageEngineException | InterruptedException e) {
+      e.printStackTrace();
+      fail();
+    } finally {
+      IoTDBDescriptor.getInstance().getConfig()
+          .setAvgSeriesPointNumberThreshold(preAvgSeriesPointNumberThreshold);
+    }
+  }
+
+  /**
+   * first file has two pages for each chunk and two chunks for each time series
+   * second file has only one page for each chunk and only one chunk for each time series
+   */
+  @Test
+  public void test4() throws SQLException {
+    String[][] retArray = {
+        {"1", "1"},
+        {"2", "2"},
+        {"3", "3"},
+        {"4", "4"},
+        {"5", "5"},
+    };
+    int preAvgSeriesPointNumberThreshold = IoTDBDescriptor.getInstance().getConfig()
+        .getAvgSeriesPointNumberThreshold();
+    try (Connection connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(2);
+
+      // first file
+      // one chunk with two pages
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(1, 1)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(2, 2)");
+      // another chunk with two pages
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(3, 3)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(4, 4)");
+      statement.execute("FLUSH");
+
+      // second file
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(5, 5)");
+      statement.execute("FLUSH");
+
+      assertTrue(waitForMergeFinish());
+
+      int cnt;
+      try (ResultSet resultSet = statement.executeQuery("SELECT s1 FROM root.sg1.d1")) {
+        cnt = 0;
+        while (resultSet.next()) {
+          long time = resultSet.getLong("Time");
+          float s1 = resultSet.getFloat("root.sg1.d1.s1");
+          assertEquals(Long.parseLong(retArray[cnt][0]), time);
+          assertEquals(Float.parseFloat(retArray[cnt][1]), s1, FLOAT_DELTA);
+          cnt++;
+        }
+      }
+      assertEquals(retArray.length, cnt);
+    } catch (StorageEngineException | InterruptedException e) {
+      e.printStackTrace();
+      fail();
+    } finally {
+      IoTDBDescriptor.getInstance().getConfig()
+          .setAvgSeriesPointNumberThreshold(preAvgSeriesPointNumberThreshold);
+    }
+  }
+
+  /**
+   * first file has only one page for each chunk and only one chunk for each time series
+   * second file has two pages for each chunk and only one chunk for each time series
+   */
+  @Test
+  public void test5() throws SQLException {
+    String[][] retArray = {
+        {"1", "1"},
+        {"2", "2"},
+        {"3", "3"},
+    };
+    int preAvgSeriesPointNumberThreshold = IoTDBDescriptor.getInstance().getConfig()
+        .getAvgSeriesPointNumberThreshold();
+    try (Connection connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(10000);
+
+      // first file
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(1, 1)");
+      statement.execute("FLUSH");
+
+      // second file
+      // two pages for one chunk
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(2, 2)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(3, 3)");
+      statement.execute("FLUSH");
+
+      assertTrue(waitForMergeFinish());
+
+      int cnt;
+      try (ResultSet resultSet = statement.executeQuery("SELECT s1 FROM root.sg1.d1")) {
+        cnt = 0;
+        while (resultSet.next()) {
+          long time = resultSet.getLong("Time");
+          float s1 = resultSet.getFloat("root.sg1.d1.s1");
+          assertEquals(Long.parseLong(retArray[cnt][0]), time);
+          assertEquals(Float.parseFloat(retArray[cnt][1]), s1, FLOAT_DELTA);
+          cnt++;
+        }
+      }
+      assertEquals(retArray.length, cnt);
+    } catch (StorageEngineException | InterruptedException e) {
+      e.printStackTrace();
+      fail();
+    } finally {
+      IoTDBDescriptor.getInstance().getConfig()
+          .setAvgSeriesPointNumberThreshold(preAvgSeriesPointNumberThreshold);
+    }
+  }
+
+  /**
+   * first file has only one page for each chunk and two chunks for each time series
+   * second file has two pages for each chunk and only one chunk for each time series
+   */
+  @Test
+  public void test6() throws SQLException {
+    String[][] retArray = {
+        {"1", "1"},
+        {"2", "2"},
+        {"3", "3"},
+        {"4", "4"},
+    };
+    int preAvgSeriesPointNumberThreshold = IoTDBDescriptor.getInstance().getConfig()
+        .getAvgSeriesPointNumberThreshold();
+    try (Connection connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(1);
+
+      // first file
+      // two chunks
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(1, 1)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(2, 2)");
+      statement.execute("FLUSH");
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(2);
+      // second file
+      // two pages for one chunk
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(3, 3)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(4, 4)");
+
+      statement.execute("FLUSH");
+
+      assertTrue(waitForMergeFinish());
+
+      int cnt;
+      try (ResultSet resultSet = statement.executeQuery("SELECT s1 FROM root.sg1.d1")) {
+        cnt = 0;
+        while (resultSet.next()) {
+          long time = resultSet.getLong("Time");
+          float s1 = resultSet.getFloat("root.sg1.d1.s1");
+          assertEquals(Long.parseLong(retArray[cnt][0]), time);
+          assertEquals(Float.parseFloat(retArray[cnt][1]), s1, FLOAT_DELTA);
+          cnt++;
+        }
+      }
+      assertEquals(retArray.length, cnt);
+    } catch (StorageEngineException | InterruptedException e) {
+      e.printStackTrace();
+      fail();
+    } finally {
+      IoTDBDescriptor.getInstance().getConfig()
+          .setAvgSeriesPointNumberThreshold(preAvgSeriesPointNumberThreshold);
+    }
+  }
+
+  /**
+   * first file has two pages for each chunk and only one chunk for each time series
+   * second file has two pages for each chunk and only one chunk for each time series
+   */
+  @Test
+  public void test7() throws SQLException {
+    String[][] retArray = {
+        {"1", "1"},
+        {"2", "2"},
+        {"3", "3"},
+        {"4", "4"}
+    };
+    int preAvgSeriesPointNumberThreshold = IoTDBDescriptor.getInstance().getConfig()
+        .getAvgSeriesPointNumberThreshold();
+    try (Connection connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(10000);
+
+      // first file
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(1, 1)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(2, 2)");
+      statement.execute("FLUSH");
+
+      // second file
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(3, 3)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(4, 4)");
+      statement.execute("FLUSH");
+
+      assertTrue(waitForMergeFinish());
+
+      int cnt;
+      try (ResultSet resultSet = statement.executeQuery("SELECT s1 FROM root.sg1.d1")) {
+        cnt = 0;
+        while (resultSet.next()) {
+          long time = resultSet.getLong("Time");
+          float s1 = resultSet.getFloat("root.sg1.d1.s1");
+          assertEquals(Long.parseLong(retArray[cnt][0]), time);
+          assertEquals(Float.parseFloat(retArray[cnt][1]), s1, FLOAT_DELTA);
+          cnt++;
+        }
+      }
+      assertEquals(retArray.length, cnt);
+    } catch (StorageEngineException | InterruptedException e) {
+      e.printStackTrace();
+      fail();
+    } finally {
+      IoTDBDescriptor.getInstance().getConfig()
+          .setAvgSeriesPointNumberThreshold(preAvgSeriesPointNumberThreshold);
+    }
+  }
+
+  /**
+   * first file has two pages for each chunk and two chunks for each time series
+   * second file has two pages for each chunk and only one chunk for each time series
+   */
+  @Test
+  public void test8() throws SQLException {
+    String[][] retArray = {
+        {"1", "1"},
+        {"2", "2"},
+        {"3", "3"},
+        {"4", "4"},
+        {"5", "5"},
+        {"6", "6"}
+    };
+    int preAvgSeriesPointNumberThreshold = IoTDBDescriptor.getInstance().getConfig()
+        .getAvgSeriesPointNumberThreshold();
+    try (Connection connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(2);
+
+      // first file
+      // one chunk with two pages
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(1, 1)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(2, 2)");
+      // another chunk with two pages
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(3, 3)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(4, 4)");
+      statement.execute("FLUSH");
+
+      // second file
+      // two pages for one chunk
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(5, 5)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(6, 6)");
+      statement.execute("FLUSH");
+
+      assertTrue(waitForMergeFinish());
+
+      int cnt;
+      try (ResultSet resultSet = statement.executeQuery("SELECT s1 FROM root.sg1.d1")) {
+        cnt = 0;
+        while (resultSet.next()) {
+          long time = resultSet.getLong("Time");
+          float s1 = resultSet.getFloat("root.sg1.d1.s1");
+          assertEquals(Long.parseLong(retArray[cnt][0]), time);
+          assertEquals(Float.parseFloat(retArray[cnt][1]), s1, FLOAT_DELTA);
+          cnt++;
+        }
+      }
+      assertEquals(retArray.length, cnt);
+    } catch (StorageEngineException | InterruptedException e) {
+      e.printStackTrace();
+      fail();
+    } finally {
+      IoTDBDescriptor.getInstance().getConfig()
+          .setAvgSeriesPointNumberThreshold(preAvgSeriesPointNumberThreshold);
+    }
+  }
+
+  /**
+   * first file has only one page for each chunk and only one chunk for each time series
+   * second file has only one page for each chunk and two chunks for each time series
+   */
+  @Test
+  public void test9() throws SQLException {
+    String[][] retArray = {
+        {"1", "1"},
+        {"2", "2"},
+        {"3", "3"},
+    };
+    int preAvgSeriesPointNumberThreshold = IoTDBDescriptor.getInstance().getConfig()
+        .getAvgSeriesPointNumberThreshold();
+    try (Connection connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(10000);
+
+      // first file
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(1, 1)");
+      statement.execute("FLUSH");
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(1);
+      // second file
+      // two pages for one chunk
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(2, 2)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(3, 3)");
+      statement.execute("FLUSH");
+
+      assertTrue(waitForMergeFinish());
+
+      int cnt;
+      try (ResultSet resultSet = statement.executeQuery("SELECT s1 FROM root.sg1.d1")) {
+        cnt = 0;
+        while (resultSet.next()) {
+          long time = resultSet.getLong("Time");
+          float s1 = resultSet.getFloat("root.sg1.d1.s1");
+          assertEquals(Long.parseLong(retArray[cnt][0]), time);
+          assertEquals(Float.parseFloat(retArray[cnt][1]), s1, FLOAT_DELTA);
+          cnt++;
+        }
+      }
+      assertEquals(retArray.length, cnt);
+    } catch (StorageEngineException | InterruptedException e) {
+      e.printStackTrace();
+      fail();
+    } finally {
+      IoTDBDescriptor.getInstance().getConfig()
+          .setAvgSeriesPointNumberThreshold(preAvgSeriesPointNumberThreshold);
+    }
+  }
+
+  /**
+   * first file has only one page for each chunk and two chunks for each time series
+   * second file has only one page for each chunk and two chunks for each time series
+   */
+  @Test
+  public void test10() throws SQLException {
+    String[][] retArray = {
+        {"1", "1"},
+        {"2", "2"},
+        {"3", "3"},
+        {"4", "4"}
+    };
+    int preAvgSeriesPointNumberThreshold = IoTDBDescriptor.getInstance().getConfig()
+        .getAvgSeriesPointNumberThreshold();
+    try (Connection connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(1);
+
+      // first file
+      // two chunks
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(1, 1)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(2, 2)");
+      statement.execute("FLUSH");
+
+      // second file
+      // two pages for one chunk
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(3, 3)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(4, 4)");
+      statement.execute("FLUSH");
+
+      assertTrue(waitForMergeFinish());
+
+      int cnt;
+      try (ResultSet resultSet = statement.executeQuery("SELECT s1 FROM root.sg1.d1")) {
+        cnt = 0;
+        while (resultSet.next()) {
+          long time = resultSet.getLong("Time");
+          float s1 = resultSet.getFloat("root.sg1.d1.s1");
+          assertEquals(Long.parseLong(retArray[cnt][0]), time);
+          assertEquals(Float.parseFloat(retArray[cnt][1]), s1, FLOAT_DELTA);
+          cnt++;
+        }
+      }
+      assertEquals(retArray.length, cnt);
+    } catch (StorageEngineException | InterruptedException e) {
+      e.printStackTrace();
+      fail();
+    } finally {
+      IoTDBDescriptor.getInstance().getConfig()
+          .setAvgSeriesPointNumberThreshold(preAvgSeriesPointNumberThreshold);
+    }
+  }
+
+  /**
+   * first file has two pages for each chunk and only one chunk for each time series
+   * second file has only one page for each chunk and two chunks for each time series
+   */
+  @Test
+  public void test11() throws SQLException {
+    String[][] retArray = {
+        {"1", "1"},
+        {"2", "2"},
+        {"3", "3"},
+        {"4", "4"}
+    };
+    int preAvgSeriesPointNumberThreshold = IoTDBDescriptor.getInstance().getConfig()
+        .getAvgSeriesPointNumberThreshold();
+    try (Connection connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(10000);
+
+      // first file
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(1, 1)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(2, 2)");
+      statement.execute("FLUSH");
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(1);
+      // second file
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(3, 3)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(4, 4)");
+      statement.execute("FLUSH");
+
+      assertTrue(waitForMergeFinish());
+
+      int cnt;
+      try (ResultSet resultSet = statement.executeQuery("SELECT s1 FROM root.sg1.d1")) {
+        cnt = 0;
+        while (resultSet.next()) {
+          long time = resultSet.getLong("Time");
+          float s1 = resultSet.getFloat("root.sg1.d1.s1");
+          assertEquals(Long.parseLong(retArray[cnt][0]), time);
+          assertEquals(Float.parseFloat(retArray[cnt][1]), s1, FLOAT_DELTA);
+          cnt++;
+        }
+      }
+      assertEquals(retArray.length, cnt);
+    } catch (StorageEngineException | InterruptedException e) {
+      e.printStackTrace();
+      fail();
+    } finally {
+      IoTDBDescriptor.getInstance().getConfig()
+          .setAvgSeriesPointNumberThreshold(preAvgSeriesPointNumberThreshold);
+    }
+  }
+
+  /**
+   * first file has two pages for each chunk and two chunks for each time series
+   * second file has only one page for each chunk and two chunks for each time series
+   */
+  @Test
+  public void test12() throws SQLException {
+    String[][] retArray = {
+        {"1", "1"},
+        {"2", "2"},
+        {"3", "3"},
+        {"4", "4"},
+        {"5", "5"},
+        {"6", "6"}
+    };
+    int preAvgSeriesPointNumberThreshold = IoTDBDescriptor.getInstance().getConfig()
+        .getAvgSeriesPointNumberThreshold();
+    try (Connection connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(2);
+      // first file
+      // one chunk with two pages
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(1, 1)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(2, 2)");
+      // another chunk with two pages
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(3, 3)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(4, 4)");
+      statement.execute("FLUSH");
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(1);
+      // second file
+      // two pages for one chunk
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(5, 5)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(6, 6)");
+      statement.execute("FLUSH");
+
+      assertTrue(waitForMergeFinish());
+
+      int cnt;
+      try (ResultSet resultSet = statement.executeQuery("SELECT s1 FROM root.sg1.d1")) {
+        cnt = 0;
+        while (resultSet.next()) {
+          long time = resultSet.getLong("Time");
+          float s1 = resultSet.getFloat("root.sg1.d1.s1");
+          assertEquals(Long.parseLong(retArray[cnt][0]), time);
+          assertEquals(Float.parseFloat(retArray[cnt][1]), s1, FLOAT_DELTA);
+          cnt++;
+        }
+      }
+      assertEquals(retArray.length, cnt);
+    } catch (StorageEngineException | InterruptedException e) {
+      e.printStackTrace();
+      fail();
+    } finally {
+      IoTDBDescriptor.getInstance().getConfig()
+          .setAvgSeriesPointNumberThreshold(preAvgSeriesPointNumberThreshold);
+    }
+  }
+
+  /**
+   * first file has only one page for each chunk and only one chunk for each time series
+   * second file has two pages for each chunk and two chunks for each time series
+   */
+  @Test
+  public void test13() throws SQLException {
+    String[][] retArray = {
+        {"1", "1"},
+        {"2", "2"},
+        {"3", "3"},
+        {"4", "4"},
+        {"5", "5"}
+    };
+    int preAvgSeriesPointNumberThreshold = IoTDBDescriptor.getInstance().getConfig()
+        .getAvgSeriesPointNumberThreshold();
+    try (Connection connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(10000);
+
+      // first file
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(1, 1)");
+      statement.execute("FLUSH");
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(2);
+      // second file
+      // one chunk with two pages
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(2, 2)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(3, 3)");
+      // another chunk with two pages
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(4, 4)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(5, 5)");
+      statement.execute("FLUSH");
+
+
+      assertTrue(waitForMergeFinish());
+
+      int cnt;
+      try (ResultSet resultSet = statement.executeQuery("SELECT s1 FROM root.sg1.d1")) {
+        cnt = 0;
+        while (resultSet.next()) {
+          long time = resultSet.getLong("Time");
+          float s1 = resultSet.getFloat("root.sg1.d1.s1");
+          assertEquals(Long.parseLong(retArray[cnt][0]), time);
+          assertEquals(Float.parseFloat(retArray[cnt][1]), s1, FLOAT_DELTA);
+          cnt++;
+        }
+      }
+      assertEquals(retArray.length, cnt);
+    } catch (StorageEngineException | InterruptedException e) {
+      e.printStackTrace();
+      fail();
+    } finally {
+      IoTDBDescriptor.getInstance().getConfig()
+          .setAvgSeriesPointNumberThreshold(preAvgSeriesPointNumberThreshold);
+    }
+  }
+
+  /**
+   * first file has only one page for each chunk and two chunks for each time series
+   * second file has two pages for each chunk and two chunks for each time series
+   */
+  @Test
+  public void test14() throws SQLException {
+    String[][] retArray = {
+        {"1", "1"},
+        {"2", "2"},
+        {"3", "3"},
+        {"4", "4"},
+        {"5", "5"},
+        {"6", "6"}
+    };
+    int preAvgSeriesPointNumberThreshold = IoTDBDescriptor.getInstance().getConfig()
+        .getAvgSeriesPointNumberThreshold();
+    try (Connection connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(1);
+
+      // first file
+      // two chunks
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(1, 1)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(2, 2)");
+      statement.execute("FLUSH");
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(2);
+      // second file
+      // one chunk with two pages
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(3, 3)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(4, 4)");
+      // another chunk with two pages
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(5, 5)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(6, 6)");
+      statement.execute("FLUSH");
+
+      assertTrue(waitForMergeFinish());
+
+      int cnt;
+      try (ResultSet resultSet = statement.executeQuery("SELECT s1 FROM root.sg1.d1")) {
+        cnt = 0;
+        while (resultSet.next()) {
+          long time = resultSet.getLong("Time");
+          float s1 = resultSet.getFloat("root.sg1.d1.s1");
+          assertEquals(Long.parseLong(retArray[cnt][0]), time);
+          assertEquals(Float.parseFloat(retArray[cnt][1]), s1, FLOAT_DELTA);
+          cnt++;
+        }
+      }
+      assertEquals(retArray.length, cnt);
+    } catch (StorageEngineException | InterruptedException e) {
+      e.printStackTrace();
+      fail();
+    } finally {
+      IoTDBDescriptor.getInstance().getConfig()
+          .setAvgSeriesPointNumberThreshold(preAvgSeriesPointNumberThreshold);
+    }
+  }
+
+  /**
+   * first file has two pages for each chunk and only one chunk for each time series
+   * second file has two pages for each chunk and two chunks for each time series
+   */
+  @Test
+  public void test15() throws SQLException {
+    String[][] retArray = {
+        {"1", "1"},
+        {"2", "2"},
+        {"3", "3"},
+        {"4", "4"},
+        {"5", "5"},
+        {"6", "6"}
+    };
+    int preAvgSeriesPointNumberThreshold = IoTDBDescriptor.getInstance().getConfig()
+        .getAvgSeriesPointNumberThreshold();
+    try (Connection connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(10000);
+
+      // first file
+      // two pages for one chunk
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(1, 1)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(2, 2)");
+      statement.execute("FLUSH");
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(2);
+      // second file
+      // one chunk with two pages
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(3, 3)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(4, 4)");
+      // another chunk with two pages
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(5, 5)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(6, 6)");
+      statement.execute("FLUSH");
+
+      assertTrue(waitForMergeFinish());
+
+      int cnt;
+      try (ResultSet resultSet = statement.executeQuery("SELECT s1 FROM root.sg1.d1")) {
+        cnt = 0;
+        while (resultSet.next()) {
+          long time = resultSet.getLong("Time");
+          float s1 = resultSet.getFloat("root.sg1.d1.s1");
+          assertEquals(Long.parseLong(retArray[cnt][0]), time);
+          assertEquals(Float.parseFloat(retArray[cnt][1]), s1, FLOAT_DELTA);
+          cnt++;
+        }
+      }
+      assertEquals(retArray.length, cnt);
+    } catch (StorageEngineException | InterruptedException e) {
+      e.printStackTrace();
+      fail();
+    } finally {
+      IoTDBDescriptor.getInstance().getConfig()
+          .setAvgSeriesPointNumberThreshold(preAvgSeriesPointNumberThreshold);
+    }
+  }
+
+  /**
+   * first file has two pages for each chunk and two chunks for each time series
+   * second file has two pages for each chunk and two chunks for each time series
+   */
+  @Test
+  public void test16() throws SQLException {
+    String[][] retArray = {
+        {"1", "1"},
+        {"2", "2"},
+        {"3", "3"},
+        {"4", "4"},
+        {"5", "5"},
+        {"6", "6"},
+        {"7", "7"},
+        {"8", "8"}
+    };
+    int preAvgSeriesPointNumberThreshold = IoTDBDescriptor.getInstance().getConfig()
+        .getAvgSeriesPointNumberThreshold();
+    try (Connection connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+
+      IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(2);
+      // first file
+      // one chunk with two pages
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(1, 1)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(2, 2)");
+      // another chunk with two pages
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(3, 3)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(4, 4)");
+      statement.execute("FLUSH");
+
+      // second file
+      // one chunk with two pages
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(5, 5)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(6, 6)");
+      // another chunk with two pages
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(7, 7)");
+      statement.execute("INSERT INTO root.sg1.d1(time,s1) values(8, 8)");
+      statement.execute("FLUSH");
+
+      assertTrue(waitForMergeFinish());
+
+      int cnt;
+      try (ResultSet resultSet = statement.executeQuery("SELECT s1 FROM root.sg1.d1")) {
+        cnt = 0;
+        while (resultSet.next()) {
+          long time = resultSet.getLong("Time");
+          float s1 = resultSet.getFloat("root.sg1.d1.s1");
+          assertEquals(Long.parseLong(retArray[cnt][0]), time);
+          assertEquals(Float.parseFloat(retArray[cnt][1]), s1, FLOAT_DELTA);
+          cnt++;
+        }
+      }
+      assertEquals(retArray.length, cnt);
+    } catch (StorageEngineException | InterruptedException e) {
+      e.printStackTrace();
+      fail();
+    } finally {
+      IoTDBDescriptor.getInstance().getConfig()
+          .setAvgSeriesPointNumberThreshold(preAvgSeriesPointNumberThreshold);
+    }
+  }
+
+
+  /**
+   * wait until merge is finished
+   */
+  private boolean waitForMergeFinish()
+      throws StorageEngineException, InterruptedException {
+    StorageGroupProcessor storageGroupProcessor = StorageEngine.getInstance()
+        .getProcessor(storageGroupPath);
+    LevelCompactionTsFileManagement tsFileManagement = (LevelCompactionTsFileManagement) storageGroupProcessor
+        .getTsFileManagement();
+
+    long startTime = System.nanoTime();
+    // get the size of level 1's tsfile list to judge whether merge is finished
+    while (tsFileManagement.getSequenceTsFileResources().get(0L).size() < 2
+        || tsFileManagement.getSequenceTsFileResources().get(0L).get(1).size() != 1) {
+      TimeUnit.MILLISECONDS.sleep(100);
+      // wait too long, just break
+      if ((System.nanoTime() - startTime) >= MAX_WAIT_TIME_FOR_MERGE) {
+        break;
+      }
+    }
+    return tsFileManagement.getSequenceTsFileResources().get(0L).get(1).size() == 1;
+  }
+}
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertTabletMultiPlanTest.java b/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertTabletMultiPlanTest.java
index c8acb43..f14eff6 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertTabletMultiPlanTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertTabletMultiPlanTest.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.qp.plan;
+package org.apache.iotdb.db.qp.physical;
 
 import java.io.IOException;
 import java.util.ArrayList;
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertTabletPlanTest.java b/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertTabletPlanTest.java
index 53905bf..6e8565a 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertTabletPlanTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertTabletPlanTest.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.qp.plan;
+package org.apache.iotdb.db.qp.physical;
 
 import java.io.IOException;
 import java.util.ArrayList;
diff --git a/server/src/test/java/org/apache/iotdb/db/writelog/recover/SeqTsFileRecoverTest.java b/server/src/test/java/org/apache/iotdb/db/writelog/recover/SeqTsFileRecoverTest.java
index be05d71..aee638f 100644
--- a/server/src/test/java/org/apache/iotdb/db/writelog/recover/SeqTsFileRecoverTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/writelog/recover/SeqTsFileRecoverTest.java
@@ -76,7 +76,6 @@ public class SeqTsFileRecoverTest {
   private WriteLogNode node;
 
   private String logNodePrefix = TestConstant.BASE_OUTPUT_PATH.concat("testRecover");
-  private String storageGroup = "target";
   private TsFileResource resource;
   private VersionController versionController = new VersionController() {
     private int i;
@@ -140,6 +139,7 @@ public class SeqTsFileRecoverTest {
       }
     }
     writer.flushAllChunkGroups();
+    writer.getIOWriter().writePlanIndices();
     writer.getIOWriter().close();
 
     node = MultiFileLogNodeManager.getInstance().getNode(logNodePrefix + tsF.getName(), () -> {
diff --git a/session/src/test/java/org/apache/iotdb/session/IoTDBSessionIteratorIT.java b/session/src/test/java/org/apache/iotdb/session/IoTDBSessionIteratorIT.java
index b237437..ae41176 100644
--- a/session/src/test/java/org/apache/iotdb/session/IoTDBSessionIteratorIT.java
+++ b/session/src/test/java/org/apache/iotdb/session/IoTDBSessionIteratorIT.java
@@ -295,7 +295,7 @@ public class IoTDBSessionIteratorIT {
     List<TSDataType> types = new ArrayList<>();
     for (int i = 1; i <= 6; i++) {
       measurements.add("s" + i);
-      types.add(TSDataType.deserialize((short) (i - 1)));
+      types.add(TSDataType.deserialize((byte) (i - 1)));
     }
 
     for (long time = 0; time < 10; time++) {
diff --git a/spark-tsfile/src/test/scala/org/apache/iotdb/spark/tsfile/HDFSInputTest.java b/spark-tsfile/src/test/scala/org/apache/iotdb/spark/tsfile/HDFSInputTest.java
index f4168b3..67538a4 100644
--- a/spark-tsfile/src/test/scala/org/apache/iotdb/spark/tsfile/HDFSInputTest.java
+++ b/spark-tsfile/src/test/scala/org/apache/iotdb/spark/tsfile/HDFSInputTest.java
@@ -66,14 +66,14 @@ public class HDFSInputTest {
 
   @Test
   public void test_read1() throws IOException {
-    int size = 1000;
+    int size = 500;
     ByteBuffer buffer = ByteBuffer.allocate(size);
     Assert.assertEquals(size, in.read(buffer));
   }
 
   @Test
   public void test_read2() throws IOException {
-    int size = 1000;
+    int size = 500;
     long pos = 20L;
     ByteBuffer buffer = ByteBuffer.allocate(size);
     Assert.assertEquals(size, in.read(buffer, pos));
diff --git a/tsfile/format-changelist.md b/tsfile/format-changelist.md
index 2cdfc33..9161d11 100644
--- a/tsfile/format-changelist.md
+++ b/tsfile/format-changelist.md
@@ -19,10 +19,12 @@
 
 -->
 
-# 0.10.x (version-2) -> 0.12.x (version-3)
+# 0.10.x/0.11.x (version-2) -> 0.12.x (version-3)
 | PR#   | Name                                                        | Author          | Changes                                                      |
 | ----- | ----------------------------------------------------------- | --------------- | ------------------------------------------------------------ |
 | 2263  | Move memtable plan index from ChunkGroupFooter to a separate marker      | jt2594838       | Add min/max plan index after MetaMarker.OPERATION_INDEX_RANGE when a memtable is flush|
+| 2184  | [IOTDB-1081] New TsFile Format      | JackieTien97      | Please see [details](https://cwiki.apache.org/confluence/display/IOTDB/New+TsFile+Format)|
+| 2445  | Remove versionInfo in Tsfile and get rid of versions in memtable      | wshao08       | Delete version info in TsFile |
 
 
 # 0.9.x (version-1) -> 0.10.x (version-2)
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/common/conf/TSFileConfig.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/common/conf/TSFileConfig.java
index 24e4dde..2193786 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/common/conf/TSFileConfig.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/common/conf/TSFileConfig.java
@@ -61,8 +61,12 @@ public class TSFileConfig implements Serializable {
   public static final Charset STRING_CHARSET = Charset.forName(STRING_ENCODING);
   public static final String CONFIG_FILE_NAME = "iotdb-engine.properties";
   public static final String MAGIC_STRING = "TsFile";
-  public static final String VERSION_NUMBER = "000002";
+  public static final String VERSION_NUMBER_V2 = "000002";
   public static final String VERSION_NUMBER_V1 = "000001";
+  /**
+   * version number is changed to use 1 byte to represent since version 3
+   */
+  public static final byte VERSION_NUMBER = 0x03;
 
   /**
    * Bloom filter constrain
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/Decoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/Decoder.java
index 6bffedd..0e43cf6 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/Decoder.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/Decoder.java
@@ -22,8 +22,6 @@ package org.apache.iotdb.tsfile.encoding.decoder;
 import java.io.IOException;
 import java.math.BigDecimal;
 import java.nio.ByteBuffer;
-
-import org.apache.iotdb.tsfile.encoding.common.EndianType;
 import org.apache.iotdb.tsfile.exception.encoding.TsFileDecodingException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
@@ -50,14 +48,14 @@ public abstract class Decoder {
   public static Decoder getDecoderByType(TSEncoding encoding, TSDataType dataType) {
     switch (encoding) {
       case PLAIN:
-        return new PlainDecoder(EndianType.BIG_ENDIAN);
+        return new PlainDecoder();
       case RLE:
         switch (dataType) {
           case BOOLEAN:
           case INT32:
-            return new IntRleDecoder(EndianType.BIG_ENDIAN);
+            return new IntRleDecoder();
           case INT64:
-            return new LongRleDecoder(EndianType.BIG_ENDIAN);
+            return new LongRleDecoder();
           case FLOAT:
           case DOUBLE:
             return new FloatDecoder(TSEncoding.valueOf(encoding.toString()), dataType);
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/FloatDecoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/FloatDecoder.java
index eaaf729..ce0bc2a 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/FloatDecoder.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/FloatDecoder.java
@@ -21,17 +21,14 @@ package org.apache.iotdb.tsfile.encoding.decoder;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.iotdb.tsfile.encoding.common.EndianType;
 import org.apache.iotdb.tsfile.encoding.encoder.FloatEncoder;
 import org.apache.iotdb.tsfile.exception.encoding.TsFileDecodingException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Decoder for float or double value using rle or two diff. For more info about
@@ -58,10 +55,10 @@ public class FloatDecoder extends Decoder {
     super(encodingType);
     if (encodingType == TSEncoding.RLE) {
       if (dataType == TSDataType.FLOAT) {
-        decoder = new IntRleDecoder(EndianType.BIG_ENDIAN);
+        decoder = new IntRleDecoder();
         logger.debug("tsfile-encoding FloatDecoder: init decoder using int-rle and float");
       } else if (dataType == TSDataType.DOUBLE) {
-        decoder = new LongRleDecoder(EndianType.BIG_ENDIAN);
+        decoder = new LongRleDecoder();
         logger.debug("tsfile-encoding FloatDecoder: init decoder using long-rle and double");
       } else {
         throw new TsFileDecodingException(String.format("data type %s is not supported by FloatDecoder", dataType));
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/IntRleDecoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/IntRleDecoder.java
index a41512f..0166446 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/IntRleDecoder.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/IntRleDecoder.java
@@ -21,15 +21,12 @@ package org.apache.iotdb.tsfile.encoding.decoder;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.encoding.bitpacking.IntPacker;
-import org.apache.iotdb.tsfile.encoding.common.EndianType;
 import org.apache.iotdb.tsfile.exception.encoding.TsFileDecodingException;
 import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Decoder for int value using rle or bit-packing.
@@ -53,8 +50,8 @@ public class IntRleDecoder extends RleDecoder {
    */
   private IntPacker packer;
 
-  public IntRleDecoder(EndianType endianType) {
-    super(endianType);
+  public IntRleDecoder() {
+    super();
     currentValue = 0;
   }
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/LongRleDecoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/LongRleDecoder.java
index 3f8da99..de27c0c 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/LongRleDecoder.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/LongRleDecoder.java
@@ -21,15 +21,12 @@ package org.apache.iotdb.tsfile.encoding.decoder;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.encoding.bitpacking.LongPacker;
-import org.apache.iotdb.tsfile.encoding.common.EndianType;
 import org.apache.iotdb.tsfile.exception.encoding.TsFileDecodingException;
 import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Decoder for long value using rle or bit-packing.
@@ -53,8 +50,8 @@ public class LongRleDecoder extends RleDecoder {
    */
   private LongPacker packer;
 
-  public LongRleDecoder(EndianType endianType) {
-    super(endianType);
+  public LongRleDecoder() {
+    super();
     currentValue = 0;
   }
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/PlainDecoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/PlainDecoder.java
index 1fd15c1..8c31cff 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/PlainDecoder.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/PlainDecoder.java
@@ -22,31 +22,16 @@ package org.apache.iotdb.tsfile.encoding.decoder;
 import java.io.IOException;
 import java.math.BigDecimal;
 import java.nio.ByteBuffer;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.iotdb.tsfile.encoding.common.EndianType;
 import org.apache.iotdb.tsfile.exception.encoding.TsFileDecodingException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
 
 public class PlainDecoder extends Decoder {
 
-  private static final Logger logger = LoggerFactory.getLogger(PlainDecoder.class);
-  private EndianType endianType;
-
-  public EndianType getEndianType() {
-    return endianType;
-  }
-
-  public void setEndianType(EndianType endianType) {
-    this.endianType = endianType;
-  }
 
-  public PlainDecoder(EndianType endianType) {
+  public PlainDecoder() {
     super(TSEncoding.PLAIN);
-    this.endianType = endianType;
   }
 
   @Override
@@ -61,7 +46,7 @@ public class PlainDecoder extends Decoder {
 
   @Override
   public int readInt(ByteBuffer buffer) {
-    return buffer.getInt();
+    return ReadWriteForEncodingUtils.readVarInt(buffer);
   }
 
   @Override
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/RleDecoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/RleDecoder.java
index 8606537..935902d 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/RleDecoder.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/RleDecoder.java
@@ -22,10 +22,8 @@ package org.apache.iotdb.tsfile.encoding.decoder;
 import java.io.IOException;
 import java.math.BigDecimal;
 import java.nio.ByteBuffer;
-
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
-import org.apache.iotdb.tsfile.encoding.common.EndianType;
 import org.apache.iotdb.tsfile.exception.encoding.TsFileDecodingException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.utils.Binary;
@@ -40,16 +38,6 @@ import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
  */
 public abstract class RleDecoder extends Decoder {
 
-  private EndianType endianType;
-
-  public EndianType getEndianType() {
-    return endianType;
-  }
-
-  public void setEndianType(EndianType endianType) {
-    this.endianType = endianType;
-  }
-
   protected TSFileConfig config = TSFileDescriptor.getInstance().getConfig();
   /**
    * mode to indicate current encoding type 0 - RLE 1 - BIT_PACKED.
@@ -87,9 +75,8 @@ public abstract class RleDecoder extends Decoder {
    * a constructor, init with endianType, default encoding is
    * <code>TSEncoding.RLE</code>.
    */
-  public RleDecoder(EndianType endianType) {
+  protected RleDecoder() {
     super(TSEncoding.RLE);
-    this.endianType = endianType;
     reset();
   }
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/FloatEncoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/FloatEncoder.java
index 873a4ba..1b4b558 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/FloatEncoder.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/FloatEncoder.java
@@ -21,8 +21,6 @@ package org.apache.iotdb.tsfile.encoding.encoder;
 
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-
-import org.apache.iotdb.tsfile.encoding.common.EndianType;
 import org.apache.iotdb.tsfile.exception.encoding.TsFileEncodingException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
@@ -66,9 +64,9 @@ public class FloatEncoder extends Encoder {
     isMaxPointNumberSaved = false;
     if (encodingType == TSEncoding.RLE) {
       if (dataType == TSDataType.FLOAT) {
-        encoder = new IntRleEncoder(EndianType.BIG_ENDIAN);
+        encoder = new IntRleEncoder();
       } else if (dataType == TSDataType.DOUBLE) {
-        encoder = new LongRleEncoder(EndianType.BIG_ENDIAN);
+        encoder = new LongRleEncoder();
       } else {
         throw new TsFileEncodingException(String.format("data type %s is not supported by FloatEncoder", dataType));
       }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/IntRleEncoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/IntRleEncoder.java
index f3bb89a..9ed819e 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/IntRleEncoder.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/IntRleEncoder.java
@@ -22,10 +22,8 @@ package org.apache.iotdb.tsfile.encoding.encoder;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.util.ArrayList;
-
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.encoding.bitpacking.IntPacker;
-import org.apache.iotdb.tsfile.encoding.common.EndianType;
 import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
 
 /**
@@ -38,11 +36,11 @@ public class IntRleEncoder extends RleEncoder<Integer> {
    */
   private IntPacker packer;
 
-  public IntRleEncoder(EndianType endianType) {
-    super(endianType);
+  public IntRleEncoder() {
+    super();
     bufferedValues = new Integer[TSFileConfig.RLE_MIN_REPEATED_NUM];
     preValue = 0;
-    values = new ArrayList<Integer>();
+    values = new ArrayList<>();
   }
 
   @Override
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/LongRleEncoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/LongRleEncoder.java
index dbbbfbb..f8a33f3 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/LongRleEncoder.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/LongRleEncoder.java
@@ -22,10 +22,8 @@ package org.apache.iotdb.tsfile.encoding.encoder;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.util.ArrayList;
-
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.encoding.bitpacking.LongPacker;
-import org.apache.iotdb.tsfile.encoding.common.EndianType;
 import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
 
 /**
@@ -41,8 +39,8 @@ public class LongRleEncoder extends RleEncoder<Long> {
   /**
    * Constructor of LongRleEncoder.
    */
-  public LongRleEncoder(EndianType endianType) {
-    super(endianType);
+  public LongRleEncoder() {
+    super();
     bufferedValues = new Long[TSFileConfig.RLE_MIN_REPEATED_NUM];
     preValue = (long) 0;
     values = new ArrayList<Long>();
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/PlainEncoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/PlainEncoder.java
index d98d9c2..5fe4ccd 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/PlainEncoder.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/PlainEncoder.java
@@ -22,39 +22,27 @@ package org.apache.iotdb.tsfile.encoding.encoder;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.math.BigDecimal;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
-import org.apache.iotdb.tsfile.encoding.common.EndianType;
 import org.apache.iotdb.tsfile.exception.encoding.TsFileEncodingException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class PlainEncoder extends Encoder {
 
   private static final Logger logger = LoggerFactory.getLogger(PlainEncoder.class);
-  private EndianType endianType;
   private TSDataType dataType;
   private int maxStringLength;
 
-  public PlainEncoder(EndianType endianType, TSDataType dataType, int maxStringLength) {
+  public PlainEncoder(TSDataType dataType, int maxStringLength) {
     super(TSEncoding.PLAIN);
-    this.endianType = endianType;
     this.dataType = dataType;
     this.maxStringLength = maxStringLength;
   }
 
-  public void setEndianType(EndianType endianType) {
-    this.endianType = endianType;
-  }
-
-  public EndianType getEndianType() {
-    return endianType;
-  }
-
   @Override
   public void encode(boolean value, ByteArrayOutputStream out) {
     if (value) {
@@ -66,46 +54,29 @@ public class PlainEncoder extends Encoder {
 
   @Override
   public void encode(short value, ByteArrayOutputStream out) {
-    if (this.endianType == EndianType.LITTLE_ENDIAN) {
-      out.write(value & 0xFF);
-      out.write((value >> 8) & 0xFF);
-    } else if (this.endianType == EndianType.BIG_ENDIAN) {
-      out.write((value >> 8) & 0xFF);
-      out.write(value & 0xFF);
-    }
+    out.write((value >> 8) & 0xFF);
+    out.write(value & 0xFF);
   }
 
   @Override
   public void encode(int value, ByteArrayOutputStream out) {
-    if (this.endianType == EndianType.LITTLE_ENDIAN) {
-      out.write(value & 0xFF);
-      out.write((value >> 8) & 0xFF);
-      out.write((value >> 16) & 0xFF);
-      out.write((value >> 24) & 0xFF);
-    } else if (this.endianType == EndianType.BIG_ENDIAN) {
-      out.write((value >> 24) & 0xFF);
-      out.write((value >> 16) & 0xFF);
-      out.write((value >> 8) & 0xFF);
-      out.write(value & 0xFF);
-    }
+    ReadWriteForEncodingUtils.writeVarInt(value, out);
   }
 
   @Override
   public void encode(long value, ByteArrayOutputStream out) {
-    if (this.endianType == EndianType.LITTLE_ENDIAN) {
-      for (int i = 0; i < 8; i++) {
-        out.write((byte) (((value) >> (i * 8)) & 0xFF));
-      }
-    } else if (this.endianType == EndianType.BIG_ENDIAN) {
-      for (int i = 7; i >= 0; i--) {
-        out.write((byte) (((value) >> (i * 8)) & 0xFF));
-      }
+    for (int i = 7; i >= 0; i--) {
+      out.write((byte) (((value) >> (i * 8)) & 0xFF));
     }
   }
 
   @Override
   public void encode(float value, ByteArrayOutputStream out) {
-    encode(Float.floatToIntBits(value), out);
+    int floatInt = Float.floatToIntBits(value);
+    out.write((floatInt >> 24) & 0xFF);
+    out.write((floatInt >> 16) & 0xFF);
+    out.write((floatInt >> 8) & 0xFF);
+    out.write(floatInt & 0xFF);
   }
 
   @Override
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/RleEncoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/RleEncoder.java
index 51003ce..8fd62f1 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/RleEncoder.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/RleEncoder.java
@@ -24,17 +24,14 @@ import java.io.IOException;
 import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.List;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
-import org.apache.iotdb.tsfile.encoding.common.EndianType;
 import org.apache.iotdb.tsfile.exception.encoding.TsFileEncodingException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Encodes values using a combination of run length encoding and bit packing,
@@ -66,16 +63,6 @@ public abstract class RleEncoder<T extends Comparable<T>> extends Encoder {
 
   private static final Logger logger = LoggerFactory.getLogger(RleEncoder.class);
 
-  private EndianType endianType;
-
-  public EndianType getEndianType() {
-    return endianType;
-  }
-
-  public void setEndianType(EndianType endianType) {
-    this.endianType = endianType;
-  }
-
   /**
    * we save all value in a list and calculate its bitwidth.
    */
@@ -134,9 +121,8 @@ public abstract class RleEncoder<T extends Comparable<T>> extends Encoder {
   /**
    * constructor.
    */
-  public RleEncoder(EndianType endianType) {
+  protected RleEncoder() {
     super(TSEncoding.RLE);
-    this.endianType = endianType;
     bytesBuffer = new ArrayList<>();
     isBitPackRun = false;
     isBitWidthSaved = false;
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/TSEncodingBuilder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/TSEncodingBuilder.java
index 0903cd1..14aa496 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/TSEncodingBuilder.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/TSEncodingBuilder.java
@@ -20,17 +20,14 @@
 package org.apache.iotdb.tsfile.encoding.encoder;
 
 import java.util.Map;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.common.constant.JsonFormatConstant;
-import org.apache.iotdb.tsfile.encoding.common.EndianType;
 import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Each subclass of TSEncodingBuilder responds a enumerate value in {@linkplain TSEncoding
@@ -105,7 +102,7 @@ public abstract class TSEncodingBuilder {
 
     @Override
     public Encoder getEncoder(TSDataType type) {
-      return new PlainEncoder(EndianType.BIG_ENDIAN, type, maxStringLength);
+      return new PlainEncoder(type, maxStringLength);
     }
 
     @Override
@@ -137,9 +134,9 @@ public abstract class TSEncodingBuilder {
       switch (type) {
         case INT32:
         case BOOLEAN:
-          return new IntRleEncoder(EndianType.BIG_ENDIAN);
+          return new IntRleEncoder();
         case INT64:
-          return new LongRleEncoder(EndianType.BIG_ENDIAN);
+          return new LongRleEncoder();
         case FLOAT:
         case DOUBLE:
           return new FloatEncoder(TSEncoding.RLE, type, maxPointNumber);
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/MetaMarker.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/MetaMarker.java
index b396a9e..74ec4a3 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/MetaMarker.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/MetaMarker.java
@@ -24,17 +24,32 @@ import java.io.IOException;
 /**
  * MetaMarker denotes the type of headers and footers. Enum is not used for space saving.
  */
+@SuppressWarnings({ "squid:S1133" }) //Deprecated code should be removed
 public class MetaMarker {
 
-  public static final byte CHUNK_GROUP_FOOTER = 0;
+  public static final byte CHUNK_GROUP_HEADER = 0;
+  /**
+   * Chunk header marker and this chunk has more than one page.
+   */
   public static final byte CHUNK_HEADER = 1;
   public static final byte SEPARATOR = 2;
+  /**
+   * @deprecated (Since TsFile version 3, the marker VERSION is no longer used in TsFile.
+   * It should be removed when TsFile upgrade to version 4)
+   */
+  @Deprecated
   public static final byte VERSION = 3;
+  
   // following this marker are two longs marking the minimum and maximum indices of operations
   // involved in the last flushed MemTable, which are generally used to support checkpoint,
   // snapshot, or backup.
   public static final byte OPERATION_INDEX_RANGE = 4;
 
+  /**
+   * Chunk header marker and this chunk has only one page.
+   */
+  public static final byte ONLY_ONE_PAGE_CHUNK_HEADER = 5;
+
   private MetaMarker() {
   }
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/footer/ChunkGroupFooter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/footer/ChunkGroupFooter.java
deleted file mode 100644
index cf7181c..0000000
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/footer/ChunkGroupFooter.java
+++ /dev/null
@@ -1,156 +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.tsfile.file.footer;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
-import org.apache.iotdb.tsfile.file.MetaMarker;
-import org.apache.iotdb.tsfile.read.reader.TsFileInput;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-
-public class ChunkGroupFooter {
-
-  private static final byte MARKER = MetaMarker.CHUNK_GROUP_FOOTER;
-
-  private String deviceID;
-
-  private long dataSize;
-
-  private int numberOfChunks;
-
-  // this field does not need to be serialized.
-  private int serializedSize;
-
-  /**
-   * constructor of CHUNK_GROUP_FOOTER.
-   *
-   * @param deviceID       device ID
-   * @param dataSize       data size
-   * @param numberOfChunks number of chunks
-   */
-  public ChunkGroupFooter(String deviceID, long dataSize, int numberOfChunks) {
-    this.deviceID = deviceID;
-    this.dataSize = dataSize;
-    this.numberOfChunks = numberOfChunks;
-    this.serializedSize =
-        Byte.BYTES + Integer.BYTES + deviceID.getBytes(TSFileConfig.STRING_CHARSET).length
-            + Long.BYTES + Integer.BYTES;
-  }
-
-  public static int getSerializedSize(String deviceID) {
-    return Byte.BYTES + Integer.BYTES + getSerializedSize(deviceID.length()) + Long.BYTES
-        + Long.BYTES;
-  }
-
-  private static int getSerializedSize(int deviceIdLength) {
-    return deviceIdLength + Long.BYTES + Integer.BYTES + Long.BYTES + Long.BYTES;
-  }
-
-  /**
-   * deserialize from inputStream.
-   *
-   * @param markerRead Whether the marker of the CHUNK_GROUP_FOOTER is read ahead.
-   */
-  public static ChunkGroupFooter deserializeFrom(InputStream inputStream, boolean markerRead)
-      throws IOException {
-    if (!markerRead) {
-      byte marker = (byte) inputStream.read();
-      if (marker != MARKER) {
-        MetaMarker.handleUnexpectedMarker(marker);
-      }
-    }
-
-    String deviceID = ReadWriteIOUtils.readString(inputStream);
-    long dataSize = ReadWriteIOUtils.readLong(inputStream);
-    int numOfChunks = ReadWriteIOUtils.readInt(inputStream);
-    return new ChunkGroupFooter(deviceID, dataSize, numOfChunks);
-  }
-
-  /**
-   * deserialize from TsFileInput.
-   *
-   * @param markerRead Whether the marker of the CHUNK_GROUP_FOOTER is read ahead.
-   */
-  public static ChunkGroupFooter deserializeFrom(TsFileInput input, long offset, boolean markerRead)
-      throws IOException {
-    long offsetVar = offset;
-    if (!markerRead) {
-      offsetVar++;
-    }
-    ByteBuffer buffer = ByteBuffer.allocate(Integer.BYTES);
-    input.read(buffer, offsetVar);
-    buffer.flip();
-    int size = buffer.getInt();
-    offsetVar += Integer.BYTES;
-    buffer = ByteBuffer.allocate(getSerializedSize(size));
-    ReadWriteIOUtils.readAsPossible(input, offsetVar, buffer);
-    buffer.flip();
-    String deviceID = ReadWriteIOUtils.readStringWithLength(buffer, size);
-    long dataSize = ReadWriteIOUtils.readLong(buffer);
-    int numOfChunks = ReadWriteIOUtils.readInt(buffer);
-    return new ChunkGroupFooter(deviceID, dataSize, numOfChunks);
-  }
-
-  public int getSerializedSize() {
-    return serializedSize;
-  }
-
-  public String getDeviceID() {
-    return deviceID;
-  }
-
-  public long getDataSize() {
-    return dataSize;
-  }
-
-  public void setDataSize(long dataSize) {
-    this.dataSize = dataSize;
-  }
-
-  public int getNumberOfChunks() {
-    return numberOfChunks;
-  }
-
-  /**
-   * serialize to outputStream.
-   *
-   * @param outputStream output stream
-   * @return length
-   * @throws IOException IOException
-   */
-  public int serializeTo(OutputStream outputStream) throws IOException {
-    int length = 0;
-    length += ReadWriteIOUtils.write(MARKER, outputStream);
-    length += ReadWriteIOUtils.write(deviceID, outputStream);
-    length += ReadWriteIOUtils.write(dataSize, outputStream);
-    length += ReadWriteIOUtils.write(numberOfChunks, outputStream);
-    return length;
-  }
-
-  @Override
-  public String toString() {
-    return "CHUNK_GROUP_FOOTER{" + "deviceID='" + deviceID + '\'' + ", dataSize=" + dataSize
-        + ", numberOfChunks="
-        + numberOfChunks + ", serializedSize=" + serializedSize;
-  }
-}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/ChunkGroupHeader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/ChunkGroupHeader.java
new file mode 100644
index 0000000..e8b86d0
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/ChunkGroupHeader.java
@@ -0,0 +1,117 @@
+/*
+ * 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.tsfile.file.header;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
+import org.apache.iotdb.tsfile.file.MetaMarker;
+import org.apache.iotdb.tsfile.read.reader.TsFileInput;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+public class ChunkGroupHeader {
+
+  private static final byte MARKER = MetaMarker.CHUNK_GROUP_HEADER;
+
+  private final String deviceID;
+
+  // this field does not need to be serialized.
+  private int serializedSize;
+
+  /**
+   * constructor of CHUNK_GROUP_HEADER.
+   *  @param deviceID       device ID
+   *
+   */
+  public ChunkGroupHeader(String deviceID) {
+    this.deviceID = deviceID;
+    this.serializedSize = getSerializedSize(deviceID);
+  }
+
+  private int getSerializedSize(String deviceID) {
+    int length = deviceID.getBytes(TSFileConfig.STRING_CHARSET).length;
+    return Byte.BYTES + ReadWriteForEncodingUtils.varIntSize(length) + length;
+  }
+
+  /**
+   * deserialize from inputStream.
+   *
+   * @param markerRead Whether the marker of the CHUNK_GROUP_HEADER is read ahead.
+   */
+  public static ChunkGroupHeader deserializeFrom(InputStream inputStream, boolean markerRead) throws IOException {
+    if (!markerRead) {
+      byte marker = (byte) inputStream.read();
+      if (marker != MARKER) {
+        MetaMarker.handleUnexpectedMarker(marker);
+      }
+    }
+
+    String deviceID = ReadWriteIOUtils.readVarIntString(inputStream);
+    return new ChunkGroupHeader(deviceID);
+  }
+
+  /**
+   * deserialize from TsFileInput.
+   *
+   * @param markerRead Whether the marker of the CHUNK_GROUP_HEADER is read ahead.
+   */
+  public static ChunkGroupHeader deserializeFrom(TsFileInput input, long offset, boolean markerRead)
+      throws IOException {
+    long offsetVar = offset;
+    if (!markerRead) {
+      offsetVar++;
+    }
+    String deviceID = input.readVarIntString(offsetVar);
+    return new ChunkGroupHeader(deviceID);
+  }
+
+  public int getSerializedSize() {
+    return serializedSize;
+  }
+
+  public String getDeviceID() {
+    return deviceID;
+  }
+
+
+  /**
+   * serialize to outputStream.
+   *
+   * @param outputStream output stream
+   * @return length
+   * @throws IOException IOException
+   */
+  public int serializeTo(OutputStream outputStream) throws IOException {
+    int length = 0;
+    length += ReadWriteIOUtils.write(MARKER, outputStream);
+    length += ReadWriteIOUtils.writeVar(deviceID, outputStream);
+    return length;
+  }
+
+  @Override
+  public String toString() {
+    return "ChunkGroupHeader{" +
+        "deviceID='" + deviceID + '\'' +
+        ", serializedSize=" + serializedSize +
+        '}';
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/ChunkHeader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/ChunkHeader.java
index 96556b8..e9e7544 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/ChunkHeader.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/ChunkHeader.java
@@ -25,6 +25,7 @@ import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.read.reader.TsFileInput;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 import java.io.IOException;
 import java.io.InputStream;
@@ -33,101 +34,119 @@ import java.nio.ByteBuffer;
 
 public class ChunkHeader {
 
+
+  /**
+   * 1 means this chunk has more than one page, so each page has its own page statistic 5 means this
+   * chunk has only one page, and this page has no page statistic
+   */
+  private byte chunkType;
   private String measurementID;
   private int dataSize;
   private TSDataType dataType;
   private CompressionType compressionType;
   private TSEncoding encodingType;
-  private int numOfPages;
 
-  // this field does not need to be serialized.
+  // the following fields do not need to be serialized.
+  private int numOfPages;
   private int serializedSize;
 
   public ChunkHeader(String measurementID, int dataSize, TSDataType dataType,
-      CompressionType compressionType,
-      TSEncoding encoding, int numOfPages) {
-    this(measurementID, dataSize, getSerializedSize(measurementID), dataType, compressionType,
-        encoding, numOfPages);
+      CompressionType compressionType, TSEncoding encoding, int numOfPages) {
+    this(numOfPages <= 1 ? MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER : MetaMarker.CHUNK_HEADER,
+        measurementID, dataSize, getSerializedSize(measurementID, dataSize), dataType,
+        compressionType,
+        encoding);
+    this.numOfPages = numOfPages;
   }
 
-  private ChunkHeader(String measurementID, int dataSize, int headerSize, TSDataType dataType,
-      CompressionType compressionType, TSEncoding encoding, int numOfPages) {
+  public ChunkHeader(byte chunkType, String measurementID, int dataSize, TSDataType dataType,
+      CompressionType compressionType, TSEncoding encoding) {
+    this(chunkType, measurementID, dataSize, getSerializedSize(measurementID, dataSize), dataType,
+        compressionType, encoding);
+  }
+
+  public ChunkHeader(byte chunkType, String measurementID, int dataSize, int headerSize,
+      TSDataType dataType, CompressionType compressionType, TSEncoding encoding) {
+    this.chunkType = chunkType;
     this.measurementID = measurementID;
     this.dataSize = dataSize;
     this.dataType = dataType;
     this.compressionType = compressionType;
-    this.numOfPages = numOfPages;
     this.encodingType = encoding;
     this.serializedSize = headerSize;
   }
 
+  /**
+   * the exact serialized size of chunk header
+   */
+  public static int getSerializedSize(String measurementID, int dataSize) {
+    int measurementIdLength = measurementID.getBytes(TSFileConfig.STRING_CHARSET).length;
+    return Byte.BYTES // chunkType
+        + ReadWriteForEncodingUtils.varIntSize(measurementIdLength) // measurementID length
+        + measurementIdLength // measurementID
+        + ReadWriteForEncodingUtils.uVarIntSize(dataSize) // dataSize
+        + TSDataType.getSerializedSize() // dataType
+        + CompressionType.getSerializedSize() // compressionType
+        + TSEncoding.getSerializedSize(); // encodingType
+  }
+
+  /**
+   * The estimated serialized size of chunk header. Only used when we don't know the actual dataSize
+   * attribute
+   */
   public static int getSerializedSize(String measurementID) {
-    return Byte.BYTES // marker
-        + Integer.BYTES // measurementID length
-        + measurementID.getBytes(TSFileConfig.STRING_CHARSET).length // measurementID
-        + Integer.BYTES // dataSize
+
+    int measurementIdLength = measurementID.getBytes(TSFileConfig.STRING_CHARSET).length;
+    return  Byte.BYTES // chunkType
+        + ReadWriteForEncodingUtils.varIntSize(measurementIdLength) // measurementID length
+        + measurementIdLength // measurementID
+        + Integer.BYTES + 1 // uVarInt dataSize
         + TSDataType.getSerializedSize() // dataType
         + CompressionType.getSerializedSize() // compressionType
-        + TSEncoding.getSerializedSize() // encodingType
-        + Integer.BYTES; // numOfPages
+        + TSEncoding.getSerializedSize(); // encodingType
   }
 
   /**
-   * deserialize from inputStream.
-   *
-   * @param markerRead Whether the marker of the CHUNK_HEADER has been read
+   * deserialize from inputStream, the marker has already been read.
    */
-  public static ChunkHeader deserializeFrom(InputStream inputStream, boolean markerRead)
+  public static ChunkHeader deserializeFrom(InputStream inputStream, byte chunkType)
       throws IOException {
-    if (!markerRead) {
-      byte marker = (byte) inputStream.read();
-      if (marker != MetaMarker.CHUNK_HEADER) {
-        MetaMarker.handleUnexpectedMarker(marker);
-      }
-    }
-
-    String measurementID = ReadWriteIOUtils.readString(inputStream);
-    int dataSize = ReadWriteIOUtils.readInt(inputStream);
-    TSDataType dataType = TSDataType.deserialize(ReadWriteIOUtils.readShort(inputStream));
-    int numOfPages = ReadWriteIOUtils.readInt(inputStream);
+    // read measurementID
+    String measurementID = ReadWriteIOUtils.readVarIntString(inputStream);
+    int dataSize = ReadWriteForEncodingUtils.readUnsignedVarInt(inputStream);
+    TSDataType dataType = ReadWriteIOUtils.readDataType(inputStream);
     CompressionType type = ReadWriteIOUtils.readCompressionType(inputStream);
     TSEncoding encoding = ReadWriteIOUtils.readEncoding(inputStream);
-    return new ChunkHeader(measurementID, dataSize, dataType, type, encoding, numOfPages);
+    return new ChunkHeader(chunkType, measurementID, dataSize, dataType, type, encoding);
   }
 
   /**
-   * deserialize from TsFileInput.
+   * deserialize from TsFileInput, the marker has not been read.
    *
-   * @param input TsFileInput
-   * @param offset offset
-   * @param chunkHeaderSize the size of chunk's header
-   * @param markerRead read marker (boolean type)
+   * @param input           TsFileInput
+   * @param offset          offset
+   * @param chunkHeaderSize the estimated size of chunk's header
    * @return CHUNK_HEADER object
    * @throws IOException IOException
    */
-  public static ChunkHeader deserializeFrom(TsFileInput input, long offset, int chunkHeaderSize,
-      boolean markerRead)
-      throws IOException {
-    long offsetVar = offset;
-    if (!markerRead) {
-      offsetVar++;
-    }
+  public static ChunkHeader deserializeFrom(TsFileInput input, long offset, int chunkHeaderSize) throws IOException {
 
     // read chunk header from input to buffer
     ByteBuffer buffer = ByteBuffer.allocate(chunkHeaderSize);
-    input.read(buffer, offsetVar);
+    input.read(buffer, offset);
     buffer.flip();
 
+    byte chunkType = buffer.get();
     // read measurementID
-    int size = buffer.getInt();
-    String measurementID = ReadWriteIOUtils.readStringWithLength(buffer, size);
-    int dataSize = ReadWriteIOUtils.readInt(buffer);
-    TSDataType dataType = TSDataType.deserialize(ReadWriteIOUtils.readShort(buffer));
-    int numOfPages = ReadWriteIOUtils.readInt(buffer);
+    String measurementID = ReadWriteIOUtils.readVarIntString(buffer);
+    int dataSize = ReadWriteForEncodingUtils.readUnsignedVarInt(buffer);
+    TSDataType dataType = ReadWriteIOUtils.readDataType(buffer);
     CompressionType type = ReadWriteIOUtils.readCompressionType(buffer);
     TSEncoding encoding = ReadWriteIOUtils.readEncoding(buffer);
-    return new ChunkHeader(measurementID, dataSize, chunkHeaderSize, dataType, type, encoding,
-        numOfPages);
+    chunkHeaderSize =
+        chunkHeaderSize - Integer.BYTES - 1 + ReadWriteForEncodingUtils.uVarIntSize(dataSize);
+    return new ChunkHeader(chunkType, measurementID, dataSize, chunkHeaderSize, dataType, type,
+        encoding);
   }
 
   public int getSerializedSize() {
@@ -155,11 +174,10 @@ public class ChunkHeader {
    */
   public int serializeTo(OutputStream outputStream) throws IOException {
     int length = 0;
-    length += ReadWriteIOUtils.write(MetaMarker.CHUNK_HEADER, outputStream);
-    length += ReadWriteIOUtils.write(measurementID, outputStream);
-    length += ReadWriteIOUtils.write(dataSize, outputStream);
+    length += ReadWriteIOUtils.write(chunkType, outputStream);
+    length += ReadWriteIOUtils.writeVar(measurementID, outputStream);
+    length += ReadWriteForEncodingUtils.writeUnsignedVarInt(dataSize, outputStream);
     length += ReadWriteIOUtils.write(dataType, outputStream);
-    length += ReadWriteIOUtils.write(numOfPages, outputStream);
     length += ReadWriteIOUtils.write(compressionType, outputStream);
     length += ReadWriteIOUtils.write(encodingType, outputStream);
     return length;
@@ -173,11 +191,10 @@ public class ChunkHeader {
    */
   public int serializeTo(ByteBuffer buffer) {
     int length = 0;
-    length += ReadWriteIOUtils.write(MetaMarker.CHUNK_HEADER, buffer);
-    length += ReadWriteIOUtils.write(measurementID, buffer);
-    length += ReadWriteIOUtils.write(dataSize, buffer);
+    length += ReadWriteIOUtils.write(chunkType, buffer);
+    length += ReadWriteIOUtils.writeVar(measurementID, buffer);
+    length += ReadWriteForEncodingUtils.writeUnsignedVarInt(dataSize, buffer);
     length += ReadWriteIOUtils.write(dataType, buffer);
-    length += ReadWriteIOUtils.write(numOfPages, buffer);
     length += ReadWriteIOUtils.write(compressionType, buffer);
     length += ReadWriteIOUtils.write(encodingType, buffer);
     return length;
@@ -208,4 +225,20 @@ public class ChunkHeader {
     this.dataSize += chunkHeader.getDataSize();
     this.numOfPages += chunkHeader.getNumOfPages();
   }
+
+  public void setDataSize(int dataSize) {
+    this.dataSize = dataSize;
+  }
+
+  public byte getChunkType() {
+    return chunkType;
+  }
+
+  public void setChunkType(byte chunkType) {
+    this.chunkType = chunkType;
+  }
+
+  public void increasePageNums(int i) {
+    numOfPages += i;
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/PageHeader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/PageHeader.java
index fa24f93..0ee8404 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/PageHeader.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/PageHeader.java
@@ -23,10 +23,9 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
-
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
 
 public class PageHeader {
 
@@ -41,25 +40,40 @@ public class PageHeader {
     this.statistics = statistics;
   }
 
-  public static int calculatePageHeaderSizeWithoutStatistics() {
-    return 2 * Integer.BYTES; // uncompressedSize, compressedSize
-  }
-
-  public static PageHeader deserializeFrom(InputStream inputStream, TSDataType dataType)
-      throws IOException {
-    int uncompressedSize = ReadWriteIOUtils.readInt(inputStream);
-    int compressedSize = ReadWriteIOUtils.readInt(inputStream);
-    Statistics statistics = Statistics.deserialize(inputStream, dataType);
+  /**
+   * max page header size without statistics
+   */
+  public static int estimateMaxPageHeaderSizeWithoutStatistics() {
+    // uncompressedSize, compressedSize
+    // because we use unsigned varInt to encode these two integer,
+    //each unsigned arInt will cost at most 5 bytes
+    return 2 * (Integer.BYTES + 1);
+  }
+
+  public static PageHeader deserializeFrom(InputStream inputStream, TSDataType dataType,
+      boolean hasStatistic) throws IOException {
+    int uncompressedSize = ReadWriteForEncodingUtils.readUnsignedVarInt(inputStream);
+    int compressedSize = ReadWriteForEncodingUtils.readUnsignedVarInt(inputStream);
+    Statistics statistics = null;
+    if (hasStatistic) {
+      statistics = Statistics.deserialize(inputStream, dataType);
+    }
     return new PageHeader(uncompressedSize, compressedSize, statistics);
   }
 
   public static PageHeader deserializeFrom(ByteBuffer buffer, TSDataType dataType) {
-    int uncompressedSize = ReadWriteIOUtils.readInt(buffer);
-    int compressedSize = ReadWriteIOUtils.readInt(buffer);
+    int uncompressedSize = ReadWriteForEncodingUtils.readUnsignedVarInt(buffer);
+    int compressedSize = ReadWriteForEncodingUtils.readUnsignedVarInt(buffer);
     Statistics statistics = Statistics.deserialize(buffer, dataType);
     return new PageHeader(uncompressedSize, compressedSize, statistics);
   }
 
+  public static PageHeader deserializeFrom(ByteBuffer buffer, Statistics chunkStatistic) {
+    int uncompressedSize = ReadWriteForEncodingUtils.readUnsignedVarInt(buffer);
+    int compressedSize = ReadWriteForEncodingUtils.readUnsignedVarInt(buffer);
+    return new PageHeader(uncompressedSize, compressedSize, chunkStatistic);
+  }
+
   public int getUncompressedSize() {
     return uncompressedSize;
   }
@@ -93,8 +107,8 @@ public class PageHeader {
   }
 
   public void serializeTo(OutputStream outputStream) throws IOException {
-    ReadWriteIOUtils.write(uncompressedSize, outputStream);
-    ReadWriteIOUtils.write(compressedSize, outputStream);
+    ReadWriteForEncodingUtils.writeUnsignedVarInt(uncompressedSize, outputStream);
+    ReadWriteForEncodingUtils.writeUnsignedVarInt(compressedSize, outputStream);
     statistics.serialize(outputStream);
   }
 
@@ -111,4 +125,14 @@ public class PageHeader {
   public void setModified(boolean modified) {
     this.modified = modified;
   }
+
+  /**
+   * max page header size without statistics
+   */
+  public int getSerializedPageSize() {
+    return ReadWriteForEncodingUtils.uVarIntSize(uncompressedSize)
+        + ReadWriteForEncodingUtils.uVarIntSize(compressedSize)
+        + (statistics == null ? 0 : statistics.getSerializedSize()) // page header
+        + compressedSize; // page data
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/package-info.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/package-info.java
deleted file mode 100644
index 9c1701a..0000000
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/package-info.java
+++ /dev/null
@@ -1,30 +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.tsfile.file.header;
-
-/**
- * CHUNK_GROUP_FOOTER and CHUNK_HEADER are used for parsing file.
- *
- * ChunkGroupMetadata and ChunkMetadata are used for locating the positions of
- * ChunkGroup (footer) and chunk (header),filtering data quickly, and thereby
- * they have statistics information.
- *
- * However, because Page has only the header structure, therefore, PageHeader
- * has the both two functions.
- */
\ No newline at end of file
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java
index 71e2596..a377b7b 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java
@@ -83,9 +83,9 @@ public class ChunkMetadata implements Accountable {
    * constructor of ChunkMetaData.
    *
    * @param measurementUid measurement id
-   * @param tsDataType time series data type
-   * @param fileOffset file offset
-   * @param statistics value statistics
+   * @param tsDataType     time series data type
+   * @param fileOffset     file offset
+   * @param statistics     value statistics
    */
   public ChunkMetadata(String measurementUid, TSDataType tsDataType, long fileOffset,
       Statistics statistics) {
@@ -142,31 +142,36 @@ public class ChunkMetadata implements Accountable {
    * @return length
    * @throws IOException IOException
    */
-  public int serializeTo(OutputStream outputStream) throws IOException {
+  public int serializeTo(OutputStream outputStream, boolean serializeStatistic) throws IOException {
     int byteLen = 0;
-
-    byteLen += ReadWriteIOUtils.write(measurementUid, outputStream);
     byteLen += ReadWriteIOUtils.write(offsetOfChunkHeader, outputStream);
-    byteLen += ReadWriteIOUtils.write(tsDataType, outputStream);
-    byteLen += statistics.serialize(outputStream);
+    if (serializeStatistic) {
+      byteLen += statistics.serialize(outputStream);
+    }
     return byteLen;
   }
 
   /**
    * deserialize from ByteBuffer.
    *
-   * @param buffer ByteBuffer
+   * @param buffer          ByteBuffer
    * @return ChunkMetaData object
    */
-  public static ChunkMetadata deserializeFrom(ByteBuffer buffer) {
+  public static ChunkMetadata deserializeFrom(ByteBuffer buffer, TimeseriesMetadata timeseriesMetadata) {
     ChunkMetadata chunkMetaData = new ChunkMetadata();
 
-    chunkMetaData.measurementUid = ReadWriteIOUtils.readString(buffer);
+    chunkMetaData.measurementUid = timeseriesMetadata.getMeasurementId();
+    chunkMetaData.tsDataType = timeseriesMetadata.getTSDataType();
     chunkMetaData.offsetOfChunkHeader = ReadWriteIOUtils.readLong(buffer);
-    chunkMetaData.tsDataType = ReadWriteIOUtils.readDataType(buffer);
-
-    chunkMetaData.statistics = Statistics.deserialize(buffer, chunkMetaData.tsDataType);
-
+    // if the TimeSeriesMetadataType is not 0, it means it has more than one chunk
+    // and each chunk's metadata has its own statistics
+    if (timeseriesMetadata.getTimeSeriesMetadataType() != 0) {
+      chunkMetaData.statistics = Statistics.deserialize(buffer, chunkMetaData.tsDataType);
+    } else {
+      // if the TimeSeriesMetadataType is 0, it means it has only one chunk
+      //and that chunk's metadata has no statistic
+      chunkMetaData.statistics = timeseriesMetadata.getStatistics();
+    }
     return chunkMetaData;
   }
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexConstructor.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexConstructor.java
index 07dd66e..ec4f50a 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexConstructor.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexConstructor.java
@@ -33,8 +33,7 @@ import org.apache.iotdb.tsfile.write.writer.TsFileOutput;
 
 public class MetadataIndexConstructor {
 
-  private static final TSFileConfig config =
-      TSFileDescriptor.getInstance().getConfig();
+  private static final TSFileConfig config = TSFileDescriptor.getInstance().getConfig();
 
   private MetadataIndexConstructor() {
     throw new IllegalStateException("Utility class");
@@ -81,7 +80,7 @@ public class MetadataIndexConstructor {
     // if not exceed the max child nodes num, ignore the device index and directly point to the measurement
     if (deviceMetadataIndexMap.size() <= config.getMaxDegreeOfIndexNode()) {
       MetadataIndexNode metadataIndexNode = new MetadataIndexNode(
-          MetadataIndexNodeType.INTERNAL_MEASUREMENT);
+          MetadataIndexNodeType.LEAF_DEVICE);
       for (Map.Entry<String, MetadataIndexNode> entry : deviceMetadataIndexMap.entrySet()) {
         metadataIndexNode.addEntry(new MetadataIndexEntry(entry.getKey(), out.getPosition()));
         entry.getValue().serializeTo(out.wrapAsStream());
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexEntry.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexEntry.java
index 5325992..73667ba 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexEntry.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexEntry.java
@@ -56,13 +56,13 @@ public class MetadataIndexEntry {
 
   public int serializeTo(OutputStream outputStream) throws IOException {
     int byteLen = 0;
-    byteLen += ReadWriteIOUtils.write(name, outputStream);
+    byteLen += ReadWriteIOUtils.writeVar(name, outputStream);
     byteLen += ReadWriteIOUtils.write(offset, outputStream);
     return byteLen;
   }
 
   public static MetadataIndexEntry deserializeFrom(ByteBuffer buffer) {
-    String name = ReadWriteIOUtils.readString(buffer);
+    String name = ReadWriteIOUtils.readVarIntString(buffer);
     long offset = ReadWriteIOUtils.readLong(buffer);
     return new MetadataIndexEntry(name, offset);
   }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexNode.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexNode.java
index 037e768..019d1c2 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexNode.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexNode.java
@@ -28,19 +28,19 @@ import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.file.metadata.enums.MetadataIndexNodeType;
 import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 public class MetadataIndexNode {
 
-  private static final TSFileConfig config =
-      TSFileDescriptor.getInstance().getConfig();
-  private List<MetadataIndexEntry> children;
+  private static final TSFileConfig config = TSFileDescriptor.getInstance().getConfig();
+  private final List<MetadataIndexEntry> children;
   private long endOffset;
 
   /**
    * type of the child node at offset
    */
-  private MetadataIndexNodeType nodeType;
+  private final MetadataIndexNodeType nodeType;
 
   public MetadataIndexNode(MetadataIndexNodeType nodeType) {
     children = new ArrayList<>();
@@ -88,7 +88,7 @@ public class MetadataIndexNode {
 
   public int serializeTo(OutputStream outputStream) throws IOException {
     int byteLen = 0;
-    byteLen += ReadWriteIOUtils.write(children.size(), outputStream);
+    byteLen += ReadWriteForEncodingUtils.writeUnsignedVarInt(children.size(), outputStream);
     for (MetadataIndexEntry metadataIndexEntry : children) {
       byteLen += metadataIndexEntry.serializeTo(outputStream);
     }
@@ -99,7 +99,7 @@ public class MetadataIndexNode {
 
   public static MetadataIndexNode deserializeFrom(ByteBuffer buffer) {
     List<MetadataIndexEntry> children = new ArrayList<>();
-    int size = ReadWriteIOUtils.readInt(buffer);
+    int size = ReadWriteForEncodingUtils.readUnsignedVarInt(buffer);
     for (int i = 0; i < size; i++) {
       children.add(MetadataIndexEntry.deserializeFrom(buffer));
     }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TimeseriesMetadata.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TimeseriesMetadata.java
index 0869643..4f93f17 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TimeseriesMetadata.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TimeseriesMetadata.java
@@ -27,10 +27,17 @@ import org.apache.iotdb.tsfile.common.cache.Accountable;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
 import org.apache.iotdb.tsfile.read.controller.IChunkMetadataLoader;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 public class TimeseriesMetadata implements Accountable {
 
+  /**
+   * 0 means this time series has only one chunk, no need to save the statistic again in chunk metadata
+   * 1 means this time series has more than one chunk, should save the statistic again in chunk metadata
+   */
+  private byte timeSeriesMetadataType;
+
   private long startOffsetOfChunkMetaDataList;
   private int chunkMetaDataListDataSize;
 
@@ -52,8 +59,10 @@ public class TimeseriesMetadata implements Accountable {
   public TimeseriesMetadata() {
   }
 
-  public TimeseriesMetadata(long startOffsetOfChunkMetaDataList, int chunkMetaDataListDataSize,
-      String measurementId, TSDataType dataType, Statistics statistics) {
+  public TimeseriesMetadata(byte timeSeriesMetadataType, long startOffsetOfChunkMetaDataList,
+      int chunkMetaDataListDataSize, String measurementId, TSDataType dataType,
+      Statistics statistics) {
+    this.timeSeriesMetadataType = timeSeriesMetadataType;
     this.startOffsetOfChunkMetaDataList = startOffsetOfChunkMetaDataList;
     this.chunkMetaDataListDataSize = chunkMetaDataListDataSize;
     this.measurementId = measurementId;
@@ -62,6 +71,7 @@ public class TimeseriesMetadata implements Accountable {
   }
 
   public TimeseriesMetadata(TimeseriesMetadata timeseriesMetadata) {
+    this.timeSeriesMetadataType = timeseriesMetadata.timeSeriesMetadataType;
     this.startOffsetOfChunkMetaDataList = timeseriesMetadata.startOffsetOfChunkMetaDataList;
     this.chunkMetaDataListDataSize = timeseriesMetadata.chunkMetaDataListDataSize;
     this.measurementId = timeseriesMetadata.measurementId;
@@ -72,10 +82,12 @@ public class TimeseriesMetadata implements Accountable {
 
   public static TimeseriesMetadata deserializeFrom(ByteBuffer buffer) {
     TimeseriesMetadata timeseriesMetaData = new TimeseriesMetadata();
-    timeseriesMetaData.setMeasurementId(ReadWriteIOUtils.readString(buffer));
+    timeseriesMetaData.setTimeSeriesMetadataType(ReadWriteIOUtils.readByte(buffer));
+    timeseriesMetaData.setMeasurementId(ReadWriteIOUtils.readVarIntString(buffer));
     timeseriesMetaData.setTSDataType(ReadWriteIOUtils.readDataType(buffer));
     timeseriesMetaData.setOffsetOfChunkMetaDataList(ReadWriteIOUtils.readLong(buffer));
-    timeseriesMetaData.setDataSizeOfChunkMetaDataList(ReadWriteIOUtils.readInt(buffer));
+    timeseriesMetaData
+        .setDataSizeOfChunkMetaDataList(ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
     timeseriesMetaData.setStatistics(Statistics.deserialize(buffer, timeseriesMetaData.dataType));
     return timeseriesMetaData;
   }
@@ -89,14 +101,24 @@ public class TimeseriesMetadata implements Accountable {
    */
   public int serializeTo(OutputStream outputStream) throws IOException {
     int byteLen = 0;
-    byteLen += ReadWriteIOUtils.write(measurementId, outputStream);
+    byteLen += ReadWriteIOUtils.write(timeSeriesMetadataType, outputStream);
+    byteLen += ReadWriteIOUtils.writeVar(measurementId, outputStream);
     byteLen += ReadWriteIOUtils.write(dataType, outputStream);
     byteLen += ReadWriteIOUtils.write(startOffsetOfChunkMetaDataList, outputStream);
-    byteLen += ReadWriteIOUtils.write(chunkMetaDataListDataSize, outputStream);
+    byteLen += ReadWriteForEncodingUtils
+        .writeUnsignedVarInt(chunkMetaDataListDataSize, outputStream);
     byteLen += statistics.serialize(outputStream);
     return byteLen;
   }
 
+  public byte getTimeSeriesMetadataType() {
+    return timeSeriesMetadataType;
+  }
+
+  public void setTimeSeriesMetadataType(byte timeSeriesMetadataType) {
+    this.timeSeriesMetadataType = timeSeriesMetadataType;
+  }
+
   public long getOffsetOfChunkMetaDataList() {
     return startOffsetOfChunkMetaDataList;
   }
@@ -161,7 +183,7 @@ public class TimeseriesMetadata implements Accountable {
   public long getRamSize() {
     return ramSize;
   }
-  
+
   public void setSeq(boolean seq) {
     isSeq = seq;
   }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TsFileMetadata.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TsFileMetadata.java
index 5a039b7..e21d222 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TsFileMetadata.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TsFileMetadata.java
@@ -26,6 +26,7 @@ import java.util.Set;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.utils.BloomFilter;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 /**
@@ -33,13 +34,6 @@ import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
  */
 public class TsFileMetadata {
 
-  // fields below are IoTDB extensions and they does not affect TsFile's
-  // stand-alone functionality
-  private int totalChunkNum;
-  // invalid means a chunk has been rewritten by merge and the chunk's data is in
-  // another new chunk
-  private int invalidChunkNum;
-
   // bloom filter
   private BloomFilter bloomFilter;
 
@@ -60,8 +54,6 @@ public class TsFileMetadata {
 
     // metadataIndex
     fileMetaData.metadataIndex = MetadataIndexNode.deserializeFrom(buffer);
-    fileMetaData.totalChunkNum = ReadWriteIOUtils.readInt(buffer);
-    fileMetaData.invalidChunkNum = ReadWriteIOUtils.readInt(buffer);
 
     // metaOffset
     long metaOffset = ReadWriteIOUtils.readLong(buffer);
@@ -69,9 +61,9 @@ public class TsFileMetadata {
 
     // read bloom filter
     if (buffer.hasRemaining()) {
-      byte[] bytes = ReadWriteIOUtils.readByteBufferWithSelfDescriptionLength(buffer).array();
-      int filterSize = ReadWriteIOUtils.readInt(buffer);
-      int hashFunctionSize = ReadWriteIOUtils.readInt(buffer);
+      byte[] bytes = ReadWriteIOUtils.readByteBufferWithSelfDescriptionLength(buffer);
+      int filterSize = ReadWriteForEncodingUtils.readUnsignedVarInt(buffer);
+      int hashFunctionSize = ReadWriteForEncodingUtils.readUnsignedVarInt(buffer);
       fileMetaData.bloomFilter = BloomFilter.buildBloomFilter(bytes, filterSize, hashFunctionSize);
     }
 
@@ -82,6 +74,10 @@ public class TsFileMetadata {
     return bloomFilter;
   }
 
+  public void setBloomFilter(BloomFilter bloomFilter) {
+    this.bloomFilter = bloomFilter;
+  }
+
   /**
    * use the given outputStream to serialize.
    *
@@ -98,9 +94,6 @@ public class TsFileMetadata {
       byteLen += ReadWriteIOUtils.write(0, outputStream);
     }
 
-    // totalChunkNum, invalidChunkNum
-    byteLen += ReadWriteIOUtils.write(totalChunkNum, outputStream);
-    byteLen += ReadWriteIOUtils.write(invalidChunkNum, outputStream);
 
     // metaOffset
     byteLen += ReadWriteIOUtils.write(metaOffset, outputStream);
@@ -120,11 +113,12 @@ public class TsFileMetadata {
     BloomFilter filter = buildBloomFilter(paths);
 
     byte[] bytes = filter.serialize();
-    byteLen += ReadWriteIOUtils.write(bytes.length, outputStream);
+    byteLen += ReadWriteForEncodingUtils.writeUnsignedVarInt(bytes.length, outputStream);
     outputStream.write(bytes);
     byteLen += bytes.length;
-    byteLen += ReadWriteIOUtils.write(filter.getSize(), outputStream);
-    byteLen += ReadWriteIOUtils.write(filter.getHashFunctionSize(), outputStream);
+    byteLen += ReadWriteForEncodingUtils.writeUnsignedVarInt(filter.getSize(), outputStream);
+    byteLen += ReadWriteForEncodingUtils
+        .writeUnsignedVarInt(filter.getHashFunctionSize(), outputStream);
     return byteLen;
   }
 
@@ -143,21 +137,6 @@ public class TsFileMetadata {
     return filter;
   }
 
-  public int getTotalChunkNum() {
-    return totalChunkNum;
-  }
-
-  public void setTotalChunkNum(int totalChunkNum) {
-    this.totalChunkNum = totalChunkNum;
-  }
-
-  public int getInvalidChunkNum() {
-    return invalidChunkNum;
-  }
-
-  public void setInvalidChunkNum(int invalidChunkNum) {
-    this.invalidChunkNum = invalidChunkNum;
-  }
 
   public long getMetaOffset() {
     return metaOffset;
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/CompressionType.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/CompressionType.java
index e596b20..661ba42 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/CompressionType.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/CompressionType.java
@@ -22,107 +22,77 @@ public enum CompressionType {
   /**
    * Do not comprocess
    */
-  UNCOMPRESSED("", 0),
+  UNCOMPRESSED("", (byte) 0),
 
   /**
    * SNAPPY
    */
-  SNAPPY(".snappy", 1),
+  SNAPPY(".snappy", (byte) 1),
 
   /**
    * GZIP
    */
-  GZIP(".gzip", 2),
+  GZIP(".gzip", (byte) 2),
 
   /**
    * LZO
    */
-  LZO(".lzo", 3),
+  LZO(".lzo", (byte) 3),
 
   /**
    * SDT
    */
-  SDT(".sdt", 4),
+  SDT(".sdt", (byte) 4),
 
   /**
    * PAA
    */
-  PAA(".paa", 5),
+  PAA(".paa", (byte) 5),
 
   /**
    * PLA
    */
-  PLA(".pla", 6),
+  PLA(".pla", (byte) 6),
 
   /**
    * LZ4
    */
-  LZ4(".lz4", 7);
+  LZ4(".lz4", (byte) 7);
 
   private final String extensionName;
-  private final int index;
+  private final byte index;
 
-  CompressionType(String extensionName, int index) {
+  CompressionType(String extensionName, byte index) {
     this.extensionName = extensionName;
     this.index = index;
   }
 
   /**
-   * deserialize short number.
+   * deserialize byte number.
    *
-   * @param compressor short number
+   * @param compressor byte number
    * @return CompressionType
    */
-  public static CompressionType deserialize(short compressor) {
-    return getCompressionType(compressor);
-  }
-
-  public static byte deserializeToByte(short compressor) {
-    //check compressor is valid
-    getCompressionType(compressor);
-    return (byte) compressor;
-  }
-
-
-  private static CompressionType getCompressionType(short compressor) {
+  public static CompressionType deserialize(byte compressor) {
     for (CompressionType compressionType : CompressionType.values()) {
       if (compressor == compressionType.index) {
         return compressionType;
       }
     }
-
     throw new IllegalArgumentException("Invalid input: " + compressor);
   }
 
-  /**
-   * give an byte to return a compression type.
-   *
-   * @param compressor byte number
-   * @return CompressionType
-   */
-  public static CompressionType byteToEnum(byte compressor) {
-    return getCompressionType(compressor);
-  }
-
   public static int getSerializedSize() {
-    return Short.BYTES;
-  }
-
-  /**
-   * serialize.
-   *
-   * @return short number
-   */
-  public short serialize() {
-    return enumToByte();
+    return Byte.BYTES;
   }
 
   /**
    * @return byte number
    */
-  public byte enumToByte() {
-    return (byte) index;
+  public byte serialize() {
+    return this.index;
   }
+
   /**
    * get extension.
    *
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/MetadataIndexNodeType.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/MetadataIndexNodeType.java
index ecc73b3..27c0c43 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/MetadataIndexNodeType.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/MetadataIndexNodeType.java
@@ -33,7 +33,31 @@ import java.nio.ByteBuffer;
  * LEAF_MEASUREMENT: leaf nodes of the index tree's device level, points to TimeseriesMetadata
  */
 public enum MetadataIndexNodeType {
-  INTERNAL_DEVICE, LEAF_DEVICE, INTERNAL_MEASUREMENT, LEAF_MEASUREMENT;
+  /**
+   * INTERNAL_DEVICE
+   */
+  INTERNAL_DEVICE((byte) 0),
+
+  /**
+   * LEAF_DEVICE
+   */
+  LEAF_DEVICE((byte) 1),
+
+  /**
+   * INTERNAL_MEASUREMENT
+   */
+  INTERNAL_MEASUREMENT((byte) 2),
+
+  /**
+   * INTERNAL_MEASUREMENT
+   */
+  LEAF_MEASUREMENT((byte) 3);
+
+  private final byte type;
+
+  MetadataIndexNodeType(byte type) {
+    this.type = type;
+  }
 
   /**
    * deserialize byte number.
@@ -42,19 +66,13 @@ public enum MetadataIndexNodeType {
    * @return MetadataIndexNodeType
    */
   public static MetadataIndexNodeType deserialize(byte i) {
-    if (i >= 4) {
-      throw new IllegalArgumentException("Invalid input: " + i);
-    }
-    switch (i) {
-      case 0:
-        return INTERNAL_DEVICE;
-      case 1:
-        return LEAF_DEVICE;
-      case 2:
-        return INTERNAL_MEASUREMENT;
-      default:
-        return LEAF_MEASUREMENT;
+    for (MetadataIndexNodeType metadataIndexNodeType : MetadataIndexNodeType.values()) {
+      if (i == metadataIndexNodeType.type) {
+        return metadataIndexNodeType;
+      }
     }
+
+    throw new IllegalArgumentException("Invalid input: " + i);
   }
 
   public static MetadataIndexNodeType deserializeFrom(ByteBuffer buffer) {
@@ -79,17 +97,6 @@ public enum MetadataIndexNodeType {
    * @return -enum type
    */
   public byte serialize() {
-    switch (this) {
-      case INTERNAL_DEVICE:
-        return 0;
-      case LEAF_DEVICE:
-        return 1;
-      case INTERNAL_MEASUREMENT:
-        return 2;
-      case LEAF_MEASUREMENT:
-        return 3;
-      default:
-        return -1;
-    }
+    return type;
   }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSDataType.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSDataType.java
index 505be1b..3e2ca80 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSDataType.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSDataType.java
@@ -24,79 +24,77 @@ import java.nio.ByteBuffer;
 import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 
 public enum TSDataType {
-  BOOLEAN, INT32, INT64, FLOAT, DOUBLE, TEXT;
+  /**
+   * BOOLEAN
+   */
+  BOOLEAN((byte) 0),
 
   /**
-   * give an integer to return a data type.
    *
-   * @param type -param to judge enum type
-   * @return -enum type
    */
-  public static TSDataType deserialize(short type) {
-    return getTsDataType(type);
-  }
+  INT32((byte) 1),
 
+  /**
+   * INT64
+   */
+  INT64((byte) 2),
 
-  private static TSDataType getTsDataType(short type) {
-    if (type >= 6 || type < 0) {
-      throw new IllegalArgumentException("Invalid input: " + type);
-    }
-    switch (type) {
-      case 0:
-        return BOOLEAN;
-      case 1:
-        return INT32;
-      case 2:
-        return INT64;
-      case 3:
-        return FLOAT;
-      case 4:
-        return DOUBLE;
-      default:
-        return TEXT;
-    }
-  }
+  /**
+   * FLOAT
+   */
+  FLOAT((byte) 3),
 
-  public static byte deserializeToByte(short type) {
-    if (type >= 6 || type < 0) {
-      throw new IllegalArgumentException("Invalid input: " + type);
-    }
-    return (byte) type;
+  /**
+   * DOUBLE
+   */
+  DOUBLE((byte) 4),
+
+  /**
+   * TEXT
+   */
+  TEXT((byte) 5);
+
+  private final byte type;
+
+  TSDataType(byte type) {
+    this.type = type;
   }
 
   /**
-   * give an byte to return a data type.
+   * give an integer to return a data type.
    *
-   * @param type byte number
-   * @return data type
+   * @param type -param to judge enum type
+   * @return -enum type
    */
-  public static TSDataType byteToEnum(byte type) {
+  public static TSDataType deserialize(byte type) {
     return getTsDataType(type);
   }
 
+
+  private static TSDataType getTsDataType(byte type) {
+    for (TSDataType tsDataType : TSDataType.values()) {
+      if (type == tsDataType.type) {
+        return tsDataType;
+      }
+    }
+
+    throw new IllegalArgumentException("Invalid input: " + type);
+  }
+
   public static TSDataType deserializeFrom(ByteBuffer buffer) {
-    return deserialize(buffer.getShort());
+    return deserialize(buffer.get());
   }
 
   public static int getSerializedSize() {
-    return Short.BYTES;
+    return Byte.BYTES;
   }
 
   public void serializeTo(ByteBuffer byteBuffer) {
-    byteBuffer.putShort(serialize());
+    byteBuffer.put(serialize());
   }
 
   public void serializeTo(DataOutputStream outputStream) throws IOException {
-    outputStream.writeShort(serialize());
-  }
-
-  /**
-   * return a serialize data type.
-   *
-   * @return -enum type
-   */
-  public short serialize() {
-    return enumToByte();
+    outputStream.write(serialize());
   }
 
   public int getDataTypeSize() {
@@ -119,22 +117,7 @@ public enum TSDataType {
   /**
    * @return byte number
    */
-  public byte enumToByte() {
-    switch (this) {
-      case BOOLEAN:
-        return 0;
-      case INT32:
-        return 1;
-      case INT64:
-        return 2;
-      case FLOAT:
-        return 3;
-      case DOUBLE:
-        return 4;
-      case TEXT:
-        return 5;
-      default:
-        return -1;
-    }
+  public byte serialize() {
+    return type;
   }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSEncoding.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSEncoding.java
index a909e47..8af115d 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSEncoding.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSEncoding.java
@@ -20,7 +20,21 @@ package org.apache.iotdb.tsfile.file.metadata.enums;
 
 public enum TSEncoding {
 
-  PLAIN, PLAIN_DICTIONARY, RLE, DIFF, TS_2DIFF, BITMAP, GORILLA_V1, REGULAR, GORILLA;
+  PLAIN((byte) 0),
+  PLAIN_DICTIONARY((byte) 1),
+  RLE((byte) 2),
+  DIFF((byte) 3),
+  TS_2DIFF((byte) 4),
+  BITMAP((byte) 5),
+  GORILLA_V1((byte) 6),
+  REGULAR( (byte) 7),
+  GORILLA((byte) 8);
+
+  private final byte type;
+
+  TSEncoding(byte type) {
+    this.type = type;
+  }
 
   /**
    * judge the encoding deserialize type.
@@ -28,55 +42,23 @@ public enum TSEncoding {
    * @param encoding -use to determine encoding type
    * @return -encoding type
    */
-  public static TSEncoding deserialize(short encoding) {
+  public static TSEncoding deserialize(byte encoding) {
     return getTsEncoding(encoding);
   }
 
-  public static byte deserializeToByte(short encoding) {
-    if (encoding < 0 || 8 < encoding) {
-      throw new IllegalArgumentException("Invalid input: " + encoding);
-    }
-    return (byte) encoding;
-  }
 
-  private static TSEncoding getTsEncoding(short encoding) {
-    if (encoding < 0 || 8 < encoding) {
-      throw new IllegalArgumentException("Invalid input: " + encoding);
-    }
-    switch (encoding) {
-      case 1:
-        return PLAIN_DICTIONARY;
-      case 2:
-        return RLE;
-      case 3:
-        return DIFF;
-      case 4:
-        return TS_2DIFF;
-      case 5:
-        return BITMAP;
-      case 6:
-        return GORILLA_V1;
-      case 7:
-        return REGULAR;
-      case 8:
-        return GORILLA;
-      default:
-        return PLAIN;
+  private static TSEncoding getTsEncoding(byte encoding) {
+    for (TSEncoding tsEncoding : TSEncoding.values()) {
+      if (encoding == tsEncoding.type) {
+        return tsEncoding;
+      }
     }
-  }
 
-  /**
-   * give an byte to return a encoding type.
-   *
-   * @param encoding byte number
-   * @return encoding type
-   */
-  public static TSEncoding byteToEnum(byte encoding) {
-    return getTsEncoding(encoding);
+    throw new IllegalArgumentException("Invalid input: " + encoding);
   }
 
   public static int getSerializedSize() {
-    return Short.BYTES;
+    return Byte.BYTES;
   }
 
   /**
@@ -84,33 +66,7 @@ public enum TSEncoding {
    *
    * @return -encoding type
    */
-  public short serialize() {
-    return enumToByte();
-  }
-
-  /**
-   * @return byte number
-   */
-  public byte enumToByte() {
-    switch (this) {
-      case PLAIN_DICTIONARY:
-        return 1;
-      case RLE:
-        return 2;
-      case DIFF:
-        return 3;
-      case TS_2DIFF:
-        return 4;
-      case BITMAP:
-        return 5;
-      case GORILLA_V1:
-        return 6;
-      case REGULAR:
-        return 7;
-      case GORILLA:
-        return 8;
-      default:
-        return 0;
-    }
+  public byte serialize() {
+    return type;
   }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/BinaryStatistics.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/BinaryStatistics.java
index 607589b..e2567f7 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/BinaryStatistics.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/BinaryStatistics.java
@@ -101,8 +101,13 @@ public class BinaryStatistics extends Statistics<Binary> {
   }
 
   @Override
-  public double getSumValue() {
-    throw new StatisticsClassException(String.format(BINARY_STATS_UNSUPPORTED_MSG, "sum"));
+  public double getSumDoubleValue() {
+    throw new StatisticsClassException(String.format(BINARY_STATS_UNSUPPORTED_MSG, "double sum"));
+  }
+
+  @Override
+  public long getSumLongValue() {
+    throw new StatisticsClassException(String.format(BINARY_STATS_UNSUPPORTED_MSG, "long sum"));
   }
 
   @Override
@@ -197,13 +202,13 @@ public class BinaryStatistics extends Statistics<Binary> {
   }
 
   @Override
-  void deserialize(InputStream inputStream) throws IOException {
+  public void deserialize(InputStream inputStream) throws IOException {
     this.firstValue = ReadWriteIOUtils.readBinary(inputStream);
     this.lastValue = ReadWriteIOUtils.readBinary(inputStream);
   }
 
   @Override
-  void deserialize(ByteBuffer byteBuffer) {
+  public void deserialize(ByteBuffer byteBuffer) {
     this.firstValue = ReadWriteIOUtils.readBinary(byteBuffer);
     this.lastValue = ReadWriteIOUtils.readBinary(byteBuffer);
   }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/BooleanStatistics.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/BooleanStatistics.java
index f7c2d1c..481d8df 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/BooleanStatistics.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/BooleanStatistics.java
@@ -32,8 +32,9 @@ public class BooleanStatistics extends Statistics<Boolean> {
 
   private boolean firstValue;
   private boolean lastValue;
+  private long sumValue;
 
-  static final int BOOLEAN_STATISTICS_FIXED_RAM_SIZE = 48;
+  static final int BOOLEAN_STATISTICS_FIXED_RAM_SIZE = 56;
 
 
   @Override
@@ -43,7 +44,7 @@ public class BooleanStatistics extends Statistics<Boolean> {
 
   @Override
   public int getStatsSize() {
-    return 2;
+    return 10;
   }
 
   /**
@@ -52,16 +53,18 @@ public class BooleanStatistics extends Statistics<Boolean> {
    * @param firstValue first boolean value
    * @param lastValue  last boolean value
    */
-  public void initializeStats(boolean firstValue, boolean lastValue) {
+  public void initializeStats(boolean firstValue, boolean lastValue, long sum) {
     this.firstValue = firstValue;
     this.lastValue = lastValue;
+    this.sumValue = sum;
   }
 
-  private void updateStats(boolean firstValue, boolean lastValue) {
+  private void updateStats(boolean lastValue, long sum) {
     this.lastValue = lastValue;
+    this.sumValue += sum;
   }
 
-  private void updateStats(boolean firstValue, boolean lastValue, long startTime, long endTime) {
+  private void updateStats(boolean firstValue, boolean lastValue, long startTime, long endTime, long sum) {
     // only if endTime greater or equals to the current endTime need we update the last value
     // only if startTime less or equals to the current startTime need we update the first value
     // otherwise, just ignore
@@ -71,15 +74,16 @@ public class BooleanStatistics extends Statistics<Boolean> {
     if (endTime >= this.getEndTime()) {
       this.lastValue = lastValue;
     }
+    this.sumValue += sum;
   }
 
   @Override
   void updateStats(boolean value) {
     if (isEmpty) {
-      initializeStats(value, value);
+      initializeStats(value, value, value ? 1 : 0);
       isEmpty = false;
     } else {
-      updateStats(value, value);
+      updateStats(value, value ? 1 : 0);
     }
   }
 
@@ -120,8 +124,13 @@ public class BooleanStatistics extends Statistics<Boolean> {
   }
 
   @Override
-  public double getSumValue() {
-    throw new StatisticsClassException("Boolean statistics does not support: sum");
+  public double getSumDoubleValue() {
+    throw new StatisticsClassException("Boolean statistics does not support: double sum");
+  }
+
+  @Override
+  public long getSumLongValue() {
+    return sumValue;
   }
 
   @Override
@@ -146,17 +155,18 @@ public class BooleanStatistics extends Statistics<Boolean> {
 
   @Override
   public ByteBuffer getSumValueBuffer() {
-    throw new StatisticsClassException("Boolean statistics do not support: sum");
+    return ReadWriteIOUtils.getByteBuffer(sumValue);
   }
 
   @Override
   protected void mergeStatisticsValue(Statistics stats) {
     BooleanStatistics boolStats = (BooleanStatistics) stats;
     if (isEmpty) {
-      initializeStats(boolStats.getFirstValue(), boolStats.getLastValue());
+      initializeStats(boolStats.getFirstValue(), boolStats.getLastValue(), boolStats.sumValue);
       isEmpty = false;
     } else {
-      updateStats(boolStats.getFirstValue(), boolStats.getLastValue(), stats.getStartTime(), stats.getEndTime());
+      updateStats(boolStats.getFirstValue(), boolStats.getLastValue(), stats.getStartTime(),
+          stats.getEndTime(), boolStats.sumValue);
     }
   }
 
@@ -182,7 +192,7 @@ public class BooleanStatistics extends Statistics<Boolean> {
 
   @Override
   public byte[] getSumValueBytes() {
-    throw new StatisticsClassException("Boolean statistics does not support: sum");
+    return BytesUtils.longToBytes(sumValue);
   }
 
   @Override
@@ -190,23 +200,29 @@ public class BooleanStatistics extends Statistics<Boolean> {
     int byteLen = 0;
     byteLen += ReadWriteIOUtils.write(firstValue, outputStream);
     byteLen += ReadWriteIOUtils.write(lastValue, outputStream);
+    byteLen += ReadWriteIOUtils.write(sumValue, outputStream);
     return byteLen;
   }
 
   @Override
-  void deserialize(InputStream inputStream) throws IOException {
+  public void deserialize(InputStream inputStream) throws IOException {
     this.firstValue = ReadWriteIOUtils.readBool(inputStream);
     this.lastValue = ReadWriteIOUtils.readBool(inputStream);
+    this.sumValue = ReadWriteIOUtils.readLong(inputStream);
   }
 
   @Override
-  void deserialize(ByteBuffer byteBuffer) {
+  public void deserialize(ByteBuffer byteBuffer) {
     this.firstValue = ReadWriteIOUtils.readBool(byteBuffer);
     this.lastValue = ReadWriteIOUtils.readBool(byteBuffer);
+    this.sumValue = ReadWriteIOUtils.readLong(byteBuffer);
   }
 
   @Override
   public String toString() {
-    return super.toString() + " [firstValue:" + firstValue + ",lastValue:" + lastValue + "]";
+    return super.toString() + " [firstValue=" + firstValue +
+        ", lastValue=" + lastValue +
+        ", sumValue=" + sumValue +
+        ']';
   }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/DoubleStatistics.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/DoubleStatistics.java
index ef7293c..1168623 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/DoubleStatistics.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/DoubleStatistics.java
@@ -18,14 +18,14 @@
  */
 package org.apache.iotdb.tsfile.file.metadata.statistics;
 
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.utils.BytesUtils;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
+import org.apache.iotdb.tsfile.exception.filter.StatisticsClassException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 public class DoubleStatistics extends Statistics<Double> {
 
@@ -143,20 +143,25 @@ public class DoubleStatistics extends Statistics<Double> {
   }
 
   @Override
-  public double getSumValue() {
+  public double getSumDoubleValue() {
     return sumValue;
   }
 
   @Override
+  public long getSumLongValue() {
+    throw new StatisticsClassException("Double statistics does not support: long sum");
+  }
+
+  @Override
   protected void mergeStatisticsValue(Statistics stats) {
     DoubleStatistics doubleStats = (DoubleStatistics) stats;
     if (this.isEmpty) {
       initializeStats(doubleStats.getMinValue(), doubleStats.getMaxValue(), doubleStats.getFirstValue(),
-          doubleStats.getLastValue(), doubleStats.getSumValue());
+          doubleStats.getLastValue(), doubleStats.sumValue);
       isEmpty = false;
     } else {
       updateStats(doubleStats.getMinValue(), doubleStats.getMaxValue(), doubleStats.getFirstValue(),
-          doubleStats.getLastValue(), doubleStats.getSumValue(), stats.getStartTime(), stats.getEndTime());
+          doubleStats.getLastValue(), doubleStats.sumValue, stats.getStartTime(), stats.getEndTime());
     }
   }
 
@@ -222,7 +227,7 @@ public class DoubleStatistics extends Statistics<Double> {
   }
 
   @Override
-  void deserialize(InputStream inputStream) throws IOException {
+  public void deserialize(InputStream inputStream) throws IOException {
     this.minValue = ReadWriteIOUtils.readDouble(inputStream);
     this.maxValue = ReadWriteIOUtils.readDouble(inputStream);
     this.firstValue = ReadWriteIOUtils.readDouble(inputStream);
@@ -231,7 +236,7 @@ public class DoubleStatistics extends Statistics<Double> {
   }
 
   @Override
-  void deserialize(ByteBuffer byteBuffer) {
+  public void deserialize(ByteBuffer byteBuffer) {
     this.minValue = ReadWriteIOUtils.readDouble(byteBuffer);
     this.maxValue = ReadWriteIOUtils.readDouble(byteBuffer);
     this.firstValue = ReadWriteIOUtils.readDouble(byteBuffer);
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/FloatStatistics.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/FloatStatistics.java
index 3080fd7..9bf7381 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/FloatStatistics.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/FloatStatistics.java
@@ -18,14 +18,14 @@
  */
 package org.apache.iotdb.tsfile.file.metadata.statistics;
 
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.utils.BytesUtils;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
+import org.apache.iotdb.tsfile.exception.filter.StatisticsClassException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 /**
  * Statistics for float type.
@@ -138,20 +138,25 @@ public class FloatStatistics extends Statistics<Float> {
   }
 
   @Override
-  public double getSumValue() {
+  public double getSumDoubleValue() {
     return sumValue;
   }
 
   @Override
+  public long getSumLongValue() {
+    throw new StatisticsClassException("Float statistics does not support: long sum");
+  }
+
+  @Override
   protected void mergeStatisticsValue(Statistics stats) {
     FloatStatistics floatStats = (FloatStatistics) stats;
     if (isEmpty) {
       initializeStats(floatStats.getMinValue(), floatStats.getMaxValue(), floatStats.getFirstValue(),
-          floatStats.getLastValue(), floatStats.getSumValue());
+          floatStats.getLastValue(), floatStats.sumValue);
       isEmpty = false;
     } else {
       updateStats(floatStats.getMinValue(), floatStats.getMaxValue(), floatStats.getFirstValue(),
-          floatStats.getLastValue(), floatStats.getSumValue(), stats.getStartTime(), stats.getEndTime());
+          floatStats.getLastValue(), floatStats.sumValue, stats.getStartTime(), stats.getEndTime());
     }
   }
 
@@ -217,7 +222,7 @@ public class FloatStatistics extends Statistics<Float> {
   }
 
   @Override
-  void deserialize(InputStream inputStream) throws IOException {
+  public void deserialize(InputStream inputStream) throws IOException {
     this.minValue = ReadWriteIOUtils.readFloat(inputStream);
     this.maxValue = ReadWriteIOUtils.readFloat(inputStream);
     this.firstValue = ReadWriteIOUtils.readFloat(inputStream);
@@ -226,7 +231,7 @@ public class FloatStatistics extends Statistics<Float> {
   }
 
   @Override
-  void deserialize(ByteBuffer byteBuffer) {
+  public void deserialize(ByteBuffer byteBuffer) {
     this.minValue = ReadWriteIOUtils.readFloat(byteBuffer);
     this.maxValue = ReadWriteIOUtils.readFloat(byteBuffer);
     this.firstValue = ReadWriteIOUtils.readFloat(byteBuffer);
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/IntegerStatistics.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/IntegerStatistics.java
index 4ca9e81..6df16a8 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/IntegerStatistics.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/IntegerStatistics.java
@@ -18,14 +18,14 @@
  */
 package org.apache.iotdb.tsfile.file.metadata.statistics;
 
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.utils.BytesUtils;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
+import org.apache.iotdb.tsfile.exception.filter.StatisticsClassException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 /**
  * Statistics for int type.
@@ -36,7 +36,7 @@ public class IntegerStatistics extends Statistics<Integer> {
   private int maxValue;
   private int firstValue;
   private int lastValue;
-  private double sumValue;
+  private long sumValue;
 
   static final int INTEGER_STATISTICS_FIXED_RAM_SIZE = 64;
 
@@ -51,7 +51,7 @@ public class IntegerStatistics extends Statistics<Integer> {
     return 24;
   }
 
-  public void initializeStats(int min, int max, int first, int last, double sum) {
+  public void initializeStats(int min, int max, int first, int last, long sum) {
     this.minValue = min;
     this.maxValue = max;
     this.firstValue = first;
@@ -59,7 +59,7 @@ public class IntegerStatistics extends Statistics<Integer> {
     this.sumValue = sum;
   }
 
-  private void updateStats(int minValue, int maxValue, int lastValue, double sumValue) {
+  private void updateStats(int minValue, int maxValue, int lastValue, long sumValue) {
     if (minValue < this.minValue) {
       this.minValue = minValue;
     }
@@ -70,7 +70,8 @@ public class IntegerStatistics extends Statistics<Integer> {
     this.lastValue = lastValue;
   }
 
-  private void updateStats(int minValue, int maxValue, int firstValue, int lastValue, double sumValue, long startTime, long endTime) {
+  private void updateStats(int minValue, int maxValue, int firstValue, int lastValue,
+      long sumValue, long startTime, long endTime) {
     if (minValue < this.minValue) {
       this.minValue = minValue;
     }
@@ -102,7 +103,6 @@ public class IntegerStatistics extends Statistics<Integer> {
       isEmpty = false;
     } else {
       updateStats(value, value, value, value);
-      isEmpty = false;
     }
   }
 
@@ -139,7 +139,12 @@ public class IntegerStatistics extends Statistics<Integer> {
   }
 
   @Override
-  public double getSumValue() {
+  public double getSumDoubleValue() {
+    throw new StatisticsClassException("Integer statistics does not support: double sum");
+  }
+
+  @Override
+  public long getSumLongValue() {
     return sumValue;
   }
 
@@ -147,14 +152,13 @@ public class IntegerStatistics extends Statistics<Integer> {
   protected void mergeStatisticsValue(Statistics stats) {
     IntegerStatistics intStats = (IntegerStatistics) stats;
     if (isEmpty) {
-      initializeStats(intStats.getMinValue(), intStats.getMaxValue(), intStats.getFirstValue(), intStats.getLastValue(),
-          intStats.getSumValue());
+      initializeStats(intStats.getMinValue(), intStats.getMaxValue(), intStats.getFirstValue(),
+          intStats.getLastValue(), intStats.sumValue);
       isEmpty = false;
     } else {
-      updateStats(intStats.getMinValue(), intStats.getMaxValue(), intStats.getFirstValue(), intStats.getLastValue(),
-          intStats.getSumValue(), stats.getStartTime(), stats.getEndTime());
+      updateStats(intStats.getMinValue(), intStats.getMaxValue(), intStats.getFirstValue(),
+          intStats.getLastValue(), intStats.sumValue, stats.getStartTime(), stats.getEndTime());
     }
-
   }
 
   @Override
@@ -204,7 +208,7 @@ public class IntegerStatistics extends Statistics<Integer> {
 
   @Override
   public byte[] getSumValueBytes() {
-    return BytesUtils.doubleToBytes(sumValue);
+    return BytesUtils.longToBytes(sumValue);
   }
 
   @Override
@@ -219,26 +223,26 @@ public class IntegerStatistics extends Statistics<Integer> {
   }
 
   @Override
-  void deserialize(InputStream inputStream) throws IOException {
+  public void deserialize(InputStream inputStream) throws IOException {
     this.minValue = ReadWriteIOUtils.readInt(inputStream);
     this.maxValue = ReadWriteIOUtils.readInt(inputStream);
     this.firstValue = ReadWriteIOUtils.readInt(inputStream);
     this.lastValue = ReadWriteIOUtils.readInt(inputStream);
-    this.sumValue = ReadWriteIOUtils.readDouble(inputStream);
+    this.sumValue = ReadWriteIOUtils.readLong(inputStream);
   }
 
   @Override
-  void deserialize(ByteBuffer byteBuffer) {
+  public void deserialize(ByteBuffer byteBuffer) {
     this.minValue = ReadWriteIOUtils.readInt(byteBuffer);
     this.maxValue = ReadWriteIOUtils.readInt(byteBuffer);
     this.firstValue = ReadWriteIOUtils.readInt(byteBuffer);
     this.lastValue = ReadWriteIOUtils.readInt(byteBuffer);
-    this.sumValue = ReadWriteIOUtils.readDouble(byteBuffer);
+    this.sumValue = ReadWriteIOUtils.readLong(byteBuffer);
   }
 
   @Override
   public String toString() {
-    return super.toString() + " [minValue:" + minValue + ",maxValue:" + maxValue + ",firstValue:" + firstValue +
-        ",lastValue:" + lastValue + ",sumValue:" + sumValue + "]";
+    return super.toString() + " [minValue:" + minValue + ",maxValue:" + maxValue + ",firstValue:"
+        + firstValue + ",lastValue:" + lastValue + ",sumValue:" + sumValue + "]";
   }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/LongStatistics.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/LongStatistics.java
index e4958d3..7d35666 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/LongStatistics.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/LongStatistics.java
@@ -18,14 +18,14 @@
  */
 package org.apache.iotdb.tsfile.file.metadata.statistics;
 
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.utils.BytesUtils;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
+import org.apache.iotdb.tsfile.exception.filter.StatisticsClassException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 public class LongStatistics extends Statistics<Long> {
 
@@ -67,7 +67,8 @@ public class LongStatistics extends Statistics<Long> {
     this.lastValue = lastValue;
   }
 
-  private void updateStats(long minValue, long maxValue, long firstValue, long lastValue, double sumValue, long startTime, long endTime) {
+  private void updateStats(long minValue, long maxValue, long firstValue, long lastValue,
+      double sumValue, long startTime, long endTime) {
     if (minValue < this.minValue) {
       this.minValue = minValue;
     }
@@ -113,11 +114,16 @@ public class LongStatistics extends Statistics<Long> {
   }
 
   @Override
-  public double getSumValue() {
+  public double getSumDoubleValue() {
     return sumValue;
   }
 
   @Override
+  public long getSumLongValue() {
+    throw new StatisticsClassException("Long statistics does not support: long sum");
+  }
+
+  @Override
   void updateStats(long value) {
     if (isEmpty) {
       initializeStats(value, value, value, value, value);
@@ -154,11 +160,11 @@ public class LongStatistics extends Statistics<Long> {
     LongStatistics longStats = (LongStatistics) stats;
     if (isEmpty) {
       initializeStats(longStats.getMinValue(), longStats.getMaxValue(), longStats.getFirstValue(),
-          longStats.getLastValue(), longStats.getSumValue());
+          longStats.getLastValue(), longStats.sumValue);
       isEmpty = false;
     } else {
-      updateStats(longStats.getMinValue(), longStats.getMaxValue(), longStats.getFirstValue(), longStats.getLastValue(),
-          longStats.getSumValue(), stats.getStartTime(), stats.getEndTime());
+      updateStats(longStats.getMinValue(), longStats.getMaxValue(), longStats.getFirstValue(),
+          longStats.getLastValue(), longStats.sumValue, stats.getStartTime(), stats.getEndTime());
     }
 
   }
@@ -225,7 +231,7 @@ public class LongStatistics extends Statistics<Long> {
   }
 
   @Override
-  void deserialize(InputStream inputStream) throws IOException {
+  public void deserialize(InputStream inputStream) throws IOException {
     this.minValue = ReadWriteIOUtils.readLong(inputStream);
     this.maxValue = ReadWriteIOUtils.readLong(inputStream);
     this.firstValue = ReadWriteIOUtils.readLong(inputStream);
@@ -234,7 +240,7 @@ public class LongStatistics extends Statistics<Long> {
   }
 
   @Override
-  void deserialize(ByteBuffer byteBuffer) {
+  public void deserialize(ByteBuffer byteBuffer) {
     this.minValue = ReadWriteIOUtils.readLong(byteBuffer);
     this.maxValue = ReadWriteIOUtils.readLong(byteBuffer);
     this.firstValue = ReadWriteIOUtils.readLong(byteBuffer);
@@ -244,7 +250,8 @@ public class LongStatistics extends Statistics<Long> {
 
   @Override
   public String toString() {
-    return super.toString() + " [minValue:" + minValue + ",maxValue:" + maxValue + ",firstValue:" + firstValue +
+    return super.toString() + " [minValue:" + minValue + ",maxValue:" + maxValue + ",firstValue:"
+        + firstValue +
         ",lastValue:" + lastValue + ",sumValue:" + sumValue + "]";
   }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/Statistics.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/Statistics.java
index 7a31494..a62fca6 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/Statistics.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/Statistics.java
@@ -28,6 +28,7 @@ import org.apache.iotdb.tsfile.exception.filter.StatisticsClassException;
 import org.apache.iotdb.tsfile.exception.write.UnknownColumnTypeException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -50,7 +51,7 @@ public abstract class Statistics<T> {
   /**
    * number of time-value points
    */
-  private long count = 0;
+  private int count = 0;
 
   private long startTime = Long.MAX_VALUE;
   private long endTime = Long.MIN_VALUE;
@@ -102,7 +103,8 @@ public abstract class Statistics<T> {
   public abstract TSDataType getType();
 
   public int getSerializedSize() {
-    return 24 // count, startTime, endTime
+    return ReadWriteForEncodingUtils.uVarIntSize(count) // count
+        + 16 // startTime, endTime
         + getStatsSize();
   }
 
@@ -110,7 +112,7 @@ public abstract class Statistics<T> {
 
   public int serialize(OutputStream outputStream) throws IOException {
     int byteLen = 0;
-    byteLen += ReadWriteIOUtils.write(count, outputStream);
+    byteLen += ReadWriteForEncodingUtils.writeUnsignedVarInt(count, outputStream);
     byteLen += ReadWriteIOUtils.write(startTime, outputStream);
     byteLen += ReadWriteIOUtils.write(endTime, outputStream);
     // value statistics of different data type
@@ -123,9 +125,9 @@ public abstract class Statistics<T> {
   /**
    * read data from the inputStream.
    */
-  abstract void deserialize(InputStream inputStream) throws IOException;
+  public abstract void deserialize(InputStream inputStream) throws IOException;
 
-  abstract void deserialize(ByteBuffer byteBuffer);
+  public abstract void deserialize(ByteBuffer byteBuffer);
 
   public abstract void setMinMaxFromBytes(byte[] minBytes, byte[] maxBytes);
 
@@ -137,7 +139,9 @@ public abstract class Statistics<T> {
 
   public abstract T getLastValue();
 
-  public abstract double getSumValue();
+  public abstract double getSumDoubleValue();
+
+  public abstract long getSumLongValue();
 
   public abstract byte[] getMinValueBytes();
 
@@ -389,7 +393,7 @@ public abstract class Statistics<T> {
   public static Statistics deserialize(InputStream inputStream, TSDataType dataType)
       throws IOException {
     Statistics statistics = getStatsByType(dataType);
-    statistics.setCount(ReadWriteIOUtils.readLong(inputStream));
+    statistics.setCount(ReadWriteForEncodingUtils.readUnsignedVarInt(inputStream));
     statistics.setStartTime(ReadWriteIOUtils.readLong(inputStream));
     statistics.setEndTime(ReadWriteIOUtils.readLong(inputStream));
     statistics.deserialize(inputStream);
@@ -399,7 +403,7 @@ public abstract class Statistics<T> {
 
   public static Statistics deserialize(ByteBuffer buffer, TSDataType dataType) {
     Statistics statistics = getStatsByType(dataType);
-    statistics.setCount(ReadWriteIOUtils.readLong(buffer));
+    statistics.setCount(ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
     statistics.setStartTime(ReadWriteIOUtils.readLong(buffer));
     statistics.setEndTime(ReadWriteIOUtils.readLong(buffer));
     statistics.deserialize(buffer);
@@ -427,7 +431,7 @@ public abstract class Statistics<T> {
     this.endTime = endTime;
   }
 
-  public void setCount(long count) {
+  public void setCount(int count) {
     this.count = count;
   }
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
index d13c631..a13ac62 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
@@ -38,8 +38,9 @@ import java.util.stream.Collectors;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.compress.IUnCompressor;
+import org.apache.iotdb.tsfile.encoding.decoder.Decoder;
 import org.apache.iotdb.tsfile.file.MetaMarker;
-import org.apache.iotdb.tsfile.file.footer.ChunkGroupFooter;
+import org.apache.iotdb.tsfile.file.header.ChunkGroupHeader;
 import org.apache.iotdb.tsfile.file.header.ChunkHeader;
 import org.apache.iotdb.tsfile.file.header.PageHeader;
 import org.apache.iotdb.tsfile.file.metadata.ChunkGroupMetadata;
@@ -51,12 +52,15 @@ import org.apache.iotdb.tsfile.file.metadata.TsFileMetadata;
 import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
 import org.apache.iotdb.tsfile.file.metadata.enums.MetadataIndexNodeType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
 import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.read.common.BatchData;
 import org.apache.iotdb.tsfile.read.common.Chunk;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.read.controller.MetadataQuerierByFileImpl;
 import org.apache.iotdb.tsfile.read.reader.TsFileInput;
+import org.apache.iotdb.tsfile.read.reader.page.PageReader;
 import org.apache.iotdb.tsfile.utils.BloomFilter;
 import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
@@ -72,11 +76,10 @@ public class TsFileSequenceReader implements AutoCloseable {
   private static final String METADATA_INDEX_NODE_DESERIALIZE_ERROR = "Something error happened while deserializing MetadataIndexNode of file {}";
   protected String file;
   protected TsFileInput tsFileInput;
-  private long fileMetadataPos;
-  private int fileMetadataSize;
+  protected long fileMetadataPos;
+  protected int fileMetadataSize;
   private ByteBuffer markerBuffer = ByteBuffer.allocate(Byte.BYTES);
-  private int totalChunkNum;
-  private TsFileMetadata tsFileMetaData;
+  protected TsFileMetadata tsFileMetaData;
   // device -> measurement -> TimeseriesMetadata
   private Map<String, Map<String, TimeseriesMetadata>> cachedDeviceMetadata = new ConcurrentHashMap<>();
   private static final ReadWriteLock cacheLock = new ReentrantReadWriteLock();
@@ -159,8 +162,8 @@ public class TsFileSequenceReader implements AutoCloseable {
   /**
    * construct function for TsFileSequenceReader.
    *
-   * @param input            the input of a tsfile. The current position should be a markder and
-   *                         then a chunk Header, rather than the magic number
+   * @param input            the input of a tsfile. The current position should be a marker and then
+   *                         a chunk Header, rather than the magic number
    * @param fileMetadataPos  the position of the file metadata in the TsFileInput from the beginning
    *                         of the input to the current position
    * @param fileMetadataSize the byte size of the file metadata in the input
@@ -208,10 +211,16 @@ public class TsFileSequenceReader implements AutoCloseable {
    * whether the file is a complete TsFile: only if the head magic and tail magic string exists.
    */
   public boolean isComplete() throws IOException {
-    return tsFileInput.size() >= TSFileConfig.MAGIC_STRING.getBytes().length * 2
-        + TSFileConfig.VERSION_NUMBER.getBytes().length
-        && (readTailMagic().equals(readHeadMagic()) || readTailMagic()
-        .equals(TSFileConfig.VERSION_NUMBER_V1));
+    long size = tsFileInput.size();
+    // TSFileConfig.MAGIC_STRING.getBytes().length * 2 for two magic string
+    // Byte.BYTES for the file version number
+    if (size >= TSFileConfig.MAGIC_STRING.getBytes().length * 2 + Byte.BYTES) {
+      String tailMagic = readTailMagic();
+      String headMagic = readHeadMagic();
+      return tailMagic.equals(headMagic);
+    } else {
+      return false;
+    }
   }
 
   /**
@@ -228,12 +237,11 @@ public class TsFileSequenceReader implements AutoCloseable {
   /**
    * this function reads version number and checks compatibility of TsFile.
    */
-  public String readVersionNumber() throws IOException {
-    ByteBuffer versionNumberBytes = ByteBuffer
-        .allocate(TSFileConfig.VERSION_NUMBER.getBytes().length);
-    tsFileInput.read(versionNumberBytes, TSFileConfig.MAGIC_STRING.getBytes().length);
-    versionNumberBytes.flip();
-    return new String(versionNumberBytes.array());
+  public byte readVersionNumber() throws IOException {
+    ByteBuffer versionNumberByte = ByteBuffer.allocate(Byte.BYTES);
+    tsFileInput.read(versionNumberByte, TSFileConfig.MAGIC_STRING.getBytes().length);
+    versionNumberByte.flip();
+    return versionNumberByte.get();
   }
 
   /**
@@ -314,7 +322,7 @@ public class TsFileSequenceReader implements AutoCloseable {
     readFileMetadata();
     MetadataIndexNode deviceMetadataIndexNode = tsFileMetaData.getMetadataIndex();
     Pair<MetadataIndexEntry, Long> metadataIndexPair = getMetadataAndEndOffset(
-        deviceMetadataIndexNode, path.getDevice(), MetadataIndexNodeType.INTERNAL_DEVICE, true);
+        deviceMetadataIndexNode, path.getDevice(), true, true);
     if (metadataIndexPair == null) {
       return null;
     }
@@ -328,7 +336,7 @@ public class TsFileSequenceReader implements AutoCloseable {
         throw e;
       }
       metadataIndexPair = getMetadataAndEndOffset(metadataIndexNode,
-          path.getMeasurement(), MetadataIndexNodeType.INTERNAL_MEASUREMENT, false);
+          path.getMeasurement(), false, false);
     }
     if (metadataIndexPair == null) {
       return null;
@@ -359,7 +367,7 @@ public class TsFileSequenceReader implements AutoCloseable {
     readFileMetadata();
     MetadataIndexNode deviceMetadataIndexNode = tsFileMetaData.getMetadataIndex();
     Pair<MetadataIndexEntry, Long> metadataIndexPair = getMetadataAndEndOffset(
-        deviceMetadataIndexNode, path.getDevice(), MetadataIndexNodeType.INTERNAL_DEVICE, true);
+        deviceMetadataIndexNode, path.getDevice(), true, true);
     if (metadataIndexPair == null) {
       return null;
     }
@@ -373,7 +381,7 @@ public class TsFileSequenceReader implements AutoCloseable {
         throw e;
       }
       metadataIndexPair = getMetadataAndEndOffset(metadataIndexNode,
-          path.getMeasurement(), MetadataIndexNodeType.INTERNAL_MEASUREMENT, false);
+          path.getMeasurement(), false, false);
     }
     if (metadataIndexPair == null) {
       return null;
@@ -401,7 +409,7 @@ public class TsFileSequenceReader implements AutoCloseable {
     readFileMetadata();
     MetadataIndexNode deviceMetadataIndexNode = tsFileMetaData.getMetadataIndex();
     Pair<MetadataIndexEntry, Long> metadataIndexPair = getMetadataAndEndOffset(
-        deviceMetadataIndexNode, device, MetadataIndexNodeType.INTERNAL_DEVICE, false);
+        deviceMetadataIndexNode, device, true, false);
     if (metadataIndexPair == null) {
       return Collections.emptyList();
     }
@@ -424,7 +432,7 @@ public class TsFileSequenceReader implements AutoCloseable {
           throw e;
         }
         measurementMetadataIndexPair = getMetadataAndEndOffset(metadataIndexNode,
-            measurementList.get(i), MetadataIndexNodeType.INTERNAL_MEASUREMENT, false);
+            measurementList.get(i), false, false);
       }
       if (measurementMetadataIndexPair == null) {
         return Collections.emptyList();
@@ -502,8 +510,8 @@ public class TsFileSequenceReader implements AutoCloseable {
     }
   }
 
-  private int binarySearchInTimeseriesMetadataList(List<TimeseriesMetadata> timeseriesMetadataList,
-      String key) {
+  protected int binarySearchInTimeseriesMetadataList(
+      List<TimeseriesMetadata> timeseriesMetadataList, String key) {
     int low = 0;
     int high = timeseriesMetadataList.size() - 1;
 
@@ -533,26 +541,29 @@ public class TsFileSequenceReader implements AutoCloseable {
   private List<String> getAllDevices(MetadataIndexNode metadataIndexNode) throws IOException {
     List<String> deviceList = new ArrayList<>();
     int metadataIndexListSize = metadataIndexNode.getChildren().size();
-    if (metadataIndexNode.getNodeType().equals(MetadataIndexNodeType.INTERNAL_MEASUREMENT)) {
-      for (MetadataIndexEntry index : metadataIndexNode.getChildren()) {
-        deviceList.add(index.getName());
-      }
-    } else {
-      for (int i = 0; i < metadataIndexListSize; i++) {
-        long endOffset = metadataIndexNode.getEndOffset();
-        if (i != metadataIndexListSize - 1) {
-          endOffset = metadataIndexNode.getChildren().get(i + 1).getOffset();
-        }
-        ByteBuffer buffer = readData(metadataIndexNode.getChildren().get(i).getOffset(), endOffset);
-        MetadataIndexNode node = MetadataIndexNode.deserializeFrom(buffer);
-        if (node.getNodeType().equals(MetadataIndexNodeType.LEAF_DEVICE)) {
-          // if node in next level is LEAF_DEVICE, put all devices in node entry into the set
-          deviceList.addAll(node.getChildren().stream().map(MetadataIndexEntry::getName).collect(
+
+    // if metadataIndexNode is LEAF_DEVICE, put all devices in node entry into the list
+    if (metadataIndexNode.getNodeType().equals(MetadataIndexNodeType.LEAF_DEVICE)) {
+      deviceList
+          .addAll(metadataIndexNode.getChildren().stream().map(MetadataIndexEntry::getName).collect(
               Collectors.toList()));
-        } else {
-          // keep traversing
-          deviceList.addAll(getAllDevices(node));
-        }
+      return deviceList;
+    }
+
+    for (int i = 0; i < metadataIndexListSize; i++) {
+      long endOffset = metadataIndexNode.getEndOffset();
+      if (i != metadataIndexListSize - 1) {
+        endOffset = metadataIndexNode.getChildren().get(i + 1).getOffset();
+      }
+      ByteBuffer buffer = readData(metadataIndexNode.getChildren().get(i).getOffset(), endOffset);
+      MetadataIndexNode node = MetadataIndexNode.deserializeFrom(buffer);
+      if (node.getNodeType().equals(MetadataIndexNodeType.LEAF_DEVICE)) {
+        // if node in next level is LEAF_DEVICE, put all devices in node entry into the list
+        deviceList.addAll(node.getChildren().stream().map(MetadataIndexEntry::getName).collect(
+            Collectors.toList()));
+      } else {
+        // keep traversing
+        deviceList.addAll(getAllDevices(node));
       }
     }
     return deviceList;
@@ -567,13 +578,14 @@ public class TsFileSequenceReader implements AutoCloseable {
    */
   public Map<String, List<ChunkMetadata>> readChunkMetadataInDevice(String device)
       throws IOException {
-    if (tsFileMetaData == null) {
-      readFileMetadata();
-    }
+    readFileMetadata();
 
     long start = 0;
     int size = 0;
     List<TimeseriesMetadata> timeseriesMetadataMap = getDeviceTimeseriesMetadata(device);
+    if (timeseriesMetadataMap.isEmpty()) {
+      return new HashMap<>();
+    }
     for (TimeseriesMetadata timeseriesMetadata : timeseriesMetadataMap) {
       if (start == 0) {
         start = timeseriesMetadata.getOffsetOfChunkMetaDataList();
@@ -583,8 +595,15 @@ public class TsFileSequenceReader implements AutoCloseable {
     // read buffer of all ChunkMetadatas of this device
     ByteBuffer buffer = readData(start, size);
     Map<String, List<ChunkMetadata>> seriesMetadata = new HashMap<>();
+    int index = 0;
+    int curSize = timeseriesMetadataMap.get(index).getDataSizeOfChunkMetaDataList();
     while (buffer.hasRemaining()) {
-      ChunkMetadata chunkMetadata = ChunkMetadata.deserializeFrom(buffer);
+      if (buffer.position() >= curSize) {
+        index++;
+        curSize += timeseriesMetadataMap.get(index).getDataSizeOfChunkMetaDataList();
+      }
+      ChunkMetadata chunkMetadata = ChunkMetadata
+          .deserializeFrom(buffer, timeseriesMetadataMap.get(index));
       seriesMetadata.computeIfAbsent(chunkMetadata.getMeasurementUid(), key -> new ArrayList<>())
           .add(chunkMetadata);
     }
@@ -621,32 +640,27 @@ public class TsFileSequenceReader implements AutoCloseable {
       String deviceId, MetadataIndexNodeType type,
       Map<String, List<TimeseriesMetadata>> timeseriesMetadataMap) throws IOException {
     try {
-      switch (type) {
-        case INTERNAL_DEVICE:
-        case LEAF_DEVICE:
-        case INTERNAL_MEASUREMENT:
-          deviceId = metadataIndex.getName();
-          MetadataIndexNode metadataIndexNode = MetadataIndexNode.deserializeFrom(buffer);
-          int metadataIndexListSize = metadataIndexNode.getChildren().size();
-          for (int i = 0; i < metadataIndexListSize; i++) {
-            long endOffset = metadataIndexNode.getEndOffset();
-            if (i != metadataIndexListSize - 1) {
-              endOffset = metadataIndexNode.getChildren().get(i + 1).getOffset();
-            }
-            ByteBuffer nextBuffer = readData(metadataIndexNode.getChildren().get(i).getOffset(),
-                endOffset);
-            generateMetadataIndex(metadataIndexNode.getChildren().get(i), nextBuffer, deviceId,
-                metadataIndexNode.getNodeType(), timeseriesMetadataMap);
-          }
-          break;
-        case LEAF_MEASUREMENT:
-          List<TimeseriesMetadata> timeseriesMetadataList = new ArrayList<>();
-          while (buffer.hasRemaining()) {
-            timeseriesMetadataList.add(TimeseriesMetadata.deserializeFrom(buffer));
+      if (type.equals(MetadataIndexNodeType.LEAF_MEASUREMENT)) {
+        List<TimeseriesMetadata> timeseriesMetadataList = new ArrayList<>();
+        while (buffer.hasRemaining()) {
+          timeseriesMetadataList.add(TimeseriesMetadata.deserializeFrom(buffer));
+        }
+        timeseriesMetadataMap.computeIfAbsent(deviceId, k -> new ArrayList<>())
+            .addAll(timeseriesMetadataList);
+      } else {
+        deviceId = metadataIndex.getName();
+        MetadataIndexNode metadataIndexNode = MetadataIndexNode.deserializeFrom(buffer);
+        int metadataIndexListSize = metadataIndexNode.getChildren().size();
+        for (int i = 0; i < metadataIndexListSize; i++) {
+          long endOffset = metadataIndexNode.getEndOffset();
+          if (i != metadataIndexListSize - 1) {
+            endOffset = metadataIndexNode.getChildren().get(i + 1).getOffset();
           }
-          timeseriesMetadataMap.computeIfAbsent(deviceId, k -> new ArrayList<>())
-              .addAll(timeseriesMetadataList);
-          break;
+          ByteBuffer nextBuffer = readData(metadataIndexNode.getChildren().get(i).getOffset(),
+              endOffset);
+          generateMetadataIndex(metadataIndexNode.getChildren().get(i), nextBuffer, deviceId,
+              metadataIndexNode.getNodeType(), timeseriesMetadataMap);
+        }
       }
     } catch (BufferOverflowException e) {
       logger.error("Something error happened while generating MetadataIndex of file {}", file);
@@ -677,7 +691,7 @@ public class TsFileSequenceReader implements AutoCloseable {
   private List<TimeseriesMetadata> getDeviceTimeseriesMetadata(String device) throws IOException {
     MetadataIndexNode metadataIndexNode = tsFileMetaData.getMetadataIndex();
     Pair<MetadataIndexEntry, Long> metadataIndexPair = getMetadataAndEndOffset(
-        metadataIndexNode, device, MetadataIndexNodeType.INTERNAL_DEVICE, true);
+        metadataIndexNode, device, true, true);
     if (metadataIndexPair == null) {
       return Collections.emptyList();
     }
@@ -697,27 +711,27 @@ public class TsFileSequenceReader implements AutoCloseable {
    *
    * @param metadataIndex given MetadataIndexNode
    * @param name          target device / measurement name
-   * @param type          target MetadataIndexNodeType, either INTERNAL_DEVICE or
-   *                      INTERNAL_MEASUREMENT. When searching for a device node,  return when it is
-   *                      not INTERNAL_DEVICE. Likewise, when searching for a measurement node,
-   *                      return when it is not INTERNAL_MEASUREMENT. This works for the situation
-   *                      when the index tree does NOT have the device level and ONLY has the
-   *                      measurement level.
-   * @param exactSearch   if is in exact search mode, return null when there is no entry with name;
-   *                      or else return the nearest MetadataIndexEntry before it (for deeper
+   * @param isDeviceLevel whether target MetadataIndexNode is device level
+   * @param exactSearch   whether is in exact search mode, return null when there is no entry with
+   *                      name; or else return the nearest MetadataIndexEntry before it (for deeper
    *                      search)
    * @return target MetadataIndexEntry, endOffset pair
    */
-  private Pair<MetadataIndexEntry, Long> getMetadataAndEndOffset(MetadataIndexNode metadataIndex,
-      String name, MetadataIndexNodeType type, boolean exactSearch) throws IOException {
+  protected Pair<MetadataIndexEntry, Long> getMetadataAndEndOffset(MetadataIndexNode metadataIndex,
+      String name, boolean isDeviceLevel, boolean exactSearch) throws IOException {
     try {
-      if (!metadataIndex.getNodeType().equals(type)) {
+      // When searching for a device node, return when it is not INTERNAL_DEVICE
+      // When searching for a measurement node, return when it is not INTERNAL_MEASUREMENT
+      if ((isDeviceLevel && !metadataIndex.getNodeType().equals(
+          MetadataIndexNodeType.INTERNAL_DEVICE)) ||
+          (!isDeviceLevel && !metadataIndex.getNodeType().equals(
+              MetadataIndexNodeType.INTERNAL_MEASUREMENT))) {
         return metadataIndex.getChildIndexEntry(name, exactSearch);
       } else {
         Pair<MetadataIndexEntry, Long> childIndexEntry = metadataIndex
             .getChildIndexEntry(name, false);
         ByteBuffer buffer = readData(childIndexEntry.left.getOffset(), childIndexEntry.right);
-        return getMetadataAndEndOffset(MetadataIndexNode.deserializeFrom(buffer), name, type,
+        return getMetadataAndEndOffset(MetadataIndexNode.deserializeFrom(buffer), name, isDeviceLevel,
             false);
       }
     } catch (BufferOverflowException e) {
@@ -733,8 +747,8 @@ public class TsFileSequenceReader implements AutoCloseable {
    * @return a CHUNK_GROUP_FOOTER
    * @throws IOException io error
    */
-  public ChunkGroupFooter readChunkGroupFooter() throws IOException {
-    return ChunkGroupFooter.deserializeFrom(tsFileInput.wrapAsInputStream(), true);
+  public ChunkGroupHeader readChunkGroupHeader() throws IOException {
+    return ChunkGroupHeader.deserializeFrom(tsFileInput.wrapAsInputStream(), true);
   }
 
   /**
@@ -745,18 +759,9 @@ public class TsFileSequenceReader implements AutoCloseable {
    * @return a CHUNK_GROUP_FOOTER
    * @throws IOException io error
    */
-  public ChunkGroupFooter readChunkGroupFooter(long position, boolean markerRead)
+  public ChunkGroupHeader readChunkGroupHeader(long position, boolean markerRead)
       throws IOException {
-    return ChunkGroupFooter.deserializeFrom(tsFileInput, position, markerRead);
-  }
-
-  public long readVersion() throws IOException {
-    ByteBuffer buffer = ByteBuffer.allocate(Long.BYTES);
-    if (ReadWriteIOUtils.readAsPossible(tsFileInput, buffer) == 0) {
-      throw new IOException("reach the end of the file.");
-    }
-    buffer.flip();
-    return buffer.getLong();
+    return ChunkGroupHeader.deserializeFrom(tsFileInput, position, markerRead);
   }
 
   public void readPlanIndex() throws IOException {
@@ -781,8 +786,8 @@ public class TsFileSequenceReader implements AutoCloseable {
    * @return a CHUNK_HEADER
    * @throws IOException io error
    */
-  public ChunkHeader readChunkHeader() throws IOException {
-    return ChunkHeader.deserializeFrom(tsFileInput.wrapAsInputStream(), true);
+  public ChunkHeader readChunkHeader(byte chunkType) throws IOException {
+    return ChunkHeader.deserializeFrom(tsFileInput.wrapAsInputStream(), chunkType);
   }
 
   /**
@@ -790,11 +795,9 @@ public class TsFileSequenceReader implements AutoCloseable {
    *
    * @param position        the file offset of this chunk's header
    * @param chunkHeaderSize the size of chunk's header
-   * @param markerRead      true if the offset does not contains the marker , otherwise false
    */
-  private ChunkHeader readChunkHeader(long position, int chunkHeaderSize, boolean markerRead)
-      throws IOException {
-    return ChunkHeader.deserializeFrom(tsFileInput, position, chunkHeaderSize, markerRead);
+  private ChunkHeader readChunkHeader(long position, int chunkHeaderSize) throws IOException {
+    return ChunkHeader.deserializeFrom(tsFileInput, position, chunkHeaderSize);
   }
 
   /**
@@ -816,38 +819,10 @@ public class TsFileSequenceReader implements AutoCloseable {
    */
   public Chunk readMemChunk(ChunkMetadata metaData) throws IOException {
     int chunkHeadSize = ChunkHeader.getSerializedSize(metaData.getMeasurementUid());
-    ChunkHeader header = readChunkHeader(metaData.getOffsetOfChunkHeader(), chunkHeadSize, false);
+    ChunkHeader header = readChunkHeader(metaData.getOffsetOfChunkHeader(), chunkHeadSize);
     ByteBuffer buffer = readChunk(metaData.getOffsetOfChunkHeader() + header.getSerializedSize(),
         header.getDataSize());
-    return new Chunk(header, buffer, metaData.getDeleteIntervalList());
-  }
-
-  /**
-   * read all Chunks of given device.
-   * <p>
-   * note that this method loads all the chunks into memory, so it needs to be invoked carefully.
-   *
-   * @param device name
-   * @return measurement -> chunks list
-   */
-  public Map<String, List<Chunk>> readChunksInDevice(String device) throws IOException {
-    List<ChunkMetadata> chunkMetadataList = new ArrayList<>();
-    Map<String, List<ChunkMetadata>> chunkMetadataInDevice = readChunkMetadataInDevice(device);
-    for (List<ChunkMetadata> chunkMetadataListInDevice : chunkMetadataInDevice.values()) {
-      chunkMetadataList.addAll(chunkMetadataListInDevice);
-    }
-
-    Map<String, List<Chunk>> chunksInDevice = new HashMap<>();
-    chunkMetadataList.sort(Comparator.comparing(ChunkMetadata::getOffsetOfChunkHeader));
-    for (ChunkMetadata chunkMetadata : chunkMetadataList) {
-      Chunk chunk = readMemChunk(chunkMetadata);
-      String measurement = chunk.getHeader().getMeasurementID();
-      if (!chunksInDevice.containsKey(measurement)) {
-        chunksInDevice.put(measurement, new ArrayList<>());
-      }
-      chunksInDevice.get(measurement).add(chunk);
-    }
-    return chunksInDevice;
+    return new Chunk(header, buffer, metaData.getDeleteIntervalList(), metaData.getStatistics());
   }
 
   /**
@@ -855,8 +830,8 @@ public class TsFileSequenceReader implements AutoCloseable {
    *
    * @param type given tsfile data type
    */
-  public PageHeader readPageHeader(TSDataType type) throws IOException {
-    return PageHeader.deserializeFrom(tsFileInput.wrapAsInputStream(), type);
+  public PageHeader readPageHeader(TSDataType type, boolean hasStatistic) throws IOException {
+    return PageHeader.deserializeFrom(tsFileInput.wrapAsInputStream(), type, hasStatistic);
   }
 
   public long position() throws IOException {
@@ -872,12 +847,7 @@ public class TsFileSequenceReader implements AutoCloseable {
   }
 
   public ByteBuffer readPage(PageHeader header, CompressionType type) throws IOException {
-    return readPage(header, type, -1);
-  }
-
-  private ByteBuffer readPage(PageHeader header, CompressionType type, long position)
-      throws IOException {
-    ByteBuffer buffer = readData(position, header.getCompressedSize());
+    ByteBuffer buffer = readData(-1, header.getCompressedSize());
     IUnCompressor unCompressor = IUnCompressor.getUnCompressor(type);
     ByteBuffer uncompressedBuffer = ByteBuffer.allocate(header.getUncompressedSize());
     if (type == CompressionType.UNCOMPRESSED) {
@@ -927,7 +897,7 @@ public class TsFileSequenceReader implements AutoCloseable {
    * @param size     the size of data that want to read
    * @return data that been read.
    */
-  private ByteBuffer readData(long position, int size) throws IOException {
+  protected ByteBuffer readData(long position, int size) throws IOException {
     ByteBuffer buffer = ByteBuffer.allocate(size);
     if (position < 0) {
       if (ReadWriteIOUtils.readAsPossible(tsFileInput, buffer) != size) {
@@ -938,7 +908,7 @@ public class TsFileSequenceReader implements AutoCloseable {
       if (actualReadSize != size) {
         throw new IOException(
             String.format("reach the end of the data. Size of data that want to read: %s,"
-                + "actual read size: %s, posiotion: %s", size, actualReadSize, position));
+                + "actual read size: %s, position: %s", size, actualReadSize, position));
       }
     }
     buffer.flip();
@@ -954,7 +924,7 @@ public class TsFileSequenceReader implements AutoCloseable {
    * @param end   the end position of data that want to read
    * @return data that been read.
    */
-  private ByteBuffer readData(long start, long end) throws IOException {
+  protected ByteBuffer readData(long start, long end) throws IOException {
     return readData(start, (int) (end - start));
   }
 
@@ -992,16 +962,14 @@ public class TsFileSequenceReader implements AutoCloseable {
     long fileOffsetOfChunk;
 
     // ChunkMetadata of current ChunkGroup
-    List<ChunkMetadata> chunkMetadataList = null;
-    String deviceID;
+    List<ChunkMetadata> chunkMetadataList = new ArrayList<>();
 
-    int headerLength = TSFileConfig.MAGIC_STRING.getBytes().length + TSFileConfig.VERSION_NUMBER
-        .getBytes().length;
+    int headerLength = TSFileConfig.MAGIC_STRING.getBytes().length + Byte.BYTES;
     if (fileSize < headerLength) {
       return TsFileCheckStatus.INCOMPATIBLE_FILE;
     }
-    if (!TSFileConfig.MAGIC_STRING.equals(readHeadMagic()) || !TSFileConfig.VERSION_NUMBER
-        .equals(readVersionNumber())) {
+    if (!TSFileConfig.MAGIC_STRING.equals(readHeadMagic()) || (TSFileConfig.VERSION_NUMBER
+        != readVersionNumber())) {
       return TsFileCheckStatus.INCOMPATIBLE_FILE;
     }
 
@@ -1014,26 +982,21 @@ public class TsFileSequenceReader implements AutoCloseable {
         return TsFileCheckStatus.COMPLETE_FILE;
       }
     }
-    boolean newChunkGroup = true;
     // not a complete file, we will recover it...
     long truncatedSize = headerLength;
     byte marker;
-    int chunkCnt = 0;
+    String lastDeviceId = null;
     List<MeasurementSchema> measurementSchemaList = new ArrayList<>();
     try {
       while ((marker = this.readMarker()) != MetaMarker.SEPARATOR) {
         switch (marker) {
           case MetaMarker.CHUNK_HEADER:
-            // this is the first chunk of a new ChunkGroup.
-            if (newChunkGroup) {
-              newChunkGroup = false;
-              chunkMetadataList = new ArrayList<>();
-            }
+          case MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER:
             fileOffsetOfChunk = this.position() - 1;
             // if there is something wrong with a chunk, we will drop the whole ChunkGroup
             // as different chunks may be created by the same insertions(sqls), and partial
             // insertion is not tolerable
-            ChunkHeader chunkHeader = this.readChunkHeader();
+            ChunkHeader chunkHeader = this.readChunkHeader(marker);
             measurementID = chunkHeader.getMeasurementID();
             MeasurementSchema measurementSchema = new MeasurementSchema(measurementID,
                 chunkHeader.getDataType(),
@@ -1041,41 +1004,97 @@ public class TsFileSequenceReader implements AutoCloseable {
             measurementSchemaList.add(measurementSchema);
             dataType = chunkHeader.getDataType();
             Statistics<?> chunkStatistics = Statistics.getStatsByType(dataType);
-            for (int j = 0; j < chunkHeader.getNumOfPages(); j++) {
-              // a new Page
-              PageHeader pageHeader = this.readPageHeader(chunkHeader.getDataType());
-              chunkStatistics.mergeStatistics(pageHeader.getStatistics());
-              this.skipPageData(pageHeader);
+            int dataSize = chunkHeader.getDataSize();
+            if (chunkHeader.getChunkType() == MetaMarker.CHUNK_HEADER) {
+              while (dataSize > 0) {
+                // a new Page
+                PageHeader pageHeader = this.readPageHeader(chunkHeader.getDataType(), true);
+                chunkStatistics.mergeStatistics(pageHeader.getStatistics());
+                this.skipPageData(pageHeader);
+                dataSize -= pageHeader.getSerializedPageSize();
+                chunkHeader.increasePageNums(1);
+              }
+            } else {
+              // only one page without statistic, we need to iterate each point to generate statistic
+              PageHeader pageHeader = this.readPageHeader(chunkHeader.getDataType(), false);
+              Decoder valueDecoder = Decoder
+                  .getDecoderByType(chunkHeader.getEncodingType(), chunkHeader.getDataType());
+              ByteBuffer pageData = readPage(pageHeader, chunkHeader.getCompressionType());
+              Decoder timeDecoder = Decoder.getDecoderByType(
+                  TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder()),
+                  TSDataType.INT64);
+              PageReader reader = new PageReader(pageHeader, pageData, chunkHeader.getDataType(),
+                  valueDecoder, timeDecoder, null);
+              BatchData batchData = reader.getAllSatisfiedPageData();
+              while (batchData.hasCurrent()) {
+                switch (dataType) {
+                  case INT32:
+                    chunkStatistics.update(batchData.currentTime(), batchData.getInt());
+                    break;
+                  case INT64:
+                    chunkStatistics.update(batchData.currentTime(), batchData.getLong());
+                    break;
+                  case FLOAT:
+                    chunkStatistics.update(batchData.currentTime(), batchData.getFloat());
+                    break;
+                  case DOUBLE:
+                    chunkStatistics.update(batchData.currentTime(), batchData.getDouble());
+                    break;
+                  case BOOLEAN:
+                    chunkStatistics.update(batchData.currentTime(), batchData.getBoolean());
+                    break;
+                  case TEXT:
+                    chunkStatistics.update(batchData.currentTime(), batchData.getBinary());
+                    break;
+                  default:
+                    throw new IOException("Unexpected type " + dataType);
+                }
+                batchData.next();
+              }
+              chunkHeader.increasePageNums(1);
             }
             currentChunk = new ChunkMetadata(measurementID, dataType, fileOffsetOfChunk,
                 chunkStatistics);
             chunkMetadataList.add(currentChunk);
-            chunkCnt++;
             break;
-          case MetaMarker.CHUNK_GROUP_FOOTER:
-            // this is a chunk group
-            // if there is something wrong with the ChunkGroup Footer, we will drop this ChunkGroup
+          case MetaMarker.CHUNK_GROUP_HEADER:
+            // if there is something wrong with the ChunkGroup Header, we will drop this ChunkGroup
             // because we can not guarantee the correctness of the deviceId.
-            ChunkGroupFooter chunkGroupFooter = this.readChunkGroupFooter();
-            deviceID = chunkGroupFooter.getDeviceID();
-            if (newSchema != null) {
-              for (MeasurementSchema tsSchema : measurementSchemaList) {
-                newSchema.putIfAbsent(new Path(deviceID, tsSchema.getMeasurementId()), tsSchema);
+            truncatedSize = this.position() - 1;
+            if (lastDeviceId != null) {
+              // schema of last chunk group
+              if (newSchema != null) {
+                for (MeasurementSchema tsSchema : measurementSchemaList) {
+                  newSchema
+                      .putIfAbsent(new Path(lastDeviceId, tsSchema.getMeasurementId()), tsSchema);
+                }
               }
+              measurementSchemaList = new ArrayList<>();
+              // last chunk group Metadata
+              chunkGroupMetadataList.add(new ChunkGroupMetadata(lastDeviceId, chunkMetadataList));
             }
-            chunkGroupMetadataList.add(new ChunkGroupMetadata(deviceID, chunkMetadataList));
-            newChunkGroup = true;
-            truncatedSize = this.position();
-
-            totalChunkNum += chunkCnt;
-            chunkCnt = 0;
-            measurementSchemaList = new ArrayList<>();
-            break;
-          case MetaMarker.VERSION:
-            truncatedSize = this.position();
+            // this is a chunk group
+            chunkMetadataList = new ArrayList<>();
+            ChunkGroupHeader chunkGroupHeader = this.readChunkGroupHeader();
+            lastDeviceId = chunkGroupHeader.getDeviceID();
             break;
           case MetaMarker.OPERATION_INDEX_RANGE:
+            truncatedSize = this.position() - 1;
+            if (lastDeviceId != null) {
+              // schema of last chunk group
+              if (newSchema != null) {
+                for (MeasurementSchema tsSchema : measurementSchemaList) {
+                  newSchema
+                      .putIfAbsent(new Path(lastDeviceId, tsSchema.getMeasurementId()), tsSchema);
+                }
+              }
+              measurementSchemaList = new ArrayList<>();
+              // last chunk group Metadata
+              chunkGroupMetadataList.add(new ChunkGroupMetadata(lastDeviceId, chunkMetadataList));
+              lastDeviceId = null;
+            }
             readPlanIndex();
+            truncatedSize = this.position();
             break;
           default:
             // the disk file is corrupted, using this file may be dangerous
@@ -1084,6 +1103,17 @@ public class TsFileSequenceReader implements AutoCloseable {
       }
       // now we read the tail of the data section, so we are sure that the last
       // ChunkGroupFooter is complete.
+      if (lastDeviceId != null) {
+        // schema of last chunk group
+        if (newSchema != null) {
+          for (MeasurementSchema tsSchema : measurementSchemaList) {
+            newSchema
+                .putIfAbsent(new Path(lastDeviceId, tsSchema.getMeasurementId()), tsSchema);
+          }
+        }
+        // last chunk group Metadata
+        chunkGroupMetadataList.add(new ChunkGroupMetadata(lastDeviceId, chunkMetadataList));
+      }
       truncatedSize = this.position() - 1;
     } catch (Exception e) {
       logger.info("TsFile {} self-check cannot proceed at position {} " + "recovered, because : {}",
@@ -1094,10 +1124,6 @@ public class TsFileSequenceReader implements AutoCloseable {
     return truncatedSize;
   }
 
-  public int getTotalChunkNum() {
-    return totalChunkNum;
-  }
-
   /**
    * get ChunkMetaDatas of given path
    *
@@ -1129,7 +1155,7 @@ public class TsFileSequenceReader implements AutoCloseable {
 
       ByteBuffer buffer = readData(startOffsetOfChunkMetadataList, dataSizeOfChunkMetadataList);
       while (buffer.hasRemaining()) {
-        chunkMetadataList.add(ChunkMetadata.deserializeFrom(buffer));
+        chunkMetadataList.add(ChunkMetadata.deserializeFrom(buffer, timeseriesMetaData));
       }
 
       // minimize the storage of an ArrayList instance.
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/BatchData.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/BatchData.java
index d17a789..5c35d21 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/BatchData.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/BatchData.java
@@ -50,7 +50,7 @@ import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsLong;
  */
 public class BatchData {
 
-  public static final int CAPACITY_THRESHOLD = TSFileConfig.ARRAY_CAPACITY_THRESHOLD;
+  protected static final int CAPACITY_THRESHOLD = TSFileConfig.ARRAY_CAPACITY_THRESHOLD;
   protected int capacity = 16;
 
   protected TSDataType dataType;
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/Chunk.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/Chunk.java
index 1968aa9..8bb7a08 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/Chunk.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/Chunk.java
@@ -18,11 +18,15 @@
  */
 package org.apache.iotdb.tsfile.read.common;
 
+import java.io.IOException;
 import java.nio.ByteBuffer;
-
 import java.util.List;
 import org.apache.iotdb.tsfile.common.cache.Accountable;
+import org.apache.iotdb.tsfile.file.MetaMarker;
 import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
+import org.apache.iotdb.tsfile.utils.PublicBAOS;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
 
 /**
  * used in query.
@@ -30,7 +34,9 @@ import org.apache.iotdb.tsfile.file.header.ChunkHeader;
 public class Chunk implements Accountable {
 
   private ChunkHeader chunkHeader;
+  private Statistics chunkStatistic;
   private ByteBuffer chunkData;
+  private boolean isFromOldFile = false;
   /**
    * A list of deleted intervals.
    */
@@ -38,10 +44,11 @@ public class Chunk implements Accountable {
 
   private long ramSize;
 
-  public Chunk(ChunkHeader header, ByteBuffer buffer, List<TimeRange> deleteIntervalList) {
+  public Chunk(ChunkHeader header, ByteBuffer buffer, List<TimeRange> deleteIntervalList, Statistics chunkStatistic) {
     this.chunkHeader = header;
     this.chunkData = buffer;
     this.deleteIntervalList = deleteIntervalList;
+    this.chunkStatistic = chunkStatistic;
   }
 
   public ChunkHeader getHeader() {
@@ -60,12 +67,76 @@ public class Chunk implements Accountable {
     this.deleteIntervalList = list;
   }
 
-  public void mergeChunk(Chunk chunk) {
-    chunkHeader.mergeChunkHeader(chunk.chunkHeader);
-    ByteBuffer newChunkData = ByteBuffer
-        .allocate(chunkData.array().length + chunk.chunkData.array().length);
-    newChunkData.put(chunkData.array());
-    newChunkData.put(chunk.chunkData.array());
+  public void mergeChunk(Chunk chunk) throws IOException {
+    int dataSize = 0;
+    // from where the page data of the merged chunk starts, if -1, it means the merged chunk has more than one page
+    int offset1 = -1;
+    // if the merged chunk has only one page, after merge with current chunk ,it will have more than page
+    // so we should add page statistics for it
+    if (chunk.chunkHeader.getChunkType() == MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER) {
+      // read the uncompressedSize and compressedSize of this page
+      ReadWriteForEncodingUtils.readUnsignedVarInt(chunk.chunkData);
+      ReadWriteForEncodingUtils.readUnsignedVarInt(chunk.chunkData);
+      // record the position from which we can reuse
+      offset1 = chunk.chunkData.position();
+      chunk.chunkData.flip();
+      // the actual size should add another page statistics size
+      dataSize += (chunk.chunkData.array().length + chunk.chunkStatistic.getSerializedSize());
+    } else {
+      // if the merge chunk already has more than one page, we can reuse all the part of its data
+      // the dataSize is equal to the before
+      dataSize += chunk.chunkData.array().length;
+    }
+    // from where the page data of the current chunk starts, if -1, it means the current chunk has more than one page
+    int offset2 = -1;
+    // if the current chunk has only one page, after merge with the merged chunk ,it will have more than page
+    // so we should add page statistics for it
+    if (chunkHeader.getChunkType() == MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER) {
+      // change the chunk type
+      chunkHeader.setChunkType(MetaMarker.CHUNK_HEADER);
+      // read the uncompressedSize and compressedSize of this page
+      ReadWriteForEncodingUtils.readUnsignedVarInt(chunkData);
+      ReadWriteForEncodingUtils.readUnsignedVarInt(chunkData);
+      // record the position from which we can reuse
+      offset2 = chunkData.position();
+      chunkData.flip();
+      // the actual size should add another page statistics size
+      dataSize += (chunkData.array().length + chunkStatistic.getSerializedSize());
+    } else {
+      // if the current chunk already has more than one page, we can reuse all the part of its data
+      // the dataSize is equal to the before
+      dataSize += chunkData.array().length;
+    }
+    chunkHeader.setDataSize(dataSize);
+    ByteBuffer newChunkData = ByteBuffer.allocate(dataSize);
+    // the current chunk has more than one page, we can use its data part directly without any changes
+    if (offset2 == -1) {
+      newChunkData.put(chunkData.array());
+    } else { // the current chunk has only one page, we need to add one page statistics for it
+      byte[] b = chunkData.array();
+      // put the uncompressedSize and compressedSize of this page
+      newChunkData.put(b, 0, offset2);
+      // add page statistics
+      PublicBAOS a = new PublicBAOS();
+      chunkStatistic.serialize(a);
+      newChunkData.put(a.getBuf(), 0, a.size());
+      // put the remaining page data
+      newChunkData.put(b, offset2, b.length - offset2);
+    }
+    // the merged chunk has more than one page, we can use its data part directly without any changes
+    if (offset1 == -1) {
+      newChunkData.put(chunk.chunkData.array());
+    } else {
+      // put the uncompressedSize and compressedSize of this page
+      byte[] b = chunk.chunkData.array();
+      newChunkData.put(b, 0, offset1);
+      // add page statistics
+      PublicBAOS a = new PublicBAOS();
+      chunk.chunkStatistic.serialize(a);
+      newChunkData.put(a.getBuf(), 0, a.size());
+      // put the remaining page data
+      newChunkData.put(b, offset1, b.length - offset1);
+    }
     chunkData = newChunkData;
   }
 
@@ -78,4 +149,16 @@ public class Chunk implements Accountable {
   public long getRamSize() {
     return ramSize;
   }
+
+  public Statistics getChunkStatistic() {
+    return chunkStatistic;
+  }
+
+  public boolean isFromOldFile() {
+    return isFromOldFile;
+  }
+
+  public void setFromOldFile(boolean isFromOldFile) {
+    this.isFromOldFile = isFromOldFile;
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/CachedChunkLoaderImpl.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/CachedChunkLoaderImpl.java
index 9c47e70..deb51b5 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/CachedChunkLoaderImpl.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/CachedChunkLoaderImpl.java
@@ -61,7 +61,8 @@ public class CachedChunkLoaderImpl implements IChunkLoader {
   @Override
   public Chunk loadChunk(ChunkMetadata chunkMetaData) throws IOException {
     Chunk chunk = chunkCache.get(chunkMetaData);
-    return new Chunk(chunk.getHeader(), chunk.getData().duplicate(), chunk.getDeleteIntervalList());
+    return new Chunk(chunk.getHeader(), chunk.getData().duplicate(), chunk.getDeleteIntervalList(),
+        chunkMetaData.getStatistics());
   }
 
   @Override
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/LocalTsFileInput.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/LocalTsFileInput.java
index de314b1..b2a228c 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/LocalTsFileInput.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/LocalTsFileInput.java
@@ -25,6 +25,7 @@ import java.nio.channels.Channels;
 import java.nio.channels.FileChannel;
 import java.nio.file.Path;
 import java.nio.file.StandardOpenOption;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -125,4 +126,23 @@ public class LocalTsFileInput implements TsFileInput {
   public int readInt() {
     throw new UnsupportedOperationException();
   }
+
+  @Override
+  public String readVarIntString(long offset) throws IOException {
+    ByteBuffer byteBuffer = ByteBuffer.allocate(5);
+    channel.read(byteBuffer, offset);
+    byteBuffer.flip();
+    int strLength = ReadWriteForEncodingUtils.readVarInt(byteBuffer);
+    if (strLength < 0) {
+      return null;
+    } else if (strLength == 0) {
+      return "";
+    }
+    ByteBuffer strBuffer = ByteBuffer.allocate(strLength);
+    int varIntLength = ReadWriteForEncodingUtils.varIntSize(strLength);
+    byte[] bytes = new byte[strLength];
+    channel.read(strBuffer, offset + varIntLength);
+    strBuffer.get(bytes, 0, strLength);
+    return new String(bytes, 0, strLength);
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/TsFileInput.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/TsFileInput.java
index b948e0c..e215d3c 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/TsFileInput.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/TsFileInput.java
@@ -143,4 +143,9 @@ public interface TsFileInput {
    * read 4 bytes from the Input and convert it to a integer.
    */
   int readInt() throws IOException;
+
+  /**
+   * read a string from the Input at the given position
+   */
+  String readVarIntString(long offset) throws IOException;
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java
index 57549a4..35b9475 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java
@@ -19,41 +19,41 @@
 
 package org.apache.iotdb.tsfile.read.reader.chunk;
 
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.LinkedList;
+import java.util.List;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.compress.IUnCompressor;
 import org.apache.iotdb.tsfile.encoding.decoder.Decoder;
+import org.apache.iotdb.tsfile.file.MetaMarker;
 import org.apache.iotdb.tsfile.file.header.ChunkHeader;
 import org.apache.iotdb.tsfile.file.header.PageHeader;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
 import org.apache.iotdb.tsfile.read.common.BatchData;
 import org.apache.iotdb.tsfile.read.common.Chunk;
 import org.apache.iotdb.tsfile.read.common.TimeRange;
-import org.apache.iotdb.tsfile.read.reader.IPageReader;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.read.reader.IChunkReader;
+import org.apache.iotdb.tsfile.read.reader.IPageReader;
 import org.apache.iotdb.tsfile.read.reader.page.PageReader;
-import org.apache.iotdb.tsfile.v1.file.utils.HeaderUtils;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.LinkedList;
-import java.util.List;
+import org.apache.iotdb.tsfile.v2.file.header.PageHeaderV2;
+import org.apache.iotdb.tsfile.v2.read.reader.page.PageReaderV2;
 
 public class ChunkReader implements IChunkReader {
 
   private ChunkHeader chunkHeader;
   private ByteBuffer chunkDataBuffer;
   private IUnCompressor unCompressor;
-  private Decoder timeDecoder = Decoder.getDecoderByType(
+  private final Decoder timeDecoder = Decoder.getDecoderByType(
       TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder()),
       TSDataType.INT64);
 
   protected Filter filter;
 
   private List<IPageReader> pageReaderList = new LinkedList<>();
-  
-  private boolean isFromOldTsFile = false;
 
   /**
    * A list of deleted intervals.
@@ -72,28 +72,24 @@ public class ChunkReader implements IChunkReader {
     this.deleteIntervalList = chunk.getDeleteIntervalList();
     chunkHeader = chunk.getHeader();
     this.unCompressor = IUnCompressor.getUnCompressor(chunkHeader.getCompressionType());
-
-
-    initAllPageReaders();
-  }
-
-  public ChunkReader(Chunk chunk, Filter filter, boolean isFromOldFile) throws IOException {
-    this.filter = filter;
-    this.chunkDataBuffer = chunk.getData();
-    this.deleteIntervalList = chunk.getDeleteIntervalList();
-    chunkHeader = chunk.getHeader();
-    this.unCompressor = IUnCompressor.getUnCompressor(chunkHeader.getCompressionType());
-    this.isFromOldTsFile = isFromOldFile;
-
-    initAllPageReaders();
+    if (chunk.isFromOldFile()) {
+      initAllPageReadersV2();
+    }
+    else {
+      initAllPageReaders(chunk.getChunkStatistic());
+    }
   }
 
-  private void initAllPageReaders() throws IOException {
+  private void initAllPageReaders(Statistics chunkStatistic) throws IOException {
     // construct next satisfied page header
     while (chunkDataBuffer.remaining() > 0) {
       // deserialize a PageHeader from chunkDataBuffer
-      PageHeader pageHeader = isFromOldTsFile ? HeaderUtils.deserializePageHeaderV1(chunkDataBuffer, chunkHeader.getDataType()) :
-          PageHeader.deserializeFrom(chunkDataBuffer, chunkHeader.getDataType());
+      PageHeader pageHeader;
+      if (chunkHeader.getChunkType() == MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER) {
+        pageHeader = PageHeader.deserializeFrom(chunkDataBuffer, chunkStatistic);
+      } else {
+        pageHeader = PageHeader.deserializeFrom(chunkDataBuffer, chunkHeader.getDataType());
+      }
       // if the current page satisfies
       if (pageSatisfied(pageHeader)) {
         pageReaderList.add(constructPageReaderForNextPage(pageHeader));
@@ -104,7 +100,6 @@ public class ChunkReader implements IChunkReader {
   }
 
 
-
   /**
    * judge if has next page whose page header satisfies the filter.
    */
@@ -158,10 +153,18 @@ public class ChunkReader implements IChunkReader {
 
     chunkDataBuffer.get(compressedPageBody);
     Decoder valueDecoder = Decoder
-            .getDecoderByType(chunkHeader.getEncodingType(), chunkHeader.getDataType());
+        .getDecoderByType(chunkHeader.getEncodingType(), chunkHeader.getDataType());
     byte[] uncompressedPageData = new byte[pageHeader.getUncompressedSize()];
-    unCompressor.uncompress(compressedPageBody,0, compressedPageBodyLength,
-        uncompressedPageData, 0);
+    try {
+      unCompressor.uncompress(compressedPageBody, 0, compressedPageBodyLength,
+          uncompressedPageData, 0);
+    } catch (Exception e) {
+      throw new IOException("Uncompress error! uncompress size: " + pageHeader.getUncompressedSize() +
+          "compressed size: " + pageHeader.getCompressedSize() +
+          "page header: " + pageHeader +
+          e.getMessage());
+    }
+
     ByteBuffer pageData = ByteBuffer.wrap(uncompressedPageData);
     PageReader reader = new PageReader(pageHeader, pageData, chunkHeader.getDataType(),
         valueDecoder, timeDecoder, filter);
@@ -181,4 +184,44 @@ public class ChunkReader implements IChunkReader {
   public List<IPageReader> loadPageReaderList() {
     return pageReaderList;
   }
+
+  // For reading TsFile V2
+  private void initAllPageReadersV2() throws IOException {
+    // construct next satisfied page header
+    while (chunkDataBuffer.remaining() > 0) {
+      // deserialize a PageHeader from chunkDataBuffer
+      PageHeader pageHeader = PageHeaderV2.deserializeFrom(chunkDataBuffer, chunkHeader.getDataType());
+      // if the current page satisfies
+      if (pageSatisfied(pageHeader)) {
+        pageReaderList.add(constructPageReaderForNextPageV2(pageHeader));
+      } else {
+        skipBytesInStreamByLength(pageHeader.getCompressedSize());
+      }
+    }
+  }
+
+  //For reading TsFile V2
+  private PageReader constructPageReaderForNextPageV2(PageHeader pageHeader)
+      throws IOException {
+    int compressedPageBodyLength = pageHeader.getCompressedSize();
+    byte[] compressedPageBody = new byte[compressedPageBodyLength];
+
+    // doesn't has a complete page body
+    if (compressedPageBodyLength > chunkDataBuffer.remaining()) {
+      throw new IOException("do not has a complete page body. Expected:" + compressedPageBodyLength
+          + ". Actual:" + chunkDataBuffer.remaining());
+    }
+
+    chunkDataBuffer.get(compressedPageBody);
+    Decoder valueDecoder = Decoder
+            .getDecoderByType(chunkHeader.getEncodingType(), chunkHeader.getDataType());
+    byte[] uncompressedPageData = new byte[pageHeader.getUncompressedSize()];
+    unCompressor.uncompress(compressedPageBody,0, compressedPageBodyLength,
+        uncompressedPageData, 0);
+    ByteBuffer pageData = ByteBuffer.wrap(uncompressedPageData);
+    PageReader reader = new PageReaderV2(pageHeader, pageData, chunkHeader.getDataType(),
+        valueDecoder, timeDecoder, filter);
+    reader.setDeleteIntervalList(deleteIntervalList);
+    return reader;
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/PageReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/PageReader.java
index 45013ca..2cac2ce 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/PageReader.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/PageReader.java
@@ -40,29 +40,29 @@ public class PageReader implements IPageReader {
 
   private PageHeader pageHeader;
 
-  private TSDataType dataType;
+  protected TSDataType dataType;
 
   /**
    * decoder for value column
    */
-  private Decoder valueDecoder;
+  protected Decoder valueDecoder;
 
   /**
    * decoder for time column
    */
-  private Decoder timeDecoder;
+  protected Decoder timeDecoder;
 
   /**
    * time column in memory
    */
-  private ByteBuffer timeBuffer;
+  protected ByteBuffer timeBuffer;
 
   /**
    * value column in memory
    */
-  private ByteBuffer valueBuffer;
+  protected ByteBuffer valueBuffer;
 
-  private Filter filter;
+  protected Filter filter;
 
   /**
    * A list of deleted intervals.
@@ -183,7 +183,7 @@ public class PageReader implements IPageReader {
     return pageHeader.isModified();
   }
 
-  private boolean isDeleted(long timestamp) {
+  protected boolean isDeleted(long timestamp) {
     while (deleteIntervalList != null && deleteCursor < deleteIntervalList.size()) {
       if (deleteIntervalList.get(deleteCursor).contains(timestamp)) {
         return true;
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteForEncodingUtils.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteForEncodingUtils.java
index 0b1f8b0..63b3372 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteForEncodingUtils.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteForEncodingUtils.java
@@ -98,14 +98,24 @@ public class ReadWriteForEncodingUtils {
   public static int readUnsignedVarInt(InputStream in) throws IOException {
     int value = 0;
     int i = 0;
-    int b;
-    while (((b = in.read()) & 0x80) != 0) {
+    int b = in.read();
+    while (b != -1 && (b & 0x80) != 0) {
       value |= (b & 0x7F) << i;
       i += 7;
+      b = in.read();
     }
     return value | (b << i);
   }
 
+  public static int readVarInt(InputStream in) throws IOException {
+    int value = readUnsignedVarInt(in);
+    int x = value >>> 1;
+    if ((value & 1) != 0) {
+      x = ~x;
+    }
+    return x;
+  }
+
   /**
    * read an unsigned var int in stream and transform it to int format.
    *
@@ -123,6 +133,15 @@ public class ReadWriteForEncodingUtils {
     return value | (b << i);
   }
 
+  public static int readVarInt(ByteBuffer buffer) {
+    int value = readUnsignedVarInt(buffer);
+    int x = value >>> 1;
+    if ((value & 1) != 0) {
+      x = ~x;
+    }
+    return x;
+  }
+
   /**
    * write a value to stream using unsigned var int format. for example, int
    * 123456789 has its binary format 00000111-01011011-11001101-00010101 (if we
@@ -133,13 +152,44 @@ public class ReadWriteForEncodingUtils {
    *
    * @param value value to write into stream
    * @param out   output stream
+   * @return the number of bytes that the value consume.
    */
-  public static void writeUnsignedVarInt(int value, ByteArrayOutputStream out) {
+  public static int writeUnsignedVarInt(int value, ByteArrayOutputStream out) {
+    int position = 1;
+    while ((value & 0xFFFFFF80) != 0L) {
+      out.write((value & 0x7F) | 0x80);
+      value >>>= 7;
+      position++;
+    }
+    out.write(value & 0x7F);
+    return position;
+  }
+
+  public static int writeVarInt(int value, ByteArrayOutputStream out) {
+    int uValue = value << 1;
+    if (value < 0) {
+      uValue = ~uValue;
+    }
+    return writeUnsignedVarInt(uValue, out);
+  }
+
+  public static int writeUnsignedVarInt(int value, OutputStream out) throws IOException {
+    int position = 1;
     while ((value & 0xFFFFFF80) != 0L) {
       out.write((value & 0x7F) | 0x80);
       value >>>= 7;
+      position++;
     }
     out.write(value & 0x7F);
+    return position;
+  }
+
+  public static int writeVarInt(int value, OutputStream out) throws IOException {
+    int uValue = value << 1;
+    if (value < 0) {
+      uValue = ~uValue;
+    }
+    return writeUnsignedVarInt(uValue, out);
   }
 
   /**
@@ -167,6 +217,46 @@ public class ReadWriteForEncodingUtils {
     return position;
   }
 
+  public static int writeVarInt(int value, ByteBuffer buffer) {
+    int uValue = value << 1;
+    if (value < 0) {
+      uValue = ~uValue;
+    }
+    return writeUnsignedVarInt(uValue, buffer);
+  }
+
+  /**
+   * Returns the encoding size in bytes of its input value.
+   * @param value the integer to be measured
+   * @return the encoding size in bytes of its input value
+   */
+  public static int varIntSize(int value) {
+    int uValue = value << 1;
+    if (value < 0) {
+      uValue = ~uValue;
+    }
+    int position = 1;
+    while ((uValue & 0xFFFFFF80) != 0L) {
+      uValue >>>= 7;
+      position++;
+    }
+    return position;
+  }
+
+  /**
+   * Returns the encoding size in bytes of its input value.
+   * @param value the unsigned integer to be measured
+   * @return the encoding size in bytes of its input value
+   */
+  public static int uVarIntSize(int value) {
+    int position = 1;
+    while ((value & 0xFFFFFF80) != 0L) {
+      value >>>= 7;
+      position++;
+    }
+    return position;
+  }
+
   /**
    * write integer value using special bit to output stream.
    *
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java
index d4b7570..3858f19 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java
@@ -353,6 +353,25 @@ public class ReadWriteIOUtils {
   }
 
   /**
+   * write string to outputStream.
+   *
+   * @return the length of string represented by byte[].
+   */
+  public static int writeVar(String s, OutputStream outputStream) throws IOException {
+    int len = 0;
+    if (s == null) {
+      len += ReadWriteForEncodingUtils.writeVarInt(-1, outputStream);
+      return len;
+    }
+
+    byte[] bytes = s.getBytes(TSFileConfig.STRING_CHARSET);
+    len += ReadWriteForEncodingUtils.writeVarInt(bytes.length, outputStream);
+    outputStream.write(bytes);
+    len += bytes.length;
+    return len;
+  }
+
+  /**
    * write string to byteBuffer.
    *
    * @return the length of string represented by byte[].
@@ -369,6 +388,18 @@ public class ReadWriteIOUtils {
     return len;
   }
 
+  public static int writeVar(String s, ByteBuffer buffer) {
+    if (s == null) {
+      return ReadWriteForEncodingUtils.writeVarInt(-1, buffer);
+    }
+    int len = 0;
+    byte[] bytes = s.getBytes();
+    len += ReadWriteForEncodingUtils.writeVarInt(bytes.length, buffer);
+    buffer.put(bytes);
+    len += bytes.length;
+    return len;
+  }
+
   /**
    * write byteBuffer.capacity and byteBuffer.array to outputStream.
    */
@@ -404,7 +435,7 @@ public class ReadWriteIOUtils {
    */
   public static int write(CompressionType compressionType, OutputStream outputStream)
       throws IOException {
-    short n = compressionType.serialize();
+    byte n = compressionType.serialize();
     return write(n, outputStream);
   }
 
@@ -412,7 +443,7 @@ public class ReadWriteIOUtils {
    * write compressionType to byteBuffer.
    */
   public static int write(CompressionType compressionType, ByteBuffer buffer) {
-    short n = compressionType.serialize();
+    byte n = compressionType.serialize();
     return write(n, buffer);
   }
 
@@ -420,12 +451,12 @@ public class ReadWriteIOUtils {
    * TSDataType.
    */
   public static int write(TSDataType dataType, OutputStream outputStream) throws IOException {
-    short n = dataType.serialize();
+    byte n = dataType.serialize();
     return write(n, outputStream);
   }
 
   public static int write(TSDataType dataType, ByteBuffer buffer) {
-    short n = dataType.serialize();
+    byte n = dataType.serialize();
     return write(n, buffer);
   }
 
@@ -433,16 +464,23 @@ public class ReadWriteIOUtils {
    * TSEncoding.
    */
   public static int write(TSEncoding encoding, OutputStream outputStream) throws IOException {
-    short n = encoding.serialize();
+    byte n = encoding.serialize();
     return write(n, outputStream);
   }
 
   public static int write(TSEncoding encoding, ByteBuffer buffer) {
-    short n = encoding.serialize();
+    byte n = encoding.serialize();
     return write(n, buffer);
   }
 
   /**
+   * read a byte var from inputStream.
+   */
+  public static byte readByte(InputStream inputStream) throws IOException {
+    return (byte) inputStream.read();
+  }
+
+  /**
    * read a short var from inputStream.
    */
   public static short readShort(InputStream inputStream) throws IOException {
@@ -576,6 +614,23 @@ public class ReadWriteIOUtils {
   }
 
   /**
+   * string length's type is varInt
+   */
+  public static String readVarIntString(InputStream inputStream) throws IOException {
+    int strLength = ReadWriteForEncodingUtils.readVarInt(inputStream);
+    if (strLength <= 0) {
+      return null;
+    }
+    byte[] bytes = new byte[strLength];
+    int readLen = inputStream.read(bytes, 0, strLength);
+    if (readLen != strLength) {
+      throw new IOException(String.format(RETURN_ERROR,
+          strLength, readLen));
+    }
+    return new String(bytes, 0, strLength);
+  }
+
+  /**
    * read string from byteBuffer.
    */
   public static String readString(ByteBuffer buffer) {
@@ -591,6 +646,21 @@ public class ReadWriteIOUtils {
   }
 
   /**
+   * string length's type is varInt
+   */
+  public static String readVarIntString(ByteBuffer buffer) {
+    int strLength = ReadWriteForEncodingUtils.readVarInt(buffer);
+    if (strLength < 0) {
+      return null;
+    } else if (strLength == 0) {
+      return "";
+    }
+    byte[] bytes = new byte[strLength];
+    buffer.get(bytes, 0, strLength);
+    return new String(bytes, 0, strLength);
+  }
+
+  /**
    * read string from byteBuffer with user define length.
    */
   public static String readStringWithLength(ByteBuffer buffer, int length) {
@@ -695,14 +765,11 @@ public class ReadWriteIOUtils {
    * <p>
    * read a int + buffer
    */
-  public static ByteBuffer readByteBufferWithSelfDescriptionLength(ByteBuffer buffer) {
-    int byteLength = readInt(buffer);
+  public static byte[] readByteBufferWithSelfDescriptionLength(ByteBuffer buffer) {
+    int byteLength = ReadWriteForEncodingUtils.readUnsignedVarInt(buffer);
     byte[] bytes = new byte[byteLength];
     buffer.get(bytes);
-    ByteBuffer byteBuffer = ByteBuffer.allocate(byteLength);
-    byteBuffer.put(bytes);
-    byteBuffer.flip();
-    return byteBuffer;
+    return bytes;
   }
 
   /**
@@ -783,32 +850,32 @@ public class ReadWriteIOUtils {
   }
 
   public static CompressionType readCompressionType(InputStream inputStream) throws IOException {
-    short n = readShort(inputStream);
+    byte n = readByte(inputStream);
     return CompressionType.deserialize(n);
   }
 
   public static CompressionType readCompressionType(ByteBuffer buffer) {
-    short n = readShort(buffer);
+    byte n = buffer.get();
     return CompressionType.deserialize(n);
   }
 
   public static TSDataType readDataType(InputStream inputStream) throws IOException {
-    short n = readShort(inputStream);
+    byte n = readByte(inputStream);
     return TSDataType.deserialize(n);
   }
 
   public static TSDataType readDataType(ByteBuffer buffer) {
-    short n = readShort(buffer);
+    byte n = buffer.get();
     return TSDataType.deserialize(n);
   }
 
   public static TSEncoding readEncoding(InputStream inputStream) throws IOException {
-    short n = readShort(inputStream);
+    byte n = readByte(inputStream);
     return TSEncoding.deserialize(n);
   }
 
   public static TSEncoding readEncoding(ByteBuffer buffer) {
-    short n = readShort(buffer);
+    byte n = buffer.get();
     return TSEncoding.deserialize(n);
   }
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/ChunkGroupMetaDataV1.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/ChunkGroupMetaDataV1.java
deleted file mode 100644
index 64ce030..0000000
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/ChunkGroupMetaDataV1.java
+++ /dev/null
@@ -1,118 +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.tsfile.v1.file.metadata;
-
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Metadata of ChunkGroup.
- */
-public class ChunkGroupMetaDataV1 {
-
-  /**
-   * Name of device, this field is not serialized.
-   */
-  private String deviceID;
-
-  /**
-   * Byte offset of the corresponding data in the file Notice: include the chunk group marker.
-   * For Hadoop and Spark.
-   */
-  private long startOffsetOfChunkGroup;
-
-  /**
-   * End Byte position of the whole chunk group in the file Notice: position after the chunk group footer.
-   * For Hadoop and Spark.
-   */
-  private long endOffsetOfChunkGroup;
-
-  /**
-   * All time series chunks in this chunk group.
-   */
-  private List<ChunkMetadataV1> chunkMetaDataList;
-
-  private long version;
-
-  private ChunkGroupMetaDataV1() {
-    chunkMetaDataList = new ArrayList<>();
-  }
-
-  /**
-   * deserialize from ByteBuffer.
-   *
-   * @param buffer ByteBuffer
-   * @return ChunkGroupMetaData object
-   */
-  public static ChunkGroupMetaDataV1 deserializeFrom(ByteBuffer buffer) {
-    ChunkGroupMetaDataV1 chunkGroupMetaData = new ChunkGroupMetaDataV1();
-
-    chunkGroupMetaData.deviceID = ReadWriteIOUtils.readString(buffer);
-    chunkGroupMetaData.startOffsetOfChunkGroup = ReadWriteIOUtils.readLong(buffer);
-    chunkGroupMetaData.endOffsetOfChunkGroup = ReadWriteIOUtils.readLong(buffer);
-    chunkGroupMetaData.version = ReadWriteIOUtils.readLong(buffer);
-
-    int size = ReadWriteIOUtils.readInt(buffer);
-
-    List<ChunkMetadataV1> chunkMetaDataList = new ArrayList<>();
-    for (int i = 0; i < size; i++) {
-      ChunkMetadataV1 metaData = ChunkMetadataV1.deserializeFrom(buffer);
-      chunkMetaDataList.add(metaData);
-    }
-    chunkGroupMetaData.chunkMetaDataList = chunkMetaDataList;
-
-    return chunkGroupMetaData;
-  }
-
-  /**
-   * add time series chunk metadata to list. THREAD NOT SAFE
-   *
-   * @param metadata time series metadata to add
-   */
-  public void addTimeSeriesChunkMetaData(ChunkMetadataV1 metadata) {
-    if (chunkMetaDataList == null) {
-      chunkMetaDataList = new ArrayList<>();
-    }
-    chunkMetaDataList.add(metadata);
-  }
-
-  public List<ChunkMetadataV1> getChunkMetaDataList() {
-    return chunkMetaDataList;
-  }
-
-  public String getDeviceID() {
-    return deviceID;
-  }
-
-  public long getStartOffsetOfChunkGroup() {
-    return startOffsetOfChunkGroup;
-  }
-
-  public long getEndOffsetOfChunkGroup() {
-    return endOffsetOfChunkGroup;
-  }
-
-  public long getVersion() {
-    return version;
-  }
-
-}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/ChunkMetadataV1.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/ChunkMetadataV1.java
deleted file mode 100644
index fee07ab..0000000
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/ChunkMetadataV1.java
+++ /dev/null
@@ -1,131 +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.tsfile.v1.file.metadata;
-
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-import org.apache.iotdb.tsfile.v1.file.metadata.statistics.StatisticsV1;
-
-import java.nio.ByteBuffer;
-/**
- * MetaData of one chunk.
- */
-public class ChunkMetadataV1 {
-
-
-  private String measurementUid;
-
-  /**
-   * Byte offset of the corresponding data in the file Notice: include the chunk header and marker.
-   */
-  private long offsetOfChunkHeader;
-
-  private long numOfPoints;
-
-  private long startTime;
-
-  private long endTime;
-
-  private TSDataType tsDataType;
-
-  /**
-   * version is used to define the order of operations(insertion, deletion, update). version is set
-   * according to its belonging ChunkGroup only when being queried, so it is not persisted.
-   */
-  private long version;
-
-  private TsDigestV1 valuesStatistics;
-
-  private ChunkMetadataV1() {
-  }
-
-  /**
-   * deserialize from ByteBuffer.
-   *
-   * @param buffer ByteBuffer
-   * @return ChunkMetaData object
-   */
-  public static ChunkMetadataV1 deserializeFrom(ByteBuffer buffer) {
-    ChunkMetadataV1 chunkMetaData = new ChunkMetadataV1();
-
-    chunkMetaData.measurementUid = ReadWriteIOUtils.readString(buffer);
-    chunkMetaData.offsetOfChunkHeader = ReadWriteIOUtils.readLong(buffer);
-    chunkMetaData.numOfPoints = ReadWriteIOUtils.readLong(buffer);
-    chunkMetaData.startTime = ReadWriteIOUtils.readLong(buffer);
-    chunkMetaData.endTime = ReadWriteIOUtils.readLong(buffer);
-    chunkMetaData.tsDataType = ReadWriteIOUtils.readDataType(buffer);
-
-    chunkMetaData.valuesStatistics = TsDigestV1.deserializeFrom(buffer);
-
-    return chunkMetaData;
-  }
-
-  public long getNumOfPoints() {
-    return numOfPoints;
-  }
-  
-  public ChunkMetadata upgradeToChunkMetadata() {
-    Statistics<?> statistics = StatisticsV1
-        .constructStatisticsFromOldChunkMetadata(this);
-    ChunkMetadata chunkMetadata = new ChunkMetadata(this.measurementUid, this.tsDataType,
-        this.offsetOfChunkHeader, statistics);
-    chunkMetadata.setFromOldTsFile(true);
-    return chunkMetadata;
-  }
-
-  /**
-   * get offset of chunk header.
-   *
-   * @return Byte offset of header of this chunk (includes the marker)
-   */
-  public long getOffsetOfChunkHeader() {
-    return offsetOfChunkHeader;
-  }
-
-  public String getMeasurementUid() {
-    return measurementUid;
-  }
-
-  public TsDigestV1 getDigest() {
-    return valuesStatistics;
-  }
-
-  public long getStartTime() {
-    return startTime;
-  }
-
-  public long getEndTime() {
-    return endTime;
-  }
-
-  public TSDataType getTsDataType() {
-    return tsDataType;
-  }
-
-  public long getVersion() {
-    return version;
-  }
-
-  public void setVersion(long version) {
-    this.version = version;
-  }
-}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/TimeseriesMetadataForV1.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/TimeseriesMetadataForV1.java
deleted file mode 100644
index 04fd8e5..0000000
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/TimeseriesMetadataForV1.java
+++ /dev/null
@@ -1,42 +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.tsfile.v1.file.metadata;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
-import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
-
-public class TimeseriesMetadataForV1 extends TimeseriesMetadata {
-  
-  private List<ChunkMetadata> chunkMetadataList;
-
-  public void setChunkMetadataList(List<ChunkMetadata> chunkMetadataList) {
-    this.chunkMetadataList = chunkMetadataList;
-  }
-
-  @Override
-  public List<ChunkMetadata> loadChunkMetadataList() throws IOException {
-    chunkMetadataLoader.setDiskChunkLoader(chunkMetadataList);
-    return chunkMetadataList;
-  }
-
-}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/TsDeviceMetadataIndexV1.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/TsDeviceMetadataIndexV1.java
deleted file mode 100644
index 2258b61..0000000
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/TsDeviceMetadataIndexV1.java
+++ /dev/null
@@ -1,78 +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.tsfile.v1.file.metadata;
-
-import java.nio.ByteBuffer;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-
-public class TsDeviceMetadataIndexV1 {
-
-  /**
-   * The offset of the TsDeviceMetadata.
-   */
-  private long offset;
-  /**
-   * The size of the TsDeviceMetadata in the disk.
-   */
-  private int len;
-  /**
-   * The start time of the device.
-   */
-  private long startTime;
-  /**
-   * The end time of the device.
-   */
-  private long endTime;
-
-  public TsDeviceMetadataIndexV1() {
-    //do nothing
-  }
-
-  /**
-   * use buffer to get a TsDeviceMetadataIndex.
-   *
-   * @param buffer -determine the index's source
-   * @return -a TsDeviceMetadataIndex
-   */
-  public static TsDeviceMetadataIndexV1 deserializeFrom(ByteBuffer buffer) {
-    TsDeviceMetadataIndexV1 index = new TsDeviceMetadataIndexV1();
-    index.offset = ReadWriteIOUtils.readLong(buffer);
-    index.len = ReadWriteIOUtils.readInt(buffer);
-    index.startTime = ReadWriteIOUtils.readLong(buffer);
-    index.endTime = ReadWriteIOUtils.readLong(buffer);
-    return index;
-  }
-
-  public long getOffset() {
-    return offset;
-  }
-
-  public int getLen() {
-    return len;
-  }
-
-  public long getStartTime() {
-    return startTime;
-  }
-
-  public long getEndTime() {
-    return endTime;
-  }
-}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/TsDeviceMetadataV1.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/TsDeviceMetadataV1.java
deleted file mode 100644
index 4441158..0000000
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/TsDeviceMetadataV1.java
+++ /dev/null
@@ -1,87 +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.tsfile.v1.file.metadata;
-
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-public class TsDeviceMetadataV1 {
-
-  /**
-   * start time for a device.
-   **/
-  private long startTime = Long.MAX_VALUE;
-
-  /**
-   * end time for a device.
-   **/
-  private long endTime = Long.MIN_VALUE;
-
-  /**
-   * Row groups in this file.
-   */
-  private List<ChunkGroupMetaDataV1> chunkGroupMetadataList = new ArrayList<>();
-
-  public TsDeviceMetadataV1() {
-    // allowed to clair an empty TsDeviceMetadata whose fields will be assigned later.
-  }
-
-
-  /**
-   * deserialize from the given buffer.
-   *
-   * @param buffer -buffer to deserialize
-   * @return -device meta data
-   */
-  public static TsDeviceMetadataV1 deserializeFrom(ByteBuffer buffer) {
-    TsDeviceMetadataV1 deviceMetadata = new TsDeviceMetadataV1();
-
-    deviceMetadata.startTime = ReadWriteIOUtils.readLong(buffer);
-    deviceMetadata.endTime = ReadWriteIOUtils.readLong(buffer);
-
-    int size = ReadWriteIOUtils.readInt(buffer);
-    if (size > 0) {
-      List<ChunkGroupMetaDataV1> chunkGroupMetaDataList = new ArrayList<>();
-      for (int i = 0; i < size; i++) {
-        chunkGroupMetaDataList.add(ChunkGroupMetaDataV1.deserializeFrom(buffer));
-      }
-      deviceMetadata.chunkGroupMetadataList = chunkGroupMetaDataList;
-    }
-
-    return deviceMetadata;
-  }
-
-  public List<ChunkGroupMetaDataV1> getChunkGroupMetaDataList() {
-    return Collections.unmodifiableList(chunkGroupMetadataList);
-  }
-
-  public long getStartTime() {
-    return startTime;
-  }
-
-  public long getEndTime() {
-    return endTime;
-  }
-
-}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/TsDigestV1.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/TsDigestV1.java
deleted file mode 100644
index a40d3be..0000000
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/TsDigestV1.java
+++ /dev/null
@@ -1,75 +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.tsfile.v1.file.metadata;
-
-import java.nio.ByteBuffer;
-
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-
-/**
- * Digest/statistics per chunk group and per page.
- */
-public class TsDigestV1 {
-
-  private ByteBuffer[] statistics;
-
-  public TsDigestV1() {
-    // allowed to declare an empty TsDigest whose fields will be assigned later.
-  }
-
-  /**
-   * use given buffer to deserialize.
-   *
-   * @param buffer -given buffer
-   * @return -an instance of TsDigest
-   */
-  public static TsDigestV1 deserializeFrom(ByteBuffer buffer) {
-    TsDigestV1 digest = new TsDigestV1();
-    int size = ReadWriteIOUtils.readInt(buffer);
-    if (size > 0) {
-      digest.statistics = new ByteBuffer[StatisticType.getTotalTypeNum()];
-      ByteBuffer value;
-      for (int i = 0; i < size; i++) {
-        short n = ReadWriteIOUtils.readShort(buffer);
-        value = ReadWriteIOUtils.readByteBufferWithSelfDescriptionLength(buffer);
-        digest.statistics[n] = value;
-      }
-    } // else left digest.statistics as null
-
-    return digest;
-  }
-
-  /**
-   * get statistics of the current object.
-   */
-  public ByteBuffer[] getStatistics() {
-    return statistics;
-  }
-
-  public enum StatisticType {
-    MIN_VALUE, MAX_VALUE, FIRST_VALUE, LAST_VALUE, SUM_VALUE;
-
-    public static int getTotalTypeNum() {
-      return StatisticType.values().length;
-    }
-
-  }
-}
\ No newline at end of file
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/TsFileMetadataV1.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/TsFileMetadataV1.java
deleted file mode 100644
index 9184154..0000000
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/TsFileMetadataV1.java
+++ /dev/null
@@ -1,106 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.tsfile.v1.file.metadata;
-
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.iotdb.tsfile.utils.BloomFilter;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
-
-/**
- * TSFileMetaData collects all metadata info and saves in its data structure.
- */
-public class TsFileMetadataV1 {
-
-  private Map<String, TsDeviceMetadataIndexV1> deviceIndexMap = new HashMap<>();
-
-  // bloom filter
-  private BloomFilter bloomFilter;
-
-  public TsFileMetadataV1() {
-    //do nothing
-  }
-
-  /**
-   * deserialize data from the buffer.
-   *
-   * @param buffer -buffer use to deserialize
-   * @return -a instance of TsFileMetaData
-   */
-  public static TsFileMetadataV1 deserializeFrom(ByteBuffer buffer) {
-    TsFileMetadataV1 fileMetaData = new TsFileMetadataV1();
-
-    int size = ReadWriteIOUtils.readInt(buffer);
-    if (size > 0) {
-      Map<String, TsDeviceMetadataIndexV1> deviceMap = new HashMap<>();
-      String key;
-      TsDeviceMetadataIndexV1 value;
-      for (int i = 0; i < size; i++) {
-        key = ReadWriteIOUtils.readString(buffer);
-        value = TsDeviceMetadataIndexV1.deserializeFrom(buffer);
-        deviceMap.put(key, value);
-      }
-      fileMetaData.deviceIndexMap = deviceMap;
-    }
-
-    size = ReadWriteIOUtils.readInt(buffer);
-    if (size > 0) {
-      for (int i = 0; i < size; i++) {
-        ReadWriteIOUtils.readString(buffer);
-        MeasurementSchema.deserializeFrom(buffer);
-      }
-    }
-
-    if (ReadWriteIOUtils.readIsNull(buffer)) {
-       ReadWriteIOUtils.readString(buffer); // createdBy String
-    }
-    ReadWriteIOUtils.readInt(buffer); // totalChunkNum
-    ReadWriteIOUtils.readInt(buffer); // invalidChunkNum
-    // read bloom filter
-    if (buffer.hasRemaining()) {
-      byte[] bytes = ReadWriteIOUtils.readByteBufferWithSelfDescriptionLength(buffer).array();
-      int filterSize = ReadWriteIOUtils.readInt(buffer);
-      int hashFunctionSize = ReadWriteIOUtils.readInt(buffer);
-      fileMetaData.bloomFilter = BloomFilter.buildBloomFilter(bytes, filterSize, hashFunctionSize);
-    }
-
-    return fileMetaData;
-  }
-
-  public BloomFilter getBloomFilter() {
-    return bloomFilter;
-  }
-
-  public Map<String, TsDeviceMetadataIndexV1> getDeviceMap() {
-    return deviceIndexMap;
-  }
-
-  public boolean containsDevice(String deltaObjUid) {
-    return this.deviceIndexMap.containsKey(deltaObjUid);
-  }
-
-  public TsDeviceMetadataIndexV1 getDeviceMetadataIndex(String deviceUid) {
-    return this.deviceIndexMap.get(deviceUid);
-  }
-
-}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/statistics/BinaryStatisticsV1.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/statistics/BinaryStatisticsV1.java
deleted file mode 100644
index 3c68be1..0000000
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/statistics/BinaryStatisticsV1.java
+++ /dev/null
@@ -1,84 +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.tsfile.v1.file.metadata.statistics;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.ByteBuffer;
-import org.apache.iotdb.tsfile.utils.Binary;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-
-/**
- * Statistics for string type.
- */
-public class BinaryStatisticsV1 extends StatisticsV1<Binary> {
-
-  private Binary min = new Binary("");
-  private Binary max = new Binary("");
-  private Binary first = new Binary("");
-  private Binary last = new Binary("");
-  private double sum;
-
-  @Override
-  public Binary getMin() {
-    return min;
-  }
-
-  @Override
-  public Binary getMax() {
-    return max;
-  }
-
-  @Override
-  public Binary getFirst() {
-    return first;
-  }
-
-  @Override
-  public Binary getLast() {
-    return last;
-  }
-
-  @Override
-  public double getSum() {
-    return sum;
-  }
-
-  @Override
-  void deserialize(ByteBuffer byteBuffer) throws IOException {
-    this.min = new Binary(
-        ReadWriteIOUtils.readByteBufferWithSelfDescriptionLength(byteBuffer).array());
-    this.max = new Binary(
-        ReadWriteIOUtils.readByteBufferWithSelfDescriptionLength(byteBuffer).array());
-    this.first = new Binary(
-        ReadWriteIOUtils.readByteBufferWithSelfDescriptionLength(byteBuffer).array());
-    this.last = new Binary(
-        ReadWriteIOUtils.readByteBufferWithSelfDescriptionLength(byteBuffer).array());
-    this.sum = ReadWriteIOUtils.readDouble(byteBuffer);
-  }
-
-  @Override
-  void deserialize(InputStream inputStream) throws IOException {
-    this.min = new Binary(ReadWriteIOUtils.readBytesWithSelfDescriptionLength(inputStream));
-    this.max = new Binary(ReadWriteIOUtils.readBytesWithSelfDescriptionLength(inputStream));
-    this.first = new Binary(ReadWriteIOUtils.readBytesWithSelfDescriptionLength(inputStream));
-    this.last = new Binary(ReadWriteIOUtils.readBytesWithSelfDescriptionLength(inputStream));
-    this.sum = ReadWriteIOUtils.readDouble(inputStream);
-  }
-}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/statistics/BooleanStatisticsV1.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/statistics/BooleanStatisticsV1.java
deleted file mode 100644
index 24e2152..0000000
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/statistics/BooleanStatisticsV1.java
+++ /dev/null
@@ -1,80 +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.tsfile.v1.file.metadata.statistics;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.ByteBuffer;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-
-/**
- * Boolean Statistics.
- */
-public class BooleanStatisticsV1 extends StatisticsV1<Boolean> {
-
-  private boolean min;
-  private boolean max;
-  private boolean first;
-  private boolean last;
-  private double sum;
-
-  @Override
-  public Boolean getMin() {
-    return min;
-  }
-
-  @Override
-  public Boolean getMax() {
-    return max;
-  }
-
-  @Override
-  public Boolean getFirst() {
-    return first;
-  }
-
-  @Override
-  public Boolean getLast() {
-    return last;
-  }
-
-  @Override
-  public double getSum() {
-    return sum;
-  }
-
-  @Override
-  void deserialize(ByteBuffer byteBuffer) throws IOException {
-    this.min = ReadWriteIOUtils.readBool(byteBuffer);
-    this.max = ReadWriteIOUtils.readBool(byteBuffer);
-    this.first = ReadWriteIOUtils.readBool(byteBuffer);
-    this.last = ReadWriteIOUtils.readBool(byteBuffer);
-    this.sum = ReadWriteIOUtils.readDouble(byteBuffer);
-  }
-
-  @Override
-  void deserialize(InputStream inputStream) throws IOException {
-    this.min = ReadWriteIOUtils.readBool(inputStream);
-    this.max = ReadWriteIOUtils.readBool(inputStream);
-    this.first = ReadWriteIOUtils.readBool(inputStream);
-    this.last = ReadWriteIOUtils.readBool(inputStream);
-    this.sum = ReadWriteIOUtils.readDouble(inputStream);
-  }
-
-}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/statistics/DoubleStatisticsV1.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/statistics/DoubleStatisticsV1.java
deleted file mode 100644
index e784b91..0000000
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/statistics/DoubleStatisticsV1.java
+++ /dev/null
@@ -1,79 +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.tsfile.v1.file.metadata.statistics;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.ByteBuffer;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-
-/**
- * Statistics for double type.
- */
-public class DoubleStatisticsV1 extends StatisticsV1<Double> {
-
-  private double min;
-  private double max;
-  private double first;
-  private double last;
-  private double sum;
-
-  @Override
-  public Double getMin() {
-    return min;
-  }
-
-  @Override
-  public Double getMax() {
-    return max;
-  }
-
-  @Override
-  public Double getFirst() {
-    return first;
-  }
-
-  @Override
-  public Double getLast() {
-    return last;
-  }
-
-  @Override
-  public double getSum() {
-    return sum;
-  }
-
-  @Override
-  void deserialize(ByteBuffer byteBuffer) throws IOException {
-    this.min = ReadWriteIOUtils.readDouble(byteBuffer);
-    this.max = ReadWriteIOUtils.readDouble(byteBuffer);
-    this.first = ReadWriteIOUtils.readDouble(byteBuffer);
-    this.last = ReadWriteIOUtils.readDouble(byteBuffer);
-    this.sum = ReadWriteIOUtils.readDouble(byteBuffer);
-  }
-
-  @Override
-  void deserialize(InputStream inputStream) throws IOException {
-    this.min = ReadWriteIOUtils.readDouble(inputStream);
-    this.max = ReadWriteIOUtils.readDouble(inputStream);
-    this.first = ReadWriteIOUtils.readDouble(inputStream);
-    this.last = ReadWriteIOUtils.readDouble(inputStream);
-    this.sum = ReadWriteIOUtils.readDouble(inputStream);
-  }
-}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/statistics/FloatStatisticsV1.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/statistics/FloatStatisticsV1.java
deleted file mode 100644
index 6a8d8e8..0000000
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/statistics/FloatStatisticsV1.java
+++ /dev/null
@@ -1,79 +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.tsfile.v1.file.metadata.statistics;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.ByteBuffer;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-
-/**
- * Statistics for float type.
- */
-public class FloatStatisticsV1 extends StatisticsV1<Float> {
-
-  private float min;
-  private float max;
-  private float first;
-  private double sum;
-  private float last;
-
-  @Override
-  public Float getMin() {
-    return min;
-  }
-
-  @Override
-  public Float getMax() {
-    return max;
-  }
-
-  @Override
-  public Float getFirst() {
-    return first;
-  }
-
-  @Override
-  public Float getLast() {
-    return last;
-  }
-
-  @Override
-  public double getSum() {
-    return sum;
-  }
-
-  @Override
-  void deserialize(ByteBuffer byteBuffer) throws IOException {
-    this.min = ReadWriteIOUtils.readFloat(byteBuffer);
-    this.max = ReadWriteIOUtils.readFloat(byteBuffer);
-    this.first = ReadWriteIOUtils.readFloat(byteBuffer);
-    this.last = ReadWriteIOUtils.readFloat(byteBuffer);
-    this.sum = ReadWriteIOUtils.readDouble(byteBuffer);
-  }
-
-  @Override
-  void deserialize(InputStream inputStream) throws IOException {
-    this.min = ReadWriteIOUtils.readFloat(inputStream);
-    this.max = ReadWriteIOUtils.readFloat(inputStream);
-    this.first = ReadWriteIOUtils.readFloat(inputStream);
-    this.last = ReadWriteIOUtils.readFloat(inputStream);
-    this.sum = ReadWriteIOUtils.readDouble(inputStream);
-  }
-}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/statistics/IntegerStatisticsV1.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/statistics/IntegerStatisticsV1.java
deleted file mode 100644
index 2511abd..0000000
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/statistics/IntegerStatisticsV1.java
+++ /dev/null
@@ -1,79 +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.tsfile.v1.file.metadata.statistics;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.ByteBuffer;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-
-/**
- * Statistics for int type.
- */
-public class IntegerStatisticsV1 extends StatisticsV1<Integer> {
-
-  private int min;
-  private int max;
-  private int first;
-  private int last;
-  private double sum;
-
-  @Override
-  public Integer getMin() {
-    return min;
-  }
-
-  @Override
-  public Integer getMax() {
-    return max;
-  }
-
-  @Override
-  public Integer getFirst() {
-    return first;
-  }
-
-  @Override
-  public Integer getLast() {
-    return last;
-  }
-
-  @Override
-  public double getSum() {
-    return sum;
-  }
-
-  @Override
-  void deserialize(ByteBuffer byteBuffer) throws IOException {
-    this.min = ReadWriteIOUtils.readInt(byteBuffer);
-    this.max = ReadWriteIOUtils.readInt(byteBuffer);
-    this.first = ReadWriteIOUtils.readInt(byteBuffer);
-    this.last = ReadWriteIOUtils.readInt(byteBuffer);
-    this.sum = ReadWriteIOUtils.readDouble(byteBuffer);
-  }
-
-  @Override
-  void deserialize(InputStream inputStream) throws IOException {
-    this.min = ReadWriteIOUtils.readInt(inputStream);
-    this.max = ReadWriteIOUtils.readInt(inputStream);
-    this.first = ReadWriteIOUtils.readInt(inputStream);
-    this.last = ReadWriteIOUtils.readInt(inputStream);
-    this.sum = ReadWriteIOUtils.readDouble(inputStream);
-  }
-}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/statistics/LongStatisticsV1.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/statistics/LongStatisticsV1.java
deleted file mode 100644
index 29beeeb..0000000
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/statistics/LongStatisticsV1.java
+++ /dev/null
@@ -1,80 +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.tsfile.v1.file.metadata.statistics;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.ByteBuffer;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-
-/**
- * Statistics for long type.
- */
-public class LongStatisticsV1 extends StatisticsV1<Long> {
-
-  private long min;
-  private long max;
-  private long first;
-  private long last;
-  private double sum;
-
-  @Override
-  public Long getMin() {
-    return min;
-  }
-
-  @Override
-  public Long getMax() {
-    return max;
-  }
-
-  @Override
-  public Long getFirst() {
-    return first;
-  }
-
-  @Override
-  public Long getLast() {
-    return last;
-  }
-
-  @Override
-  public double getSum() {
-    return sum;
-  }
-
-  @Override
-  void deserialize(ByteBuffer byteBuffer) throws IOException {
-    this.min = ReadWriteIOUtils.readLong(byteBuffer);
-    this.max = ReadWriteIOUtils.readLong(byteBuffer);
-    this.first = ReadWriteIOUtils.readLong(byteBuffer);
-    this.last = ReadWriteIOUtils.readLong(byteBuffer);
-    this.sum = ReadWriteIOUtils.readDouble(byteBuffer);
-  }
-
-  @Override
-  void deserialize(InputStream inputStream) throws IOException {
-    this.min = ReadWriteIOUtils.readLong(inputStream);
-    this.max = ReadWriteIOUtils.readLong(inputStream);
-    this.first = ReadWriteIOUtils.readLong(inputStream);
-    this.last = ReadWriteIOUtils.readLong(inputStream);
-    this.sum = ReadWriteIOUtils.readDouble(inputStream);
-  }
-
-}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/statistics/StatisticsV1.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/statistics/StatisticsV1.java
deleted file mode 100644
index ea91dd3..0000000
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/metadata/statistics/StatisticsV1.java
+++ /dev/null
@@ -1,225 +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.tsfile.v1.file.metadata.statistics;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.ByteBuffer;
-
-import org.apache.iotdb.tsfile.exception.write.UnknownColumnTypeException;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.file.metadata.statistics.BinaryStatistics;
-import org.apache.iotdb.tsfile.file.metadata.statistics.BooleanStatistics;
-import org.apache.iotdb.tsfile.file.metadata.statistics.DoubleStatistics;
-import org.apache.iotdb.tsfile.file.metadata.statistics.FloatStatistics;
-import org.apache.iotdb.tsfile.file.metadata.statistics.IntegerStatistics;
-import org.apache.iotdb.tsfile.file.metadata.statistics.LongStatistics;
-import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
-import org.apache.iotdb.tsfile.utils.Binary;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-import org.apache.iotdb.tsfile.v1.file.metadata.ChunkMetadataV1;
-import org.apache.iotdb.tsfile.v1.file.metadata.TsDigestV1;
-
-/**
- * This class is used for recording statistic information of each measurement in a delta file. While
- * writing processing, the processor records the digest information. Statistics includes maximum,
- * minimum and null value count up to version 0.0.1.<br> Each data type extends this Statistic as
- * super class.<br>
- *
- * @param <T> data type for Statistics
- */
-public abstract class StatisticsV1<T> {
-
-  /**
-   * static method providing statistic instance for respective data type.
-   *
-   * @param type - data type
-   * @return Statistics
-   */
-  public static StatisticsV1 getStatsByType(TSDataType type) {
-    switch (type) {
-      case INT32:
-        return new IntegerStatisticsV1();
-      case INT64:
-        return new LongStatisticsV1();
-      case TEXT:
-        return new BinaryStatisticsV1();
-      case BOOLEAN:
-        return new BooleanStatisticsV1();
-      case DOUBLE:
-        return new DoubleStatisticsV1();
-      case FLOAT:
-        return new FloatStatisticsV1();
-      default:
-        throw new UnknownColumnTypeException(type.toString());
-    }
-  }
-
-  public static StatisticsV1 deserialize(InputStream inputStream, TSDataType dataType)
-      throws IOException {
-    StatisticsV1<?> statistics = getStatsByType(dataType);
-    statistics.deserialize(inputStream);
-    return statistics;
-  }
-
-  public static StatisticsV1 deserialize(ByteBuffer buffer, TSDataType dataType) throws IOException {
-    StatisticsV1<?> statistics = getStatsByType(dataType);
-    statistics.deserialize(buffer);
-    return statistics;
-  }
-
-  /**
-   * For upgrading 0.9.x/v1 -> 0.10/v2
-   */
-  public static Statistics upgradeOldStatistics(StatisticsV1<?> oldstatistics,
-      TSDataType dataType, int numOfValues, long maxTimestamp, long minTimestamp) {
-    Statistics<?> statistics = Statistics.getStatsByType(dataType);
-    statistics.setStartTime(minTimestamp);
-    statistics.setEndTime(maxTimestamp);
-    statistics.setCount(numOfValues);
-    statistics.setEmpty(false);
-    switch (dataType) {
-      case INT32:
-        ((IntegerStatistics) statistics)
-        .initializeStats(((IntegerStatisticsV1) oldstatistics).getMin(),
-            ((IntegerStatisticsV1) oldstatistics).getMax(),
-            ((IntegerStatisticsV1) oldstatistics).getFirst(),
-            ((IntegerStatisticsV1) oldstatistics).getLast(),
-            ((IntegerStatisticsV1) oldstatistics).getSum());
-        break;
-      case INT64:
-        ((LongStatistics) statistics)
-        .initializeStats(((LongStatisticsV1) oldstatistics).getMin(),
-            ((LongStatisticsV1) oldstatistics).getMax(),
-            ((LongStatisticsV1) oldstatistics).getFirst(),
-            ((LongStatisticsV1) oldstatistics).getLast(),
-            ((LongStatisticsV1) oldstatistics).getSum());
-        break;
-      case TEXT:
-        ((BinaryStatistics) statistics)
-        .initializeStats(((BinaryStatisticsV1) oldstatistics).getFirst(),
-            ((BinaryStatisticsV1) oldstatistics).getLast());
-        break;
-      case BOOLEAN:
-        ((BooleanStatistics) statistics)
-        .initializeStats(((BooleanStatisticsV1) oldstatistics).getFirst(),
-            ((BooleanStatisticsV1) oldstatistics).getLast());
-        break;
-      case DOUBLE:
-        ((DoubleStatistics) statistics)
-        .initializeStats(((DoubleStatisticsV1) oldstatistics).getMin(),
-            ((DoubleStatisticsV1) oldstatistics).getMax(),
-            ((DoubleStatisticsV1) oldstatistics).getFirst(),
-            ((DoubleStatisticsV1) oldstatistics).getLast(),
-            ((DoubleStatisticsV1) oldstatistics).getSum());
-        break;
-      case FLOAT:
-        ((FloatStatistics) statistics)
-        .initializeStats(((FloatStatisticsV1) oldstatistics).getMin(),
-            ((FloatStatisticsV1) oldstatistics).getMax(),
-            ((FloatStatisticsV1) oldstatistics).getFirst(),
-            ((FloatStatisticsV1) oldstatistics).getLast(),
-            ((FloatStatisticsV1) oldstatistics).getSum());
-        break;
-      default:
-        throw new UnknownColumnTypeException(statistics.getType()
-            .toString());
-    }
-    return statistics;
-  }
-
-  /**
-   * For upgrading 0.9.x/v1 -> 0.10.x/v2
-   */
-  public static Statistics constructStatisticsFromOldChunkMetadata(ChunkMetadataV1 oldChunkMetadata) {
-    Statistics<?> statistics = Statistics.getStatsByType(oldChunkMetadata.getTsDataType());
-    statistics.setStartTime(oldChunkMetadata.getStartTime());
-    statistics.setEndTime(oldChunkMetadata.getEndTime());
-    statistics.setCount(oldChunkMetadata.getNumOfPoints());
-    statistics.setEmpty(false);
-    TsDigestV1 tsDigest = oldChunkMetadata.getDigest();
-    ByteBuffer[] buffers = tsDigest.getStatistics();
-    switch (statistics.getType()) {
-      case INT32:
-        ((IntegerStatistics) statistics)
-        .initializeStats(ReadWriteIOUtils.readInt(buffers[0]),
-            ReadWriteIOUtils.readInt(buffers[1]),
-            ReadWriteIOUtils.readInt(buffers[2]),
-            ReadWriteIOUtils.readInt(buffers[3]),
-            ReadWriteIOUtils.readDouble(buffers[4]));
-        break;
-      case INT64:
-        ((LongStatistics) statistics)
-        .initializeStats(ReadWriteIOUtils.readLong(buffers[0]),
-            ReadWriteIOUtils.readLong(buffers[1]),
-            ReadWriteIOUtils.readLong(buffers[2]),
-            ReadWriteIOUtils.readLong(buffers[3]),
-            ReadWriteIOUtils.readDouble(buffers[4]));
-        break;
-      case TEXT:
-        ((BinaryStatistics) statistics)
-        .initializeStats(new Binary(buffers[2].array()),
-            new Binary(buffers[3].array()));
-        break;
-      case BOOLEAN:
-        ((BooleanStatistics) statistics)
-        .initializeStats(ReadWriteIOUtils.readBool(buffers[2]),
-            ReadWriteIOUtils.readBool(buffers[3]));
-        break;
-      case DOUBLE:
-        ((DoubleStatistics) statistics)
-        .initializeStats(ReadWriteIOUtils.readDouble(buffers[0]),
-            ReadWriteIOUtils.readDouble(buffers[1]),
-            ReadWriteIOUtils.readDouble(buffers[2]),
-            ReadWriteIOUtils.readDouble(buffers[3]),
-            ReadWriteIOUtils.readDouble(buffers[4]));
-        break;
-      case FLOAT:
-        ((FloatStatistics) statistics)
-        .initializeStats(ReadWriteIOUtils.readFloat(buffers[0]),
-            ReadWriteIOUtils.readFloat(buffers[1]),
-            ReadWriteIOUtils.readFloat(buffers[2]),
-            ReadWriteIOUtils.readFloat(buffers[3]),
-            ReadWriteIOUtils.readDouble(buffers[4]));
-        break;
-      default:
-        throw new UnknownColumnTypeException(statistics.getType()
-            .toString());
-    }
-    return statistics;
-  }
-
-  public abstract T getMin();
-
-  public abstract T getMax();
-
-  public abstract T getFirst();
-
-  public abstract T getLast();
-
-  public abstract double getSum();
-
-  /**
-   * read data from the inputStream.
-   */
-  abstract void deserialize(InputStream inputStream) throws IOException;
-
-  abstract void deserialize(ByteBuffer byteBuffer) throws IOException;
-
-}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/utils/HeaderUtils.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/utils/HeaderUtils.java
deleted file mode 100644
index 403db9a..0000000
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/file/utils/HeaderUtils.java
+++ /dev/null
@@ -1,141 +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.tsfile.v1.file.utils;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.ByteBuffer;
-
-import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
-import org.apache.iotdb.tsfile.file.MetaMarker;
-import org.apache.iotdb.tsfile.file.header.ChunkHeader;
-import org.apache.iotdb.tsfile.file.header.PageHeader;
-import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
-import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
-import org.apache.iotdb.tsfile.read.reader.TsFileInput;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-import org.apache.iotdb.tsfile.v1.file.metadata.statistics.StatisticsV1;
-
-public class HeaderUtils {
-  
-  private HeaderUtils() {
-  }
-  
-  public static PageHeader deserializePageHeaderV1(InputStream inputStream, TSDataType dataType)
-      throws IOException {
-    int uncompressedSize = ReadWriteIOUtils.readInt(inputStream);
-    int compressedSize = ReadWriteIOUtils.readInt(inputStream);
-    int numOfValues = ReadWriteIOUtils.readInt(inputStream);
-    long maxTimestamp = ReadWriteIOUtils.readLong(inputStream);
-    long minTimestamp = ReadWriteIOUtils.readLong(inputStream);
-    StatisticsV1<?> oldstatistics = StatisticsV1.deserialize(inputStream, dataType);
-    Statistics<?> statistics = StatisticsV1.upgradeOldStatistics(oldstatistics, dataType, 
-        numOfValues, maxTimestamp, minTimestamp);
-    return new PageHeader(uncompressedSize, compressedSize, statistics);
-  }
-
-  public static PageHeader deserializePageHeaderV1(ByteBuffer buffer, TSDataType dataType)
-      throws IOException {
-    int uncompressedSize = ReadWriteIOUtils.readInt(buffer);
-    int compressedSize = ReadWriteIOUtils.readInt(buffer);
-    int numOfValues = ReadWriteIOUtils.readInt(buffer);
-    long maxTimestamp = ReadWriteIOUtils.readLong(buffer);
-    long minTimestamp = ReadWriteIOUtils.readLong(buffer);
-    StatisticsV1<?> oldstatistics = StatisticsV1.deserialize(buffer, dataType);
-    Statistics<?> statistics = StatisticsV1.upgradeOldStatistics(oldstatistics, dataType, 
-        numOfValues, maxTimestamp, minTimestamp);
-    return new PageHeader(uncompressedSize, compressedSize, statistics);
-  }
-
-  /**
-   * deserialize from inputStream.
-   *
-   * @param markerRead Whether the marker of the CHUNK_HEADER has been read
-   */
-  public static ChunkHeader deserializeChunkHeaderV1(InputStream inputStream, boolean markerRead) 
-      throws IOException {
-    if (!markerRead) {
-      byte marker = (byte) inputStream.read();
-      if (marker != MetaMarker.CHUNK_HEADER) {
-        MetaMarker.handleUnexpectedMarker(marker);
-      }
-    }
-
-    String measurementID = ReadWriteIOUtils.readString(inputStream);
-    int dataSize = ReadWriteIOUtils.readInt(inputStream);
-    TSDataType dataType = TSDataType.deserialize(ReadWriteIOUtils.readShort(inputStream));
-    int numOfPages = ReadWriteIOUtils.readInt(inputStream);
-    CompressionType type = ReadWriteIOUtils.readCompressionType(inputStream);
-    TSEncoding encoding = ReadWriteIOUtils.readEncoding(inputStream);
-    // read maxTombstoneTime from old TsFile, has been removed in newer versions of TsFile
-    ReadWriteIOUtils.readLong(inputStream);
-    return new ChunkHeader(measurementID, dataSize, dataType, type, encoding,
-        numOfPages);
-  }
-
-  /**
-   * deserialize from TsFileInput.
-   *
-   * @param input           TsFileInput
-   * @param offset          offset
-   * @param chunkHeaderSize the size of chunk's header
-   * @param markerRead      read marker (boolean type)
-   * @return CHUNK_HEADER object
-   * @throws IOException IOException
-   */
-  public static ChunkHeader deserializeChunkHeaderV1(TsFileInput input, long offset,
-      int chunkHeaderSize, boolean markerRead) throws IOException {
-    long offsetVar = offset;
-    if (!markerRead) {
-      offsetVar++;
-    }
-
-    // read chunk header from input to buffer
-    ByteBuffer buffer = ByteBuffer.allocate(chunkHeaderSize);
-    input.read(buffer, offsetVar);
-    buffer.flip();
-
-    // read measurementID
-    int size = buffer.getInt();
-    String measurementID = ReadWriteIOUtils.readStringWithLength(buffer, size);
-    int dataSize = ReadWriteIOUtils.readInt(buffer);
-    TSDataType dataType = TSDataType.deserialize(ReadWriteIOUtils.readShort(buffer));
-    int numOfPages = ReadWriteIOUtils.readInt(buffer);
-    CompressionType type = ReadWriteIOUtils.readCompressionType(buffer);
-    TSEncoding encoding = ReadWriteIOUtils.readEncoding(buffer);
-    // read maxTombstoneTime from old TsFile, has been removed in newer versions of TsFile
-    ReadWriteIOUtils.readLong(buffer);
-    return new ChunkHeader(measurementID, dataSize, dataType, type, encoding, numOfPages);
-  }
-
-  public static int getSerializedSizeV1(String measurementID) {
-    return Byte.BYTES // marker
-        + Integer.BYTES // measurementID length
-        + measurementID.getBytes(TSFileConfig.STRING_CHARSET).length // measurementID
-        + Integer.BYTES // dataSize
-        + TSDataType.getSerializedSize() // dataType
-        + CompressionType.getSerializedSize() // compressionType
-        + TSEncoding.getSerializedSize() // encodingType
-        + Integer.BYTES // numOfPages
-        + Long.BYTES;  // maxTombstoneTime
-  }
-}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/read/TsFileSequenceReaderForV1.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/read/TsFileSequenceReaderForV1.java
deleted file mode 100644
index 91990be..0000000
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/v1/read/TsFileSequenceReaderForV1.java
+++ /dev/null
@@ -1,409 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.tsfile.v1.read;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
-import org.apache.iotdb.tsfile.file.header.ChunkHeader;
-import org.apache.iotdb.tsfile.file.header.PageHeader;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
-import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
-import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
-import org.apache.iotdb.tsfile.read.common.Chunk;
-import org.apache.iotdb.tsfile.read.common.Path;
-import org.apache.iotdb.tsfile.read.reader.TsFileInput;
-import org.apache.iotdb.tsfile.utils.BloomFilter;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-import org.apache.iotdb.tsfile.v1.file.metadata.ChunkGroupMetaDataV1;
-import org.apache.iotdb.tsfile.v1.file.metadata.ChunkMetadataV1;
-import org.apache.iotdb.tsfile.v1.file.metadata.TsDeviceMetadataV1;
-import org.apache.iotdb.tsfile.v1.file.metadata.TsDeviceMetadataIndexV1;
-import org.apache.iotdb.tsfile.v1.file.metadata.TsFileMetadataV1;
-import org.apache.iotdb.tsfile.v1.file.metadata.TimeseriesMetadataForV1;
-import org.apache.iotdb.tsfile.v1.file.utils.HeaderUtils;
-
-public class TsFileSequenceReaderForV1 extends TsFileSequenceReader {
-
-  private long fileMetadataPos;
-  private int fileMetadataSize;
-  private TsFileMetadataV1 oldTsFileMetaData;
-  // device -> measurement -> TimeseriesMetadata
-  private Map<String, Map<String, TimeseriesMetadata>> cachedDeviceMetadataFromOldFile = new ConcurrentHashMap<>();
-  private static final ReadWriteLock cacheLock = new ReentrantReadWriteLock();
-  private boolean cacheDeviceMetadata;
-
-  /**
-   * Create a file reader of the given file. The reader will read the tail of the file to get the
-   * file metadata size.Then the reader will skip the first TSFileConfig.MAGIC_STRING.getBytes().length
-   * + TSFileConfig.NUMBER_VERSION.getBytes().length bytes of the file for preparing reading real
-   * data.
-   *
-   * @param file the data file
-   * @throws IOException If some I/O error occurs
-   */
-  public TsFileSequenceReaderForV1(String file) throws IOException {
-    super(file, true);
-  }
-
-  /**
-   * construct function for TsFileSequenceReader.
-   *
-   * @param file -given file name
-   * @param loadMetadataSize -whether load meta data size
-   */
-  public TsFileSequenceReaderForV1(String file, boolean loadMetadataSize) throws IOException {
-    super(file, loadMetadataSize);
-  }
-
-  /**
-   * Create a file reader of the given file. The reader will read the tail of the file to get the
-   * file metadata size.Then the reader will skip the first TSFileConfig.MAGIC_STRING.getBytes().length
-   * + TSFileConfig.NUMBER_VERSION.getBytes().length bytes of the file for preparing reading real
-   * data.
-   *
-   * @param input given input
-   */
-  public TsFileSequenceReaderForV1(TsFileInput input) throws IOException {
-    this(input, true);
-  }
-
-  /**
-   * construct function for TsFileSequenceReader.
-   *
-   * @param input -given input
-   * @param loadMetadataSize -load meta data size
-   */
-  public TsFileSequenceReaderForV1(TsFileInput input, boolean loadMetadataSize) throws IOException {
-    super(input, loadMetadataSize);
-  }
-
-  /**
-   * construct function for TsFileSequenceReader.
-   *
-   * @param input the input of a tsfile. The current position should be a markder and then a chunk
-   * Header, rather than the magic number
-   * @param fileMetadataPos the position of the file metadata in the TsFileInput from the beginning
-   * of the input to the current position
-   * @param fileMetadataSize the byte size of the file metadata in the input
-   */
-  public TsFileSequenceReaderForV1(TsFileInput input, long fileMetadataPos, int fileMetadataSize) {
-    super(input, fileMetadataPos, fileMetadataSize);
-    this.fileMetadataPos = fileMetadataPos;
-    this.fileMetadataSize = fileMetadataSize;
-  }
-
-  @Override
-  public void loadMetadataSize() throws IOException {
-    ByteBuffer metadataSize = ByteBuffer.allocate(Integer.BYTES);
-    if (readTailMagic().equals(TSFileConfig.MAGIC_STRING)) {
-      tsFileInput.read(metadataSize,
-          tsFileInput.size() - TSFileConfig.MAGIC_STRING.getBytes().length - Integer.BYTES);
-      metadataSize.flip();
-      // read file metadata size and position
-      fileMetadataSize = ReadWriteIOUtils.readInt(metadataSize);
-      fileMetadataPos = tsFileInput.size() - TSFileConfig.MAGIC_STRING.getBytes().length
-          - Integer.BYTES - fileMetadataSize;
-    }
-  }
-  
-  public TsFileMetadataV1 readOldFileMetadata() throws IOException {
-    if (oldTsFileMetaData == null) {
-      oldTsFileMetaData = TsFileMetadataV1
-          .deserializeFrom(readDataFromOldFile(fileMetadataPos, fileMetadataSize));
-    }
-    return oldTsFileMetaData;
-  }
-
-  /**
-   * this function does not modify the position of the file reader.
-   *
-   * @throws IOException io error
-   */
-  @Override
-  public BloomFilter readBloomFilter() throws IOException {
-    readOldFileMetadata();
-    return oldTsFileMetaData.getBloomFilter();
-  }
-
-  /**
-   * this function reads measurements and TimeseriesMetaDatas in given device Thread Safe
-   *
-   * @param device name
-   * @return the map measurementId -> TimeseriesMetaData in one device
-   * @throws IOException io error
-   */
-  @Override
-  public Map<String, TimeseriesMetadata> readDeviceMetadata(String device) throws IOException {
-    if (!cacheDeviceMetadata) {
-      return constructDeviceMetadataFromOldFile(device);
-    }
-
-    cacheLock.readLock().lock();
-    try {
-      if (cachedDeviceMetadataFromOldFile.containsKey(device)) {
-        return cachedDeviceMetadataFromOldFile.get(device);
-      }
-    } finally {
-      cacheLock.readLock().unlock();
-    }
-
-    cacheLock.writeLock().lock();
-    try {
-      if (cachedDeviceMetadataFromOldFile.containsKey(device)) {
-        return cachedDeviceMetadataFromOldFile.get(device);
-      }
-      readOldFileMetadata();
-      if (!oldTsFileMetaData.containsDevice(device)) {
-        return new HashMap<>();
-      }
-      Map<String, TimeseriesMetadata> deviceMetadata = constructDeviceMetadataFromOldFile(device);
-      cachedDeviceMetadataFromOldFile.put(device, deviceMetadata);
-      return deviceMetadata;
-    } finally {
-      cacheLock.writeLock().unlock();
-    }
-  }
-
-  /**
-   * for old TsFile
-   * this function does not modify the position of the file reader.
-   */
-  private Map<String, TimeseriesMetadata> constructDeviceMetadataFromOldFile(String device)
-      throws IOException {
-    Map<String, TimeseriesMetadata> newDeviceMetadata = new HashMap<>();
-    readOldFileMetadata();
-    TsDeviceMetadataIndexV1 index = oldTsFileMetaData.getDeviceMetadataIndex(device);
-    // read TsDeviceMetadata from file
-    TsDeviceMetadataV1 tsDeviceMetadata = readOldTsDeviceMetaData(index);
-    if (tsDeviceMetadata == null) {
-      return newDeviceMetadata;
-    }
-
-    Map<String, List<ChunkMetadata>> measurementChunkMetaMap = new HashMap<>();
-    // get all ChunkMetaData of this path included in all ChunkGroups of this device
-    for (ChunkGroupMetaDataV1 chunkGroupMetaData : tsDeviceMetadata.getChunkGroupMetaDataList()) {
-      List<ChunkMetadataV1> chunkMetaDataListInOneChunkGroup = chunkGroupMetaData.getChunkMetaDataList();
-      for (ChunkMetadataV1 oldChunkMetadata : chunkMetaDataListInOneChunkGroup) {
-        oldChunkMetadata.setVersion(chunkGroupMetaData.getVersion());
-        measurementChunkMetaMap.computeIfAbsent(oldChunkMetadata.getMeasurementUid(), key -> new ArrayList<>())
-          .add(oldChunkMetadata.upgradeToChunkMetadata());
-      }
-    }
-    measurementChunkMetaMap.forEach((measurementId, chunkMetadataList) -> {
-      if (!chunkMetadataList.isEmpty()) {
-        TimeseriesMetadataForV1 timeseiresMetadata = new TimeseriesMetadataForV1();
-        timeseiresMetadata.setMeasurementId(measurementId);
-        timeseiresMetadata.setTSDataType(chunkMetadataList.get(0).getDataType());
-        Statistics<?> statistics = Statistics.getStatsByType(chunkMetadataList.get(0).getDataType());
-        for (ChunkMetadata chunkMetadata : chunkMetadataList) {
-          statistics.mergeStatistics(chunkMetadata.getStatistics());
-        }
-        timeseiresMetadata.setStatistics(statistics);
-        timeseiresMetadata.setChunkMetadataList(chunkMetadataList);
-        newDeviceMetadata.put(measurementId, timeseiresMetadata);
-      }
-    });
-    return newDeviceMetadata;
-  }
-
-  /**
-   * for old TsFile
-   * this function does not modify the position of the file reader.
-   */
-  private TsDeviceMetadataV1 readOldTsDeviceMetaData(TsDeviceMetadataIndexV1 index) 
-      throws IOException {
-    if (index == null) {
-      return null;
-    }
-    return TsDeviceMetadataV1.deserializeFrom(readDataFromOldFile(index.getOffset(), index.getLen()));
-  }
-
-  @Override
-  public TimeseriesMetadata readTimeseriesMetadata(Path path) throws IOException {
-    return getTimeseriesMetadataFromOldFile(path);
-  }
-
-  @Override
-  public List<TimeseriesMetadata> readTimeseriesMetadata(String device, Set<String> measurements)
-      throws IOException {
-    return getTimeseriesMetadataFromOldFile(device, measurements);
-  }
-
-  /**
-   *  for 0.9.x/v1 TsFile
-   */
-  private TimeseriesMetadata getTimeseriesMetadataFromOldFile(Path path) throws IOException {
-    Map<String, TimeseriesMetadata> deviceMetadata = 
-        constructDeviceMetadataFromOldFile(path.getDevice());
-    return deviceMetadata.get(path.getMeasurement());
-  }
-
-  /**
-   *  for 0.9.x/v1 TsFile
-   */
-  private List<TimeseriesMetadata> getTimeseriesMetadataFromOldFile(String device, Set<String> measurements)
-      throws IOException {
-    Map<String, TimeseriesMetadata> deviceMetadata = 
-        constructDeviceMetadataFromOldFile(device);
-    List<TimeseriesMetadata> resultTimeseriesMetadataList = new ArrayList<>();
-    for (Entry<String, TimeseriesMetadata> entry : deviceMetadata.entrySet()) {
-      if (measurements.contains(entry.getKey())) {
-        resultTimeseriesMetadataList.add(entry.getValue());
-      }
-    }
-    return resultTimeseriesMetadataList;
-  }
-
-  /**
-   * read the chunk's header.
-   *
-   * @param position the file offset of this chunk's header
-   * @param chunkHeaderSize the size of chunk's header
-   * @param markerRead true if the offset does not contains the marker , otherwise false
-   */
-  private ChunkHeader readChunkHeaderFromOldFile(long position, int chunkHeaderSize, boolean markerRead)
-      throws IOException {
-    return HeaderUtils.deserializeChunkHeaderV1(tsFileInput, position, chunkHeaderSize, markerRead);
-  }
-
-  /**
-   * notice, this function will modify channel's position.
-   *
-   * @param dataSize the size of chunkdata
-   * @param position the offset of the chunk data
-   * @return the pages of this chunk
-   */
-  private ByteBuffer readChunkFromOldFile(long position, int dataSize) throws IOException {
-    return readDataFromOldFile(position, dataSize);
-  }
-
-  /**
-   * read memory chunk.
... 3253 lines suppressed ...