You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2022/01/15 06:15:28 UTC

[iotdb] branch master updated: [IOTDB-1892] Add TsfileSelfCheckTool in server (#4674)

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

qiaojialin 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 46ee27b  [IOTDB-1892] Add TsfileSelfCheckTool in server (#4674)
46ee27b is described below

commit 46ee27b19fcbdb7cc3c2153c9d583a2da6bcc695
Author: Hang Zhang <34...@users.noreply.github.com>
AuthorDate: Sat Jan 15 14:14:51 2022 +0800

    [IOTDB-1892] Add TsfileSelfCheckTool in server (#4674)
---
 .../UserGuide/System-Tools/TsFileSelfCheck-Tool.md |  42 ++++
 .../UserGuide/System-Tools/TsFileSelfCheck-Tool.md |  42 ++++
 .../TsFileTimeseriesMetadataException.java         |  16 +-
 .../apache/iotdb/db/tools/TsFileSelfCheckTool.java | 190 ++++++++++++++++
 .../apache/iotdb/db/tools/TsFileSketchTool.java    |   1 +
 .../iotdb/db/tools/TsFileSelfCheckToolTest.java    | 239 +++++++++++++++++++++
 .../TsFileStatisticsMistakesException.java}        |  10 +-
 .../file/metadata/statistics/BinaryStatistics.java |  15 ++
 .../metadata/statistics/BooleanStatistics.java     |  17 ++
 .../file/metadata/statistics/DoubleStatistics.java |  20 ++
 .../file/metadata/statistics/FloatStatistics.java  |  20 ++
 .../metadata/statistics/IntegerStatistics.java     |  19 ++
 .../file/metadata/statistics/LongStatistics.java   |  19 ++
 .../iotdb/tsfile/read/TsFileCheckStatus.java       |   7 +-
 .../iotdb/tsfile/read/TsFileSequenceReader.java    | 135 ++++++++++++
 15 files changed, 776 insertions(+), 16 deletions(-)

diff --git a/docs/UserGuide/System-Tools/TsFileSelfCheck-Tool.md b/docs/UserGuide/System-Tools/TsFileSelfCheck-Tool.md
new file mode 100644
index 0000000..ef0c8ef
--- /dev/null
+++ b/docs/UserGuide/System-Tools/TsFileSelfCheck-Tool.md
@@ -0,0 +1,42 @@
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+
+# TsFileSelfCheck Tool
+IoTDB Server provides the TsFile self check tool. At present, the tool can check the basic format of the TsFile file, the correctness of TimeseriesMetadata, and the correctness and consistency of the Statistics stored in each part of the TsFile.
+
+## Use
+Step 1:Create an object instance of TsFileSelfCheckTool class.
+
+``` java
+TsFileSelfCheckTool tool = new TsFileSelfCheckTool();
+```
+
+Step 2:Call the check method of the self check tool. The first parameter path is the path of the TsFile to be checked. The second parameter is whether to check only the Magic String and Version Number at the beginning and end of TsFile.
+
+``` java
+tool.check(path, false);
+```
+
+* There are four return values of the check method.
+* The return value is 0, which means that the TsFile self check is error-free.
+* The return value is -1, which means that TsFile has inconsistencies in Statistics. There will be two specific exceptions, one is that the Statistics of TimeSeriesMetadata is inconsistent with the Statistics of the aggregated statistics of ChunkMetadata. The other is that the Statistics of ChunkMetadata is inconsistent with the Statistics of Page aggregation statistics in the Chunk indexed by it.
+* The return value is -2, which means that the TsFile version is not compatible.
+* The return value is -3, which means that the TsFile file does not exist in the given path.
\ No newline at end of file
diff --git a/docs/zh/UserGuide/System-Tools/TsFileSelfCheck-Tool.md b/docs/zh/UserGuide/System-Tools/TsFileSelfCheck-Tool.md
new file mode 100644
index 0000000..023006e
--- /dev/null
+++ b/docs/zh/UserGuide/System-Tools/TsFileSelfCheck-Tool.md
@@ -0,0 +1,42 @@
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+
+# TsFile 自检工具
+IoTDB Server 提供了 TsFile 自检工具,目前该工具可以检查 TsFile 文件中的基本格式、TimeseriesMetadata 的正确性以及 TsFile 中各部分存储的 Statistics 的正确性和一致性。
+
+## 使用
+第一步:创建一个 TsFileSelfCheckTool 类的对象。
+
+``` java
+TsFileSelfCheckTool tool = new TsFileSelfCheckTool();
+```
+
+第二步:调用自检工具的 check 方法。第一个参数 path 是要检测的 TsFile 的路径。第二个参数是是否只检测 TsFile 开头和结尾的 Magic String 和 Version Number。
+
+``` java
+tool.check(path, false);
+```
+
+* check 方法的返回值有四种。
+* 返回值为 0 表示 TsFile 自检无错。
+* 返回值为 -1 表示 TsFile 存在 Statistics 不一致问题。具体会有两种异常,一种是 TimeSeriesMetadata 的 Statistics 与其后面的 ChunkMetadata 的聚合统计的 Statistics 不一致。另一种是 ChunkMetadata 的 Statistics 与其索引的 Chunk 中的 Page 聚合统计的 Statistics 不一致。
+* 返回值为 -2 表示 TsFile 版本不兼容。
+* 返回值为 -3 表示给定路径不存在 TsFile 文件。
\ No newline at end of file
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileCheckStatus.java b/server/src/main/java/org/apache/iotdb/db/exception/TsFileTimeseriesMetadataException.java
similarity index 73%
copy from tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileCheckStatus.java
copy to server/src/main/java/org/apache/iotdb/db/exception/TsFileTimeseriesMetadataException.java
index 9bb5ea5..a923a5a 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileCheckStatus.java
+++ b/server/src/main/java/org/apache/iotdb/db/exception/TsFileTimeseriesMetadataException.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <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
@@ -16,11 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.db.exception;
 
-package org.apache.iotdb.tsfile.read;
+public class TsFileTimeseriesMetadataException extends Exception {
 
-public class TsFileCheckStatus {
-  public static final long COMPLETE_FILE = -1;
-  public static final long INCOMPATIBLE_FILE = -3;
-  public static final long FILE_NOT_FOUND = -4;
+  public TsFileTimeseriesMetadataException(String message) {
+    super(message);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/tools/TsFileSelfCheckTool.java b/server/src/main/java/org/apache/iotdb/db/tools/TsFileSelfCheckTool.java
new file mode 100644
index 0000000..1309fdf
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/tools/TsFileSelfCheckTool.java
@@ -0,0 +1,190 @@
+/*
+ * 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.tools;
+
+import org.apache.iotdb.db.exception.TsFileTimeseriesMetadataException;
+import org.apache.iotdb.tsfile.exception.TsFileStatisticsMistakesException;
+import org.apache.iotdb.tsfile.file.metadata.MetadataIndexEntry;
+import org.apache.iotdb.tsfile.file.metadata.MetadataIndexNode;
+import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
+import org.apache.iotdb.tsfile.file.metadata.enums.MetadataIndexNodeType;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+public class TsFileSelfCheckTool {
+
+  private static final Logger logger = LoggerFactory.getLogger(TsFileSelfCheckTool.class);
+
+  private Map<Long, Pair<Path, TimeseriesMetadata>> getTimeseriesMetadataMapWithReader(
+      TsFileSelfCheckToolReader reader) throws Exception {
+    Map<Long, Pair<Path, TimeseriesMetadata>> timeseriesMetadataMap = null;
+    timeseriesMetadataMap = reader.getAllTimeseriesMetadataWithOffset();
+    return timeseriesMetadataMap;
+  }
+
+  public Map<Long, Pair<Path, TimeseriesMetadata>> getTimeseriesMetadataMapWithPath(String filename)
+      throws IOException, TsFileTimeseriesMetadataException {
+    TsFileSelfCheckToolReader reader = new TsFileSelfCheckToolReader(filename);
+    Map<Long, Pair<Path, TimeseriesMetadata>> timeseriesMetadataMap = null;
+    try {
+      timeseriesMetadataMap = getTimeseriesMetadataMapWithReader(reader);
+    } catch (Exception e) {
+      logger.error("Error occurred while getting all TimeseriesMetadata with offset in TsFile.");
+      throw new TsFileTimeseriesMetadataException(
+          "Error occurred while getting all TimeseriesMetadata with offset in TsFile.");
+    }
+    return timeseriesMetadataMap;
+  }
+
+  /**
+   * @param filename The path of TsFile.
+   * @param fastFinish If true, the method will only check the format of head (Magic String TsFile,
+   *     Version Number) and tail (Magic String TsFile) of TsFile.
+   * @return There are four return values of the check method. The return value is 0, which means
+   *     that the TsFile self check is error-free. The return value is -1, which means that TsFile
+   *     has inconsistencies in Statistics. There will be two specific exceptions, one is that the
+   *     Statistics of TimeSeriesMetadata is inconsistent with the Statistics of the aggregated
+   *     statistics of ChunkMetadata. The other is that the Statistics of ChunkMetadata is
+   *     inconsistent with the Statistics of Page aggregation statistics in the Chunk indexed by it.
+   *     The return value is -2, which means that the TsFile version is not compatible. The return
+   *     value is -3, which means that the TsFile file does not exist in the given path.
+   */
+  public long check(String filename, boolean fastFinish)
+      throws IOException, TsFileStatisticsMistakesException, TsFileTimeseriesMetadataException {
+    logger.info("file path: " + filename);
+    TsFileSelfCheckToolReader reader = new TsFileSelfCheckToolReader(filename);
+    Map<Long, Pair<Path, TimeseriesMetadata>> timeseriesMetadataMap = null;
+    long res = -1;
+    try {
+      timeseriesMetadataMap = getTimeseriesMetadataMapWithReader(reader);
+      res = reader.selfCheckWithInfo(filename, fastFinish, timeseriesMetadataMap);
+    } catch (TsFileStatisticsMistakesException e) {
+      throw e;
+    } catch (Exception e) {
+      logger.error("Error occurred while getting all TimeseriesMetadata with offset in TsFile.");
+      throw new TsFileTimeseriesMetadataException(
+          "Error occurred while getting all TimeseriesMetadata with offset in TsFile.");
+    } finally {
+      reader.close();
+    }
+    return res;
+  }
+
+  private class TsFileSelfCheckToolReader extends TsFileSequenceReader {
+    public TsFileSelfCheckToolReader(String file) throws IOException {
+      super(file);
+    }
+
+    /**
+     * Traverse the metadata index from MetadataIndexEntry to get TimeseriesMetadatas
+     *
+     * @param metadataIndex MetadataIndexEntry
+     * @param buffer byte buffer
+     * @param deviceId String
+     * @param timeseriesMetadataMap map: deviceId -> timeseriesMetadata list
+     * @param needChunkMetadata deserialize chunk metadata list or not
+     */
+    private void generateMetadataIndexWithOffset(
+        long startOffset,
+        MetadataIndexEntry metadataIndex,
+        ByteBuffer buffer,
+        String deviceId,
+        MetadataIndexNodeType type,
+        Map<Long, Pair<Path, TimeseriesMetadata>> timeseriesMetadataMap,
+        boolean needChunkMetadata)
+        throws IOException {
+      try {
+        if (type.equals(MetadataIndexNodeType.LEAF_MEASUREMENT)) {
+          while (buffer.hasRemaining()) {
+            long pos = startOffset + buffer.position();
+            TimeseriesMetadata timeseriesMetadata =
+                TimeseriesMetadata.deserializeFrom(buffer, needChunkMetadata);
+            timeseriesMetadataMap.put(
+                pos,
+                new Pair<>(
+                    new Path(deviceId, timeseriesMetadata.getMeasurementId()), timeseriesMetadata));
+          }
+        } else {
+          // deviceId should be determined by LEAF_DEVICE node
+          if (type.equals(MetadataIndexNodeType.LEAF_DEVICE)) {
+            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);
+            generateMetadataIndexWithOffset(
+                metadataIndexNode.getChildren().get(i).getOffset(),
+                metadataIndexNode.getChildren().get(i),
+                nextBuffer,
+                deviceId,
+                metadataIndexNode.getNodeType(),
+                timeseriesMetadataMap,
+                needChunkMetadata);
+          }
+        }
+      } catch (BufferOverflowException e) {
+        throw e;
+      }
+    }
+
+    public Map<Long, Pair<Path, TimeseriesMetadata>> getAllTimeseriesMetadataWithOffset()
+        throws IOException {
+      if (tsFileMetaData == null) {
+        readFileMetadata();
+      }
+      MetadataIndexNode metadataIndexNode = tsFileMetaData.getMetadataIndex();
+      Map<Long, Pair<Path, TimeseriesMetadata>> timeseriesMetadataMap = new TreeMap<>();
+      List<MetadataIndexEntry> metadataIndexEntryList = metadataIndexNode.getChildren();
+      for (int i = 0; i < metadataIndexEntryList.size(); i++) {
+        MetadataIndexEntry metadataIndexEntry = metadataIndexEntryList.get(i);
+        long endOffset = tsFileMetaData.getMetadataIndex().getEndOffset();
+        if (i != metadataIndexEntryList.size() - 1) {
+          endOffset = metadataIndexEntryList.get(i + 1).getOffset();
+        }
+        ByteBuffer buffer = readData(metadataIndexEntry.getOffset(), endOffset);
+        generateMetadataIndexWithOffset(
+            metadataIndexEntry.getOffset(),
+            metadataIndexEntry,
+            buffer,
+            null,
+            metadataIndexNode.getNodeType(),
+            timeseriesMetadataMap,
+            false);
+      }
+      return timeseriesMetadataMap;
+    }
+  }
+}
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 947c1b7..45db458 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
@@ -134,6 +134,7 @@ public class TsFileSketchTool {
     printlnBoth(
         pw,
         "---------------------------------- TsFile Sketch End ----------------------------------");
+    reader.close();
     pw.close();
   }
 
diff --git a/server/src/test/java/org/apache/iotdb/db/tools/TsFileSelfCheckToolTest.java b/server/src/test/java/org/apache/iotdb/db/tools/TsFileSelfCheckToolTest.java
new file mode 100644
index 0000000..8082003
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/tools/TsFileSelfCheckToolTest.java
@@ -0,0 +1,239 @@
+/*
+ * 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.tools;
+
+import org.apache.iotdb.db.exception.TsFileTimeseriesMetadataException;
+import org.apache.iotdb.tsfile.exception.TsFileStatisticsMistakesException;
+import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
+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.LongStatistics;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.iotdb.tsfile.write.TsFileWriter;
+import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.Schema;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.*;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.*;
+
+public class TsFileSelfCheckToolTest {
+
+  String path =
+      "data"
+          .concat(File.separator)
+          .concat("data")
+          .concat(File.separator)
+          .concat("sequence")
+          .concat(File.separator)
+          .concat("root.sg1")
+          .concat(File.separator)
+          .concat("0")
+          .concat(File.separator)
+          .concat("0")
+          .concat(File.separator);
+  String device = "root.device_0";
+
+  private static final Logger logger = LoggerFactory.getLogger(TsFileSelfCheckToolTest.class);
+
+  public void setUp(String filePath) throws Exception {
+    try {
+      File f = FSFactoryProducer.getFSFactory().getFile(filePath);
+      if (f.exists() && !f.delete()) {
+        throw new RuntimeException("can not delete " + f.getAbsolutePath());
+      }
+
+      Schema schema = new Schema();
+
+      String sensorPrefix = "sensor_";
+      // the number of rows to include in the tablet
+      int rowNum = 1000000;
+      // the number of values to include in the tablet
+      int sensorNum = 10;
+
+      List<MeasurementSchema> measurementSchemas = new ArrayList<>();
+      // add measurements into file schema (all with INT64 data type)
+      for (int i = 0; i < sensorNum; i++) {
+        MeasurementSchema measurementSchema =
+            new MeasurementSchema(sensorPrefix + (i + 1), TSDataType.INT64, TSEncoding.TS_2DIFF);
+        measurementSchemas.add(measurementSchema);
+        schema.registerTimeseries(
+            new Path(device),
+            new MeasurementSchema(sensorPrefix + (i + 1), TSDataType.INT64, TSEncoding.TS_2DIFF));
+      }
+
+      // add measurements into TSFileWriter
+      try (TsFileWriter tsFileWriter = new TsFileWriter(f, schema)) {
+
+        // construct the tablet
+        Tablet tablet = new Tablet(device, measurementSchemas);
+
+        long[] timestamps = tablet.timestamps;
+        Object[] values = tablet.values;
+
+        long timestamp = 1;
+        long value = 1000000L;
+
+        for (int r = 0; r < rowNum; r++, value++) {
+          int row = tablet.rowSize++;
+          timestamps[row] = timestamp++;
+          for (int i = 0; i < sensorNum; i++) {
+            long[] sensor = (long[]) values[i];
+            sensor[row] = value;
+          }
+          // write Tablet to TsFile
+          if (tablet.rowSize == tablet.getMaxRowNumber()) {
+            tsFileWriter.write(tablet);
+            tablet.reset();
+          }
+        }
+        // write Tablet to TsFile
+        if (tablet.rowSize != 0) {
+          tsFileWriter.write(tablet);
+          tablet.reset();
+        }
+      }
+    } catch (Exception e) {
+      throw new Exception("meet error in TsFileWrite with tablet", e);
+    }
+  }
+
+  @Test
+  public void tsFileSelfCheckToolCompleteTest() throws Exception {
+    String fileName = "1-0-0-1.tsfile";
+    String filePath = path.concat(fileName);
+    setUp(filePath);
+    TsFileSelfCheckTool tool = new TsFileSelfCheckTool();
+    try {
+      tool.check(filePath, false);
+    } catch (IOException
+        | TsFileStatisticsMistakesException
+        | TsFileTimeseriesMetadataException e) {
+      fail(e.getMessage());
+    }
+    tearDown(filePath);
+  }
+
+  @Test
+  public void tsFileSelfCheckToolWithStatisticsModifiedTest()
+      throws IOException, TsFileTimeseriesMetadataException, Exception {
+    String fileName = "1-0-0-2.tsfile";
+    String filePath = path.concat(fileName);
+    setUp(filePath);
+    Map<Long, Pair<Path, TimeseriesMetadata>> timeseriesMetadataMap =
+        new TsFileSelfCheckTool().getTimeseriesMetadataMapWithPath(filePath);
+    for (Map.Entry<Long, Pair<Path, TimeseriesMetadata>> entry : timeseriesMetadataMap.entrySet()) {
+      TimeseriesMetadata timeseriesMetadata = entry.getValue().right;
+      Long pos = entry.getKey();
+      LongStatistics statistics = (LongStatistics) timeseriesMetadata.getStatistics();
+      statistics.initializeStats(666, 1999999, 1000000, 1999999, 0);
+
+      RandomAccessFile raf = new RandomAccessFile(filePath, "rw");
+      ByteArrayOutputStream bo = new ByteArrayOutputStream();
+      int serialLength = ReadWriteIOUtils.write(timeseriesMetadata.getTimeSeriesMetadataType(), bo);
+      serialLength += ReadWriteIOUtils.writeVar(timeseriesMetadata.getMeasurementId(), bo);
+      serialLength += ReadWriteIOUtils.write(timeseriesMetadata.getTSDataType(), bo);
+      serialLength +=
+          ReadWriteForEncodingUtils.writeUnsignedVarInt(
+              timeseriesMetadata.getDataSizeOfChunkMetaDataList(), bo);
+      serialLength += statistics.serialize(bo);
+      logger.info("serialLength: " + serialLength);
+      byte[] serialArr = bo.toByteArray();
+      raf.seek(pos);
+      raf.write(serialArr, 0, serialArr.length);
+      bo.close();
+      raf.close();
+
+      // We only modify one statistics of TimeseriesMetadata in TsFile to test the check method, so
+      // we break here
+      break;
+    }
+
+    TsFileSelfCheckTool tool = new TsFileSelfCheckTool();
+    try {
+      tool.check(filePath, false);
+      fail("No exception thrown.");
+    } catch (TsFileStatisticsMistakesException e) {
+      // In fact, what we are modifying is the Statistics of TimeseriesMetadata. It should be
+      // reported that
+      // TimeseriesMetadata is inconsistent with the Statistics of the subsequent ChunkMetadata
+      // aggregation statistics.
+      // But because the self check method first checks the aggregate statistics of ChunkMetadata
+      // and the page behind
+      // the chunk at its index position and TsFile is initialized to TimeseriesMetadata and
+      // followed by a
+      // ChunkMetadata, the Statistics of ChunkMetadata here uses the Statistics of
+      // TimeseriesMetadata.
+      // Therefore, Chunk's Statistics error will be reported.
+      assertEquals("Chunk exists statistics mistakes at position 22", e.getMessage());
+    }
+    tearDown(filePath);
+  }
+
+  @Test
+  public void tsFileSelfCheckToolWithRandomModifiedTest() throws IOException, Exception {
+
+    String fileName = "1-0-0-3.tsfile";
+    String filePath = path.concat(fileName);
+    setUp(filePath);
+
+    RandomAccessFile raf = new RandomAccessFile(filePath, "rw");
+    ByteArrayOutputStream bo = new ByteArrayOutputStream();
+    ReadWriteIOUtils.write(100, bo);
+    byte[] serialArr = bo.toByteArray();
+    // timeseriesMetadata begins at 878364
+    // randomly modify timeseriesMetadata region
+    raf.seek(878375);
+    raf.write(serialArr, 0, serialArr.length);
+    bo.close();
+    raf.close();
+
+    TsFileSelfCheckTool tool = new TsFileSelfCheckTool();
+    try {
+      tool.check(filePath, false);
+      fail("No exception thrown.");
+    } catch (TsFileTimeseriesMetadataException e) {
+      assertEquals(
+          "Error occurred while getting all TimeseriesMetadata with offset in TsFile.",
+          e.getMessage());
+    }
+    tearDown(filePath);
+  }
+
+  public void tearDown(String filePath) {
+    try {
+      FileUtils.forceDelete(new File(filePath));
+    } catch (IOException e) {
+      fail(e.getMessage());
+    }
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileCheckStatus.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/exception/TsFileStatisticsMistakesException.java
similarity index 78%
copy from tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileCheckStatus.java
copy to tsfile/src/main/java/org/apache/iotdb/tsfile/exception/TsFileStatisticsMistakesException.java
index 9bb5ea5..0a8e309 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileCheckStatus.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/exception/TsFileStatisticsMistakesException.java
@@ -16,11 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.tsfile.exception;
 
-package org.apache.iotdb.tsfile.read;
+public class TsFileStatisticsMistakesException extends TsFileRuntimeException {
 
-public class TsFileCheckStatus {
-  public static final long COMPLETE_FILE = -1;
-  public static final long INCOMPATIBLE_FILE = -3;
-  public static final long FILE_NOT_FOUND = -4;
+  public TsFileStatisticsMistakesException(String message) {
+    super(message);
+  }
 }
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 593db6c..440844f 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
@@ -28,6 +28,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
+import java.util.Objects;
 
 /** Statistics for string type. */
 public class BinaryStatistics extends Statistics<Binary> {
@@ -216,6 +217,20 @@ public class BinaryStatistics extends Statistics<Binary> {
   }
 
   @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    if (!super.equals(o)) return false;
+    BinaryStatistics that = (BinaryStatistics) o;
+    return Objects.equals(firstValue, that.firstValue) && Objects.equals(lastValue, that.lastValue);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(super.hashCode(), firstValue, lastValue);
+  }
+
+  @Override
   public String toString() {
     return super.toString() + " [firstValue:" + firstValue + ",lastValue:" + lastValue + "]";
   }
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 83ed06f..05a46aa 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
@@ -27,6 +27,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
+import java.util.Objects;
 
 public class BooleanStatistics extends Statistics<Boolean> {
 
@@ -222,6 +223,22 @@ public class BooleanStatistics extends Statistics<Boolean> {
   }
 
   @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    if (!super.equals(o)) return false;
+    BooleanStatistics that = (BooleanStatistics) o;
+    return firstValue == that.firstValue
+        && lastValue == that.lastValue
+        && sumValue == that.sumValue;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(super.hashCode(), firstValue, lastValue, sumValue);
+  }
+
+  @Override
   public String toString() {
     return super.toString()
         + " [firstValue="
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 11d7dbd..d8a5831 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
@@ -27,6 +27,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
+import java.util.Objects;
 
 public class DoubleStatistics extends Statistics<Double> {
 
@@ -263,6 +264,25 @@ public class DoubleStatistics extends Statistics<Double> {
   }
 
   @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    if (!super.equals(o)) return false;
+    double e = 0.00001;
+    DoubleStatistics that = (DoubleStatistics) o;
+    return Math.abs(that.minValue - minValue) < e
+        && Math.abs(that.maxValue - maxValue) < e
+        && Math.abs(that.firstValue - firstValue) < e
+        && Math.abs(that.lastValue - lastValue) < e
+        && Math.abs(that.sumValue - sumValue) < e;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(super.hashCode(), minValue, maxValue, firstValue, lastValue, sumValue);
+  }
+
+  @Override
   public String toString() {
     return super.toString()
         + " [minValue:"
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 2870ed2..3380d5d 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
@@ -27,6 +27,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
+import java.util.Objects;
 
 /** Statistics for float type. */
 public class FloatStatistics extends Statistics<Float> {
@@ -255,6 +256,25 @@ public class FloatStatistics extends Statistics<Float> {
   }
 
   @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    if (!super.equals(o)) return false;
+    double e = 0.00001;
+    FloatStatistics that = (FloatStatistics) o;
+    return Math.abs(that.minValue - minValue) < e
+        && Math.abs(that.maxValue - maxValue) < e
+        && Math.abs(that.firstValue - firstValue) < e
+        && Math.abs(that.lastValue - lastValue) < e
+        && Math.abs(that.sumValue - sumValue) < e;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(super.hashCode(), minValue, maxValue, firstValue, lastValue, sumValue);
+  }
+
+  @Override
   public String toString() {
     return super.toString()
         + " [minValue:"
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 831a697..3cc2dcd 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
@@ -27,6 +27,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
+import java.util.Objects;
 
 /** Statistics for int type. */
 public class IntegerStatistics extends Statistics<Integer> {
@@ -255,6 +256,24 @@ public class IntegerStatistics extends Statistics<Integer> {
   }
 
   @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    if (!super.equals(o)) return false;
+    IntegerStatistics that = (IntegerStatistics) o;
+    return minValue == that.minValue
+        && maxValue == that.maxValue
+        && firstValue == that.firstValue
+        && lastValue == that.lastValue
+        && sumValue == that.sumValue;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(super.hashCode(), minValue, maxValue, firstValue, lastValue, sumValue);
+  }
+
+  @Override
   public String toString() {
     return super.toString()
         + " [minValue:"
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 849bd41..80741c2 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
@@ -27,6 +27,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
+import java.util.Objects;
 
 public class LongStatistics extends Statistics<Long> {
 
@@ -264,6 +265,24 @@ public class LongStatistics extends Statistics<Long> {
   }
 
   @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    if (!super.equals(o)) return false;
+    LongStatistics that = (LongStatistics) o;
+    return minValue == that.minValue
+        && maxValue == that.maxValue
+        && firstValue == that.firstValue
+        && lastValue == that.lastValue
+        && Double.compare(that.sumValue, sumValue) == 0;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(super.hashCode(), minValue, maxValue, firstValue, lastValue, sumValue);
+  }
+
+  @Override
   public String toString() {
     return super.toString()
         + " [minValue:"
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileCheckStatus.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileCheckStatus.java
index 9bb5ea5..d63aa83 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileCheckStatus.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileCheckStatus.java
@@ -20,7 +20,8 @@
 package org.apache.iotdb.tsfile.read;
 
 public class TsFileCheckStatus {
-  public static final long COMPLETE_FILE = -1;
-  public static final long INCOMPATIBLE_FILE = -3;
-  public static final long FILE_NOT_FOUND = -4;
+  public static final long COMPLETE_FILE = 0;
+  public static final long FILE_EXISTS_MISTAKES = -1;
+  public static final long INCOMPATIBLE_FILE = -2;
+  public static final long FILE_NOT_FOUND = -3;
 }
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 d99b3ba..bd42af8 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
@@ -23,6 +23,7 @@ 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.TsFileRuntimeException;
+import org.apache.iotdb.tsfile.exception.TsFileStatisticsMistakesException;
 import org.apache.iotdb.tsfile.file.MetaMarker;
 import org.apache.iotdb.tsfile.file.header.ChunkGroupHeader;
 import org.apache.iotdb.tsfile.file.header.ChunkHeader;
@@ -1403,6 +1404,140 @@ public class TsFileSequenceReader implements AutoCloseable {
   }
 
   /**
+   * Self Check the file and return whether the file is safe.
+   *
+   * @param filename the path of file
+   * @param fastFinish if true, the method will only check the format of head (Magic String TsFile,
+   *     Version Number) and tail (Magic String TsFile) of TsFile.
+   * @return the status of TsFile
+   */
+  public long selfCheckWithInfo(
+      String filename,
+      boolean fastFinish,
+      Map<Long, Pair<Path, TimeseriesMetadata>> timeseriesMetadataMap)
+      throws IOException, TsFileStatisticsMistakesException {
+    String message = " exists statistics mistakes at position ";
+    File checkFile = FSFactoryProducer.getFSFactory().getFile(filename);
+    if (!checkFile.exists()) {
+      return TsFileCheckStatus.FILE_NOT_FOUND;
+    }
+    long fileSize = checkFile.length();
+    logger.info("file length: " + fileSize);
+
+    int headerLength = TSFileConfig.MAGIC_STRING.getBytes().length + Byte.BYTES;
+    if (fileSize < headerLength) {
+      return TsFileCheckStatus.INCOMPATIBLE_FILE;
+    }
+    try {
+      if (!TSFileConfig.MAGIC_STRING.equals(readHeadMagic())
+          || (TSFileConfig.VERSION_NUMBER != readVersionNumber())) {
+        return TsFileCheckStatus.INCOMPATIBLE_FILE;
+      }
+      tsFileInput.position(headerLength);
+      if (isComplete()) {
+        loadMetadataSize();
+        if (fastFinish) {
+          return TsFileCheckStatus.COMPLETE_FILE;
+        }
+      }
+    } catch (IOException e) {
+      logger.error("Error occurred while fast checking TsFile.");
+      throw e;
+    }
+    for (Map.Entry<Long, Pair<Path, TimeseriesMetadata>> entry : timeseriesMetadataMap.entrySet()) {
+      TimeseriesMetadata timeseriesMetadata = entry.getValue().right;
+      TSDataType dataType = timeseriesMetadata.getTSDataType();
+      Statistics<? extends Serializable> timeseriesMetadataSta = timeseriesMetadata.getStatistics();
+      Statistics<? extends Serializable> chunkMetadatasSta = Statistics.getStatsByType(dataType);
+      for (IChunkMetadata chunkMetadata : getChunkMetadataList(entry.getValue().left)) {
+        long tscheckStatus = TsFileCheckStatus.COMPLETE_FILE;
+        try {
+          tscheckStatus = checkChunkAndPagesStatistics(chunkMetadata);
+        } catch (IOException e) {
+          logger.error("Error occurred while checking the statistics of chunk and its pages");
+          throw e;
+        }
+        if (tscheckStatus == TsFileCheckStatus.FILE_EXISTS_MISTAKES) {
+          throw new TsFileStatisticsMistakesException(
+              "Chunk" + message + chunkMetadata.getOffsetOfChunkHeader());
+        }
+        chunkMetadatasSta.mergeStatistics(chunkMetadata.getStatistics());
+      }
+      if (!timeseriesMetadataSta.equals(chunkMetadatasSta)) {
+        long timeseriesMetadataPos = entry.getKey();
+        throw new TsFileStatisticsMistakesException(
+            "TimeseriesMetadata" + message + timeseriesMetadataPos);
+      }
+    }
+    return TsFileCheckStatus.COMPLETE_FILE;
+  }
+
+  public long checkChunkAndPagesStatistics(IChunkMetadata chunkMetadata) throws IOException {
+    long offsetOfChunkHeader = chunkMetadata.getOffsetOfChunkHeader();
+    tsFileInput.position(offsetOfChunkHeader);
+    byte marker = this.readMarker();
+    ChunkHeader chunkHeader = this.readChunkHeader(marker);
+    TSDataType dataType = chunkHeader.getDataType();
+    Statistics<? extends Serializable> chunkStatistics = Statistics.getStatsByType(dataType);
+    int dataSize = chunkHeader.getDataSize();
+    if (((byte) (chunkHeader.getChunkType() & 0x3F)) == 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);
+    }
+    if (chunkMetadata.getStatistics().equals(chunkStatistics)) {
+      return TsFileCheckStatus.COMPLETE_FILE;
+    }
+    return TsFileCheckStatus.FILE_EXISTS_MISTAKES;
+  }
+
+  /**
    * get ChunkMetaDatas of given path, and throw exception if path not exists
    *
    * @param path timeseries path