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 2022/08/12 10:29:59 UTC

[GitHub] [iotdb] qiaojialin commented on a diff in pull request #6782: [IOTDB-3771] Fix cannot take snapshot when the data dir and snapshot dir is on different disk

qiaojialin commented on code in PR #6782:
URL: https://github.com/apache/iotdb/pull/6782#discussion_r944328796


##########
server/src/main/java/org/apache/iotdb/db/engine/snapshot/SnapshotTaker.java:
##########
@@ -117,83 +210,154 @@ public boolean takeFullSnapshot(String snapshotDirPath, boolean flushBeforeSnaps
     } finally {
       manager.readUnlock();
     }
+    return true;
+  }
 
-    LOGGER.info(
-        "Successfully take snapshot for {}-{}, snapshot directory is {}",
-        dataRegion.getLogicalStorageGroupName(),
-        dataRegion.getDataRegionId(),
-        snapshotDirPath);
+  private boolean createSnapshotInAnotherFs(File snapshotDir) {
+    try {
+      snapshotLogger.logSnapshotType(SnapshotLogger.SnapshotType.REMOTE_FS);
+    } catch (IOException e) {
+      LOGGER.error("Fail to create snapshot", e);
+      cleanUpWhenFail(snapshotDir);
+      return false;
+    }
+    for (String dataDir : IoTDBDescriptor.getInstance().getConfig().getDataDirs()) {
+      File seqDir =
+          new File(
+              dataDir,
+              "sequence"
+                  + File.separator
+                  + dataRegion.getStorageGroupName()
+                  + File.separator
+                  + dataRegion.getDataRegionId());
+      File unseqDir =
+          new File(
+              dataDir,
+              "unsequence"
+                  + File.separator
+                  + dataRegion.getStorageGroupName()
+                  + File.separator
+                  + dataRegion.getDataRegionId());
+      File localSeqSnapshotDir =
+          new File(
+              dataDir,
+              "snapshot"
+                  + File.separator
+                  + "sequence"
+                  + File.separator
+                  + snapshotDir.getName()
+                  + File.separator
+                  + dataRegion.getStorageGroupName()
+                  + File.separator
+                  + dataRegion.getDataRegionId());
+      File localUnseqSnapshotDir =
+          new File(
+              dataDir,
+              "snapshot"
+                  + File.separator
+                  + "unsequence"
+                  + File.separator
+                  + snapshotDir.getName()
+                  + File.separator
+                  + dataRegion.getStorageGroupName()
+                  + File.separator
+                  + dataRegion.getDataRegionId());
+      if (!localSeqSnapshotDir.mkdirs()) {
+        LOGGER.warn("Failed to create local snapshot dir {}", localSeqSnapshotDir);
+        return false;
+      }
+      List<Long> timePartitions = dataRegion.getTimePartitions();
+      TsFileManager manager = dataRegion.getTsFileManager();
+      manager.readLock();
+      for (long timePartition : timePartitions) {
+        createSnapshotForTimePartition(snapshotDir, seqDir, localSeqSnapshotDir, timePartition);
+        createSnapshotForTimePartition(snapshotDir, unseqDir, localUnseqSnapshotDir, timePartition);
+      }
+    }
 
     return true;
   }
 
-  private List<String> getAllDataDirOfOnePartition(boolean sequence, long timePartition) {
-    String[] dataDirs = IoTDBDescriptor.getInstance().getConfig().getDataDirs();
-    List<String> resultDirs = new LinkedList<>();
+  private void createSnapshotForTimePartition(
+      File snapshotDir, File dataDir, File localSeqSnapshotDir, long timePartition) {
+    File seqTimePartitionDir = new File(dataDir, String.valueOf(timePartition));
+    File seqTimePartitionSnapshotDir = new File(localSeqSnapshotDir, String.valueOf(timePartition));
 
-    for (String dataDir : dataDirs) {
-      resultDirs.add(
-          dataDir
-              + File.separator
-              + (sequence
-                  ? IoTDBConstant.SEQUENCE_FLODER_NAME
-                  : IoTDBConstant.UNSEQUENCE_FLODER_NAME)
-              + File.separator
-              + dataRegion.getLogicalStorageGroupName()
-              + File.separator
-              + dataRegion.getDataRegionId()
-              + File.separator
-              + timePartition
-              + File.separator);
+    try {
+      createFileSnapshotToTargetOne(seqTimePartitionDir, seqTimePartitionSnapshotDir);
+    } catch (IOException e) {
+      LOGGER.error(
+          "Failed to create snapshot for {}-{}",
+          dataRegion.getStorageGroupName(),
+          dataRegion.getDataRegionId(),
+          e);
+      cleanUpWhenFail(snapshotDir);
     }
-    return resultDirs;
   }
 
-  private void createFileSnapshot(List<String> sourceDirPaths, boolean sequence, long timePartition)
-      throws IOException {
-    File timePartitionDir =
-        new File(sequence ? seqBaseDir : unseqBaseDir, String.valueOf(timePartition));
-    if (!timePartitionDir.exists() && !timePartitionDir.mkdirs()) {
-      throw new IOException(
-          String.format("%s not exists and cannot create it", timePartitionDir.getAbsolutePath()));
+  private void createFileSnapshotToTargetOne(File sourceDir, File targetDir) throws IOException {
+    File[] files =
+        sourceDir.listFiles(
+            (dir, name) ->
+                name.endsWith(TsFileConstant.TSFILE_SUFFIX)
+                    || name.endsWith(TsFileResource.RESOURCE_SUFFIX)
+                    || name.endsWith(ModificationFile.FILE_SUFFIX)
+                    || name.endsWith(ModificationFile.COMPACTION_FILE_SUFFIX)
+                    || name.endsWith(CompactionLogger.INNER_COMPACTION_LOG_NAME_SUFFIX)
+                    || name.endsWith(CompactionLogger.CROSS_COMPACTION_LOG_NAME_SUFFIX)
+                    || name.endsWith(IoTDBConstant.INNER_COMPACTION_TMP_FILE_SUFFIX)
+                    || name.endsWith(IoTDBConstant.CROSS_COMPACTION_TMP_FILE_SUFFIX));

Review Comment:
   move to a unified class: StorageEngineFile



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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org