You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ratis.apache.org by GitBox <gi...@apache.org> on 2022/08/24 18:35:40 UTC

[GitHub] [ratis] szetszwo commented on a diff in pull request #718: RATIS-1677. Do not auto format RaftStorage in RECOVER.

szetszwo commented on code in PR #718:
URL: https://github.com/apache/ratis/pull/718#discussion_r954157445


##########
ratis-server/src/main/java/org/apache/ratis/server/storage/StorageImplUtils.java:
##########
@@ -17,38 +17,157 @@
  */
 package org.apache.ratis.server.storage;
 
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.server.RaftServerConfigKeys;
-import org.apache.ratis.util.IOUtils;
-import org.apache.ratis.util.JavaUtils;
-import org.apache.ratis.util.TimeDuration;
+import org.apache.ratis.server.RaftServerConfigKeys.Log;
+import org.apache.ratis.server.storage.RaftStorage.StartupOption;
+import org.apache.ratis.util.Preconditions;
+import org.apache.ratis.util.SizeInBytes;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.concurrent.TimeUnit;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static org.apache.ratis.server.RaftServer.Division.LOG;
 
 public final class StorageImplUtils {
+  private static final File[] EMPTY_FILE_ARRAY = {};
 
   private StorageImplUtils() {
     //Never constructed
   }
 
+  public static SnapshotManager newSnapshotManager(RaftPeerId id) {
+    return new SnapshotManager(id);
+  }
+
   /** Create a {@link RaftStorageImpl}. */
-  public static RaftStorageImpl newRaftStorage(File dir, RaftServerConfigKeys.Log.CorruptionPolicy logCorruptionPolicy,
-      RaftStorage.StartupOption option, long storageFeeSpaceMin) throws IOException {
-    RaftStorage.LOG.debug("newRaftStorage: {}, {}, {}, {}",dir, logCorruptionPolicy, option, storageFeeSpaceMin);
+  public static RaftStorageImpl newRaftStorage(File dir, SizeInBytes freeSpaceMin,
+      RaftStorage.StartupOption option, Log.CorruptionPolicy logCorruptionPolicy) {
+    return new RaftStorageImpl(dir, freeSpaceMin, option, logCorruptionPolicy);
+  }
+
+  private static List<File> getExistingStorageSubs(List<File> volumes, String targetSubDir,
+      Map<File, Integer> dirsPerVol) {
+    return volumes.stream().flatMap(volume -> {
+          final File[] dirs = Optional.ofNullable(volume.listFiles()).orElse(EMPTY_FILE_ARRAY);
+          Optional.ofNullable(dirsPerVol).ifPresent(map -> map.put(volume, dirs.length));
+          return Arrays.stream(dirs);
+        }).filter(dir -> targetSubDir.equals(dir.getName()))
+        .collect(Collectors.toList());
+  }
+
+  /**
+   * Get a list of existing subdirectories matching the given storage directory name from the given root directories.
+   */
+  static List<File> getExistingStorageSubs(String storageDirName, StartupOption option,
+      List<File> rootDirs, Map<File, Integer> dirsPerVol) throws IOException {
+    Preconditions.assertEmpty(dirsPerVol, "dirsPerVol");
+    final List<File> existingSubs = getExistingStorageSubs(rootDirs, storageDirName, dirsPerVol);
+    final int size = existingSubs.size();
+    if (option == StartupOption.RECOVER) {
+      if (size > 1) {
+        throw new IOException("Failed to " + option + ": More than one existing directories found " + existingSubs
+            + " for " + storageDirName);
+      } else if (size == 0) {
+        throw new IOException("Failed to " + option + ": Storage directory not found for " + storageDirName
+            + " from " + rootDirs);
+      }
+    } else if (option == StartupOption.FORMAT) {
+      if (size > 0) {
+        throw new IOException("Failed to " + option + ": One or more existing directories found " + existingSubs
+            + " for " + storageDirName);
+      }

Review Comment:
   @lokeshj1703 , thanks for the suggestion!  If these checks are moved inside, then it needs to pass the variables for the error messages.  Let's create a class so that the variables will be available.



-- 
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: issues-unsubscribe@ratis.apache.org

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