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 2021/01/19 02:46:26 UTC

[GitHub] [iotdb] jt2594838 commented on a change in pull request #2184: [IOTDB-1081] New TsFile Format

jt2594838 commented on a change in pull request #2184:
URL: https://github.com/apache/iotdb/pull/2184#discussion_r559284013



##########
File path: server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java
##########
@@ -334,18 +342,117 @@ private void checkUnClosedTsFileV2InFolders(List<String> folders) {
           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);
+            // If upgrading from v0.11.2 to v0.12, there may be some Unclosed merging files.
+            // We have to delete these files before upgrading
+            File[] unmergedTsfiles = fsFactory
+                .listFilesBySuffix(partitionDir.toString(), "0" + TsFileConstant.TSFILE_SUFFIX);
+            File[] unmergedResources = fsFactory
+                .listFilesBySuffix(partitionDir.toString(), "0" + TsFileResource.RESOURCE_SUFFIX);
+            if (unmergedTsfiles.length != unmergedResources.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);
+            }
+            deleteMergeingTsFiles(tsfiles, resources);
           }
         }
       }
     }
   }
+
+  private void deleteMergeingTsFiles(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)) {
+        if (!tsfile.delete()) {
+          logger.error("Failed to delete merging tsfile {}", tsfile);
+          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;
+        }
+        File upgradeFolder = fsFactory.getFile(storageGroup, IoTDBConstant.UPGRADE_FOLDER_NAME);
+        // 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());
+        }
+        for (File partitionDir : storageGroup.listFiles()) {
+          if (!partitionDir.isDirectory() || 
+              partitionDir.getName().equals(IoTDBConstant.UPGRADE_FOLDER_NAME)) {
+            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);
+          // move the old files to upgrade folder if exists
+          if (oldTsfileArray.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());
+            }

Review comment:
       It seems `upgradeFolder` has been created in line402.

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -1748,10 +1717,39 @@ private void upgradeTsFileResourceCallBack(TsFileResource tsFileResource) {
           }
         }
       }
-      if (StorageEngine.getInstance().countUpgradeFiles() == 0) {
-        UpgradeSevice.getINSTANCE().stop();
+//      if (StorageEngine.getInstance().countUpgradeFiles() == 0) {
+//        UpgradeSevice.getINSTANCE().stop();
+//      }

Review comment:
       Remove this if it is truly unused.

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java
##########
@@ -352,6 +352,25 @@ public static int write(String s, OutputStream outputStream) throws IOException
     return len;
   }
 
+  /**
+   * 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();

Review comment:
       Use `TsFileConfig.STRING_CHARSET`.

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TimeseriesMetadata.java
##########
@@ -27,10 +27,17 @@
 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;

Review comment:
       How about changing it to a boolean and using a more accurate name?

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java
##########
@@ -142,31 +142,32 @@ public TSDataType getDataType() {
    * @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 (timeseriesMetadata.getTimeSeriesMetadataType() != 0) {

Review comment:
       Better to wrap this in a more semantic way.

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/ChunkHeader.java
##########
@@ -33,101 +34,119 @@
 
 public class ChunkHeader {
 
+
+  /**
+   * 1 means this chunk has more than one page, so each page has its own page statistic 4 means this
+   * chunk has only one page, and this page has no page statistic
+   */
+  private byte chunkType;

Review comment:
       Better to use some final static fields to explain this. And is it really necessary to use the numbers from Marker? It is confusing when you have 1 and 4, but there are no 2 and 3.

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -1748,10 +1717,39 @@ private void upgradeTsFileResourceCallBack(TsFileResource tsFileResource) {
           }
         }
       }
-      if (StorageEngine.getInstance().countUpgradeFiles() == 0) {
-        UpgradeSevice.getINSTANCE().stop();
+//      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();
+        UpgradeLog.writeUpgradeLogFile(
+            resource.getTsFile().getAbsolutePath() + "," + UpgradeCheckStatus.UPGRADE_SUCCESS);

Review comment:
       Just to confirm, what if the storage group name contains a comma?

##########
File path: server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java
##########
@@ -319,118 +104,133 @@ private ByteBuffer readData(long position, int size) throws IOException {
    * @throws IOException, WriteProcessException
    */
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  public void upgradeFile(List<TsFileResource> upgradedResources)
+  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:
+          case MetaMarker.CHUNK_GROUP_HEADER:
             // this is the footer of a ChunkGroup.
-            ChunkGroupFooter chunkGroupFooter = this.readChunkGroupFooter();
+            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:
+            reader.readVersion();
+            // write plan indices
+            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));
       }
     } 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());
+  private boolean checkIfNeedToDecode(TSDataType dataType, TSEncoding encoding,
+      PageHeader pageHeader) {
+    return dataType == TSDataType.BOOLEAN ||
+        dataType == TSDataType.TEXT ||
+        (dataType == TSDataType.INT32 && encoding == TSEncoding.PLAIN) ||

Review comment:
       It would be better to add a comment and explain why such data types need decoding.

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
##########
@@ -567,13 +576,14 @@ private int binarySearchInTimeseriesMetadataList(List<TimeseriesMetadata> timese
    */
   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<>();

Review comment:
       If the result is read-only, Collections.emptyMap should be enough.

##########
File path: 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_FOOTER.
+   *  @param deviceID       device ID
+   *
+   */
+  public ChunkGroupHeader(String deviceID) {
+    this.deviceID = deviceID;
+    this.serializedSize = getSerializedSize(deviceID);
+  }
+
+  public static int getSerializedSize(String deviceID) {
+    int length = deviceID.getBytes(TSFileConfig.STRING_CHARSET).length;
+    return Byte.BYTES + ReadWriteForEncodingUtils.varIntSize(length) + length;
+  }

Review comment:
       `String.getBytes` is a bit wasteful here as the conversion result is discarded immediately, but I think maybe you can make this method non-static and cache the returned `byte[]` in this object, and use it when `serializeTo` is called.
   The same technique may be applied to other structures where string length is calculated before actually serialized.




----------------------------------------------------------------
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