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/05/18 09:07:43 UTC

[GitHub] [iotdb] THUMarkLau commented on a diff in pull request #5923: [IOTDB-2977] Take and load snapshot for DataRegionStateMachine

THUMarkLau commented on code in PR #5923:
URL: https://github.com/apache/iotdb/pull/5923#discussion_r875656399


##########
server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java:
##########
@@ -1055,6 +1055,23 @@ protected void getSeriesSchemas(InsertPlan insertPlan, DataRegion processor)
     }
   }
 
+  public String getSystemDir() {
+    return systemDir;

Review Comment:
   Resolve, I use `IoTDBDescriptor.getInstance().getConfig().getSystemDir()` instead.



##########
server/src/main/java/org/apache/iotdb/db/engine/snapshot/SnapshotTaker.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.engine.snapshot;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.compaction.log.CompactionLogger;
+import org.apache.iotdb.db.engine.modification.ModificationFile;
+import org.apache.iotdb.db.engine.snapshot.exception.DirectoryNotLegalException;
+import org.apache.iotdb.db.engine.storagegroup.DataRegion;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * SnapshotTaker takes data snapshot for a DataRegion in one time. It does so by creating hard link
+ * for files or copying them. SnapshotTaker supports two different ways of snapshot: Full Snapshot
+ * and Incremental Snapshot. The former takes a snapshot for all files in an empty directory, and
+ * the latter takes a snapshot based on the snapshot that took before.
+ */
+public class SnapshotTaker {
+  private static final Logger LOGGER = LoggerFactory.getLogger(SnapshotTaker.class);
+  private final DataRegion dataRegion;
+  public static String SNAPSHOT_FILE_INFO_SEP_STR = "_";
+
+  public SnapshotTaker(DataRegion dataRegion) {
+    this.dataRegion = dataRegion;
+  }
+
+  public boolean takeFullSnapshot(String snapshotDirPath)
+      throws DirectoryNotLegalException, IOException {
+    File snapshotDir = new File(snapshotDirPath);
+    if (snapshotDir.exists()
+        && snapshotDir.listFiles() != null
+        && snapshotDir.listFiles().length > 0) {
+      // the directory should be empty or not exists
+      throw new DirectoryNotLegalException(
+          String.format("%s already exists and is not empty", snapshotDirPath));
+    }
+
+    if (!snapshotDir.exists() && !snapshotDir.mkdirs()) {
+      throw new IOException(String.format("Failed to create directory %s", snapshotDir));
+    }
+
+      dataRegion.syncCloseAllWorkingTsFileProcessors();
+
+    List<Long> timePartitions = dataRegion.getTimePartitions();
+    for (Long timePartition : timePartitions) {
+      List<String> seqDataDirs = getAllDataDirOfOnePartition(true, timePartition);
+
+      try {
+        createFileSnapshot(seqDataDirs, snapshotDir, true, timePartition);
+      } catch (IOException e) {
+        LOGGER.error("Fail to create snapshot", e);
+        return false;

Review Comment:
   Resolve



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