You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2019/03/20 06:56:50 UTC

[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #100: [IOTDB-39]Refactor auto repair for TsFile reader and TsFile writer

jt2594838 commented on a change in pull request #100: [IOTDB-39]Refactor auto repair for TsFile  reader and TsFile writer
URL: https://github.com/apache/incubator-iotdb/pull/100#discussion_r267203489
 
 

 ##########
 File path: tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
 ##########
 @@ -427,4 +429,152 @@ public int readRaw(long position, int length, ByteBuffer target) throws IOExcept
         .readAsPossible(tsFileInput.wrapAsFileChannel(), target, position, length);
   }
 
+  /**
+   * Self Check the file and return the position before where the data is safe.
+   *
+   * @param newSchema @OUT.  the measurement schema in the file will be added into
+   * this parameter.
+   * @param newMetaData @OUT can not be null, the chunk group metadta in the file will be added into
+   * this parameter.
+   * @param fastFinish if true and the file is complete, then newSchema and newMetaData parameter
+   * will be not modified.
+   * @return the position of the file that is fine. All data after the position in the file should
+   * be truncated.
+   */
+  public long selfCheck(Map<String, MeasurementSchema> newSchema,
+      List<ChunkGroupMetaData> newMetaData, boolean fastFinish) throws IOException {
+    File checkFile = new File(this.file);
+    long fileSize;
+    if (!checkFile.exists()) {
+      return TsFileCheckStatus.FILE_NOT_FOUND;
+    } else {
+      fileSize = checkFile.length();
+    }
+    ChunkMetaData currentChunk;
+    String measurementID;
+    TSDataType dataType;
+    long fileOffsetOfChunk;
+    long startTimeOfChunk = 0;
+    long endTimeOfChunk = 0;
+    long numOfPoints = 0;
+
+    ChunkGroupMetaData currentChunkGroup;
+    List<ChunkMetaData> chunks = null;
+    String deviceID;
+    long startOffsetOfChunkGroup = 0;
+    long endOffsetOfChunkGroup;
+    long versionOfChunkGroup = 0;
+    boolean haveReadAnUnverifiedGroupFooter = false;
+    boolean newGroup = true;
+
+    if (fileSize < TSFileConfig.MAGIC_STRING.length()) {
+      return TsFileCheckStatus.INCOMPATIBLE_FILE;
+    }
+    String magic = readHeadMagic(true);
+    if (!magic.equals(TSFileConfig.MAGIC_STRING)) {
+      return TsFileCheckStatus.INCOMPATIBLE_FILE;
+    }
+
+    if (fileSize == TSFileConfig.MAGIC_STRING.length()) {
+      return TsFileCheckStatus.ONLY_MAGIC_HEAD;
+    } else if (readTailMagic().equals(magic)) {
+      loadMetadataSize();
+      if (fastFinish) {
+        return TsFileCheckStatus.COMPLETE_FILE;
+      }
+    }
+
+    // not a complete file, we will recover it...
+    long truncatedPosition = magicStringBytes.length;
+    boolean goon = true;
+    byte marker;
+    try {
+      while (goon && (marker = this.readMarker()) != MetaMarker.SEPARATOR) {
+        switch (marker) {
+          case MetaMarker.CHUNK_HEADER:
+            //this is a chunk.
+            if (haveReadAnUnverifiedGroupFooter) {
+              //now we are sure that the last ChunkGroupFooter is complete.
+              haveReadAnUnverifiedGroupFooter = false;
+              truncatedPosition = this.position() - 1;
+              newGroup = true;
+            }
+            if (newGroup) {
+              chunks = new ArrayList<>();
+              startOffsetOfChunkGroup = this.position() - 1;
+              newGroup = false;
+            }
+            //if there is something wrong with a chunk, we will drop this part of data
+            // (the whole ChunkGroup)
+            ChunkHeader header = this.readChunkHeader();
+            measurementID = header.getMeasurementID();
+            if (newSchema != null) {
+              newSchema.putIfAbsent(measurementID,
+                  new MeasurementSchema(measurementID, header.getDataType(),
+                      header.getEncodingType(), header.getCompressionType()));
+            }
+            dataType = header.getDataType();
+            fileOffsetOfChunk = this.position() - 1;
+            if (header.getNumOfPages() > 0) {
+              PageHeader pageHeader = this.readPageHeader(header.getDataType());
+              numOfPoints += pageHeader.getNumOfValues();
+              startTimeOfChunk = pageHeader.getMinTimestamp();
+              endTimeOfChunk = pageHeader.getMaxTimestamp();
+              this.skipPageData(pageHeader);
+            }
+            for (int j = 1; j < header.getNumOfPages() - 1; j++) {
+              //a new Page
+              PageHeader pageHeader = this.readPageHeader(header.getDataType());
+              this.skipPageData(pageHeader);
+            }
+            if (header.getNumOfPages() > 1) {
+              PageHeader pageHeader = this.readPageHeader(header.getDataType());
+              endTimeOfChunk = pageHeader.getMaxTimestamp();
+              this.skipPageData(pageHeader);
+            }
+            currentChunk = new ChunkMetaData(measurementID, dataType, fileOffsetOfChunk,
+                startTimeOfChunk, endTimeOfChunk);
+            currentChunk.setNumOfPoints(numOfPoints);
+            chunks.add(currentChunk);
+            numOfPoints = 0;
+            break;
+          case MetaMarker.CHUNK_GROUP_FOOTER:
+            //this is a chunk group
+            //if there is something wrong with the chunkGroup Footer, we will drop this part of data
+            //because we can not guarantee the correction of the deviceId.
+            ChunkGroupFooter chunkGroupFooter = this.readChunkGroupFooter();
+            deviceID = chunkGroupFooter.getDeviceID();
+            endOffsetOfChunkGroup = this.position();
+            currentChunkGroup = new ChunkGroupMetaData(deviceID, chunks, startOffsetOfChunkGroup);
+            currentChunkGroup.setEndOffsetOfChunkGroup(endOffsetOfChunkGroup);
+            currentChunkGroup.setVersion(versionOfChunkGroup++);
+            newMetaData.add(currentChunkGroup);
+            // though we have read the current ChunkMetaData from Disk, it may be incomplete.
+            // because if the file only loses one byte, the ChunkMetaData.deserialize() returns ok,
+            // while the last filed of the ChunkMetaData is incorrect.
+            // So, only reading the next MASK, can make sure that this ChunkMetaData is complete.
+            haveReadAnUnverifiedGroupFooter = true;
+            break;
+
+          default:
+            // it is impossible that we read an incorrect data.
+            MetaMarker.handleUnexpectedMarker(marker);
+            goon = false;
+        }
+      }
+      //now we read the tail of the file, so we are sure that the last ChunkGroupFooter is complete.
+      truncatedPosition = this.position() - 1;
 
 Review comment:
   What does this 1 mean if you have read all markers?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services